init push

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

View File

@ -0,0 +1,225 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/recursive_cte/ob_fake_cte_table.h"
namespace oceanbase {
using namespace common;
namespace sql {
int ObFakeCTETable::ObFakeCTETableOperatorCtx::get_next_row(const common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
if (!has_valid_data()) {
ret = OB_ITER_END;
} else {
row = pump_row_;
empty_ = true;
}
return ret;
}
void ObFakeCTETable::ObFakeCTETableOperatorCtx::reuse()
{
pump_row_ = nullptr;
empty_ = true;
}
int ObFakeCTETable::ObFakeCTETableOperatorCtx::add_row(common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
common::ObNewRow* new_row = nullptr;
common::ObNewRow* old_row = nullptr;
if (has_valid_data()) {
LOG_DEBUG("Cur row count may be 0", K(lbt()), KPC(row), KPC(pump_row_));
}
common::ObNewRow& row_template = get_cur_row();
for (int64_t i = 0; i < column_involved_offset_.count(); ++i) {
int64_t offset = column_involved_offset_.at(i);
if (offset != OB_INVALID_INDEX) {
ObObj& fresh_obj = row->get_cell(offset);
row_template.cells_[i] = fresh_obj;
} else {
// do nothing
}
}
if (OB_FAIL(ObPhyOperator::deep_copy_row(row_template, new_row, alloc_))) {
LOG_WARN("Failed to deep copy new row", K(row_template), K(ret));
} else {
old_row = pump_row_;
pump_row_ = new_row;
empty_ = false;
if (nullptr != old_row) {
alloc_.free(old_row);
}
}
return ret;
}
int ObFakeCTETable::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_table_ctx = nullptr;
if (OB_ISNULL(cte_table_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Get physical operator context failed", K(ret), K_(id));
} else if (nullptr != cte_table_ctx->pump_row_) {
cte_table_ctx->empty_ = false;
}
return ret;
}
void ObFakeCTETable::reset()
{
ObNoChildrenPhyOperator::reset();
}
void ObFakeCTETable::reuse()
{
ObNoChildrenPhyOperator::reuse();
}
int ObFakeCTETable::has_valid_data(ObExecContext& ctx, bool& result) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_table_ctx = nullptr;
if (OB_ISNULL(cte_table_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Get physical operator context failed", K(ret), K_(id));
} else {
result = cte_table_ctx->has_valid_data();
}
return ret;
}
int64_t ObFakeCTETable::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV("pump table", column_involved_offset_);
return pos;
}
int ObFakeCTETable::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = nullptr;
if (OB_FAIL(inner_create_operator_ctx(ctx, op_ctx))) {
LOG_WARN("Inner create operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Op ctx is null", K(ret));
}
return ret;
}
int ObFakeCTETable::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = nullptr;
if (OB_NOT_NULL(op_ctx = GET_PHY_OPERATOR_CTX(ObPhyOperatorCtx, ctx, get_id()))) {
// do nothing
} else if (OB_FAIL(open_self(ctx))) {
LOG_WARN("Open fake cte table failed", K(ret));
}
return ret;
}
int ObFakeCTETable::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_fake_table_ctx = nullptr;
if (OB_ISNULL(cte_fake_table_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, ctx, get_id()))) {
LOG_DEBUG("get_phy_operator_ctx failed", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
} else {
cte_fake_table_ctx->reuse();
}
return ret;
}
int ObFakeCTETable::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
{
return CREATE_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, ctx, get_id(), get_type(), op_ctx);
}
int ObFakeCTETable::inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_ctx = nullptr;
if (OB_FAIL(try_check_status(exec_ctx))) {
LOG_WARN("Failed to check physical plan status", K(ret));
} else if ((OB_ISNULL(cte_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, exec_ctx, get_id())))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Failed to get fake cte ctx", K(ret));
} else if (OB_FAIL(cte_ctx->get_next_row(row))) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next sort row from recursive inner data", K(ret));
}
}
return ret;
}
int ObFakeCTETable::add_row(ObExecContext& exec_ctx, common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_ctx = nullptr;
ObPhyOperatorCtx* op_ctx = nullptr;
if (OB_FAIL(try_open_and_get_operator_ctx(exec_ctx, op_ctx))) {
LOG_WARN("Failed to init operator context", K(ret));
} else if (OB_ISNULL(cte_ctx = static_cast<ObFakeCTETableOperatorCtx*>(op_ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Failed to get fake ctx", K(ret));
} else if (OB_FAIL(try_check_status(exec_ctx))) {
LOG_WARN("Failed to check physical plan status", K(ret));
} else if (OB_FAIL(cte_ctx->add_row(row))) {
LOG_WARN("Add row to fake cte table op failed", K(ret));
}
return ret;
}
int ObFakeCTETable::set_empty(ObExecContext& exec_ctx) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_table_ctx = nullptr;
if (OB_ISNULL(cte_table_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, exec_ctx, get_id()))) {
// still not open
} else {
cte_table_ctx->empty_ = true;
}
return ret;
}
int ObFakeCTETable::open_self(ObExecContext& exec_ctx) const
{
int ret = OB_SUCCESS;
ObFakeCTETableOperatorCtx* cte_fake_table_ctx = nullptr;
if (OB_FAIL(init_op_ctx(exec_ctx))) {
LOG_WARN("Failed to init operator context", K(ret));
} else if (OB_ISNULL(cte_fake_table_ctx = GET_PHY_OPERATOR_CTX(ObFakeCTETableOperatorCtx, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Get physical operator context failed", K(ret), K_(id));
} else if (OB_FAIL(init_cur_row(*cte_fake_table_ctx, true))) {
LOG_WARN("Init current row failed", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < column_involved_offset_.count(); ++i) {
ret = cte_fake_table_ctx->column_involved_offset_.push_back(column_involved_offset_.at(i));
}
}
return ret;
}
OB_SERIALIZE_MEMBER((ObFakeCTETable, ObNoChildrenPhyOperator), column_involved_offset_);
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,90 @@
/**
* 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_FAKE_CTE_TABLE_H
#define _OB_FAKE_CTE_TABLE_H 1
#include "sql/engine/ob_phy_operator.h"
#include "lib/container/ob_array.h"
#include "lib/allocator/page_arena.h"
#include "lib/string/ob_string.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/engine/ob_no_children_phy_operator.h"
#include "sql/engine/table/ob_table_scan.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObFakeCTETable : public ObNoChildrenPhyOperator {
OB_UNIS_VERSION_V(1);
public:
class ObFakeCTETableOperatorCtx : public ObPhyOperatorCtx {
public:
explicit ObFakeCTETableOperatorCtx(ObExecContext& ctx)
: ObPhyOperatorCtx(ctx), empty_(false), pump_row_(nullptr), alloc_(ctx.get_allocator())
{}
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
inline bool has_valid_data()
{
return !empty_;
}
int get_next_row(const common::ObNewRow*& row);
int add_row(common::ObNewRow*& row);
void reuse();
public:
bool empty_;
common::ObNewRow* pump_row_;
common::ObSEArray<int64_t, 64> column_involved_offset_;
ObIAllocator& alloc_;
};
public:
explicit ObFakeCTETable(common::ObIAllocator& alloc) : ObNoChildrenPhyOperator(alloc), column_involved_offset_(alloc)
{}
virtual ~ObFakeCTETable()
{}
virtual void reset();
virtual void reuse();
int has_valid_data(ObExecContext& ctx, bool& result) const;
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
OB_INLINE virtual bool need_filter_row() const
{
return true;
}
// virtual int get_next_row(ObExecContext &ctx, const common::ObNewRow *&row) const;
virtual int rescan(ObExecContext& ctx) const;
virtual int init_op_ctx(ObExecContext& ctx) const;
virtual int inner_open(ObExecContext& ctx) const;
virtual int inner_close(ObExecContext& ctx) const;
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
int open_self(ObExecContext& ctx) const;
int set_empty(ObExecContext& exec_ctx) const;
int add_row(ObExecContext& exec_ctx, common::ObNewRow*& row) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObFakeCTETable);
// index refers to index in output_, and value refers to offset of column in cte table.
common::ObFixedArray<int64_t, common::ObIAllocator> column_involved_offset_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_FAKE_CTE_TABLE_H */

View File

@ -0,0 +1,178 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/recursive_cte/ob_fake_cte_table_op.h"
#include "ob_search_method_op.h"
namespace oceanbase {
using namespace common;
namespace sql {
OB_SERIALIZE_MEMBER((ObFakeCTETableSpec, ObOpSpec), column_involved_offset_, column_involved_exprs_);
int ObFakeCTETableOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_FAIL(try_check_status())) {
LOG_WARN("Failed to check physical plan status", K(ret));
} else if (!has_valid_data()) {
ret = OB_ITER_END;
} else if (OB_ISNULL(pump_row_) && OB_UNLIKELY(MY_SPEC.column_involved_exprs_.count() != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("pump row is null", K(ret));
} else if (OB_LIKELY(NULL != pump_row_) && OB_FAIL(pump_row_->to_expr(MY_SPEC.column_involved_exprs_, eval_ctx_))) {
LOG_WARN("Stored row to expr failed", K(ret));
} else {
empty_ = true;
}
return ret;
}
void ObFakeCTETableOp::reuse()
{
pump_row_ = nullptr;
empty_ = true;
}
int ObFakeCTETableOp::add_row(ObChunkDatumStore::StoredRow* row)
{
int ret = OB_SUCCESS;
const ObChunkDatumStore::StoredRow* new_row = nullptr;
ObChunkDatumStore::StoredRow* old_row = nullptr;
if (OB_UNLIKELY(0 == MY_SPEC.column_involved_offset_.count())) {
empty_ = false;
} else if (OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fake cte table add row count != output_ count", KPC(row));
} else if (OB_FAIL(deep_copy_row(
row, new_row, MY_SPEC.column_involved_offset_, ObSearchMethodOp::ROW_EXTRA_SIZE, allocator_))) {
LOG_WARN("fail to deep copy stored row", K(ret));
} else {
old_row = const_cast<ObChunkDatumStore::StoredRow*>(pump_row_);
pump_row_ = new_row;
empty_ = false;
if (nullptr != old_row) {
allocator_.free(old_row);
}
}
return ret;
}
int ObFakeCTETableOp::rescan()
{
int ret = OB_SUCCESS;
if (pump_row_ != nullptr) {
empty_ = false;
}
return ret;
}
int ObFakeCTETableOp::inner_open()
{
int ret = OB_SUCCESS;
if (MY_SPEC.column_involved_exprs_.count() != MY_SPEC.column_involved_offset_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid fake cte table spec", K(ret));
}
return ret;
}
int ObFakeCTETableOp::inner_close()
{
int ret = OB_SUCCESS;
reuse();
return ret;
}
int ObFakeCTETableOp::copy_datums(ObChunkDatumStore::StoredRow* row, common::ObDatum* datums, int64_t cnt,
const common::ObIArray<int64_t>& chosen_datums, char* buf, const int64_t size, const int64_t row_size,
const uint32_t row_extend_size)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == row || row->payload_ != buf || size < 0 || nullptr == datums || chosen_datums.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), KP(buf), K(size), K(datums));
} else {
row->cnt_ = static_cast<uint32_t>(chosen_datums.count());
int64_t pos = sizeof(ObDatum) * row->cnt_ + row_extend_size;
row->row_size_ = static_cast<int32_t>(row_size);
for (int64_t i = 0; OB_SUCC(ret) && i < row->cnt_; ++i) {
int64_t idx = chosen_datums.at(i);
if (OB_UNLIKELY(idx >= cnt)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), KP(row->payload_), KP(buf), K(size), K(datums), K(idx), K(cnt));
} else {
ObDatum* datum = new (&row->cells()[i]) ObDatum();
if (OB_FAIL(datum->deep_copy(datums[idx], buf, size, pos))) {
LOG_WARN(
"failed to copy datum", K(ret), K(i), K(pos), K(size), K(row_size), K(datums[idx]), K(datums[idx].len_));
}
}
}
}
return ret;
}
int ObFakeCTETableOp::deep_copy_row(const ObChunkDatumStore::StoredRow* src_row,
const ObChunkDatumStore::StoredRow*& dst_row, const ObIArray<int64_t>& chosen_index, int64_t extra_size,
ObIAllocator& allocator)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(src_row) || OB_ISNULL(src_row->cells())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("src row is null", K(ret), K(src_row));
} else if (chosen_index.empty()) {
dst_row = nullptr;
} else {
char* buf = nullptr;
int64_t row_size = sizeof(ObDatum) * chosen_index.count();
for (int64_t i = 0; OB_SUCC(ret) && i < chosen_index.count(); i++) {
int64_t idx = chosen_index.at(i);
if (OB_UNLIKELY(idx >= src_row->cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("chosen index greater than src_row count", K(ret), K(chosen_index), K(src_row->cnt_));
} else {
row_size += src_row->cells()[idx].len_;
}
}
if (OB_SUCC(ret)) {
int64_t buffer_len = 0;
int64_t head_size = sizeof(ObChunkDatumStore::StoredRow);
int64_t pos = head_size;
ObChunkDatumStore::StoredRow* new_row = nullptr;
buffer_len = row_size + head_size + extra_size;
if (OB_ISNULL(buf = reinterpret_cast<char*>(allocator.alloc(buffer_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc buf failed", K(ret));
} else if (OB_ISNULL(new_row = new (buf) ObChunkDatumStore::StoredRow())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to new row", K(ret));
} else if (OB_FAIL(copy_datums(new_row,
const_cast<ObDatum*>(src_row->cells()),
src_row->cnt_,
chosen_index,
buf + pos,
buffer_len - head_size,
row_size,
extra_size))) {
LOG_WARN("failed to deep copy row", K(ret), K(buffer_len), K(row_size));
} else {
dst_row = new_row;
}
}
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,82 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef _OB_FAKE_CTE_TABLE_OP_H
#define _OB_FAKE_CTE_TABLE_OP_H 1
#include "lib/container/ob_array.h"
#include "lib/allocator/page_arena.h"
#include "lib/string/ob_string.h"
#include "sql/engine/ob_operator.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObFakeCTETableSpec : public ObOpSpec {
OB_UNIS_VERSION_V(1);
public:
explicit ObFakeCTETableSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type), column_involved_offset_(alloc), column_involved_exprs_(alloc)
{}
virtual ~ObFakeCTETableSpec()
{}
// index refers to index in output_, and value refers to offset of column in cte table.
common::ObFixedArray<int64_t, common::ObIAllocator> column_involved_offset_;
common::ObFixedArray<ObExpr*, common::ObIAllocator> column_involved_exprs_;
};
class ObFakeCTETableOp : public ObOperator {
public:
explicit ObFakeCTETableOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input), empty_(false), pump_row_(nullptr), allocator_(exec_ctx.get_allocator())
{}
inline virtual void destroy()
{
ObOperator::destroy();
}
inline bool has_valid_data()
{
return !empty_;
}
void reuse();
virtual int rescan() override;
virtual int inner_open() override;
virtual int inner_close() override;
virtual int inner_get_next_row() override;
inline void set_empty()
{
empty_ = true;
}
int add_row(ObChunkDatumStore::StoredRow* row);
int copy_datums(ObChunkDatumStore::StoredRow* row, common::ObDatum* datums, int64_t cnt,
const common::ObIArray<int64_t>& chosen_datums, char* buf, const int64_t size, const int64_t row_size,
const uint32_t row_extend_size);
// copy rows of chosen_index from src_row to dst_row.
int deep_copy_row(const ObChunkDatumStore::StoredRow* src_row, const ObChunkDatumStore::StoredRow*& dst_row,
const common::ObIArray<int64_t>& chosen_index, int64_t extra_size, common::ObIAllocator& allocator);
const static int64_t ROW_EXTRA_SIZE = 0;
private:
bool empty_;
const ObChunkDatumStore::StoredRow* pump_row_;
ObIAllocator& allocator_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_FAKE_CTE_TABLE_OP_H */

View File

@ -0,0 +1,461 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_double_children_phy_operator.h"
#include "sql/engine/set/ob_merge_set_operator.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "ob_recursive_inner_data.h"
namespace oceanbase {
namespace sql {
int ObRecursiveInnerData::init(const ObIArray<int64_t>& cte_pseudo_column_row_desc)
{
int ret = OB_SUCCESS;
if (OB_FAIL(cte_pseudo_column_row_desc_.init(ObCTEPseudoColumn::CTE_PSEUDO_COLUMN_CNT))) {
LOG_WARN("Failed to init cte_pseudo_column_row_desc_", K(ret));
} else if (ObCTEPseudoColumn::CTE_PSEUDO_COLUMN_CNT != cte_pseudo_column_row_desc.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Wrong cte pseudo column row desc", K(ret), K(cte_pseudo_column_row_desc.count()));
} else if (OB_FAIL(cte_pseudo_column_row_desc_.assign(cte_pseudo_column_row_desc))) {
LOG_WARN("Failed to assign pseudo row desc", K(ret));
} else if (OB_FAIL(dfs_pump_.init())) {
LOG_WARN("Failed to init depth first search pump", K(ret));
}
return ret;
}
int ObRecursiveInnerData::add_search_column(ObSortColumn col)
{
int ret = OB_SUCCESS;
if (OB_FAIL(search_by_col_lists_.push_back(col))) {
LOG_WARN("Add search col info failed", K(ret));
} else if (OB_FAIL(dfs_pump_.add_sort_column(col))) {
LOG_WARN("Add search col info failed", K(ret));
} else if (OB_FAIL(bfs_pump_.add_sort_column(col))) {
LOG_WARN("Add search col info failed", K(ret));
}
return ret;
}
int ObRecursiveInnerData::add_cycle_column(common::ObColumnInfo col)
{
int ret = OB_SUCCESS;
if (OB_FAIL(cycle_by_col_lists_.push_back(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
} else if (OB_FAIL(dfs_pump_.add_cycle_by_column(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
} else if (OB_FAIL(bfs_pump_.add_cycle_by_column(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
}
return ret;
}
int ObRecursiveInnerData::get_all_data_from_left_child(ObExecContext& exec_ctx)
{
int ret = OB_SUCCESS;
const ObNewRow* input_row = nullptr;
uint64_t left_rows_count = 0;
if (OB_ISNULL(left_op_) || OB_ISNULL(right_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Left_op_ or right_op_ is nullptr", K(ret), K(left_op_), K(right_op_));
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(left_op_->get_next_row(exec_ctx, input_row))) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next row", K(ret));
}
} else {
LOG_DEBUG("Get rows from left op: ", KPC(input_row), K(left_op_->get_type()));
++left_rows_count;
if (SearchStrategyType::BREADTH_FRIST == search_type_) {
if (OB_FAIL(bfs_pump_.add_row(input_row))) {
LOG_WARN("Failed to add row", K(ret));
}
} else if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(dfs_pump_.add_row(input_row))) {
LOG_WARN("Failed to add row", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected search strategy", K(ret), K(search_type_));
}
}
}
}
if (OB_ITER_END == ret) {
ret = (left_rows_count == 0) ? OB_ITER_END : OB_SUCCESS;
}
return ret;
}
int ObRecursiveInnerData::get_all_data_from_right_child(ObExecContext& exec_ctx)
{
int ret = OB_SUCCESS;
const ObNewRow* input_row = nullptr;
if (OB_ISNULL(left_op_) || OB_ISNULL(right_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Left_op_ or right_op_ is nullptr", K(ret), K(left_op_), K(right_op_));
} else {
while (OB_SUCC(ret) && OB_SUCC(right_op_->get_next_row(exec_ctx, input_row))) {
LOG_DEBUG("Get rows from right op: ", K(ret), KPC(input_row), K(right_op_->get_type()));
if (OB_ITER_END == ret) {
} else if (OB_SUCC(ret) && SearchStrategyType::BREADTH_FRIST == search_type_) {
LOG_DEBUG("Get rows from right op: ", KPC(input_row));
if (OB_FAIL(bfs_pump_.add_row(input_row))) {
LOG_WARN("Failed to add row", K(ret));
}
} else if (OB_SUCC(ret) && SearchStrategyType::DEPTH_FRIST == search_type_) {
LOG_DEBUG("Get rows from right op: ", KPC(input_row));
if (OB_FAIL(dfs_pump_.add_row(input_row))) {
LOG_WARN("Failed to add row", K(ret));
}
} else {
}
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
return ret;
}
int ObRecursiveInnerData::try_format_output_row(const ObNewRow*& output_row)
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
// try to get row again
if (!result_output_.empty()) {
if (OB_FAIL(result_output_.pop_front(result_node))) {
LOG_WARN("Get result output failed", K(ret));
} else if (OB_ISNULL(result_node.row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Get a null result output", K(ret));
}
} else {
ret = OB_ITER_END;
}
// try format row
if (OB_SUCC(ret)) {
LOG_DEBUG("Result row", KPC(result_node.row_));
if (OB_FAIL(assign_to_cur_row(*cur_row_, *result_node.row_))) {
LOG_WARN("Failed to assign input row to cur row", K(ret));
} else if (OB_FAIL(add_pseudo_column(cur_row_, result_node.is_cycle_))) {
LOG_WARN("Add pseudo column failed", K(ret));
} else {
output_row = cur_row_;
LOG_DEBUG("Format row", K(cte_pseudo_column_row_desc_));
}
}
return ret;
}
int ObRecursiveInnerData::depth_first_union(ObExecContext& exec_ctx, const bool sort /*=true*/)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(dfs_pump_.finish_add_row(sort))) {
LOG_WARN("Failed to add row", K(ret));
} else if (OB_FAIL(set_fake_cte_table_empty(exec_ctx))) {
LOG_WARN("Set fake cte table to empty failed", K(ret));
} else if (OB_FAIL(dfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(exec_ctx, node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
}
return ret;
}
int ObRecursiveInnerData::fake_cte_table_add_row(ObExecContext& exec_ctx, ObTreeNode& node)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(pump_operator_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Fake cte table op can not be nullptr", K(ret));
} else if (OB_FAIL(pump_operator_->add_row(exec_ctx, node.row_))) {
LOG_WARN("Fake cte table add row failed", K(ret));
} else if (SearchStrategyType::BREADTH_FRIST == search_type_ && OB_FAIL(bfs_pump_.update_parent_node(node))) {
LOG_WARN("Failed to update last bst node stask", K(ret), K(node));
} else if (SearchStrategyType::DEPTH_FRIST == search_type_ && OB_FAIL(dfs_pump_.adjust_stack(node))) {
LOG_WARN("Failed to adjust stask", K(ret), K(node));
}
return ret;
}
int ObRecursiveInnerData::breadth_first_union(ObExecContext& exec_ctx, bool left_branch, bool& continue_search)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(set_fake_cte_table_empty(exec_ctx))) {
LOG_WARN("Set fake cte table to empty failed", K(ret));
} else if (OB_FAIL(bfs_pump_.add_result_rows())) {
LOG_WARN("Failed to finish add row", K(ret));
} else if (OB_FAIL(bfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
continue_search = false;
if (OB_FAIL(start_new_level(exec_ctx, left_branch))) {
LOG_WARN("Failed to start new level", K(ret));
}
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(exec_ctx, node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
} else if (search_by_col_lists_.empty()) {
/**
* when sort_collations_ is not empty, we need to sort all sibling nodes,
* so continue_search is true.
* when sort_collations_ is empty, we can output each node when it's generated,
* so continue_search is false.
*/
continue_search = false;
}
return ret;
}
int ObRecursiveInnerData::start_new_level(ObExecContext& exec_ctx, bool left_branch)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(bfs_pump_.finish_add_row(!left_branch))) {
LOG_WARN("Failed to finish add row", K(ret));
} else if (OB_FAIL(bfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
state_ = RecursiveUnionState::R_UNION_END;
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(exec_ctx, node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
}
return ret;
}
int ObRecursiveInnerData::try_get_left_rows(ObExecContext& exec_ctx, const common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
const bool sort = false;
if (OB_FAIL(get_all_data_from_left_child(exec_ctx))) {
if (OB_ITER_END == ret) {
// do nothing
} else {
LOG_WARN("Get row from left child failed", K(ret));
}
} else {
if (SearchStrategyType::BREADTH_FRIST == search_type_) {
bool continue_search = false;
if (OB_FAIL(breadth_first_union(exec_ctx, true, continue_search))) {
LOG_WARN("Breadth first union failed", K(ret));
}
} else if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(depth_first_union(exec_ctx, sort))) {
LOG_WARN("Depth first union failed", K(ret));
}
} else {
// never get there
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(try_format_output_row(row))) {
LOG_WARN("Failed to get next row", K(ret));
}
}
return ret;
}
int ObRecursiveInnerData::try_get_right_rows(ObExecContext& exec_ctx, const common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(get_all_data_from_right_child(exec_ctx))) {
LOG_WARN("Get row from right child failed", K(ret));
} else if (OB_FAIL(right_op_->rescan(exec_ctx))) {
LOG_WARN("Recursive union right children rescan failed", K(ret));
} else if (dfs_pump_.empty()) {
// do nothing
} else if (OB_FAIL(depth_first_union(exec_ctx))) {
LOG_WARN("Depth first union failed", K(ret));
}
} else if (SearchStrategyType::BREADTH_FRIST == search_type_) {
bool continue_search = true;
while (OB_SUCC(ret) && continue_search) {
if (OB_FAIL(get_all_data_from_right_child(exec_ctx))) {
LOG_WARN("Get row from right child failed", K(ret));
} else if (OB_FAIL(right_op_->rescan(exec_ctx))) {
LOG_WARN("Recursive union right children rescan failed", K(ret));
} else if (bfs_pump_.empty()) {
break;
} else if (OB_FAIL(breadth_first_union(exec_ctx, false, continue_search))) {
LOG_WARN("Breadth first union failed", K(ret));
}
}
}
// try to get row again
if (OB_SUCC(ret)) {
if (OB_FAIL(try_format_output_row(row))) {
if (ret != OB_ITER_END) {
LOG_WARN("Failed to get next row", K(ret));
}
}
}
return ret;
}
int ObRecursiveInnerData::get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
if (!result_output_.empty()) {
if (OB_FAIL(try_format_output_row(row))) {
LOG_WARN("Format output row failed", K(ret));
} else {
}
} else if (RecursiveUnionState::R_UNION_READ_LEFT == state_) {
if (OB_FAIL(try_get_left_rows(exec_ctx, row))) {
if (ret != OB_ITER_END) {
LOG_WARN("Get left rows failed", K(ret));
} else {
state_ = RecursiveUnionState::R_UNION_END;
}
} else {
state_ = R_UNION_READ_RIGHT;
}
} else if (RecursiveUnionState::R_UNION_READ_RIGHT == state_) {
if (OB_FAIL(try_get_right_rows(exec_ctx, row))) {
if (ret != OB_ITER_END) {
LOG_WARN("Get right rows failed", K(ret));
} else {
state_ = RecursiveUnionState::R_UNION_END;
}
} else {
}
} else if (RecursiveUnionState::R_UNION_END == state_) {
ret = OB_ITER_END;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected state", K(ret), K(state_));
}
return ret;
}
int ObRecursiveInnerData::add_pseudo_column(const ObNewRow* output_row, bool cycle /*default false*/)
{
int ret = OB_SUCCESS;
if (!search_by_col_lists_.empty()) {
if (OB_ISNULL(output_row) || OB_UNLIKELY(!output_row->is_valid()) || OB_ISNULL(calc_buf_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid parameter", K(ret), KPC(output_row));
} else if (OB_UNLIKELY(ObCTEPseudoColumn::CTE_PSEUDO_COLUMN_CNT != cte_pseudo_column_row_desc_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid pseudo_column_row_desc", K(ret));
} else if (ObMergeSetOperator::UNUSED_POS != cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_SEARCH]) {
int64_t cell_idx = cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_SEARCH];
if (cell_idx > output_row->count_ - 1) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid cell idx", K(ret), K(cell_idx), K(output_row->count_));
} else {
ObObj val;
int64_t cur_level = ordering_column_;
common::number::ObNumber num;
if (OB_FAIL(num.from(cur_level, *calc_buf_))) {
LOG_WARN("Failed to create ObNumber", K(ret));
} else {
val.set_number(num);
}
output_row->cells_[cell_idx] = val;
++ordering_column_;
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, search by clause without pseudo info", K(ret));
}
}
if (OB_SUCC(ret) && !cycle_by_col_lists_.empty()) {
if (OB_ISNULL(output_row) || OB_UNLIKELY(!output_row->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid parameter", K(ret), KPC(output_row));
} else if (OB_UNLIKELY(ObCTEPseudoColumn::CTE_PSEUDO_COLUMN_CNT != cte_pseudo_column_row_desc_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid pseudo_column_row_desc", K(ret));
} else if (ObMergeSetOperator::UNUSED_POS != cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_CYCLE]) {
int64_t cell_idx = cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_CYCLE];
if (cell_idx > output_row->count_ - 1) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid cell idx", K(cell_idx), K(output_row->count_), K(ret));
} else {
output_row->cells_[cell_idx] = cycle ? cycle_value_ : non_cycle_value_;
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, cycle by clause without pseudo info", K(ret));
}
}
return ret;
}
int ObRecursiveInnerData::rescan(ObExecContext& ctx)
{
UNUSED(ctx);
int ret = OB_SUCCESS;
state_ = R_UNION_READ_LEFT;
result_output_.reset();
ordering_column_ = 1;
dfs_pump_.reuse();
bfs_pump_.reuse();
return ret;
}
int ObRecursiveInnerData::set_fake_cte_table_empty(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(pump_operator_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The fake cte table is null", K(ret));
} else if (OB_FAIL(pump_operator_->set_empty(ctx))) {
LOG_WARN("Set fake cte table empty failed", K(ret));
}
return ret;
}
void ObRecursiveInnerData::destroy()
{
stored_row_buf_.reset();
}
int ObRecursiveInnerData::assign_to_cur_row(ObNewRow& cur_row, const ObNewRow& input_row)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(cur_row.count_ < input_row.projector_size_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN(
"Cur column count is not enough to store child row", K(ret), K_(cur_row.count), K_(input_row.projector_size));
} else {
for (int64_t i = 0; i < input_row.get_count(); ++i) {
cur_row.cells_[i] = input_row.get_cell(i);
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,145 @@
/**
* 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_RECURSIVE_INNER_DATA_
#define OB_RECURSIVE_INNER_DATA_
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_double_children_phy_operator.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "sql/engine/sort/ob_base_sort.h"
#include "sql/engine/sort/ob_specific_columns_sort.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "ob_search_method.h"
#include "ob_fake_cte_table.h"
namespace oceanbase {
namespace sql {
class ObRecursiveInnerData {
using ObTreeNode = ObSearchMethod::ObTreeNode;
friend class ObRecursiveUnionAllOperatorCtx;
friend class ObRecursiveUnionAll;
public:
struct RowComparer;
enum RecursiveUnionState { R_UNION_BEGIN, R_UNION_READ_LEFT, R_UNION_READ_RIGHT, R_UNION_END, R_UNION_STATE_COUNT };
enum SearchStrategyType { DEPTH_FRIST, BREADTH_FRIST };
public:
explicit ObRecursiveInnerData(common::ObIAllocator& alloc)
: state_(RecursiveUnionState::R_UNION_READ_LEFT),
stored_row_buf_(ObModIds::OB_SQL_CTE_ROW),
pump_operator_(nullptr),
left_op_(nullptr),
right_op_(nullptr),
search_type_(SearchStrategyType::BREADTH_FRIST),
search_by_col_lists_(),
cycle_by_col_lists_(),
result_output_(stored_row_buf_),
cte_pseudo_column_row_desc_(alloc),
cur_row_(nullptr),
cycle_value_(),
non_cycle_value_(),
ordering_column_(1),
dfs_pump_(stored_row_buf_),
bfs_pump_(stored_row_buf_),
calc_buf_(NULL)
{}
~ObRecursiveInnerData() = default;
inline void set_left_child(ObPhyOperator* op)
{
left_op_ = op;
};
inline void set_right_child(ObPhyOperator* op)
{
right_op_ = op;
};
inline void set_fake_cte_table(const ObFakeCTETable* cte_table)
{
pump_operator_ = cte_table;
};
inline void set_search_strategy(ObRecursiveInnerData::SearchStrategyType strategy)
{
search_type_ = strategy;
};
int add_search_column(ObSortColumn col);
int add_cycle_column(common::ObColumnInfo);
int get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row);
int rescan(ObExecContext& ctx);
int set_fake_cte_table_empty(ObExecContext& ctx);
int init(const common::ObIArray<int64_t>& cte_pseudo_column_row_desc);
void set_op_schema_objs(const common::ObIArray<ObOpSchemaObj>& op_schema_objs)
{
dfs_pump_.set_op_schema_objs(op_schema_objs);
}
void set_calc_buf(common::ObIAllocator* calc_buf)
{
calc_buf_ = calc_buf;
}
private:
void destroy();
int add_pseudo_column(const ObNewRow* row, bool cycle = false);
int try_get_left_rows(ObExecContext& exec_ctx, const ObNewRow*& row);
int try_get_right_rows(ObExecContext& exec_ctx, const ObNewRow*& row);
int try_format_output_row(const ObNewRow*& output_row);
/**
* left child of recursive union is called plan a, and right child is called plan b
* plan a generate initial data, plan b is executed recursively.
*/
int get_all_data_from_left_child(ObExecContext& exec_ctx);
int get_all_data_from_right_child(ObExecContext& exec_ctx);
int depth_first_union(ObExecContext& exec_ctx, const bool sort = true);
int breadth_first_union(ObExecContext& exec_ctx, bool left_branch, bool& continue_search);
int start_new_level(ObExecContext& exec_ctx, bool left_branch);
// output one row to fake_cte_table operator,use it as next intput of plan b.
int fake_cte_table_add_row(ObExecContext& exec_ctx, ObTreeNode& node);
int assign_to_cur_row(ObNewRow& cur_row, const ObNewRow& input_row);
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRecursiveInnerData);
private:
RecursiveUnionState state_;
common::ObArenaAllocator stored_row_buf_;
const ObFakeCTETable* pump_operator_;
ObPhyOperator* left_op_;
ObPhyOperator* right_op_;
// BFS or DFS
SearchStrategyType search_type_;
common::ObSEArray<ObSortColumn, 32> search_by_col_lists_;
common::ObSEArray<common::ObColumnInfo, 32> cycle_by_col_lists_;
common::ObList<ObTreeNode, common::ObIAllocator> result_output_;
common::ObFixedArray<int64_t, common::ObIAllocator> cte_pseudo_column_row_desc_;
// cur row
common::ObNewRow* cur_row_;
// cycle value
common::ObObj cycle_value_;
// non-cycle value
common::ObObj non_cycle_value_;
/**
* Oracle explaination:
* The ordering_column is automatically added to the column list for the query name.
* The query that selects from query_name can include an ORDER BY on ordering_column to return the rows in the order
* that was specified by the SEARCH clause.
*/
int64_t ordering_column_;
ObDepthFisrtSearch dfs_pump_;
ObBreadthFisrtSearch bfs_pump_;
common::ObIAllocator* calc_buf_;
};
} // end namespace sql
} // end namespace oceanbase
#endif

View File

@ -0,0 +1,391 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "ob_recursive_union_all_op.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "ob_recursive_inner_data_op.h"
namespace oceanbase {
namespace sql {
int ObRecursiveInnerDataOp::init(const ObExpr* search_expr, const ObExpr* cycle_expr)
{
int ret = OB_SUCCESS;
search_expr_ = search_expr;
cycle_expr_ = cycle_expr;
if (OB_FAIL(dfs_pump_.init())) {
LOG_WARN("Failed to init depth first search pump", K(ret));
}
return ret;
}
int ObRecursiveInnerDataOp::get_all_data_from_left_child()
{
int ret = OB_SUCCESS;
uint64_t left_rows_count = 0;
if (OB_ISNULL(left_op_) || OB_ISNULL(right_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Left_op_ or right_op_ is nullptr", K(ret), K(left_op_), K(right_op_));
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(left_op_->get_next_row())) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next row", K(ret));
}
} else {
++left_rows_count;
if (SearchStrategyType::BREADTH_FRIST == search_type_) {
if (OB_FAIL(bfs_pump_.add_row(left_op_->get_spec().output_, eval_ctx_))) {
LOG_WARN("Failed to add row", K(ret));
}
} else if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(dfs_pump_.add_row(left_op_->get_spec().output_, eval_ctx_))) {
LOG_WARN("Failed to add row", K(ret));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected search strategy", K(ret), K(search_type_));
}
}
}
}
if (OB_ITER_END == ret) {
// no rows from left child, end immediately.
ret = (left_rows_count == 0) ? OB_ITER_END : OB_SUCCESS;
}
return ret;
}
int ObRecursiveInnerDataOp::get_all_data_from_right_child()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(left_op_) || OB_ISNULL(right_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Left_op_ or right_op_ is nullptr", K(ret), K(left_op_), K(right_op_));
} else {
while (OB_SUCC(ret) && OB_SUCC(right_op_->get_next_row())) {
if (OB_ITER_END == ret) {
} else if (OB_SUCC(ret) && SearchStrategyType::BREADTH_FRIST == search_type_) {
if (OB_FAIL(bfs_pump_.add_row(right_op_->get_spec().output_, eval_ctx_))) { // add stored row
LOG_WARN("Failed to add row", K(ret));
}
} else if (OB_SUCC(ret) && SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(dfs_pump_.add_row(right_op_->get_spec().output_, eval_ctx_))) {
LOG_WARN("Failed to add row", K(ret));
}
} else {
}
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
return ret;
}
int ObRecursiveInnerDataOp::try_format_output_row()
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
// try to get row again
if (!result_output_.empty()) {
if (OB_FAIL(result_output_.pop_front(result_node))) {
LOG_WARN("Get result output failed", K(ret));
} else if (OB_ISNULL(result_node.stored_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Get a null result output", K(ret));
}
} else {
ret = OB_ITER_END;
}
// try format row
if (OB_SUCC(ret)) {
if (OB_FAIL(assign_to_cur_row(result_node.stored_row_))) {
LOG_WARN("Failed to assign input row to cur row", K(ret));
} else if (OB_FAIL(add_pseudo_column(result_node.is_cycle_))) {
LOG_WARN("Add pseudo column failed", K(ret));
}
}
return ret;
}
int ObRecursiveInnerDataOp::depth_first_union(const bool sort /*=true*/)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(dfs_pump_.finish_add_row(sort))) {
LOG_WARN("Failed to add row", K(ret));
} else if (OB_FAIL(set_fake_cte_table_empty())) {
LOG_WARN("Set fake cte table to empty failed", K(ret));
} else if (OB_FAIL(dfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
}
return ret;
}
int ObRecursiveInnerDataOp::fake_cte_table_add_row(ObTreeNode& node)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(pump_operator_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Fake cte table op can not be nullptr", K(ret));
} else if (OB_FAIL(pump_operator_->add_row(node.stored_row_))) {
LOG_WARN("Fake cte table add row failed", K(ret));
} else if (SearchStrategyType::BREADTH_FRIST == search_type_ && OB_FAIL(bfs_pump_.update_parent_node(node))) {
LOG_WARN("Failed to update last bst node stask", K(ret), K(node));
} else if (SearchStrategyType::DEPTH_FRIST == search_type_ && OB_FAIL(dfs_pump_.adjust_stack(node))) {
LOG_WARN("Failed to adjust stask", K(ret), K(node));
}
return ret;
}
int ObRecursiveInnerDataOp::breadth_first_union(bool left_branch, bool& continue_search)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(set_fake_cte_table_empty())) {
LOG_WARN("Set fake cte table to empty failed", K(ret));
} else if (OB_FAIL(bfs_pump_.add_result_rows())) {
LOG_WARN("Failed to finish add row", K(ret));
} else if (OB_FAIL(bfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
continue_search = false;
if (OB_FAIL(start_new_level(left_branch))) {
LOG_WARN("Failed to start new level", K(ret));
}
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
} else if (sort_collations_.empty()) {
/**
* when sort_collations_ is not empty, we need to sort all sibling nodes,
* so continue_search is true.
* when sort_collations_ is empty, we can output each node when it's generated,
* so continue_search is false.
*/
continue_search = false;
}
return ret;
}
int ObRecursiveInnerDataOp::start_new_level(bool left_branch)
{
int ret = OB_SUCCESS;
ObTreeNode node;
if (OB_FAIL(bfs_pump_.finish_add_row(!left_branch))) {
LOG_WARN("Failed to finish add row", K(ret));
} else if (OB_FAIL(bfs_pump_.get_next_non_cycle_node(result_output_, node))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
state_ = RecursiveUnionState::R_UNION_END;
} else {
LOG_WARN("Failed to get next non cycle node", K(ret));
}
} else if (OB_FAIL(fake_cte_table_add_row(node))) {
LOG_WARN("Fake cte table add row failed", K(ret));
}
return ret;
}
int ObRecursiveInnerDataOp::try_get_left_rows()
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
const bool sort = false;
if (OB_FAIL(get_all_data_from_left_child())) {
if (OB_ITER_END == ret) {
// do nothing
} else {
LOG_WARN("Get row from left child failed", K(ret));
}
} else {
if (SearchStrategyType::BREADTH_FRIST == search_type_) {
bool continue_search = false;
if (OB_FAIL(breadth_first_union(true, continue_search))) {
LOG_WARN("Breadth first union failed", K(ret));
}
} else if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(depth_first_union(sort))) {
LOG_WARN("Depth first union failed", K(ret));
}
} else {
// never get there
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(try_format_output_row())) {
LOG_WARN("Failed to get next row", K(ret));
}
}
return ret;
}
int ObRecursiveInnerDataOp::try_get_right_rows()
{
int ret = OB_SUCCESS;
ObTreeNode result_node;
if (SearchStrategyType::DEPTH_FRIST == search_type_) {
if (OB_FAIL(get_all_data_from_right_child())) {
LOG_WARN("Get row from right child failed", K(ret));
} else if (OB_FAIL(right_op_->rescan())) {
LOG_WARN("Recursive union right children rescan failed", K(ret));
} else if (dfs_pump_.empty()) {
// do nothing
} else if (OB_FAIL(depth_first_union())) {
LOG_WARN("Depth first union failed", K(ret));
}
} else if (SearchStrategyType::BREADTH_FRIST == search_type_) {
bool continue_search = true;
while (OB_SUCC(ret) && continue_search) {
if (OB_FAIL(get_all_data_from_right_child())) {
LOG_WARN("Get row from right child failed", K(ret));
} else if (OB_FAIL(right_op_->rescan())) {
LOG_WARN("Recursive union right children rescan failed", K(ret));
} else if (bfs_pump_.empty()) {
break;
} else if (OB_FAIL(breadth_first_union(false, continue_search))) {
LOG_WARN("Breadth first union failed", K(ret));
}
}
}
// try to get row again
if (OB_SUCC(ret)) {
if (OB_FAIL(try_format_output_row())) {
if (ret != OB_ITER_END) {
LOG_WARN("Failed to get next row", K(ret));
}
}
}
return ret;
}
int ObRecursiveInnerDataOp::get_next_row()
{
int ret = OB_SUCCESS;
if (!result_output_.empty()) {
if (OB_FAIL(try_format_output_row())) {
LOG_WARN("Format output row failed", K(ret));
} else {
}
} else if (RecursiveUnionState::R_UNION_READ_LEFT == state_) {
if (OB_FAIL(try_get_left_rows())) {
if (ret != OB_ITER_END) {
LOG_WARN("Get left rows failed", K(ret));
} else {
state_ = RecursiveUnionState::R_UNION_END;
}
} else {
state_ = R_UNION_READ_RIGHT;
}
} else if (RecursiveUnionState::R_UNION_READ_RIGHT == state_) {
if (OB_FAIL(try_get_right_rows())) {
if (ret != OB_ITER_END) {
LOG_WARN("Get right rows failed", K(ret));
} else {
state_ = RecursiveUnionState::R_UNION_END;
}
} else {
}
} else if (RecursiveUnionState::R_UNION_END == state_) {
ret = OB_ITER_END;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected state", K(ret), K(state_));
}
return ret;
}
int ObRecursiveInnerDataOp::add_pseudo_column(bool cycle /*default false*/)
{
int ret = OB_SUCCESS;
if (nullptr != search_expr_) {
int64_t cur_level = ordering_column_;
common::number::ObNumber res_num;
char buf_alloc[common::number::ObNumber::MAX_BYTE_LEN];
ObDataBuffer allocator(buf_alloc, common::number::ObNumber::MAX_BYTE_LEN);
if (OB_FAIL(res_num.from(cur_level, allocator))) {
LOG_WARN("fail to create obnumber", K(ret));
} else {
search_expr_->locate_datum_for_write(eval_ctx_).set_number(res_num);
search_expr_->get_eval_info(eval_ctx_).evaluated_ = true;
++ordering_column_;
}
} else {
// there is no search column or it's not in the output.
}
if (nullptr != cycle_expr_) {
if (OB_FAIL(cycle_expr_->deep_copy_datum(eval_ctx_, cycle ? cycle_value_ : non_cycle_value_))) {
LOG_WARN("expr datum deep copy failed", K(ret));
} else {
cycle_expr_->get_eval_info(eval_ctx_).evaluated_ = true;
}
}
return ret;
}
int ObRecursiveInnerDataOp::rescan()
{
int ret = OB_SUCCESS;
state_ = R_UNION_READ_LEFT;
result_output_.reset();
ordering_column_ = 1;
dfs_pump_.reuse();
bfs_pump_.reuse();
return ret;
}
int ObRecursiveInnerDataOp::set_fake_cte_table_empty()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(pump_operator_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The fake cte table is null", K(ret));
} else {
pump_operator_->set_empty();
}
return ret;
}
void ObRecursiveInnerDataOp::destroy()
{
stored_row_buf_.reset();
}
int ObRecursiveInnerDataOp::assign_to_cur_row(ObChunkDatumStore::StoredRow* stored_row)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(stored_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("stored row is null", K(stored_row));
} else {
if (OB_SUCC(ret)) {
if (OB_FAIL(stored_row->to_expr(output_union_exprs_, eval_ctx_))) {
LOG_WARN("stored row to exprs failed", K(ret));
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,157 @@
/**
* 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_RECURSIVE_INNER_DATA_OP_
#define OB_RECURSIVE_INNER_DATA_OP_
#include "sql/engine/ob_operator.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "sql/engine/sort/ob_base_sort.h"
#include "sql/engine/sort/ob_specific_columns_sort.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "ob_search_method_op.h"
#include "ob_fake_cte_table_op.h"
#include "sql/engine/ob_operator.h"
namespace oceanbase {
namespace sql {
class ObRecursiveInnerDataOp {
using ObTreeNode = ObSearchMethodOp::ObTreeNode;
friend class ObRecursiveUnionAllOp;
friend class ObRecursiveUnionAllSpec;
public:
struct RowComparer;
enum RecursiveUnionState { R_UNION_BEGIN, R_UNION_READ_LEFT, R_UNION_READ_RIGHT, R_UNION_END, R_UNION_STATE_COUNT };
enum SearchStrategyType { DEPTH_FRIST, BREADTH_FRIST };
public:
explicit ObRecursiveInnerDataOp(ObEvalCtx& eval_ctx, ObExecContext& exec_ctx, const ExprFixedArray& left_output,
const common::ObIArray<ObSortFieldCollation>& sort_collations,
const common::ObIArray<uint64_t>& cycle_by_col_lists, const common::ObIArray<ObExpr*>& output_union_exprs)
: state_(RecursiveUnionState::R_UNION_READ_LEFT),
stored_row_buf_(ObModIds::OB_SQL_CTE_ROW),
pump_operator_(nullptr),
left_op_(nullptr),
right_op_(nullptr),
search_type_(SearchStrategyType::BREADTH_FRIST),
sort_collations_(sort_collations),
result_output_(stored_row_buf_),
search_expr_(nullptr),
cycle_expr_(nullptr),
cycle_value_(),
non_cycle_value_(),
cte_columns_(nullptr),
ordering_column_(1),
dfs_pump_(stored_row_buf_, left_output, sort_collations, cycle_by_col_lists),
bfs_pump_(stored_row_buf_, left_output, sort_collations, cycle_by_col_lists),
eval_ctx_(eval_ctx),
ctx_(exec_ctx),
output_union_exprs_(output_union_exprs)
{}
~ObRecursiveInnerDataOp() = default;
inline void set_left_child(ObOperator* op)
{
left_op_ = op;
};
inline void set_right_child(ObOperator* op)
{
right_op_ = op;
};
inline void set_fake_cte_table(ObFakeCTETableOp* cte_table)
{
pump_operator_ = cte_table;
};
inline void set_search_strategy(ObRecursiveInnerDataOp::SearchStrategyType strategy)
{
search_type_ = strategy;
}
int add_sort_collation(ObSortFieldCollation sort_collation);
int add_cycle_column(uint64_t index);
int add_cmp_func(ObCmpFunc cmp_func);
int get_next_row();
int rescan();
int set_fake_cte_table_empty();
int init(const ObExpr* search_expr, const ObExpr* cycle_expr);
void set_cte_column_exprs(common::ObIArray<ObExpr*>* exprs)
{
cte_columns_ = exprs;
}
private:
void destroy();
int add_pseudo_column(bool cycle = false);
int try_get_left_rows();
int try_get_right_rows();
int try_format_output_row();
/**
* The left child of a recursive union is called plan a, and the right child is called plan b
* plan a will produce initial data, and the recursive union itself controls the progress of recursion.
* The right child is a plan executed recursively
*/
int get_all_data_from_left_child();
int get_all_data_from_right_child();
// In depth-first recursion, perform row UNION ALL operations
int depth_first_union(const bool sort = true);
// In breadth-first recursion, the UNION ALL operation of the row is performed
int breadth_first_union(bool left_branch, bool& continue_search);
int start_new_level(bool left_branch);
// output a row to the fake cte table operator,
// which will be used as the input for the plan b later
int fake_cte_table_add_row(ObTreeNode& node);
// set value for cte table column expr
int assign_to_cur_row(ObChunkDatumStore::StoredRow* stored_row);
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRecursiveInnerDataOp);
private:
RecursiveUnionState state_;
common::ObArenaAllocator stored_row_buf_;
ObFakeCTETableOp* pump_operator_;
ObOperator* left_op_;
ObOperator* right_op_;
// Mark depth first or breadth first
SearchStrategyType search_type_;
// Sort by which columns
const common::ObIArray<ObSortFieldCollation>& sort_collations_;
// The data to be output to the next operator, R in pseudo code
common::ObList<ObTreeNode, common::ObIAllocator> result_output_;
// pseudo column
const ObExpr* search_expr_;
const ObExpr* cycle_expr_;
// cycle value
ObDatum cycle_value_;
// non-cycle value
ObDatum non_cycle_value_;
common::ObIArray<ObExpr*>* cte_columns_;
/**
* represent search breadth/depth first by xxx set ordering_column.
* Oracle explain it as:
* The ordering_column is automatically added to the column list for the query name.
* The query that selects from query_name can include an ORDER BY on ordering_column to return
* the rows in the order that was specified by the SEARCH clause.
*/
int64_t ordering_column_;
// depth first
ObDepthFisrtSearchOp dfs_pump_;
// breadth first
ObBreadthFisrtSearchOp bfs_pump_;
ObEvalCtx& eval_ctx_;
ObExecContext& ctx_;
const common::ObIArray<ObExpr*>& output_union_exprs_;
};
} // end namespace sql
} // end namespace oceanbase
#endif

