init push

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

View File

@ -0,0 +1,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

View 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_ */

View File

@ -0,0 +1,82 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#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

View 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_ */

View 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

View 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

View 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

View 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_

View 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

View 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

View 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

View 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_

View 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

View 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_

View 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;
}

View 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

View 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

View 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

View 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

View 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_

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_

View 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

View 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