init push
This commit is contained in:
123
src/sql/engine/set/ob_append.cpp
Normal file
123
src/sql/engine/set/ob_append.cpp
Normal file
@ -0,0 +1,123 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_append.h"
|
||||
#include "common/object/ob_object.h"
|
||||
#include "sql/engine/expr/ob_expr_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
class ObAppend::ObAppendCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObAppendCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), current_child_op_idx_(0)
|
||||
{}
|
||||
~ObAppendCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
public:
|
||||
int64_t current_child_op_idx_;
|
||||
};
|
||||
|
||||
ObAppend::ObAppend(common::ObIAllocator& alloc) : ObMultiChildrenPhyOperator(alloc)
|
||||
{}
|
||||
|
||||
ObAppend::~ObAppend()
|
||||
{}
|
||||
|
||||
void ObAppend::reset()
|
||||
{
|
||||
ObMultiChildrenPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObAppend::reuse()
|
||||
{
|
||||
ObMultiChildrenPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObAppend::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAppendCtx* append_ctx = NULL;
|
||||
if (OB_ISNULL(append_ctx = GET_PHY_OPERATOR_CTX(ObAppendCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_NULL_VALUE;
|
||||
} else if (append_ctx->current_child_op_idx_ >= child_num_ || append_ctx->current_child_op_idx_ < 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected error. current child op idx is out of range", K(ret), K(append_ctx->current_child_op_idx_));
|
||||
} else if (OB_ISNULL(child_array_[append_ctx->current_child_op_idx_])) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("child_op_ is null");
|
||||
} else if (child_array_[append_ctx->current_child_op_idx_]->is_dml_operator()) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
ret = child_array_[append_ctx->current_child_op_idx_]->get_next_row(ctx, row);
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAppend::get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAppendCtx* append_ctx = NULL;
|
||||
if (OB_ISNULL(append_ctx = GET_PHY_OPERATOR_CTX(ObAppendCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_NULL_VALUE;
|
||||
} else {
|
||||
while (true) {
|
||||
ret = inner_get_next_row(ctx, row);
|
||||
if (OB_SUCC(ret)) {
|
||||
break;
|
||||
} else if (OB_ITER_END == ret) {
|
||||
if (append_ctx->current_child_op_idx_ < child_num_ - 1) {
|
||||
++append_ctx->current_child_op_idx_;
|
||||
// go on iterating
|
||||
} else {
|
||||
// iterate ended. really.
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("inner get next row failed", K(ret));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAppend::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
UNUSED(op_ctx);
|
||||
return CREATE_PHY_OPERATOR_CTX(ObAppendCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
|
||||
int ObAppend::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("init operator context failed", K(ret), K_(id));
|
||||
} else if (OB_FAIL(handle_op_ctx(ctx))) {
|
||||
LOG_WARN("handle op ctx failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
55
src/sql/engine/set/ob_append.h
Normal file
55
src/sql/engine/set/ob_append.h
Normal file
@ -0,0 +1,55 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_OB_APPEND_H_
|
||||
#define OCEANBASE_SQL_OB_APPEND_H_
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_multi_children_phy_operator.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObAppend : public ObMultiChildrenPhyOperator {
|
||||
public:
|
||||
class ObAppendCtx;
|
||||
explicit ObAppend(common::ObIAllocator& alloc);
|
||||
~ObAppend();
|
||||
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
int inner_open(ObExecContext& ctx) 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 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;
|
||||
/**
|
||||
* @note not need to iterator a row to parent operator, so forbid this function
|
||||
*/
|
||||
int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObAppend);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_OB_APPEND_H_ */
|
||||
82
src/sql/engine/set/ob_append_op.cpp
Normal file
82
src/sql/engine/set/ob_append_op.cpp
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.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_append_op.h"
|
||||
#include "common/object/ob_object.h"
|
||||
#include "sql/engine/expr/ob_expr_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
OB_SERIALIZE_MEMBER((ObAppendSpec, ObOpSpec));
|
||||
|
||||
int ObAppendOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (current_child_op_idx_ >= MY_SPEC.get_child_cnt() || current_child_op_idx_ < 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected error. current child op idx is out of range", K(ret), K(current_child_op_idx_));
|
||||
} else if (OB_ISNULL(MY_SPEC.get_child(current_child_op_idx_))) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("child_op_ is null", K(ret));
|
||||
} else if (MY_SPEC.get_child(current_child_op_idx_)->is_dml_operator()) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
const ObOpSpec* spec = MY_SPEC.get_child(current_child_op_idx_);
|
||||
ObOperatorKit* kit = ctx_.get_operator_kit(spec->id_);
|
||||
if (OB_ISNULL(kit) || OB_ISNULL(kit->op_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(*spec));
|
||||
} else {
|
||||
ret = kit->op_->get_next_row();
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAppendOp::get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
while (true) {
|
||||
ret = inner_get_next_row();
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("append op output", "output", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_));
|
||||
break;
|
||||
} else if (OB_ITER_END == ret) {
|
||||
if (current_child_op_idx_ < MY_SPEC.get_child_cnt() - 1) {
|
||||
++current_child_op_idx_;
|
||||
// go on iterating
|
||||
} else {
|
||||
// iterate ended. really.
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("inner get next row failed", K(ret));
|
||||
break;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAppendOp::inner_open()
|
||||
{
|
||||
// do nothing
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
70
src/sql/engine/set/ob_append_op.h
Normal file
70
src/sql/engine/set/ob_append_op.h
Normal file
@ -0,0 +1,70 @@
|
||||
/**
|
||||
* 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_APPEND_OP_H_
|
||||
#define OCEANBASE_SQL_OB_APPEND_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObAppendSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION(1);
|
||||
|
||||
public:
|
||||
ObAppendSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type)
|
||||
{}
|
||||
virtual ~ObAppendSpec(){};
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObAppendSpec);
|
||||
};
|
||||
|
||||
class ObAppendOp : public ObOperator {
|
||||
public:
|
||||
ObAppendOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input), current_child_op_idx_(0)
|
||||
{}
|
||||
~ObAppendOp()
|
||||
{
|
||||
destroy();
|
||||
}
|
||||
|
||||
int inner_open() override;
|
||||
virtual void destroy()
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
/**
|
||||
* @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() override;
|
||||
/**
|
||||
* @note not need to iterator a row to parent operator, so forbid this function
|
||||
*/
|
||||
int get_next_row();
|
||||
|
||||
public:
|
||||
int64_t current_child_op_idx_;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObAppendOp);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_OB_APPEND_H_ */
|
||||
197
src/sql/engine/set/ob_hash_except.cpp
Normal file
197
src/sql/engine/set/ob_hash_except.cpp
Normal file
@ -0,0 +1,197 @@
|
||||
/**
|
||||
* 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_hash_except.h"
|
||||
#include "lib/utility/utility.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
|
||||
namespace sql {
|
||||
|
||||
class ObHashExcept::ObHashExceptCtx : public ObHashSetOperatorCtx {
|
||||
explicit ObHashExceptCtx(ObExecContext& ctx) : ObHashSetOperatorCtx(ctx)
|
||||
{}
|
||||
|
||||
~ObHashExceptCtx()
|
||||
{}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashExceptCtx);
|
||||
friend class ObHashExcept;
|
||||
};
|
||||
|
||||
ObHashExcept::ObHashExcept(common::ObIAllocator& alloc) : ObHashSetOperator(alloc)
|
||||
{}
|
||||
|
||||
ObHashExcept::~ObHashExcept()
|
||||
{}
|
||||
|
||||
int ObHashExcept::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObHashExceptCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
|
||||
int ObHashExcept::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
return ObHashSetOperator::inner_open(ctx);
|
||||
}
|
||||
|
||||
int ObHashExcept::build_hash_table_by_part(ObExecContext& ctx, ObHashExceptCtx* except_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool found = false;
|
||||
while (OB_SUCC(ret) && !found) {
|
||||
if (OB_FAIL(except_ctx->hp_infras_.get_next_pair_partition(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to get next pair partitions", K(ret));
|
||||
} else if (!except_ctx->hp_infras_.has_cur_part(InputSide::RIGHT)) {
|
||||
if (OB_FAIL(except_ctx->hp_infras_.get_next_partition(InputSide::LEFT))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next partition", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.resize(except_ctx->hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
} else {
|
||||
found = true;
|
||||
except_ctx->hp_infras_.switch_left();
|
||||
}
|
||||
} else if (!except_ctx->hp_infras_.has_cur_part(InputSide::LEFT)) {
|
||||
// left: empty right: yes
|
||||
if (OB_FAIL(except_ctx->hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(build_hash_table(ctx, false))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else {
|
||||
found = true;
|
||||
except_ctx->hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashExcept::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashExceptCtx* except_ctx = NULL;
|
||||
const ObNewRow* cur_row = NULL;
|
||||
bool left_has_row = false;
|
||||
bool got_row = false;
|
||||
bool inserted = false;
|
||||
bool exists = false;
|
||||
const HashPartCols* part_cols = nullptr;
|
||||
const ObChunkRowStore::StoredRow* store_row = nullptr;
|
||||
if (OB_ISNULL(except_ctx = GET_PHY_OPERATOR_CTX(ObHashExceptCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else if (except_ctx->first_get_left_) {
|
||||
if (OB_FAIL(except_ctx->is_left_has_row(ctx, left_has_row))) {
|
||||
LOG_WARN("failed to judge left has row", K(ret));
|
||||
} else if (!left_has_row) {
|
||||
ret = OB_ITER_END;
|
||||
except_ctx->iter_end_ = true;
|
||||
} else if (OB_FAIL(ObHashSetOperator::init_hash_partition_infras(ctx))) {
|
||||
LOG_WARN("failed to init hash partition infras", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table(ctx, true))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
except_ctx->hp_infras_.switch_left();
|
||||
}
|
||||
} else if (except_ctx->iter_end_) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!except_ctx->has_got_part_) {
|
||||
if (OB_FAIL(except_ctx->get_left_row(ctx, cur_row))) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get left row", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = except_ctx->hp_infras_.get_left_next_row(store_row, cur_row);
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get next dumped partition
|
||||
if (OB_FAIL(except_ctx->hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!except_ctx->has_got_part_) {
|
||||
except_ctx->has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(except_ctx->hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table_by_part(ctx, except_ctx))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
except_ctx->iter_end_ = true;
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status(ctx))) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.exists_row(cur_row, part_cols))) {
|
||||
LOG_WARN("failed to probe exists row", K(ret));
|
||||
} else if (OB_NOT_NULL(part_cols)) {
|
||||
// exists
|
||||
} else {
|
||||
// row is not exists in hash table
|
||||
if (except_ctx->hp_infras_.has_right_dumped() || except_ctx->hp_infras_.has_left_dumped()) {
|
||||
// dump row
|
||||
if (!except_ctx->hp_infras_.has_left_dumped() &&
|
||||
OB_FAIL(except_ctx->hp_infras_.create_dumped_partitions(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to create dump partitions", K(ret));
|
||||
} else if (OB_FAIL(except_ctx->hp_infras_.insert_row_on_partitions(cur_row))) {
|
||||
LOG_WARN("failed to insert row into partitions", K(ret));
|
||||
}
|
||||
} else {
|
||||
// insert and return row
|
||||
if (OB_FAIL(except_ctx->hp_infras_.insert_row(cur_row, exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
} else if (inserted) {
|
||||
got_row = true;
|
||||
row = cur_row;
|
||||
}
|
||||
}
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(copy_cur_row_by_projector(*except_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashExcept::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOperator::inner_close(ctx))) {
|
||||
LOG_WARN("failed to close", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
43
src/sql/engine/set/ob_hash_except.h
Normal file
43
src/sql/engine/set/ob_hash_except.h
Normal file
@ -0,0 +1,43 @@
|
||||
/**
|
||||
* 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_HASH_EXCEPT_H
|
||||
#define OB_HASH_EXCEPT_H
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObHashExcept : public ObHashSetOperator {
|
||||
private:
|
||||
class ObHashExceptCtx;
|
||||
|
||||
public:
|
||||
explicit ObHashExcept(common::ObIAllocator& alloc);
|
||||
|
||||
virtual ~ObHashExcept();
|
||||
|
||||
private:
|
||||
int build_hash_table_by_part(ObExecContext& ctx, ObHashExceptCtx* except_ctx) const;
|
||||
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
|
||||
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashExcept);
|
||||
};
|
||||
} // namespace sql
|
||||
} // end namespace oceanbase
|
||||
#endif // OB_HASH_EXCEPT_H
|
||||
185
src/sql/engine/set/ob_hash_except_op.cpp
Normal file
185
src/sql/engine/set/ob_hash_except_op.cpp
Normal file
@ -0,0 +1,185 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_hash_except_op.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/basic/ob_hash_partitioning_infrastructure_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObHashExceptSpec::ObHashExceptSpec(ObIAllocator& alloc, const ObPhyOperatorType type) : ObHashSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObHashExceptSpec, ObHashSetSpec));
|
||||
|
||||
ObHashExceptOp::ObHashExceptOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObHashSetOp(exec_ctx, spec, input)
|
||||
{}
|
||||
|
||||
int ObHashExceptOp::inner_open()
|
||||
{
|
||||
return ObHashSetOp::inner_open();
|
||||
}
|
||||
|
||||
int ObHashExceptOp::inner_close()
|
||||
{
|
||||
return ObHashSetOp::inner_close();
|
||||
}
|
||||
|
||||
int ObHashExceptOp::rescan()
|
||||
{
|
||||
return ObHashSetOp::rescan();
|
||||
}
|
||||
|
||||
void ObHashExceptOp::destroy()
|
||||
{
|
||||
return ObHashSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObHashExceptOp::build_hash_table_by_part()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool found = false;
|
||||
while (OB_SUCC(ret) && !found) {
|
||||
if (OB_FAIL(hp_infras_.get_next_pair_partition(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to get next pair partitions", K(ret));
|
||||
} else if (!hp_infras_.has_cur_part(InputSide::RIGHT)) {
|
||||
if (OB_FAIL(hp_infras_.get_next_partition(InputSide::LEFT))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next partition", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
} else {
|
||||
found = true;
|
||||
hp_infras_.switch_left();
|
||||
}
|
||||
} else if (!hp_infras_.has_cur_part(InputSide::LEFT)) {
|
||||
// left: empty right: yes
|
||||
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(build_hash_table(false))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else {
|
||||
found = true;
|
||||
hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashExceptOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool left_has_row = false;
|
||||
bool got_row = false;
|
||||
bool inserted = false;
|
||||
bool exists = false;
|
||||
const ObHashPartCols* part_cols = nullptr;
|
||||
const ObChunkDatumStore::StoredRow* store_row = nullptr;
|
||||
const common::ObIArray<ObExpr*>* cur_exprs = nullptr;
|
||||
clear_evaluated_flag();
|
||||
if (first_get_left_) {
|
||||
if (OB_FAIL(is_left_has_row(left_has_row))) {
|
||||
LOG_WARN("failed to judge left has row", K(ret));
|
||||
} else if (!left_has_row) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(ObHashSetOp::init_hash_partition_infras())) {
|
||||
LOG_WARN("failed to init hash partition infras", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table(true))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!has_got_part_) {
|
||||
if (OB_FAIL(get_left_row())) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get left row", K(ret));
|
||||
}
|
||||
} else {
|
||||
cur_exprs = &left_->get_spec().output_;
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.get_left_next_row(store_row, MY_SPEC.output_))) {
|
||||
} else {
|
||||
cur_exprs = &MY_SPEC.output_;
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get next dumped partition
|
||||
if (OB_FAIL(hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!has_got_part_) {
|
||||
has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table_by_part())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.exists_row(*cur_exprs, part_cols))) {
|
||||
LOG_WARN("failed to probe exists row", K(ret));
|
||||
} else if (OB_NOT_NULL(part_cols)) {
|
||||
// exists
|
||||
} else {
|
||||
// row is not exists in hash table
|
||||
if (hp_infras_.has_right_dumped() || hp_infras_.has_left_dumped()) {
|
||||
// dump row
|
||||
if (!hp_infras_.has_left_dumped() && OB_FAIL(hp_infras_.create_dumped_partitions(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to create dump partitions", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.insert_row_on_partitions(*cur_exprs))) {
|
||||
LOG_WARN("failed to insert row into partitions", K(ret));
|
||||
}
|
||||
} else {
|
||||
// insert and return row
|
||||
if (OB_FAIL(OB_FAIL(hp_infras_.insert_row(*cur_exprs, exists, inserted)))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
} else if (inserted) {
|
||||
got_row = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret) && !has_got_part_) {
|
||||
if (OB_FAIL(convert_row(*cur_exprs, MY_SPEC.output_))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
47
src/sql/engine/set/ob_hash_except_op.h
Normal file
47
src/sql/engine/set/ob_hash_except_op.h
Normal file
@ -0,0 +1,47 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObHashExceptSpec : public ObHashSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObHashExceptSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObHashExceptOp : public ObHashSetOp {
|
||||
public:
|
||||
ObHashExceptOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
~ObHashExceptOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
|
||||
private:
|
||||
int build_hash_table_by_part();
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_OP_H_
|
||||
183
src/sql/engine/set/ob_hash_intersect.cpp
Normal file
183
src/sql/engine/set/ob_hash_intersect.cpp
Normal file
@ -0,0 +1,183 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_hash_intersect.h"
|
||||
#include "lib/utility/utility.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
|
||||
namespace sql {
|
||||
|
||||
class ObHashIntersect::ObHashIntersectCtx : public ObHashSetOperatorCtx {
|
||||
explicit ObHashIntersectCtx(ObExecContext& ctx) : ObHashSetOperatorCtx(ctx)
|
||||
{}
|
||||
|
||||
~ObHashIntersectCtx()
|
||||
{}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashIntersectCtx);
|
||||
friend class ObHashIntersect;
|
||||
};
|
||||
|
||||
ObHashIntersect::ObHashIntersect(common::ObIAllocator& alloc) : ObHashSetOperator(alloc)
|
||||
{}
|
||||
|
||||
ObHashIntersect::~ObHashIntersect()
|
||||
{}
|
||||
|
||||
int ObHashIntersect::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObHashIntersectCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
|
||||
int ObHashIntersect::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
return ObHashSetOperator::inner_open(ctx);
|
||||
}
|
||||
|
||||
int ObHashIntersect::build_hash_table_by_part(ObExecContext& ctx, ObHashIntersectCtx* intersect_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool found = false;
|
||||
while (OB_SUCC(ret) && !found) {
|
||||
if (OB_FAIL(intersect_ctx->hp_infras_.get_next_pair_partition(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to get next pair partitions", K(ret));
|
||||
} else if (!intersect_ctx->hp_infras_.has_cur_part(InputSide::RIGHT)) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (!intersect_ctx->hp_infras_.has_cur_part(InputSide::LEFT)) {
|
||||
// right part has no matched left part
|
||||
if (OB_FAIL(intersect_ctx->hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(build_hash_table(ctx, false))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else if (OB_FAIL(intersect_ctx->hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else {
|
||||
found = true;
|
||||
intersect_ctx->hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashIntersect::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashIntersectCtx* intersect_ctx = NULL;
|
||||
const ObNewRow* cur_row = NULL;
|
||||
bool left_has_row = false;
|
||||
bool got_row = false;
|
||||
const ObChunkRowStore::StoredRow* store_row = nullptr;
|
||||
const HashPartCols* part_cols = nullptr;
|
||||
if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObHashIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else if (intersect_ctx->first_get_left_) {
|
||||
if (OB_FAIL(intersect_ctx->is_left_has_row(ctx, left_has_row))) {
|
||||
LOG_WARN("failed to judge left has row", K(ret));
|
||||
} else if (!left_has_row) {
|
||||
ret = OB_ITER_END;
|
||||
intersect_ctx->iter_end_ = true;
|
||||
} else if (OB_FAIL(ObHashSetOperator::init_hash_partition_infras(ctx))) {
|
||||
LOG_WARN("failed to init hash partition infras", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table(ctx, true))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
intersect_ctx->hp_infras_.switch_left();
|
||||
}
|
||||
} else if (intersect_ctx->iter_end_) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!intersect_ctx->has_got_part_) {
|
||||
if (OB_FAIL(intersect_ctx->get_left_row(ctx, cur_row))) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get left row", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = intersect_ctx->hp_infras_.get_left_next_row(store_row, cur_row);
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get next dumped partition
|
||||
if (OB_FAIL(intersect_ctx->hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!intersect_ctx->has_got_part_) {
|
||||
intersect_ctx->has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(intersect_ctx->hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(intersect_ctx->hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(intersect_ctx->hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table_by_part(ctx, intersect_ctx))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
intersect_ctx->iter_end_ = true;
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status(ctx))) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(intersect_ctx->hp_infras_.exists_row(cur_row, part_cols))) {
|
||||
LOG_WARN("failed to probe exists row", K(ret));
|
||||
} else if (OB_NOT_NULL(part_cols)) {
|
||||
// LOG_TRACE("trace part cols store row",
|
||||
// K(*part_cols->store_row_), K(part_cols->store_row_->is_match()));
|
||||
if (!part_cols->store_row_->is_match()) {
|
||||
// exists & not return
|
||||
got_row = true;
|
||||
row = cur_row;
|
||||
part_cols->store_row_->set_is_match(true);
|
||||
}
|
||||
} else {
|
||||
// row is not exists in hash table
|
||||
if (intersect_ctx->hp_infras_.has_right_dumped()) {
|
||||
// dump left row if right is dumpe
|
||||
if (!intersect_ctx->hp_infras_.has_left_dumped() &&
|
||||
OB_FAIL(intersect_ctx->hp_infras_.create_dumped_partitions(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to create dump partitions", K(ret));
|
||||
} else if (OB_FAIL(intersect_ctx->hp_infras_.insert_row_on_partitions(cur_row))) {
|
||||
LOG_WARN("failed to insert row into partitions", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(copy_cur_row_by_projector(*intersect_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashIntersect::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOperator::inner_close(ctx))) {
|
||||
LOG_WARN("failed to close", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
42
src/sql/engine/set/ob_hash_intersect.h
Normal file
42
src/sql/engine/set/ob_hash_intersect.h
Normal file
@ -0,0 +1,42 @@
|
||||
/**
|
||||
* 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_HASH_INTERSECT_H
|
||||
#define OB_HASH_INTERSECT_H
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObHashIntersect : public ObHashSetOperator {
|
||||
private:
|
||||
class ObHashIntersectCtx;
|
||||
|
||||
public:
|
||||
explicit ObHashIntersect(common::ObIAllocator& alloc);
|
||||
|
||||
virtual ~ObHashIntersect();
|
||||
|
||||
private:
|
||||
int build_hash_table_by_part(ObExecContext& ctx, ObHashIntersectCtx* intersect_ctx) const;
|
||||
int get_next_group_part(ObExecContext& ctx, ObHashIntersectCtx* intersect_ctx) const;
|
||||
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashIntersect);
|
||||
};
|
||||
} // namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OB_HASH_INTERSECT_H
|
||||
171
src/sql/engine/set/ob_hash_intersect_op.cpp
Normal file
171
src/sql/engine/set/ob_hash_intersect_op.cpp
Normal file
@ -0,0 +1,171 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_hash_intersect_op.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/basic/ob_hash_partitioning_infrastructure_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObHashIntersectSpec::ObHashIntersectSpec(ObIAllocator& alloc, const ObPhyOperatorType type) : ObHashSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObHashIntersectSpec, ObHashSetSpec));
|
||||
|
||||
ObHashIntersectOp::ObHashIntersectOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObHashSetOp(exec_ctx, spec, input)
|
||||
{}
|
||||
|
||||
int ObHashIntersectOp::inner_open()
|
||||
{
|
||||
return ObHashSetOp::inner_open();
|
||||
}
|
||||
|
||||
int ObHashIntersectOp::inner_close()
|
||||
{
|
||||
return ObHashSetOp::inner_close();
|
||||
}
|
||||
|
||||
int ObHashIntersectOp::rescan()
|
||||
{
|
||||
return ObHashSetOp::rescan();
|
||||
}
|
||||
|
||||
void ObHashIntersectOp::destroy()
|
||||
{
|
||||
return ObHashSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObHashIntersectOp::build_hash_table_by_part()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool found = false;
|
||||
while (OB_SUCC(ret) && !found) {
|
||||
if (OB_FAIL(hp_infras_.get_next_pair_partition(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to get next pair partitions", K(ret));
|
||||
} else if (!hp_infras_.has_cur_part(InputSide::RIGHT)) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (!hp_infras_.has_cur_part(InputSide::LEFT)) {
|
||||
// right part has no matched left part
|
||||
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(build_hash_table(false))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else {
|
||||
found = true;
|
||||
hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashIntersectOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool left_has_row = false;
|
||||
bool got_row = false;
|
||||
const ObChunkDatumStore::StoredRow* store_row = nullptr;
|
||||
const ObHashPartCols* part_cols = nullptr;
|
||||
const common::ObIArray<ObExpr*>* cur_exprs = nullptr;
|
||||
clear_evaluated_flag();
|
||||
if (first_get_left_) {
|
||||
if (OB_FAIL(is_left_has_row(left_has_row))) {
|
||||
LOG_WARN("failed to judge left has row", K(ret));
|
||||
} else if (!left_has_row) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(ObHashSetOp::init_hash_partition_infras())) {
|
||||
LOG_WARN("failed to init hash partition infras", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table(true))) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
} else {
|
||||
hp_infras_.switch_left();
|
||||
}
|
||||
}
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!has_got_part_) {
|
||||
if (OB_FAIL(get_left_row())) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get left row", K(ret));
|
||||
}
|
||||
} else {
|
||||
cur_exprs = &left_->get_spec().output_;
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.get_left_next_row(store_row, MY_SPEC.output_))) {
|
||||
} else {
|
||||
cur_exprs = &MY_SPEC.output_;
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get next dumped partition
|
||||
if (OB_FAIL(hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!has_got_part_) {
|
||||
has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(build_hash_table_by_part())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to build hash table", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.exists_row(*cur_exprs, part_cols))) {
|
||||
LOG_WARN("failed to probe exists row", K(ret));
|
||||
} else if (OB_NOT_NULL(part_cols)) {
|
||||
// LOG_TRACE("trace part cols store row",
|
||||
// K(*part_cols->store_row_), K(part_cols->store_row_->is_match()));
|
||||
if (!part_cols->store_row_->is_match()) {
|
||||
// exists & not return
|
||||
got_row = true;
|
||||
part_cols->store_row_->set_is_match(true);
|
||||
}
|
||||
} else {
|
||||
// row is not exists in hash table
|
||||
if (hp_infras_.has_right_dumped()) {
|
||||
// dump left row if right is dumpe
|
||||
if (!hp_infras_.has_left_dumped() && OB_FAIL(hp_infras_.create_dumped_partitions(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to create dump partitions", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.insert_row_on_partitions(*cur_exprs))) {
|
||||
LOG_WARN("failed to insert row into partitions", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret) && !has_got_part_) {
|
||||
if (OB_FAIL(convert_row(*cur_exprs, MY_SPEC.output_))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
47
src/sql/engine/set/ob_hash_intersect_op.h
Normal file
47
src/sql/engine/set/ob_hash_intersect_op.h
Normal file
@ -0,0 +1,47 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObHashIntersectSpec : public ObHashSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObHashIntersectSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObHashIntersectOp : public ObHashSetOp {
|
||||
public:
|
||||
ObHashIntersectOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
~ObHashIntersectOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
|
||||
private:
|
||||
int build_hash_table_by_part();
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_OP_H_
|
||||
221
src/sql/engine/set/ob_hash_set_op.cpp
Normal file
221
src/sql/engine/set/ob_hash_set_op.cpp
Normal file
@ -0,0 +1,221 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_op.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/basic/ob_hash_partitioning_infrastructure_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObHashSetSpec::ObHashSetSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObSetSpec(alloc, type), hash_funcs_(alloc)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObHashSetSpec, ObSetSpec), hash_funcs_);
|
||||
|
||||
ObHashSetOp::ObHashSetOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
first_get_left_(true),
|
||||
has_got_part_(false),
|
||||
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
|
||||
sql_mem_processor_(profile_),
|
||||
hp_infras_()
|
||||
{}
|
||||
|
||||
int ObHashSetOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(left_) || OB_ISNULL(right_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: left or right is null", K(ret), K(left_), K(right_));
|
||||
} else if (OB_FAIL(ObOperator::inner_open())) {
|
||||
LOG_WARN("failed to inner open", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObHashSetOp::reset()
|
||||
{
|
||||
first_get_left_ = true;
|
||||
has_got_part_ = false;
|
||||
hp_infras_.reset();
|
||||
}
|
||||
|
||||
int ObHashSetOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::inner_close())) {
|
||||
LOG_WARN("failed to inner close", K(ret));
|
||||
} else {
|
||||
reset();
|
||||
sql_mem_processor_.unregister_profile();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
} else {
|
||||
reset();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObHashSetOp::destroy()
|
||||
{
|
||||
hp_infras_.~ObHashPartInfrastructure();
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int ObHashSetOp::is_left_has_row(bool& left_has_row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
left_has_row = true;
|
||||
if (OB_FAIL(left_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
left_has_row = false;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("failed to get next row from left op", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOp::get_left_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (first_get_left_) {
|
||||
first_get_left_ = false;
|
||||
} else {
|
||||
if (OB_FAIL(left_->get_next_row())) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("child operator get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOp::build_hash_table(bool from_child)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObChunkDatumStore::StoredRow* store_row = NULL;
|
||||
bool inserted = false;
|
||||
if (!from_child) {
|
||||
if (OB_FAIL(hp_infras_.open_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to open cur part", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::RIGHT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
} else if (OB_FAIL(sql_mem_processor_.init(&ctx_.get_allocator(),
|
||||
ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
hp_infras_.get_cur_part_file_size(InputSide::RIGHT),
|
||||
spec_.type_,
|
||||
spec_.id_,
|
||||
&ctx_))) {
|
||||
LOG_WARN("failed to init sql mem processor", K(ret));
|
||||
}
|
||||
}
|
||||
hp_infras_.switch_right();
|
||||
bool has_exists = false;
|
||||
while (OB_SUCC(ret)) {
|
||||
if (from_child) {
|
||||
if (OB_FAIL(right_->get_next_row())) {
|
||||
} else if (OB_FAIL(hp_infras_.insert_row(right_->get_spec().output_, has_exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.get_right_next_row(store_row, get_spec().output_))) {
|
||||
} else if (OB_FAIL(hp_infras_.insert_row(get_spec().output_, has_exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("check status exit", K(ret));
|
||||
}
|
||||
} // end of while
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish insert row", K(ret));
|
||||
} else if (!from_child && OB_FAIL(hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOp::init_hash_partition_infras()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t est_rows = get_spec().rows_;
|
||||
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx_, get_spec().px_est_size_factor_, est_rows, est_rows))) {
|
||||
LOG_WARN("failed to get px size", K(ret));
|
||||
} else if (OB_FAIL(sql_mem_processor_.init(&ctx_.get_allocator(),
|
||||
ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
est_rows * get_spec().width_,
|
||||
get_spec().type_,
|
||||
get_spec().id_,
|
||||
&ctx_))) {
|
||||
LOG_WARN("failed to init sql mem processor", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.init(ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
GCONF.is_sql_operator_dump_enabled() && !(GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2250),
|
||||
true,
|
||||
true,
|
||||
2,
|
||||
&sql_mem_processor_))) {
|
||||
LOG_WARN("failed to init hash partition infrastructure", K(ret));
|
||||
} else {
|
||||
const ObHashSetSpec& spec = static_cast<const ObHashSetSpec&>(get_spec());
|
||||
int64_t est_bucket_num = hp_infras_.est_bucket_count(est_rows, get_spec().width_);
|
||||
if (OB_FAIL(hp_infras_.set_funcs(&spec.hash_funcs_, &spec.sort_collations_, &spec.sort_cmp_funs_, &eval_ctx_))) {
|
||||
LOG_WARN("failed to set funcs", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to start round", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.init_hash_table(est_bucket_num))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOp::convert_row(const common::ObIArray<ObExpr*>& src_exprs, const common::ObIArray<ObExpr*>& dst_exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (dst_exprs.count() != src_exprs.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: exprs is not match", K(ret), K(src_exprs.count()), K(dst_exprs.count()));
|
||||
} else {
|
||||
ObDatum* src_datum = nullptr;
|
||||
for (uint32_t i = 0; i < dst_exprs.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(src_exprs.at(i)->eval(eval_ctx_, src_datum))) {
|
||||
LOG_WARN("failed to eval expr", K(ret), K(i));
|
||||
} else {
|
||||
dst_exprs.at(i)->locate_expr_datum(eval_ctx_) = *src_datum;
|
||||
dst_exprs.at(i)->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
}
|
||||
// LOG_TRACE("trace convert row", K(ret), K(dst_exprs.count()), K(src_exprs.count()));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
66
src/sql/engine/set/ob_hash_set_op.h
Normal file
66
src/sql/engine/set/ob_hash_set_op.h
Normal file
@ -0,0 +1,66 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_SET_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_HASH_SET_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_set_op.h"
|
||||
#include "share/datum/ob_datum_funcs.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
#include "sql/engine/basic/ob_hash_partitioning_infrastructure_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObHashSetSpec : public ObSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObHashSetSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObSetSpec, K_(hash_funcs));
|
||||
ObHashFuncs hash_funcs_;
|
||||
};
|
||||
|
||||
class ObHashSetOp : public ObOperator {
|
||||
public:
|
||||
ObHashSetOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
~ObHashSetOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
|
||||
protected:
|
||||
void reset();
|
||||
int is_left_has_row(bool& left_has_row);
|
||||
int get_left_row();
|
||||
|
||||
int build_hash_table(bool from_child);
|
||||
int init_hash_partition_infras();
|
||||
int convert_row(const common::ObIArray<ObExpr*>& src_exprs, const common::ObIArray<ObExpr*>& dst_exprs);
|
||||
|
||||
protected:
|
||||
// used by intersect and except
|
||||
bool first_get_left_;
|
||||
bool has_got_part_;
|
||||
ObSqlWorkAreaProfile profile_;
|
||||
ObSqlMemMgrProcessor sql_mem_processor_;
|
||||
ObHashPartInfrastructure<ObHashPartCols, ObHashPartStoredRow> hp_infras_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_SET_OP_H_
|
||||
320
src/sql/engine/set/ob_hash_set_operator.cpp
Normal file
320
src/sql/engine/set/ob_hash_set_operator.cpp
Normal file
@ -0,0 +1,320 @@
|
||||
/**
|
||||
* 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_hash_set_operator.h"
|
||||
#include "lib/allocator/ob_mod_define.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "share/ob_cluster_version.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::common;
|
||||
|
||||
int ObHashSetOperator::HashCols::init(const ObNewRow* row, const ObIArray<ObCollationType>* col_collation)
|
||||
{
|
||||
row_ = row;
|
||||
col_collation_ = col_collation;
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
uint64_t ObHashSetOperator::HashCols::hash() const
|
||||
{
|
||||
uint64_t result = 0;
|
||||
if (!OB_ISNULL(col_collation_) && !OB_ISNULL(row_) && row_->is_valid()) {
|
||||
int64_t N = col_collation_->count();
|
||||
const ObObj* cells = row_->cells_;
|
||||
const int32_t* projector = row_->projector_;
|
||||
for (int64_t i = 0; i < N; ++i) {
|
||||
int64_t real_index = row_->projector_size_ > 0 ? projector[i] : i;
|
||||
const ObObj& cell = cells[real_index];
|
||||
result = cell.is_string_type() ? cell.varchar_hash(col_collation_->at(i), result) : cell.hash(result);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
bool ObHashSetOperator::HashCols::operator==(const ObHashSetOperator::HashCols& other) const
|
||||
{
|
||||
bool result = true;
|
||||
const ObObj* lcell = NULL;
|
||||
const ObObj* rcell = NULL;
|
||||
int64_t real_idx = -1;
|
||||
if (OB_ISNULL(col_collation_)) {
|
||||
result = false;
|
||||
} else {
|
||||
int64_t N = col_collation_->count();
|
||||
for (int32_t i = 0; i < N && result; ++i) {
|
||||
if (NULL != row_ && row_->is_valid()) {
|
||||
real_idx = row_->projector_size_ > 0 ? row_->projector_[i] : i;
|
||||
lcell = &row_->cells_[real_idx];
|
||||
}
|
||||
if (NULL != row_ && row_->is_valid()) {
|
||||
real_idx = other.row_->projector_size_ > 0 ? other.row_->projector_[i] : i;
|
||||
rcell = &other.row_->cells_[real_idx];
|
||||
}
|
||||
if (NULL == lcell || NULL == rcell) {
|
||||
result = false;
|
||||
} else {
|
||||
result = lcell->is_equal(*rcell, col_collation_->at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
int ObHashSetOperator::ObHashSetOperatorCtx::is_left_has_row(ObExecContext& ctx, bool& left_has_row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
left_has_row = true;
|
||||
if (OB_ISNULL(left_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op is null", K(ret), K(left_op_));
|
||||
} else if (OB_FAIL(left_op_->get_next_row(ctx, first_left_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
left_has_row = false;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("failed to get next row from left op", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObHashSetOperator::ObHashSetOperatorCtx::reset()
|
||||
{
|
||||
first_get_left_ = true;
|
||||
has_got_part_ = false;
|
||||
iter_end_ = false;
|
||||
first_left_row_ = NULL;
|
||||
hp_infras_.reset();
|
||||
}
|
||||
|
||||
int ObHashSetOperator::ObHashSetOperatorCtx::get_left_row(ObExecContext& ctx, const ObNewRow*& cur_row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(left_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op is null", K(ret), K(left_op_));
|
||||
} else if (first_get_left_) {
|
||||
if (OB_ISNULL(first_left_row_)) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
cur_row = first_left_row_;
|
||||
first_left_row_ = NULL;
|
||||
}
|
||||
first_get_left_ = false;
|
||||
} else if (OB_FAIL(left_op_->get_next_row(ctx, cur_row))) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("child operator get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObHashSetOperator::ObHashSetOperator(common::ObIAllocator& alloc) : ObSetOperator(alloc)
|
||||
{}
|
||||
|
||||
ObHashSetOperator::~ObHashSetOperator()
|
||||
{}
|
||||
|
||||
int ObHashSetOperator::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashSetOperatorCtx* hash_ctx = NULL;
|
||||
if (OB_FAIL(ObSetOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan child operator failed", K(ret));
|
||||
} else if (OB_ISNULL(hash_ctx = GET_PHY_OPERATOR_CTX(ObHashSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to get except ctx", K(ret), K(hash_ctx));
|
||||
} else {
|
||||
hash_ctx->reset();
|
||||
hash_ctx->left_op_ = get_child(FIRST_CHILD);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObHashSetOperator::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_KV(N_DISTINCT, distinct_, "collation_types", cs_types_);
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObHashSetOperator::init_hash_partition_infras(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashSetOperatorCtx* hash_ctx = NULL;
|
||||
int64_t est_rows = get_rows();
|
||||
if (OB_ISNULL(hash_ctx = GET_PHY_OPERATOR_CTX(ObHashSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K(ctx), K_(id));
|
||||
} else if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx, px_est_size_factor_, get_rows(), est_rows))) {
|
||||
LOG_WARN("failed to get px size", K(ret));
|
||||
} else if (OB_FAIL(hash_ctx->sql_mem_processor_.init(&hash_ctx->exec_ctx_.get_allocator(),
|
||||
hash_ctx->exec_ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
est_rows * get_width(),
|
||||
get_type(),
|
||||
get_id(),
|
||||
&hash_ctx->exec_ctx_))) {
|
||||
LOG_WARN("failed to init sql mem processor", K(ret));
|
||||
} else if (OB_FAIL(hash_ctx->hp_infras_.init(ctx.get_my_session()->get_effective_tenant_id(),
|
||||
GCONF.is_sql_operator_dump_enabled() && !(GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2250),
|
||||
true,
|
||||
true,
|
||||
2,
|
||||
&hash_ctx->sql_mem_processor_))) {
|
||||
LOG_WARN("failed to init hash partition infrastructure", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; i < cs_types_.count() && OB_SUCC(ret); ++i) {
|
||||
ObColumnInfo column_info;
|
||||
column_info.index_ = i;
|
||||
column_info.cs_type_ = cs_types_.at(i);
|
||||
if (OB_FAIL(hash_ctx->hp_infras_.add_part_col_idx(column_info))) {
|
||||
LOG_WARN("failed to add part column index", K(ret));
|
||||
}
|
||||
}
|
||||
int64_t est_bucket_num = hash_ctx->hp_infras_.est_bucket_count(est_rows, get_width());
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hash_ctx->hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to start round", K(ret));
|
||||
} else if (OB_FAIL(hash_ctx->hp_infras_.init_hash_table(est_bucket_num))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOperator::build_hash_table(ObExecContext& ctx, bool from_child) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashSetOperatorCtx* hash_ctx = NULL;
|
||||
const ObNewRow* cur_row = NULL;
|
||||
const ObChunkRowStore::StoredRow* store_row = NULL;
|
||||
bool inserted = false;
|
||||
if (OB_ISNULL(hash_ctx = GET_PHY_OPERATOR_CTX(ObHashSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else {
|
||||
if (!from_child) {
|
||||
if (OB_FAIL(hash_ctx->hp_infras_.open_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to open cur part", K(ret));
|
||||
} else if (OB_FAIL(hash_ctx->hp_infras_.resize(hash_ctx->hp_infras_.get_cur_part_row_cnt(InputSide::RIGHT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
} else if (OB_FAIL(hash_ctx->sql_mem_processor_.init(&hash_ctx->exec_ctx_.get_allocator(),
|
||||
hash_ctx->exec_ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
hash_ctx->hp_infras_.get_cur_part_file_size(InputSide::RIGHT),
|
||||
get_type(),
|
||||
get_id(),
|
||||
&hash_ctx->exec_ctx_))) {
|
||||
LOG_WARN("failed to init sql mem processor", K(ret));
|
||||
}
|
||||
}
|
||||
// switch to right, dump right partition
|
||||
hash_ctx->hp_infras_.switch_right();
|
||||
bool has_exists = false;
|
||||
while (OB_SUCC(ret)) {
|
||||
if (from_child) {
|
||||
ret = get_child(SECOND_CHILD)->get_next_row(ctx, cur_row);
|
||||
} else {
|
||||
ret = hash_ctx->hp_infras_.get_right_next_row(store_row, cur_row);
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_check_status(ctx))) {
|
||||
LOG_WARN("check status exit", K(ret));
|
||||
} else if (OB_ISNULL(cur_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null", K(ret), K(cur_row));
|
||||
} else if (OB_FAIL(hash_ctx->hp_infras_.insert_row(cur_row, has_exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
}
|
||||
} // end of while
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(hash_ctx->hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish insert row", K(ret));
|
||||
} else if (!from_child && OB_FAIL(hash_ctx->hp_infras_.close_cur_part(InputSide::RIGHT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOperator::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashSetOperatorCtx* hash_ctx = NULL;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to init op ctx", K(ret));
|
||||
} else if (OB_ISNULL(hash_ctx = GET_PHY_OPERATOR_CTX(ObHashSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else {
|
||||
hash_ctx->left_op_ = get_child(FIRST_CHILD);
|
||||
;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashSetOperator::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashSetOperatorCtx* hash_ctx = NULL;
|
||||
if (OB_ISNULL(hash_ctx = GET_PHY_OPERATOR_CTX(ObHashSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else {
|
||||
hash_ctx->reset();
|
||||
hash_ctx->sql_mem_processor_.unregister_profile();
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObHashSetOperator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_UNIS_ENCODE(distinct_);
|
||||
OB_UNIS_ENCODE(cs_types_);
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObPhyOperator::serialize(buf, buf_len, pos);
|
||||
}
|
||||
OB_UNIS_ENCODE(child_num_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObHashSetOperator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_distinct = false;
|
||||
child_num_ = 2; // for compatibility, set a default two child op value
|
||||
OB_UNIS_DECODE(is_distinct);
|
||||
OB_UNIS_DECODE(cs_types_);
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObPhyOperator::deserialize(buf, data_len, pos);
|
||||
}
|
||||
OB_UNIS_DECODE(child_num_);
|
||||
if (OB_SUCC(ret)) {
|
||||
set_distinct(is_distinct);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObHashSetOperator)
|
||||
{
|
||||
int64_t len = 0;
|
||||
OB_UNIS_ADD_LEN(distinct_);
|
||||
OB_UNIS_ADD_LEN(cs_types_);
|
||||
len += ObPhyOperator::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(child_num_);
|
||||
return len;
|
||||
}
|
||||
110
src/sql/engine/set/ob_hash_set_operator.h
Normal file
110
src/sql/engine/set/ob_hash_set_operator.h
Normal file
@ -0,0 +1,110 @@
|
||||
/**
|
||||
* 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_HASH_SET_OPERATOR_H
|
||||
#define OB_HASH_SET_OPERATOR_H
|
||||
|
||||
#include "sql/engine/set/ob_set_operator.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "lib/hash/ob_hashset.h"
|
||||
#include "sql/engine/basic/ob_hash_partitioning_infrastructure.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObHashSetOperator : public ObSetOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
protected:
|
||||
class HashCols {
|
||||
public:
|
||||
explicit HashCols() : row_(NULL), col_collation_(NULL)
|
||||
{}
|
||||
~HashCols()
|
||||
{}
|
||||
int init(const common::ObNewRow* row, const common::ObIArray<common::ObCollationType>* col_collation);
|
||||
uint64_t hash() const;
|
||||
bool operator==(const HashCols& other) const;
|
||||
|
||||
public:
|
||||
const common::ObNewRow* row_;
|
||||
const common::ObIArray<common::ObCollationType>* col_collation_;
|
||||
};
|
||||
|
||||
class ObHashSetOperatorCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
static const int64_t MIN_BUCKET_COUNT = 10000;
|
||||
static const int64_t MAX_BUCKET_COUNT = 500000;
|
||||
static const int64_t HASH_SET_BUCKET_RATIO = 10;
|
||||
|
||||
explicit ObHashSetOperatorCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx),
|
||||
first_get_left_(true),
|
||||
has_got_part_(false),
|
||||
iter_end_(false),
|
||||
first_left_row_(NULL),
|
||||
left_op_(NULL),
|
||||
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
|
||||
sql_mem_processor_(profile_),
|
||||
hp_infras_()
|
||||
{}
|
||||
virtual ~ObHashSetOperatorCtx()
|
||||
{}
|
||||
|
||||
virtual void reset();
|
||||
void destroy() override
|
||||
{
|
||||
hp_infras_.~ObBasicHashPartInfrastructure();
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
int is_left_has_row(ObExecContext& ctx, bool& left_has_row);
|
||||
int get_left_row(ObExecContext& ctx, const common::ObNewRow*& cur_row);
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashSetOperatorCtx);
|
||||
friend class ObHashSetOperator;
|
||||
|
||||
protected:
|
||||
// used by intersect and except
|
||||
bool first_get_left_;
|
||||
bool has_got_part_;
|
||||
bool iter_end_;
|
||||
const ObNewRow* first_left_row_;
|
||||
ObPhyOperator* left_op_;
|
||||
ObSqlWorkAreaProfile profile_;
|
||||
ObSqlMemMgrProcessor sql_mem_processor_;
|
||||
ObBasicHashPartInfrastructure<HashPartCols, ObPartStoredRow> hp_infras_;
|
||||
};
|
||||
|
||||
public:
|
||||
explicit ObHashSetOperator(common::ObIAllocator& alloc);
|
||||
~ObHashSetOperator();
|
||||
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
protected:
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
|
||||
int build_hash_table(ObExecContext& ctx, bool from_child) const;
|
||||
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
virtual int init_hash_partition_infras(ObExecContext& ctx) const;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashSetOperator);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OB_HASH_SET_OPERATOR_H
|
||||
189
src/sql/engine/set/ob_hash_union.cpp
Normal file
189
src/sql/engine/set/ob_hash_union.cpp
Normal file
@ -0,0 +1,189 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_hash_union.h"
|
||||
#include "lib/utility/utility.h"
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
|
||||
namespace sql {
|
||||
class ObHashUnion::ObHashUnionCtx : public ObHashSetOperatorCtx {
|
||||
public:
|
||||
explicit ObHashUnionCtx(ObExecContext& ctx) : ObHashSetOperatorCtx(ctx), cur_child_op_(NULL), is_left_child_(true)
|
||||
{}
|
||||
|
||||
virtual ~ObHashUnionCtx()
|
||||
{}
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashUnionCtx);
|
||||
|
||||
protected:
|
||||
ObPhyOperator* cur_child_op_;
|
||||
bool is_left_child_;
|
||||
|
||||
friend class ObHashUnion;
|
||||
};
|
||||
|
||||
ObHashUnion::ObHashUnion(common::ObIAllocator& alloc) : ObHashSetOperator(alloc)
|
||||
{}
|
||||
|
||||
ObHashUnion::~ObHashUnion()
|
||||
{}
|
||||
|
||||
int ObHashUnion::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashUnionCtx* union_ctx = NULL;
|
||||
if (OB_FAIL(ObHashSetOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan child operator failed", K(ret));
|
||||
} else if (OB_ISNULL(union_ctx = GET_PHY_OPERATOR_CTX(ObHashUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to get union ctx", K(ret), K(union_ctx));
|
||||
} else {
|
||||
union_ctx->cur_child_op_ = get_child(FIRST_CHILD);
|
||||
union_ctx->is_left_child_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnion::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObHashUnionCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
|
||||
int ObHashUnion::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashUnionCtx* union_ctx = NULL;
|
||||
if (init_op_ctx(ctx)) {
|
||||
LOG_WARN("failed to init operator context", K(ret));
|
||||
} else if (OB_ISNULL(union_ctx = GET_PHY_OPERATOR_CTX(ObHashUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K(ctx), K_(id));
|
||||
} else {
|
||||
union_ctx->cur_child_op_ = get_child(FIRST_CHILD);
|
||||
union_ctx->is_left_child_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnion::get_child_next_row(ObExecContext& ctx, ObHashUnionCtx* union_ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = union_ctx->cur_child_op_->get_next_row(ctx, row);
|
||||
if (OB_ITER_END == ret) {
|
||||
if (union_ctx->is_left_child_) {
|
||||
ret = OB_SUCCESS;
|
||||
union_ctx->is_left_child_ = false;
|
||||
union_ctx->cur_child_op_ = get_child(SECOND_CHILD);
|
||||
ret = union_ctx->cur_child_op_->get_next_row(ctx, row);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// If bkt_created_ false, create bucket fist.
|
||||
// If not matched int hash buckets, store in bucket and return this row.
|
||||
// If matched, get next row until get row not matched.
|
||||
// Once the left child is exhausted, continue to visist the right child
|
||||
int ObHashUnion::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObHashUnionCtx* union_ctx = NULL;
|
||||
const ObNewRow* cur_row = NULL;
|
||||
if (OB_ISNULL(union_ctx = GET_PHY_OPERATOR_CTX(ObHashUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(union_ctx->cur_child_op_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("cur_child_op is null", K(ret), K(union_ctx->cur_child_op_));
|
||||
} else if (union_ctx->first_get_left_) {
|
||||
if (OB_FAIL(ObHashSetOperator::init_hash_partition_infras(ctx))) {
|
||||
LOG_WARN("failed to get next row", K(ret));
|
||||
}
|
||||
union_ctx->first_get_left_ = false;
|
||||
} else if (union_ctx->iter_end_) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
HashCols hash_cols;
|
||||
bool got_row = false;
|
||||
bool has_exists = false;
|
||||
bool inserted = false;
|
||||
const ObChunkRowStore::StoredRow* store_row = nullptr;
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!union_ctx->has_got_part_) {
|
||||
ret = get_child_next_row(ctx, union_ctx, cur_row);
|
||||
} else {
|
||||
ret = union_ctx->hp_infras_.get_left_next_row(store_row, cur_row);
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get dumped partition
|
||||
if (OB_FAIL(union_ctx->hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!union_ctx->has_got_part_) {
|
||||
union_ctx->has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(union_ctx->hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(try_check_status(ctx))) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.get_next_partition(InputSide::LEFT))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to create dumped partitions", K(ret));
|
||||
} else {
|
||||
union_ctx->iter_end_ = true;
|
||||
}
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.resize(union_ctx->hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(cur_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null", K(ret), K(cur_row));
|
||||
} else if (OB_FAIL(union_ctx->hp_infras_.insert_row(cur_row, has_exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
} else if (has_exists) {
|
||||
// Already in hash map, do nothing
|
||||
} else if (inserted) {
|
||||
got_row = true;
|
||||
row = cur_row;
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(copy_cur_row_by_projector(*union_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnion::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOperator::inner_close(ctx))) {
|
||||
LOG_WARN("failed to close", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
46
src/sql/engine/set/ob_hash_union.h
Normal file
46
src/sql/engine/set/ob_hash_union.h
Normal file
@ -0,0 +1,46 @@
|
||||
/**
|
||||
* 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 SQL_ENGINE_SET_OB_HASH_UNION
|
||||
#define SQL_ENGINE_SET_OB_HASH_UNION
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObHashUnion : public ObHashSetOperator {
|
||||
private:
|
||||
class ObHashUnionCtx;
|
||||
|
||||
public:
|
||||
explicit ObHashUnion(common::ObIAllocator& alloc);
|
||||
|
||||
virtual ~ObHashUnion();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
private:
|
||||
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
|
||||
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
|
||||
int get_child_next_row(ObExecContext& ctx, ObHashUnionCtx* union_ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObHashUnion);
|
||||
};
|
||||
} // namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif
|
||||
152
src/sql/engine/set/ob_hash_union_op.cpp
Normal file
152
src/sql/engine/set/ob_hash_union_op.cpp
Normal file
@ -0,0 +1,152 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_hash_union_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObHashUnionSpec::ObHashUnionSpec(ObIAllocator& alloc, const ObPhyOperatorType type) : ObHashSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObHashUnionSpec, ObHashSetSpec));
|
||||
|
||||
ObHashUnionOp::ObHashUnionOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObHashSetOp(exec_ctx, spec, input), cur_child_op_(nullptr), is_left_child_(true)
|
||||
{}
|
||||
|
||||
int ObHashUnionOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOp::inner_open())) {
|
||||
LOG_WARN("failed to inner open", K(ret));
|
||||
} else {
|
||||
cur_child_op_ = left_;
|
||||
is_left_child_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnionOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOp::inner_close())) {
|
||||
LOG_WARN("failed to inner close", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnionOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObHashSetOp::rescan())) {
|
||||
LOG_WARN("failed to rescan child operator", K(ret));
|
||||
} else {
|
||||
cur_child_op_ = left_;
|
||||
is_left_child_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObHashUnionOp::destroy()
|
||||
{
|
||||
ObHashSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObHashUnionOp::get_child_next_row()
|
||||
{
|
||||
int ret = cur_child_op_->get_next_row();
|
||||
if (OB_ITER_END == ret) {
|
||||
if (is_left_child_) {
|
||||
ret = OB_SUCCESS;
|
||||
is_left_child_ = false;
|
||||
cur_child_op_ = right_;
|
||||
ret = cur_child_op_->get_next_row();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObHashUnionOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (OB_ISNULL(cur_child_op_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("cur_child_op is null", K(ret), K(cur_child_op_));
|
||||
} else if (first_get_left_) {
|
||||
if (OB_FAIL(ObHashSetOp::init_hash_partition_infras())) {
|
||||
LOG_WARN("failed to get next row", K(ret));
|
||||
}
|
||||
first_get_left_ = false;
|
||||
}
|
||||
bool got_row = false;
|
||||
bool has_exists = false;
|
||||
bool inserted = false;
|
||||
const ObChunkDatumStore::StoredRow* store_row = nullptr;
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
if (!has_got_part_) {
|
||||
ret = get_child_next_row();
|
||||
} else {
|
||||
ret = hp_infras_.get_left_next_row(store_row, MY_SPEC.set_exprs_);
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
// get dumped partition
|
||||
if (OB_FAIL(hp_infras_.finish_insert_row())) {
|
||||
LOG_WARN("failed to finish to insert row", K(ret));
|
||||
} else if (!has_got_part_) {
|
||||
has_got_part_ = true;
|
||||
} else {
|
||||
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to close cur part", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hp_infras_.end_round())) {
|
||||
LOG_WARN("failed to end round", K(ret));
|
||||
} else if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("failed to check status", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.start_round())) {
|
||||
LOG_WARN("failed to open round", K(ret));
|
||||
} else if (OB_FAIL(hp_infras_.get_next_partition(InputSide::LEFT))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to create dumped partitions", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
|
||||
LOG_WARN("failed to open cur part");
|
||||
} else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
|
||||
LOG_WARN("failed to init hash table", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hp_infras_.insert_row(
|
||||
has_got_part_ ? MY_SPEC.set_exprs_ : cur_child_op_->get_spec().output_, has_exists, inserted))) {
|
||||
LOG_WARN("failed to insert row", K(ret));
|
||||
} else if (has_exists) {
|
||||
// Already in hash map, do nothing
|
||||
} else if (inserted) {
|
||||
got_row = true;
|
||||
}
|
||||
} // end of while
|
||||
if (OB_SUCC(ret) && !has_got_part_) {
|
||||
if (OB_FAIL(convert_row(cur_child_op_->get_spec().output_, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
51
src/sql/engine/set/ob_hash_union_op.h
Normal file
51
src/sql/engine/set/ob_hash_union_op.h
Normal file
@ -0,0 +1,51 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_hash_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObHashUnionSpec : public ObHashSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObHashUnionSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObHashUnionOp : public ObHashSetOp {
|
||||
public:
|
||||
ObHashUnionOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
~ObHashUnionOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
|
||||
private:
|
||||
int get_child_next_row();
|
||||
|
||||
private:
|
||||
ObOperator* cur_child_op_;
|
||||
bool is_left_child_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_OP_H_
|
||||
322
src/sql/engine/set/ob_merge_except.cpp
Normal file
322
src/sql/engine/set/ob_merge_except.cpp
Normal file
@ -0,0 +1,322 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_merge_except.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "common/object/ob_object.h"
|
||||
#include "common/row/ob_row_util.h"
|
||||
#include "sql/engine/expr/ob_expr_operator.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
|
||||
namespace sql {
|
||||
class ObMergeExcept::ObMergeExceptCtx : public ObMergeSetOperatorCtx {
|
||||
public:
|
||||
explicit ObMergeExceptCtx(ObExecContext& ctx) : ObMergeSetOperatorCtx(ctx), right_iter_end_(false), right_row_(NULL)
|
||||
{
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
void reset()
|
||||
{
|
||||
right_iter_end_ = false;
|
||||
right_row_ = NULL;
|
||||
need_skip_init_row_ = true;
|
||||
for (int64_t i = 0; i < last_output_row_.count_; ++i) {
|
||||
last_output_row_.cells_[i].set_null();
|
||||
}
|
||||
}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObMergeSetOperatorCtx::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
bool right_iter_end_;
|
||||
const ObNewRow* right_row_;
|
||||
friend class ObMergeExcept;
|
||||
};
|
||||
|
||||
ObMergeExcept::ObMergeExcept(common::ObIAllocator& alloc) : ObMergeSetOperator(alloc), get_next_row_func_(NULL)
|
||||
{}
|
||||
|
||||
ObMergeExcept::~ObMergeExcept()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
}
|
||||
|
||||
void ObMergeExcept::reset()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reset();
|
||||
}
|
||||
|
||||
void ObMergeExcept::reuse()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reuse();
|
||||
}
|
||||
|
||||
void ObMergeExcept::set_distinct(bool is_distinct)
|
||||
{
|
||||
ObMergeSetOperator::set_distinct(is_distinct);
|
||||
if (is_distinct) {
|
||||
get_next_row_func_ = &ObMergeExcept::distinct_get_next_row;
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeExcept::all_get_next_row;
|
||||
}
|
||||
}
|
||||
|
||||
int ObMergeExcept::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id(), PHY_MERGE_EXCEPT, op_ctx);
|
||||
}
|
||||
|
||||
int ObMergeExcept::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeExceptCtx* except_ctx = NULL;
|
||||
if (OB_FAIL(ObMergeSetOperator::inner_open(ctx))) {
|
||||
LOG_WARN("fail to open set operator", K(ret));
|
||||
} else if (OB_ISNULL(except_ctx = GET_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get except context", K(ctx), K_(id), K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeExcept::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeExceptCtx* except_ctx = NULL;
|
||||
except_ctx = GET_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id());
|
||||
if (OB_ISNULL(except_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got except context is NULL", K(ret));
|
||||
} else {
|
||||
except_ctx->reset();
|
||||
if (OB_FAIL(ObMergeSetOperator::rescan(ctx))) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeExcept::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObMergeSetOperator::inner_close(ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeExcept::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeExceptCtx* intersect_ctx = NULL;
|
||||
if (OB_ISNULL(get_next_row_func_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get_next_row_func_ is NULL", K(ret));
|
||||
} else if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get merge intersect context failed", K(ret));
|
||||
} else if (OB_FAIL((this->*get_next_row_func_)(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(copy_cur_row_by_projector(*intersect_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeExcept::distinct_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObNewRow* left_row = NULL;
|
||||
ObMergeExceptCtx* merge_except_ctx = NULL;
|
||||
ObPhyOperator* left_op = NULL;
|
||||
ObPhyOperator* right_op = NULL;
|
||||
|
||||
if (OB_ISNULL(merge_except_ctx = GET_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context", K(ctx), K_(id), K(ret));
|
||||
} else if (OB_ISNULL(left_op = get_child(FIRST_CHILD)) || OB_ISNULL(right_op = get_child(SECOND_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op or right_op is NULL", K(left_op), K(right_op), K(ret));
|
||||
} else {
|
||||
bool is_get_first_row = true;
|
||||
while (OB_SUCC(ret) && OB_SUCC(do_strict_distinct(*left_op, ctx, merge_except_ctx->last_output_row_, left_row))) {
|
||||
if (is_get_first_row) {
|
||||
if (OB_FAIL(merge_except_ctx->store_last_row(*left_row))) {
|
||||
LOG_WARN("fail to store last row", K(ret), K(*left_row));
|
||||
} else {
|
||||
is_get_first_row = false;
|
||||
}
|
||||
}
|
||||
break_outer_loop = merge_except_ctx->right_iter_end_;
|
||||
while (OB_SUCCESS == ret && !merge_except_ctx->right_iter_end_) {
|
||||
if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_row is NULL", K_(id), K(ret));
|
||||
} else if (NULL != merge_except_ctx->right_row_) {
|
||||
if (OB_FAIL(strict_compare(*left_row, *merge_except_ctx->right_row_, cmp))) {
|
||||
LOG_WARN("compare input_row with right_row failed", K(*left_row), K_(*merge_except_ctx->right_row), K(ret));
|
||||
} else if (cmp < 0) {
|
||||
// input_row is not in the right set, output input_row
|
||||
break_outer_loop = true;
|
||||
break;
|
||||
} else if (0 == cmp) {
|
||||
// break to get another left row
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, merge_except_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
merge_except_ctx->right_iter_end_ = true;
|
||||
merge_except_ctx->right_row_ = NULL;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
} else {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, merge_except_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
merge_except_ctx->right_iter_end_ = true;
|
||||
merge_except_ctx->right_row_ = NULL;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break; // cmp < 0 implement that the left_row is not in the right set, so output it
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_UNLIKELY(OB_SUCCESS != ret && OB_ITER_END != ret)) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else if (OB_SUCC(ret)) {
|
||||
row = left_row;
|
||||
if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is NULL", K(ret));
|
||||
} else if (OB_FAIL(merge_except_ctx->store_last_row(*row))) {
|
||||
LOG_WARN("fail to store last row", K(ret), K(*row));
|
||||
} else {
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
/**
|
||||
* the algorithm of get_next row:
|
||||
* state mechine, use two while loop to control the state transition
|
||||
* state 1: get next row from left operator, if in the end of iteration, exit,
|
||||
* otherwise, jump to state 2.
|
||||
* state 2: if the right_row doesn't exist, jump to state 3, otherwise, jump to state 4.
|
||||
* state 3: get next row from right operator, if in the end of iterator, output left_row,
|
||||
* otherwise, jump to state 4.
|
||||
* state 4: compare left_row with right_row, if left_row < right_row, jump to state 5,
|
||||
* if left_row = right_row, jump to state 6, otherwise, jump to state 7
|
||||
* state 5: output left row, exit
|
||||
* state 6: get next row from right operator, if in the end of iterator, right_iter_end_=true,
|
||||
* otherwise, jump to state 1.
|
||||
* state 7: get next row from right operator, if in the end of iterator, output left_row,
|
||||
* otherwise, jump to state 4.
|
||||
*/
|
||||
int ObMergeExcept::all_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObNewRow* left_row = NULL;
|
||||
ObMergeExceptCtx* merge_except_ctx = NULL;
|
||||
ObPhyOperator* left_op = NULL;
|
||||
ObPhyOperator* right_op = NULL;
|
||||
|
||||
if (OB_ISNULL(merge_except_ctx = GET_PHY_OPERATOR_CTX(ObMergeExceptCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context", K(ctx), K_(id), K(ret));
|
||||
} else if (OB_ISNULL(left_op = get_child(FIRST_CHILD)) || OB_ISNULL(right_op = get_child(SECOND_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op or right_op is NULL", K(left_op), K(right_op), K(ret));
|
||||
} else {
|
||||
while (OB_SUCCESS == ret && OB_SUCC(left_op->get_next_row(ctx, left_row))) {
|
||||
break_outer_loop = merge_except_ctx->right_iter_end_;
|
||||
while (OB_SUCCESS == ret && !merge_except_ctx->right_iter_end_) {
|
||||
if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left row is NULL", K(ret));
|
||||
} else if (NULL != merge_except_ctx->right_row_) {
|
||||
if (OB_FAIL(strict_compare(*left_row, *merge_except_ctx->right_row_, cmp))) {
|
||||
LOG_WARN("compare input_row with right_row failed", K(*left_row), K_(*merge_except_ctx->right_row), K(ret));
|
||||
} else if (cmp < 0) {
|
||||
// input_row is not in the right set, output input_row
|
||||
break_outer_loop = true;
|
||||
break;
|
||||
} else if (0 == cmp) {
|
||||
// break to get another left row
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, merge_except_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
merge_except_ctx->right_iter_end_ = true;
|
||||
merge_except_ctx->right_row_ = NULL;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, merge_except_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
merge_except_ctx->right_iter_end_ = true;
|
||||
merge_except_ctx->right_row_ = NULL;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break; // cmp < 0 implement that the left_row is not in the right set, so output it
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else if (OB_SUCC(ret)) {
|
||||
row = left_row;
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
71
src/sql/engine/set/ob_merge_except.h
Normal file
71
src/sql/engine/set/ob_merge_except.h
Normal file
@ -0,0 +1,71 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_MERGE_EXCEPT_H_
|
||||
#define OCEANBASE_SQL_ENGINE_MERGE_EXCEPT_H_
|
||||
|
||||
#include "common/row/ob_row.h"
|
||||
#include "sql/engine/set/ob_merge_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObMergeExcept : public ObMergeSetOperator {
|
||||
private:
|
||||
class ObMergeExceptCtx;
|
||||
|
||||
public:
|
||||
explicit ObMergeExcept(common::ObIAllocator& alloc);
|
||||
virtual ~ObMergeExcept();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
|
||||
virtual void set_distinct(bool is_distinct);
|
||||
|
||||
private:
|
||||
int distinct_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
int all_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) 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 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;
|
||||
/**
|
||||
* @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;
|
||||
|
||||
private:
|
||||
typedef int (ObMergeExcept::*GetNextRowFunc)(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
GetNextRowFunc get_next_row_func_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMergeExcept);
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_MERGE_EXCEPT_H_ */
|
||||
128
src/sql/engine/set/ob_merge_except_op.cpp
Normal file
128
src/sql/engine/set/ob_merge_except_op.cpp
Normal file
@ -0,0 +1,128 @@
|
||||
/**
|
||||
* 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_merge_except_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObMergeExceptSpec::ObMergeExceptSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObMergeSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMergeExceptSpec, ObMergeSetSpec));
|
||||
|
||||
ObMergeExceptOp::ObMergeExceptOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObMergeSetOp(exec_ctx, spec, input), right_iter_end_(false), first_got_right_row_(true)
|
||||
{}
|
||||
|
||||
int ObMergeExceptOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(nullptr == left_ || nullptr == right_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: left or right is null", K(ret));
|
||||
} else if (OB_FAIL(ObMergeSetOp::inner_open())) {
|
||||
LOG_WARN("failed to init open", K(ret));
|
||||
} else {
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeExceptOp::inner_close()
|
||||
{
|
||||
return ObMergeSetOp::inner_close();
|
||||
}
|
||||
|
||||
int ObMergeExceptOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObMergeSetOp::rescan())) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
} else {
|
||||
right_iter_end_ = false;
|
||||
first_got_right_row_ = true;
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMergeExceptOp::destroy()
|
||||
{
|
||||
ObMergeSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObMergeExceptOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObIArray<ObExpr*>* left_row = NULL;
|
||||
clear_evaluated_flag();
|
||||
while (OB_SUCC(ret) && OB_SUCC(do_strict_distinct(*left_, last_row_.store_row_, left_row))) {
|
||||
break_outer_loop = right_iter_end_;
|
||||
while (OB_SUCC(ret) && !right_iter_end_) {
|
||||
if (!first_got_right_row_) {
|
||||
if (OB_FAIL(cmp_(right_->get_spec().output_, *left_row, eval_ctx_, cmp))) {
|
||||
LOG_WARN("cmp_ input_row with right_row failed", K(*left_row), K(ret));
|
||||
} else if (cmp > 0) {
|
||||
// input_row is not in the right set, output input_row
|
||||
break_outer_loop = true;
|
||||
break;
|
||||
} else if (0 == cmp) {
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
right_iter_end_ = true;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ret));
|
||||
}
|
||||
} else {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
first_got_right_row_ = false;
|
||||
if (OB_FAIL(right_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
right_iter_end_ = true;
|
||||
break_outer_loop = true;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break; // cmp < 0 implement that the left_row is not in the right set, so output it
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(convert_row(*left_row, MY_SPEC.output_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
} else if (OB_FAIL(last_row_.save_store_row(*left_row, eval_ctx_, 0))) {
|
||||
LOG_WARN("failed to save right row", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
46
src/sql/engine/set/ob_merge_except_op.h
Normal file
46
src/sql/engine/set/ob_merge_except_op.h
Normal file
@ -0,0 +1,46 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_MERGE_EXCEPT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_MERGE_EXCEPT_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMergeExceptSpec : public ObMergeSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMergeExceptSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObMergeExceptOp : public ObMergeSetOp {
|
||||
public:
|
||||
ObMergeExceptOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
private:
|
||||
bool right_iter_end_;
|
||||
bool first_got_right_row_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_BASIC_OB_SET_OB_MERGE_EXCEPT_OP_H_ */
|
||||
300
src/sql/engine/set/ob_merge_intersect.cpp
Normal file
300
src/sql/engine/set/ob_merge_intersect.cpp
Normal file
@ -0,0 +1,300 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_merge_intersect.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
|
||||
namespace sql {
|
||||
class ObMergeIntersect::ObMergeIntersectCtx : public ObMergeSetOperatorCtx {
|
||||
public:
|
||||
explicit ObMergeIntersectCtx(ObExecContext& ctx)
|
||||
: ObMergeSetOperatorCtx(ctx), right_iter_end_(false), right_row_(NULL)
|
||||
{
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
void reset()
|
||||
{
|
||||
right_iter_end_ = false;
|
||||
right_row_ = NULL;
|
||||
need_skip_init_row_ = true;
|
||||
for (int64_t i = 0; i < last_output_row_.count_; ++i) {
|
||||
last_output_row_.cells_[i].set_null();
|
||||
}
|
||||
}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObMergeSetOperatorCtx::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
bool right_iter_end_;
|
||||
const ObNewRow* right_row_;
|
||||
|
||||
friend class ObMergeIntersect;
|
||||
};
|
||||
|
||||
ObMergeIntersect::ObMergeIntersect(common::ObIAllocator& alloc) : ObMergeSetOperator(alloc), get_next_row_func_(NULL)
|
||||
{}
|
||||
|
||||
ObMergeIntersect::~ObMergeIntersect()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
}
|
||||
|
||||
void ObMergeIntersect::reset()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reset();
|
||||
}
|
||||
|
||||
void ObMergeIntersect::reuse()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reuse();
|
||||
}
|
||||
|
||||
void ObMergeIntersect::set_distinct(bool is_distinct)
|
||||
{
|
||||
ObMergeSetOperator::set_distinct(is_distinct);
|
||||
if (is_distinct) {
|
||||
get_next_row_func_ = &ObMergeIntersect::distinct_get_next_row;
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeIntersect::all_get_next_row;
|
||||
}
|
||||
}
|
||||
|
||||
int ObMergeIntersect::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeIntersectCtx* intersect_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(get_next_row_func_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("merge_intersect not init", K(ret));
|
||||
} else if (OB_FAIL(ObMergeSetOperator::inner_open(ctx))) {
|
||||
LOG_WARN("failed to open double child operators", K(ret));
|
||||
} else if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_NULL_VALUE;
|
||||
LOG_WARN("fail to get merge intersect context", K(ctx), K_(id), K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersect::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeIntersectCtx* set_ctx = NULL;
|
||||
if (OB_ISNULL(set_ctx = GET_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to get operator ctx", K(ret));
|
||||
} else {
|
||||
set_ctx->reset();
|
||||
ret = ObMergeSetOperator::rescan(ctx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersect::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
return ObMergeSetOperator::inner_close(ctx);
|
||||
}
|
||||
|
||||
/*
|
||||
int ObMergeIntersect::close(ObExecContext &ctx) const
|
||||
{
|
||||
return ObSetOperator::close(ctx);
|
||||
}
|
||||
*/
|
||||
int ObMergeIntersect::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeIntersectCtx* intersect_ctx = NULL;
|
||||
if (OB_ISNULL(get_next_row_func_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("merge_intersect not init", K(ret));
|
||||
} else if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get merge intersect context failed", K(ret));
|
||||
} else if (OB_FAIL((this->*get_next_row_func_)(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(copy_cur_row_by_projector(*intersect_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersect::distinct_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObNewRow* left_row = NULL;
|
||||
ObMergeIntersectCtx* intersect_ctx = NULL;
|
||||
ObPhyOperator* left_op = NULL;
|
||||
ObPhyOperator* right_op = NULL;
|
||||
|
||||
if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context", K(ctx), K_(id), K(ret));
|
||||
} else if (OB_ISNULL(left_op = get_child(FIRST_CHILD)) || OB_ISNULL(right_op = get_child(SECOND_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op or right_op is NULL", K(left_op), K(right_op), K(ret));
|
||||
} else {
|
||||
bool is_get_first_row = true;
|
||||
while (OB_SUCCESS == ret && OB_SUCC(do_strict_distinct(*left_op, ctx, intersect_ctx->last_output_row_, left_row))) {
|
||||
if (is_get_first_row) {
|
||||
if (OB_FAIL(intersect_ctx->store_last_row(*left_row))) {
|
||||
LOG_WARN("fail to store last row", K(ret), K(*left_row));
|
||||
} else {
|
||||
is_get_first_row = false;
|
||||
}
|
||||
}
|
||||
while (OB_SUCCESS == ret && !intersect_ctx->right_iter_end_) {
|
||||
if (NULL != intersect_ctx->right_row_) {
|
||||
if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got left row is NULL", K(ret));
|
||||
} else if (OB_FAIL(strict_compare(*left_row, *intersect_ctx->right_row_, cmp))) {
|
||||
LOG_WARN("compare input_row with right_row failed", K(*left_row), K_(*intersect_ctx->right_row), K(ret));
|
||||
} else if (cmp < 0) {
|
||||
// input_row is not in the right set, break to get another left row
|
||||
break;
|
||||
} else if (0 == cmp) {
|
||||
break_outer_loop = true;
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, intersect_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
intersect_ctx->right_iter_end_ = true;
|
||||
intersect_ctx->right_row_ = NULL;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, intersect_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
intersect_ctx->right_iter_end_ = true;
|
||||
intersect_ctx->right_row_ = NULL;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else if (OB_SUCC(ret)) {
|
||||
row = left_row;
|
||||
if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got left_row is NULL", K(ret));
|
||||
} else if (OB_FAIL(intersect_ctx->store_last_row(*row))) {
|
||||
LOG_WARN("fail to store last row", K(*row), K(ret));
|
||||
} else {
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersect::all_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObNewRow* left_row = NULL;
|
||||
ObMergeIntersectCtx* intersect_ctx = NULL;
|
||||
ObPhyOperator* left_op = NULL;
|
||||
ObPhyOperator* right_op = NULL;
|
||||
|
||||
if (OB_ISNULL(intersect_ctx = GET_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context", K(ctx), K_(id), K(ret));
|
||||
} else if (OB_ISNULL(left_op = get_child(FIRST_CHILD)) || OB_ISNULL(right_op = get_child(SECOND_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op or right_op is NULL", K(left_op), K(right_op), K(ret));
|
||||
} else {
|
||||
while (OB_SUCCESS == ret && OB_SUCC(left_op->get_next_row(ctx, left_row))) {
|
||||
while (OB_SUCCESS == ret && !intersect_ctx->right_iter_end_) {
|
||||
if (NULL != intersect_ctx->right_row_) {
|
||||
if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got left row is NULL", K(ret));
|
||||
} else if (OB_FAIL(strict_compare(*left_row, *intersect_ctx->right_row_, cmp))) {
|
||||
LOG_WARN("compare input_row with right_row failed", K(*left_row), K_(*intersect_ctx->right_row), K(ret));
|
||||
} else if (cmp < 0) {
|
||||
// input_row is not in the right set, break to get another left row
|
||||
break;
|
||||
} else if (0 == cmp) {
|
||||
break_outer_loop = true;
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, intersect_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
intersect_ctx->right_iter_end_ = true;
|
||||
intersect_ctx->right_row_ = NULL;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
if (OB_FAIL(right_op->get_next_row(ctx, intersect_ctx->right_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
intersect_ctx->right_iter_end_ = true;
|
||||
intersect_ctx->right_row_ = NULL;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ctx), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else if (OB_SUCC(ret)) {
|
||||
row = left_row;
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersect::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObMergeIntersectCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
71
src/sql/engine/set/ob_merge_intersect.h
Normal file
71
src/sql/engine/set/ob_merge_intersect.h
Normal file
@ -0,0 +1,71 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_MERGE_INTERSECT_H_
|
||||
#define OCEANBASE_SQL_ENGINE_MERGE_INTERSECT_H_
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_operator.h"
|
||||
#include "common/row/ob_row.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObMergeIntersect : public ObMergeSetOperator {
|
||||
private:
|
||||
class ObMergeIntersectCtx;
|
||||
|
||||
public:
|
||||
explicit ObMergeIntersect(common::ObIAllocator& alloc);
|
||||
virtual ~ObMergeIntersect();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
|
||||
virtual void set_distinct(bool is_distinct);
|
||||
|
||||
private:
|
||||
int distinct_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
int all_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) 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 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;
|
||||
/**
|
||||
* @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;
|
||||
|
||||
private:
|
||||
typedef int (ObMergeIntersect::*GetNextRowFunc)(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
GetNextRowFunc get_next_row_func_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMergeIntersect);
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_MERGE_INTERSECT_H_ */
|
||||
119
src/sql/engine/set/ob_merge_intersect_op.cpp
Normal file
119
src/sql/engine/set/ob_merge_intersect_op.cpp
Normal file
@ -0,0 +1,119 @@
|
||||
/**
|
||||
* 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_merge_intersect_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObMergeIntersectSpec::ObMergeIntersectSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObMergeSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMergeIntersectSpec, ObMergeSetSpec));
|
||||
|
||||
ObMergeIntersectOp::ObMergeIntersectOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObMergeSetOp(exec_ctx, spec, input), right_iter_end_(false), first_got_right_row_(true)
|
||||
{}
|
||||
|
||||
int ObMergeIntersectOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(nullptr == left_ || nullptr == right_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: left or right is null", K(ret));
|
||||
} else if (OB_FAIL(ObMergeSetOp::inner_open())) {
|
||||
LOG_WARN("failed to init open", K(ret));
|
||||
} else {
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeIntersectOp::inner_close()
|
||||
{
|
||||
return ObMergeSetOp::inner_close();
|
||||
}
|
||||
|
||||
int ObMergeIntersectOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObMergeSetOp::rescan())) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
} else {
|
||||
right_iter_end_ = false;
|
||||
first_got_right_row_ = true;
|
||||
need_skip_init_row_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMergeIntersectOp::destroy()
|
||||
{
|
||||
ObMergeSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObMergeIntersectOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
bool break_outer_loop = false;
|
||||
const ObIArray<ObExpr*>* left_row = NULL;
|
||||
clear_evaluated_flag();
|
||||
while (OB_SUCC(ret) && OB_SUCC(do_strict_distinct(*left_, last_row_.store_row_, left_row))) {
|
||||
while (OB_SUCC(ret) && !right_iter_end_) {
|
||||
if (!first_got_right_row_) {
|
||||
if (OB_FAIL(cmp_(right_->get_spec().output_, *left_row, eval_ctx_, cmp))) {
|
||||
LOG_WARN("cmp_ input_row with right_row failed", K(*left_row), K(ret));
|
||||
} else if (cmp >= 0) {
|
||||
break_outer_loop = 0 == cmp ? true : false;
|
||||
break;
|
||||
} else {
|
||||
if (OB_FAIL(right_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
right_iter_end_ = true;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// get first row
|
||||
first_got_right_row_ = false;
|
||||
if (OB_FAIL(right_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
right_iter_end_ = true;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator row", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (break_outer_loop) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(convert_row(*left_row, MY_SPEC.output_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
} else if (OB_FAIL(last_row_.save_store_row(*left_row, eval_ctx_, 0))) {
|
||||
LOG_WARN("failed to save right row", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
46
src/sql/engine/set/ob_merge_intersect_op.h
Normal file
46
src/sql/engine/set/ob_merge_intersect_op.h
Normal file
@ -0,0 +1,46 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_MERGE_INTERSECT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_MERGE_INTERSECT_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMergeIntersectSpec : public ObMergeSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMergeIntersectSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObMergeIntersectOp : public ObMergeSetOp {
|
||||
public:
|
||||
ObMergeIntersectOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
private:
|
||||
bool right_iter_end_;
|
||||
bool first_got_right_row_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_BASIC_OB_SET_OB_MERGE_INTERSECT_OP_H_ */
|
||||
198
src/sql/engine/set/ob_merge_set_op.cpp
Normal file
198
src/sql/engine/set/ob_merge_set_op.cpp
Normal file
@ -0,0 +1,198 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObMergeSetSpec::ObMergeSetSpec(ObIAllocator& alloc, const ObPhyOperatorType type) : ObSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMergeSetSpec, ObSetSpec));
|
||||
|
||||
ObMergeSetOp::ObMergeSetOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
alloc_(ObModIds::OB_SQL_MERGE_GROUPBY, OB_MALLOC_NORMAL_BLOCK_SIZE, OB_SERVER_TENANT_ID, ObCtxIds::WORK_AREA),
|
||||
last_row_(alloc_),
|
||||
cmp_(),
|
||||
need_skip_init_row_(false)
|
||||
{}
|
||||
|
||||
int ObMergeSetOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(left_) || OB_ISNULL(right_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: left or right is null", K(ret), K(left_), K(right_));
|
||||
} else {
|
||||
const ObMergeSetSpec& spec = static_cast<const ObMergeSetSpec&>(get_spec());
|
||||
if (OB_FAIL(cmp_.init(&spec.sort_collations_, &spec.sort_cmp_funs_))) {
|
||||
LOG_WARN("failed to init compare function", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::inner_close()
|
||||
{
|
||||
return ObOperator::inner_close();
|
||||
}
|
||||
|
||||
int ObMergeSetOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
last_row_.reset();
|
||||
alloc_.reset();
|
||||
need_skip_init_row_ = false;
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMergeSetOp::destroy()
|
||||
{
|
||||
last_row_.reset();
|
||||
alloc_.reset();
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int ObMergeSetOp::do_strict_distinct(
|
||||
ObOperator& child_op, const ObIArray<ObExpr*>& compare_row, const ObIArray<ObExpr*>*& output_row, int& cmp)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_break = false;
|
||||
while (OB_SUCC(ret) && !is_break) {
|
||||
if (OB_FAIL(child_op.get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next row", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(cmp_(compare_row, child_op.get_spec().output_, eval_ctx_, cmp))) {
|
||||
LOG_WARN("strict compare with last_row failed", K(ret), K(compare_row));
|
||||
} else if (0 != cmp) {
|
||||
is_break = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
output_row = &child_op.get_spec().output_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::convert_row(const ObChunkDatumStore::StoredRow* sr, const common::ObIArray<ObExpr*>& exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(sr) || sr->cnt_ != exprs.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: store row is null or column count is not match", K(ret));
|
||||
} else {
|
||||
for (uint32_t i = 0; i < sr->cnt_; ++i) {
|
||||
exprs.at(i)->locate_expr_datum(eval_ctx_) = sr->cells()[i];
|
||||
exprs.at(i)->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
LOG_DEBUG("trace convert row", K(ret), K(sr->cnt_), K(ROWEXPR2STR(eval_ctx_, exprs)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::convert_row(const common::ObIArray<ObExpr*>& src_exprs, const common::ObIArray<ObExpr*>& dst_exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (dst_exprs.count() != src_exprs.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: exprs is not match", K(ret), K(src_exprs.count()), K(dst_exprs.count()));
|
||||
} else {
|
||||
ObDatum* src_datum = nullptr;
|
||||
for (uint32_t i = 0; i < dst_exprs.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(src_exprs.at(i)->eval(eval_ctx_, src_datum))) {
|
||||
LOG_WARN("failed to eval expr", K(ret), K(i));
|
||||
} else {
|
||||
dst_exprs.at(i)->locate_expr_datum(eval_ctx_) = *src_datum;
|
||||
dst_exprs.at(i)->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG(
|
||||
"trace convert row", K(ret), K(ROWEXPR2STR(eval_ctx_, dst_exprs)), K(ROWEXPR2STR(eval_ctx_, src_exprs)));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::Compare::init(
|
||||
const common::ObIArray<ObSortFieldCollation>* sort_collations, const common::ObIArray<common::ObCmpFunc>* cmp_funcs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(nullptr == sort_collations || nullptr == cmp_funcs) ||
|
||||
sort_collations->count() != cmp_funcs->count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("compare info is null", K(ret), K(sort_collations), K(cmp_funcs));
|
||||
} else {
|
||||
sort_collations_ = sort_collations;
|
||||
cmp_funcs_ = cmp_funcs;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::Compare::operator()(
|
||||
const ObChunkDatumStore::StoredRow& l, const common::ObIArray<ObExpr*>& r, ObEvalCtx& eval_ctx, int& cmp)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
cmp = 0;
|
||||
const ObDatum* lcells = l.cells();
|
||||
ObDatum* r_datum = nullptr;
|
||||
for (int64_t i = 0; i < sort_collations_->count() && OB_SUCC(ret); i++) {
|
||||
int64_t idx = sort_collations_->at(i).field_idx_;
|
||||
if (OB_FAIL(r.at(idx)->eval(eval_ctx, r_datum))) {
|
||||
LOG_WARN("failed to get expr value", K(ret), K(i));
|
||||
} else {
|
||||
cmp = cmp_funcs_->at(i).cmp_func_(lcells[idx], *r_datum);
|
||||
if (0 != cmp) {
|
||||
cmp = sort_collations_->at(i).is_ascending_ ? cmp : -cmp;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOp::Compare::operator()(
|
||||
const common::ObIArray<ObExpr*>& l, const common::ObIArray<ObExpr*>& r, ObEvalCtx& eval_ctx, int& cmp)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
cmp = 0;
|
||||
ObDatum* l_datum = nullptr;
|
||||
ObDatum* r_datum = nullptr;
|
||||
for (int64_t i = 0; i < sort_collations_->count() && OB_SUCC(ret); i++) {
|
||||
int64_t idx = sort_collations_->at(i).field_idx_;
|
||||
if (OB_FAIL(l.at(idx)->eval(eval_ctx, l_datum))) {
|
||||
LOG_WARN("failed to get expr value", K(ret), K(i));
|
||||
} else if (OB_FAIL(r.at(idx)->eval(eval_ctx, r_datum))) {
|
||||
} else {
|
||||
cmp = cmp_funcs_->at(i).cmp_func_(*l_datum, *r_datum);
|
||||
LOG_DEBUG(
|
||||
"debug compare merge set op", K(EXPR2STR(eval_ctx, *l.at(idx))), K(EXPR2STR(eval_ctx, *r.at(idx))), K(cmp));
|
||||
if (0 != cmp) {
|
||||
cmp = sort_collations_->at(i).is_ascending_ ? cmp : -cmp;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
119
src/sql/engine/set/ob_merge_set_op.h
Normal file
119
src/sql/engine/set/ob_merge_set_op.h
Normal file
@ -0,0 +1,119 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_MERGE_SET_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_MERGE_SET_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_set_op.h"
|
||||
#include "share/datum/ob_datum_funcs.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMergeSetSpec : public ObSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMergeSetSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObMergeSetOp : public ObOperator {
|
||||
public:
|
||||
ObMergeSetOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
class Compare {
|
||||
public:
|
||||
Compare() : sort_collations_(nullptr), cmp_funcs_(nullptr), ret_code_(common::OB_SUCCESS)
|
||||
{}
|
||||
int init(const common::ObIArray<ObSortFieldCollation>* sort_collations,
|
||||
const common::ObIArray<common::ObCmpFunc>* cmp_funcs);
|
||||
int operator()(
|
||||
const common::ObIArray<ObExpr*>& l, const common::ObIArray<ObExpr*>& r, ObEvalCtx& eval_ctx, int& cmp);
|
||||
int operator()(
|
||||
const ObChunkDatumStore::StoredRow& l, const common::ObIArray<ObExpr*>& r, ObEvalCtx& eval_ctx, int& cmp);
|
||||
const common::ObIArray<ObSortFieldCollation>* sort_collations_;
|
||||
const common::ObIArray<common::ObCmpFunc>* cmp_funcs_;
|
||||
int ret_code_;
|
||||
};
|
||||
|
||||
protected:
|
||||
int do_strict_distinct(ObOperator& child_op, const common::ObIArray<ObExpr*>& compare_row,
|
||||
const common::ObIArray<ObExpr*>*& row, int& cmp);
|
||||
template <typename T>
|
||||
int do_strict_distinct(ObOperator& child_op, const T* compare_row, const common::ObIArray<ObExpr*>*& output_row);
|
||||
|
||||
int convert_row(const ObChunkDatumStore::StoredRow* sr, const common::ObIArray<ObExpr*>& exprs);
|
||||
int convert_row(const common::ObIArray<ObExpr*>& src_exprs, const common::ObIArray<ObExpr*>& dst_exprs);
|
||||
|
||||
bool get_need_skip_init_row() const
|
||||
{
|
||||
return need_skip_init_row_;
|
||||
}
|
||||
void set_need_skip_init_row(bool need_skip_init_row)
|
||||
{
|
||||
need_skip_init_row_ = need_skip_init_row;
|
||||
}
|
||||
|
||||
protected:
|
||||
common::ObArenaAllocator alloc_;
|
||||
ObChunkDatumStore::LastStoredRow<> last_row_;
|
||||
Compare cmp_;
|
||||
bool need_skip_init_row_;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
int ObMergeSetOp::do_strict_distinct(
|
||||
ObOperator& child_op, const T* compare_row, const common::ObIArray<ObExpr*>*& output_row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp_ret = 0;
|
||||
bool is_break = false;
|
||||
while (OB_SUCC(ret) && !is_break) {
|
||||
if (OB_FAIL(child_op.get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SQL_ENG_LOG(WARN, "failed to get next row", K(ret));
|
||||
}
|
||||
} else if (OB_UNLIKELY(get_need_skip_init_row())) {
|
||||
set_need_skip_init_row(false);
|
||||
is_break = true;
|
||||
if (OB_NOT_NULL(compare_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "first row: compare row must be null", K(ret));
|
||||
} else if (OB_FAIL(last_row_.save_store_row(child_op.get_spec().output_, eval_ctx_, 0))) {
|
||||
SQL_ENG_LOG(WARN, "failed to save right row", K(ret));
|
||||
}
|
||||
} else if (OB_NOT_NULL(compare_row)) {
|
||||
if (OB_FAIL(cmp_(*compare_row, child_op.get_spec().output_, eval_ctx_, cmp_ret))) {
|
||||
SQL_ENG_LOG(WARN, "strict compare with last_row failed", K(ret), K(compare_row));
|
||||
} else if (0 != cmp_ret) {
|
||||
is_break = true;
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "unexpected status", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
output_row = &child_op.get_spec().output_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_MERGE_SET_OP_H_
|
||||
310
src/sql/engine/set/ob_merge_set_operator.cpp
Normal file
310
src/sql/engine/set/ob_merge_set_operator.cpp
Normal file
@ -0,0 +1,310 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_merge_set_operator.h"
|
||||
#include "common/object/ob_object.h"
|
||||
#include "sql/engine/expr/ob_expr_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
const int64_t ObMergeSetOperator::UNUSED_POS = -2;
|
||||
const int64_t ObMergeSetOperator::ObMergeSetOperatorCtx::OB_ROW_BUF_SIZE = OB_MAX_ROW_LENGTH;
|
||||
ObMergeSetOperator::ObMergeSetOperator(common::ObIAllocator& alloc)
|
||||
: ObSetOperator(alloc), set_directions_(alloc), cte_pseudo_column_row_desc_(alloc), map_array_(alloc)
|
||||
{}
|
||||
|
||||
ObMergeSetOperator::~ObMergeSetOperator()
|
||||
{}
|
||||
|
||||
void ObMergeSetOperator::reset()
|
||||
{
|
||||
set_directions_.reset();
|
||||
cte_pseudo_column_row_desc_.reset();
|
||||
map_array_.reset();
|
||||
ObSetOperator::reset();
|
||||
}
|
||||
|
||||
void ObMergeSetOperator::reuse()
|
||||
{
|
||||
set_directions_.reuse();
|
||||
cte_pseudo_column_row_desc_.reuse();
|
||||
map_array_.reuse();
|
||||
ObSetOperator::reuse();
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::init(int64_t count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObSetOperator::init(count))) {
|
||||
LOG_WARN("failed to init set operator", K(count), K(ret));
|
||||
} else if (OB_FAIL(init_set_directions(count))) {
|
||||
LOG_WARN("failed to init_set_directions", K(count), K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::init_cte_pseudo_column()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(cte_pseudo_column_row_desc_.init(CTE_PSEUDO_COLUMN_CNT))) {
|
||||
LOG_WARN("init cte_pseudo_column_row_desc_ failed");
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < CTE_PSEUDO_COLUMN_CNT; ++i) {
|
||||
if (OB_FAIL(cte_pseudo_column_row_desc_.push_back(ObMergeSetOperator::UNUSED_POS))) {
|
||||
LOG_WARN("fail to push back pos", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::set_map_array(const ObIArray<int64_t>& map_array)
|
||||
{
|
||||
return map_array_.assign(map_array);
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::strict_compare(const ObNewRow& row1, const ObNewRow& row2, int& cmp) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
cmp = 0;
|
||||
if (row1.is_invalid() || row2.is_invalid() || row1.get_count() != row2.get_count() ||
|
||||
row1.get_count() != cs_types_.count() || row1.get_count() != set_directions_.count()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument",
|
||||
K(row1.get_count()),
|
||||
K(row2.get_count()),
|
||||
"collation types",
|
||||
cs_types_.count(),
|
||||
"set_directions",
|
||||
set_directions_.count(),
|
||||
K(ret));
|
||||
}
|
||||
if (0 != map_array_.count()) {
|
||||
if (map_array_.count() != row1.get_count() || map_array_.count() != row2.get_count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected error", K(map_array_), K(row1), K(row2));
|
||||
} else {
|
||||
int64_t pos = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row1.get_count(); ++i) {
|
||||
pos = map_array_.at(i);
|
||||
if ((cmp = row1.get_cell(pos).compare(row2.get_cell(pos), cs_types_.at(pos))) != 0) {
|
||||
// take consideration of order_type of columns
|
||||
cmp *= set_directions_.at(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row1.get_count(); ++i) {
|
||||
if ((cmp = row1.get_cell(i).compare(row2.get_cell(i), cs_types_.at(i))) != 0) {
|
||||
// take consideration of order_type of columns
|
||||
cmp *= set_directions_.at(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::strict_compare(const ObNewRow& row1, const ObNewRow& row2, bool& equal) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
|
||||
if (OB_FAIL(strict_compare(row1, row2, cmp))) {
|
||||
LOG_WARN("strict compare row failed", K(row1), K(row2));
|
||||
} else {
|
||||
equal = (cmp == 0);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::do_strict_distinct(
|
||||
ObPhyOperator& child_op, ObExecContext& ctx, const ObNewRow& compare_row, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool equal = false;
|
||||
bool is_break = false;
|
||||
ObMergeSetOperatorCtx* set_ctx = GET_PHY_OPERATOR_CTX(ObMergeSetOperatorCtx, ctx, get_id());
|
||||
if (nullptr == set_ctx) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to get physical operator", K(ctx), K_(id));
|
||||
}
|
||||
while (OB_SUCC(ret) && !is_break && OB_SUCC(child_op.get_next_row(ctx, row))) {
|
||||
if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null");
|
||||
} else if (OB_UNLIKELY(set_ctx->get_need_skip_init_row())) {
|
||||
set_ctx->set_need_skip_init_row(false);
|
||||
is_break = true;
|
||||
} else if (OB_FAIL(strict_compare(compare_row, *row, equal))) {
|
||||
LOG_WARN("strict compare with last_row failed", K(ret), K(compare_row), K(*row));
|
||||
} else if (!equal) {
|
||||
is_break = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::do_strict_distinct(
|
||||
ObPhyOperator& child_op, ObExecContext& ctx, const ObNewRow& compare_row, const ObNewRow*& row, int& cmp) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_break = false;
|
||||
while (OB_SUCC(ret) && !is_break && OB_SUCC(child_op.get_next_row(ctx, row))) {
|
||||
if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null");
|
||||
} else if (OB_FAIL(strict_compare(compare_row, *row, cmp))) {
|
||||
LOG_WARN("strict compare with last_row failed", K(ret), K(compare_row), K(*row));
|
||||
} else if (0 != cmp) {
|
||||
is_break = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeSetOperatorCtx* set_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(get_child(FIRST_CHILD))) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("left op is null");
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to init operator context", K(ret));
|
||||
} else if (OB_ISNULL(set_ctx = GET_PHY_OPERATOR_CTX(ObMergeSetOperatorCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to get physical operator", K(ctx), K_(id));
|
||||
} else if (is_distinct()) {
|
||||
if (OB_FAIL(set_ctx->alloc_last_row_buf(get_child(FIRST_CHILD)->get_output_count()))) {
|
||||
LOG_WARN("failed to alloc last row buffer", K(ret), "column_count", get_child(FIRST_CHILD)->get_output_count());
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeSetOperatorCtx* set_ctx = NULL;
|
||||
if (OB_ISNULL(set_ctx = GET_PHY_OPERATOR_CTX(ObMergeSetOperatorCtx, 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 {
|
||||
set_ctx->free_last_row_buf();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::add_set_direction(ObOrderDirection direction_)
|
||||
{
|
||||
|
||||
int32_t set_direction = (is_ascending_direction(direction_) ? static_cast<int32_t>(CMP_DIRECTION_ASC)
|
||||
: static_cast<int32_t>(CMP_DIRECTION_DESC));
|
||||
return set_directions_.push_back(set_direction);
|
||||
}
|
||||
|
||||
int64_t ObMergeSetOperator::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_KV(N_DISTINCT,
|
||||
distinct_,
|
||||
"collation_types",
|
||||
cs_types_,
|
||||
"set_directions_",
|
||||
set_directions_,
|
||||
"cte_pseido_column_",
|
||||
cte_pseudo_column_row_desc_,
|
||||
"map_array_",
|
||||
map_array_);
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObMergeSetOperator::add_cte_pseudo_column(const ObPseudoColumnRawExpr* expr, int64_t pos)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(expr) || pos < 0 || cte_pseudo_column_row_desc_.count() != ObCTEPseudoColumn::CTE_PSEUDO_COLUMN_CNT) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid parameter", K(pos), KPC(expr), K(cte_pseudo_column_row_desc_.count()), K(ret));
|
||||
} else {
|
||||
ObItemType expr_type = expr->get_expr_type();
|
||||
switch (expr_type) {
|
||||
case T_CTE_SEARCH_COLUMN:
|
||||
cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_SEARCH] = pos;
|
||||
break;
|
||||
case T_CTE_CYCLE_COLUMN:
|
||||
cte_pseudo_column_row_desc_[ObCTEPseudoColumn::CTE_CYCLE] = pos;
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid expr", KPC(expr), K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObMergeSetOperator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_UNIS_ENCODE(distinct_);
|
||||
OB_UNIS_ENCODE(cs_types_);
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObPhyOperator::serialize(buf, buf_len, pos);
|
||||
}
|
||||
OB_UNIS_ENCODE(set_directions_);
|
||||
OB_UNIS_ENCODE(cte_pseudo_column_row_desc_);
|
||||
OB_UNIS_ENCODE(map_array_);
|
||||
OB_UNIS_ENCODE(child_num_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObMergeSetOperator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
child_num_ = 2; // for compatibility, set a default two child op value
|
||||
bool is_distinct = false;
|
||||
OB_UNIS_DECODE(is_distinct);
|
||||
OB_UNIS_DECODE(cs_types_);
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObPhyOperator::deserialize(buf, data_len, pos);
|
||||
}
|
||||
OB_UNIS_DECODE(set_directions_);
|
||||
OB_UNIS_DECODE(cte_pseudo_column_row_desc_);
|
||||
OB_UNIS_DECODE(map_array_);
|
||||
OB_UNIS_DECODE(child_num_);
|
||||
if (OB_SUCC(ret)) {
|
||||
set_distinct(is_distinct);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObMergeSetOperator)
|
||||
{
|
||||
int64_t len = 0;
|
||||
OB_UNIS_ADD_LEN(distinct_);
|
||||
OB_UNIS_ADD_LEN(cs_types_);
|
||||
len += ObPhyOperator::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(set_directions_);
|
||||
OB_UNIS_ADD_LEN(cte_pseudo_column_row_desc_);
|
||||
OB_UNIS_ADD_LEN(map_array_);
|
||||
OB_UNIS_ADD_LEN(child_num_);
|
||||
return len;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
169
src/sql/engine/set/ob_merge_set_operator.h
Normal file
169
src/sql/engine/set/ob_merge_set_operator.h
Normal file
@ -0,0 +1,169 @@
|
||||
/**
|
||||
* 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_MERGE_SET_OPERATOR_H_
|
||||
#define OCEANBASE_SQL_OB_MERGE_SET_OPERATOR_H_
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_double_children_phy_operator.h"
|
||||
#include "sql/engine/set/ob_set_operator.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
enum ObCTEPseudoColumn { CTE_SEARCH = 0, CTE_CYCLE = 1, CTE_PSEUDO_COLUMN_CNT = 2 };
|
||||
|
||||
class ObMergeSetOperator : public ObSetOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
protected:
|
||||
class ObMergeSetOperatorCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObMergeSetOperatorCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx), last_output_row_(), last_row_buf_(nullptr), need_skip_init_row_(false)
|
||||
{}
|
||||
|
||||
inline int alloc_last_row_buf(const int64_t column_count)
|
||||
{
|
||||
void* ptr = NULL;
|
||||
int ret = common::OB_SUCCESS;
|
||||
int64_t row_size = column_count * sizeof(common::ObObj);
|
||||
if (OB_UNLIKELY(NULL == (last_row_buf_ = exec_ctx_.get_allocator().alloc(OB_ROW_BUF_SIZE)))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_ENG_LOG(ERROR, "alloc last row buffer failed", K(OB_ROW_BUF_SIZE));
|
||||
} else if (OB_UNLIKELY(NULL == (ptr = exec_ctx_.get_allocator().alloc(row_size)))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_ENG_LOG(ERROR, "alloc last row cell memory failed", K(row_size));
|
||||
} else {
|
||||
last_output_row_.cells_ = new (ptr) common::ObObj[column_count];
|
||||
last_output_row_.count_ = column_count;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int store_last_row(const common::ObNewRow& row)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
common::ObDataBuffer data_buf(static_cast<char*>(last_row_buf_), OB_ROW_BUF_SIZE);
|
||||
|
||||
if (row.is_invalid() || last_output_row_.is_invalid() || row.get_count() > last_output_row_.get_count()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid_argument", K(row), K_(last_output_row));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row.get_count(); ++i) {
|
||||
if (OB_FAIL(common::ob_write_obj(data_buf, row.get_cell(i), last_output_row_.cells_[i]))) {
|
||||
SQL_ENG_LOG(WARN, "write obj failed", K(ret), K(i), K(row));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
void free_last_row_buf()
|
||||
{
|
||||
if (last_row_buf_) {
|
||||
exec_ctx_.get_allocator().free(last_row_buf_);
|
||||
last_row_buf_ = NULL;
|
||||
}
|
||||
if (last_output_row_.cells_) {
|
||||
exec_ctx_.get_allocator().free(last_output_row_.cells_);
|
||||
last_output_row_.cells_ = NULL;
|
||||
}
|
||||
}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
inline bool get_need_skip_init_row()
|
||||
{
|
||||
return need_skip_init_row_;
|
||||
}
|
||||
inline void set_need_skip_init_row(bool b)
|
||||
{
|
||||
need_skip_init_row_ = b;
|
||||
}
|
||||
|
||||
protected:
|
||||
common::ObNewRow last_output_row_;
|
||||
void* last_row_buf_;
|
||||
static const int64_t OB_ROW_BUF_SIZE;
|
||||
bool need_skip_init_row_;
|
||||
};
|
||||
|
||||
public:
|
||||
explicit ObMergeSetOperator(common::ObIAllocator& alloc);
|
||||
~ObMergeSetOperator();
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
int init(int64_t count);
|
||||
|
||||
int add_set_direction(ObOrderDirection direction);
|
||||
int add_cte_pseudo_column(const ObPseudoColumnRawExpr* expr, int64_t pos);
|
||||
int init_cte_pseudo_column();
|
||||
int set_map_array(const ObIArray<int64_t>& map_array);
|
||||
|
||||
protected:
|
||||
int init_set_directions(int64_t count);
|
||||
int strict_compare(const common::ObNewRow& row1, const common::ObNewRow& row2, int& cmp) const;
|
||||
int strict_compare(const common::ObNewRow& row1, const common::ObNewRow& row2, bool& equal) const;
|
||||
/**
|
||||
* @brief get a row distinct with the specified row
|
||||
* @param child_op[in], the child operator
|
||||
* @param compare_row[in], the specified row
|
||||
* @param row[out], the output row
|
||||
* @return if success, return OB_SUCCESS,
|
||||
* if iterator end, return OB_ITER_END, otherwise, return errno
|
||||
*/
|
||||
int do_strict_distinct(ObPhyOperator& child_op, ObExecContext& ctx, const common::ObNewRow& compare_row,
|
||||
const common::ObNewRow*& row) const;
|
||||
int do_strict_distinct(ObPhyOperator& child_op, ObExecContext& ctx, const common::ObNewRow& compare_row,
|
||||
const common::ObNewRow*& row, int& cmp) const;
|
||||
/**
|
||||
* @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 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;
|
||||
// disallow copy
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMergeSetOperator);
|
||||
|
||||
public:
|
||||
static const int64_t UNUSED_POS;
|
||||
|
||||
protected:
|
||||
static const int32_t CMP_DIRECTION_ASC = 1;
|
||||
static const int32_t CMP_DIRECTION_DESC = -1;
|
||||
common::ObFixedArray<int32_t, common::ObIAllocator> set_directions_;
|
||||
common::ObFixedArray<int64_t, common::ObIAllocator> cte_pseudo_column_row_desc_;
|
||||
common::ObFixedArray<int64_t, common::ObIAllocator> map_array_;
|
||||
};
|
||||
inline int ObMergeSetOperator::init_set_directions(int64_t count)
|
||||
{
|
||||
return init_array_size<>(set_directions_, count);
|
||||
}
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_OB_SET_OPERATOR_H_ */
|
||||
362
src/sql/engine/set/ob_merge_union.cpp
Normal file
362
src/sql/engine/set/ob_merge_union.cpp
Normal file
@ -0,0 +1,362 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/set/ob_merge_union.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "lib/allocator/ob_malloc.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "common/object/ob_object.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
//#include "sql/engine/expr/ob_expr_promotion_util.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
// REGISTER_PHY_OPERATOR(ObMergeUnion, PHY_MERGE_UNION);
|
||||
|
||||
class ObMergeUnion::ObMergeUnionCtx : public ObMergeSetOperatorCtx {
|
||||
public:
|
||||
explicit ObMergeUnionCtx(ObExecContext& ctx)
|
||||
: ObMergeSetOperatorCtx(ctx),
|
||||
cur_child_op_(NULL),
|
||||
next_child_op_idx_(1),
|
||||
got_first_row_(false),
|
||||
candidate_output_row_(NULL),
|
||||
candidate_child_op_(NULL)
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObMergeSetOperatorCtx::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
ObPhyOperator* cur_child_op_;
|
||||
int64_t next_child_op_idx_;
|
||||
bool got_first_row_;
|
||||
const ObNewRow* candidate_output_row_;
|
||||
ObPhyOperator* candidate_child_op_;
|
||||
|
||||
friend class ObMergeUnion;
|
||||
};
|
||||
|
||||
ObMergeUnion::ObMergeUnion(common::ObIAllocator& alloc) : ObMergeSetOperator(alloc), get_next_row_func_(NULL)
|
||||
{}
|
||||
|
||||
ObMergeUnion::~ObMergeUnion()
|
||||
{}
|
||||
|
||||
void ObMergeUnion::reset()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reset();
|
||||
}
|
||||
|
||||
void ObMergeUnion::reuse()
|
||||
{
|
||||
get_next_row_func_ = NULL;
|
||||
ObMergeSetOperator::reuse();
|
||||
}
|
||||
|
||||
int ObMergeUnion::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
|
||||
{
|
||||
return CREATE_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id(), get_type(), op_ctx);
|
||||
}
|
||||
|
||||
int ObMergeUnion::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(inner_create_operator_ctx(ctx, op_ctx))) {
|
||||
LOG_WARN("create p == hysical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got op_ctx is NULL", K(ret));
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/*
|
||||
* When UNION ALL, we get results from two query one by one
|
||||
*/
|
||||
int ObMergeUnion::all_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeUnionCtx* merge_union_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(merge_union_ctx = GET_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(ctx), K_(id));
|
||||
} else if (OB_ISNULL(merge_union_ctx->cur_child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(ctx), K_(id));
|
||||
} else if (OB_FAIL(merge_union_ctx->cur_child_op_->get_next_row(ctx, row))) {
|
||||
// get next row with the next child operator
|
||||
while (OB_ITER_END == ret && merge_union_ctx->next_child_op_idx_ < get_child_num()) {
|
||||
merge_union_ctx->cur_child_op_ = get_child(merge_union_ctx->next_child_op_idx_);
|
||||
++merge_union_ctx->next_child_op_idx_;
|
||||
if (OB_ISNULL(merge_union_ctx->cur_child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(ctx), K_(id));
|
||||
} else {
|
||||
ret = merge_union_ctx->cur_child_op_->get_next_row(ctx, row);
|
||||
}
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnion::get_first_row(ObExecContext& ctx, ObMergeUnionCtx& union_ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
const ObNewRow* left_row = NULL;
|
||||
const ObNewRow* right_row = NULL;
|
||||
ObPhyOperator* left_op = NULL;
|
||||
ObPhyOperator* right_op = NULL;
|
||||
if (OB_ISNULL(left_op = get_child(FIRST_CHILD)) || OB_ISNULL(right_op = get_child(SECOND_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("left_op or right_op is NULL", K(left_op), K(right_op), K(ret));
|
||||
} else if (OB_FAIL(left_op->get_next_row(ctx, left_row))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
// switch to the right operator
|
||||
union_ctx.candidate_output_row_ = NULL;
|
||||
union_ctx.candidate_child_op_ = NULL;
|
||||
union_ctx.cur_child_op_ = right_op;
|
||||
ret = union_ctx.cur_child_op_->get_next_row(ctx, row);
|
||||
if (OB_SUCCESS != ret && OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next row", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (OB_ISNULL(left_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("got left_row is NULL", K(ret));
|
||||
} else if (OB_FAIL(do_strict_distinct(*right_op, ctx, *left_row, right_row, cmp))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
union_ctx.candidate_child_op_ = NULL;
|
||||
union_ctx.candidate_output_row_ = NULL;
|
||||
union_ctx.cur_child_op_ = left_op;
|
||||
row = left_row;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator's row", K(ret));
|
||||
}
|
||||
} else {
|
||||
union_ctx.candidate_child_op_ = cmp < 0 ? right_op : left_op;
|
||||
union_ctx.candidate_output_row_ = cmp < 0 ? right_row : left_row;
|
||||
union_ctx.cur_child_op_ = cmp < 0 ? left_op : right_op;
|
||||
row = cmp < 0 ? left_row : right_row;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
/**
|
||||
* When UNION DISTINCT, we consider that left and right query already in ordered.
|
||||
* cur_child_operator: get current row from this operator in the first place
|
||||
* candidate_output_row && candidate_child_operator: the candidate output row from the candidate
|
||||
* child operator
|
||||
*
|
||||
* in distinct_get_nexr_row, we get the next row from the cur_child_operator as input_row at first,
|
||||
* input_row is distinct with the last output row
|
||||
* if in the end of the cur_child_opertor iterator, we must output the candidate_output_row
|
||||
* as the current row and switch the cur_child_operator to candidate_child_operator to get next row,
|
||||
*
|
||||
* if in the end of the candidate_child_operator iterator, we only need to get next row from the
|
||||
* cur_child_operator, and don't need to compare with candidate_child_operator's row
|
||||
*
|
||||
* if cur_child_operator and candidate_child_operator are present, we need to compare input_row
|
||||
* with the candidate_output_row, if input_row is less than candidate_output_row, return input_row
|
||||
* as the result, if input_row equal to candidate_output_row, return input_row as the result and
|
||||
* get the distinct candidate_output_row from the candidate_child_operator, otherwise, return
|
||||
* candidate_output_row as the result and switch candidate_child_operator with cur_child_operator
|
||||
* for the next iteration
|
||||
*/
|
||||
int ObMergeUnion::distinct_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
const ObNewRow* input_row = NULL;
|
||||
ObMergeUnionCtx* merge_union_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(merge_union_ctx = GET_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(ctx), K_(id));
|
||||
} else {
|
||||
if (merge_union_ctx->got_first_row_) {
|
||||
int cur_child_err = OB_SUCCESS;
|
||||
int candidate_child_err = OB_SUCCESS;
|
||||
if (OB_ISNULL(merge_union_ctx->cur_child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cur_child_op is NULL", K(ret));
|
||||
} else if (OB_UNLIKELY(
|
||||
(OB_SUCCESS !=
|
||||
(cur_child_err = do_strict_distinct(
|
||||
*merge_union_ctx->cur_child_op_, ctx, merge_union_ctx->last_output_row_, input_row))))) {
|
||||
if (OB_ITER_END == cur_child_err) {
|
||||
if (OB_LIKELY(NULL != merge_union_ctx->candidate_child_op_)) {
|
||||
// current operator in the end of iterator, so switch to the candidate operator
|
||||
merge_union_ctx->cur_child_op_ = merge_union_ctx->candidate_child_op_;
|
||||
merge_union_ctx->candidate_child_op_ = NULL;
|
||||
row = merge_union_ctx->candidate_output_row_;
|
||||
merge_union_ctx->candidate_output_row_ = NULL;
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
} else {
|
||||
ret = cur_child_err;
|
||||
LOG_WARN("failed to do_strict_distinct", K(ret));
|
||||
}
|
||||
} else if (NULL == merge_union_ctx->candidate_child_op_) {
|
||||
row = input_row;
|
||||
} else if (OB_UNLIKELY(NULL == input_row || NULL == merge_union_ctx->candidate_output_row_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("input row is NULL or candidate_output_row_ is NULL",
|
||||
K(input_row),
|
||||
K(merge_union_ctx->candidate_output_row_),
|
||||
K(ret));
|
||||
} else if (OB_FAIL(strict_compare(*input_row, *merge_union_ctx->candidate_output_row_, cmp))) {
|
||||
LOG_WARN("compatible compare failed", K(ret));
|
||||
} else if (0 == cmp) {
|
||||
// left row equal to right row
|
||||
row = input_row;
|
||||
|
||||
candidate_child_err = do_strict_distinct(
|
||||
*merge_union_ctx->candidate_child_op_, ctx, *input_row, merge_union_ctx->candidate_output_row_);
|
||||
|
||||
if (OB_SUCCESS != candidate_child_err) {
|
||||
if (OB_ITER_END == candidate_child_err) {
|
||||
// candidate operator in the end of row iteration, candidate operator not exist
|
||||
merge_union_ctx->candidate_child_op_ = NULL;
|
||||
merge_union_ctx->candidate_output_row_ = NULL;
|
||||
} else {
|
||||
ret = candidate_child_err;
|
||||
LOG_WARN("candidate child operator get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (cmp < 0) {
|
||||
// output current row
|
||||
row = input_row;
|
||||
} else if (cmp > 0) {
|
||||
// output candidate row and switch candidate operator to current operator for next iteration
|
||||
ObPhyOperator* tmp_op = NULL;
|
||||
row = merge_union_ctx->candidate_output_row_;
|
||||
merge_union_ctx->candidate_output_row_ = input_row;
|
||||
tmp_op = merge_union_ctx->candidate_child_op_;
|
||||
merge_union_ctx->candidate_child_op_ = merge_union_ctx->cur_child_op_;
|
||||
merge_union_ctx->cur_child_op_ = tmp_op;
|
||||
} else {
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(merge_union_ctx->store_last_row(*row))) {
|
||||
LOG_WARN("storage current row for next compare failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// the first row, not need to compare the row
|
||||
// first, get next row
|
||||
if (OB_FAIL(get_first_row(ctx, *merge_union_ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get first row failed", K(ret));
|
||||
}
|
||||
}
|
||||
// second, storage current row
|
||||
if (OB_SUCC(ret)) {
|
||||
merge_union_ctx->got_first_row_ = true;
|
||||
if (OB_FAIL(merge_union_ctx->store_last_row(*row))) {
|
||||
LOG_WARN("storage current row for next compare failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMergeUnion::set_distinct(bool is_distinct)
|
||||
{
|
||||
ObMergeSetOperator::set_distinct(is_distinct);
|
||||
if (is_distinct) {
|
||||
get_next_row_func_ = &ObMergeUnion::distinct_get_next_row;
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeUnion::all_get_next_row;
|
||||
}
|
||||
}
|
||||
|
||||
int ObMergeUnion::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeUnionCtx* merge_union_ctx = NULL;
|
||||
|
||||
if (OB_FAIL(ObMergeSetOperator::inner_open(ctx))) {
|
||||
LOG_WARN("failed to open set operator", K(ret));
|
||||
} else if (OB_ISNULL(merge_union_ctx = GET_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ctx), K_(id), K(ret));
|
||||
} else {
|
||||
merge_union_ctx->cur_child_op_ = get_child(FIRST_CHILD);
|
||||
merge_union_ctx->next_child_op_idx_ = 1;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnion::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeUnionCtx* union_ctx = NULL;
|
||||
if (OB_ISNULL(union_ctx = GET_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN(" failed to get operator context ", K(ret));
|
||||
} else {
|
||||
union_ctx->cur_child_op_ = get_child(FIRST_CHILD);
|
||||
union_ctx->next_child_op_idx_ = 1;
|
||||
union_ctx->got_first_row_ = false;
|
||||
union_ctx->candidate_output_row_ = NULL;
|
||||
union_ctx->candidate_child_op_ = NULL;
|
||||
ret = ObMergeSetOperator::rescan(ctx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnion::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
return ObMergeSetOperator::inner_close(ctx);
|
||||
}
|
||||
|
||||
int ObMergeUnion::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMergeUnionCtx* union_ctx = NULL;
|
||||
if (OB_ISNULL(get_next_row_func_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("get_next_row_func is NULL", K(ret));
|
||||
} else if (OB_ISNULL(union_ctx = GET_PHY_OPERATOR_CTX(ObMergeUnionCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get merge intersect context failed");
|
||||
} else if (OB_FAIL((this->*get_next_row_func_)(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(copy_cur_row_by_projector(*union_ctx, row))) {
|
||||
LOG_WARN("copy current row with projector failed", K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
88
src/sql/engine/set/ob_merge_union.h
Normal file
88
src/sql/engine/set/ob_merge_union.h
Normal file
@ -0,0 +1,88 @@
|
||||
/**
|
||||
* 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_ENGINE_MERGE_UNION_H_
|
||||
#define OCEANBASE_SQL_ENGINE_MERGE_UNION_H_
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace common {
|
||||
class ObNewRow;
|
||||
}
|
||||
namespace sql {
|
||||
class ObMergeUnion : public ObMergeSetOperator {
|
||||
private:
|
||||
class ObMergeUnionCtx;
|
||||
|
||||
public:
|
||||
explicit ObMergeUnion(common::ObIAllocator& alloc);
|
||||
virtual ~ObMergeUnion();
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
virtual void set_distinct(bool is_distinct);
|
||||
|
||||
private:
|
||||
int get_first_row(ObExecContext& ctx, ObMergeUnionCtx& union_ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief get next row expected the same row in the same group
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], output row
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int distinct_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief the function to get next row without distinct attribution
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], output row
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int all_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) 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;
|
||||
virtual int init_op_ctx(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;
|
||||
/**
|
||||
* @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;
|
||||
|
||||
typedef int (ObMergeUnion::*GetNextRowFunc)(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
private:
|
||||
GetNextRowFunc get_next_row_func_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMergeUnion);
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_MERGE_UNION_H_ */
|
||||
294
src/sql/engine/set/ob_merge_union_op.cpp
Normal file
294
src/sql/engine/set/ob_merge_union_op.cpp
Normal file
@ -0,0 +1,294 @@
|
||||
/**
|
||||
* 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_merge_union_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObMergeUnionSpec::ObMergeUnionSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObMergeSetSpec(alloc, type)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMergeUnionSpec, ObMergeSetSpec));
|
||||
|
||||
ObMergeUnionOp::ObMergeUnionOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObMergeSetOp(exec_ctx, spec, input),
|
||||
cur_child_op_(nullptr),
|
||||
candidate_child_op_(nullptr),
|
||||
candidate_output_row_(nullptr),
|
||||
next_child_op_idx_(1),
|
||||
first_got_row_(true),
|
||||
get_next_row_func_(nullptr)
|
||||
{}
|
||||
|
||||
int ObMergeUnionOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObMergeSetOp::inner_open())) {
|
||||
LOG_WARN("failed to init open", K(ret));
|
||||
} else {
|
||||
cur_child_op_ = left_;
|
||||
next_child_op_idx_ = 1;
|
||||
if (MY_SPEC.is_distinct_) {
|
||||
if (OB_UNLIKELY(2 != get_child_cnt())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected merge union distinct", K(ret), K(get_child_cnt()));
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeUnionOp::distinct_get_next_row;
|
||||
}
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeUnionOp::all_get_next_row;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnionOp::inner_close()
|
||||
{
|
||||
return ObMergeSetOp::inner_close();
|
||||
}
|
||||
|
||||
int ObMergeUnionOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
cur_child_op_ = left_;
|
||||
next_child_op_idx_ = 1;
|
||||
first_got_row_ = true;
|
||||
candidate_output_row_ = nullptr;
|
||||
candidate_child_op_ = nullptr;
|
||||
if (MY_SPEC.is_distinct_) {
|
||||
get_next_row_func_ = &ObMergeUnionOp::distinct_get_next_row;
|
||||
} else {
|
||||
get_next_row_func_ = &ObMergeUnionOp::all_get_next_row;
|
||||
}
|
||||
if (OB_FAIL(ObMergeSetOp::rescan())) {}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMergeUnionOp::destroy()
|
||||
{
|
||||
ObMergeSetOp::destroy();
|
||||
}
|
||||
|
||||
int ObMergeUnionOp::get_first_row(const ObIArray<ObExpr*>*& output_row)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
const ObIArray<ObExpr*>* right_row = nullptr;
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(left_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
// switch to the right operator
|
||||
candidate_output_row_ = NULL;
|
||||
candidate_child_op_ = NULL;
|
||||
cur_child_op_ = right_;
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(cur_child_op_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
output_row = &cur_child_op_->get_spec().output_;
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(do_strict_distinct(*right_, left_->get_spec().output_, right_row, cmp))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
candidate_child_op_ = NULL;
|
||||
candidate_output_row_ = NULL;
|
||||
cur_child_op_ = left_;
|
||||
output_row = &left_->get_spec().output_;
|
||||
} else {
|
||||
LOG_WARN("fail to get right operator's row", K(ret));
|
||||
}
|
||||
} else {
|
||||
candidate_child_op_ = cmp < 0 ? right_ : left_;
|
||||
candidate_output_row_ = cmp < 0 ? &right_->get_spec().output_ : &left_->get_spec().output_;
|
||||
cur_child_op_ = cmp < 0 ? left_ : right_;
|
||||
output_row = cmp < 0 ? &left_->get_spec().output_ : &right_->get_spec().output_;
|
||||
LOG_DEBUG("trace first row", K(ROWEXPR2STR(eval_ctx_, *output_row)), K(cmp), K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* copy from ob_merge_union.cpp
|
||||
* When UNION DISTINCT, we consider that left and right query already in ordered.
|
||||
* cur_child_operator: get current row from this operator in the first place
|
||||
* candidate_output_row && candidate_child_operator: the candidate output row from the candidate
|
||||
* child operator
|
||||
*
|
||||
* in distinct_get_nexr_row, we get the next row from the cur_child_operator as input_row at first,
|
||||
* input_row is distinct with the last output row
|
||||
* if in the end of the cur_child_opertor iterator, we must output the candidate_output_row
|
||||
* as the current row and switch the cur_child_operator to candidate_child_operator to get next row,
|
||||
*
|
||||
* if in the end of the candidate_child_operator iterator, we only need to get next row from the
|
||||
* cur_child_operator, and don't need to cmp_( with candidate_child_operator's row
|
||||
*
|
||||
* if cur_child_operator and candidate_child_operator are present, we need to cmp_( input_row
|
||||
* with the candidate_output_row, if input_row is less than candidate_output_row, return input_row
|
||||
* as the result, if input_row equal to candidate_output_row, return input_row as the result and
|
||||
* get the distinct candidate_output_row from the candidate_child_operator, otherwise, return
|
||||
* candidate_output_row as the result and switch candidate_child_operator with cur_child_operator
|
||||
* for the next iteration
|
||||
*/
|
||||
int ObMergeUnionOp::distinct_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cmp = 0;
|
||||
const ObIArray<ObExpr*>* input_row = nullptr;
|
||||
clear_evaluated_flag();
|
||||
if (!first_got_row_) {
|
||||
int cur_child_err = OB_SUCCESS;
|
||||
int candidate_child_err = OB_SUCCESS;
|
||||
if (OB_ISNULL(cur_child_op_) || OB_ISNULL(last_row_.store_row_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cur_child_op is NULL or last row is null", K(ret));
|
||||
} else if (OB_UNLIKELY((OB_SUCCESS !=
|
||||
(cur_child_err = do_strict_distinct(*cur_child_op_, last_row_.store_row_, input_row))))) {
|
||||
if (OB_ITER_END == cur_child_err) {
|
||||
if (OB_LIKELY(NULL != candidate_child_op_)) {
|
||||
// current operator in the end of iterator, so switch to the candidate operator
|
||||
cur_child_op_ = candidate_child_op_;
|
||||
candidate_child_op_ = NULL;
|
||||
if (OB_FAIL(convert_row(*candidate_output_row_, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
candidate_output_row_ = NULL;
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
} else {
|
||||
ret = cur_child_err;
|
||||
LOG_WARN("failed to do_strict_distinct", K(ret));
|
||||
}
|
||||
} else if (NULL == candidate_child_op_) {
|
||||
if (OB_FAIL(convert_row(*input_row, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
} else if (OB_UNLIKELY(NULL == input_row || NULL == candidate_output_row_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("input row is NULL or candidate_output_row_ is NULL", K(input_row), K(candidate_output_row_), K(ret));
|
||||
} else if (OB_FAIL(cmp_(*input_row, *candidate_output_row_, eval_ctx_, cmp))) {
|
||||
LOG_WARN("compatible cmp_( failed", K(ret));
|
||||
} else if (0 == cmp) {
|
||||
// left row equal to right row
|
||||
if (OB_FAIL(convert_row(*input_row, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
candidate_child_err = do_strict_distinct(*candidate_child_op_, input_row, candidate_output_row_);
|
||||
|
||||
if (OB_SUCCESS != candidate_child_err) {
|
||||
if (OB_ITER_END == candidate_child_err) {
|
||||
// candidate operator in the end of row iteration, candidate operator not exist
|
||||
candidate_child_op_ = NULL;
|
||||
candidate_output_row_ = NULL;
|
||||
} else {
|
||||
ret = candidate_child_err;
|
||||
LOG_WARN("candidate child operator get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (cmp < 0) {
|
||||
// output current row
|
||||
if (OB_FAIL(convert_row(*input_row, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
} else if (cmp > 0) {
|
||||
// output candidate row and switch candidate operator to current operator for next iteration
|
||||
ObOperator* tmp_op = NULL;
|
||||
if (OB_FAIL(convert_row(*candidate_output_row_, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
candidate_output_row_ = input_row;
|
||||
tmp_op = candidate_child_op_;
|
||||
candidate_child_op_ = cur_child_op_;
|
||||
cur_child_op_ = tmp_op;
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(last_row_.save_store_row(MY_SPEC.set_exprs_, eval_ctx_, 0))) {
|
||||
LOG_WARN("storage current row for next cmp_( failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// the first row, not need to cmp_( the row
|
||||
// first, get next row
|
||||
const ObIArray<ObExpr*>* child_row = nullptr;
|
||||
first_got_row_ = false;
|
||||
if (OB_FAIL(get_first_row(child_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get first row failed", K(ret));
|
||||
}
|
||||
}
|
||||
// second, storage current row
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(convert_row(*child_row, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
} else if (OB_FAIL(last_row_.save_store_row(*child_row, eval_ctx_, 0))) {
|
||||
LOG_WARN("storage current row for next cmp_( failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("trace output row", K(ROWEXPR2STR(eval_ctx_, MY_SPEC.set_exprs_)), K(cmp));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnionOp::all_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (OB_ISNULL(cur_child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(MY_SPEC.id_));
|
||||
} else if (OB_FAIL(cur_child_op_->get_next_row())) {
|
||||
// get next row with the next child operator
|
||||
while (OB_ITER_END == ret && next_child_op_idx_ < get_child_cnt()) {
|
||||
cur_child_op_ = get_child(next_child_op_idx_);
|
||||
++next_child_op_idx_;
|
||||
if (OB_ISNULL(cur_child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator failed", K(MY_SPEC.id_));
|
||||
} else if (OB_FAIL(cur_child_op_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(convert_row(cur_child_op_->get_spec().output_, MY_SPEC.set_exprs_))) {
|
||||
LOG_WARN("failed to convert row", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMergeUnionOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(get_next_row_func_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("get_next_row_func is NULL", K(ret));
|
||||
} else if (OB_FAIL((this->*get_next_row_func_)())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
56
src/sql/engine/set/ob_merge_union_op.h
Normal file
56
src/sql/engine/set/ob_merge_union_op.h
Normal file
@ -0,0 +1,56 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_MERGE_UNION_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_MERGE_UNION_OP_H_
|
||||
|
||||
#include "sql/engine/set/ob_merge_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMergeUnionSpec : public ObMergeSetSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMergeUnionSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
};
|
||||
|
||||
class ObMergeUnionOp : public ObMergeSetOp {
|
||||
public:
|
||||
ObMergeUnionOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual void destroy() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
private:
|
||||
int get_first_row(const ObIArray<ObExpr*>*& output_row);
|
||||
int distinct_get_next_row();
|
||||
int all_get_next_row();
|
||||
|
||||
private:
|
||||
typedef int (ObMergeUnionOp::*GetNextRowFunc)();
|
||||
ObOperator* cur_child_op_;
|
||||
ObOperator* candidate_child_op_;
|
||||
const ObIArray<ObExpr*>* candidate_output_row_;
|
||||
int64_t next_child_op_idx_;
|
||||
bool first_got_row_;
|
||||
GetNextRowFunc get_next_row_func_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_BASIC_OB_SET_OB_MERGE_UNION_OP_H_ */
|
||||
28
src/sql/engine/set/ob_set_op.cpp
Normal file
28
src/sql/engine/set/ob_set_op.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/set/ob_set_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObSetSpec::ObSetSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), is_distinct_(false), set_exprs_(alloc), sort_collations_(alloc), sort_cmp_funs_(alloc)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObSetSpec, ObOpSpec), is_distinct_, set_exprs_, sort_collations_, sort_cmp_funs_);
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
47
src/sql/engine/set/ob_set_op.h
Normal file
47
src/sql/engine/set/ob_set_op.h
Normal file
@ -0,0 +1,47 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SET_OB_SET_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_SET_OB_SET_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/sort/ob_sort_basic_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObSetSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObSetSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(is_distinct), K_(sort_collations));
|
||||
bool is_distinct_;
|
||||
ExprFixedArray set_exprs_;
|
||||
ObSortCollations sort_collations_;
|
||||
ObSortFuncs sort_cmp_funs_;
|
||||
};
|
||||
|
||||
// class ObSetOp : public ObOperator
|
||||
// {
|
||||
// public:
|
||||
// ObSetOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input)
|
||||
// : ObOperator(exec_ctx, spec, input)
|
||||
// {}
|
||||
// virtual ~ObSetOp() = 0;
|
||||
// };
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SET_OB_SET_OP_H_
|
||||
146
src/sql/engine/set/ob_set_operator.cpp
Normal file
146
src/sql/engine/set/ob_set_operator.cpp
Normal file
@ -0,0 +1,146 @@
|
||||
/**
|
||||
* 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_set_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
ObSetOperator::ObSetOperator(common::ObIAllocator& alloc)
|
||||
: ObPhyOperator(alloc), distinct_(false), cs_types_(alloc), child_num_(0), child_array_(NULL)
|
||||
{}
|
||||
|
||||
ObSetOperator::~ObSetOperator()
|
||||
{}
|
||||
|
||||
void ObSetOperator::reset()
|
||||
{
|
||||
distinct_ = false;
|
||||
cs_types_.reset();
|
||||
child_num_ = 0;
|
||||
child_array_ = NULL;
|
||||
ObPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObSetOperator::reuse()
|
||||
{
|
||||
distinct_ = false;
|
||||
cs_types_.reuse();
|
||||
child_num_ = 0;
|
||||
child_array_ = NULL;
|
||||
ObPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObSetOperator::init(int64_t count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(init_collation_types(count))) {
|
||||
LOG_WARN("failed to init_collation_types", K(count), K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObSetOperator::set_distinct(bool is_distinct)
|
||||
{
|
||||
distinct_ = is_distinct;
|
||||
}
|
||||
|
||||
int ObSetOperator::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
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");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true /* need create cells */))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSetOperator::create_child_array(int64_t child_op_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (child_op_size > 0) {
|
||||
void* ptr = NULL;
|
||||
size_t buf_size = static_cast<size_t>(child_op_size * sizeof(ObPhyOperator*));
|
||||
CK(OB_NOT_NULL(my_phy_plan_));
|
||||
if (OB_ISNULL(ptr = my_phy_plan_->get_allocator().alloc(buf_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("create child array failed", K(ret), K(child_op_size));
|
||||
} else {
|
||||
memset(ptr, 0, buf_size);
|
||||
child_array_ = static_cast<ObPhyOperator**>(ptr);
|
||||
child_num_ = static_cast<int32_t>(child_op_size);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObPhyOperator* ObSetOperator::get_child(int32_t child_idx) const
|
||||
{
|
||||
ObPhyOperator* ret = NULL;
|
||||
if (OB_LIKELY(child_idx >= 0 && child_idx < child_num_)) {
|
||||
ret = child_array_[child_idx];
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSetOperator::set_child(int32_t child_idx, ObPhyOperator& child_operator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(child_array_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret), K(child_array_));
|
||||
} else if (OB_UNLIKELY(child_idx < 0 || child_idx >= child_num_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_ERROR("invalid child idx", K(child_idx));
|
||||
} else {
|
||||
child_array_[child_idx] = &child_operator;
|
||||
child_operator.set_parent(this);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSetOperator::accept(ObPhyOperatorVisitor& visitor) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(child_num_ < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid argument", K(child_num_));
|
||||
} else if (OB_FAIL(visitor.pre_visit(*this))) { // pre-visit
|
||||
LOG_WARN("fail to pre-visit", K(*this));
|
||||
}
|
||||
|
||||
// visit children
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < child_num_; ++i) {
|
||||
if (OB_ISNULL(child_array_[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid argument", K(child_array_[i]));
|
||||
} else if (OB_FAIL(child_array_[i]->accept(visitor))) {
|
||||
LOG_WARN("fail to accept left op", K(*child_array_[i]));
|
||||
}
|
||||
}
|
||||
|
||||
// post-visit
|
||||
if (OB_SUCC(ret) && OB_FAIL(visitor.post_visit(*this))) {
|
||||
LOG_WARN("fail to post-visit", K(*this));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // end namespace oceanbase
|
||||
89
src/sql/engine/set/ob_set_operator.h
Normal file
89
src/sql/engine/set/ob_set_operator.h
Normal file
@ -0,0 +1,89 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OB_SET_OPERATOR_H
|
||||
#define OB_SET_OPERATOR_H
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSetOperator : public ObPhyOperator {
|
||||
public:
|
||||
ObSetOperator();
|
||||
explicit ObSetOperator(common::ObIAllocator& alloc);
|
||||
~ObSetOperator();
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
virtual int init(int64_t count);
|
||||
|
||||
virtual void set_distinct(bool is_distinct);
|
||||
bool is_distinct() const;
|
||||
int add_collation_type(common::ObCollationType cs_type);
|
||||
|
||||
protected:
|
||||
int init_collation_types(int64_t count);
|
||||
|
||||
/**
|
||||
* @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 = 0;
|
||||
|
||||
virtual int set_child(int32_t child_idx, ObPhyOperator& child_operator) override;
|
||||
virtual int create_child_array(int64_t child_op_size) override;
|
||||
virtual ObPhyOperator* get_child(int32_t child_idx) const override;
|
||||
virtual int32_t get_child_num() const override
|
||||
{
|
||||
return child_num_;
|
||||
}
|
||||
virtual int accept(ObPhyOperatorVisitor& visitor) const override;
|
||||
|
||||
// disallow copy
|
||||
DISALLOW_COPY_AND_ASSIGN(ObSetOperator);
|
||||
|
||||
protected:
|
||||
bool distinct_;
|
||||
common::ObFixedArray<common::ObCollationType, common::ObIAllocator> cs_types_;
|
||||
int32_t child_num_;
|
||||
// Reminder: serialization ofdistinct_/cs_types_/child_num_ are implemented in
|
||||
// ObHashSetOperator and ObMergeSetOperator
|
||||
// To add more fileds, you need add their serialization of members in these subclasses
|
||||
ObPhyOperator** child_array_;
|
||||
};
|
||||
inline bool ObSetOperator::is_distinct() const
|
||||
{
|
||||
return distinct_;
|
||||
}
|
||||
inline int ObSetOperator::add_collation_type(common::ObCollationType cs_type)
|
||||
{
|
||||
return cs_types_.push_back(cs_type);
|
||||
}
|
||||
inline int ObSetOperator::init_collation_types(int64_t count)
|
||||
{
|
||||
return init_array_size<>(cs_types_, count);
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OB_SET_OPERATOR_H
|
||||
Reference in New Issue
Block a user