View File

@ -0,0 +1,242 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/recursive_cte/ob_recursive_union_all.h"
#include "common/object/ob_object.h"
#include "sql/engine/expr/ob_expr_operator.h"
namespace oceanbase {
using namespace common;
namespace sql {
int ObRecursiveUnionAll::ObRecursiveUnionAllOperatorCtx::ctx_close()
{
int ret = OB_SUCCESS;
inner_data_.search_by_col_lists_.reset();
inner_data_.cycle_by_col_lists_.reset();
inner_data_.result_output_.reset();
inner_data_.pump_operator_ = nullptr;
inner_data_.left_op_ = nullptr;
inner_data_.right_op_ = nullptr;
return ret;
}
ObRecursiveUnionAll::ObRecursiveUnionAll(common::ObIAllocator& alloc)
: ObMergeSetOperator(alloc),
search_by_col_lists_(alloc),
cycle_by_col_lists_(alloc),
pump_operator_(nullptr),
strategy_(ObRecursiveInnerData::SearchStrategyType::BREADTH_FRIST),
cycle_value_(alloc),
cycle_default_value_(alloc)
{}
ObRecursiveUnionAll::~ObRecursiveUnionAll()
{}
void ObRecursiveUnionAll::reset()
{
cycle_by_col_lists_.reset();
search_by_col_lists_.reset();
ObMergeSetOperator::reset();
}
void ObRecursiveUnionAll::reuse()
{
search_by_col_lists_.reuse();
cycle_by_col_lists_.reuse();
ObMergeSetOperator::reuse();
}
int ObRecursiveUnionAll::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
LOG_DEBUG("Do recursive union rescan");
ObRecursiveUnionAllOperatorCtx* cte_ctx = nullptr;
if (OB_ISNULL(cte_ctx = GET_PHY_OPERATOR_CTX(ObRecursiveUnionAllOperatorCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Failed to get physical operator", K(ret), K(ctx), K_(id));
} else if (OB_FAIL(cte_ctx->inner_data_.rescan(ctx))) {
LOG_WARN("Failed to rescan inner data", K(ret));
} else if (OB_FAIL(ObPhyOperator::rescan(ctx))) {
LOG_WARN("Operator rescan failed", K(ret));
}
return ret;
}
int ObRecursiveUnionAll::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObRecursiveUnionAllOperatorCtx* cte_ctx = nullptr;
ObObj cycle_value;
ObObj non_cycle_value;
ObNewRow empty_row;
if (OB_ISNULL(get_child(FIRST_CHILD))) {
ret = OB_NOT_INIT;
LOG_WARN("Left op is null", K(ret));
} else if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("Failed to init operator context", K(ret), K(ret));
} else if (OB_ISNULL(cte_ctx = GET_PHY_OPERATOR_CTX(ObRecursiveUnionAllOperatorCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Failed to get physical operator", K(ret), K(ctx), K_(id));
} else if (OB_FAIL(wrap_expr_ctx(ctx, cte_ctx->expr_ctx_))) {
LOG_WARN("Failed to wrap expr ctx", K(ret));
} else if (OB_FAIL(cte_ctx->inner_data_.init(cte_pseudo_column_row_desc_))) {
LOG_WARN("Failed to create hash filter", K(ret));
} else if (!cycle_value_.is_empty() && OB_FAIL(cycle_value_.calc(cte_ctx->expr_ctx_, empty_row, cycle_value))) {
LOG_WARN("Failed to calculate cycle value", K(ret));
} else if (!cycle_default_value_.is_empty() &&
OB_FAIL(cycle_default_value_.calc(cte_ctx->expr_ctx_, empty_row, non_cycle_value))) {
LOG_WARN("Failed to calculate non-cycle value", K(ret));
} else {
cte_ctx->inner_data_.set_left_child(get_child(FIRST_CHILD));
cte_ctx->inner_data_.set_right_child(get_child(SECOND_CHILD));
cte_ctx->inner_data_.set_fake_cte_table(pump_operator_);
cte_ctx->inner_data_.set_search_strategy(strategy_);
cte_ctx->inner_data_.cycle_value_ = cycle_value;
cte_ctx->inner_data_.non_cycle_value_ = non_cycle_value;
cte_ctx->inner_data_.set_op_schema_objs(get_op_schema_objs());
cte_ctx->inner_data_.set_calc_buf(&cte_ctx->get_calc_buf());
ARRAY_FOREACH(search_by_col_lists_, i)
{
if (OB_FAIL(cte_ctx->inner_data_.add_search_column(search_by_col_lists_.at(i)))) {
LOG_WARN("Add search col to inner data failed", K(ret));
}
}
ARRAY_FOREACH(cycle_by_col_lists_, i)
{
if (OB_FAIL(cte_ctx->inner_data_.add_cycle_column(cycle_by_col_lists_.at(i)))) {
LOG_WARN("Add cycle col to inner data failed", K(ret));
}
}
}
return ret;
}
int ObRecursiveUnionAll::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObRecursiveUnionAllOperatorCtx* cte_ctx = nullptr;
if (OB_ISNULL(cte_ctx = GET_PHY_OPERATOR_CTX(ObRecursiveUnionAllOperatorCtx, ctx, get_id()))) {
LOG_DEBUG("The operator has not been opened.", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
} else if (OB_FAIL(cte_ctx->ctx_close())) {
LOG_WARN("ctx close failed", K(ret));
}
return ret;
}
int ObRecursiveUnionAll::inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObRecursiveUnionAllOperatorCtx* cte_ctx = nullptr;
if (OB_FAIL(try_check_status(exec_ctx))) {
LOG_WARN("Failed to check physical plan status", K(ret));
} else if ((OB_ISNULL(cte_ctx = GET_PHY_OPERATOR_CTX(ObRecursiveUnionAllOperatorCtx, exec_ctx, get_id())))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Failed to get recursive union ctx", K(ret));
} else if (OB_FAIL(cte_ctx->inner_data_.get_next_row(exec_ctx, row))) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next sort row from recursive inner data", K(ret));
}
}
return ret;
}
int ObRecursiveUnionAll::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = nullptr;
if (OB_FAIL(inner_create_operator_ctx(ctx, op_ctx))) {
LOG_WARN("Inner create operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op_ctx is null", K(ret));
} else if (OB_FAIL(init_cur_row(*op_ctx, true /* need create cells */))) {
LOG_WARN("Init current row failed", K(ret));
} else {
ObRecursiveUnionAllOperatorCtx* union_ctx = static_cast<ObRecursiveUnionAllOperatorCtx*>(op_ctx);
union_ctx->set_cur_row();
}
return ret;
}
int ObRecursiveUnionAll::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
{
return CREATE_PHY_OPERATOR_CTX(ObRecursiveUnionAllOperatorCtx, ctx, get_id(), get_type(), op_ctx);
}
int ObRecursiveUnionAll::set_cycle_pseudo_values(ObSqlExpression& v, ObSqlExpression& d_v)
{
int ret = OB_SUCCESS;
if (OB_FAIL(cycle_value_.assign(v))) {
LOG_WARN("Failed to add cycle value", K(ret));
} else if (OB_FAIL(cycle_default_value_.assign(d_v))) {
LOG_WARN("Failed to add non-cycle value", K(ret));
}
return ret;
}
int64_t ObRecursiveUnionAll::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV("strategy ", strategy_, "search_by_col_lists", search_by_col_lists_, "cycle_by_col_lists_", cycle_by_col_lists_);
return pos;
}
OB_DEF_SERIALIZE(ObRecursiveUnionAll)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObMergeSetOperator::serialize(buf, buf_len, pos))) {
LOG_WARN("failed to serialize merge set operator", K(ret));
} else if (OB_FAIL(search_by_col_lists_.serialize(buf, buf_len, pos))) {
LOG_WARN("failed to serialize search_by_col_list", K(ret));
} else if (OB_FAIL(cycle_by_col_lists_.serialize(buf, buf_len, pos))) {
LOG_WARN("failed to serialize cycle_by_col_list", K(ret));
} else {
OB_UNIS_ENCODE(strategy_);
OB_UNIS_ENCODE(cycle_value_);
OB_UNIS_ENCODE(cycle_default_value_);
}
return ret;
}
OB_DEF_DESERIALIZE(ObRecursiveUnionAll)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObMergeSetOperator::deserialize(buf, data_len, pos))) {
LOG_WARN("failed to deserialize merge set operator", K(ret));
} else if (OB_FAIL(search_by_col_lists_.deserialize(buf, data_len, pos))) {
LOG_WARN("failed to deserialize search_by_col_list", K(ret));
} else if (OB_FAIL(cycle_by_col_lists_.deserialize(buf, data_len, pos))) {
LOG_WARN("failed to deserialize cycle_by_col_list", K(ret));
} else {
OB_UNIS_DECODE(strategy_);
OB_UNIS_DECODE(cycle_value_);
OB_UNIS_DECODE(cycle_default_value_);
}
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObRecursiveUnionAll)
{
int64_t len = 0;
len += ObMergeSetOperator::get_serialize_size();
OB_UNIS_ADD_LEN(search_by_col_lists_);
OB_UNIS_ADD_LEN(cycle_by_col_lists_);
OB_UNIS_ADD_LEN(strategy_);
OB_UNIS_ADD_LEN(cycle_value_);
OB_UNIS_ADD_LEN(cycle_default_value_);
return len;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,135 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_OB_RECURSIVE_UNION_ALL_OPERATOR_H_
#define OCEANBASE_SQL_OB_RECURSIVE_UNION_ALL_OPERATOR_H_
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/set/ob_merge_set_operator.h"
#include "sql/engine/ob_double_children_phy_operator.h"
#include "sql/engine/sort/ob_specific_columns_sort.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "ob_fake_cte_table.h"
#include "ob_recursive_inner_data.h"
namespace oceanbase {
namespace sql {
class ObRecursiveUnionAll : public ObMergeSetOperator {
OB_UNIS_VERSION_V(1);
protected:
class ObRecursiveUnionAllOperatorCtx : public ObPhyOperatorCtx {
public:
explicit ObRecursiveUnionAllOperatorCtx(ObExecContext& ctx)
: ObPhyOperatorCtx(ctx), inner_data_(ctx.get_allocator())
{}
~ObRecursiveUnionAllOperatorCtx()
{}
virtual void destroy()
{
inner_data_.~ObRecursiveInnerData();
ObPhyOperatorCtx::destroy_base();
}
void set_search_strategy(ObRecursiveInnerData::SearchStrategyType strategy)
{
inner_data_.set_search_strategy(strategy);
}
void set_cur_row()
{
inner_data_.cur_row_ = &get_cur_row();
}
int init();
int ctx_close();
public:
common::ObExprCtx expr_ctx_;
ObRecursiveInnerData inner_data_;
};
public:
explicit ObRecursiveUnionAll(common::ObIAllocator& alloc);
~ObRecursiveUnionAll();
virtual void reset();
virtual void reuse();
virtual int rescan(ObExecContext& ctx) const;
inline void set_search_strategy(ObRecursiveInnerData::SearchStrategyType strategy)
{
strategy_ = strategy;
};
inline void set_fake_cte_table(ObFakeCTETable* cte_table)
{
pump_operator_ = cte_table;
};
int set_cycle_pseudo_values(ObSqlExpression& v, ObSqlExpression& d_v);
protected:
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
/**
* @brief create operator context, only child operator can know it's specific operator type,
* so must be overwrited by child operator,
* @param ctx[in], execute context
* @param op_ctx[out], the pointer of operator context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief: called by get_next_row(), get a row from the child operator or row_store
* @param: ctx[in], execute context
* @param: row[out], ObSqlRow an obj array and row_size
*/
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRecursiveUnionAll);
public:
common::ObFixedArray<ObSortColumn, common::ObIAllocator> search_by_col_lists_;
common::ObFixedArray<common::ObColumnInfo, common::ObIAllocator> cycle_by_col_lists_;
protected:
static const int32_t CMP_DIRECTION_ASC = 1;
static const int32_t CMP_DIRECTION_DESC = -1;
const ObFakeCTETable* pump_operator_;
ObRecursiveInnerData::SearchStrategyType strategy_;
ObSqlExpression cycle_value_;
ObSqlExpression cycle_default_value_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SQL_OB_SET_OPERATOR_H_ */

View File

@ -0,0 +1,132 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/recursive_cte/ob_recursive_union_all_op.h"
#include "sql/engine/expr/ob_expr_operator.h"
namespace oceanbase {
using namespace common;
namespace sql {
const int64_t ObRecursiveUnionAllSpec::UNUSED_POS = -2;
int ObRecursiveUnionAllOp::inner_close()
{
int ret = OB_SUCCESS;
inner_data_.result_output_.reset();
inner_data_.pump_operator_ = nullptr;
inner_data_.left_op_ = nullptr;
inner_data_.right_op_ = nullptr;
return ret;
}
ObRecursiveUnionAllSpec::ObRecursiveUnionAllSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type),
sort_collations_(alloc),
cycle_by_col_lists_(alloc),
output_union_exprs_(alloc),
pump_operator_id_(OB_INVALID_ID),
search_expr_(nullptr),
cycle_expr_(nullptr),
strategy_(ObRecursiveInnerDataOp::SearchStrategyType::BREADTH_FRIST),
cycle_value_(nullptr),
cycle_default_value_(nullptr)
{}
ObRecursiveUnionAllSpec::~ObRecursiveUnionAllSpec()
{}
OB_SERIALIZE_MEMBER((ObRecursiveUnionAllSpec, ObOpSpec), sort_collations_, cycle_by_col_lists_, output_union_exprs_,
pump_operator_id_, search_expr_, cycle_expr_, strategy_, cycle_value_, cycle_default_value_);
int ObRecursiveUnionAllOp::rescan()
{
int ret = OB_SUCCESS;
if (OB_FAIL(inner_data_.rescan())) {
LOG_WARN("Failed to rescan inner data", K(ret));
} else if (OB_FAIL(ObOperator::rescan())) {
LOG_WARN("Operator rescan failed", K(ret));
}
return ret;
}
int ObRecursiveUnionAllOp::inner_open()
{
int ret = OB_SUCCESS;
ObDatum* cycle_value = NULL;
ObDatum* non_cycle_value = NULL;
const ObExpr* cycle_expr = MY_SPEC.cycle_value_;
const ObExpr* cycle_default_expr = MY_SPEC.cycle_default_value_;
ObOperatorKit* op_kit = nullptr;
if (OB_ISNULL(left_)) {
ret = OB_NOT_INIT;
LOG_WARN("Left op is null", K(ret));
} else if (OB_FAIL(inner_data_.init(MY_SPEC.search_expr_, MY_SPEC.cycle_expr_))) {
LOG_WARN("Failed to create hash filter", K(ret));
} else if (NULL != cycle_expr && OB_FAIL(cycle_expr->eval(eval_ctx_, cycle_value))) {
LOG_WARN("Failed to calculate cycle value", K(ret));
} else if (NULL != cycle_expr && OB_FAIL(inner_data_.cycle_value_.deep_copy(*cycle_value, ctx_.get_allocator()))) {
LOG_WARN("datum deep copy failed", K(ret));
} else if (NULL != cycle_default_expr && OB_FAIL(cycle_default_expr->eval(eval_ctx_, non_cycle_value))) {
LOG_WARN("Failed to calculate non-cycle value", K(ret));
} else if (NULL != cycle_default_expr &&
OB_FAIL(inner_data_.non_cycle_value_.deep_copy(*non_cycle_value, ctx_.get_allocator()))) {
LOG_WARN("datum deep copy failed", K(ret));
} else if (OB_ISNULL(op_kit = ctx_.get_operator_kit(MY_SPEC.pump_operator_id_)) || OB_ISNULL(op_kit->op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get ObOperater from exec ctx failed",
K(MY_SPEC.pump_operator_id_),
K(op_kit),
K(MY_SPEC.search_expr_),
K(MY_SPEC.strategy_));
} else {
inner_data_.set_left_child(left_);
inner_data_.set_right_child(right_);
LOG_DEBUG(
"recursive union all inner open", K(MY_SPEC.output_), K(MY_SPEC.left_->output_), K(MY_SPEC.right_->output_));
inner_data_.set_fake_cte_table(static_cast<ObFakeCTETableOp*>(op_kit->op_));
inner_data_.set_search_strategy(MY_SPEC.strategy_);
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.get_left()->get_output_count(); i++) {
const ObExpr* expr = MY_SPEC.get_left()->output_.at(i);
if (OB_ISNULL(expr) || OB_ISNULL(expr->basic_funcs_) || OB_ISNULL(expr->basic_funcs_->null_first_cmp_) ||
OB_ISNULL(expr->basic_funcs_->null_last_cmp_) || OB_ISNULL(expr->basic_funcs_->default_hash_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("left output expr is null or basic_funcs_ is null", K(ret));
}
}
}
return ret;
}
int ObRecursiveUnionAllOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_FAIL(try_check_status())) {
LOG_WARN("Failed to check physical plan status", K(ret));
} else if (OB_FAIL(inner_data_.get_next_row())) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next sort row from recursive inner data", K(ret));
}
}
return ret;
}
int ObRecursiveUnionAllSpec::set_cycle_pseudo_values(ObExpr* v, ObExpr* d_v)
{
int ret = OB_SUCCESS;
cycle_value_ = v;
cycle_default_value_ = d_v;
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,116 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_OB_RECURSIVE_UNION_ALL_OP_H_
#define OCEANBASE_SQL_OB_RECURSIVE_UNION_ALL_OP_H_
#include "sql/engine/set/ob_merge_set_op.h"
#include "sql/engine/sort/ob_specific_columns_sort.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/allocator/ob_malloc.h"
#include "ob_fake_cte_table_op.h"
#include "ob_recursive_inner_data_op.h"
namespace oceanbase {
namespace sql {
class ObRecursiveUnionAllSpec : public ObOpSpec {
OB_UNIS_VERSION_V(1);
public:
explicit ObRecursiveUnionAllSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
~ObRecursiveUnionAllSpec();
friend class ObRecursiveUnionAllOp;
void set_search_pseudo_column(ObExpr* expr)
{
search_expr_ = expr;
}
void set_cycle_pseudo_column(ObExpr* expr)
{
cycle_expr_ = expr;
}
inline void set_search_strategy(ObRecursiveInnerDataOp::SearchStrategyType strategy)
{
strategy_ = strategy;
}
inline void set_fake_cte_table(uint64_t cte_table_id)
{
pump_operator_id_ = cte_table_id;
};
int set_cycle_pseudo_values(ObExpr* v, ObExpr* d_v);
static const int64_t UNUSED_POS;
protected:
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
// virtual int64_t to_string_kv(char *buf, const int64_t buf_len) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRecursiveUnionAllSpec);
public:
common::ObFixedArray<ObSortFieldCollation, common::ObIAllocator> sort_collations_;
common::ObFixedArray<uint64_t, common::ObIAllocator> cycle_by_col_lists_;
// T_OP_UNION expression in the output of recursive uion all operator.
// copy datums of inner_data rows to datums of these exprs.
common::ObFixedArray<ObExpr*, common::ObIAllocator> output_union_exprs_;
protected:
static const int32_t CMP_DIRECTION_ASC = 1;
static const int32_t CMP_DIRECTION_DESC = -1;
uint64_t pump_operator_id_;
ObExpr* search_expr_;
ObExpr* cycle_expr_;
ObRecursiveInnerDataOp::SearchStrategyType strategy_;
ObExpr* cycle_value_;
ObExpr* cycle_default_value_;
};
class ObRecursiveUnionAllOp : public ObOperator {
public:
explicit ObRecursiveUnionAllOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input),
inner_data_(*exec_ctx.get_eval_ctx(), exec_ctx, // spec.output_,
MY_SPEC.get_left()->output_, MY_SPEC.sort_collations_, MY_SPEC.cycle_by_col_lists_,
MY_SPEC.output_union_exprs_)
{}
~ObRecursiveUnionAllOp()
{}
virtual int inner_open() override;
virtual int inner_close() override;
virtual int inner_get_next_row() override;
virtual int rescan() override;
virtual void destroy()
{
inner_data_.~ObRecursiveInnerDataOp();
ObOperator::destroy();
}
void set_search_strategy(ObRecursiveInnerDataOp::SearchStrategyType strategy)
{
inner_data_.set_search_strategy(strategy);
}
const ObRecursiveUnionAllSpec& get_spec() const
{
return static_cast<const ObRecursiveUnionAllSpec&>(spec_);
}
public:
ObRecursiveInnerDataOp inner_data_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SQL_OB_SET_OPERATOR_H_ */

View File

@ -0,0 +1,476 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/ob_phy_operator.h"
#include "ob_search_method.h"
using namespace oceanbase::sql;
using namespace oceanbase::common;
int ObSearchMethod::reuse()
{
input_rows_.reuse();
sort_.reuse();
return OB_SUCCESS;
}
int ObSearchMethod::add_row(const common::ObNewRow* input_row)
{
int ret = OB_SUCCESS;
ObNewRow* new_row = nullptr;
if (input_rows_.empty() && 0 == input_rows_.get_capacity() && OB_FAIL(input_rows_.reserve(INIT_ROW_COUNT))) {
LOG_WARN("Failed to pre allocate array", K(ret));
} else if (OB_ISNULL(input_row)) {
ret = OB_BAD_NULL_ERROR;
LOG_WARN("Input row is null", K(ret));
} else if (OB_FAIL(ObPhyOperator::deep_copy_row(*input_row, new_row, allocator_))) {
LOG_WARN("Deep copy input row failed", K(ret));
} else if (OB_FAIL(input_rows_.push_back(new_row))) {
LOG_WARN("Push new row to result input error", K(ret));
} else {
LOG_DEBUG("Add a new row", KPC(new_row));
}
return ret;
}
int ObSearchMethod::sort_input_rows()
{
int ret = OB_SUCCESS;
bool need_sort = false;
// set sort column
if (OB_FAIL(sort_.set_sort_columns(sort_columns_, 0))) {
LOG_WARN("Failed to set sort columns", K(ret));
}
// sort
for (int64_t i = 0; OB_SUCC(ret) && i < input_rows_.count(); i++) {
if (OB_FAIL(sort_.add_row(*input_rows_.at(i), need_sort, false))) {
LOG_WARN("Failed to add row", K(ret));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(sort_.sort_rows())) {
LOG_WARN("Sort result input row failed", K(ret));
} else {
input_rows_.reuse();
const ObNewRow* row = nullptr;
while (OB_SUCC(sort_.get_next_row(row))) {
if (OB_FAIL(input_rows_.push_back(const_cast<common::ObNewRow*>(row)))) {
LOG_WARN("Failed to push back row", K(ret));
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
}
// reuse
sort_.reuse();
return ret;
}
int ObSearchMethod::sort_rownodes(ObArray<ObTreeNode>& sort_array)
{
int ret = OB_SUCCESS;
if (!sort_array.empty()) {
LOG_DEBUG("Sort row nodes", K(sort_array.count()));
ObTreeNode* first_row = &sort_array.at(0);
ObNodeComparer comparer(sort_columns_, &ret);
std::sort(first_row, first_row + sort_array.count(), comparer);
if (OB_SUCCESS != ret) {
LOG_WARN("Failed to do sort", K(ret));
}
}
return ret;
}
int ObSearchMethod::add_sort_column(ObSortColumn col)
{
int ret = OB_SUCCESS;
if (OB_FAIL(sort_columns_.push_back(col))) {
LOG_WARN("Add sort col info failed", K(ret));
}
return ret;
}
int ObSearchMethod::add_cycle_by_column(ObColumnInfo col)
{
int ret = OB_SUCCESS;
if (OB_FAIL(cycle_by_columns_.push_back(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
}
return ret;
}
int ObSearchMethod::is_same_row(ObNewRow& row_1st, ObNewRow& row_2nd, bool& is_cycle)
{
int ret = OB_SUCCESS;
if (row_1st.get_count() != row_2nd.get_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Inconformity row schema", K(ret), K(row_1st), K(row_2nd));
} else if (cycle_by_columns_.empty()) {
// detect whole row
is_cycle = false;
if (row_1st == row_2nd) {
is_cycle = true;
ret = OB_ERR_CYCLE_FOUND_IN_RECURSIVE_CTE;
LOG_WARN("Cycle detected while executing recursive WITH query", K(ret));
}
} else {
// detect some obj cell
is_cycle = true;
for (int64_t i = 0; OB_SUCC(ret) && i < cycle_by_columns_.count(); ++i) {
uint64_t index = cycle_by_columns_.at(i).index_;
if (index >= row_1st.get_count() || index >= row_2nd.get_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Column index out of range", K(ret));
} else if (row_1st.get_cell(index) != row_2nd.get_cell(index)) {
is_cycle = false;
break;
}
} // end for
}
return ret;
}
int ObDepthFisrtSearch::add_cycle_by_column(ObColumnInfo col)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObSearchMethod::add_cycle_by_column(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
} else if (OB_FAIL(hash_col_idx_.push_back(col))) {
LOG_WARN("Add cycle col info failed", K(ret));
}
return ret;
}
int ObDepthFisrtSearch::reuse()
{
ObSearchMethod::reuse();
last_node_level_ = UINT64_MAX;
hash_filter_rows_.reuse();
return OB_SUCCESS;
}
int ObDepthFisrtSearch::finish_add_row(bool sort)
{
int ret = OB_SUCCESS;
if (sort && OB_FAIL(sort_input_rows())) {
LOG_WARN("Sort input rows failed", K(ret));
} else if (input_rows_.empty()) {
if (current_search_path_.empty()) {
// do nothing
} else {
int64_t count = current_search_path_.count();
common::ObNewRow* tmp_row = current_search_path_.at(count - 1);
ObHashCols hash_col;
hash_col.init(tmp_row, &hash_col_idx_);
if (OB_FAIL(hash_filter_rows_.erase_refactored(hash_col))) {
LOG_WARN("Earse rows from the hash map failed", K(ret));
} else if (OB_FAIL(recycle_rows_.push_back(tmp_row))) {
LOG_WARN("Failed to push back rows", K(ret));
} else {
current_search_path_.pop_back();
}
}
} else {
for (int64_t i = input_rows_.count() - 1; OB_SUCC(ret) && i >= 0; i--) {
ObTreeNode node;
node.row_ = input_rows_.at(i);
node.tree_level_ = (UINT64_MAX == last_node_level_) ? 0 : last_node_level_ + 1;
if (OB_FAIL(is_depth_cycle_node(node))) {
LOG_WARN("Check cycle node failed", K(ret));
} else if (OB_FAIL(search_stack_.push_front(node))) {
LOG_WARN("Push data to result hold stack failed", K(ret));
} else {
}
} // end for
}
input_rows_.reuse();
if (OB_SUCC(ret) && !recycle_rows_.empty()) {
ARRAY_FOREACH(recycle_rows_, i)
{
allocator_.free(recycle_rows_.at(i));
}
recycle_rows_.reuse();
}
return ret;
}
int ObDepthFisrtSearch::is_depth_cycle_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObHashCols hash_col;
hash_col.init(node.row_, &hash_col_idx_);
if (OB_FAIL(hash_filter_rows_.exist_refactored(hash_col))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else if (OB_HASH_EXIST == ret) {
ret = OB_SUCCESS;
node.is_cycle_ = true;
if (cycle_by_columns_.empty()) {
ret = OB_ERR_CYCLE_FOUND_IN_RECURSIVE_CTE;
LOG_WARN("Cycle detected while executing recursive WITH query", K(ret));
}
} else {
LOG_WARN("Failed to find in hashmap", K(ret));
}
} else {
// succ
}
return ret;
}
int ObDepthFisrtSearch::adjust_stack(ObTreeNode& node)
{
int ret = OB_SUCCESS;
last_node_level_ = node.tree_level_;
if (0 == hash_col_idx_.count()) {
// init hash column
for (int64_t i = 0; i < node.row_->get_count(); ++i) {
ObColumnInfo col_info;
col_info.cs_type_ = node.row_->get_cell(i).get_collation_type();
col_info.index_ = i;
hash_col_idx_.push_back(col_info);
}
}
if (current_search_path_.count() < node.tree_level_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The current node level is 2 or more large than last node level", K(ret));
} else if (current_search_path_.count() > node.tree_level_) {
/**
* Tree of all rows is:
* A
* AA AB
* AAA AAB ABA ABB
* current_search_path_ is A->AA->AAB
* when current node goes to AB, path will be A->AB
*/
int64_t count = current_search_path_.count();
int64_t pop_times = count - node.tree_level_;
common::ObNewRow* tmp_row = nullptr;
while (OB_SUCC(ret) && pop_times > 0) {
--pop_times;
ObHashCols hash_col;
if (OB_FAIL(current_search_path_.pop_back(tmp_row))) {
LOG_WARN("Failed to pop stack", K(ret));
} else if (FALSE_IT(hash_col.init(tmp_row, &hash_col_idx_))) {
} else if (OB_FAIL(hash_filter_rows_.erase_refactored(hash_col))) {
LOG_WARN("Earse rows from the hash map failed", K(ret));
} else if (OB_FAIL(recycle_rows_.push_back(tmp_row))) {
LOG_WARN("Failed to push back rows", K(ret));
}
}
}
// add stack
if (OB_SUCC(ret) && current_search_path_.count() == node.tree_level_) {
ObHashCols hash_col;
hash_col.init(node.row_, &hash_col_idx_);
if (OB_FAIL(current_search_path_.push_back(node.row_))) {
LOG_WARN("Push new row to result record failed", K(ret));
} else if (OB_FAIL(hash_filter_rows_.set_refactored(hash_col))) {
LOG_WARN("Failed to insert row to hashmap", K(ret));
}
}
return ret;
}
int ObDepthFisrtSearch::get_next_non_cycle_node(
ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObTreeNode non_cycle_node;
bool got_row = false;
while (OB_SUCC(ret) && !search_stack_.empty()) {
if (OB_FAIL(search_stack_.pop_front(non_cycle_node))) {
LOG_WARN("Search stack pop back failed", K(ret));
} else if (OB_FAIL(result_output.push_back(non_cycle_node))) {
LOG_WARN("Push back data to result output failed", K(ret));
} else if (non_cycle_node.is_cycle_) {
if (OB_FAIL(recycle_rows_.push_back(non_cycle_node.row_))) {
LOG_WARN("Failed to push back rows", K(ret));
}
} else {
got_row = true;
node = non_cycle_node;
break;
}
} // end while
if (OB_SUCC(ret) && !got_row) {
ret = OB_ITER_END;
}
return ret;
}
int ObBreadthFisrtSearch::add_new_level()
{
int ret = OB_SUCCESS;
ObTreeNode new_level_node;
new_level_node.row_ = nullptr;
if (OB_FAIL(search_queue_.push_back(new_level_node))) {
LOG_WARN("Push back data to result hold queue failed", K(ret));
}
return ret;
}
int ObBreadthFisrtSearch::reuse()
{
ObSearchMethod::reuse();
current_parent_node_ = &bst_root_;
bst_root_.child_num_ = 0;
bst_root_.children_ = nullptr;
bst_root_.parent_ = nullptr;
bst_root_.row_ = nullptr;
search_queue_.reset();
search_results_.reuse();
return OB_SUCCESS;
}
int ObBreadthFisrtSearch::init_new_nodes(ObBFSTreeNode* last_bstnode, int64_t child_num)
{
int ret = OB_SUCCESS;
void* childs_ptr = nullptr;
// init memory of a node
if (OB_UNLIKELY(0 == child_num)) {
// do nothing
} else if (OB_ISNULL(last_bstnode)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Last bst node can not be null", K(ret));
} else if (OB_UNLIKELY(last_bstnode->child_num_ != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Last bst node can not be ini twice", K(ret), KPC(last_bstnode));
} else if (OB_ISNULL(childs_ptr = allocator_.alloc(sizeof(ObBFSTreeNode*) * child_num))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("Alloc memory for row failed", "size", child_num * sizeof(ObBFSTreeNode*), K(ret));
} else {
last_bstnode->children_ = (ObBFSTreeNode**)childs_ptr;
last_bstnode->child_num_ = child_num;
}
return ret;
}
int ObBreadthFisrtSearch::is_breadth_cycle_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObBFSTreeNode* tmp = current_parent_node_;
ObNewRow* row = node.row_;
if (OB_ISNULL(tmp) || OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The last_bstnode and row an not be null", K(ret), KPC(row));
} else {
// row_ of bst_root_ is empty.
while (OB_SUCC(ret) && OB_NOT_NULL(tmp) && OB_NOT_NULL(tmp->row_)) {
ObNewRow* row_1st = row;
ObNewRow* row_2nd = tmp->row_;
if (OB_FAIL(is_same_row(*row_1st, *row_2nd, node.is_cycle_))) {
LOG_WARN("Failed to compare the two row", K(ret), KPC(row_1st), KPC(row_2nd));
} else if (node.is_cycle_) {
break;
} else {
tmp = tmp->parent_;
}
}
}
return ret;
}
int ObBreadthFisrtSearch::get_next_non_cycle_node(
ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObTreeNode non_cycle_node;
bool got_row = false;
while (OB_SUCC(ret) && !search_queue_.empty()) {
if (OB_FAIL(search_queue_.pop_front(non_cycle_node))) {
LOG_WARN("Get row from hold queue failed", K(ret));
} else if (OB_FAIL(result_output.push_back(non_cycle_node))) {
LOG_WARN("Failed to push row to output ", K(ret));
} else if (non_cycle_node.is_cycle_) {
if (OB_FAIL(recycle_rows_.push_back(non_cycle_node.row_))) {
LOG_WARN("Failed to push back rows", K(ret));
}
} else {
got_row = true;
node = non_cycle_node;
break;
}
} // end while
if (OB_SUCC(ret) && !got_row) {
ret = OB_ITER_END;
}
return ret;
}
int ObBreadthFisrtSearch::update_parent_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
current_parent_node_ = node.in_bstree_node_;
return ret;
}
int ObBreadthFisrtSearch::add_result_rows()
{
int ret = OB_SUCCESS;
if (OB_FAIL(init_new_nodes(current_parent_node_, input_rows_.count()))) {
LOG_WARN("Failed to init new bst node", K(ret));
} else {
ARRAY_FOREACH(input_rows_, i)
{
void* ptr = nullptr;
ObBFSTreeNode* tmp = nullptr;
ObTreeNode node;
node.row_ = input_rows_.at(i);
// breadth search tree
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObBFSTreeNode)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("Alloc memory for row failed", "size", sizeof(ObBFSTreeNode), K(ret));
} else {
tmp = new (ptr) ObBFSTreeNode();
tmp->row_ = input_rows_.at(i);
tmp->parent_ = current_parent_node_;
current_parent_node_->children_[i] = tmp;
node.in_bstree_node_ = tmp;
if (OB_FAIL(is_breadth_cycle_node(node))) {
LOG_WARN("Find cycle failed", K(ret));
} else if (OB_FAIL(search_results_.push_back(node))) {
LOG_WARN("Push back data to layer_results failed", K(ret));
} else {
LOG_DEBUG("Result node", K(node));
}
}
}
}
input_rows_.reuse();
return ret;
}
int ObBreadthFisrtSearch::finish_add_row(bool sort)
{
int ret = OB_SUCCESS;
if (sort && OB_FAIL(sort_rownodes(search_results_))) {
LOG_WARN("Failed to sort results", K(ret));
} else if (!search_queue_.empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The last result still has residual", K(search_queue_), K(ret));
} else {
ARRAY_FOREACH(search_results_, i)
{
if (OB_FAIL(search_queue_.push_back(search_results_.at(i)))) {
LOG_WARN("Push back failed", K(ret));
}
}
}
search_results_.reuse();
return ret;
}

View File

@ -0,0 +1,211 @@
/**
* 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_DEPTH_FIRST_SEARCH_H_
#define OB_DEPTH_FIRST_SEARCH_H_
#include "sql/engine/sort/ob_base_sort.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "lib/allocator/ob_malloc.h"
#include "lib/list/ob_list.h"
#include "common/row/ob_row.h"
namespace oceanbase {
namespace sql {
class ObSearchMethod {
public:
typedef struct _BreadthFirstSearchTreeNode {
_BreadthFirstSearchTreeNode() : child_num_(0), row_(nullptr), children_(nullptr), parent_(nullptr)
{}
int64_t child_num_;
common::ObNewRow* row_;
struct _BreadthFirstSearchTreeNode** children_;
struct _BreadthFirstSearchTreeNode* parent_;
TO_STRING_KV("row ", row_, "child_num_", child_num_);
} ObBFSTreeNode;
typedef struct _TreeNode {
_TreeNode() : is_cycle_(false), tree_level_(0), row_(nullptr), in_bstree_node_(nullptr)
{}
bool is_cycle_;
uint64_t tree_level_;
common::ObNewRow* row_;
ObBFSTreeNode* in_bstree_node_;
TO_STRING_KV("is tree level", tree_level_, "is cycle", is_cycle_, "row ", row_)
} ObTreeNode;
struct ObNodeComparer {
explicit ObNodeComparer(const common::ObIArray<ObSortColumn>& sort_columns, int* err)
: sort_columns_(sort_columns), err_(err)
{}
bool operator()(const ObTreeNode& r1, const ObTreeNode& r2)
{
bool bret = false;
if (OB_UNLIKELY(common::OB_SUCCESS != *err_)) {
// do nothing if we already have an error,
// so we can finish the sort process ASAP.
} else {
int cmp = 0;
for (int64_t i = 0; common::OB_SUCCESS == *err_ && 0 == cmp && i < sort_columns_.count(); ++i) {
int64_t idx = sort_columns_.at(i).index_;
cmp = r1.row_->get_cell(idx).compare(r2.row_->get_cell(idx), sort_columns_.at(i).cs_type_);
if (cmp < 0) {
bret = sort_columns_.at(i).is_ascending();
} else if (cmp > 0) {
bret = !sort_columns_.at(i).is_ascending();
} else {
}
} // end for
}
return bret;
}
private:
const common::ObIArray<ObSortColumn>& sort_columns_;
int* err_;
};
static const int64_t INIT_ROW_COUNT = 1 << 7l;
// initial size of hash table for loop search
static const int64_t CTE_SET_NUM = 1 << 5l;
public:
explicit ObSearchMethod(common::ObIAllocator& allocator)
: allocator_(allocator), input_rows_(), sort_columns_(), cycle_by_columns_(), sort_(), op_schema_objs_(nullptr){};
virtual ~ObSearchMethod() = default;
virtual int finish_add_row(bool sort) = 0;
virtual int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) = 0;
virtual int empty() = 0;
virtual int reuse();
virtual int add_cycle_by_column(common::ObColumnInfo col);
int add_row(const common::ObNewRow* row);
int sort_input_rows();
int sort_rownodes(common::ObArray<ObTreeNode>& sort_array);
int add_sort_column(ObSortColumn col);
void set_op_schema_objs(const common::ObIArray<ObOpSchemaObj>& op_schema_objs)
{
op_schema_objs_ = &op_schema_objs;
}
int is_same_row(common::ObNewRow& row_1st, common::ObNewRow& row_2nd, bool& is_cycle);
int64_t count()
{
return input_rows_.count();
}
protected:
common::ObIAllocator& allocator_;
common::ObArray<common::ObNewRow*> input_rows_;
common::ObSEArray<ObSortColumn, 32> sort_columns_;
common::ObSEArray<common::ObColumnInfo, 32> cycle_by_columns_;
common::ObArray<common::ObNewRow*> recycle_rows_;
ObBaseSort sort_;
const common::ObIArray<ObOpSchemaObj>* op_schema_objs_;
};
class ObDepthFisrtSearch : public ObSearchMethod {
typedef common::hash::ObHashSet<ObHashCols, common::hash::NoPthreadDefendMode> RowMap;
public:
ObDepthFisrtSearch(common::ObIAllocator& allocator)
: ObSearchMethod(allocator),
hash_filter_rows_(),
hash_col_idx_(),
last_node_level_(UINT64_MAX),
current_search_path_(),
search_stack_(allocator_)
{}
virtual ~ObDepthFisrtSearch()
{
if (hash_filter_rows_.created()) {
hash_filter_rows_.destroy();
}
}
virtual int finish_add_row(bool sort) override;
virtual int reuse() override;
virtual int empty() override
{
return search_stack_.empty() && input_rows_.empty();
}
virtual int add_cycle_by_column(common::ObColumnInfo col) override;
int init()
{
return hash_filter_rows_.create(CTE_SET_NUM);
}
int adjust_stack(ObTreeNode& node);
int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) override;
private:
int is_depth_cycle_node(ObTreeNode& node);
private:
RowMap hash_filter_rows_;
common::ObSEArray<common::ObColumnInfo, 32> hash_col_idx_;
// record level of current row in the tree.
uint64_t last_node_level_;
common::ObArray<common::ObNewRow*> current_search_path_;
common::ObList<ObTreeNode, common::ObIAllocator> search_stack_;
};
class ObBreadthFisrtSearch : public ObSearchMethod {
public:
ObBreadthFisrtSearch(common::ObIAllocator& allocator)
: ObSearchMethod(allocator),
bst_root_(),
current_parent_node_(&bst_root_),
search_queue_(allocator),
search_results_()
{}
virtual ~ObBreadthFisrtSearch() = default;
virtual int finish_add_row(bool sort) override;
virtual int reuse() override;
virtual int empty() override
{
return input_rows_.empty() && search_queue_.empty() && search_results_.empty();
}
int add_result_rows();
int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) override;
int update_parent_node(ObTreeNode& node);
private:
int init_new_nodes(ObBFSTreeNode* last_bstnode, int64_t child_num);
int is_breadth_cycle_node(ObTreeNode& node);
int add_new_level();
private:
ObBFSTreeNode bst_root_;
/**
* A
* AA AB
* AAA AAB ABA ABB
* when current_parent_node_ is AA:
* search_queue_ contains AA and AB
* search_results_ contains AAA and AAB
*/
ObBFSTreeNode* current_parent_node_;
common::ObList<ObTreeNode, common::ObIAllocator> search_queue_;
common::ObArray<ObTreeNode> search_results_;
};
} // namespace sql
} // namespace oceanbase
#endif

