init push
This commit is contained in:
225
src/sql/engine/recursive_cte/ob_fake_cte_table.cpp
Normal file
225
src/sql/engine/recursive_cte/ob_fake_cte_table.cpp
Normal 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
|
||||
90
src/sql/engine/recursive_cte/ob_fake_cte_table.h
Normal file
90
src/sql/engine/recursive_cte/ob_fake_cte_table.h
Normal 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 */
|
||||
178
src/sql/engine/recursive_cte/ob_fake_cte_table_op.cpp
Normal file
178
src/sql/engine/recursive_cte/ob_fake_cte_table_op.cpp
Normal 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
|
||||
82
src/sql/engine/recursive_cte/ob_fake_cte_table_op.h
Normal file
82
src/sql/engine/recursive_cte/ob_fake_cte_table_op.h
Normal 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 */
|
||||
461
src/sql/engine/recursive_cte/ob_recursive_inner_data.cpp
Normal file
461
src/sql/engine/recursive_cte/ob_recursive_inner_data.cpp
Normal 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
|
||||
145
src/sql/engine/recursive_cte/ob_recursive_inner_data.h
Normal file
145
src/sql/engine/recursive_cte/ob_recursive_inner_data.h
Normal 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
|
||||
391
src/sql/engine/recursive_cte/ob_recursive_inner_data_op.cpp
Normal file
391
src/sql/engine/recursive_cte/ob_recursive_inner_data_op.cpp
Normal 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
|
||||
157
src/sql/engine/recursive_cte/ob_recursive_inner_data_op.h
Normal file
157
src/sql/engine/recursive_cte/ob_recursive_inner_data_op.h
Normal 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
|
||||
242
src/sql/engine/recursive_cte/ob_recursive_union_all.cpp
Normal file
242
src/sql/engine/recursive_cte/ob_recursive_union_all.cpp
Normal 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
|
||||
135
src/sql/engine/recursive_cte/ob_recursive_union_all.h
Normal file
135
src/sql/engine/recursive_cte/ob_recursive_union_all.h
Normal 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_ */
|
||||
132
src/sql/engine/recursive_cte/ob_recursive_union_all_op.cpp
Normal file
132
src/sql/engine/recursive_cte/ob_recursive_union_all_op.cpp
Normal 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
|
||||
116
src/sql/engine/recursive_cte/ob_recursive_union_all_op.h
Normal file
116
src/sql/engine/recursive_cte/ob_recursive_union_all_op.h
Normal 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_ */
|
||||
476
src/sql/engine/recursive_cte/ob_search_method.cpp
Normal file
476
src/sql/engine/recursive_cte/ob_search_method.cpp
Normal 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;
|
||||
}
|
||||
211
src/sql/engine/recursive_cte/ob_search_method.h
Normal file
211
src/sql/engine/recursive_cte/ob_search_method.h
Normal 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
|
||||
485
src/sql/engine/recursive_cte/ob_search_method_op.cpp
Normal file
485
src/sql/engine/recursive_cte/ob_search_method_op.cpp
Normal 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;
|
||||
}
|
||||
268
src/sql/engine/recursive_cte/ob_search_method_op.h
Normal file
268
src/sql/engine/recursive_cte/ob_search_method_op.h
Normal 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
|
||||
Reference in New Issue
Block a user