[FEAT MERGE] 4.3 optimizer enhancement

Co-authored-by: 2149 <260391947@qq.com>
Co-authored-by: akaError <lzg020616@163.com>
Co-authored-by: jinmaoli <lijinmao.csmaster@gmail.com>
This commit is contained in:
yinyj17 2023-12-16 14:12:43 +00:00 committed by ant-ob-hengtang
parent 438a70b2b8
commit 37fe7ce4eb
50 changed files with 3205 additions and 1059 deletions

View File

@ -1785,6 +1785,9 @@ DEF_STR_WITH_CHECKER(sql_protocol_min_tls_version, OB_CLUSTER_PARAMETER, "none",
DEF_MODE_WITH_PARSER(_obkv_feature_mode, OB_CLUSTER_PARAMETER, "", common::ObKvFeatureModeParser,
"_obkv_feature_mode is a option list to control specified OBKV features on/off.",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(_enable_optimizer_qualify_filter, OB_TENANT_PARAMETER, "True",
"Enable extracting qualify filters for window function",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(_enable_range_extraction_for_not_in, OB_TENANT_PARAMETER, "True",
"Enable extract query range for not in predicate",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));

View File

@ -1136,7 +1136,6 @@ ob_set_subtarget(ob_sql resolver_mv
ob_set_subtarget(ob_sql resolver_expr
resolver/expr/ob_expr_relation_analyzer.cpp
resolver/expr/ob_raw_expr.cpp
resolver/expr/ob_raw_expr_canonicalizer_impl.cpp
resolver/expr/ob_raw_expr_copier.cpp
resolver/expr/ob_raw_expr_deduce_type.cpp
resolver/expr/ob_raw_expr_info_extractor.cpp

View File

@ -575,8 +575,9 @@ ObSortOpImpl::ObSortOpImpl(ObMonitorNode &op_monitor_info)
op_type_(PHY_INVALID), op_id_(UINT64_MAX), exec_ctx_(nullptr), stored_rows_(nullptr),
io_event_observer_(nullptr), buckets_(NULL), max_bucket_cnt_(0), part_hash_nodes_(NULL),
max_node_cnt_(0), part_cnt_(0), topn_cnt_(INT64_MAX), outputted_rows_cnt_(0),
is_fetch_with_ties_(false), topn_heap_(NULL), ties_array_pos_(0), ties_array_(),
last_ties_row_(NULL), rows_(NULL)
is_fetch_with_ties_(false), topn_heap_(), ties_array_pos_(0), last_ties_row_(NULL),
pt_buckets_(NULL), use_partition_topn_sort_(false), heap_nodes_(), cur_heap_idx_(0),
rows_(NULL)
{
}
@ -585,6 +586,43 @@ ObSortOpImpl::~ObSortOpImpl()
reset();
}
int ObSortOpImpl::init_topn()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(topn_heap_ = OB_NEWx(TopnHeapNode, (&mem_context_->get_malloc_allocator()),
comp_, &mem_context_->get_malloc_allocator()))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
}
return ret;
}
int ObSortOpImpl::init_partition_topn()
{
int ret = OB_SUCCESS;
uint64_t bucket_cnt = 16;//next_pow2(std::max(16L, rows.count()));
uint64_t shift_right = __builtin_clzll(bucket_cnt);
ObIAllocator &alloc = mem_context_->get_malloc_allocator();
if (max_bucket_cnt_ < bucket_cnt) {
if (NULL != pt_buckets_) {
alloc.free(pt_buckets_);
pt_buckets_ = NULL;
max_bucket_cnt_ = 0;
}
pt_buckets_ = (PartHeapNode **)alloc.alloc(sizeof(PartHeapNode*) * bucket_cnt);
if (OB_ISNULL(pt_buckets_)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc memory", K(ret));
} else {
max_bucket_cnt_ = bucket_cnt;
MEMSET(pt_buckets_, 0, sizeof(PartHeapNode*) * bucket_cnt);
}
} else {
MEMSET(pt_buckets_, 0, sizeof(PartHeapNode*) * bucket_cnt);
}
return ret;
}
// Set the note in ObPrefixSortImpl::init(): %sort_columns may be zero, to compatible with
// the wrong generated prefix sort.
int ObSortOpImpl::init(
@ -627,7 +665,8 @@ int ObSortOpImpl::init(
exec_ctx_ = exec_ctx;
part_cnt_ = part_cnt;
topn_cnt_ = topn_cnt;
use_heap_sort_ = is_topn_sort();
use_heap_sort_ = is_topn_sort() && part_cnt_ == 0;
use_partition_topn_sort_ = is_topn_sort() && part_cnt_ > 0;
is_fetch_with_ties_ = is_fetch_with_ties;
int64_t batch_size = eval_ctx_->max_batch_size_;
lib::ContextParam param;
@ -645,11 +684,10 @@ int ObSortOpImpl::init(
0, /* row_extra_size */
default_block_size))) {
LOG_WARN("init row store failed", K(ret));
} else if (is_topn_sort()
&& OB_ISNULL(topn_heap_ = OB_NEWx(TopnHeap, (&mem_context_->get_malloc_allocator()),
comp_, &mem_context_->get_malloc_allocator()))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else if (use_heap_sort_ && OB_FAIL(init_topn())) {
LOG_WARN("init topn failed", K(ret));
} else if (use_partition_topn_sort_ && OB_FAIL(init_partition_topn())) {
LOG_WARN("init partition topn failed", K(ret));
} else if (batch_size > 0
&& OB_ISNULL(stored_rows_ = static_cast<ObChunkDatumStore::StoredRow **>(
mem_context_->get_malloc_allocator().alloc(
@ -680,11 +718,14 @@ int ObSortOpImpl::init(
}
if (OB_SUCC(ret)) {
inited_ = true;
if (!is_topn_sort()) {
if (use_partition_topn_sort_) {
//rows_ will be set to heap data of current heap after adding the first row
rows_ = &quick_sort_array_;
} else {
} else if (use_heap_sort_) {
rows_ = &(const_cast<common::ObIArray<ObChunkDatumStore::StoredRow *> &>
(topn_heap_->get_heap_data()));
(topn_heap_->heap_.get_heap_data()));
} else {
rows_ = &quick_sort_array_;
}
}
}
@ -703,14 +744,6 @@ void ObSortOpImpl::reuse()
row_idx_ = 0;
next_stored_row_func_ = &ObSortOpImpl::array_next_stored_row;
ties_array_pos_ = 0;
if (0 != ties_array_.count()) {
for (int64_t i = 0; i < ties_array_.count(); ++i) {
sql_mem_processor_.alloc(-1 * ties_array_[i]->get_max_size());
mem_context_->get_malloc_allocator().free(ties_array_[i]);
ties_array_[i] = NULL;
}
}
ties_array_.reset();
while (!sort_chunks_.is_empty()) {
ObSortOpChunk *chunk = sort_chunks_.remove_first();
chunk->~ObSortOpChunk();
@ -724,15 +757,13 @@ void ObSortOpImpl::reuse()
heap_iter_begin_ = false;
if (NULL != ems_heap_) {
ems_heap_->reset();
}
if (NULL != topn_heap_) {
for (int64_t i = 0; i < topn_heap_->count(); ++i) {
sql_mem_processor_.alloc(-1 *
static_cast<SortStoredRow *>(topn_heap_->at(i))->get_max_size());
mem_context_->get_malloc_allocator().free(static_cast<SortStoredRow *>(topn_heap_->at(i)));
topn_heap_->at(i) = NULL;
}
topn_heap_->reset();
} else if (use_partition_topn_sort_) {
heap_nodes_.reset();
reuse_part_topn_heap();
cur_heap_idx_ = 0;
topn_heap_ = NULL;
} else if (NULL != topn_heap_) {
reuse_topn_heap(topn_heap_);
}
}
@ -767,13 +798,10 @@ void ObSortOpImpl::reset()
is_fetch_with_ties_ = false;
rows_ = NULL;
ties_array_pos_ = 0;
if (0 != ties_array_.count()) {
for (int64_t i = 0; i < ties_array_.count(); ++i) {
mem_context_->get_malloc_allocator().free(ties_array_[i]);
ties_array_[i] = NULL;
}
}
ties_array_.reset();
// for partition topn sort
cur_heap_idx_ = 0;
heap_nodes_.reset();
// for partition topn end
if (NULL != mem_context_) {
if (NULL != imms_heap_) {
imms_heap_->~IMMSHeap();
@ -797,12 +825,12 @@ void ObSortOpImpl::reset()
mem_context_->get_malloc_allocator().free(part_hash_nodes_);
part_hash_nodes_ = NULL;
}
if (NULL != pt_buckets_) {
mem_context_->get_malloc_allocator().free(pt_buckets_);
pt_buckets_ = NULL;
}
if (NULL != topn_heap_) {
for (int64_t i = 0; i < topn_heap_->count(); ++i) {
mem_context_->get_malloc_allocator().free(static_cast<SortStoredRow *>(topn_heap_->at(i)));
topn_heap_->at(i) = NULL;
}
topn_heap_->~TopnHeap();
topn_heap_->~TopnHeapNode();
mem_context_->get_malloc_allocator().free(topn_heap_);
topn_heap_ = NULL;
}
@ -842,8 +870,9 @@ int ObSortOpImpl::build_chunk(const int64_t level, Input &input, int64_t extra_s
chunk->datum_store_.set_allocator(mem_context_->get_malloc_allocator());
chunk->datum_store_.set_callback(&sql_mem_processor_);
chunk->datum_store_.set_io_event_observer(io_event_observer_);
int64_t total_size = 0;
while (OB_SUCC(ret)) {
if (!is_fetch_with_ties_ && stored_row_cnt >= topn_cnt_) {
if (use_heap_sort_ && !is_fetch_with_ties_ && stored_row_cnt >= topn_cnt_) {
break;
} else if (OB_FAIL(input(datum_store, src_store_row))) {
if (OB_ITER_END != ret) {
@ -858,9 +887,9 @@ int ObSortOpImpl::build_chunk(const int64_t level, Input &input, int64_t extra_s
stored_row_cnt++;
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::SORT_SORTED_ROW_COUNT;
op_monitor_info_.otherstat_1_value_ += 1;
total_size += src_store_row->row_size_;
}
}
// 必须强制先dump,然后finish dump才有效
if (OB_FAIL(ret)) {
} else if (OB_FAIL(chunk->datum_store_.dump(false, true))) {
@ -1093,7 +1122,7 @@ int ObSortOpImpl::add_row(const common::ObIArray<ObExpr*> &exprs,
const ObChunkDatumStore::StoredRow *&store_row)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(use_heap_sort_ && need_dump())) {
if (OB_UNLIKELY((use_heap_sort_ || use_partition_topn_sort_) && need_dump())) {
bool dumped = false;
if (OB_FAIL(preprocess_dump(dumped))) {
LOG_WARN("failed preprocess dump", K(ret));
@ -1104,12 +1133,33 @@ int ObSortOpImpl::add_row(const common::ObIArray<ObExpr*> &exprs,
if (OB_FAIL(ret)) {
} else if (use_heap_sort_) {
ret = add_heap_sort_row(exprs, store_row);
} else if (use_partition_topn_sort_) {
ret = add_part_heap_sort_row(exprs, store_row);
} else {
ret = add_quick_sort_row(exprs, store_row);
}
return ret;
}
int ObSortOpImpl::add_part_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
const ObChunkDatumStore::StoredRow *&store_row)
{
int ret = OB_SUCCESS;
bool is_cur_block_row = true;
SortStoredRow *new_row = NULL;
if (OB_ISNULL(mem_context_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("mem_context is not initialized", K(ret));
} else if (topn_cnt_ <= 0) {
ret = OB_ITER_END;
} else if (OB_FAIL(locate_current_heap(exprs))) {
LOG_WARN("failed to locate heap", K(ret));
} else if (OB_FAIL(add_heap_sort_row(exprs, store_row))) {
LOG_WARN("add heap sort row failed", K(ret));
}
return ret;
}
int ObSortOpImpl::add_quick_sort_batch(const common::ObIArray<ObExpr *> &exprs,
const ObBitVector &skip,
const int64_t batch_size,
@ -1143,7 +1193,7 @@ int ObSortOpImpl::add_batch(const common::ObIArray<ObExpr *> &exprs,
int64_t *append_row_count = nullptr)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(use_heap_sort_ && need_dump())) {
if (OB_UNLIKELY((use_heap_sort_ || use_partition_topn_sort_) && need_dump())) {
bool dumped = false;
if (OB_FAIL(preprocess_dump(dumped))) {
LOG_WARN("failed preprocess dump", K(ret));
@ -1154,6 +1204,8 @@ int ObSortOpImpl::add_batch(const common::ObIArray<ObExpr *> &exprs,
if (OB_FAIL(ret)) {
} else if (use_heap_sort_) {
ret = add_heap_sort_batch(exprs, skip, batch_size, start_pos, append_row_count);
} else if (use_partition_topn_sort_) {
ret = add_part_heap_sort_batch(exprs, skip, batch_size, start_pos, append_row_count);
} else {
ret = add_quick_sort_batch(exprs, skip, batch_size, start_pos, append_row_count);
}
@ -1188,7 +1240,7 @@ int ObSortOpImpl::add_batch(const common::ObIArray<ObExpr *> &exprs,
const uint16_t selector[], const int64_t size)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(use_heap_sort_ && need_dump())) {
if (OB_UNLIKELY((use_heap_sort_ || use_partition_topn_sort_) && need_dump())) {
if (OB_FAIL(do_dump())) {
LOG_WARN("failed to do topn dump", K(ret));
}
@ -1196,6 +1248,8 @@ int ObSortOpImpl::add_batch(const common::ObIArray<ObExpr *> &exprs,
if (OB_FAIL(ret)) {
} else if (use_heap_sort_) {
ret = add_heap_sort_batch(exprs, skip, batch_size, selector, size);
} else if (use_partition_topn_sort_) {
ret = add_part_heap_sort_batch(exprs, skip, batch_size, selector, size);
} else {
ret = add_quick_sort_batch(exprs, skip, batch_size, selector, size);
}
@ -1394,9 +1448,75 @@ int ObSortOpImpl::do_partition_sort(common::ObIArray<ObChunkDatumStore::StoredRo
return ret;
}
int ObSortOpImpl::do_partition_topn_sort() {
int ret = OB_SUCCESS;
heap_nodes_.reuse();
for (int64_t idx = 0; OB_SUCC(ret) && idx < max_bucket_cnt_; ++idx) {
PartHeapNode *hash_node = pt_buckets_[idx];
int64_t cur_bucket_cnt = 0;
int64_t hash_expr_cnt = 1;
cur_bucket_cnt = heap_nodes_.count();
while(hash_node != NULL) {
TopnHeapNode &cur_heap = hash_node->topn_heap_node_;
if (0 == cur_heap.heap_.count()) {
//do nothing
} else if (topn_cnt_ < cur_heap.heap_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("topn is less than array_count", K(ret), K(topn_cnt_), K(cur_heap.heap_.count()));
} else if (OB_FAIL(heap_nodes_.push_back(&cur_heap))) {
LOG_WARN("failed to add into heap nodes", K(ret));
}
hash_node = hash_node->hash_node_next_;
}
if (OB_SUCC(ret)) {
//sort heaps which is in the same bucket
comp_.set_cmp_range(0, part_cnt_ + hash_expr_cnt);
std::sort(heap_nodes_.begin() + cur_bucket_cnt,
heap_nodes_.end(),
TopnHeapNodeComparer(comp_));
//sort rows in heap
if (OB_SUCC(ret) && part_cnt_ + hash_expr_cnt < comp_.get_cnt()) {
comp_.set_cmp_range(part_cnt_ + hash_expr_cnt, comp_.get_cnt());
for (int64_t i = cur_bucket_cnt; OB_SUCC(ret) && i < heap_nodes_.count(); ++i) {
TopnHeapNode *cur_heap = heap_nodes_.at(i);
ObIArray<ObChunkDatumStore::StoredRow *> &heap_rows = (const_cast<common::ObIArray<ObChunkDatumStore::StoredRow *> &>
(cur_heap->heap_.get_heap_data()));
if (enable_encode_sortkey_) {
bool can_encode = true;
ObAdaptiveQS aqs(heap_rows, mem_context_->get_malloc_allocator());
if (OB_FAIL(aqs.init(heap_rows, mem_context_->get_malloc_allocator(), 0, cur_heap->heap_.count(), can_encode))) {
LOG_WARN("failed to init aqs", K(ret));
} else if (can_encode) {
aqs.sort(0, cur_heap->heap_.count());
} else {
enable_encode_sortkey_ = false;
comp_.enable_encode_sortkey_ = false;
std::sort(&heap_rows.at(0), &heap_rows.at(0) + heap_rows.count(), CopyableComparer(comp_));
if (OB_SUCCESS != comp_.ret_) {
ret = comp_.ret_;
LOG_WARN("compare failed", K(ret));
}
}
} else {
std::sort(&heap_rows.at(0), &heap_rows.at(0) + heap_rows.count(), CopyableComparer(comp_));
}
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::SORT_SORTED_ROW_COUNT;
op_monitor_info_.otherstat_1_value_ += cur_heap->heap_.count();
}
} else {
//partition limit, do nothing
}
}
}
comp_.set_cmp_range(0, comp_.get_cnt());
return ret;
}
int ObSortOpImpl::do_dump()
{
int ret = OB_SUCCESS;
ObSEArray<std::pair<int64_t, int64_t>, 4> pt_bucket_cnts;
int64_t pt_bucket_dumped = 0;
if (!is_inited()) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
@ -1405,22 +1525,40 @@ int ObSortOpImpl::do_dump()
LOG_WARN("invalid argument", K(ret));
} else if (OB_FAIL(sort_inmem_data())) {
LOG_WARN("sort in-memory data failed", K(ret));
} else {
}
if (OB_SUCC(ret)) {
const int64_t level = 0;
if (!need_imms()) {
if (use_partition_topn_sort_) {
int64_t cur_heap_idx = 0;
int64_t row_idx = 0;
const int64_t level = 0;
auto input = [&](ObChunkDatumStore *&rs, const ObChunkDatumStore::StoredRow *&row) {
if (OB_FAIL(part_topn_heap_next(cur_heap_idx, row_idx, row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get row from part topn heap failed", K(ret));
}
} else {
rs = &datum_store_;
}
return ret;
};
if (OB_FAIL(build_chunk(level, input))) {
LOG_WARN("build chunk failed", K(ret));
}
} else if (!need_imms()) {
int64_t row_pos = 0;
int64_t ties_array_pos = 0;
auto input = [&](ObChunkDatumStore *&rs, const ObChunkDatumStore::StoredRow *&row) {
int ret = OB_SUCCESS;
if (row_pos >= rows_->count()
&& ties_array_pos >= ties_array_.count()) {
&& !(NULL != topn_heap_ && ties_array_pos < topn_heap_->ties_array_.count())) {
ret = OB_ITER_END;
} else if (row_pos < rows_->count()) {
row = rows_->at(row_pos);
rs = &datum_store_;
row_pos += 1;
} else {
row = ties_array_.at(ties_array_pos);
} else if (NULL != topn_heap_) {
row = topn_heap_->ties_array_.at(ties_array_pos);
rs = &datum_store_;
ties_array_pos += 1;
}
@ -1445,33 +1583,24 @@ int ObSortOpImpl::do_dump()
LOG_WARN("build chunk failed", K(ret));
}
}
if (OB_SUCC(ret) && use_heap_sort_) {
if (NULL != mem_context_ && NULL != topn_heap_) {
for (int64_t i = 0; i < topn_heap_->count(); ++i) {
sql_mem_processor_.alloc(-1 *
static_cast<SortStoredRow *>(topn_heap_->at(i))->get_max_size());
mem_context_->get_malloc_allocator().free(
static_cast<SortStoredRow *>(topn_heap_->at(i)));
topn_heap_->at(i) = NULL;
}
topn_heap_->~TopnHeap();
mem_context_->get_malloc_allocator().free(topn_heap_);
topn_heap_ = NULL;
}
if (0 != ties_array_.count()) {
for (int64_t i = 0; i < ties_array_.count(); ++i) {
sql_mem_processor_.alloc(-1 * ties_array_[i]->get_max_size());
mem_context_->get_malloc_allocator().free(ties_array_[i]);
ties_array_[i] = NULL;
}
}
ties_array_.reset();
reuse_topn_heap(topn_heap_);
topn_heap_->~TopnHeapNode();
mem_context_->get_malloc_allocator().free(topn_heap_);
topn_heap_ = NULL;
got_first_row_ = false;
use_heap_sort_ = false;
rows_ = &quick_sort_array_;
}
if (OB_SUCC(ret) && use_partition_topn_sort_) {
heap_nodes_.reset();
reuse_part_topn_heap();
topn_heap_ = NULL;
got_first_row_ = false;
rows_ = &quick_sort_array_;
}
if (OB_SUCC(ret)) {
heap_iter_begin_ = false;
row_idx_ = 0;
@ -1486,6 +1615,43 @@ int ObSortOpImpl::do_dump()
return ret;
}
void ObSortOpImpl::reuse_topn_heap(TopnHeapNode *topn_heap) {
if (NULL != topn_heap && NULL != mem_context_) {
for (int64_t i = 0; i < topn_heap->heap_.count(); ++i) {
sql_mem_processor_.alloc(-1 *
static_cast<SortStoredRow *>(topn_heap->heap_.at(i))->get_max_size());
mem_context_->get_malloc_allocator().free(static_cast<SortStoredRow *>(topn_heap->heap_.at(i)));
topn_heap->heap_.at(i) = NULL;
}
topn_heap->heap_.reset();
for (int64_t i = 0; i < topn_heap->ties_array_.count(); ++i) {
sql_mem_processor_.alloc(-1 * topn_heap->ties_array_[i]->get_max_size());
mem_context_->get_malloc_allocator().free(topn_heap->ties_array_[i]);
topn_heap->ties_array_[i] = NULL;
}
topn_heap->ties_array_.reset();
}
}
void ObSortOpImpl::reuse_part_topn_heap() {
for (int64_t i = 0; i < max_bucket_cnt_; ++i) {
if (mem_context_ != NULL) {
PartHeapNode *hash_node = pt_buckets_[i];
while(hash_node != NULL) {
PartHeapNode *cur_node = hash_node;
if (topn_heap_ == &(cur_node->topn_heap_node_)) {
topn_heap_ = NULL;
}
hash_node = cur_node->hash_node_next_;
reuse_topn_heap(&cur_node->topn_heap_node_);
cur_node->~PartHeapNode();
mem_context_->get_malloc_allocator().free(cur_node);
}
pt_buckets_[i] = NULL;
}
}
}
int ObSortOpImpl::build_ems_heap(int64_t &merge_ways)
{
int ret = OB_SUCCESS;
@ -1646,6 +1812,41 @@ int ObSortOpImpl::imms_heap_next(const ObChunkDatumStore::StoredRow *&store_row)
return ret;
}
int ObSortOpImpl::part_topn_heap_next(int64_t &cur_heap_idx,
int64_t &cur_heap_row_idx,
const ObChunkDatumStore::StoredRow *&store_row)
{
int ret = OB_SUCCESS;
TopnHeapNode *cur_heap = NULL;
store_row = NULL;
if (cur_heap_idx < 0) {
ret = OB_ARRAY_OUT_OF_RANGE;
} else {
while (OB_SUCC(ret) && NULL == store_row) {
if (cur_heap_idx >= heap_nodes_.count()) {
ret = OB_ITER_END;
} else {
cur_heap = heap_nodes_.at(cur_heap_idx);
if (OB_ISNULL(cur_heap)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid cur_heap", K(ret));
} else if (cur_heap_row_idx < cur_heap->heap_.count()) {
store_row = cur_heap->heap_.at(cur_heap_row_idx);
cur_heap_row_idx += 1;
} else if (cur_heap_row_idx - cur_heap->heap_.count() < cur_heap->ties_array_.count()) {
store_row = cur_heap->ties_array_.at(cur_heap_row_idx - cur_heap->heap_.count());
cur_heap_row_idx += 1;
} else {
//switch heap
cur_heap_idx += 1;
cur_heap_row_idx = 0;
}
}
}
}
return ret;
}
int ObSortOpImpl::sort_inmem_data()
{
int ret = OB_SUCCESS;
@ -1654,20 +1855,15 @@ int ObSortOpImpl::sort_inmem_data()
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (!rows_->empty()) {
if (!use_heap_sort_ && (local_merge_sort_ || sorted_)) {
if (!use_heap_sort_ && !use_partition_topn_sort_ &&
(local_merge_sort_ || sorted_)) {
// row already in order, do nothing.
} else {
int64_t begin = 0;
if (need_imms()) {
// is increment sort (rows add after sort()), sort the last add rows
for (int64_t i = rows_->count() - 1; i >= 0; i--) {
if (NULL == rows_->at(i)) {
begin = i + 1;
break;
}
}
}
if (part_cnt_ > 0) {
if (use_partition_topn_sort_) {
heap_nodes_.reuse();
do_partition_topn_sort();
} else if (part_cnt_ > 0) {
do_partition_sort(*rows_, begin, rows_->count());
} else if (enable_encode_sortkey_) {
bool can_encode = true;
@ -1752,8 +1948,14 @@ int ObSortOpImpl::sort()
LOG_WARN("init iterator failed", K(ret));
} else {
if (!need_imms()) {
row_idx_ = 0;
next_stored_row_func_ = &ObSortOpImpl::array_next_stored_row;
if (use_partition_topn_sort_) {
cur_heap_idx_ = 0;
row_idx_ = 0;
next_stored_row_func_ = &ObSortOpImpl::part_heap_next_stored_row;
} else {
row_idx_ = 0;
next_stored_row_func_ = &ObSortOpImpl::array_next_stored_row;
}
} else {
next_stored_row_func_ = &ObSortOpImpl::imms_heap_next_stored_row;
}
@ -1811,6 +2013,7 @@ int ObSortOpImpl::sort()
}
if (OB_SUCC(ret)) {
// set iteration age for batch iteration.
set_blk_holder(&blk_holder_);
next_stored_row_func_ = &ObSortOpImpl::ems_heap_next_stored_row;
}
@ -1818,18 +2021,116 @@ int ObSortOpImpl::sort()
return ret;
}
int ObSortOpImpl::locate_current_heap(const common::ObIArray<ObExpr*> &exprs)
{
int ret = OB_SUCCESS;
PartHeapNode *new_heap_node = NULL;
int64_t hash_idx = sort_collations_->at(0).field_idx_;
uint64_t bucket_cnt = max_bucket_cnt_;
uint64_t shift_right = __builtin_clzll(bucket_cnt) + 1;
ObDatum *part_datum = NULL;
uint64_t pos = 0;
if (OB_ISNULL(exprs.at(hash_idx))) {
pos = 0;
} else if (OB_FAIL(exprs.at(hash_idx)->eval(*eval_ctx_, part_datum))) {
LOG_WARN("expression evaluate failed", K(ret));
} else {
pos = part_datum->get_uint64() >> shift_right; // high n bit
}
if (OB_FAIL(ret)) {
} else if (pos > bucket_cnt - 1) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid pos", K(ret));
} else {
PartHeapNode *exist = NULL;
if (OB_FAIL(locate_current_heap_in_bucket(pt_buckets_[pos], exprs, exist))) {
LOG_WARN("locate current heap in bucket failed", K(ret));
} else if (NULL == exist) {
TopnHeap *new_heap = NULL;
ObIAllocator &alloc = mem_context_->get_malloc_allocator();
if (OB_ISNULL(new_heap_node = OB_NEWx(PartHeapNode, &alloc, comp_, &alloc))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
new_heap_node->hash_node_next_ = pt_buckets_[pos];
pt_buckets_[pos] = new_heap_node;
topn_heap_ = &new_heap_node->topn_heap_node_;
}
} else {
topn_heap_ = &(exist->topn_heap_node_);
}
}
if (OB_SUCC(ret)) {
rows_ = &(const_cast<common::ObIArray<ObChunkDatumStore::StoredRow *> &>
(topn_heap_->heap_.get_heap_data()));
}
return ret;
}
int ObSortOpImpl::locate_current_heap_in_bucket(PartHeapNode *first_node,
const common::ObIArray<ObExpr*> &exprs,
PartHeapNode *&exist)
{
int ret = OB_SUCCESS;
ObSEArray<ObDatum *, 4> part_datums;
exist = first_node;
bool find_same_heap = false;
if (NULL != exist) {
for (int64_t i = 0; OB_SUCC(ret) && i <= part_cnt_; ++i) {
int64_t idx = sort_collations_->at(i).field_idx_;
ObDatum *part_datum = NULL;
if (OB_ISNULL(exprs.at(idx))) {
if (OB_FAIL(part_datums.push_back(NULL))) {
LOG_WARN("push back part datums of new row failed", K(ret));
}
} else if (OB_FAIL(exprs.at(idx)->eval(*eval_ctx_, part_datum))) {
LOG_WARN("expression evaluate failed", K(ret));
} else if (OB_FAIL(part_datums.push_back(part_datum))) {
LOG_WARN("push back part datums of new row failed", K(ret));
}
}
}
while (OB_SUCC(ret) && NULL != exist && !find_same_heap) {
if (exist->topn_heap_node_.heap_.count() > 0) {
const ObChunkDatumStore::StoredRow *top_row = exist->topn_heap_node_.heap_.top();
find_same_heap = true;
if (OB_ISNULL(top_row)) {
find_same_heap = false;
}
for (int64_t i = 0; OB_SUCC(ret) && find_same_heap && i <= part_cnt_; ++i) {
uint32_t idx = sort_collations_->at(i).field_idx_;
int cmp_ret = 0;
if (OB_ISNULL(part_datums.at(i))) {
find_same_heap = top_row->cells()[idx].is_null();
} else if (OB_FAIL(sort_cmp_funs_->at(i).cmp_func_(*part_datums.at(i),
top_row->cells()[idx],
cmp_ret))) {
LOG_WARN("failed to compare", K(ret));
} else {
find_same_heap = (0 == cmp_ret);
}
}
}
if (!find_same_heap) {
exist = exist->hash_node_next_;
}
}
return ret;
}
int ObSortOpImpl::array_next_stored_row(
const ObChunkDatumStore::StoredRow *&sr)
{
int ret = OB_SUCCESS;
if (row_idx_ >= rows_->count()
&& ties_array_pos_ >= ties_array_.count()) {
&& (NULL == topn_heap_
|| ties_array_pos_ >= topn_heap_->ties_array_.count())) {
ret = OB_ITER_END;
} else if (row_idx_ < rows_->count()) {
sr = rows_->at(row_idx_);
row_idx_ += 1;
} else {
sr = ties_array_.at(ties_array_pos_);
sr = topn_heap_->ties_array_.at(ties_array_pos_);
ties_array_pos_ += 1;
}
return ret;
@ -1859,6 +2160,23 @@ int ObSortOpImpl::ems_heap_next_stored_row(
return ret;
}
int ObSortOpImpl::part_heap_next_stored_row(const ObChunkDatumStore::StoredRow *&sr)
{
int ret = OB_SUCCESS;
if (OB_FAIL(part_topn_heap_next(cur_heap_idx_, row_idx_, sr))) {
if (OB_ITER_END != ret) {
LOG_WARN("get row from part topn heap failed", K(ret));
} else {
// Reset status when iterating end, because we will add rows and sort again after dumped to disk.
cur_heap_idx_ = 0;
row_idx_ = 0;
heap_nodes_.reset();
sr = nullptr;
}
}
return ret;
}
int ObSortOpImpl::rewind()
{
int ret = OB_SUCCESS;
@ -1920,7 +2238,7 @@ int ObSortOpImpl::get_next_batch(const common::ObIArray<ObExpr*> &exprs,
if (OB_ITER_END != ret) {
LOG_WARN("failed to get next batch stored rows", K(ret));
}
} else if (read_rows > 0 && OB_FAIL(adjust_topn_read_rows(stored_rows_, read_rows))) {
} else if (read_rows > 0 && !use_partition_topn_sort_ && OB_FAIL(adjust_topn_read_rows(stored_rows_, read_rows))) {
LOG_WARN("failed to adjust read rows with ties", K(ret));
} else {
ObChunkDatumStore::Iterator::attach_rows(exprs, *eval_ctx_,
@ -1951,7 +2269,7 @@ int ObSortOpImpl::add_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
bool updated = false;
if (OB_FAIL(sql_mem_processor_.update_max_available_mem_size_periodically(
&mem_context_->get_malloc_allocator(),
[&](int64_t cur_cnt){ return topn_heap_->count() > cur_cnt; },
[&](int64_t cur_cnt){ return topn_heap_->heap_.count() > cur_cnt; },
updated))) {
LOG_WARN("failed to get max available memory size", K(ret));
} else if (updated && OB_FAIL(sql_mem_processor_.update_used_mem_size(mem_context_->used()))) {
@ -1960,7 +2278,7 @@ int ObSortOpImpl::add_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
}
if (OB_FAIL(ret)) {
} else if (topn_heap_->count() == topn_cnt_ - outputted_rows_cnt_) {
} else if (topn_heap_->heap_.count() == topn_cnt_ - outputted_rows_cnt_) {
if (is_fetch_with_ties_ && OB_FAIL(adjust_topn_heap_with_ties(exprs, store_row))) {
LOG_WARN("failed to adjust topn heap with ties", K(ret));
} else if (!is_fetch_with_ties_ && OB_FAIL(adjust_topn_heap(exprs, store_row))) {
@ -1969,12 +2287,12 @@ int ObSortOpImpl::add_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
} else { // push back array
SortStoredRow *new_row = NULL;
ObIAllocator &alloc = mem_context_->get_malloc_allocator();
int64_t topn_heap_size = topn_heap_->count();
int64_t topn_heap_size = topn_heap_->heap_.count();
if (OB_FAIL(copy_to_row(exprs, alloc, new_row))) {
LOG_WARN("failed to generate new row", K(ret));
} else if (OB_FAIL(topn_heap_->push(new_row))) {
} else if (OB_FAIL(topn_heap_->heap_.push(new_row))) {
LOG_WARN("failed to push back row", K(ret));
if (topn_heap_->count() == topn_heap_size) {
if (topn_heap_->heap_.count() == topn_heap_size) {
mem_context_->get_malloc_allocator().free(new_row);
new_row = NULL;
}
@ -2034,8 +2352,64 @@ int ObSortOpImpl::add_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
LOG_WARN("check need sort failed", K(ret));
} else if (OB_NOT_NULL(store_row)) {
stored_rows_[i] = const_cast<ObChunkDatumStore::StoredRow *>(store_row);
} else if (OB_NOT_NULL(topn_heap_) && OB_NOT_NULL(topn_heap_->top())) {
stored_rows_[i] = const_cast<ObChunkDatumStore::StoredRow *>(topn_heap_->top());
} else if (OB_NOT_NULL(topn_heap_) && OB_NOT_NULL(topn_heap_->heap_.top())) {
stored_rows_[i] = const_cast<ObChunkDatumStore::StoredRow *>(topn_heap_->heap_.top());
} else {
ret = OB_ERR_UNEXPECTED;
}
}
return ret;
}
int ObSortOpImpl::add_part_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
const ObBitVector &skip,
const int64_t batch_size,
const int64_t start_pos /* 0 */,
int64_t *append_row_count)
{
int ret = OB_SUCCESS;
int64_t row_count = 0;
const ObChunkDatumStore::StoredRow *store_row = NULL;
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(*eval_ctx_);
batch_info_guard.set_batch_size(batch_size);
for (int64_t i = start_pos; OB_SUCC(ret) && i < batch_size; i++) {
if (skip.at(i)) {
continue;
}
batch_info_guard.set_batch_idx(i);
if (OB_FAIL(add_part_heap_sort_row(exprs, store_row))) {
LOG_WARN("failed to add topn row", K(ret));
}
row_count++;
}
if (OB_NOT_NULL(append_row_count)) {
*append_row_count = row_count;
}
return ret;
}
// if less than heap size or replace heap top, store row is new row
// otherwise, store row will not change as last result obtained.
// here, strored_rows_ only used to fetch prev_row in prefix sort batch.
int ObSortOpImpl::add_part_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
const ObBitVector &skip,
const int64_t batch_size,
const uint16_t selector[],
const int64_t size)
{
int ret = OB_SUCCESS;
const ObChunkDatumStore::StoredRow *store_row = NULL;
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(*eval_ctx_);
batch_info_guard.set_batch_size(batch_size);
for (int64_t i = 0; i < size && OB_SUCC(ret); i++) {
int64_t idx = selector[i];
batch_info_guard.set_batch_idx(idx);
if (OB_FAIL(add_part_heap_sort_row(exprs, store_row))) {
LOG_WARN("check need sort failed", K(ret));
} else if (OB_NOT_NULL(store_row)) {
stored_rows_[i] = const_cast<ObChunkDatumStore::StoredRow *>(store_row);
} else if (OB_NOT_NULL(topn_heap_) && OB_NOT_NULL(topn_heap_->heap_.top())) {
stored_rows_[i] = const_cast<ObChunkDatumStore::StoredRow *>(topn_heap_->heap_.top());
} else {
ret = OB_ERR_UNEXPECTED;
}
@ -2050,16 +2424,19 @@ int ObSortOpImpl::adjust_topn_heap(const common::ObIArray<ObExpr*> &exprs,
if (OB_ISNULL(mem_context_) || OB_ISNULL(topn_heap_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("mem_context or heap is not initialized", K(ret));
} else if (OB_ISNULL(topn_heap_->top())) {
} else if (OB_ISNULL(topn_heap_->heap_.top())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error.top of the heap is NULL", K(ret), K(topn_heap_->count()));
} else if (!topn_heap_->empty()) {
if (comp_(&exprs, topn_heap_->top(), *eval_ctx_)) {
LOG_WARN("unexpected error.top of the heap is NULL", K(ret), K(topn_heap_->heap_.count()));
} else if (!topn_heap_->heap_.empty()) {
if (use_partition_topn_sort_) {
comp_.set_cmp_range(part_cnt_ + 1 /*hash expr cnt*/, comp_.get_cnt());
}
if (comp_(&exprs, topn_heap_->heap_.top(), *eval_ctx_)) {
ObIAllocator &alloc = mem_context_->get_malloc_allocator();
SortStoredRow* new_row = NULL;
if (OB_FAIL(copy_to_topn_row(exprs, alloc, new_row))) {
LOG_WARN("failed to generate new row", K(ret));
} else if (OB_FAIL(topn_heap_->replace_top(new_row))) {
} else if (OB_FAIL(topn_heap_->heap_.replace_top(new_row))) {
LOG_WARN("failed to replace top", K(ret));
} else {
store_row = new_row;
@ -2067,6 +2444,9 @@ int ObSortOpImpl::adjust_topn_heap(const common::ObIArray<ObExpr*> &exprs,
} else {
ret = comp_.ret_;
}
if (OB_SUCC(ret) && use_partition_topn_sort_) {
comp_.set_cmp_range(0, comp_.get_cnt());
}
}
return ret;
}
@ -2084,27 +2464,31 @@ int ObSortOpImpl::adjust_topn_heap_with_ties(const common::ObIArray<ObExpr*> &ex
if (OB_ISNULL(mem_context_) || OB_ISNULL(topn_heap_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("mem_context or heap is not initialized", K(ret));
} else if (OB_ISNULL(topn_heap_->top())) {
} else if (OB_ISNULL(topn_heap_->heap_.top())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error.top of the heap is NULL", K(ret), K(topn_heap_->count()));
} else if (!topn_heap_->empty()) {
int cmp = comp_.with_ties_cmp(&exprs, topn_heap_->top(), *eval_ctx_);
LOG_WARN("unexpected error.top of the heap is NULL", K(ret), K(topn_heap_->heap_.count()));
} else if (!topn_heap_->heap_.empty()) {
int cmp = 0;
bool is_alloced = false;
bool add_ties_array = false;
SortStoredRow *new_row = NULL;
SortStoredRow *copy_pre_heap_top_row = NULL;
ObIAllocator &alloc = mem_context_->get_malloc_allocator();
SortStoredRow *pre_heap_top_row = static_cast<SortStoredRow *>(topn_heap_->top());
SortStoredRow *pre_heap_top_row = static_cast<SortStoredRow *>(topn_heap_->heap_.top());
if (use_partition_topn_sort_) {
comp_.set_cmp_range(part_cnt_ + 1 /*hash expr cnt*/, comp_.get_cnt());
}
cmp = comp_.with_ties_cmp(&exprs, topn_heap_->heap_.top(), *eval_ctx_);
if (OB_FAIL(comp_.ret_) || cmp < 0) {
/* do nothing */
} else if (0 == cmp) {
// equal to heap top, add row to ties array
int64_t ties_array_size = ties_array_.count();
int64_t ties_array_size = topn_heap_->ties_array_.count();
if (OB_FAIL(copy_to_row(exprs, alloc, new_row))) {
LOG_WARN("failed to generate new row", K(ret));
} else if (OB_FAIL(ties_array_.push_back(new_row))) {
} else if (OB_FAIL(topn_heap_->ties_array_.push_back(new_row))) {
LOG_WARN("failed to push back ties array", K(ret));
if (ties_array_size == ties_array_.count()) {
if (ties_array_size == topn_heap_->ties_array_.count()) {
mem_context_->get_malloc_allocator().free(new_row);
new_row = NULL;
}
@ -2116,26 +2500,26 @@ int ObSortOpImpl::adjust_topn_heap_with_ties(const common::ObIArray<ObExpr*> &ex
LOG_WARN("failed to generate new row", K(ret));
} else if (OB_FAIL(copy_to_topn_row(exprs, alloc, new_row))) {
LOG_WARN("failed to generate new row", K(ret));
} else if (OB_FAIL(topn_heap_->replace_top(new_row))) {
} else if (OB_FAIL(topn_heap_->heap_.replace_top(new_row))) {
LOG_WARN("failed to replace top", K(ret));
} else if (OB_FALSE_IT(cmp = comp_.with_ties_cmp(copy_pre_heap_top_row, topn_heap_->top()))) {
} else if (OB_FALSE_IT(cmp = comp_.with_ties_cmp(copy_pre_heap_top_row, topn_heap_->heap_.top()))) {
} else if (OB_FAIL(comp_.ret_)) {
/* do nothing */
} else if (0 != cmp) {
// previous heap top not equal to new heap top, clear ties array
LOG_DEBUG("in memory topn sort with ties clear ties array",
KPC(new_row), KPC(copy_pre_heap_top_row));
if (0 != ties_array_.count()) {
for (int64_t i = 0; i < ties_array_.count(); ++i) {
sql_mem_processor_.alloc(-1 * ties_array_[i]->get_max_size());
inmem_row_size_ -= ties_array_[i]->get_max_size();
mem_context_->get_malloc_allocator().free(ties_array_[i]);
ties_array_[i] = NULL;
if (0 != topn_heap_->ties_array_.count()) {
for (int64_t i = 0; i < topn_heap_->ties_array_.count(); ++i) {
sql_mem_processor_.alloc(-1 * topn_heap_->ties_array_[i]->get_max_size());
inmem_row_size_ -= topn_heap_->ties_array_[i]->get_max_size();
mem_context_->get_malloc_allocator().free(topn_heap_->ties_array_[i]);
topn_heap_->ties_array_[i] = NULL;
}
}
ties_array_.reset();
topn_heap_->ties_array_.reset();
store_row = new_row;
} else if (OB_FAIL(ties_array_.push_back(copy_pre_heap_top_row))) {
} else if (OB_FAIL(topn_heap_->ties_array_.push_back(copy_pre_heap_top_row))) {
LOG_WARN("failed to push back ties array", K(ret));
} else {
// previous heap top equal to new heap top, add previous heap top to ties array
@ -2144,6 +2528,9 @@ int ObSortOpImpl::adjust_topn_heap_with_ties(const common::ObIArray<ObExpr*> &ex
LOG_DEBUG("in memory topn sort with ties add ties array",
KPC(new_row), KPC(copy_pre_heap_top_row));
}
if (OB_SUCC(ret) && use_partition_topn_sort_) {
comp_.set_cmp_range(0, comp_.get_cnt());
}
if (!add_ties_array && OB_NOT_NULL(copy_pre_heap_top_row)) {
sql_mem_processor_.alloc(-1 * copy_pre_heap_top_row->get_max_size());
inmem_row_size_ -= copy_pre_heap_top_row->get_max_size();
@ -2160,12 +2547,12 @@ int ObSortOpImpl::copy_to_topn_row(const common::ObIArray<ObExpr*> &exprs,
SortStoredRow *&new_row)
{
int ret = OB_SUCCESS;
SortStoredRow *top_row = static_cast<SortStoredRow *>(topn_heap_->top());
SortStoredRow *top_row = static_cast<SortStoredRow *>(topn_heap_->heap_.top());
if (OB_FAIL(copy_to_row(exprs, alloc, top_row))) {
LOG_WARN("failed to copy to row", K(ret));
} else {
new_row = top_row;
topn_heap_->top() = static_cast<ObChunkDatumStore::StoredRow *>(top_row);
topn_heap_->heap_.top() = static_cast<ObChunkDatumStore::StoredRow *>(top_row);
}
return ret;
}

View File

@ -107,7 +107,7 @@ public:
int ret = OB_SUCCESS;
if (OB_FAIL(add_row(expr, store_row))) {
SQL_ENG_LOG(WARN, "failed to add row", K(ret));
} else if (use_heap_sort_) {
} else if (use_heap_sort_ || use_partition_topn_sort_) {
sort_need_dump = false;
} else {
sort_need_dump = need_dump();
@ -141,7 +141,7 @@ public:
int ret = OB_SUCCESS;
if (OB_FAIL(add_batch(exprs, skip, batch_size, start_pos, append_row_count))) {
SQL_ENG_LOG(WARN, "failed to add batch", K(ret));
} else if (use_heap_sort_) {
} else if (use_heap_sort_ || use_partition_topn_sort_) {
sort_need_dump = false;
} else {
sort_need_dump = need_dump();
@ -179,7 +179,7 @@ public:
if (!is_inited()) {
ret = common::OB_NOT_INIT;
SQL_ENG_LOG(WARN, "not init", K(ret));
} else if (outputted_rows_cnt_ >= topn_cnt_ && !is_fetch_with_ties_) {
} else if (outputted_rows_cnt_ >= topn_cnt_ && !is_fetch_with_ties_ && !use_partition_topn_sort_) {
ret = OB_ITER_END;
} else {
blk_holder_.release();
@ -188,7 +188,7 @@ public:
reuse();
}
}
if (OB_FAIL(ret)) {
if (OB_FAIL(ret) || use_partition_topn_sort_) {
} else if (NULL != last_ties_row_ && 0 != comp_.with_ties_cmp(sr, last_ties_row_)) {
ret = OB_ITER_END;
} else if (OB_UNLIKELY(OB_FAIL(comp_.ret_))) {
@ -436,6 +436,42 @@ protected:
{ get_extra_info().max_size_ = max_size; }
};
typedef common::ObBinaryHeap<ObChunkDatumStore::StoredRow *, Compare> TopnHeap;
struct TopnHeapNode
{
TopnHeapNode(Compare &cmp, common::ObIAllocator *allocator = NULL): heap_(cmp, allocator), ties_array_() {}
~TopnHeapNode() {
heap_.reset();
ties_array_.reset();
}
TO_STRING_EMPTY();
TopnHeap heap_;
common::ObArray<SortStoredRow *> ties_array_;
};
struct PartHeapNode
{
PartHeapNode(Compare &cmp, common::ObIAllocator *allocator = NULL):
hash_node_next_(NULL),
topn_heap_node_(cmp, allocator) {}
~PartHeapNode() {hash_node_next_ = NULL; topn_heap_node_.~TopnHeapNode(); }
PartHeapNode *hash_node_next_;
TopnHeapNode topn_heap_node_;
TO_STRING_EMPTY();
};
class TopnHeapNodeComparer
{
public:
TopnHeapNodeComparer(Compare &compare) : compare_(compare) {}
bool operator()(const TopnHeapNode *l, const TopnHeapNode *r)
{
return compare_(l->heap_.top(),
r->heap_.top());
}
Compare &compare_;
};
int get_next_row(const common::ObIArray<ObExpr*> &exprs, const ObChunkDatumStore::StoredRow *&sr)
{
int ret = common::OB_SUCCESS;
@ -459,10 +495,12 @@ protected:
bool need_imms() const
{
return !use_heap_sort_ && rows_->count() > datum_store_.get_row_cnt();
return !use_heap_sort_ && !use_partition_topn_sort_
&& rows_->count() > datum_store_.get_row_cnt();
}
int sort_inmem_data();
int do_dump();
int inner_dump_part_topn_heap();
template <typename Input>
int build_chunk(const int64_t level, Input &input, int64_t extra_size = 0);
@ -472,6 +510,9 @@ protected:
int heap_next(Heap &heap, const NextFunc &func, Item &item);
int ems_heap_next(ObSortOpChunk *&chunk);
int imms_heap_next(const ObChunkDatumStore::StoredRow *&store_row);
int imms_partition_topn_next(const ObChunkDatumStore::StoredRow *&store_row);
int part_topn_heap_next(int64_t &cur_heap_array_idx, int64_t &cur_heap_idx,
const ObChunkDatumStore::StoredRow *&store_row);
int array_next_stored_row(
const ObChunkDatumStore::StoredRow *&sr);
@ -479,6 +520,8 @@ protected:
const ObChunkDatumStore::StoredRow *&sr);
int ems_heap_next_stored_row(
const ObChunkDatumStore::StoredRow *&sr);
int part_heap_next_stored_row(
const ObChunkDatumStore::StoredRow *&sr);
// 这里need dump外加两个条件: 1) data_size > expect_size 2) mem_used > global_bound
// 为什么如此,原因在于expect size可能是one pass size,所以数据大于expect size,
@ -510,8 +553,13 @@ protected:
int is_equal_part(const ObChunkDatumStore::StoredRow *l, const ObChunkDatumStore::StoredRow *r, bool &is_equal);
int do_partition_sort(common::ObIArray<ObChunkDatumStore::StoredRow *> &rows,
const int64_t rows_begin, const int64_t rows_end);
int do_partition_topn_sort();
void set_blk_holder(ObChunkDatumStore::IteratedBlockHolder *blk_holder);
bool is_in_same_heap(const SortStoredRow *l,
const SortStoredRow*r);
// for topn sort
int init_topn();
void reuse_topn_heap(TopnHeapNode *topn_heap);
int add_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
const ObChunkDatumStore::StoredRow *&store_row);
int add_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
@ -543,13 +591,32 @@ protected:
SortStoredRow *&new_row);
int generate_last_ties_row(const ObChunkDatumStore::StoredRow *orign_row);
int adjust_topn_read_rows(ObChunkDatumStore::StoredRow **stored_rows, int64_t &read_cnt);
// for partition topn
int init_partition_topn();
void reuse_part_topn_heap();
int locate_current_heap(const common::ObIArray<ObExpr*> &exprs);
int locate_current_heap_in_bucket(PartHeapNode *first_node,
const common::ObIArray<ObExpr*> &exprs,
PartHeapNode *&exist);
void clean_up_topn_heap(TopnHeap *&topn_heap);
int add_part_heap_sort_row(const common::ObIArray<ObExpr*> &exprs,
const ObChunkDatumStore::StoredRow *&store_row);
int add_part_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
const ObBitVector &skip,
const int64_t batch_size,
const int64_t start_pos /* 0 */,
int64_t *append_row_count = nullptr);
int add_part_heap_sort_batch(const common::ObIArray<ObExpr *> &exprs,
const ObBitVector &skip,
const int64_t batch_size,
const uint16_t selector[],
const int64_t size);
DISALLOW_COPY_AND_ASSIGN(ObSortOpImpl);
protected:
typedef common::ObBinaryHeap<ObChunkDatumStore::StoredRow **, Compare, 16> IMMSHeap;
typedef common::ObBinaryHeap<ObSortOpChunk *, Compare, MAX_MERGE_WAYS> EMSHeap;
typedef common::ObBinaryHeap<ObChunkDatumStore::StoredRow *, Compare> TopnHeap;
//typedef common::ObBinaryHeap<ObChunkDatumStore::StoredRow *, Compare> TopnHeap;
static const int64_t MAX_ROW_CNT = 268435456; // (2G / 8)
static const int64_t STORE_ROW_HEADER_SIZE = sizeof(SortStoredRow);
static const int64_t STORE_ROW_EXTRA_SIZE = sizeof(uint64_t);
@ -601,10 +668,14 @@ protected:
// for topn sort
bool use_heap_sort_;
bool is_fetch_with_ties_;
TopnHeap *topn_heap_;
TopnHeapNode *topn_heap_;
int64_t ties_array_pos_;
common::ObArray<SortStoredRow *> ties_array_;
ObChunkDatumStore::StoredRow *last_ties_row_;
// for window function partition topn sort
PartHeapNode **pt_buckets_;
bool use_partition_topn_sort_;
ObSEArray<TopnHeapNode*, 16> heap_nodes_;
int64_t cur_heap_idx_;
common::ObIArray<ObChunkDatumStore::StoredRow *> *rows_;
ObChunkDatumStore::IteratedBlockHolder blk_holder_;
};

View File

@ -8304,6 +8304,7 @@ int ObLogPlan::allocate_sort_and_exchange_as_top(ObLogicalOperator *&top,
const OrderItem *hash_sortkey)
{
int ret = OB_SUCCESS;
bool is_part_topn = (NULL != hash_sortkey) && (NULL != topn_expr);
if (OB_ISNULL(top)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
@ -8313,7 +8314,7 @@ int ObLogPlan::allocate_sort_and_exchange_as_top(ObLogicalOperator *&top,
} else { /*do nothing*/ }
} else {
// allocate push down limit if necessary
if (NULL != topn_expr && !need_sort) {
if (NULL != topn_expr && !need_sort && !is_part_topn) {
bool is_pushed = false;
if (!is_fetch_with_ties &&
OB_FAIL(try_push_limit_into_table_scan(top, topn_expr, topn_expr, NULL, is_pushed))) {
@ -8392,15 +8393,15 @@ int ObLogPlan::allocate_sort_and_exchange_as_top(ObLogicalOperator *&top,
sort_keys,
real_prefix_pos,
real_local_order,
NULL,
false,
topn_expr,
is_fetch_with_ties,
hash_sortkey))) {
LOG_WARN("failed to allocate sort as top", K(ret));
} else { /*do nothing*/ }
}
// allocate final limit if necessary
if (OB_SUCC(ret) && NULL != topn_expr && exch_info.is_pq_local()) {
if (OB_SUCC(ret) && NULL != topn_expr && exch_info.is_pq_local() && !is_part_topn) {
if (OB_FAIL(allocate_limit_as_top(top,
topn_expr,
NULL,

View File

@ -248,7 +248,11 @@ const char *ObLogSort::get_name() const
{
const char *ret = NULL;
if (NULL != topn_expr_) {
ret = "TOP-N SORT";
if (part_cnt_ > 0) {
ret = "PARTITION TOP-N SORT";
} else {
ret = "TOP-N SORT";
}
} else if (NULL == topk_limit_expr_ && prefix_pos_ <= 0 && part_cnt_ > 0) {
ret = "PARTITION SORT";
}
@ -403,7 +407,33 @@ int ObLogSort::inner_est_cost(const int64_t parallel, double child_card, double
if (OB_FAIL(ObOptEstCost::cost_sort(cost_info, op_cost, opt_ctx))) {
LOG_WARN("failed to calc cost", K(ret), K(child->get_type()));
} else if (NULL != topn_expr_) {
double_topn_count = std::min(double_topn_count * parallel, child_card);
if (part_cnt_ > 0) {
//partition topn sort
ObSEArray<ObRawExpr*, 4> part_exprs;
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); ++i) {
if (i < cost_info.part_cnt_) {
if (OB_FAIL(part_exprs.push_back(sort_keys_.at(i).expr_))) {
LOG_WARN("fail to push back expr", K(ret));
}
}
}
if (OB_SUCC(ret)) {
double child_rows = child_card / parallel;
double distinct_parts = child_rows;
if (OB_FAIL(ObOptSelectivity::calculate_distinct(get_plan()->get_update_table_metas(),
get_plan()->get_selectivity_ctx(),
part_exprs,
child_rows,
distinct_parts))) {
LOG_WARN("failed to calculate distinct", K(ret));
} else if (OB_UNLIKELY(distinct_parts < 1.0 || distinct_parts > child_rows)) {
distinct_parts = child_rows;
}
double_topn_count = std::min(distinct_parts * double_topn_count * parallel, child_card);
}
} else {
double_topn_count = std::min(double_topn_count * parallel, child_card);
}
}
}
return ret;

View File

@ -236,27 +236,63 @@ uint64_t ObLogWindowFunction::hash(uint64_t seed) const
return seed;
}
int ObLogWindowFunction::est_cost()
int ObLogWindowFunction::inner_est_cost(double child_card, double child_width, double &op_cost)
{
int ret = OB_SUCCESS;
int64_t parallel = 0;
op_cost = 0.0;
if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan is null", K(ret));
} else if (OB_UNLIKELY((parallel = get_parallel()) < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected parallel degree", K(parallel), K(ret));
} else if (OB_FAIL(ObOptEstCost::cost_window_function(child_card / parallel,
child_width,
win_exprs_.count(),
op_cost,
get_plan()->get_optimizer_context()))) {
LOG_WARN("calculate cost of window function failed", K(ret));
} else {
ObSEArray<ObBasicCostInfo, 1> cost_infos;
double filter_op_cost = 0.0;
if (OB_SUCC(ret) && !filter_exprs_.empty()) {
if (OB_FAIL(cost_infos.push_back(ObBasicCostInfo(child_card, get_cost(), get_width(),is_exchange_allocated())))) {
LOG_WARN("push back cost info failed.", K(ret));
} else {
common::ObBitSet<> dummy_onetime;
common::ObBitSet<> dummy_init;
ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context();
ObSubplanFilterCostInfo info(cost_infos, dummy_onetime, dummy_init);
if (OB_FAIL(ObOptEstCost::cost_subplan_filter(info, filter_op_cost, opt_ctx))) {
LOG_WARN("failed to calculate the cost of subplan filter", K(ret));
}
}
}
op_cost += filter_op_cost;
}
return ret;
}
int ObLogWindowFunction::est_cost()
{
int ret = OB_SUCCESS;
ObLogicalOperator *first_child = NULL;
double child_card = 0.0;
double child_width = 0.0;
double sel = 0.0;
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 if (OB_UNLIKELY((parallel = get_parallel()) < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected parallel degree", K(parallel), K(ret));
} else if (OB_FAIL(ObOptEstCost::cost_window_function(first_child->get_card() / parallel,
first_child->get_width(),
win_exprs_.count(),
op_cost_,
get_plan()->get_optimizer_context()))) {
} else if (OB_FAIL(get_child_est_info(child_card, child_width, sel))) {
LOG_WARN("get child est info failed", K(ret));
} else if (OB_FAIL(inner_est_cost(child_card, child_width, op_cost_))) {
LOG_WARN("calculate cost of window function failed", K(ret));
} else {
set_card(first_child->get_card());
set_cost(first_child->get_cost() + op_cost_);
set_op_cost(op_cost_);
set_card(child_card * sel);
}
return ret;
}
@ -266,28 +302,72 @@ int ObLogWindowFunction::do_re_est_cost(EstimateCostInfo &param, double &card, d
int ret = OB_SUCCESS;
const int64_t parallel = param.need_parallel_;
ObLogicalOperator *child = NULL;
double child_card = 0.0;
double child_width = 0.0;
double sel = 0.0;
if (OB_ISNULL(get_plan()) ||
OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(child), K(ret));
} else if (OB_FAIL(get_child_est_info(child_card, child_width, sel))) {
LOG_WARN("get child est info failed", K(ret));
} else {
double child_card = child->get_card();
double child_cost = child->get_cost();
ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context();
if (is_block_op()) {
param.need_row_count_ = -1; //reset need row count
}
if (OB_FAIL(SMART_CALL(child->re_est_cost(param, child_card, child_cost)))) {
else if (sel <= OB_DOUBLE_EPSINON || param.need_row_count_ >= sel * child_card) {
param.need_row_count_ = -1;
} else if (param.need_row_count_ > 0) {
param.need_row_count_ /= sel;
}
if (OB_FAIL(ret)) {
//do nothing
} else if (OB_FAIL(SMART_CALL(child->re_est_cost(param, child_card, child_cost)))) {
LOG_WARN("failed to re est exchange cost", K(ret));
} else if (OB_FAIL(ObOptEstCost::cost_window_function(child_card / parallel,
child->get_width(),
win_exprs_.count(),
op_cost,
opt_ctx))) {
} else if (OB_FAIL(inner_est_cost(child_card, child_width, op_cost))) {
LOG_WARN("calculate cost of window function failed", K(ret));
} else {
cost = child_cost + op_cost;
card = child_card;
card = sel * child_card;
}
}
return ret;
}
int ObLogWindowFunction::get_child_est_info(double &child_card, double &child_width, double &selectivity)
{
int ret = OB_SUCCESS;
selectivity = 1.0;
ObLogicalOperator *child = get_child(ObLogicalOperator::first_child);
if (OB_ISNULL(child) || OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(child));
} else {
child_card = child->get_card();
child_width = child->get_width();
}
if (OB_FAIL(ret)) {
} else if (0 == child_card) {
selectivity = 1.0;
} else if (!filter_exprs_.empty()) {
if (OB_FALSE_IT(get_plan()->get_selectivity_ctx().init_op_ctx(&get_output_equal_sets(),
use_topn_sort_ ? origin_sort_card_ : get_card()))) {
} else if (OB_FAIL(ObOptSelectivity::calculate_selectivity(get_plan()->get_update_table_metas(),
get_plan()->get_selectivity_ctx(),
filter_exprs_,
selectivity,
get_plan()->get_predicate_selectivities()))) {
LOG_WARN("failed to calc selectivity", K(ret));
} else if (use_topn_sort_) {
//calc the card of window func operator without pushing down the topn filter first.
//win_card / child_card is the precise selectivity
double win_card = selectivity * origin_sort_card_;
selectivity = win_card / child_card;
if (selectivity > 1.0) {
selectivity = 1.0;
}
}
}
return ret;
@ -436,7 +516,8 @@ int ObLogWindowFunction::add_win_dist_options(const ObLogicalOperator *op,
win_func->get_window_exprs(),
win_func->get_win_dist_algo(),
win_func->is_push_down(),
win_func->get_use_hash_sort()))) {
win_func->get_use_hash_sort(),
win_func->get_use_topn_sort()))) {
LOG_WARN("failed to add win dist option", K(ret));
}
}
@ -470,6 +551,7 @@ int ObLogWindowFunction::print_used_hint(PlanText &plan_text)
hint_match = hint_opt.algo_ == outline_opt.algo_
&& hint_opt.use_hash_sort_ == outline_opt.use_hash_sort_
&& hint_opt.is_push_down_ == outline_opt.is_push_down_
&& hint_opt.use_topn_sort_ == outline_opt.use_topn_sort_
&& (hint_opt.win_func_idxs_.empty()
|| is_array_equal(hint_opt.win_func_idxs_, outline_opt.win_func_idxs_));
}

View File

@ -31,12 +31,14 @@ namespace sql
: ObLogicalOperator(plan),
algo_(WinDistAlgo::WIN_DIST_INVALID),
use_hash_sort_(false),
use_topn_sort_(false),
single_part_parallel_(false),
range_dist_parallel_(false),
role_type_(WindowFunctionRoleType::NORMAL),
rd_sort_keys_cnt_(0),
rd_pby_sort_cnt_(0),
wf_aggr_status_expr_(NULL)
wf_aggr_status_expr_(NULL),
origin_sort_card_(0.0)
{}
virtual ~ObLogWindowFunction() {}
virtual int get_explain_name_internal(char *buf,
@ -53,6 +55,8 @@ namespace sql
virtual int est_cost() override;
virtual int est_width() override;
virtual int do_re_est_cost(EstimateCostInfo &param, double &card, double &op_cost, double &cost) override;
int get_child_est_info(double &child_card, double &child_width, double &selectivity);
int inner_est_cost(double child_card, double child_width, double &op_cost);
virtual int get_op_exprs(ObIArray<ObRawExpr*> &all_exprs) override;
virtual int is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) override;
virtual int compute_op_ordering() override;
@ -98,6 +102,10 @@ namespace sql
WinDistAlgo get_win_dist_algo() const { return algo_; }
void set_use_hash_sort(const bool use_hash_sort) { use_hash_sort_ = use_hash_sort; }
bool get_use_hash_sort() const { return use_hash_sort_; }
void set_use_topn_sort(const bool use_topn_sort) { use_topn_sort_ = use_topn_sort; }
bool get_use_topn_sort() const { return use_topn_sort_; }
void set_origin_sort_card(double origin_sort_card) { origin_sort_card_ = origin_sort_card; }
double get_origin_sort_card() { return origin_sort_card_; }
virtual int get_plan_item_info(PlanText &plan_text,
ObSqlPlanItem &plan_item) override;
virtual int print_outline_data(PlanText &plan_text) override;
@ -111,6 +119,7 @@ namespace sql
// for print PQ_DISTRIBUTE_WINDOW hint outline
WinDistAlgo algo_;
bool use_hash_sort_;
bool use_topn_sort_;
// Single partition (no partition by) window function parallel process, need the PX COORD
// to collect the partial result and broadcast the final result to each worker.
@ -143,6 +152,9 @@ namespace sql
// for reporting window function adaptive pushdown
ObOpPseudoColumnRawExpr *wf_aggr_status_expr_;
common::ObSEArray<bool, 8, common::ModulePageAllocator, true> pushdown_info_;
//for est_cost when use topn sort
double origin_sort_card_;
};
}
}

View File

@ -498,6 +498,14 @@ int ObOptEstCostModel::cost_sort(const ObSortCostInfo &cost_info,
// get_next_row获取下层算子行的代价
cost += cost_params_.get_cpu_tuple_cost(sys_stat_) * cost_info.rows_;
}
} else if (cost_info.part_cnt_ > 0 && cost_info.topn_ >= 0) {
//part topn sort/part topn limit
if (OB_FAIL(cost_part_topn_sort(cost_info, order_exprs, order_types, cost))) {
LOG_WARN("failed to calc part cost", K(ret));
} else {
// get_next_row获取下层算子行的代价
cost += cost_params_.get_cpu_tuple_cost(sys_stat_) * cost_info.rows_;
}
} else if (cost_info.topn_ >= 0) {
//top-n sort
if (OB_FAIL(cost_topn_sort(cost_info, order_types, cost))) {
@ -644,6 +652,70 @@ int ObOptEstCostModel::cost_part_sort(const ObSortCostInfo &cost_info,
return ret;
}
int ObOptEstCostModel::cost_part_topn_sort(const ObSortCostInfo &cost_info,
const ObIArray<ObRawExpr *> &order_exprs,
const ObIArray<ObExprResType> &order_col_types,
double &cost)
{
int ret = OB_SUCCESS;
cost = 0.0;
double real_sort_cost = 0.0;
double material_cost = 0.0;
double calc_hash_cost = 0.0;
double rows = cost_info.rows_;
double width = cost_info.width_;
double distinct_parts = rows;
ObSEArray<ObRawExpr*, 4> part_exprs;
ObSEArray<ObExprResType, 4> sort_types;
for (int64_t i = 0; OB_SUCC(ret) && i < order_exprs.count(); ++i) {
if (i < cost_info.part_cnt_) {
if (OB_FAIL(part_exprs.push_back(order_exprs.at(i)))) {
LOG_WARN("fail to push back expr", K(ret));
}
} else {
if (OB_FAIL(sort_types.push_back(order_col_types.at(i)))) {
LOG_WARN("fail to push back type", K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ObOptSelectivity::calculate_distinct(*cost_info.table_metas_,
*cost_info.sel_ctx_,
part_exprs,
rows,
distinct_parts))) {
LOG_WARN("failed to calculate distinct", K(ret));
} else if (OB_UNLIKELY(distinct_parts < 1.0 || distinct_parts > rows)) {
distinct_parts = rows;
}
}
if (OB_SUCC(ret)) {
//partition topn sort
double topn = cost_info.topn_;
double one_part_rows = rows;
if (distinct_parts != 0) {
one_part_rows = rows / distinct_parts;
}
if (topn > one_part_rows) {
topn = one_part_rows;
}
material_cost = cost_material(topn, width) * distinct_parts;
if (sort_types.count() > 0 && OB_FAIL(cost_topn_sort_inner(sort_types, one_part_rows, topn, real_sort_cost))) {
LOG_WARN("failed to calc cost", K(ret));
} else {
real_sort_cost = real_sort_cost * distinct_parts;
calc_hash_cost = cost_hash(rows, part_exprs) + rows * cost_params_.get_build_hash_per_row_cost(sys_stat_) / 2.0;
cost = material_cost + real_sort_cost + calc_hash_cost;
LOG_TRACE("OPT: [COST PARTITION TOPN SORT]", K(cost), K(calc_hash_cost), K(material_cost),
K(real_sort_cost), K(rows), K(width), K(topn), K(cost_info.part_cnt_));
}
}
return ret;
}
int ObOptEstCostModel::cost_prefix_sort(const ObSortCostInfo &cost_info,
const ObIArray<ObRawExpr *> &order_exprs,
const int64_t topn_count,

View File

@ -804,6 +804,10 @@ protected:
const ObIArray<ObRawExpr *> &order_exprs,
const ObIArray<ObExprResType> &order_col_types,
double &cost);
int cost_part_topn_sort(const ObSortCostInfo &cost_info,
const ObIArray<ObRawExpr *> &order_exprs,
const ObIArray<ObExprResType> &order_col_types,
double &cost);
int cost_prefix_sort(const ObSortCostInfo &cost_info,
const ObIArray<ObRawExpr *> &order_exprs,

View File

@ -1287,15 +1287,18 @@ int ObSelectLogPlan::allocate_window_function_as_top(const WinDistAlgo dist_algo
const bool match_parallel,
const bool is_partition_wise,
const bool use_hash_sort,
const bool use_topn_sort,
const ObIArray<OrderItem> &sort_keys,
ObLogicalOperator *&top)
ObLogicalOperator *&top,
const ObIArray<ObRawExpr *> *qualify_filters,
double origin_sort_card)
{
const int32_t role_type = ObLogWindowFunction::WindowFunctionRoleType::NORMAL;
const int64_t range_dist_keys_cnt = 0;
const int64_t range_dist_pby_prefix = 0;
return allocate_window_function_as_top(dist_algo, win_exprs, match_parallel, is_partition_wise,
use_hash_sort, role_type, sort_keys, range_dist_keys_cnt,
range_dist_pby_prefix, top, NULL, NULL);
use_hash_sort, use_topn_sort, role_type, sort_keys, range_dist_keys_cnt,
range_dist_pby_prefix, top, qualify_filters, origin_sort_card, NULL, NULL);
}
int ObSelectLogPlan::allocate_window_function_as_top(const WinDistAlgo dist_algo,
@ -1303,11 +1306,14 @@ int ObSelectLogPlan::allocate_window_function_as_top(const WinDistAlgo dist_algo
const bool match_parallel,
const bool is_partition_wise,
const bool use_hash_sort,
const bool use_topn_sort,
const int32_t role_type,
const ObIArray<OrderItem> &sort_keys,
const int64_t range_dist_keys_cnt,
const int64_t range_dist_pby_prefix,
ObLogicalOperator *&top,
const ObIArray<ObRawExpr *> *qualify_filters,
double origin_sort_card,
ObOpPseudoColumnRawExpr *wf_aggr_status_expr, /* default null */
const ObIArray<bool> *pushdown_info /* default null */)
{
@ -1332,6 +1338,7 @@ int ObSelectLogPlan::allocate_window_function_as_top(const WinDistAlgo dist_algo
window_function->set_child(ObLogicalOperator::first_child, top);
window_function->set_role_type(ObLogWindowFunction::WindowFunctionRoleType(role_type));
window_function->set_use_hash_sort(use_hash_sort);
window_function->set_use_topn_sort(use_topn_sort);
if (range_dist_keys_cnt > 0) {
window_function->set_ragne_dist_parallel(true);
window_function->set_rd_pby_sort_cnt(range_dist_pby_prefix);
@ -1347,6 +1354,13 @@ int ObSelectLogPlan::allocate_window_function_as_top(const WinDistAlgo dist_algo
window_function->set_aggr_status_expr(wf_aggr_status_expr);
}
}
if (OB_SUCC(ret)) {
if (NULL != qualify_filters && OB_FAIL(window_function->get_filter_exprs().assign(*qualify_filters))) {
LOG_WARN("assign win filters failed", K(ret));
} else if (use_topn_sort) {
window_function->set_origin_sort_card(origin_sort_card);
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(window_function->compute_property())) {
LOG_WARN("failed to compute property", K(ret));
@ -4737,6 +4751,7 @@ int ObSelectLogPlan::candi_allocate_window_function()
} else if (OB_FAIL(candi_allocate_subplan_filter_for_exprs(candi_subquery_exprs))) {
LOG_WARN("failed to do allocate subplan filter", K(ret));
} else if (OB_FAIL(candi_allocate_window_function_with_hint(stmt->get_window_func_exprs(),
stmt->get_qualify_filters(),
win_func_plans))) {
LOG_WARN("failed to allocate window function with hint", K(ret));
} else if (!win_func_plans.empty()) {
@ -4744,6 +4759,7 @@ int ObSelectLogPlan::candi_allocate_window_function()
} else if (OB_FAIL(get_log_plan_hint().check_status())) {
LOG_WARN("failed to generate plans with hint", K(ret));
} else if (OB_FAIL(candi_allocate_window_function(stmt->get_window_func_exprs(),
stmt->get_qualify_filters(),
win_func_plans))) {
LOG_WARN("failed to allocate window function", K(ret));
} else {
@ -4773,6 +4789,7 @@ int ObSelectLogPlan::candi_allocate_window_function()
// 4. method matched
// window functions in win_func_exprs must keep the same ordering with win_func_exprs_ in stmt
int ObSelectLogPlan::candi_allocate_window_function_with_hint(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
const ObIArray<ObRawExpr*> &qualify_filters,
common::ObIArray<CandidatePlan> &total_plans)
{
int ret = OB_SUCCESS;
@ -4802,7 +4819,8 @@ int ObSelectLogPlan::candi_allocate_window_function_with_hint(const ObIArray<ObW
orig_top->get_output_equal_sets(),
orig_top->get_output_const_exprs(),
orig_top->get_card(),
orig_top->get_is_at_most_one_row());
orig_top->get_is_at_most_one_row(),
qualify_filters);
while (OB_SUCC(ret) && !candi_plans.empty() && !remaining_exprs.empty()) {
tmp_plans.reuse();
if (OB_FAIL(init_win_func_helper_with_hint(candi_plans,
@ -4834,8 +4852,10 @@ int ObSelectLogPlan::candi_allocate_window_function_with_hint(const ObIArray<ObW
}
}
}
if (OB_SUCC(ret) && OB_FAIL(total_plans.assign(candi_plans))) {
LOG_WARN("failed to assign candidate plans", K(ret));
if (OB_SUCC(ret)) {
if (OB_FAIL(total_plans.assign(candi_plans))) {
LOG_WARN("failed to assign candidate plans", K(ret));
}
}
}
return ret;
@ -4944,6 +4964,9 @@ int ObSelectLogPlan::init_win_func_helper_with_hint(const ObIArray<CandidatePlan
win_func_helper.wf_aggr_status_expr_))) {
LOG_WARN("failed to build inner wf aggr status expr", K(ret));
}
if (OB_SUCC(ret) && is_valid) {
win_func_helper.need_qualify_filter_ = remaining_exprs.empty();
}
LOG_TRACE("finish init win_func_helper with hint. ", K(is_valid), K(win_func_helper));
}
return ret;
@ -4968,6 +4991,20 @@ int ObSelectLogPlan::calc_win_func_helper_with_hint(const ObLogicalOperator *op,
LOG_WARN("failed to calc ndvs and pby oby prefix", K(ret));
} else if (OB_FAIL(calc_partition_count(win_func_helper))) {
LOG_WARN("failed to get partition count", K(ret));
} else if (OB_ISNULL(win_func_helper.win_dist_hint_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else {
const ObWindowDistHint *hint = win_func_helper.win_dist_hint_;
const ObWindowDistHint::WinDistOption &option = hint->get_win_dist_options().at(win_func_helper.win_op_idx_);
if (!option.use_topn_sort_) {
win_func_helper.enable_topn_ = false;
win_func_helper.topn_const_ = NULL;
} else if (OB_FAIL(init_wf_topn_option(win_func_helper, true))) {
LOG_WARN("choose topn filter failed", K(ret));
} else if (!win_func_helper.enable_topn_) {
is_valid = false;
}
}
return ret;
}
@ -5017,6 +5054,7 @@ int ObSelectLogPlan::check_win_dist_method_valid(const WinFuncOpHelper &win_func
}
int ObSelectLogPlan::candi_allocate_window_function(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
const ObIArray<ObRawExpr*> &qualify_filters,
ObIArray<CandidatePlan> &total_plans)
{
int ret = OB_SUCCESS;
@ -5036,13 +5074,15 @@ int ObSelectLogPlan::candi_allocate_window_function(const ObIArray<ObWinFunRawEx
orig_top->get_output_equal_sets(),
orig_top->get_output_const_exprs(),
orig_top->get_card(),
orig_top->get_is_at_most_one_row());
orig_top->get_is_at_most_one_row(),
qualify_filters);
for (int64_t i = 0; OB_SUCC(ret) && i < candi_plans.count(); ++i) {
OPT_TRACE("generate window function for plan:", candi_plans.at(i));
if (OB_FAIL(generate_window_functions_plan(win_func_helper,
status_exprs,
total_plans,
candi_plans.at(i)))) {
candi_plans.at(i)
))) {
LOG_WARN("failed to allocate window functions", K(ret));
} else { /*do nothing*/ }
}
@ -5095,6 +5135,7 @@ int ObSelectLogPlan::generate_window_functions_plan(WinFuncOpHelper &win_func_he
methods,
si,
status_exprs,
remaining_exprs.empty(),
win_func_helper))) {
LOG_WARN("failed to init win func helper", K(ret));
} else {
@ -5151,9 +5192,9 @@ int ObSelectLogPlan::create_one_window_function(CandidatePlan &candidate_plan,
part_cnt))) {
LOG_WARN("failed to check win func need sort", K(ret));
} else if (OB_FAIL(create_range_list_dist_win_func(top,
win_func_helper,
part_cnt,
all_plans))) {
win_func_helper,
part_cnt,
all_plans))) {
LOG_WARN("failed to create range list dist window functions", K(ret));
} else if (OB_FAIL(create_none_dist_win_func(top,
win_func_helper,
@ -5275,6 +5316,7 @@ int ObSelectLogPlan::init_win_func_helper(const ObIArray<ObWinFunRawExpr*> &orde
const ObIArray<WinDistAlgo> &methods,
const int64_t splict_idx,
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
bool is_last_group,
WinFuncOpHelper &win_func_helper)
{
int ret = OB_SUCCESS;
@ -5294,6 +5336,12 @@ int ObSelectLogPlan::init_win_func_helper(const ObIArray<ObWinFunRawExpr*> &orde
ObOpPseudoColumnRawExpr *status_expr = NULL;
const int64_t start = 0 == splict_idx ? 0 : split.at(splict_idx - 1);
const int64_t end = split.at(splict_idx);
if (!is_last_group || splict_idx != split.count() - 1) {
//only alloc qualify filter on the top win func op
win_func_helper.need_qualify_filter_ = false;
} else {
win_func_helper.need_qualify_filter_ = true;
}
for (int64_t i = start; OB_SUCC(ret) && i < end; ++i) {
if (OB_FAIL(win_func_helper.ordered_win_func_exprs_.push_back(ordered_win_func_exprs.at(i)))
|| OB_FAIL(win_func_helper.pby_oby_prefixes_.push_back(pby_oby_prefixes.at(i)))) {
@ -5325,6 +5373,11 @@ int ObSelectLogPlan::init_win_func_helper(const ObIArray<ObWinFunRawExpr*> &orde
} else {
status_exprs.at(win_func_helper.win_op_idx_) = win_func_helper.wf_aggr_status_expr_;
}
if (OB_SUCC(ret)) {
if (OB_FAIL(init_wf_topn_option(win_func_helper, false))) {
LOG_WARN("choose topn filter failed", K(ret));
}
}
LOG_TRACE("finish init win_func_helper. ", K(win_func_helper));
}
return ret;
@ -5936,6 +5989,8 @@ int ObSelectLogPlan::create_none_dist_win_func(ObLogicalOperator *top,
const ObIArray<ObWinFunRawExpr*> &win_func_exprs = win_func_helper.ordered_win_func_exprs_;
const ObIArray<OrderItem> &sort_keys = win_func_helper.sort_keys_;
bool is_local_order = false;
ObRawExpr *topn_const = NULL;
bool is_with_ties = false;
if (OB_ISNULL(top)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
@ -5965,17 +6020,25 @@ int ObSelectLogPlan::create_none_dist_win_func(ObLogicalOperator *top,
}
if (OB_SUCC(ret) && need_normal_sort) {
ObLogicalOperator *normal_sort_top= top;
ObLogicalOperator *normal_sort_top = top;
bool use_topn = win_func_helper.enable_topn_
&& win_func_helper.partition_exprs_.empty()
&& NULL != win_func_helper.topn_const_;
if (OB_FAIL(allocate_sort_and_exchange_as_top(normal_sort_top, exch_info, sort_keys, need_sort,
prefix_pos, is_local_order))) {
prefix_pos, is_local_order,
use_topn ? win_func_helper.topn_const_ : NULL, /* topn_expr */
use_topn ? win_func_helper.is_fetch_with_ties_ : false /* is_fetch_with_ties */))) {
LOG_WARN("failed to allocate sort and exchange as top", K(ret));
} else if (OB_FAIL(allocate_window_function_as_top(WinDistAlgo::WIN_DIST_NONE,
win_func_exprs,
single_part_parallel,
is_partition_wise,
false, /* use hash sort */
use_topn, /* use topn sort */
sort_keys,
normal_sort_top))) {
normal_sort_top,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL,
use_topn ? win_func_helper.origin_sort_card_ : 0))) {
LOG_WARN("failed to allocate window function as top", K(ret));
} else if (OB_FAIL(all_plans.push_back(CandidatePlan(normal_sort_top)))) {
LOG_WARN("failed to push back", K(ret));
@ -5985,12 +6048,15 @@ int ObSelectLogPlan::create_none_dist_win_func(ObLogicalOperator *top,
if (OB_SUCC(ret) && need_hash_sort) {
ObLogicalOperator *hash_sort_top= top;
OrderItem hash_sortkey;
bool use_part_topn = win_func_helper.enable_topn_
&& prefix_pos == 0
&& NULL != win_func_helper.topn_const_;
if (OB_FAIL(create_hash_sortkey(part_cnt, sort_keys, hash_sortkey))) {
LOG_WARN("failed to create hash sort key", K(ret), K(part_cnt), K(sort_keys));
} else if (OB_FAIL(allocate_sort_and_exchange_as_top(hash_sort_top, exch_info, sort_keys, need_sort,
prefix_pos, is_local_order,
NULL, /* topn_expr */
false, /* is_fetch_with_ties */
use_part_topn ? win_func_helper.topn_const_ : NULL, /* topn_expr */
use_part_topn ? win_func_helper.is_fetch_with_ties_ : false, /* is_fetch_with_ties */
&hash_sortkey))) {
LOG_WARN("failed to allocate sort and exchange as top", K(ret));
} else if (OB_FAIL(allocate_window_function_as_top(WinDistAlgo::WIN_DIST_NONE,
@ -5998,8 +6064,11 @@ int ObSelectLogPlan::create_none_dist_win_func(ObLogicalOperator *top,
single_part_parallel,
is_partition_wise,
true, /* use hash sort */
use_part_topn, /* use partition topn sort */
sort_keys,
hash_sort_top))) {
hash_sort_top,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL,
use_part_topn ? win_func_helper.origin_sort_card_ : 0))) {
LOG_WARN("failed to allocate window function as top", K(ret));
} else if (OB_FAIL(all_plans.push_back(CandidatePlan(hash_sort_top)))) {
LOG_WARN("failed to push back", K(ret));
@ -6064,11 +6133,14 @@ int ObSelectLogPlan::create_range_list_dist_win_func(ObLogicalOperator *top,
single_part_parallel,
is_partition_wise,
false, /* use hash sort */
false, /* use hash topn sort */
ObLogWindowFunction::WindowFunctionRoleType::NORMAL,
sort_keys,
range_dist_keys.count(),
pby_prefix,
top))) {
top,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL,
0))) {
LOG_WARN("failed to allocate window function as top", K(ret));
} else if (OB_FAIL(all_plans.push_back(CandidatePlan(top)))) {
LOG_WARN("failed to push back", K(ret));
@ -6169,6 +6241,8 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
bool is_partition_wise = false;
const ObIArray<ObWinFunRawExpr*> &win_func_exprs = win_func_helper.ordered_win_func_exprs_;
const ObIArray<OrderItem> &sort_keys = win_func_helper.sort_keys_;
ObRawExpr *topn_const = NULL;
bool is_with_ties = false;
if (OB_ISNULL(top)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
@ -6195,6 +6269,9 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
}
if (OB_SUCC(ret) && need_normal_sort) {
ObLogicalOperator *normal_sort_top= top;
bool use_topn = win_func_helper.enable_topn_
&& win_func_helper.partition_exprs_.empty()
&& NULL != win_func_helper.topn_const_;
if (!need_pushdown &&
OB_FAIL(create_normal_hash_dist_win_func(normal_sort_top,
win_func_exprs,
@ -6202,7 +6279,11 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
sort_keys,
need_sort,
prefix_pos,
NULL))) {
NULL,
use_topn ? win_func_helper.topn_const_ : NULL,
use_topn ? win_func_helper.is_fetch_with_ties_ : false,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL,
use_topn ? win_func_helper.origin_sort_card_ : 0))) {
LOG_WARN("failed to create normal hash dist window function", K(ret));
} else if (need_pushdown &&
OB_FAIL(create_pushdown_hash_dist_win_func(normal_sort_top,
@ -6212,7 +6293,8 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
win_func_helper.wf_aggr_status_expr_,
need_sort,
prefix_pos,
NULL))) {
NULL,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL))) {
LOG_WARN("failed to create push down hash dist window function", K(ret));
} else if (OB_FAIL(all_plans.push_back(CandidatePlan(normal_sort_top)))) {
LOG_WARN("failed to push back", K(ret));
@ -6222,6 +6304,9 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
if (OB_SUCC(ret) && need_hash_sort) {
ObLogicalOperator *hash_sort_top= top;
OrderItem hash_sortkey;
bool use_part_topn = win_func_helper.enable_topn_
&& prefix_pos == 0
&& NULL != win_func_helper.topn_const_;
if (OB_FAIL(create_hash_sortkey(part_cnt, sort_keys, hash_sortkey))) {
LOG_WARN("failed to create hash sort key", K(ret), K(part_cnt), K(sort_keys));
} else if (!need_pushdown &&
@ -6231,7 +6316,11 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
sort_keys,
need_sort,
prefix_pos,
&hash_sortkey))) {
&hash_sortkey,
use_part_topn ? win_func_helper.topn_const_ : NULL,
use_part_topn ? win_func_helper.is_fetch_with_ties_ : false,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL,
use_part_topn ? win_func_helper.origin_sort_card_ : 0))) {
LOG_WARN("failed to create normal hash dist window function", K(ret));
} else if (need_pushdown &&
OB_FAIL(create_pushdown_hash_dist_win_func(hash_sort_top,
@ -6241,7 +6330,8 @@ int ObSelectLogPlan::create_hash_dist_win_func(ObLogicalOperator *top,
win_func_helper.wf_aggr_status_expr_,
need_sort,
prefix_pos,
&hash_sortkey))) {
&hash_sortkey,
win_func_helper.need_qualify_filter_ ? &win_func_helper.qualify_filters_ : NULL))) {
LOG_WARN("failed to create push down hash dist window function", K(ret));
} else if (OB_FAIL(all_plans.push_back(CandidatePlan(hash_sort_top)))) {
LOG_WARN("failed to push back", K(ret));
@ -6256,7 +6346,11 @@ int ObSelectLogPlan::create_normal_hash_dist_win_func(ObLogicalOperator *&top,
const ObIArray<OrderItem> &sort_keys,
const int64_t need_sort,
const int64_t prefix_pos,
OrderItem *hash_sortkey)
OrderItem *hash_sortkey,
ObRawExpr *topn_const,
bool is_fetch_with_ties,
const ObIArray<ObRawExpr*> *qualify_filters,
double origin_sort_card)
{
int ret = OB_SUCCESS;
ObExchangeInfo exch_info;
@ -6270,8 +6364,8 @@ int ObSelectLogPlan::create_normal_hash_dist_win_func(ObLogicalOperator *&top,
need_sort,
prefix_pos,
top->get_is_local_order(),
NULL, /* topn_expr */
false, /* is_fetch_with_ties */
topn_const, /* topn_expr */
is_fetch_with_ties, /* is_fetch_with_ties */
hash_sortkey))) {
LOG_WARN("failed to allocate sort and exchange as top", K(ret));
} else if (OB_FAIL(allocate_window_function_as_top(WinDistAlgo::WIN_DIST_HASH,
@ -6279,8 +6373,11 @@ int ObSelectLogPlan::create_normal_hash_dist_win_func(ObLogicalOperator *&top,
false, /* match_parallel */
false, /*is_partition_wise*/
NULL != hash_sortkey, /* use hash sort */
NULL != topn_const, /* use topn sort */
sort_keys,
top))) {
top,
qualify_filters,
origin_sort_card))) {
LOG_WARN("failed to allocate window function as top", K(ret));
}
return ret;
@ -6293,7 +6390,8 @@ int ObSelectLogPlan::create_pushdown_hash_dist_win_func(ObLogicalOperator *&top,
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
const int64_t need_sort,
const int64_t prefix_pos,
OrderItem *hash_sortkey)
OrderItem *hash_sortkey,
const ObIArray<ObRawExpr*> *qualify_filters)
{
int ret = OB_SUCCESS;
ObExchangeInfo exch_info;
@ -6319,11 +6417,14 @@ int ObSelectLogPlan::create_pushdown_hash_dist_win_func(ObLogicalOperator *&top,
false, /* match_parallel */
false, /* is_partition_wise */
NULL != hash_sortkey, /* use hash sort */
false, /* use hash topn sort */
ObLogWindowFunction::WindowFunctionRoleType::PARTICIPATOR,
sort_keys,
range_dist_keys_cnt,
range_dist_pby_prefix,
top,
NULL,
0.0,
wf_aggr_status_expr,
&pushdown_info))) {
LOG_WARN("failed to allocate window function as top", K(ret));
@ -6348,11 +6449,14 @@ int ObSelectLogPlan::create_pushdown_hash_dist_win_func(ObLogicalOperator *&top,
false, /* match_parallel */
false, /* is_partition_wise */
NULL != hash_sortkey, /* use hash sort */
false, /* use hash topn sort */
ObLogWindowFunction::WindowFunctionRoleType::CONSOLIDATOR,
sort_keys,
range_dist_keys_cnt,
range_dist_pby_prefix,
top,
qualify_filters,
0.0,
wf_aggr_status_expr,
&pushdown_info))) {
LOG_WARN("failed to allocate window function as top", K(ret));
@ -6493,6 +6597,33 @@ int ObSelectLogPlan::check_wf_range_dist_supported(ObWinFunRawExpr *win_expr,
return ret;
}
int ObSelectLogPlan::check_wf_part_topn_supported(const common::ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
const ObIArray<ObRawExpr*> &partition_exprs,
bool &can_wf_topn)
{
int ret = OB_SUCCESS;
int64_t part_cnt = partition_exprs.count();
can_wf_topn = true;
for (int64 i = 0; OB_SUCC(ret) && can_wf_topn && i < winfunc_exprs.count(); ++i) {
ObWinFunRawExpr *win_expr = winfunc_exprs.at(i);
if (OB_ISNULL(win_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (win_expr->get_partition_exprs().count() != part_cnt) {
can_wf_topn = false;
} else if (win_expr->get_upper().type_ != BoundType::BOUND_UNBOUNDED
|| win_expr->get_lower().type_ != BoundType::BOUND_UNBOUNDED) {
can_wf_topn = false;
} else {
ObItemType wf_type = win_expr->get_func_type();
can_wf_topn = T_WIN_FUN_ROW_NUMBER == wf_type
|| T_WIN_FUN_RANK == wf_type
|| T_WIN_FUN_DENSE_RANK == wf_type;
}
}
return ret;
}
int ObSelectLogPlan::check_wf_pushdown_supported(ObWinFunRawExpr *win_expr,
bool &can_wf_pushdown)
{
@ -6534,6 +6665,112 @@ int ObSelectLogPlan::check_wf_pushdown_supported(ObWinFunRawExpr *win_expr,
return ret;
}
//topn option should be inited after sort keys are inited
int ObSelectLogPlan::init_wf_topn_option(WinFuncOpHelper &win_func_helper, bool wf_topn_hint)
{
int ret = OB_SUCCESS;
const ObIArray<ObWinFunRawExpr *> &winfunc_exprs = win_func_helper.ordered_win_func_exprs_;
const ObIArray<ObRawExpr *> &filter_exprs = win_func_helper.qualify_filters_;
win_func_helper.is_fetch_with_ties_ = false;
win_func_helper.topn_const_ = NULL;
if (winfunc_exprs.count() != win_func_helper.all_win_func_exprs_.count()) {
//not only one group
win_func_helper.enable_topn_ = false;
} else if (wf_topn_hint) {
win_func_helper.enable_topn_ = true;
} else if (OB_ISNULL(get_optimizer_context().get_session_info())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret), K(get_optimizer_context().get_session_info()));
} else {
win_func_helper.enable_topn_ = get_optimizer_context().get_session_info()->is_qualify_filter_enabled();
}
if (OB_FAIL(ret) || !win_func_helper.enable_topn_) {
//do nothing
} else if (check_wf_part_topn_supported(winfunc_exprs,
win_func_helper.partition_exprs_,
win_func_helper.enable_topn_)) {
LOG_WARN("check partition topn supported failed", K(ret));
} else if (win_func_helper.enable_topn_) {
for (int64_t i = 0; OB_SUCC(ret) && NULL == win_func_helper.topn_const_ && i < filter_exprs.count(); ++i) {
ObRawExpr *const_expr = NULL;
bool ties_flag = false;
bool is_topn_filter = false;
ObWinFunRawExpr *win_expr = NULL;
if (OB_FAIL(ObTransformUtils::is_winfunc_topn_filter(winfunc_exprs, filter_exprs.at(i), is_topn_filter,
const_expr, ties_flag, win_expr))) {
LOG_WARN("check whether the filter is a winfunc topn filter failed", K(ret));
} else if (is_topn_filter) {
//order by must be the same as the sort keys
if (OB_ISNULL(win_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else {
const common::ObIArray<OrderItem> &order_items = win_expr->get_order_items();
if (order_items.count() + win_func_helper.part_cnt_ != win_func_helper.sort_keys_.count()) {
is_topn_filter = false;
} else {
for (int64_t j = 0; is_topn_filter && j < order_items.count(); ++j) {
if (order_items.at(j) != win_func_helper.sort_keys_.at(j + win_func_helper.part_cnt_)) {
is_topn_filter = false;
}
}
}
if (is_topn_filter) {
win_func_helper.topn_const_ = const_expr;
win_func_helper.is_fetch_with_ties_ = ties_flag;
}
}
}
}
if (OB_SUCC(ret)
&& NULL == win_func_helper.topn_const_) {
win_func_helper.enable_topn_ = false;
}
}
if (OB_SUCC(ret)
&& NULL != win_func_helper.topn_const_
&& !ob_is_integer_type(win_func_helper.topn_const_->get_result_type().get_type())) {
//cast topn expr to int
ObRawExpr *topn_with_cast = NULL;
ObRawExpr *topn_without_cast = NULL;
ObExprResType res_type;
bool need_cast = false;
bool ignore_err = false;
const ObExprResType &src_type = win_func_helper.topn_const_->get_result_type();
res_type.set_int();
res_type.set_precision(ObAccuracy::DDL_DEFAULT_ACCURACY[ObIntType].precision_);
res_type.set_scale(DEFAULT_SCALE_FOR_INTEGER);
if (OB_FAIL(ObRawExprUtils::check_need_cast_expr(src_type, res_type, need_cast, ignore_err))) {
LOG_WARN("failed to check need cast expr", K(ret), K(src_type), K(res_type));
} else if (!need_cast) {
// do nothing
} else if (OB_ISNULL((topn_without_cast = ObRawExprUtils::skip_implicit_cast(win_func_helper.topn_const_)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (OB_FAIL(ObRawExprUtils::try_add_cast_expr_above(&get_optimizer_context().get_expr_factory(),
get_optimizer_context().get_session_info(),
*topn_without_cast,
res_type,
topn_with_cast))) {
LOG_WARN("create cast expr for stmt failed", K(ret));
} else {
win_func_helper.topn_const_ = topn_with_cast;
}
}
if (OB_SUCC(ret) && win_func_helper.enable_topn_) {
ObLogicalOperator *best_plan = NULL;
if (OB_FAIL(candidates_.get_best_plan(best_plan))) {
LOG_WARN("failed to get best plan", K(ret));
} else if (OB_ISNULL(best_plan)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else {
win_func_helper.origin_sort_card_ = best_plan->get_card();
}
}
return ret;
}
int ObSelectLogPlan::get_pushdown_window_function_exchange_info(
const ObIArray<ObWinFunRawExpr *> &win_exprs,
ObLogicalOperator *op,

View File

@ -483,7 +483,8 @@ private:
const EqualSets &equal_sets,
const ObIArray<ObRawExpr*> &const_exprs,
const double card,
const bool is_at_most_one_row)
const bool is_at_most_one_row,
const ObIArray<ObRawExpr*> &qualify_filters)
: all_win_func_exprs_(all_win_func_exprs),
win_dist_hint_(win_dist_hint),
explicit_hint_(explicit_hint),
@ -499,7 +500,13 @@ private:
force_pushdown_(false),
part_cnt_(false),
win_op_idx_(false),
wf_aggr_status_expr_(NULL)
wf_aggr_status_expr_(NULL),
need_qualify_filter_(false),
qualify_filters_(qualify_filters),
enable_topn_(false),
topn_const_(NULL),
is_fetch_with_ties_(false),
origin_sort_card_(0.0)
{
}
virtual ~WinFuncOpHelper() {}
@ -529,6 +536,14 @@ private:
ObArray<double> sort_key_ndvs_;
ObArray<std::pair<int64_t, int64_t>> pby_oby_prefixes_;
ObArray<OrderItem> sort_keys_;
//if true, add winfunc filters to winfunc operator
bool need_qualify_filter_;
const ObIArray<ObRawExpr*> &qualify_filters_;
//if true, only generate partition topn plans. if false, only generate no partition topn plans.
bool enable_topn_;
ObRawExpr* topn_const_;
bool is_fetch_with_ties_;
double origin_sort_card_;
TO_STRING_KV(K_(win_dist_method),
K_(win_op_idx),
@ -540,7 +555,11 @@ private:
K_(part_cnt),
K_(ordered_win_func_exprs),
K_(win_dist_hint),
K_(explicit_hint));
K_(explicit_hint),
K_(enable_topn),
K_(topn_const),
K_(is_fetch_with_ties),
K_(qualify_filters));
};
private:
@ -554,8 +573,10 @@ private:
int candi_allocate_window_function();
int candi_allocate_window_function_with_hint(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
const ObIArray<ObRawExpr*> &qualify_filters,
common::ObIArray<CandidatePlan> &total_plans);
int candi_allocate_window_function(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
const ObIArray<ObRawExpr*> &qualify_filters,
ObIArray<CandidatePlan> &total_plans);
int create_one_window_function(CandidatePlan &candidate_plan,
const WinFuncOpHelper &win_func_helper,
@ -592,7 +613,11 @@ private:
const ObIArray<OrderItem> &sort_keys,
const int64_t need_sort,
const int64_t prefix_pos,
OrderItem *hash_sortkey);
OrderItem *hash_sortkey,
ObRawExpr *topn_const,
bool is_fetch_with_ties,
const ObIArray<ObRawExpr*> *qualify_filters,
double origin_sort_card);
int create_pushdown_hash_dist_win_func(ObLogicalOperator *&top,
const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
const ObIArray<OrderItem> &sort_keys,
@ -600,7 +625,8 @@ private:
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
const int64_t need_sort,
const int64_t prefix_pos,
OrderItem *hash_sortkey);
OrderItem *hash_sortkey,
const ObIArray<ObRawExpr*> *qualify_filters);
int check_is_win_func_hint_valid(const ObIArray<ObWinFunRawExpr*> &all_win_exprs,
const ObWindowDistHint *hint,
bool &is_valid);
@ -613,10 +639,11 @@ private:
bool &is_valid);
int check_win_dist_method_valid(const WinFuncOpHelper &win_func_helper,
bool &is_valid);
int generate_window_functions_plan(WinFuncOpHelper &win_func_helper,
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
ObIArray<CandidatePlan> &total_plans,
CandidatePlan &orig_candidate_plan);
int generate_window_functions_plan(WinFuncOpHelper &win_func_helper,
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
ObIArray<CandidatePlan> &total_plans,
CandidatePlan &orig_candidate_plan);
int check_win_func_need_sort(const ObLogicalOperator &top,
const WinFuncOpHelper &win_func_helper,
bool &need_sort,
@ -637,6 +664,7 @@ private:
const ObIArray<WinDistAlgo> &methods,
const int64_t splict_idx,
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
bool is_last_group,
WinFuncOpHelper &win_func_helper);
int get_next_group_window_exprs(const ObIArray<OrderItem> &op_ordering,
WinFuncOpHelper &win_func_helper,
@ -734,6 +762,7 @@ private:
*/
int create_merge_window_function_plan(ObLogicalOperator *&top,
const ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
const ObRawExpr *limit_expr,
const ObIArray<OrderItem> &sort_keys,
const ObIArray<ObRawExpr*> &partition_exprs,
WinDistAlgo dist_method,
@ -744,8 +773,13 @@ private:
int64_t prefix_pos,
int64_t part_cnt);
//init topn_filter,topn_const and is_fetch_with_ties flag
int init_wf_topn_option(WinFuncOpHelper &win_func_helper, bool wf_topn_hint);
int create_hash_window_function_plan(ObLogicalOperator *&top,
const ObIArray<ObWinFunRawExpr*> &adjusted_winfunc_exprs,
const ObIArray<ObRawExpr *> &qualify_filter_exprs,
const ObRawExpr *limit_expr,
const ObIArray<OrderItem> &sort_keys,
const ObIArray<ObRawExpr*> &partition_exprs,
const OrderItem &hash_sortkey,
@ -768,6 +802,10 @@ private:
int check_wf_pushdown_supported(ObWinFunRawExpr *win_expr, bool &supported);
int check_wf_part_topn_supported(const common::ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
const ObIArray<ObRawExpr*> &partition_exprs,
bool &can_wf_topn);
int get_pushdown_window_function_exchange_info(const ObIArray<ObWinFunRawExpr *> &win_exprs,
ObLogicalOperator *op,
ObExchangeInfo &exch_info);
@ -779,18 +817,24 @@ private:
const bool match_parallel,
const bool is_partition_wise,
const bool use_hash_sort,
const bool use_topn_sort,
const ObIArray<OrderItem> &sort_keys,
ObLogicalOperator *&top);
ObLogicalOperator *&top,
const ObIArray<ObRawExpr *> *qualify_filters,
double origin_sort_card);
int allocate_window_function_as_top(const WinDistAlgo dist_algo,
const ObIArray<ObWinFunRawExpr *> &win_exprs,
const bool match_parallel,
const bool is_partition_wise,
const bool use_hash_sort,
const bool use_topn_sort,
const int32_t role_type,
const ObIArray<OrderItem> &sort_keys,
const int64_t range_dist_keys_cnt,
const int64_t range_dist_pby_prefix,
ObLogicalOperator *&top,
const ObIArray<ObRawExpr *> *qualify_filters,
double origin_sort_card,
ObOpPseudoColumnRawExpr *wf_aggr_status_expr = NULL,
const ObIArray<bool> *pushdown_info = NULL);

View File

@ -1008,6 +1008,7 @@ Timestamp{whitespace}?\"[^\"]*\" {
<hint>NO_USE_HASH_AGGREGATION { return NO_USE_HASH_AGGREGATION; }
<hint>PARTITION_SORT { return PARTITION_SORT; }
<hint>NO_PARTITION_SORT { return NO_PARTITION_SORT; }
<hint>WF_TOPN { return WF_TOPN; }
<hint>USE_LATE_MATERIALIZATION { return USE_LATE_MATERIALIZATION; }
<hint>NO_USE_LATE_MATERIALIZATION { return NO_USE_LATE_MATERIALIZATION; }
<hint>TRACE_LOG { return TRACE_LOG; }

View File

@ -181,7 +181,7 @@ LEADING_HINT ORDERED
USE_NL USE_MERGE USE_HASH NO_USE_HASH NO_USE_MERGE NO_USE_NL
USE_NL_MATERIALIZATION NO_USE_NL_MATERIALIZATION
USE_HASH_AGGREGATION NO_USE_HASH_AGGREGATION
PARTITION_SORT NO_PARTITION_SORT
PARTITION_SORT NO_PARTITION_SORT WF_TOPN
USE_LATE_MATERIALIZATION NO_USE_LATE_MATERIALIZATION
PX_JOIN_FILTER NO_PX_JOIN_FILTER PX_PART_JOIN_FILTER NO_PX_PART_JOIN_FILTER
PQ_MAP PQ_DISTRIBUTE PQ_DISTRIBUTE_WINDOW PQ_SET RANDOM_LOCAL BROADCAST BC2HOST LIST
@ -395,7 +395,7 @@ END_P SET_VAR DELIMITER
%type <node> case_arg when_clause_list when_clause case_default
%type <node> window_function opt_partition_by generalized_window_clause win_rows_or_range win_preceding_or_following win_interval win_bounding win_window opt_win_window win_fun_lead_lag_params respect_or_ignore opt_respect_or_ignore_nulls win_fun_first_last_params first_or_last opt_from_first_or_last new_generalized_window_clause new_generalized_window_clause_with_blanket opt_named_windows named_windows named_window
%type <node> win_dist_list win_dist_desc
%type <ival> opt_hash_sort_and_pushdown
%type <ival> opt_hash_sort_and_pushdown opts_hash_sort_and_pushdown
%type <node> update_asgn_list update_asgn_factor
%type <node> update_basic_stmt delete_basic_stmt
%type <node> table_element_list table_element column_definition column_definition_ref column_definition_list column_name_list
@ -10373,26 +10373,32 @@ win_dist_desc
;
win_dist_desc:
'(' intnum_list ')' distribute_method opt_hash_sort_and_pushdown
'(' intnum_list ')' distribute_method opts_hash_sort_and_pushdown
{
ParseNode *win_func_idxs = NULL;
merge_nodes(win_func_idxs, result, T_WIN_FUNC_IDX_LIST, $2);
malloc_non_terminal_node($$, result->malloc_pool_, T_METHOD_OPT, 2, win_func_idxs, $4);
$4->value_ = $5[0];
}
| distribute_method opt_hash_sort_and_pushdown
| distribute_method opts_hash_sort_and_pushdown
{
malloc_non_terminal_node($$, result->malloc_pool_, T_METHOD_OPT, 2, NULL, $1);
$1->value_ = $2[0];
}
;
opt_hash_sort_and_pushdown:
opts_hash_sort_and_pushdown:
/*empty*/
{
$$[0] = 0;
}
| PARTITION_SORT
| opts_hash_sort_and_pushdown opt_hash_sort_and_pushdown
{
$$[0] = $1[0] | $2[0];
}
opt_hash_sort_and_pushdown:
PARTITION_SORT
{
$$[0] = 1;
}
@ -10400,13 +10406,9 @@ opt_hash_sort_and_pushdown:
{
$$[0] = 2;
}
| PARTITION_SORT PUSHDOWN
| WF_TOPN
{
$$[0] = 3;
}
| PUSHDOWN PARTITION_SORT
{
$$[0] = 3;
$$[0] = 4;
}
;

View File

@ -20,7 +20,6 @@
#include "sql/resolver/dml/ob_dml_resolver.h"
#include "sql/resolver/expr/ob_raw_expr.h"
#include "sql/resolver/expr/ob_raw_expr_util.h"
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include "sql/resolver/expr/ob_raw_expr_info_extractor.h"
#include "sql/resolver/expr/ob_raw_expr_info_extractor.h"
#include "sql/resolver/dml/ob_view_table_resolver.h"
@ -32,7 +31,6 @@
#include "sql/parser/parse_malloc.h"
#include "sql/session/ob_sql_session_info.h"
#include "share/schema/ob_table_schema.h"
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include "sql/resolver/ob_resolver_utils.h"
#include "sql/optimizer/ob_optimizer_util.h"
#include "sql/resolver/dml/ob_default_value_utils.h"
@ -6066,22 +6064,13 @@ int ObDMLResolver::resolve_and_split_sql_expr(const ParseNode &node, ObIArray<Ob
ctx.stmt_ = static_cast<ObStmt*>(get_stmt());
ctx.query_ctx_ = params_.query_ctx_;
ctx.session_info_ = params_.session_info_;
ObRawExprCanonicalizerImpl canonicalizer(ctx);
if (OB_FAIL(resolve_sql_expr(node, expr))) {
LOG_WARN("resolve sql expr failed", K(ret));
} else if ( !is_resolving_view_ && OB_FAIL(canonicalizer.canonicalize(expr))) { // canonicalize expression
LOG_WARN("resolve canonicalize expression", K(ret));
} else if (OB_FAIL(expr->formalize(session_info_))) {
LOG_WARN("failed to formalize expr", K(ret));
} else if (expr->get_expr_type() == T_OP_AND) {
// no T_OP_AND under another T_OP_AND, which is ensured by canonicalize
ObOpRawExpr *and_expr = static_cast<ObOpRawExpr *>(expr);
for (int64_t i = 0; OB_SUCC(ret) && i < and_expr->get_param_count(); i++) {
ObRawExpr *sub_expr = and_expr->get_param_expr(i);
OZ((and_exprs.push_back)(sub_expr));
}
} else {
OZ((and_exprs.push_back)(expr));
} else if (OB_FAIL(ObTransformUtils::flatten_expr(expr, and_exprs))) {
//canonicalizer move to rewrite, T_OP_AND may under T_OP_AND, so flatten expr
LOG_WARN("fail to flatten_expr", K(ret));
}
} else {
for (int i = 0; OB_SUCC(ret) && i < node.num_child_; i++) {
@ -14367,16 +14356,19 @@ int ObDMLResolver::resolve_win_dist_option(const ParseNode *option,
const ParseNode *idx_node = NULL;
const int64_t hash_sort_flag = 1;
const int64_t push_down_flag = 1 << 1;
const int64_t topn_flag = 1 << 2;
switch (dist_method->type_) {
case T_DISTRIBUTE_NONE: {
dist_option.algo_ = WinDistAlgo::WIN_DIST_NONE;
dist_option.use_hash_sort_ = dist_method->value_ & hash_sort_flag;
dist_option.use_topn_sort_ = dist_method->value_ & topn_flag;
break;
}
case T_DISTRIBUTE_HASH: {
dist_option.algo_ = WinDistAlgo::WIN_DIST_HASH;
dist_option.use_hash_sort_ = dist_method->value_ & hash_sort_flag;
dist_option.is_push_down_ = dist_method->value_ & push_down_flag;
dist_option.use_topn_sort_ = dist_method->value_ & topn_flag;
break;
}
case T_DISTRIBUTE_RANGE: dist_option.algo_ = WinDistAlgo::WIN_DIST_RANGE; break;

View File

@ -2644,7 +2644,8 @@ int ObWindowDistHint::add_win_dist_option(const ObIArray<ObWinFunRawExpr*> &all_
const ObIArray<ObWinFunRawExpr*> &cur_win_funcs,
const WinDistAlgo algo,
const bool is_push_down,
const bool use_hash_sort)
const bool use_hash_sort,
const bool use_topn_sort)
{
int ret = OB_SUCCESS;
ObSEArray<int64_t, 4> win_func_idxs;
@ -2657,7 +2658,7 @@ int ObWindowDistHint::add_win_dist_option(const ObIArray<ObWinFunRawExpr*> &all_
LOG_WARN("failed to push back", K(ret));
}
}
if (OB_SUCC(ret) && add_win_dist_option(win_func_idxs, algo, is_push_down, use_hash_sort)) {
if (OB_SUCC(ret) && add_win_dist_option(win_func_idxs, algo, is_push_down, use_hash_sort, use_topn_sort)) {
LOG_WARN("failed to add win dist option", K(ret));
}
return ret;
@ -2666,7 +2667,8 @@ int ObWindowDistHint::add_win_dist_option(const ObIArray<ObWinFunRawExpr*> &all_
int ObWindowDistHint::add_win_dist_option(const ObIArray<int64_t> &win_func_idxs,
const WinDistAlgo algo,
const bool is_push_down,
const bool use_hash_sort)
const bool use_hash_sort,
const bool use_topn_sort)
{
int ret = OB_SUCCESS;
int64_t idx = win_dist_options_.count();
@ -2677,6 +2679,7 @@ int ObWindowDistHint::add_win_dist_option(const ObIArray<int64_t> &win_func_idxs
win_dist_option.algo_ = algo;
win_dist_option.is_push_down_ = is_push_down;
win_dist_option.use_hash_sort_ = use_hash_sort;
win_dist_option.use_topn_sort_ = use_topn_sort;
if (win_dist_option.win_func_idxs_.assign(win_func_idxs)) {
LOG_WARN("failed to add win dist option", K(ret));
}
@ -2715,6 +2718,8 @@ int ObWindowDistHint::WinDistOption::print_win_dist_option(PlanText &plan_text)
LOG_WARN("failed to print win func sort", K(ret));
} else if (is_push_down_ && OB_FAIL(BUF_PRINTF(" PUSHDOWN"))) {
LOG_WARN("failed to print win func push down", K(ret));
} else if (use_topn_sort_ && OB_FAIL(BUF_PRINTF(" WF_TOPN"))) {
LOG_WARN("failed to print win func sort", K(ret));
}
return ret;
}
@ -2726,7 +2731,8 @@ bool ObWindowDistHint::WinDistOption::is_valid() const
bret = false;
} else if (WinDistAlgo::WIN_DIST_HASH != algo_ && is_push_down_) {
bret = false;
} else if (WinDistAlgo::WIN_DIST_HASH != algo_ && WinDistAlgo::WIN_DIST_NONE != algo_ && use_hash_sort_) {
} else if (WinDistAlgo::WIN_DIST_HASH != algo_ && WinDistAlgo::WIN_DIST_NONE != algo_
&& (use_hash_sort_ || use_topn_sort_)) {
bret = false;
} else {
for (int64_t i = 0; bret && i < win_func_idxs_.count(); ++i) {
@ -2741,6 +2747,7 @@ int ObWindowDistHint::WinDistOption::assign(const WinDistOption& other)
int ret = OB_SUCCESS;
algo_ = other.algo_;
use_hash_sort_ = other.use_hash_sort_;
use_topn_sort_ = other.use_topn_sort_;
is_push_down_ = other.is_push_down_;
if (OB_FAIL(win_func_idxs_.assign(other.win_func_idxs_))) {
LOG_WARN("failed to assign", K(ret));
@ -2752,6 +2759,7 @@ void ObWindowDistHint::WinDistOption::reset()
{
algo_ = WinDistAlgo::WIN_DIST_INVALID;
use_hash_sort_ = false;
use_topn_sort_ = false;
is_push_down_ = false;
win_func_idxs_.reuse();
}

View File

@ -960,7 +960,8 @@ public:
common::ObSEArray<int64_t, 2, common::ModulePageAllocator, true> win_func_idxs_;
bool use_hash_sort_; // use hash sort for none/hash dist method
bool is_push_down_; // push down window function for hash dist method
TO_STRING_KV(K_(algo), K_(win_func_idxs), K_(use_hash_sort), K_(is_push_down));
bool use_topn_sort_; // use part topn sort for none/hash dist method
TO_STRING_KV(K_(algo), K_(win_func_idxs), K_(use_hash_sort), K_(is_push_down), K_(use_topn_sort));
};
const ObIArray<WinDistOption> &get_win_dist_options() const { return win_dist_options_; }
@ -969,11 +970,13 @@ public:
const ObIArray<ObWinFunRawExpr*> &cur_win_funcs,
const WinDistAlgo algo,
const bool is_push_down,
const bool use_hash_sort);
const bool use_hash_sort,
const bool use_topn_sort);
int add_win_dist_option(const ObIArray<int64_t> &win_func_idxs,
const WinDistAlgo algo,
const bool is_push_down,
const bool use_hash_sort);
const bool use_hash_sort,
const bool use_topn_sort);
static const char* get_dist_algo_str(WinDistAlgo dist_algo);
virtual int print_hint_desc(PlanText &plan_text) const override;

View File

@ -22,7 +22,6 @@
#include "share/inner_table/ob_inner_table_schema.h"
#include "sql/ob_sql_utils.h"
#include "sql/resolver/expr/ob_raw_expr_info_extractor.h"
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include "sql/resolver/dml/ob_aggr_expr_push_up_analyzer.h"
#include "sql/resolver/dml/ob_group_by_checker.h"
#include "sql/resolver/expr/ob_raw_expr.h"

View File

@ -119,6 +119,15 @@ int ObSelectStmt::add_window_func_expr(ObWinFunRawExpr *expr)
return ret;
}
int ObSelectStmt::set_qualify_filters(common::ObIArray<ObRawExpr *> &exprs)
{
int ret = OB_SUCCESS;
if (OB_FAIL(qualify_filters_.assign(exprs))) {
LOG_WARN("failed to add expr", K(ret));
}
return ret;
}
int ObSelectStmt::remove_window_func_expr(ObWinFunRawExpr *expr)
{
int ret = OB_SUCCESS;
@ -179,6 +188,8 @@ int ObSelectStmt::assign(const ObSelectStmt &other)
LOG_WARN("assign other start with failed", K(ret));
} else if (OB_FAIL(win_func_exprs_.assign(other.win_func_exprs_))) {
LOG_WARN("assign window function exprs failed", K(ret));
} else if (OB_FAIL(qualify_filters_.assign(other.qualify_filters_))) {
LOG_WARN("assign window function filter exprs failed", K(ret));
} else if (OB_FAIL(connect_by_exprs_.assign(other.connect_by_exprs_))) {
LOG_WARN("assign other connect by failed", K(ret));
} else if (OB_FAIL(connect_by_prior_exprs_.assign(other.connect_by_prior_exprs_))) {
@ -247,6 +258,8 @@ int ObSelectStmt::deep_copy_stmt_struct(ObIAllocator &allocator,
LOG_WARN("deep copy agg item failed", K(ret));
} else if (OB_FAIL(expr_copier.copy(other.win_func_exprs_, win_func_exprs_))) {
LOG_WARN("deep copy window function expr failed", K(ret));
} else if (OB_FAIL(expr_copier.copy(other.qualify_filters_, qualify_filters_))) {
LOG_WARN("deep copy window function expr failed", K(ret));
} else if (OB_FAIL(expr_copier.copy(other.start_with_exprs_, start_with_exprs_))) {
LOG_WARN("deep copy start with exprs failed", K(ret));
} else if (OB_FAIL(expr_copier.copy(other.connect_by_exprs_, connect_by_exprs_))) {
@ -440,6 +453,11 @@ int ObSelectStmt::iterate_stmt_expr(ObStmtExprVisitor &visitor)
} else { /* do nothing*/ }
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(visitor.visit(qualify_filters_, SCOPE_QUALIFY_FILTER))) {
LOG_WARN("failed to visit winfunc exprs", K(ret));
}
}
if (OB_SUCC(ret) && NULL != into_item_) {
for (int64_t i = 0; OB_SUCC(ret) && i < into_item_->pl_vars_.count(); ++i) {
if (OB_FAIL(visitor.visit(into_item_->pl_vars_.at(i), SCOPE_SELECT_INTO))) {

View File

@ -573,6 +573,11 @@ public:
ObWinFunRawExpr* get_window_func_expr(int64_t i) { return win_func_exprs_.at(i); }
int64_t get_window_func_count() const { return win_func_exprs_.count(); }
int remove_window_func_expr(ObWinFunRawExpr *expr);
const common::ObIArray<ObRawExpr *> &get_qualify_filters() const { return qualify_filters_; };
common::ObIArray<ObRawExpr *> &get_qualify_filters() { return qualify_filters_; };
int64_t get_qualify_filters_count() const { return qualify_filters_.count(); };
bool has_window_function_filter() const { return qualify_filters_.count() != 0; }
int set_qualify_filters(common::ObIArray<ObRawExpr *> &exprs);
void set_children_swapped() { children_swapped_ = true; }
bool get_children_swapped() const { return children_swapped_; }
const ObString* get_select_alias(const char *col_name, uint64_t table_id, uint64_t col_id);
@ -643,6 +648,8 @@ private:
common::ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> having_exprs_;
common::ObSEArray<ObAggFunRawExpr*, 8, common::ModulePageAllocator, true> agg_items_;
common::ObSEArray<ObWinFunRawExpr*, 8, common::ModulePageAllocator, true> win_func_exprs_;
//a child set of the filters in the parent stmts, only used for partition topn sort
common::ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> qualify_filters_;
common::ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> start_with_exprs_;
common::ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> connect_by_exprs_;
common::ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> connect_by_prior_exprs_;

View File

@ -45,7 +45,8 @@ enum DmlStmtScope {
SCOPE_INSERT_DESC,
SCOPE_INSERT_VECTOR,
SCOPE_RETURNING,
SCOPE_DICT_FIELDS // 16
SCOPE_DICT_FIELDS,
SCOPE_QUALIFY_FILTER
};
class ObStmtExprVisitor

View File

@ -660,19 +660,24 @@ bool ObRawExpr::same_as(const ObRawExpr &expr,
ObExprEqualCheckContext *check_context) const
{
bool bret = false;
int ret = OB_SUCCESS;
if (this == &expr) {
bret = true;
} else {
if (NULL != check_context) {
check_context->recursion_level_ += 1;
}
const ObRawExpr *l = get_same_identify(this, check_context);
const ObRawExpr *r = get_same_identify(&expr, check_context);
bret = l->inner_same_as(*r, check_context);
ret = SMART_CALL(bret = l->inner_same_as(*r, check_context));
if (NULL != check_context) {
check_context->recursion_level_ -= 1;
if (OB_SIZE_OVERFLOW == ret) {
bret = false;
check_context->error_code_ = ret;
LOG_WARN("check smart call fail", K(ret));
} else {
check_context->recursion_level_ -= 1;
}
}
if (bret) {

View File

@ -1372,7 +1372,8 @@ struct ObExprEqualCheckContext
param_expr_(),
need_check_deterministic_(false),
ignore_param_(false),
ora_numeric_compare_(false)
ora_numeric_compare_(false),
error_code_(0)
{ }
ObExprEqualCheckContext(bool need_check_deterministic)
: override_const_compare_(false),
@ -1385,7 +1386,8 @@ struct ObExprEqualCheckContext
param_expr_(),
need_check_deterministic_(need_check_deterministic),
ignore_param_(false),
ora_numeric_compare_(false)
ora_numeric_compare_(false),
error_code_(0)
{ }
virtual ~ObExprEqualCheckContext() {}
struct ParamExprPair
@ -1435,6 +1437,7 @@ struct ObExprEqualCheckContext
param_expr_.reset();
need_check_deterministic_ = false;
ignore_param_ = false;
error_code_ = 0;
}
bool override_const_compare_;
bool override_column_compare_;
@ -1448,6 +1451,7 @@ struct ObExprEqualCheckContext
bool need_check_deterministic_;
bool ignore_param_; // only compare structure of expr
bool ora_numeric_compare_;
int64_t error_code_; //error code to return
};
struct ObExprParamCheckContext : ObExprEqualCheckContext

View File

@ -1,47 +0,0 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef _OB_RAW_EXPR_CANONICALIZER_H
#define _OB_RAW_EXPR_CANONICALIZER_H 1
#include "sql/resolver/expr/ob_raw_expr.h"
namespace oceanbase
{
namespace sql
{
// 表达式规范化
class ObRawExprCanonicalizer
{
public:
ObRawExprCanonicalizer() {}
virtual ~ObRawExprCanonicalizer() {}
/**
* Canonicalize the expression.
*
* @param expr [in/out]
*
* @return error code
*/
virtual int canonicalize(ObRawExpr *&expr) = 0;
private:
// types and constants
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRawExprCanonicalizer);
// function members
private:
// data members
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_RAW_EXPR_CANONICALIZER_H */

View File

@ -1,569 +0,0 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_RESV
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include "lib/json/ob_json_print_utils.h"
#include "sql/resolver/expr/ob_raw_expr_util.h"
#include "sql/resolver/expr/ob_raw_expr_resolver_impl.h"
#include "sql/rewrite/ob_stmt_comparer.h"
#include "sql/ob_sql_context.h"
#include "common/ob_smart_call.h"
namespace oceanbase
{
using namespace common;
namespace sql
{
ObRawExprCanonicalizerImpl::ObRawExprCanonicalizerImpl(ObExprResolveContext &ctx)
: ctx_(ctx)
{}
int ObRawExprCanonicalizerImpl::canonicalize(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (OB_FAIL(push_not(expr))) {
LOG_WARN("push not expr failed", K(ret));
} else if (OB_FAIL(cluster_and_or(expr))) {
LOG_WARN("cluster and or failed", K(ret));
} else if (OB_FAIL(pull_similar_expr(expr))) {
LOG_WARN("pull similar expr failed", K(ret));
} else if (OB_FAIL(expr->extract_info())) {
LOG_WARN("failed to extract info", K(ret), K(*expr));
}
return ret;
}
/**
* @brief ObRawExprCanonicalizerImpl::cluster_and_or
* and(and(a,b),c) => and (a,b,c)
* or(or(a,b),c) => or(a,b,c)
* @param expr
* @return
*/
int ObRawExprCanonicalizerImpl::cluster_and_or(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null", K(ret), K(expr));
}
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
if (OB_FAIL(SMART_CALL(cluster_and_or(expr->get_param_expr(i))))) {
LOG_WARN("failed to cluster and or", K(ret));
}
}
if (OB_SUCC(ret) && (expr->get_expr_type() == T_OP_AND ||
expr->get_expr_type() == T_OP_OR)) {
ObSEArray<ObRawExpr *, 4> new_param_exprs;
const ObItemType expr_type = expr->get_expr_type();
bool is_valid = false;
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
ObRawExpr *param = NULL;
if (OB_ISNULL(param = expr->get_param_expr(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(ret));
} else if (param->get_expr_type() == expr_type) {
is_valid = true;
if (OB_FAIL(append(new_param_exprs,
static_cast<ObOpRawExpr *>(param)->get_param_exprs()))) {
LOG_WARN("failed to append param exprs", K(ret));
}
} else if (OB_FAIL(new_param_exprs.push_back(param))) {
LOG_WARN("failed to push back param", K(ret));
}
}
if (OB_SUCC(ret) && is_valid) {
ObOpRawExpr *op_expr = static_cast<ObOpRawExpr *>(expr);
if (OB_FAIL(op_expr->get_param_exprs().assign(new_param_exprs))) {
LOG_WARN("failed to assign new param exprs", K(ret));
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::pull_similar_expr(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
if (OB_FAIL(SMART_CALL(pull_similar_expr(expr->get_param_expr(i))))) {
LOG_WARN("failed to pull similar expr", K(ret));
}
}
if (OB_SUCC(ret) && (expr->get_expr_type() == T_OP_OR ||
expr->get_expr_type() == T_OP_AND)) {
// A OR A => A, A and A => A
if (OB_FAIL(remove_duplicate_conds(expr))) {
LOG_WARN("failed to remove duplicate condition", K(ret));
} else if (expr->get_expr_type() == T_OP_OR) {
if (OB_FAIL(pull_and_factor(expr))) {
LOG_WARN("failed to pull and factor", K(ret));
}
} else if (expr->get_expr_type() == T_OP_AND) {
if (OB_FAIL(pull_parallel_expr(expr))) {
LOG_WARN("failed to pull parallel expr", K(ret));
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::remove_duplicate_conds(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op expr is null", K(ret), K(expr));
} else if (expr->has_flag(CNT_OUTER_JOIN_SYMBOL)) {
// do nothing
} else if (expr->get_expr_type() == T_OP_OR ||
expr->get_expr_type() == T_OP_AND) {
ObOpRawExpr *op_expr = static_cast<ObOpRawExpr *>(expr);
ObSEArray<ObRawExpr *, 4> param_conds;
for (int64_t i = 0; OB_SUCC(ret) && i < op_expr->get_param_count(); ++i) {
if (OB_FAIL(add_var_to_array_no_dup(param_conds, op_expr->get_param_expr(i)))) {
LOG_WARN("failed to append array no dup", K(ret));
}
}
if (OB_SUCC(ret)) {
if (param_conds.count() == op_expr->get_param_count()) {
// do nothing
} else if (param_conds.count() == 1) {
expr = param_conds.at(0);
} else if (OB_FAIL(op_expr->get_param_exprs().assign(param_conds))) {
LOG_WARN("failed to assign param conditions", K(ret));
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::pull_and_factor(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (OB_LIKELY(NULL != expr) && T_OP_OR == expr->get_expr_type()) {
ObRawExpr *short_expr = NULL;
int64_t factor_num = INT64_MAX;
// continue processing when one 'and'(ObOpRawExpr) exist at least
bool do_handle = false;
bool expr_copied = false;
ObOpRawExpr *m_expr = static_cast<ObOpRawExpr *>(expr);
for (int64_t i = 0; OB_SUCC(ret) && i < m_expr->get_param_count(); ++i) {
const ObRawExpr *and_expr = m_expr->get_param_expr(i);
// at least one 'and' expr
if (OB_ISNULL(and_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("and_expr is null");
} else {
if (and_expr->get_param_count() > 1) {
do_handle = true;
}
if (T_OP_AND != and_expr->get_expr_type()) {
short_expr = m_expr->get_param_expr(i);
factor_num = 1;
} else if (and_expr->get_param_count() < factor_num) {
short_expr = m_expr->get_param_expr(i);
factor_num = and_expr->get_param_count();
}
// find 'and' expr and short shortest expr
// do not need to go on
if (do_handle && 1 == factor_num) {
break;
}
}
}
if (OB_SUCC(ret) && do_handle) {
bool is_match = false;
ObQueryCtx *query_ctx = NULL;
ObStmtCompareContext stmt_compare_context;
ObArray<ObRawExpr *> candidate_factors;
if (OB_ISNULL(short_expr) ||
OB_ISNULL(ctx_.stmt_) ||
OB_ISNULL(query_ctx = ctx_.stmt_->get_query_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(short_expr),
K(ctx_.stmt_), K(query_ctx));
} else if (T_OP_AND == short_expr->get_expr_type()) {
for (int64_t i = 0; OB_SUCC(ret) && i < short_expr->get_param_count(); ++i) {
if (OB_FAIL(candidate_factors.push_back(short_expr->get_param_expr(i)))) {
LOG_WARN("construct candidate factors failed", K(ret));
}
}
} else if (OB_FAIL(candidate_factors.push_back(short_expr))) {
LOG_WARN("cons candidate factors failed", K(ret));
}
if (OB_SUCC(ret)) {
ObOpRawExpr *new_and = NULL;
stmt_compare_context.init(&query_ctx->calculable_items_);
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(T_OP_AND, new_and))) {
LOG_WARN("alloc ObOpRawExpr failed", K(ret));
} else if (OB_ISNULL(new_and)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("new_and is null");
} else {
new_and->set_data_type(ObTinyIntType);
}
bool need_or = true;
ObSEArray<int64_t, 4> idxs;
ObSEArray<ObPCParamEqualInfo, 8> equal_params;
for (int64_t i = 0; OB_SUCC(ret) && need_or && i < candidate_factors.count(); ++i) {
idxs.reset();
equal_params.reset();
if (OB_ISNULL(candidate_factors.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("candidate factor param expr is null", K(i));
}
for (int64_t j = 0; OB_SUCC(ret) && j < m_expr->get_param_count(); ++j) {
ObOpRawExpr *and_expr = NULL;
// 1. ignore NULL 'or expr'
if (OB_ISNULL(m_expr->get_param_expr(j))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("m_expr param is null", K(j), K(*m_expr));
}
// 2. seem whole none-'or expr' as integrity
// E.g. A or (A and B) ==> A
else if (T_OP_AND != m_expr->get_param_expr(j)->get_expr_type()) {
if (OB_FAIL(ObStmtComparer::is_same_condition(candidate_factors.at(i),
m_expr->get_param_expr(j),
stmt_compare_context,
is_match))) {
LOG_WARN("failed to check is condition equal", K(ret));
} else if (!is_match) {
/*do nothing*/
} else if (m_expr->has_flag(CNT_OUTER_JOIN_SYMBOL)) {
/* don't eliminate or expr in (+), to keep the error information compatible with oracle
* e,g,. select * from t1,t2 where t1.c1(+) = t2.c1 or t1.c1(+) = t2.c1 should raise ORA-1719
*/
} else if (OB_FAIL(append(equal_params,
stmt_compare_context.equal_param_info_))) {
LOG_WARN("failed to append expr", K(ret));
} else if (OB_FAIL(idxs.push_back(-1 /*and factor index*/))) {
LOG_WARN("failed to push back pos", K(ret));
} else { /*do nothing*/ }
}
// 3. find factor in and list
else { // and_expr != NULL
and_expr = static_cast<ObOpRawExpr *>(m_expr->get_param_expr(j));
for (int64_t k = 0; OB_SUCC(ret) && k < and_expr->get_param_count(); ++k) {
if (OB_FAIL(ObStmtComparer::is_same_condition(candidate_factors.at(i),
and_expr->get_param_expr(k),
stmt_compare_context,
is_match))) {
LOG_WARN("failed to check is condition equal", K(ret));
} else if (!is_match) {
/*do nothing*/
} else if (OB_FAIL(append(equal_params,
stmt_compare_context.equal_param_info_))) {
LOG_WARN("failed to append expr", K(ret));
} else if (OB_FAIL(idxs.push_back(k))) {
LOG_WARN("failed to push back pos", K(ret));
} else {
break;
}
}
}
}
/* is and factor */
if (OB_SUCC(ret) && idxs.count() == m_expr->get_param_count()) {
// 1. add and factor to new and expr
if (OB_FAIL(new_and->add_param_expr(candidate_factors.at(i)))) {
LOG_WARN("failed to add param expr", K(ret));
}
// 2. remove from or
if (OB_SUCC(ret) && !expr_copied) {
// deep copy T_OP_OR and T_OP_AND to make expr unshared
ObRawExprCopier expr_copier(ctx_.expr_factory_);
ObOpRawExpr *new_or_expr = NULL;
ObArray<ObRawExpr *> new_candidate_factors;
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(T_OP_OR, new_or_expr))) {
LOG_WARN("alloc ObOpRawExpr failed", K(ret));
} else if (OB_ISNULL(new_or_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("new_or_expr is null");
}
for (int64_t j = 0; OB_SUCC(ret) && j < m_expr->get_param_count(); ++j) {
ObRawExpr *new_or_param = m_expr->get_param_expr(j);
if (T_OP_AND == new_or_param->get_expr_type() &&
OB_FAIL(expr_copier.copy_expr_node(new_or_param, new_or_param))) {
LOG_WARN("failed to copy expr node", K(ret));
} else if (OB_FAIL(new_or_expr->add_param_expr(new_or_param))) {
LOG_WARN("failed to add param expr", K(ret));
}
}
if (OB_SUCC(ret)) {
m_expr = new_or_expr;
expr = new_or_expr;
expr_copied = true;
}
}
for (int64_t j = 0; OB_SUCC(ret) && j < idxs.count(); ++j) {
ObOpRawExpr *and_expr = NULL;
if (OB_ISNULL(m_expr->get_param_expr(j))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("m_expr is null");
} else if (idxs.at(j) < 0) {
// whole or expr is and factor
// no need to consider OR condition any more
// E.g. A or (A and B) ==> A
need_or = false;
} else if (T_OP_AND != m_expr->get_param_expr(j)->get_expr_type()) {
// only 1 factor, but not and factor. idxs.count() == m_expr->get_param_count() can not be satisfied
ret = OB_ERR_UNEXPECTED;
} else {
// must be and factor in and list
and_expr = static_cast<ObOpRawExpr *>(m_expr->get_param_expr(j));
if (OB_FAIL(and_expr->remove_param_expr(idxs.at(j)))) {
LOG_WARN("remove param expr failed", K(ret));
} else if (1 == and_expr->get_param_count()) {
m_expr->get_param_expr(j) = and_expr->get_param_expr(0);
}
}
}
//这里会导致m_expr的树形结构发生变化,所以and or关系也有可能发生变化,所以需要做一次相邻 and or的提升
if (OB_SUCC(ret)) {
if (OB_FAIL(append(query_ctx->all_equal_param_constraints_, equal_params))) {
LOG_WARN("failed to append expr", K(ret));
} else {
ret = cluster_and_or(expr);
LOG_DEBUG("succeed to extract common expression", K(*expr), K(equal_params));
}
}
}
}
// and factor is found
if (OB_SUCC(ret) && new_and->get_param_count() > 0) {
if (need_or) {
if (OB_FAIL(new_and->add_param_expr(m_expr))) {
LOG_WARN("failed to add param expr", K(ret));
} else {
expr = new_and;
}
} else if (1 == new_and->get_param_count() && !expr->has_flag(CNT_OUTER_JOIN_SYMBOL)) {
ObRawExpr *const_false = NULL;
ObSEArray<ObRawExpr*,2> new_exprs;
if (OB_FAIL(ObRawExprUtils::build_const_bool_expr(&ctx_.expr_factory_, const_false, false))) {
LOG_WARN("failed to build const expr", K(ret));
} else if (OB_FAIL(new_exprs.push_back(const_false))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (OB_FAIL(new_exprs.push_back(new_and->get_param_expr(0)))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (OB_FAIL(ObRawExprUtils::build_or_exprs(ctx_.expr_factory_, new_exprs, expr))) {
LOG_WARN("failed to build or expr", K(ret));
}
} else {
expr = new_and;
}
}
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::pull_parallel_expr(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (expr && (expr->get_expr_type() == T_OP_AND || expr->get_expr_type() == T_OP_OR)) {
bool has_sub = false;
ObRawExpr::ExprClass expr_class = expr->get_expr_class();
if (ObRawExpr::EXPR_OPERATOR != expr_class){
LOG_WARN("Unexpected expression type", K(expr_class));
} else {
ObOpRawExpr *parent_expr = static_cast<ObOpRawExpr *>(expr);
for (int64_t i = 0; OB_SUCC(ret) && i < parent_expr->get_param_count(); ++i) {
ObRawExpr *sub_expr = parent_expr->get_param_expr(i);
if (sub_expr->get_expr_type() == parent_expr->get_expr_type()) {
if (OB_FAIL(SMART_CALL(pull_parallel_expr(sub_expr)))) {
LOG_WARN("Cluster AND or OR expression failed");
} else {
has_sub = true;
}
}
}
if (OB_SUCC(ret) && has_sub) {
ObOpRawExpr* tmp = nullptr;
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(parent_expr->get_expr_type(), tmp))) {
LOG_WARN("failed to create raw expr", K(ret));
} else if (OB_ISNULL(tmp)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get null expr", K(ret));
} else if (OB_FAIL(tmp->assign(*parent_expr))) {
LOG_WARN("failed to assign expr", K(ret));
} else {
parent_expr->clear_child();
for (int64_t i = 0; OB_SUCC(ret) && i < tmp->get_param_count(); ++i) {
ObRawExpr *sub_expr = tmp->get_param_expr(i);
if (OB_ISNULL(sub_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sub_expr is null", K(i));
} else if (sub_expr->get_expr_type() == parent_expr->get_expr_type()) {
ObRawExpr::ExprClass sub_expr_class = sub_expr->get_expr_class();
if (ObRawExpr::EXPR_OPERATOR != sub_expr_class) {
LOG_WARN("Unexpect Expression class", K(sub_expr_class));
} else {
ObOpRawExpr *child_expr = static_cast<ObOpRawExpr *>(sub_expr);
for (int64_t j = 0; OB_SUCC(ret) && j < child_expr->get_param_count(); j++) {
if (OB_FAIL(parent_expr->add_param_expr(child_expr->get_param_expr(j)))) {
LOG_WARN("Pull AND or OR expression failed", K(ret));
}
}
}
} else {
if (OB_FAIL(parent_expr->add_param_expr(sub_expr))) {
LOG_WARN("Pull AND or OR expression failed");
}
}
}
}
if (OB_SUCC(ret) && OB_NOT_NULL(tmp)) {
tmp->reset();
}
}
if (OB_SUCC(ret)) {
if (expr->get_expr_type() == T_OP_AND && expr->get_param_count() == 1) {
expr = expr->get_param_expr(0);
}
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::push_not(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
if (expr != NULL && expr->has_flag(CNT_NOT)) {
if (expr->has_flag(IS_NOT)) {
if (OB_FAIL(do_push_not(expr))) {
LOG_WARN("failed to do push not expr", K(ret));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
if (OB_FAIL(SMART_CALL(push_not(expr->get_param_expr(i))))) {
LOG_WARN("failed to push not expr", K(ret));
}
}
}
return ret;
}
int ObRawExprCanonicalizerImpl::do_push_not(ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
ObRawExpr *child = NULL;
ObItemType opp_type = T_MIN_OP;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null", K(ret));
} else if (!expr->is_op_expr() || expr->get_expr_type() != T_OP_NOT) {
// do nothing
} else if (OB_ISNULL(child = expr->get_param_expr(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child expr is null", K(ret));
} else if (!child->is_op_expr()) {
// do nothing
} else if (child->get_expr_type() == T_OP_NOT) {
ObRawExpr *param = NULL;
if (OB_ISNULL(param = child->get_param_expr(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param is null", K(ret));
} else if (!IS_BOOL_OP(param->get_expr_type())) {
// do nothing
} else if (OB_FAIL(push_not(param))) {
LOG_WARN("failed to push not", K(ret));
} else {
expr = param;
}
} else if (child->get_expr_type() == T_OP_OR ||
child->get_expr_type() == T_OP_AND) {
ObItemType new_type = (child->get_expr_type() == T_OP_OR) ? T_OP_AND : T_OP_OR;
ObOpRawExpr *new_expr = NULL;
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(new_type, new_expr))) {
LOG_WARN("failed to create raw expr", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < child->get_param_count(); ++i) {
ObOpRawExpr *not_expr = NULL;
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(T_OP_NOT, not_expr))) {
LOG_WARN("failed to create not expr", K(ret));
} else if (not_expr->add_param_expr(child->get_param_expr(i))) {
LOG_WARN("failed to add param expr", K(ret));
} else if (OB_FAIL(new_expr->add_param_expr(not_expr))) {
LOG_WARN("failed to add param expr", K(ret));
} else if (OB_FAIL(not_expr->add_flag(IS_NOT))) {
LOG_WARN("failed to add not flag", K(ret));
}
}
if (OB_SUCC(ret)) {
expr = new_expr;
}
} else if ((opp_type = get_opposite_op(child->get_expr_type())) != T_MIN_OP) {
ObItemType new_type = opp_type;
ObOpRawExpr *new_expr = NULL;
if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(new_type, new_expr))) {
LOG_WARN("failed to create raw expr", K(ret));
} else if (OB_FAIL(new_expr->get_param_exprs().assign(
static_cast<ObOpRawExpr*>(child)->get_param_exprs()))) {
LOG_WARN("failed to assign param exprs", K(ret));
} else {
expr = new_expr;
}
}
return ret;
}
ObRawExprCanonicalizerImpl::ObOppositeOpPair ObRawExprCanonicalizerImpl::OPPOSITE_PAIRS[] =
{
// T_MIN_OP means no opposite operator
{T_OP_EQ, T_OP_NE},
{T_OP_LE, T_OP_GT},
{T_OP_LT, T_OP_GE},
{T_OP_GE, T_OP_LT},
{T_OP_GT, T_OP_LE},
{T_OP_NE, T_OP_EQ},
{T_OP_IS, T_OP_IS_NOT},
{T_OP_IS_NOT, T_OP_IS},
{T_OP_BTW, T_OP_NOT_BTW},
{T_OP_NOT_BTW, T_OP_BTW},
{T_OP_NOT, T_MIN_OP},
{T_OP_AND, T_MIN_OP},
{T_OP_OR, T_MIN_OP},
{T_OP_IN, T_OP_NOT_IN},
{T_OP_NOT_IN, T_OP_IN},
{T_OP_NSEQ, T_MIN_OP},
};
ObItemType ObRawExprCanonicalizerImpl::get_opposite_op(ObItemType type)
{
ObItemType ret = T_MIN_OP;
int32_t low = 0;
int32_t mid = 0;
int32_t high = ARRAYSIZEOF(OPPOSITE_PAIRS) - 1;
while (low <= high) {
mid = low + (high - low) / 2;
if (OPPOSITE_PAIRS[mid].original_ == type) {
ret = OPPOSITE_PAIRS[mid].opposite_;
break;
} else if (OPPOSITE_PAIRS[mid].original_ > type) {
high = mid - 1;
} else {
low = mid + 1;
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -1,58 +0,0 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef _OB_RAW_EXPR_CANONICALIZER_IMPL_H
#define _OB_RAW_EXPR_CANONICALIZER_IMPL_H
#include "sql/resolver/expr/ob_raw_expr_canonicalizer.h"
namespace oceanbase
{
namespace sql
{
class ObRawExprCanonicalizerImpl : public ObRawExprCanonicalizer
{
public:
explicit ObRawExprCanonicalizerImpl(ObExprResolveContext &ctx);
virtual ~ObRawExprCanonicalizerImpl() {}
virtual int canonicalize(ObRawExpr *&expr);
private:
// types and constants
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRawExprCanonicalizerImpl);
// function members
int push_not(ObRawExpr *&expr);
int do_push_not(ObRawExpr *&expr);
int remove_duplicate_conds(ObRawExpr *&expr);
int pull_parallel_expr(ObRawExpr *&expr);
int pull_and_factor(ObRawExpr *&expr);
int pull_similar_expr(ObRawExpr *&expr);
int cluster_and_or(ObRawExpr *&expr);
private:
struct ObOppositeOpPair
{
ObItemType original_;
ObItemType opposite_;
};
static ObItemType get_opposite_op(ObItemType type);
static ObOppositeOpPair OPPOSITE_PAIRS[];
private:
// data members
ObExprResolveContext &ctx_;
// ObIAllocator &allocator_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_RAW_EXPR_CANONICALIZER_IMPL_H */

View File

@ -5720,6 +5720,37 @@ int ObRawExprUtils::build_nvl_expr(ObRawExprFactory &expr_factory, const ColumnI
return ret;
}
int ObRawExprUtils::build_nvl_expr(ObRawExprFactory &expr_factory,
ObRawExpr *param_expr1,
ObRawExpr *param_expr2,
ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
ObSysFunRawExpr *nvl_func_expr = NULL;
if (OB_ISNULL(param_expr1) || OB_ISNULL(param_expr2)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("fail to build length expr", K(param_expr1), K(param_expr2), K(ret));
} else if (OB_FAIL(expr_factory.create_raw_expr(T_FUN_SYS_NVL, nvl_func_expr))) {
LOG_WARN("fail to create raw expr", K(ret));
} else if (OB_ISNULL(nvl_func_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("func expr is null", K(nvl_func_expr), K(ret));
} else {
nvl_func_expr->set_expr_type(T_FUN_SYS_NVL);
nvl_func_expr->set_func_name(ObString::make_string(N_NVL));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(nvl_func_expr->add_param_expr(param_expr1))) {
LOG_WARN("fail to add param expr", K(ret));
} else if (OB_FAIL(nvl_func_expr->add_param_expr(param_expr2))) {
LOG_WARN("fail to add param expr", K(ret));
} else {
expr = nvl_func_expr;
}
}
return ret;
}
int ObRawExprUtils::build_lnnvl_expr(ObRawExprFactory &expr_factory,
ObRawExpr *param_expr,
ObRawExpr *&expr)

View File

@ -735,6 +735,7 @@ public:
static int build_null_expr(ObRawExprFactory &expr_factory, ObRawExpr *&expr);
static int build_nvl_expr(ObRawExprFactory &expr_factory, const ColumnItem *column_item, ObRawExpr *&expr);
static int build_nvl_expr(ObRawExprFactory &expr_factory, const ColumnItem *column_item, ObRawExpr *&expr1, ObRawExpr *&expr2);
static int build_nvl_expr(ObRawExprFactory &expr_factory, ObRawExpr *param_expr1, ObRawExpr *param_expr2, ObRawExpr *&expr);
static int build_lnnvl_expr(ObRawExprFactory &expr_factory,
ObRawExpr *param_expr,
ObRawExpr *&lnnvl_expr);

View File

@ -39,6 +39,7 @@ void ObStmtMapInfo::reset()
select_item_map_.reset();
is_select_item_equal_ = false;
is_distinct_equal_ = false;
is_qualify_filter_equal_ = false;
equal_param_map_.reset();
view_select_item_map_.reset();
}
@ -74,6 +75,7 @@ int ObStmtMapInfo::assign(const ObStmtMapInfo& other)
is_order_equal_ = other.is_order_equal_;
is_select_item_equal_ = other.is_select_item_equal_;
is_distinct_equal_ = other.is_distinct_equal_;
is_qualify_filter_equal_ = other.is_qualify_filter_equal_;
}
return ret;
}
@ -590,6 +592,21 @@ int ObStmtComparer::check_stmt_containment(const ObDMLStmt *first,
}
}
// check qualify filters
if (OB_SUCC(ret) && QueryRelation::QUERY_UNCOMPARABLE != relation) {
first_count = first_sel->get_qualify_filters_count();
second_count = second_sel->get_qualify_filters_count();
if (0 == first_count && 0 == second_count) {
map_info.is_qualify_filter_equal_ = true;
} else if (!ObOptimizerUtil::same_exprs(first_sel->get_qualify_filters(),
second_sel->get_qualify_filters())) {
relation = QueryRelation::QUERY_UNCOMPARABLE;
LOG_TRACE("succeed to check qualify filters", K(relation), K(map_info));
} else {
map_info.is_qualify_filter_equal_ = true;
}
}
// check distinct exprs
if (OB_SUCC(ret) && QueryRelation::QUERY_UNCOMPARABLE != relation) {
if ((!first_sel->has_distinct() && !second_sel->has_distinct()) ||

View File

@ -53,6 +53,7 @@ enum QueryRelation
bool is_order_equal_;
bool is_select_item_equal_;
bool is_distinct_equal_;
bool is_qualify_filter_equal_;
//如果from item是generated table,需要记录ref query的select item map关系
//如果是set stmt,每个set query对应的映射关系也记录在view_select_item_map_
@ -65,7 +66,8 @@ enum QueryRelation
is_having_equal_(false),
is_order_equal_(false),
is_select_item_equal_(false),
is_distinct_equal_(false)
is_distinct_equal_(false),
is_qualify_filter_equal_(false)
{}
void reset();

View File

@ -69,6 +69,15 @@ int ObTransformPreProcess::transform_one_stmt(common::ObIArray<ObParentDMLStmt>
OB_FAIL(formalize_limit_expr(*stmt))) {
LOG_WARN("formalize stmt fialed", K(ret));
} else {
if (OB_SUCC(ret)) {
if (OB_FAIL(flatten_conditions(stmt, is_happened))) {
LOG_WARN("failed to flatten_condition", K(ret));
} else {
trans_happened |= is_happened;
OPT_TRACE("flatten condition:", is_happened);
LOG_TRACE("succeed to flatten_condition", K(is_happened));
}
}
if (OB_SUCC(ret) && parent_stmts.empty()) {
if (OB_FAIL(expand_correlated_cte(stmt, is_happened))) {
LOG_WARN("failed to expand correlated cte", K(ret));
@ -10125,6 +10134,122 @@ int ObTransformPreProcess::check_is_correlated_cte(ObSelectStmt *stmt, ObIArray<
return ret;
}
int ObTransformPreProcess::flatten_conditions(ObDMLStmt *stmt, bool &trans_happened)
{
int ret = OB_SUCCESS;
bool flatten_where = false;
bool flatten_having = false;
bool flatten_semi_info = false;
bool flatten_join = false;
bool flatten_start_with = false;
bool flatten_match_condition = false;
if (OB_ISNULL(stmt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpect null pointer error", K(stmt), K(ret));
} else {
if (OB_FAIL(do_flatten_conditions(stmt, stmt->get_condition_exprs(), flatten_where))) {
LOG_WARN("flatten_where_condition_expr failed", K(ret));
} else {
//simplify having expr
if (!stmt->is_select_stmt()) {
//do nothing
} else if (OB_FAIL(do_flatten_conditions(stmt, static_cast<ObSelectStmt*>(stmt)->get_having_exprs(), flatten_having))) {
LOG_WARN("flatten_having_condition_expr failed", K(ret));
} else if (stmt->is_hierarchical_query()) {
if (OB_FAIL(do_flatten_conditions(stmt, static_cast<ObSelectStmt*>(stmt)->get_start_with_exprs(), flatten_start_with))) {
LOG_WARN("flatten_start_with failed", K(ret));
}
}
if (OB_SUCC(ret) && stmt->is_merge_stmt()) {
if (OB_FAIL(do_flatten_conditions(stmt, static_cast<ObMergeStmt*>(stmt)->get_match_condition_exprs(), flatten_match_condition))) {
LOG_WARN("flatten_match_condition failed", K(ret));
}
}
if (OB_FAIL(ret)) {
//do nothing
} else if (stmt->is_insert_stmt()) {
//do nothing
} else {
//simplify semi info
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_semi_infos().count(); ++i) {
if (OB_ISNULL(stmt->get_semi_infos().at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpect null semi info", K(stmt->get_semi_infos().at(i)), K(ret));
} else if (OB_FAIL(do_flatten_conditions(stmt, stmt->get_semi_infos().at(i)->semi_conditions_, flatten_semi_info))) {
LOG_WARN("flatten_semi_info failed", K(ret));
}
}
//simplify join condition expr
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_joined_tables().count(); i++) {
if (OB_ISNULL(stmt->get_joined_tables().at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpect null joined table item", K(stmt->get_joined_tables().at(i)), K(ret));
} else if (OB_FAIL(recursive_flatten_join_conditions(stmt, stmt->get_joined_tables().at(i), flatten_join))) {
LOG_WARN("flatten_join_condition_expr failed", K(ret));
}
}
}
}
if (OB_SUCC(ret)) {
trans_happened = flatten_where | flatten_having | flatten_start_with |
flatten_match_condition | flatten_semi_info| flatten_join;
}
}
return ret;
}
int ObTransformPreProcess::recursive_flatten_join_conditions(ObDMLStmt *stmt, TableItem *table, bool &trans_happened)
{
int ret = OB_SUCCESS;
JoinedTable *join_table = NULL;
trans_happened = false;
bool cur_happened = false;
bool left_happened = false;
bool right_happened = false;
if (OB_ISNULL(stmt) || OB_ISNULL(table)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null pointer", K(stmt), K(table), K(ret));
} else if (!table->is_joined_table()) {
/*do nothing*/
} else if (OB_ISNULL(join_table = static_cast<JoinedTable*>(table)) ||
OB_ISNULL(join_table->left_table_) || OB_ISNULL(join_table->right_table_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(join_table), K(join_table->left_table_), K(join_table));
} else if (OB_FAIL(do_flatten_conditions(stmt, join_table->join_conditions_, cur_happened))) {
LOG_WARN("failed to flatten join conditions", K(ret));
} else if (OB_FAIL(SMART_CALL(recursive_flatten_join_conditions(stmt,
join_table->left_table_,
left_happened)))) {
LOG_WARN("failed to flatten left child join condition exprs", K(ret));
} else if (OB_FAIL(SMART_CALL(recursive_flatten_join_conditions(stmt,
join_table->right_table_,
right_happened)))) {
LOG_WARN("failed to flatten right child join condition exprs", K(ret));
} else {
trans_happened = cur_happened | left_happened | right_happened;
}
return ret;
}
int ObTransformPreProcess::do_flatten_conditions(ObDMLStmt *stmt, ObIArray<ObRawExpr*> &conditions, bool &trans_happened)
{
int ret = OB_SUCCESS;
trans_happened = false;
bool flatten_happend = false;
if (OB_ISNULL(stmt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpect null pointer error", K(stmt), K(ret));
} else if (conditions.count() == 0) {
//do nothing
} else if (OB_FAIL(ObTransformUtils::flatten_and_or_xor(ctx_, conditions, &flatten_happend))) {
LOG_WARN("flatten_and_or_xor failed", K(ret));
} else if (trans_happened) {
trans_happened = flatten_happend ;
OPT_TRACE(" flatten_happend:", flatten_happend);
}
return ret;
}
int ObTransformPreProcess::check_can_transform_insert_only_merge_into(const ObMergeStmt *merge_stmt,
bool &is_valid)
{

View File

@ -636,6 +636,10 @@ struct DistinctObjMeta
int check_exec_param_correlated(const ObRawExpr *expr, bool &is_correlated);
int check_is_correlated_cte(ObSelectStmt *stmt, ObIArray<ObSelectStmt *> &visited_cte, bool &is_correlated);
int convert_join_preds_vector_to_scalar(JoinedTable &joined_table, bool &trans_happened);
int flatten_conditions(ObDMLStmt *stmt, bool &trans_happened);
int recursive_flatten_join_conditions(ObDMLStmt *stmt, TableItem *table, bool &trans_happened);
int do_flatten_conditions(ObDMLStmt *stmt, ObIArray<ObRawExpr*> &conditions, bool &trans_happened);
private:
DISALLOW_COPY_AND_ASSIGN(ObTransformPreProcess);
};

View File

@ -899,6 +899,8 @@ int ObTransformPredicateMoveAround::generate_pullup_predicates(
LOG_WARN("failed to append conditions", K(ret));
} else if (OB_FAIL(append(filter_preds, select_stmt.get_having_exprs()))) {
LOG_WARN("failed to append having conditions", K(ret));
} else if (OB_FAIL(gather_basic_qualify_filter(select_stmt, filter_preds))) {
LOG_WARN("failed to gather qualify filters", K(ret));
} else if (OB_FAIL(remove_simple_op_null_condition(select_stmt, filter_preds))) {
LOG_WARN("fail to chck and remove const simple conditions", K(ret));
} else if (OB_FAIL(append(local_preds, input_pullup_preds))) {
@ -912,6 +914,23 @@ int ObTransformPredicateMoveAround::generate_pullup_predicates(
return ret;
}
int ObTransformPredicateMoveAround::gather_basic_qualify_filter(ObSelectStmt &stmt,
ObIArray<ObRawExpr*> &preds)
{
int ret = OB_SUCCESS;
ObIArray<ObRawExpr *> &qualify_filters = stmt.get_qualify_filters();
for (int64_t i = 0; OB_SUCC(ret) && i < stmt.get_qualify_filters_count(); ++i) {
ObRawExpr *expr = qualify_filters.at(i);
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (!expr->has_flag(CNT_WINDOW_FUNC) && OB_FAIL(preds.push_back(expr))) {
LOG_WARN("push back pred failed", K(ret));
}
}
return ret;
}
int ObTransformPredicateMoveAround::gather_pullup_preds_from_semi_outer_join(ObDMLStmt &stmt,
ObIArray<ObRawExpr*> &preds,
bool remove_preds /* default false*/)
@ -1260,11 +1279,13 @@ int ObTransformPredicateMoveAround::pushdown_predicates(
} else {
const uint64_t pushdown_pred_count = pushdown_preds.count();
bool is_happened = false;
bool has_distinct = false;
if (OB_FAIL(stmt->has_rownum(has_rownum))) {
LOG_WARN("failed to check stmt has rownum", K(ret));
} else if (stmt->is_select_stmt()) {
has_group = sel_stmt->has_group_by();
has_winfunc = sel_stmt->has_window_function();
has_distinct = sel_stmt->has_distinct();
}
if (OB_SUCC(ret)) {
if (stmt->has_limit() || stmt->has_sequence() ||
@ -1276,7 +1297,10 @@ int ObTransformPredicateMoveAround::pushdown_predicates(
// but with rownum, it is impossible
OPT_TRACE("stmt has rownum, can not pushdown into where");
} else if (has_winfunc) {
if (OB_FAIL(pushdown_through_winfunc(*sel_stmt, pushdown_preds, candi_preds))) {
if (!has_distinct && !sel_stmt->is_dblink_stmt()
&& OB_FAIL(pushdown_into_qualify_filter(pushdown_preds, *sel_stmt, is_happened))) {
LOG_WARN("extract winfunc topn exprs failed", K(ret));
} else if (OB_FAIL(pushdown_through_winfunc(*sel_stmt, pushdown_preds, candi_preds))) {
LOG_WARN("failed to push down predicates throught winfunc", K(ret));
}
} else if (OB_FAIL(candi_preds.assign(pushdown_preds))) {
@ -2124,6 +2148,57 @@ int ObTransformPredicateMoveAround::pushdown_through_winfunc(
return ret;
}
//extract topn filters for ranking window functions and pushdown into qualify_filters_
int ObTransformPredicateMoveAround::pushdown_into_qualify_filter(ObIArray<ObRawExpr *> &predicates, ObSelectStmt &sel_stmt, bool &is_happened)
{
int ret = OB_SUCCESS;
ObSEArray<ObRawExpr *, 4> qualify_filters;
ObSEArray<ObRawExpr *, 4> remain_exprs;
if (OB_ISNULL(ctx_) && OB_ISNULL(ctx_->session_info_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_3_0_0) {
//do nothing
} else if (!ctx_->session_info_->is_qualify_filter_enabled()) {
//do nothing
} else if (OB_FAIL(qualify_filters.assign(sel_stmt.get_qualify_filters()))) {
LOG_WARN("assign window function filter expr failed", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < predicates.count(); ++i) {
ObRawExpr *pred = NULL;
bool is_topn_pred = false;
ObRawExpr *dummy_expr = NULL;
ObWinFunRawExpr *dummy_win_expr = NULL;
bool dummy_flag;
if (OB_ISNULL(pred = predicates.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("predicate is null", K(ret));
} else if (OB_FAIL(ObTransformUtils::is_winfunc_topn_filter(sel_stmt.get_window_func_exprs(), pred,
is_topn_pred, dummy_expr, dummy_flag, dummy_win_expr))) {
LOG_WARN("check whether the filter is a winfunc topn filter failed", K(ret));
} else if (is_topn_pred) {
if (ObPredicateDeduce::find_equal_expr(qualify_filters, pred)) {
// the condition has been ensured
} else if (OB_FAIL(qualify_filters.push_back(pred))) {
LOG_WARN("push back topn filter failed", K(ret));
} else {
is_happened = true;
}
} else if (OB_FAIL(remain_exprs.push_back(pred))) {
LOG_WARN("push back filter failed", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(sel_stmt.set_qualify_filters(qualify_filters))) {
LOG_WARN("assign topn filters failed", K(ret));
} else if (OB_FAIL(predicates.assign(remain_exprs))) {
LOG_WARN("assign remain filters failed", K(ret));
}
}
}
return ret;
}
/**
* @brief ObTransformPredicateMoveAround::check_pushdown_validity
* group by
@ -2186,7 +2261,7 @@ int ObTransformPredicateMoveAround::pushdown_through_groupby(
pushed = true;
}
if (OB_SUCC(ret) && !pushed) {
if (T_OP_OR == pred->get_expr_type()) {
if (T_OP_OR == pred->get_expr_type() && !ObOptimizerUtil::find_equal_expr(ctx_->push_down_filters_, pred)) {
//对于having c1 > 1 or (c1 < 0 and count(*) > 1)
//可以拆分出c1 > 1 or c1 < 0下推过group by
ObRawExpr *new_pred = NULL;
@ -2206,6 +2281,12 @@ int ObTransformPredicateMoveAround::pushdown_through_groupby(
LOG_WARN("failed to push back new having expr", K(ret));
}
}
//no matter new_pred is valid, pred not need to generate new_pred and try push again
if (OB_FAIL(ret)) {
//do nothing
} else if (OB_FAIL(ctx_->push_down_filters_.push_back(pred))) {
LOG_WARN("failed to append table filters", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(stmt.get_having_exprs().assign(new_having_exprs))) {

View File

@ -276,6 +276,9 @@ private:
int pushdown_through_winfunc(ObSelectStmt &stmt,
ObIArray<ObRawExpr *> &predicates,
ObIArray<ObRawExpr *> &down_preds);
int pushdown_into_qualify_filter(ObIArray<ObRawExpr *> &predicates,
ObSelectStmt &sel_stmt,
bool &is_happened);
int pushdown_through_groupby(ObSelectStmt &stmt,
ObIArray<ObRawExpr *> &output_predicates);
@ -374,6 +377,8 @@ private:
int append_condition_array(ObIArray<ObRawExprCondition *> &conditions, int count, ObRawExprCondition *value);
int gather_basic_qualify_filter(ObSelectStmt &stmt, ObIArray<ObRawExpr*> &preds);
private:
typedef ObSEArray<ObRawExpr *, 4> PullupPreds;
ObArenaAllocator allocator_;

View File

@ -870,6 +870,9 @@ int ObTransformQueryPushDown::push_down_stmt_exprs(ObSelectStmt *select_stmt,
} else if (OB_FAIL(append(view_stmt->get_window_func_exprs(),
select_stmt->get_window_func_exprs()))) {
LOG_WARN("append select_stmt window func exprs to view stmt window func exprs failed", K(ret));
} else if (OB_FAIL(append(view_stmt->get_qualify_filters(),
select_stmt->get_qualify_filters()))) {
LOG_WARN("append select_stmt window func filters to view stmt window func filters failed", K(ret));
} else {
if (!view_stmt->is_from_pivot()) {
view_stmt->set_from_pivot(select_stmt->is_from_pivot());

View File

@ -66,6 +66,7 @@ void ObTransformerCtx::reset()
used_trans_hints_.reset();
groupby_pushdown_stmts_.reset();
is_spm_outline_ = false;
push_down_filters_.reset();
}
int ObTransformerCtx::add_src_hash_val(const ObString &src_str)

View File

@ -66,7 +66,8 @@ struct ObTransformerCtx
outline_trans_hints_(),
used_trans_hints_(),
groupby_pushdown_stmts_(),
is_spm_outline_(false)
is_spm_outline_(false),
push_down_filters_()
{ }
virtual ~ObTransformerCtx() {}
@ -126,6 +127,7 @@ struct ObTransformerCtx
ObSEArray<uint64_t, 4> groupby_pushdown_stmts_;
/* end used for hint and outline below */
bool is_spm_outline_;
ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> push_down_filters_;
};
enum TransMethod

File diff suppressed because it is too large Load Diff

View File

@ -164,8 +164,66 @@ private:
ObIArray<ObRawExpr *> &new_exprs);
int check_remove_ora_decode_valid(ObRawExpr *&expr, int64_t &result_idx, bool &is_valid);
int try_remove_ora_decode(ObRawExpr *&expr, ObRawExpr *&new_expr);
};
int canonicalize_conditions(ObDMLStmt *stmt, bool &trans_happened);
int recursive_canonicalize_join_conditions(ObDMLStmt *stmt, TableItem *table, bool &trans_happened);
int do_canonicalize(ObDMLStmt *stmt, ObIArray<ObRawExpr*> &conditions, bool &trans_happened);
int push_not(ObIArray<ObRawExpr*> &conditions,
bool &trans_happened);
int recursive_push_not(ObRawExpr *&expr,
hash::ObHashMap<uint64_t, ObRawExpr *> &push_expr_map,
bool &trans_happened);
int do_push_not(ObRawExpr *&expr,
hash::ObHashMap<uint64_t, ObRawExpr *> &push_expr_map,
bool &trans_happened);
int reverse_cmp_type_of_align_date4cmp(ObRawExpr* expr,
ObRawExprFactory *expr_factory,
const ObItemType cmp_type,
bool is_left);
int get_opposite_op(ObItemType type, ObItemType& opposit_type);
int remove_duplicate_exprs(ObIArray<ObRawExpr*> &conditions, bool &trans_happened);
int do_remove(ObRawExpr* &expr, bool &trans_happened);
int pull_similar_expr(ObDMLStmt *stmt, ObIArray<ObRawExpr*> &conditions, bool &trans_happened);
int recursive_pull_similar(ObDMLStmt *stmt, ObRawExpr* &expr, bool &trans_happened);
int do_pull_similar(ObDMLStmt *stmt,
ObRawExpr* &expr,
ObIArray<ObRawExpr*> &exprs,
ObItemType expr_type,
bool &trans_happened);
int gen_not_intersect_param(ObRawExpr* &expr,
ObIArray<ObSEArray<ObRawExpr *, 4>> &params_sets,
ObItemType expr_type);
int get_intersection(ObDMLStmt *stmt,
ObIArray<ObSEArray<ObRawExpr *, 4>> &params_sets,
ObIArray<ObRawExpr *> &intersection);
int remove_intersect_item(ObIArray<ObSEArray<ObRawExpr *, 4>> &params_sets, ObIArray<ObRawExpr *> &intersection);
int get_params_array(ObIArray<ObRawExpr*> &exprs,
ObRawExpr *&expr,
ObItemType parent_type);
int convert_case_when_predicate(ObDMLStmt *stmt, bool &trans_happened);
int try_convert_case_when_by_then_exprs(ObRawExpr *&expr,
ObIArray<ObRawExpr *> &extracted_preds,
bool &trans_happened);
int check_convert_then_exprs_validity(ObRawExpr *parent_expr,
ObCaseOpRawExpr *&case_expr,
bool &is_all_false,
bool &is_all_true,
int64_t &reserved_branch_cnt,
ObIArray<ObRawExpr*> &true_exprs,
ObIArray<ObRawExpr*> &false_null_exprs,
ObIArray<ObRawExpr*> &extracted_preds,
bool &is_valid);
int get_preds_for_convert_case_when_by_then(ObCaseOpRawExpr *case_expr,
bool is_all_true,
bool is_all_false,
int64_t reserved_branch_cnt,
ObIArray<ObRawExpr*> &extracted_preds);
int add_constraint_for_convert_case_when_by_then(ObIArray<ObRawExpr *> &false_null_exprs,
ObIArray<ObRawExpr *> &true_exprs);
int build_nvl_bool_exprs(ObIArray<ObRawExpr *> &exprs, const bool boolean);
int build_nvl_bool_expr(ObRawExpr *expr, const bool boolean, ObRawExpr *&nvl_expr);
};
}
}

View File

@ -632,6 +632,12 @@ int ObTransformSimplifyWinfunc::do_remove_stmt_win(ObSelectStmt *select_stmt,
if (OB_SUCC(ret) && OB_FAIL(select_stmt->replace_relation_exprs(exprs, new_exprs))) {
LOG_WARN("select_stmt replace inner stmt expr failed", K(ret), K(select_stmt));
}
if (OB_SUCC(ret)) {
//check qualify filters
if (OB_FAIL(ObTransformUtils::pushdown_qualify_filters(select_stmt))) {
LOG_WARN("check pushdown qualify filters failed", K(ret));
}
}
}
return ret;
}

View File

@ -1914,7 +1914,8 @@ bool ObTransformSubqueryCoalesce::check_subquery_can_coalesce(const ObStmtMapInf
map_info.is_semi_info_equal_ &&
map_info.is_cond_equal_ &&
map_info.is_group_equal_ &&
map_info.is_having_equal_;
map_info.is_having_equal_ &&
map_info.is_qualify_filter_equal_;
}
int ObTransformSubqueryCoalesce::coalesce_subquery(StmtCompareHelper &helper,

View File

@ -1715,6 +1715,109 @@ int ObTransformUtils::flatten_expr(ObRawExpr *expr,
return ret;
}
int ObTransformUtils::flatten_and_or_xor(ObTransformerCtx *ctx,
ObIArray<ObRawExpr*> &conditions,
bool *trans_happened)
{
int ret = OB_SUCCESS;
ObRawExprFactory *factory = NULL;
ObSEArray<ObRawExpr *, 4> new_param_exprs;
bool happend = false;
if (OB_ISNULL(ctx) || OB_ISNULL(factory = ctx->expr_factory_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpect null pointer error", K(ctx), K(factory), K(ret));
} else if (NULL != trans_happened) {
*trans_happened = false;
}
for (int64_t i = 0; OB_SUCC(ret) && i < conditions.count(); ++i) {
if (OB_ISNULL(conditions.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(conditions.at(i)), K(ret));
} else if (OB_FAIL(flatten_and_or_xor(conditions.at(i), trans_happened))) {
LOG_WARN("failed to do_flatten", K(ret));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < conditions.count(); ++i) {
ObRawExpr *param = NULL;
if (OB_ISNULL(param = conditions.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(conditions.at(i)), K(ret));
} else if (param->get_expr_type() == T_OP_AND) {
happend = true;
for (int64_t j = 0; OB_SUCC(ret) && j < param->get_param_count(); ++j) {
ObRawExpr* temp = NULL;
if (OB_ISNULL(param->get_param_expr(j))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(param->get_param_expr(j)), K(ret));
} else if (OB_FAIL(ObRawExprUtils::try_create_bool_expr(param->get_param_expr(j), temp, *factory))) {
LOG_WARN("try create bool expr failed", K(ret));
} else if (OB_FAIL(new_param_exprs.push_back(temp))) {
LOG_WARN("failed to append param exprs", K(ret));
}
}
} else if (OB_FAIL(new_param_exprs.push_back(param))) {
LOG_WARN("failed to push back param", K(ret));
}
}
if (OB_SUCC(ret) && happend) {
if (OB_FAIL(conditions.assign(new_param_exprs))) {
LOG_WARN("failed to assign new param exprs", K(ret));
} else if (NULL != trans_happened) {
*trans_happened = true;
}
}
return ret;
}
int ObTransformUtils::flatten_and_or_xor(ObRawExpr* expr, bool *trans_happened)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null", K(ret), K(expr));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
if (OB_ISNULL(expr->get_param_expr(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(expr->get_param_expr(i)), K(ret));
} else if (OB_FAIL(SMART_CALL(flatten_and_or_xor(expr->get_param_expr(i), trans_happened)))) {
LOG_WARN("failed to cluster and or", K(ret));
}
}
if (OB_SUCC(ret) && (expr->get_expr_type() == T_OP_AND ||
expr->get_expr_type() == T_OP_OR ||
expr->get_expr_type() == T_OP_XOR)) {
ObSEArray<ObRawExpr *, 4> new_param_exprs;
const ObItemType expr_type = expr->get_expr_type();
bool happend = false;
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
ObRawExpr *param = NULL;
if (OB_ISNULL(param = expr->get_param_expr(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(expr->get_param_expr(i)), K(ret));
} else if (param->get_expr_type() == expr_type) {
happend = true;
if (OB_FAIL(append(new_param_exprs,
static_cast<ObOpRawExpr *>(param)->get_param_exprs()))) {
LOG_WARN("failed to append param exprs", K(ret));
}
} else if (OB_FAIL(new_param_exprs.push_back(param))) {
LOG_WARN("failed to push back param", K(ret));
}
}
if (OB_SUCC(ret) && happend) {
ObOpRawExpr *op_expr = static_cast<ObOpRawExpr *>(expr);
if (OB_FAIL(op_expr->get_param_exprs().assign(new_param_exprs))) {
LOG_WARN("failed to assign new param exprs", K(ret));
} else if (NULL != trans_happened) {
*trans_happened = true;
}
}
}
}
return ret;
}
int ObTransformUtils::find_not_null_expr(const ObDMLStmt &stmt,
ObRawExpr *&not_null_expr,
bool &is_valid,
@ -7426,6 +7529,33 @@ int ObTransformUtils::create_inline_view(ObTransformerCtx *ctx,
view_table->for_update_ = true;
}
}
if (OB_SUCC(ret) && stmt->is_select_stmt()) {
//check qualify filters
if (OB_FAIL(pushdown_qualify_filters(static_cast<ObSelectStmt *>(stmt)))) {
LOG_WARN("check pushdown qualify filters failed", K(ret));
}
}
return ret;
}
int ObTransformUtils::pushdown_qualify_filters(ObSelectStmt *stmt)
{
int ret = OB_SUCCESS;
bool need_pushdown = true;
//if there are no winfunc, move qualify filters to having/where
if (stmt->get_qualify_filters_count() > 0 && 0 == stmt->get_window_func_count()) {
if (!stmt->has_group_by()) {
if (OB_FAIL(append(stmt->get_condition_exprs(), stmt->get_qualify_filters()))) {
LOG_WARN("failed to append condition exprs", K(ret));
} else {
stmt->get_qualify_filters().reset();
}
} else if (OB_FAIL(append(stmt->get_having_exprs(), stmt->get_qualify_filters()))) {
LOG_WARN("move qualify filters to having failed", K(ret));
} else {
stmt->get_qualify_filters().reset();
}
}
return ret;
}
@ -8204,6 +8334,87 @@ int ObTransformUtils::build_case_when_expr(ObDMLStmt &stmt,
return ret;
}
int ObTransformUtils::build_case_when_expr(ObTransformerCtx *ctx,
ObIArray<ObRawExpr*> &when_exprs,
ObIArray<ObRawExpr*> &then_exprs,
ObRawExpr *default_expr,
ObCaseOpRawExpr *&case_expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ctx) || OB_ISNULL(ctx->expr_factory_) || OB_ISNULL(ctx->session_info_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr is null", K(ret), K(ctx));
} else if (when_exprs.count() != then_exprs.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param expr count missmatch", K(ret), K(when_exprs.count()), K(then_exprs.count()));
} else if (OB_FAIL(ctx->expr_factory_->create_raw_expr(T_OP_CASE, case_expr))) {
LOG_WARN("failed to create case expr", K(ret));
} else if (OB_ISNULL(case_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null expr", K(ret));
} else if (OB_FALSE_IT(case_expr->set_default_param_expr(default_expr))) {
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < when_exprs.count(); i++) {
if (OB_FAIL(case_expr->add_when_param_expr(when_exprs.at(i)))) {
LOG_WARN("failed to add when expr", K(ret));
} else if (OB_FAIL(case_expr->add_then_param_expr(then_exprs.at(i)))) {
LOG_WARN("failed to add then expr", K(ret));
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(case_expr->formalize(ctx->session_info_))) {
LOG_WARN("failed to formalize case expr", K(ret));
} else if (OB_FAIL(case_expr->pull_relation_id())) {
LOG_WARN("failed to pull relation id and levels", K(ret));
}
return ret;
}
int ObTransformUtils::check_error_free_expr(ObRawExpr *expr, bool &is_error_free)
{
int ret = OB_SUCCESS;
is_error_free = true;
bool temp_flag = true;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null expr", K(ret));
} else if (expr->has_flag(CNT_SO_UDF) || expr->has_flag(CNT_PL_UDF) || expr->has_flag(CNT_SUB_QUERY)
|| expr->has_flag(CNT_VALUES) || expr->has_flag(CNT_ONETIME) || expr->has_flag(CNT_STATE_FUNC)
|| expr->has_flag(CNT_ENUM_OR_SET) || expr->has_flag(CNT_SEQ_EXPR) || expr->has_flag(CNT_VOLATILE_CONST)
|| expr->has_flag(CNT_DYNAMIC_USER_VARIABLE) || expr->has_flag(CNT_CUR_TIME)) {
// disable by flag
is_error_free = false;
} else if (expr->get_expr_type() == T_OP_DIV || expr->get_expr_type() == T_OP_AGG_DIV
|| expr->get_expr_type() == T_OP_INT_DIV) {
// disable by type
is_error_free = false;
} else if (expr->get_expr_type() == T_FUN_SYS_CAST) {
// case-by-case determination
if (OB_FAIL(ObOptimizerUtil::is_lossless_column_cast(expr, temp_flag))) {
LOG_WARN("failed to check lossless cast", K(ret));
} else {
is_error_free &= temp_flag;
}
} else if (expr->is_column_ref_expr()) {
ObColumnRefRawExpr *col = static_cast<ObColumnRefRawExpr *>(expr);
if (OB_NOT_NULL(col) && col->is_generated_column()) {
is_error_free = false;
}
}
// check child exprs recursively
for (int64_t i = 0; OB_SUCC(ret) && is_error_free && i < expr->get_param_count(); i++) {
if (OB_FAIL(SMART_CALL(check_error_free_expr(expr->get_param_expr(i), temp_flag)))) {
LOG_WARN("failed to check error free expr", K(ret));
} else {
is_error_free &= temp_flag;
}
}
return ret;
}
int ObTransformUtils::compare_const_expr_result(ObTransformerCtx *ctx,
ObRawExpr *expr,
ObItemType op_type,
@ -14382,5 +14593,75 @@ int ObTransformUtils::check_child_projection_validity(const ObSelectStmt *child_
return ret;
}
//check whether the filter can be used as a partition topn filter for winfunc_exprs
int ObTransformUtils::is_winfunc_topn_filter(const ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
ObRawExpr *filter,
bool &is_topn_filter,
ObRawExpr * &topn_const_expr,
bool &is_fetch_with_ties,
ObWinFunRawExpr *&win_expr)
{
int ret = OB_SUCCESS;
is_topn_filter = false;
topn_const_expr = NULL;
is_fetch_with_ties = false;
if (OB_ISNULL(filter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (T_OP_EQ > filter->get_expr_type() || T_OP_GT < filter->get_expr_type()) {
//do nothing
} else if (OB_ISNULL(filter->get_param_expr(0)) || OB_ISNULL(filter->get_param_expr(1))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("param is null", K(ret), KP(filter->get_param_expr(0)), KP(filter->get_param_expr(0)));
} else if (filter->get_param_expr(0)->is_static_const_expr()
&& filter->get_param_expr(1)->is_win_func_expr()) {
if (T_OP_EQ == filter->get_expr_type()
|| T_OP_NSEQ == filter->get_expr_type()
|| T_OP_GE == filter->get_expr_type()
|| T_OP_GT == filter->get_expr_type()) {
is_topn_filter = true;
win_expr = static_cast<ObWinFunRawExpr *>(filter->get_param_expr(1));
topn_const_expr = filter->get_param_expr(0);
}
} else if (filter->get_param_expr(1)->is_static_const_expr()
&& filter->get_param_expr(0)->is_win_func_expr()) {
if (T_OP_EQ == filter->get_expr_type()
|| T_OP_NSEQ == filter->get_expr_type()
|| T_OP_LE == filter->get_expr_type()
|| T_OP_LT == filter->get_expr_type()) {
is_topn_filter = true;
win_expr = static_cast<ObWinFunRawExpr *>(filter->get_param_expr(0));
topn_const_expr = filter->get_param_expr(1);
}
}
if (OB_SUCC(ret) && is_topn_filter) {
is_topn_filter = false;
ObItemType func_type = win_expr->get_func_type();
if (!ObOptimizerUtil::find_item(winfunc_exprs, win_expr)) {
// do nothing
} else if (BOUND_UNBOUNDED != win_expr->get_upper().type_
|| BOUND_UNBOUNDED != win_expr->get_lower().type_) {
// do nothing
} else {
switch (win_expr->get_func_type()) {
case T_WIN_FUN_RANK: {
is_topn_filter = true;
is_fetch_with_ties = true;
break;
}
case T_WIN_FUN_ROW_NUMBER: {
is_topn_filter = true;
is_fetch_with_ties = false;
break;
}
default: {
break;
}
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -431,7 +431,8 @@ public:
static int flatten_expr(ObRawExpr *expr,
common::ObIArray<ObRawExpr*> &flattened_exprs);
static int flatten_and_or_xor(ObTransformerCtx *ctx, ObIArray<ObRawExpr*> &conditions, bool *trans_happened = NULL);
static int flatten_and_or_xor(ObRawExpr* expr, bool *trans_happened = NULL);
static int find_not_null_expr(const ObDMLStmt &stmt,
ObRawExpr *&not_null_expr,
bool &is_valid,
@ -1255,6 +1256,18 @@ public:
ObRawExpr *default_expr,
ObRawExpr *&out_expr,
ObTransformerCtx *ctx);
static int build_case_when_expr(ObTransformerCtx *ctx,
ObIArray<ObRawExpr*> &when_exprs,
ObIArray<ObRawExpr*> &then_exprs,
ObRawExpr *default_expr,
ObCaseOpRawExpr *&case_expr);
/**
* @brief check_error_free_expr
* Judging whether an expression has a high risk of reporting errors during execution.
* @note The rules are mainly based on historical experience, results are not guaranteed to be accurate.
* Please use with care.
*/
static int check_error_free_expr(ObRawExpr *expr, bool &is_error_free);
static int build_row_expr(ObRawExprFactory& expr_factory,
common::ObIArray<ObRawExpr*>& param_exprs,
ObOpRawExpr*& row_expr);
@ -1871,6 +1884,14 @@ public:
static int check_child_projection_validity(const ObSelectStmt *child_stmt,
ObRawExpr *expr,
bool &is_valid);
static int is_winfunc_topn_filter(const ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
ObRawExpr *filter,
bool &is_topn_filter,
ObRawExpr * &topn_const_expr,
bool &is_fetch_with_ties,
ObWinFunRawExpr *&win_expr);
static int pushdown_qualify_filters(ObSelectStmt *stmt);
private:
static int inner_get_lazy_left_join(ObDMLStmt *stmt,
TableItem *table,

View File

@ -513,6 +513,17 @@ bool ObSQLSessionInfo::is_index_skip_scan_enabled() const
return bret;
}
bool ObSQLSessionInfo::is_qualify_filter_enabled() const
{
bool bret = false;
int64_t tenant_id = get_effective_tenant_id();
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
if (tenant_config.is_valid()) {
bret = tenant_config->_enable_optimizer_qualify_filter;
}
return bret;
}
int ObSQLSessionInfo::is_enable_range_extraction_for_not_in(bool &enabled) const
{
int ret = OB_SUCCESS;

View File

@ -1168,6 +1168,7 @@ public:
bool is_in_range_optimization_enabled() const;
int is_better_inlist_enabled(bool &enabled) const;
bool is_index_skip_scan_enabled() const;
bool is_qualify_filter_enabled() const;
int is_enable_range_extraction_for_not_in(bool &enabled) const;
bool is_var_assign_use_das_enabled() const;
int is_adj_index_cost_enabled(bool &enabled, int64_t &stats_cost_percent) const;

View File

@ -287,6 +287,7 @@ _enable_hash_join_processor
_enable_in_range_optimization
_enable_newsort
_enable_new_sql_nio
_enable_optimizer_qualify_filter
_enable_oracle_priv_check
_enable_parallel_minor_merge
_enable_parallel_redo_logging

View File

@ -2,7 +2,6 @@ sql_unittest(test_raw_expr_print_visitor expr/test_raw_expr_print_visitor.cpp)
sql_unittest(test_raw_expr_to_str expr/test_raw_expr_to_str.cpp)
sql_unittest(test_raw_expr_hash expr/test_raw_expr_hash.cpp)
sql_unittest(test_raw_expr_resolver expr/test_raw_expr_resolver.cpp)
sql_unittest(test_raw_expr_canonicalizer expr/test_raw_expr_canonicalizer.cpp)
sql_unittest(test_sql_bitset)
sql_unittest(ddl_resolver)
#sql_unittest(test_resolver)

View File

@ -1,127 +0,0 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "sql/test_sql_utils.h"
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include <gtest/gtest.h>
#include "lib/utility/ob_test_util.h"
#include "sql/resolver/expr/ob_raw_expr_util.h"
#include "sql/resolver/ob_stmt.h"
#include "sql/ob_sql_context.h"
#include "lib/json/ob_json_print_utils.h"
#define private public
#include "observer/ob_server.h"
#undef private
#include <fstream>
#include <iterator>
using namespace oceanbase::common;
using namespace oceanbase::sql;
class TestRawExprCanonicalizer: public ::testing::Test
{
public:
TestRawExprCanonicalizer();
virtual ~TestRawExprCanonicalizer();
virtual void SetUp();
virtual void TearDown();
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(TestRawExprCanonicalizer);
protected:
// function members
void canon(const char* expr, const char *&canon_expr);
protected:
// data members
};
TestRawExprCanonicalizer::TestRawExprCanonicalizer()
{
}
TestRawExprCanonicalizer::~TestRawExprCanonicalizer()
{
}
void TestRawExprCanonicalizer::SetUp()
{
}
void TestRawExprCanonicalizer::TearDown()
{
}
void TestRawExprCanonicalizer::canon(const char* expr, const char *&canon_expr)
{
ObArray<ObQualifiedName> columns;
ObArray<ObVarInfo> sys_vars;
ObArray<ObSubQueryInfo> sub_query_info;
ObArray<ObAggFunRawExpr*> aggr_exprs;
ObArray<ObWinFunRawExpr*> win_exprs;
ObArray<ObUDFInfo> udf_info;
const char *expr_str = expr;
ObArenaAllocator allocator(ObModIds::TEST);
ObRawExprFactory expr_factory(allocator);
ObTimeZoneInfo tz_info;
ObStmt stmt;
ObQueryCtx query_ctx;
ObNameCaseMode case_mode = OB_NAME_CASE_INVALID;
ObExprResolveContext ctx(expr_factory, &tz_info, case_mode);
stmt.query_ctx_ = &query_ctx;
ctx.connection_charset_ = ObCharset::get_default_charset();
ctx.dest_collation_ = ObCharset::get_default_collation(ctx.connection_charset_);
ctx.stmt_ = &stmt;
ObSQLSessionInfo session;
ctx.session_info_ = &session;
OBSERVER.init_version();
ObRawExpr *raw_expr = NULL;
OK(ObRawExprUtils::make_raw_expr_from_str(expr_str, strlen(expr_str), ctx, raw_expr, columns,
sys_vars, &sub_query_info, aggr_exprs ,win_exprs, udf_info));
_OB_LOG(DEBUG, "================================================================");
_OB_LOG(DEBUG, "%s", expr);
_OB_LOG(DEBUG, "%s", CSJ(raw_expr));
ObRawExprCanonicalizerImpl canon(ctx);
OK(canon.canonicalize(raw_expr));
canon_expr = CSJ(raw_expr);
_OB_LOG(DEBUG, "canon_expr=%s", canon_expr);
}
TEST_F(TestRawExprCanonicalizer, basic_test)
{
set_compat_mode(Worker::CompatMode::MYSQL);
std::ifstream if_tests("./expr/test_raw_expr_canonicalizer.test");
ASSERT_TRUE(if_tests.is_open());
std::string line;
const char* canon_expr = NULL;
std::ofstream of_result("./expr/test_raw_expr_canonicalizer.tmp");
ASSERT_TRUE(of_result.is_open());
while (std::getline(if_tests, line)) {
of_result << line << std::endl;
canon(line.c_str(), canon_expr);
of_result << canon_expr << std::endl;
}
of_result.close();
std::ifstream if_result("./expr/test_raw_expr_canonicalizer.tmp");
ASSERT_TRUE(if_result.is_open());
std::istream_iterator<std::string> it_result(if_result);
std::ifstream if_expected("./expr/test_raw_expr_canonicalizer.result");
ASSERT_TRUE(if_expected.is_open());
std::istream_iterator<std::string> it_expected(if_expected);
ASSERT_TRUE(std::equal(it_result, std::istream_iterator<std::string>(), it_expected));
std::remove("./test_raw_expr_canonicalizer.tmp");
}
int main(int argc, char **argv)
{
::testing::InitGoogleTest(&argc,argv);
return RUN_ALL_TESTS();
}

View File

@ -1,15 +0,0 @@
not (10 > 5 and 100 < 9)
not not a > b
not a > b
not a = b
not a < b
not a is true
not a between 1 and 100
(1 and 2) and (3 and 4) and 5
(1 or 2) or (3 or 4 or 5)
(1 and 2) or (3 and 1)
not(not(a))
A or (A And B) or (A And C)
(A and B) or (A and C and D)
(A and B) or (A and C)
(A and B and C) or (A and B and D)

View File

@ -16,7 +16,6 @@
#include "sql/resolver/expr/ob_raw_expr.h"
#include "sql/resolver/expr/ob_raw_expr_util.h"
#include "sql/resolver/expr/ob_raw_expr_print_visitor.h"
#include "sql/resolver/expr/ob_raw_expr_canonicalizer_impl.h"
#include "sql/resolver/expr/ob_raw_expr_resolver_impl.h"
#include "sql/ob_sql_init.h"
#include "../test_sql_utils.h"