View File

@ -0,0 +1,485 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/ob_operator.h"
#include "ob_search_method_op.h"
using namespace oceanbase::sql;
using namespace oceanbase::common;
int ObSearchMethodOp::reuse()
{
input_rows_.reuse();
return OB_SUCCESS;
}
uint64_t ObSearchMethodOp::ObCycleHash::inner_hash() const
{
uint64_t result = 99194853094755497L;
if (OB_ISNULL(hash_col_idx_) || OB_ISNULL(exprs_) || OB_ISNULL(row_)) {
} else {
int64_t col_count = hash_col_idx_->count();
ObExpr* expr = NULL;
const ObDatum* datum = NULL;
uint64_t idx = 0;
for (int64_t i = 0; i < col_count; i++) {
idx = hash_col_idx_->at(i);
if (OB_UNLIKELY(idx >= exprs_->count()) || OB_ISNULL(expr = exprs_->at(idx)) || OB_ISNULL(expr->basic_funcs_)) {
} else {
datum = &row_->cells()[idx];
result = expr->basic_funcs_->wy_hash_(*datum, result);
}
}
}
return result;
}
bool ObSearchMethodOp::ObCycleHash::operator==(const ObCycleHash& other) const
{
bool result = true;
if (OB_ISNULL(hash_col_idx_) || OB_ISNULL(row_) || OB_ISNULL(exprs_) || OB_ISNULL(other.row_)) {
result = false;
} else {
const ObDatum* lcell = row_->cells();
const ObDatum* rcell = other.row_->cells();
int64_t col_count = hash_col_idx_->count();
ObExpr* expr = NULL;
for (int64_t i = 0; result && i < col_count; i++) {
int64_t idx = hash_col_idx_->at(i);
if (OB_UNLIKELY(idx >= exprs_->count()) || OB_ISNULL(expr = exprs_->at(idx)) || OB_ISNULL(expr->basic_funcs_)) {
} else {
result = (0 == expr->basic_funcs_->null_first_cmp_(lcell[idx], rcell[idx]));
}
}
}
return result;
}
int ObSearchMethodOp::add_row(const ObIArray<ObExpr*>& exprs, ObEvalCtx& eval_ctx)
{
int ret = OB_SUCCESS;
ObChunkDatumStore::LastStoredRow<> last_row(allocator_);
if (input_rows_.empty() && 0 == input_rows_.get_capacity() && OB_FAIL(input_rows_.reserve(INIT_ROW_COUNT))) {
LOG_WARN("Failed to pre allocate array", K(ret));
} else if (OB_UNLIKELY(exprs.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("exprs empty", K(ret));
} else if (OB_FAIL(last_row.save_store_row(exprs, eval_ctx, ROW_EXTRA_SIZE))) {
LOG_WARN("save store row failed", K(ret));
} else if (OB_ISNULL(last_row.store_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("stored_row of last_stored_row is null", K(ret));
} else if (OB_FAIL(input_rows_.push_back(last_row.store_row_))) {
LOG_WARN("Push new row to result input error", K(ret));
} else {
}
return ret;
}
int ObSearchMethodOp::sort_input_rows()
{
int ret = OB_SUCCESS;
// sort
if (input_rows_.count() > 0) {
ObChunkDatumStore::StoredRow** first_row = &input_rows_.at(0);
ObNodeComparer comparer(sort_collations_, left_output_, &ret);
std::sort(first_row, first_row + input_rows_.count(), comparer);
}
return ret;
}
int ObSearchMethodOp::sort_rownodes(ObArray<ObTreeNode>& sort_array)
{
int ret = OB_SUCCESS;
if (!sort_array.empty()) {
LOG_DEBUG("Sort row nodes", K(sort_array.count()));
ObTreeNode* first_row = &sort_array.at(0);
ObNodeComparer comparer(sort_collations_, left_output_, &ret);
std::sort(first_row, first_row + sort_array.count(), comparer);
if (OB_SUCCESS != ret) {
LOG_WARN("Failed to do sort", K(ret));
}
}
return ret;
}
int ObSearchMethodOp::is_same_row(
ObChunkDatumStore::StoredRow& row_1st, ObChunkDatumStore::StoredRow& row_2nd, bool& is_cycle)
{
int ret = OB_SUCCESS;
const ObDatum* cells_1st = row_1st.cells();
const ObDatum* cells_2nd = row_2nd.cells();
if (OB_UNLIKELY(row_1st.cnt_ != row_2nd.cnt_ || 0 == row_1st.cnt_) || OB_ISNULL(cells_1st) || OB_ISNULL(cells_2nd)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Inconformity row schema", K(ret), K(row_1st), K(row_2nd));
} else if (cycle_by_columns_.empty()) {
// detect whole row
is_cycle = true;
for (int64_t i = 0; i < left_output_.count(); i++) {
if (0 != left_output_.at(i)->basic_funcs_->null_first_cmp_(cells_1st[i], cells_2nd[i])) {
is_cycle = false;
break;
}
}
if (is_cycle) {
ret = OB_ERR_CYCLE_FOUND_IN_RECURSIVE_CTE;
LOG_WARN("Cycle detected while executing recursive WITH query", K(ret));
}
} else {
// detect some datum
is_cycle = true;
for (int64_t i = 0; OB_SUCC(ret) && i < cycle_by_columns_.count(); ++i) {
uint64_t index = cycle_by_columns_.at(i);
if (index >= row_1st.cnt_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Column index out of range", K(ret));
} else if (0 != left_output_.at(index)->basic_funcs_->null_first_cmp_(cells_1st[index], cells_2nd[index])) {
is_cycle = false;
break;
}
} // end for
}
return ret;
}
int ObDepthFisrtSearchOp::init()
{
int ret = OB_SUCCESS;
if (OB_FAIL(hash_filter_rows_.create(CTE_SET_NUM))) {
LOG_WARN("row map init failed", K(ret));
} else {
if (cycle_by_columns_.empty()) {
for (int64_t i = 0; OB_SUCC(ret) && i < left_output_.count(); ++i) {
if (OB_FAIL(hash_col_idx_.push_back(i))) {
LOG_WARN("push back failed");
}
}
} else if (OB_FAIL(hash_col_idx_.assign(cycle_by_columns_))) {
LOG_WARN("assign hash col idx failed", K(ret));
}
}
return ret;
}
int ObDepthFisrtSearchOp::reuse()
{
ObSearchMethodOp::reuse();
last_node_level_ = UINT64_MAX;
hash_filter_rows_.reuse();
return OB_SUCCESS;
}
int ObDepthFisrtSearchOp::finish_add_row(bool sort)
{
int ret = OB_SUCCESS;
if (sort && OB_FAIL(sort_input_rows())) {
LOG_WARN("Sort input rows failed", K(ret));
} else if (input_rows_.empty()) {
if (current_search_path_.empty()) {
// do nothing
} else {
int64_t count = current_search_path_.count();
ObChunkDatumStore::StoredRow* tmp_row = current_search_path_.at(count - 1);
ObCycleHash cycle_hash(tmp_row, &hash_col_idx_, &left_output_);
if (OB_FAIL(hash_filter_rows_.erase_refactored(cycle_hash))) {
LOG_WARN("Earse rows from the hash map failed", K(ret));
} else if (OB_FAIL(recycle_rows_.push_back(tmp_row))) {
LOG_WARN("Failed to push back rows", K(ret));
} else {
current_search_path_.pop_back();
}
}
} else {
for (int64_t i = input_rows_.count() - 1; OB_SUCC(ret) && i >= 0; i--) {
ObTreeNode node;
node.stored_row_ = input_rows_.at(i);
node.tree_level_ = (UINT64_MAX == last_node_level_) ? 0 : last_node_level_ + 1;
if (OB_FAIL(is_depth_cycle_node(node))) {
LOG_WARN("Check cycle node failed", K(ret));
} else if (OB_FAIL(search_stack_.push_front(node))) {
LOG_WARN("Push data to result hold stack failed", K(ret));
} else {
}
} // end for
}
input_rows_.reuse();
if (OB_SUCC(ret) && !recycle_rows_.empty()) {
ARRAY_FOREACH(recycle_rows_, i)
{
allocator_.free(recycle_rows_.at(i));
}
recycle_rows_.reuse();
}
return ret;
}
int ObDepthFisrtSearchOp::is_depth_cycle_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObCycleHash cycle_hash(node.stored_row_, &hash_col_idx_, &left_output_);
if (OB_FAIL(hash_filter_rows_.exist_refactored(cycle_hash))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else if (OB_HASH_EXIST == ret) {
ret = OB_SUCCESS;
node.is_cycle_ = true;
if (cycle_by_columns_.empty()) {
ret = OB_ERR_CYCLE_FOUND_IN_RECURSIVE_CTE;
LOG_WARN("Cycle detected while executing recursive WITH query", K(ret));
}
} else {
LOG_WARN("Failed to find in hashmap", K(ret));
}
} else {
// succ
}
return ret;
}
int ObDepthFisrtSearchOp::adjust_stack(ObTreeNode& node)
{
int ret = OB_SUCCESS;
last_node_level_ = node.tree_level_;
if (current_search_path_.count() < node.tree_level_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The current node level is 2 or more large than last node level", K(ret));
} else if (current_search_path_.count() > node.tree_level_) {
/**
* Tree of all rows is:
* A
* AA AB
* AAA AAB ABA ABB
* current_search_path_ is A->AA->AAB
* when current node goes to AB, path will be A->AB
*/
int64_t count = current_search_path_.count();
int64_t pop_times = count - node.tree_level_;
ObChunkDatumStore::StoredRow* tmp_row = nullptr;
while (OB_SUCC(ret) && pop_times > 0) {
--pop_times;
if (OB_FAIL(current_search_path_.pop_back(tmp_row))) {
LOG_WARN("Failed to pop stack", K(ret));
} else {
ObCycleHash cycle_hash(tmp_row, &hash_col_idx_, &left_output_);
if (OB_FAIL(hash_filter_rows_.erase_refactored(cycle_hash))) {
LOG_WARN("Earse rows from the hash map failed", K(ret));
} else if (OB_FAIL(recycle_rows_.push_back(tmp_row))) {
LOG_WARN("Failed to push back rows", K(ret));
}
}
}
}
// add stack
if (OB_SUCC(ret) && current_search_path_.count() == node.tree_level_) {
ObCycleHash cycle_hash(node.stored_row_, &hash_col_idx_, &left_output_);
if (OB_FAIL(current_search_path_.push_back(node.stored_row_))) {
LOG_WARN("Push new row to result record failed", K(ret));
} else if (OB_FAIL(hash_filter_rows_.set_refactored(cycle_hash))) {
LOG_WARN("Failed to insert row to hashmap", K(ret));
}
}
return ret;
}
int ObDepthFisrtSearchOp::get_next_non_cycle_node(
ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObTreeNode non_cycle_node;
bool got_row = false;
while (OB_SUCC(ret) && !search_stack_.empty()) {
if (OB_FAIL(search_stack_.pop_front(non_cycle_node))) {
LOG_WARN("Search stack pop back failed", K(ret));
} else if (OB_FAIL(result_output.push_back(non_cycle_node))) {
LOG_WARN("Push back data to result output failed", K(ret));
} else if (non_cycle_node.is_cycle_) {
if (OB_FAIL(recycle_rows_.push_back(non_cycle_node.stored_row_))) {
LOG_WARN("Failed to push back rows", K(ret));
}
} else {
got_row = true;
node = non_cycle_node;
break;
}
} // end while
if (OB_SUCC(ret) && !got_row) {
ret = OB_ITER_END;
}
return ret;
}
int ObBreadthFisrtSearchOp::add_new_level()
{
int ret = OB_SUCCESS;
ObTreeNode new_level_node;
new_level_node.stored_row_ = nullptr;
if (OB_FAIL(search_queue_.push_back(new_level_node))) {
LOG_WARN("Push back data to result hold queue failed", K(ret));
}
return ret;
}
int ObBreadthFisrtSearchOp::reuse()
{
ObSearchMethodOp::reuse();
current_parent_node_ = &bst_root_;
bst_root_.child_num_ = 0;
bst_root_.children_ = nullptr;
bst_root_.parent_ = nullptr;
bst_root_.stored_row_ = nullptr;
search_queue_.reset();
search_results_.reuse();
return OB_SUCCESS;
}
int ObBreadthFisrtSearchOp::init_new_nodes(ObBFSTreeNode* last_bstnode, int64_t child_num)
{
int ret = OB_SUCCESS;
void* childs_ptr = nullptr;
if (OB_UNLIKELY(0 == child_num)) {
// do nothing
} else if (OB_ISNULL(last_bstnode)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Last bst node can not be null", K(ret));
} else if (OB_UNLIKELY(last_bstnode->child_num_ != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Last bst node can not be ini twice", K(ret), KPC(last_bstnode));
} else if (OB_ISNULL(childs_ptr = allocator_.alloc(sizeof(ObBFSTreeNode*) * child_num))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("Alloc memory for row failed", "size", child_num * sizeof(ObBFSTreeNode*), K(ret));
} else {
last_bstnode->children_ = (ObBFSTreeNode**)childs_ptr;
last_bstnode->child_num_ = child_num;
}
return ret;
}
int ObBreadthFisrtSearchOp::is_breadth_cycle_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObBFSTreeNode* tmp = current_parent_node_;
ObChunkDatumStore::StoredRow* row = node.stored_row_;
if (OB_ISNULL(tmp) || OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The last_bstnode and row an not be null", K(ret), KPC(row));
} else {
// bst_root_ row_ is null
while (OB_SUCC(ret) && OB_NOT_NULL(tmp) && OB_NOT_NULL(tmp->stored_row_)) {
ObChunkDatumStore::StoredRow* row_1st = row;
ObChunkDatumStore::StoredRow* row_2nd = tmp->stored_row_;
// NOTE: cycle check cost lost of time via bianque perf diagnosis
if (OB_FAIL(is_same_row(*row_1st, *row_2nd, node.is_cycle_))) {
LOG_WARN("Failed to compare the two row", K(ret), KPC(row_1st), KPC(row_2nd));
} else if (node.is_cycle_) {
break;
} else {
tmp = tmp->parent_;
}
}
}
return ret;
}
int ObBreadthFisrtSearchOp::get_next_non_cycle_node(
ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node)
{
int ret = OB_SUCCESS;
ObTreeNode non_cycle_node;
bool got_row = false;
while (OB_SUCC(ret) && !search_queue_.empty()) {
if (OB_FAIL(search_queue_.pop_front(non_cycle_node))) {
LOG_WARN("Get row from hold queue failed", K(ret));
} else if (OB_FAIL(result_output.push_back(non_cycle_node))) {
LOG_WARN("Failed to push row to output ", K(ret));
} else if (non_cycle_node.is_cycle_) {
if (OB_FAIL(recycle_rows_.push_back(non_cycle_node.stored_row_))) {
LOG_WARN("Failed to push back rows", K(ret));
}
} else {
got_row = true;
node = non_cycle_node;
break;
}
} // end while
if (OB_SUCC(ret) && !got_row) {
ret = OB_ITER_END;
}
return ret;
}
int ObBreadthFisrtSearchOp::update_parent_node(ObTreeNode& node)
{
int ret = OB_SUCCESS;
current_parent_node_ = node.in_bstree_node_;
return ret;
}
int ObBreadthFisrtSearchOp::add_result_rows()
{
int ret = OB_SUCCESS;
if (OB_FAIL(init_new_nodes(current_parent_node_, input_rows_.count()))) {
LOG_WARN("Failed to init new bst node", K(ret));
} else {
ARRAY_FOREACH(input_rows_, i)
{
void* ptr = nullptr;
ObBFSTreeNode* tmp = nullptr;
ObTreeNode node;
node.stored_row_ = input_rows_.at(i);
// breadth search tree
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObBFSTreeNode)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("Alloc memory for row failed", "size", sizeof(ObBFSTreeNode), K(ret));
} else {
tmp = new (ptr) ObBFSTreeNode();
tmp->stored_row_ = input_rows_.at(i);
tmp->parent_ = current_parent_node_;
current_parent_node_->children_[i] = tmp;
node.in_bstree_node_ = tmp;
if (OB_FAIL(is_breadth_cycle_node(node))) {
LOG_WARN("Find cycle failed", K(ret));
} else if (OB_FAIL(search_results_.push_back(node))) {
LOG_WARN("Push back data to layer_results failed", K(ret));
} else {
LOG_DEBUG("Result node", K(node));
}
}
}
}
input_rows_.reuse();
return ret;
}
int ObBreadthFisrtSearchOp::finish_add_row(bool sort)
{
int ret = OB_SUCCESS;
if (sort && OB_FAIL(sort_rownodes(search_results_))) {
LOG_WARN("Failed to sort results", K(ret));
} else if (!search_queue_.empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("The last result still has residual", K(search_queue_), K(ret));
} else {
ARRAY_FOREACH(search_results_, i)
{
if (OB_FAIL(search_queue_.push_back(search_results_.at(i)))) {
LOG_WARN("Push back failed", K(ret));
}
}
}
search_results_.reuse();
return ret;
}

View File

@ -0,0 +1,268 @@
/**
* 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_DEPTH_FIRST_SEARCH_OP_H_
#define OB_DEPTH_FIRST_SEARCH_OP_H_
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "lib/allocator/ob_malloc.h"
#include "lib/list/ob_list.h"
#include "share/datum/ob_datum_funcs.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
#include "sql/engine/sort/ob_sort_basic_info.h"
namespace oceanbase {
namespace sql {
class ObSearchMethodOp {
public:
typedef struct _BreadthFirstSearchTreeNode {
_BreadthFirstSearchTreeNode() : child_num_(0), stored_row_(nullptr), children_(nullptr), parent_(nullptr)
{}
int64_t child_num_;
ObChunkDatumStore::StoredRow* stored_row_;
struct _BreadthFirstSearchTreeNode** children_;
struct _BreadthFirstSearchTreeNode* parent_;
TO_STRING_KV("row ", stored_row_, "child_num_", child_num_);
} ObBFSTreeNode;
typedef struct _TreeNode {
_TreeNode() : is_cycle_(false), tree_level_(0), stored_row_(nullptr), in_bstree_node_(nullptr)
{}
bool is_cycle_;
uint64_t tree_level_;
ObChunkDatumStore::StoredRow* stored_row_;
ObBFSTreeNode* in_bstree_node_;
TO_STRING_KV("is tree level", tree_level_, "is cycle", is_cycle_, "row", stored_row_);
} ObTreeNode;
struct ObNodeComparer {
explicit ObNodeComparer(
const common::ObIArray<ObSortFieldCollation>& sort_collations, const common::ObIArray<ObExpr*>& exprs, int* err)
: sort_collations_(sort_collations), exprs_(exprs), err_(err)
{}
bool operator()(const ObTreeNode& r1, const ObTreeNode& r2)
{
bool bret = false;
const ObChunkDatumStore::StoredRow* l = r1.stored_row_;
const ObChunkDatumStore::StoredRow* r = r2.stored_row_;
return cmp_stored_row(l, r);
}
bool operator()(const ObChunkDatumStore::StoredRow* l, const ObChunkDatumStore::StoredRow* r)
{
return cmp_stored_row(l, r);
}
inline bool cmp_stored_row(const ObChunkDatumStore::StoredRow* l, const ObChunkDatumStore::StoredRow* r)
{
bool bret = false;
if (OB_UNLIKELY(common::OB_SUCCESS != *err_)) {
// do nothing if we already have an error,
// so we can finish the sort process ASAP.
} else if (OB_ISNULL(l) || OB_ISNULL(r) || (l->cnt_ != r->cnt_)) {
*err_ = OB_ERR_UNEXPECTED;
SQL_LOG(WARN, "invalid parameter", KPC(l), KPC(r), K(*err_));
} else {
const ObDatum* lcells = l->cells();
const ObDatum* rcells = r->cells();
int cmp = 0;
for (int64_t i = 0; OB_SUCCESS == *err_ && 0 == cmp && i < sort_collations_.count(); i++) {
const int64_t idx = sort_collations_.at(i).field_idx_;
if (idx >= exprs_.count()) {
*err_ = OB_ERR_UNEXPECTED;
SQL_LOG(WARN, "compare column id greater than exprs count", K(*err_), K(idx), K(exprs_.count()));
} else {
bool null_first = (NULL_FIRST == sort_collations_.at(i).null_pos_);
ObExprCmpFuncType cmp_func = null_first ? exprs_.at(idx)->basic_funcs_->null_first_cmp_
: exprs_.at(idx)->basic_funcs_->null_last_cmp_;
cmp = cmp_func(lcells[idx], rcells[idx]);
if (cmp < 0) {
bret = sort_collations_.at(i).is_ascending_;
} else if (cmp > 0) {
bret = !sort_collations_.at(i).is_ascending_;
}
}
}
}
return bret;
}
private:
const common::ObIArray<ObSortFieldCollation>& sort_collations_;
const common::ObIArray<ObExpr*>& exprs_;
int* err_;
};
class ObCycleHash {
public:
ObCycleHash() : row_(NULL), hash_col_idx_(NULL), exprs_(NULL), hash_val_(0)
{}
ObCycleHash(const ObChunkDatumStore::StoredRow* row, const common::ObIArray<uint64_t>* hash_col_idx,
const common::ObIArray<ObExpr*>* exprs)
: row_(row), hash_col_idx_(hash_col_idx), exprs_(exprs), hash_val_(0)
{}
~ObCycleHash()
{}
uint64_t hash() const
{
if (hash_val_ == 0) {
hash_val_ = inner_hash();
}
return hash_val_;
}
uint64_t inner_hash() const;
bool operator==(const ObCycleHash& other) const;
public:
const ObChunkDatumStore::StoredRow* row_;
const common::ObIArray<uint64_t>* hash_col_idx_;
const common::ObIArray<ObExpr*>* exprs_;
mutable uint64_t hash_val_;
};
static const int64_t INIT_ROW_COUNT = 1 << 7l;
// initial size of hash table for loop search
static const int64_t CTE_SET_NUM = 1 << 5l;
public:
explicit ObSearchMethodOp(common::ObIAllocator& allocator, const ExprFixedArray& left_output,
const common::ObIArray<ObSortFieldCollation>& sort_collations, const common::ObIArray<uint64_t>& cycle_by_columns)
: allocator_(allocator),
input_rows_(),
sort_collations_(sort_collations),
cycle_by_columns_(cycle_by_columns),
left_output_(left_output){};
virtual ~ObSearchMethodOp() = default;
virtual int finish_add_row(bool sort) = 0;
virtual int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) = 0;
virtual int empty() = 0;
virtual int reuse();
int add_row(const ObIArray<ObExpr*>& exprs, ObEvalCtx& eval_ctx);
int sort_input_rows();
int sort_rownodes(common::ObArray<ObTreeNode>& sort_array);
int is_same_row(ObChunkDatumStore::StoredRow& row_1st, ObChunkDatumStore::StoredRow& row_2nd, bool& is_cycle);
int64_t count()
{
return input_rows_.count();
}
const static int64_t ROW_EXTRA_SIZE = 0;
protected:
// hard code seed, 24bit max prime number
static const int64_t HASH_SEED = 16777213;
common::ObIAllocator& allocator_;
common::ObArray<ObChunkDatumStore::StoredRow*> input_rows_;
const common::ObIArray<ObSortFieldCollation>& sort_collations_;
const common::ObIArray<uint64_t>& cycle_by_columns_;
common::ObArray<ObChunkDatumStore::StoredRow*> recycle_rows_;
const ExprFixedArray& left_output_;
};
class ObDepthFisrtSearchOp : public ObSearchMethodOp {
typedef common::hash::ObHashSet<ObCycleHash, common::hash::NoPthreadDefendMode> RowMap;
public:
ObDepthFisrtSearchOp(common::ObIAllocator& allocator, const ExprFixedArray& left_output,
const common::ObIArray<ObSortFieldCollation>& sort_collations, const common::ObIArray<uint64_t>& cycle_by_columns)
: ObSearchMethodOp(allocator, left_output, sort_collations, cycle_by_columns),
hash_filter_rows_(),
hash_col_idx_(),
last_node_level_(UINT64_MAX),
current_search_path_(),
search_stack_(allocator_)
{}
virtual ~ObDepthFisrtSearchOp()
{
if (hash_filter_rows_.created()) {
hash_filter_rows_.destroy();
}
}
virtual int finish_add_row(bool sort) override;
virtual int reuse() override;
virtual int empty() override
{
return search_stack_.empty() && input_rows_.empty();
}
int init();
int adjust_stack(ObTreeNode& node);
int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) override;
private:
int is_depth_cycle_node(ObTreeNode& node);
private:
RowMap hash_filter_rows_;
common::ObSEArray<uint64_t, 32> hash_col_idx_;
// record level of current row in the tree.
uint64_t last_node_level_;
common::ObArray<ObChunkDatumStore::StoredRow*> current_search_path_;
common::ObList<ObTreeNode, common::ObIAllocator> search_stack_;
};
class ObBreadthFisrtSearchOp : public ObSearchMethodOp {
public:
ObBreadthFisrtSearchOp(common::ObIAllocator& allocator, const ExprFixedArray& left_output,
const common::ObIArray<ObSortFieldCollation>& sort_collations, const common::ObIArray<uint64_t>& cycle_by_columns)
: ObSearchMethodOp(allocator, left_output, sort_collations, cycle_by_columns),
bst_root_(),
current_parent_node_(&bst_root_),
search_queue_(allocator),
search_results_()
{}
virtual ~ObBreadthFisrtSearchOp() = default;
virtual int finish_add_row(bool sort) override;
virtual int reuse() override;
virtual int empty() override
{
return input_rows_.empty() && search_queue_.empty() && search_results_.empty();
}
int add_result_rows();
int get_next_non_cycle_node(
common::ObList<ObTreeNode, common::ObIAllocator>& result_output, ObTreeNode& node) override;
int update_parent_node(ObTreeNode& node);
private:
int init_new_nodes(ObBFSTreeNode* last_bstnode, int64_t child_num);
int is_breadth_cycle_node(ObTreeNode& node);
int add_new_level();
private:
ObBFSTreeNode bst_root_;
/**
* A
* AA AB
* AAA AAB ABA ABB
* when current_parent_node_ is AA:
* search_queue_ contains AA and AB
* search_results_ contains AAA and AAB
*/
ObBFSTreeNode* current_parent_node_;
common::ObList<ObTreeNode, common::ObIAllocator> search_queue_;
common::ObArray<ObTreeNode> search_results_;
};
} // namespace sql
} // namespace oceanbase
#endif