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,842 @@
/**
* 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/subquery/ob_subplan_filter.h"
#include "common/row/ob_row_iterator.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/session/ob_sql_session_info.h"
namespace oceanbase {
using namespace common;
namespace sql {
class ObSubPlanFilter::ObSubPlanIterator : public ObNewRowIterator {
public:
ObSubPlanIterator(ObExecContext& ctx, const ObPhyOperator& op)
: ObNewRowIterator(),
ctx_(ctx),
op_(op),
onetime_plan_(false),
init_plan_(false),
inited_(false),
row_store_(),
row_store_it_(),
cur_row_()
{}
~ObSubPlanIterator()
{}
void set_tenant_id(uint64_t tenant_id)
{
row_store_.set_tenant_id(tenant_id);
}
int get_next_row(ObNewRow*& row)
{
int ret = OB_SUCCESS;
const ObNewRow* tmp_row = NULL;
if (init_plan_ && inited_) {
if (OB_SUCC(row_store_it_.get_next_row(cur_row_))) {
row = &cur_row_;
}
} else {
if (OB_SUCC(op_.get_next_row(ctx_, tmp_row))) {
row = const_cast<ObNewRow*>(tmp_row);
}
}
return ret;
}
void reset()
{
if (onetime_plan_) {
// for onetime expr
} else if (init_plan_) {
// for init plan
row_store_it_ = row_store_.begin();
} else {
int ret = OB_SUCCESS;
ObExecContext::ObPlanRestartGuard restart_plan(ctx_);
if (OB_FAIL(op_.rescan(ctx_))) {
BACKTRACE(ERROR, true, "failed to do rescan");
}
}
}
void reuse()
{
inited_ = false;
row_store_.reuse();
row_store_it_.reset();
}
void set_init_plan()
{
init_plan_ = true;
}
bool is_init_plan()
{
return init_plan_;
}
void set_onetime_plan()
{
onetime_plan_ = true;
}
bool is_onetime_plan()
{
return onetime_plan_;
}
int prepare_init_plan()
{
int ret = OB_SUCCESS;
if (!inited_) {
if (NULL == cur_row_.cells_) {
if (OB_FAIL(init_cur_row())) {
LOG_WARN("failed to init cur_row", K(ret));
} else { /*do nothing*/
}
} else { /*do nothing*/
}
ObNewRow* row = NULL;
while (OB_SUCC(ret) && OB_SUCC(get_next_row(row))) {
if (OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row is null");
} else if (OB_FAIL(row_store_.add_row(*row))) {
LOG_WARN("failed to add row to row store", K(ret));
} else { /*do nothing*/
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
inited_ = true;
row_store_it_ = row_store_.begin();
}
} else { /*do nothing*/
}
return ret;
}
int init_cur_row()
{
int ret = common::OB_SUCCESS;
void* ptr = NULL;
int64_t column_count = op_.get_projector_size();
if (OB_UNLIKELY(column_count <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(column_count));
} else if (OB_UNLIKELY(NULL == (ptr = ctx_.get_allocator().alloc(column_count * sizeof(common::ObObj))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc memory for row failed", K(column_count * sizeof(common::ObObj)));
} else {
cur_row_.cells_ = new (ptr) ObObj[column_count];
cur_row_.count_ = column_count;
}
return ret;
}
private:
ObExecContext& ctx_;
const ObPhyOperator& op_;
bool onetime_plan_;
bool init_plan_;
bool inited_;
common::ObRowStore row_store_;
ObRowStore::Iterator row_store_it_;
common::ObNewRow cur_row_;
};
class ObSubPlanFilter::ObSubPlanFilterCtx : public ObPhyOperatorCtx {
explicit ObSubPlanFilterCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), subplan_iters_()
{}
~ObSubPlanFilterCtx()
{
for (int64_t i = 0; i < subplan_iters_.count(); ++i) {
ObNewRowIterator*& iter = subplan_iters_.at(i);
if (iter != NULL) {
iter->~ObNewRowIterator();
iter = NULL;
}
}
}
virtual void destroy()
{
subplan_iters_.~ObSEArray<ObNewRowIterator*, 16>();
ObPhyOperatorCtx::destroy_base();
}
private:
ObSEArray<ObNewRowIterator*, 16> subplan_iters_;
friend class ObSubPlanFilter;
};
ObSubPlanFilter::ObSubPlanFilter(ObIAllocator& alloc)
: ObMultiChildrenPhyOperator(alloc),
rescan_params_(alloc),
onetime_exprs_(alloc),
init_plan_idxs_(ModulePageAllocator(alloc)),
one_time_idxs_(ModulePageAllocator(alloc)),
update_set_(false)
{}
ObSubPlanFilter::~ObSubPlanFilter()
{}
void ObSubPlanFilter::reset()
{
rescan_params_.reset();
onetime_exprs_.reset();
init_plan_idxs_.reset();
one_time_idxs_.reset();
ObMultiChildrenPhyOperator::reset();
}
void ObSubPlanFilter::reuse()
{
rescan_params_.reuse();
onetime_exprs_.reuse();
init_plan_idxs_.reuse();
one_time_idxs_.reuse();
ObMultiChildrenPhyOperator::reuse();
}
int ObSubPlanFilter::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperator* child_op = NULL;
ObSubPlanFilterCtx* subplan_ctx = NULL;
ObPhysicalPlanCtx* plan_ctx = NULL;
if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, ctx, get_id())) ||
OB_ISNULL(plan_ctx = ctx.get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed", K(ret));
} else { /*do nothing*/
}
for (int64_t i = 0; OB_SUCC(ret) && i < rescan_params_.count(); ++i) {
int64_t idx = rescan_params_.at(i).second;
plan_ctx->get_param_store_for_update().at(idx).set_null();
LOG_DEBUG("prepare_rescan_params", K(ret), K(i), K(idx));
}
for (int32_t i = 1; OB_SUCC(ret) && i < get_child_num(); ++i) {
if (OB_ISNULL(child_op = get_child(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get child", K(i), K(child_op), K(get_child_num()), K(ret));
} else if (OB_FAIL(child_op->rescan(ctx))) {
LOG_WARN("rescan child operator failed",
K(ret),
"op_type",
ob_phy_operator_type_str(get_type()),
"child op_type",
ob_phy_operator_type_str(child_op->get_type()));
} else { /*do nothing*/
}
}
for (int32_t i = 1; OB_SUCC(ret) && i < get_child_num(); ++i) {
ObSubPlanIterator* subplan_iter = NULL;
subplan_iter = static_cast<ObSubPlanIterator*>(subplan_ctx->subplan_iters_.at(i - 1));
if (OB_ISNULL(subplan_iter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subplan_iter is null");
} else if (init_plan_idxs_.has_member(i)) {
subplan_iter->reuse();
if (OB_FAIL(subplan_iter->prepare_init_plan())) {
LOG_WARN("prepare init plan failed", K(ret), K(i));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(prepare_onetime_exprs(ctx))) {
LOG_WARN("prepare onetime exprs failed", K(ret));
} else if (OB_ISNULL(child_op = get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get child", K(0), K(child_op), K(get_child_num()), K(ret));
} else if (OB_FAIL(child_op->rescan(ctx))) {
LOG_WARN("failed to do rescan", K(ret));
} else {
subplan_ctx->is_filtered_has_set_ = false;
// subplan_ctx->op_monitor_info_.increase_value(RESCAN_TIMES);
}
}
return ret;
}
int ObSubPlanFilter::switch_iterator(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperator* child_op = NULL;
ObSubPlanFilterCtx* subplan_ctx = NULL;
if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed", K(ret));
} else if (OB_ISNULL(child_op = get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("main query child operator is null", K(ret));
} else if (OB_FAIL(child_op->switch_iterator(ctx))) {
if (OB_ITER_END != ret) {
LOG_WARN("swtich child operator iterator failed", K(ret));
}
} else {
subplan_ctx->is_filtered_has_set_ = false;
++subplan_ctx->expr_ctx_.cur_array_index_;
}
return ret;
}
OB_DEF_SERIALIZE(ObSubPlanFilter)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObMultiChildrenPhyOperator::serialize(buf, buf_len, pos))) {
LOG_WARN("serialize child phy operator failed", K(ret));
} else {
// serialize rescan_params_
OB_UNIS_ENCODE(rescan_params_.count());
for (int64_t i = 0; OB_SUCC(ret) && i < rescan_params_.count(); ++i) {
ObSqlExpression* sql_expr = rescan_params_.at(i).first;
int64_t param_idx = rescan_params_.at(i).second;
if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
}
OB_UNIS_ENCODE(*sql_expr);
OB_UNIS_ENCODE(param_idx);
}
// serialize onetime_exprs_
if (OB_SUCC(ret)) {
OB_UNIS_ENCODE(onetime_exprs_.count());
}
for (int64_t i = 0; OB_SUCC(ret) && i < onetime_exprs_.count(); ++i) {
ObSqlExpression* sql_expr = onetime_exprs_.at(i).first;
int64_t param_idx = onetime_exprs_.at(i).second;
if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
}
OB_UNIS_ENCODE(*sql_expr);
OB_UNIS_ENCODE(param_idx);
}
// serialize init_plan_idxs_ and one_time_idxs_
if (OB_SUCC(ret)) {
if (OB_FAIL(init_plan_idxs_.serialize(buf, buf_len, pos))) {
LOG_WARN("serialize init plan idxs failed", K(ret));
} else if (OB_FAIL(one_time_idxs_.serialize(buf, buf_len, pos))) {
LOG_WARN("serialize one time idxs failed", K(ret));
} else { /*do nothing*/
}
} else { /*do nothing*/
}
OB_UNIS_ENCODE(update_set_);
}
return ret;
}
OB_DEF_DESERIALIZE(ObSubPlanFilter)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObMultiChildrenPhyOperator::deserialize(buf, data_len, pos))) {
LOG_WARN("deserialize child phy operator failed", K(ret));
} else if (OB_ISNULL(my_phy_plan_)) {
ret = OB_NOT_INIT;
LOG_WARN("my_phy_plan_ is null");
} else {
// deserialize rescan_params_
int64_t param_count = 0;
OB_UNIS_DECODE(param_count);
if (OB_FAIL(init_rescan_param(param_count))) {
LOG_WARN("fail to init rescan_param", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < param_count; ++i) {
ObSqlExpression* sql_expr = NULL;
int64_t param_idx = OB_INVALID_INDEX;
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, sql_expr))) {
LOG_WARN("make sql expression failed", K(ret));
} else if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
} else {
OB_UNIS_DECODE(*sql_expr);
OB_UNIS_DECODE(param_idx);
}
if (OB_SUCC(ret)) {
if (OB_FAIL(rescan_params_.push_back(std::pair<ObSqlExpression*, int64_t>(sql_expr, param_idx)))) {
LOG_WARN("push back rescan params failed", K(ret));
}
}
}
// deserialize onetime_exprs_
OB_UNIS_DECODE(param_count);
if (OB_SUCC(ret) && OB_FAIL(init_onetime_exprs(param_count))) {
LOG_WARN("fail to init onetime expr", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < param_count; ++i) {
ObSqlExpression* sql_expr = NULL;
int64_t param_idx = OB_INVALID_INDEX;
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, sql_expr))) {
LOG_WARN("make sql expression failed", K(ret));
} else if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
} else {
OB_UNIS_DECODE(*sql_expr);
OB_UNIS_DECODE(param_idx);
}
if (OB_SUCC(ret)) {
if (OB_FAIL(onetime_exprs_.push_back(std::pair<ObSqlExpression*, int64_t>(sql_expr, param_idx)))) {
LOG_WARN("push back rescan params failed", K(ret));
}
}
}
// serialize init_plan_idxs_ and one_time_idxs_
if (OB_SUCC(ret)) {
if (OB_FAIL(init_plan_idxs_.deserialize(buf, data_len, pos))) {
LOG_WARN("serialize init plan idxs failed", K(ret));
} else if (OB_FAIL(one_time_idxs_.deserialize(buf, data_len, pos))) {
LOG_WARN("serialize init plan idxs failed", K(ret));
} else { /*do nothing*/
}
} else { /*do nothing*/
}
OB_UNIS_DECODE(update_set_);
}
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObSubPlanFilter)
{
int64_t len = 0;
len = ObMultiChildrenPhyOperator::get_serialize_size();
OB_UNIS_ADD_LEN(rescan_params_.count());
for (int64_t i = 0; i < rescan_params_.count(); ++i) {
ObSqlExpression* sql_expr = rescan_params_.at(i).first;
int64_t param_idx = rescan_params_.at(i).second;
if (sql_expr != NULL) {
OB_UNIS_ADD_LEN(*sql_expr);
OB_UNIS_ADD_LEN(param_idx);
}
}
OB_UNIS_ADD_LEN(onetime_exprs_.count());
for (int64_t i = 0; i < onetime_exprs_.count(); ++i) {
ObSqlExpression* sql_expr = onetime_exprs_.at(i).first;
int64_t param_idx = onetime_exprs_.at(i).second;
if (sql_expr != NULL) {
OB_UNIS_ADD_LEN(*sql_expr);
OB_UNIS_ADD_LEN(param_idx);
}
}
len += init_plan_idxs_.get_serialize_size();
len += one_time_idxs_.get_serialize_size();
OB_UNIS_ADD_LEN(update_set_);
return len;
}
int ObSubPlanFilter::open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperator* child_ptr = NULL;
for (int32_t i = 1; OB_SUCC(ret) && i < get_child_num(); ++i) {
if (OB_ISNULL(child_ptr = get_child(i))) {
ret = OB_NOT_INIT;
LOG_WARN("failed to get child", K(i), K(child_ptr), K(get_child_num()), K(ret));
} else if (OB_FAIL(child_ptr->open(ctx))) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open child operator failed", K(ret));
}
} else {
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(inner_open(ctx))) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open this operator failed", K(ret));
}
} else if (OB_FAIL(handle_op_ctx(ctx))) {
LOG_WARN("handle op ctx failed ", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(child_ptr = get_child(0))) {
ret = OB_NOT_INIT;
LOG_WARN("failed to get child", K(0), K(child_ptr), K(get_child_num()), K(ret));
} else if (OB_FAIL(child_ptr->open(ctx))) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open child operator failed", K(ret));
}
} else { /*do nothing*/
}
}
return ret;
}
int ObSubPlanFilter::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObSubPlanFilterCtx* subplan_ctx = NULL;
ObSQLSessionInfo* session = NULL;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("init operator context failed", K(ret));
} else if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed", K(ret));
} else if (OB_ISNULL(session = ctx.get_my_session())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get session", K(ret), K(session));
} else {
void* ptr = NULL;
ObSubPlanIterator* subplan_iter = NULL;
for (int32_t i = 1; OB_SUCC(ret) && i < get_child_num(); ++i) {
if (OB_UNLIKELY(NULL == (ptr = ctx.get_allocator().alloc(sizeof(ObSubPlanIterator))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc subplan iterator failed", "size", sizeof(ObSubPlanIterator));
} else {
subplan_iter = new (ptr) ObSubPlanIterator(ctx, *get_child(i));
subplan_iter->set_tenant_id(session->get_effective_tenant_id());
if (OB_FAIL(subplan_ctx->subplan_iters_.push_back(subplan_iter))) {
LOG_WARN("push back subplan iter failed", K(ret), K(i));
subplan_iter->~ObSubPlanIterator();
subplan_iter = NULL;
} else {
if (init_plan_idxs_.has_member(i)) {
subplan_iter->set_init_plan();
if (OB_FAIL(subplan_iter->prepare_init_plan())) {
LOG_WARN("prepare init plan failed", K(ret), K(i));
} else { /*do nothing*/
}
} else if (one_time_idxs_.has_member(i)) {
subplan_iter->set_onetime_plan();
} else { /*do nothing*/
}
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(prepare_onetime_exprs(ctx))) {
LOG_WARN("prepare onetime exprs failed", K(ret));
} else { /*do nothing*/
}
}
}
return ret;
}
int ObSubPlanFilter::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObSubPlanFilterCtx* subplan_ctx = NULL;
if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, 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 {
for (int64_t i = 0; OB_SUCC(ret) && i < subplan_ctx->subplan_iters_.count(); ++i) {
ObNewRowIterator*& subplan_iter = subplan_ctx->subplan_iters_.at(i);
if (subplan_iter != NULL) {
subplan_iter->~ObNewRowIterator();
subplan_iter = NULL;
}
}
}
return ret;
}
int ObSubPlanFilter::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("create physical 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, update_set_ || need_copy_row_for_compute()))) {
LOG_WARN("create current row failed", K(ret));
}
return ret;
}
int ObSubPlanFilter::wrap_expr_ctx(ObExecContext& exec_ctx, common::ObExprCtx& expr_ctx) const
{
int ret = OB_SUCCESS;
ObSubPlanFilterCtx* subplan_ctx = NULL;
if (OB_FAIL(ObPhyOperator::wrap_expr_ctx(exec_ctx, expr_ctx))) {
LOG_WARN("wrap_expr_ctx failed", K(ret));
} else if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get_phy_operator_ctx failed", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
} else {
expr_ctx.subplan_iters_ = &(subplan_ctx->subplan_iters_);
}
return ret;
}
int ObSubPlanFilter::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObPhyOperator* child_op = NULL;
ObSubPlanFilterCtx* subplan_ctx = NULL;
if (OB_ISNULL(child_op = get_child(0))) {
ret = OB_NOT_INIT;
LOG_WARN("child operator is null");
} else if (OB_FAIL(child_op->get_next_row(ctx, row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from child operator failed", K(ret));
}
} else if (OB_ISNULL(row)) {
LOG_WARN("row is null");
} else if (OB_FAIL(prepare_rescan_params(ctx, *row))) {
LOG_WARN("prepare rescan params failed", K(ret));
} else {
ObNewRowIterator* subplan_iter = NULL;
if (OB_ISNULL(subplan_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanFilterCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed");
}
ObExecContext::ObPlanRestartGuard restart_plan(ctx);
for (int32_t i = 1; OB_SUCC(ret) && i < get_child_num(); ++i) {
if (one_time_idxs_.has_member(i)) {
} else if (init_plan_idxs_.has_member(i)) {
if (OB_ISNULL(subplan_iter = subplan_ctx->subplan_iters_.at(i - 1))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subplan iter is null");
} else {
subplan_iter->reset();
}
} else if (OB_ISNULL(child_op = get_child(i))) {
ret = OB_NOT_INIT;
LOG_WARN("child operator is null", K(i));
} else if (OB_FAIL(child_op->rescan(ctx))) {
LOG_WARN("rescan child operator failed", K(ret), K(i));
}
}
}
if (OB_SUCC(ret)) {
if (update_set_) {
OZ(handle_update_set(subplan_ctx, row));
} else {
OZ(copy_cur_row(*subplan_ctx, row));
}
}
return ret;
}
int ObSubPlanFilter::prepare_rescan_params(ObExecContext& ctx, const ObNewRow& row) const
{
int ret = OB_SUCCESS;
ObExprCtx expr_ctx;
ObPhysicalPlanCtx* plan_ctx = NULL;
if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("physical plan context is null");
} else if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
LOG_WARN("wrap expr ctx failed", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < rescan_params_.count(); ++i) {
ObObjParam result;
ObSqlExpression* sql_expr = rescan_params_.at(i).first;
int64_t param_idx = rescan_params_.at(i).second;
if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
} else if (OB_FAIL(sql_expr->calc(expr_ctx, row, result))) {
LOG_WARN("sql expr calc failed", K(ret), K(*sql_expr), K(row));
} else {
result.set_param_meta();
plan_ctx->get_param_store_for_update().at(param_idx) = result;
}
}
}
return ret;
}
int ObSubPlanFilter::add_rescan_param(ObSqlExpression* sql_expr, int64_t param_idx)
{
return rescan_params_.push_back(std::pair<ObSqlExpression*, int64_t>(sql_expr, param_idx));
}
int ObSubPlanFilter::construct_array_params(ObExecContext& ctx) const
{
int ret = OB_NOT_SUPPORTED;
LOG_WARN("not supported", K(ctx));
return ret;
}
int ObSubPlanFilter::prepare_onetime_exprs(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObExprCtx expr_ctx;
ObPhysicalPlanCtx* plan_ctx = NULL;
if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("physical plan context is null");
} else if (OB_LIKELY(plan_ctx->get_bind_array_count() > 0)) {
if (OB_FAIL(construct_array_params(ctx))) {
LOG_WARN("construct array params failed", K(ret));
}
} else if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
LOG_WARN("wrap expr ctx failed", K(ret));
} else {
ObNewRow row;
for (int64_t i = 0; OB_SUCC(ret) && i < onetime_exprs_.count(); ++i) {
ObObj tmp;
ObObjParam result;
ObSqlExpression* sql_expr = onetime_exprs_.at(i).first;
int64_t param_idx = onetime_exprs_.at(i).second;
if (OB_ISNULL(sql_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_expr is null");
} else if (OB_FAIL(sql_expr->calc(expr_ctx, row, tmp))) {
LOG_WARN("sql expr calc failed", K(ret), K(*sql_expr), K(row));
} else if (OB_FAIL(ob_write_obj(ctx.get_allocator(), tmp, result))) {
LOG_WARN("deep copy obj failed", K(ret));
} else {
result.set_param_meta();
plan_ctx->get_param_store_for_update().at(param_idx) = result;
}
}
}
return ret;
}
int ObSubPlanFilter::add_onetime_expr(ObSqlExpression* sql_expr, int64_t param_idx)
{
return onetime_exprs_.push_back(std::pair<ObSqlExpression*, int64_t>(sql_expr, param_idx));
}
int64_t ObSubPlanFilter::to_string_kv(char* buf, int64_t buf_len) const
{
int64_t pos = 0;
// rescan_params_
J_NAME(N_RESCAN_PARAM);
J_COLON();
J_ARRAY_START();
for (int64_t i = 0; i < rescan_params_.count() - 1; ++i) {
int64_t index = rescan_params_.at(i).second;
ObSqlExpression* expr = rescan_params_.at(i).first;
if (expr != NULL) {
J_OBJ_START();
J_KV(K(index), N_EXPR, expr);
J_OBJ_END();
J_COMMA();
}
}
if (rescan_params_.count() > 0) {
int64_t index = rescan_params_.at(rescan_params_.count() - 1).second;
ObSqlExpression* expr = rescan_params_.at(rescan_params_.count() - 1).first;
if (expr != NULL) {
J_OBJ_START();
J_KV(K(index), N_EXPR, expr);
J_OBJ_END();
}
}
J_ARRAY_END();
J_COMMA();
// onetime_exprs_
J_NAME(N_ONETIME_FILTER);
J_COLON();
J_ARRAY_START();
for (int64_t i = 0; i < onetime_exprs_.count() - 1; ++i) {
int64_t index = onetime_exprs_.at(i).second;
ObSqlExpression* expr = onetime_exprs_.at(i).first;
if (expr != NULL) {
J_OBJ_START();
J_KV(K(index), N_EXPR, expr);
J_OBJ_END();
J_COMMA();
}
}
if (onetime_exprs_.count() > 0) {
int64_t index = onetime_exprs_.at(onetime_exprs_.count() - 1).second;
ObSqlExpression* expr = onetime_exprs_.at(onetime_exprs_.count() - 1).first;
if (expr != NULL) {
J_OBJ_START();
J_KV(K(index), N_EXPR, expr);
J_OBJ_END();
}
}
J_ARRAY_END();
J_COMMA();
// init_plan_idxs_ and onetime_idxs_
J_KV(K_(init_plan_idxs), K_(one_time_idxs));
return pos;
}
int ObSubPlanFilter::handle_update_set(ObSubPlanFilterCtx* subplan_ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(row) || OB_ISNULL(subplan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row is null unexpected", K(ret));
} else {
bool no_row = false;
ObNewRow* subquery_row = NULL;
ObNewRowIterator* row_iter = nullptr;
ObSEArray<ObNewRowIterator*, 16>& row_iters = subplan_ctx->subplan_iters_;
if (1 != row_iters.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("too many subplan unexpected", K(ret), K(row_iters.count()));
} else {
row_iter = row_iters.at(0);
if (OB_ISNULL(row_iter) || OB_FAIL(row_iter->get_next_row(subquery_row))) {
if (OB_LIKELY(OB_ITER_END == ret)) {
ret = OB_SUCCESS;
no_row = true;
}
}
}
if (OB_SUCC(ret)) {
ObNewRow& cur_row = subplan_ctx->cur_row_;
int64_t real_count = row->get_count();
for (int64_t i = 0; OB_SUCC(ret) && i < real_count; ++i) {
int64_t real_idx = row->projector_size_ > 0 ? row->projector_[i] : i;
if (OB_UNLIKELY(real_idx >= row->count_ || real_idx >= cur_row.count_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(real_idx), K_(row->count), K_(cur_row.count));
} else {
cur_row.cells_[real_idx] = row->cells_[real_idx];
}
}
if (OB_SUCC(ret)) {
if (!no_row) {
for (int64_t i = 0; OB_SUCC(ret) && i < subquery_row->get_count(); i++) {
// subquery row's memory will be release after asign the obj to current row
// so need to call ob_write_obj to deep copy this obj at here
if (OB_UNLIKELY(row->count_ + i >= cur_row.count_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(row->count_ + i), K_(cur_row.count), K(ret));
} else if (OB_FAIL(ob_write_obj(*subplan_ctx->expr_ctx_.calc_buf_,
subquery_row->get_cell(i),
cur_row.cells_[row->count_ + i]))) {
LOG_WARN("write obj failed", K(ret));
}
}
} else {
ObObj null_obj;
for (int64_t i = row->count_; i < cur_row.count_; i++) {
cur_row.cells_[i] = null_obj;
}
}
row = &cur_row;
}
}
if (OB_SUCC(ret)) {
ObNewRow* tmp_row = NULL;
if (OB_UNLIKELY(OB_SUCCESS == (ret = row_iter->get_next_row(tmp_row)))) {
ret = OB_ERR_MORE_THAN_ONE_ROW;
LOG_WARN("subquery too many rows", K(ret));
} else if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,120 @@
/**
* 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_SRC_SQL_ENGINE_SUBQUERY_OB_SUBPLAN_FILTER_H_
#define OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_SUBPLAN_FILTER_H_
#include "lib/container/ob_bit_set.h"
#include "lib/container/ob_fixed_array.h"
#include "sql/engine/ob_multi_children_phy_operator.h"
namespace oceanbase {
namespace sql {
class ObSubPlanFilter : public ObMultiChildrenPhyOperator {
OB_UNIS_VERSION(1);
class ObSubPlanFilterCtx;
class ObSubPlanIterator;
public:
explicit ObSubPlanFilter(common::ObIAllocator& alloc);
virtual ~ObSubPlanFilter();
void reset();
void reuse();
int rescan(ObExecContext& ctx) const;
/**
* @brief add rescan param to subplan filter
* @param sql_expr[in], the sql expression of rescan param
* @param param_idx[in], rescan param index
* @return if success, return OB_SUCCESS
*/
int add_rescan_param(ObSqlExpression* sql_expr, int64_t param_idx);
int add_onetime_expr(ObSqlExpression* sql_expr, int64_t param_idx);
void add_initplan_idxs(const common::ObBitSet<>& idxs)
{
init_plan_idxs_.add_members2(idxs);
}
void add_onetime_idxs(const common::ObBitSet<>& idxs)
{
one_time_idxs_.add_members2(idxs);
}
int init_rescan_param(int64_t count)
{
return init_array_size<>(rescan_params_, count);
}
int init_onetime_exprs(int64_t count)
{
return init_array_size<>(onetime_exprs_, count);
}
void set_update_set(bool update_set)
{
update_set_ = update_set;
}
bool is_update_set()
{
return update_set_;
}
virtual int open(ObExecContext& ctx) const;
virtual int switch_iterator(ObExecContext& ctx) const override;
private:
/**
* @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], ObNewRow 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;
/**
* @brief wrap the object of ObExprCtx, and reset calc_buf
* @param exec_ctx[in], execute context
* @param expr_ctx[out], sql expression calculate buffer context
* @return if success, return OB_SUCCESS
*/
virtual int wrap_expr_ctx(ObExecContext& exec_ctx, common::ObExprCtx& expr_ctx) const;
int prepare_rescan_params(ObExecContext& ctx, const common::ObNewRow& row) const;
int prepare_onetime_exprs(ObExecContext& ctx) const;
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
int handle_update_set(ObSubPlanFilterCtx* subplan_ctx, const common::ObNewRow*& row) const;
int construct_array_params(ObExecContext& ctx) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObSubPlanFilter);
private:
common::ObFixedArray<std::pair<ObSqlExpression*, int64_t>, common::ObIAllocator> rescan_params_;
common::ObFixedArray<std::pair<ObSqlExpression*, int64_t>, common::ObIAllocator> onetime_exprs_;
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> init_plan_idxs_;
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> one_time_idxs_;
bool update_set_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_SUBPLAN_FILTER_H_ */

View File

@ -0,0 +1,504 @@
/**
* 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_subplan_filter_op.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObSubQueryIterator::ObSubQueryIterator(ObOperator& op)
: op_(op), onetime_plan_(false), init_plan_(false), inited_(false), iterated_(false)
{}
int ObSubQueryIterator::start()
{
int ret = OB_SUCCESS;
if (iterated_) {
const bool reset_onetime_plan = true;
reset(reset_onetime_plan);
}
return ret;
}
int ObSubQueryIterator::get_next_row()
{
int ret = OB_SUCCESS;
iterated_ = true;
if (init_plan_ && inited_) {
ret = store_it_.get_next_row(get_output(), op_.get_eval_ctx());
} else {
ret = op_.get_next_row();
}
return ret;
}
void ObSubQueryIterator::reset(const bool reset_onetime_plan /* = false */)
{
int ret = OB_SUCCESS;
if (onetime_plan_ && !reset_onetime_plan) {
// for onetime expr
} else if (init_plan_) {
// for init plan
if (OB_FAIL(store_.begin(store_it_, ObChunkDatumStore::BLOCK_SIZE))) {
BACKTRACE(ERROR, true, "failed to rewind iterator");
}
} else {
ObExecContext::ObPlanRestartGuard restart_plan(op_.get_exec_ctx());
if (OB_FAIL(op_.rescan())) {
BACKTRACE(ERROR, true, "failed to do rescan");
}
}
iterated_ = false;
}
void ObSubQueryIterator::reuse()
{
inited_ = false;
iterated_ = false;
store_it_.reset();
store_.reset();
}
int ObSubQueryIterator::prepare_init_plan()
{
int ret = OB_SUCCESS;
if (!inited_) {
if (!store_.is_inited()) {
// TODO : use auto memory management
OZ(store_.init(1L << 20, // 1MB memory limit
GET_MY_SESSION(op_.get_exec_ctx())->get_effective_tenant_id()));
}
while (OB_SUCC(ret) && OB_SUCC(get_next_row())) {
OZ(store_.add_row(get_output(), &op_.get_eval_ctx()));
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
inited_ = true;
iterated_ = false;
OZ(store_it_.init(&store_, ObChunkDatumStore::BLOCK_SIZE));
}
}
return ret;
}
ObSubPlanFilterSpec::ObSubPlanFilterSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type),
rescan_params_(alloc),
onetime_exprs_(alloc),
init_plan_idxs_(ModulePageAllocator(alloc)),
one_time_idxs_(ModulePageAllocator(alloc)),
update_set_(alloc)
{}
OB_SERIALIZE_MEMBER(
(ObSubPlanFilterSpec, ObOpSpec), rescan_params_, onetime_exprs_, init_plan_idxs_, one_time_idxs_, update_set_);
DEF_TO_STRING(ObSubPlanFilterSpec)
{
int64_t pos = 0;
J_OBJ_START();
J_NAME("op_spec");
J_COLON();
pos += ObOpSpec::to_string(buf + pos, buf_len - pos);
J_COMMA();
J_KV(K_(rescan_params), K_(onetime_exprs), K_(init_plan_idxs), K_(one_time_idxs), K_(update_set));
J_OBJ_END();
return pos;
}
ObSubPlanFilterOp::ObSubPlanFilterOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input), update_set_mem_(NULL)
{}
ObSubPlanFilterOp::~ObSubPlanFilterOp()
{
destroy_subplan_iters();
destroy_update_set_mem();
}
void ObSubPlanFilterOp::destroy_subplan_iters()
{
FOREACH_CNT(it, subplan_iters_)
{
if (NULL != *it) {
(*it)->~Iterator();
*it = NULL;
}
}
subplan_iters_.reset();
}
void ObSubPlanFilterOp::destroy()
{
destroy_subplan_iters();
destroy_update_set_mem();
ObOperator::destroy();
}
int ObSubPlanFilterOp::set_param_null()
{
int ret = OB_SUCCESS;
ObDatum null_datum;
null_datum.set_null();
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.rescan_params_.count(); ++i) {
OZ(MY_SPEC.rescan_params_.at(i).update_dynamic_param(eval_ctx_, null_datum));
LOG_DEBUG("prepare_rescan_params", K(ret), K(i));
}
return ret;
}
int ObSubPlanFilterOp::rescan()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_FAIL(set_param_null())) {
LOG_WARN("failed to set param null", K(ret));
}
for (int32_t i = 1; OB_SUCC(ret) && i < child_cnt_; ++i) {
if (OB_FAIL(children_[i]->rescan())) {
LOG_WARN("rescan child operator failed", K(ret), "op", op_name(), "child", children_[i]->op_name());
}
}
for (int32_t i = 1; OB_SUCC(ret) && i < child_cnt_; ++i) {
Iterator* iter = subplan_iters_.at(i - 1);
if (OB_ISNULL(iter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subplan_iter is null", K(ret));
} else if (MY_SPEC.init_plan_idxs_.has_member(i)) {
iter->reuse();
if (OB_FAIL(iter->prepare_init_plan())) {
LOG_WARN("prepare init plan failed", K(ret), K(i));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(prepare_onetime_exprs())) {
LOG_WARN("prepare onetime exprs failed", K(ret));
} else if (OB_FAIL(child_->rescan())) {
LOG_WARN("failed to do rescan", K(ret));
} else {
startup_passed_ = spec_.startup_filters_.empty();
}
}
return ret;
}
int ObSubPlanFilterOp::switch_iterator()
{
int ret = OB_SUCCESS;
if (OB_FAIL(child_->switch_iterator())) {
if (OB_ITER_END != ret) {
LOG_WARN("swtich child operator iterator failed", K(ret));
}
} else {
startup_passed_ = spec_.startup_filters_.empty();
}
return ret;
}
int ObSubPlanFilterOp::open()
{
int ret = OB_SUCCESS;
opened_ = true;
CK(child_cnt_ >= 2);
// reset param:
if (OB_FAIL(set_param_null())) {
LOG_WARN("failed to set param null", K(ret));
}
for (int32_t i = 1; OB_SUCC(ret) && i < child_cnt_; ++i) {
if (OB_ISNULL(children_[i])) {
ret = OB_NOT_INIT;
LOG_WARN("failed to get child", K(ret), K(i));
} else if (OB_FAIL(children_[i]->open())) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open child operator failed", K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(init_evaluated_flags())) {
LOG_WARN("init evaluated flags failed", K(ret));
} else if (OB_FAIL(inner_open())) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open this operator failed", K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(child_)) {
ret = OB_NOT_INIT;
LOG_WARN("failed to get child", K(ret));
} else if (OB_FAIL(child_->open())) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("Open child operator failed", K(ret));
}
}
}
return ret;
}
int ObSubPlanFilterOp::inner_open()
{
int ret = OB_SUCCESS;
CK(child_cnt_ >= 2);
if (OB_SUCC(ret)) {
// generate row iterator for each subquery
OZ(subplan_iters_.prepare_allocate(child_cnt_ - 1));
for (int32_t i = 1; OB_SUCC(ret) && i < child_cnt_; ++i) {
void* ptr = ctx_.get_allocator().alloc(sizeof(Iterator));
Iterator*& iter = subplan_iters_.at(i - 1);
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc subplan iterator failed", K(ret), "size", sizeof(Iterator));
} else {
iter = new (ptr) Iterator(*children_[i]);
if (MY_SPEC.init_plan_idxs_.has_member(i)) {
iter->set_init_plan();
OZ(iter->prepare_init_plan());
} else if (MY_SPEC.one_time_idxs_.has_member(i)) {
iter->set_onetime_plan();
}
}
}
OZ(prepare_onetime_exprs());
}
return ret;
}
int ObSubPlanFilterOp::inner_close()
{
destroy_subplan_iters();
destroy_update_set_mem();
return OB_SUCCESS;
}
int ObSubPlanFilterOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_FAIL(child_->get_next_row())) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from child operator failed", K(ret));
}
} else if (OB_FAIL(prepare_rescan_params())) {
LOG_WARN("prepare rescan params failed", K(ret));
} else {
ObExecContext::ObPlanRestartGuard restart_plan(ctx_);
for (int32_t i = 1; OB_SUCC(ret) && i < child_cnt_; ++i) {
//// rescan for each subquery
if (MY_SPEC.one_time_idxs_.has_member(i)) {
// need no rescan, skip
} else if (MY_SPEC.init_plan_idxs_.has_member(i)) {
Iterator* iter = subplan_iters_.at(i - 1);
if (OB_ISNULL(iter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subplan iter is null");
} else {
iter->reset();
}
} else if (OB_FAIL(children_[i]->rescan())) {
LOG_WARN("rescan child operator failed", K(ret), K(i));
}
}
}
if (OB_SUCC(ret)) {
if (!MY_SPEC.update_set_.empty()) {
OZ(handle_update_set());
}
}
return ret;
}
int ObSubPlanFilterOp::prepare_rescan_params()
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.rescan_params_.count(); ++i) {
OZ(MY_SPEC.rescan_params_.at(i).set_dynamic_param(eval_ctx_));
}
return ret;
}
// int ObSubPlanFilter::construct_array_params(ObExecContext &ctx) const
// {
// int ret = OB_SUCCESS;
// ObExprCtx expr_ctx;
// ObPhysicalPlanCtx *plan_ctx = NULL;
// if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
// ret = OB_ERR_UNEXPECTED;
// LOG_WARN("physical plan context is null");
// } else if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
// LOG_WARN("wrap expr ctx failed", K(ret));
// } else if (OB_UNLIKELY(onetime_exprs_.count() != get_child_num() - 1)) {
// ret = OB_ERR_UNEXPECTED;
// LOG_WARN("onetime exprs isn't match with child operator count",
// K(ret), K(onetime_exprs_.count()), K(get_child_num()));
// } else {
// ObNewRow empty_row;
// ObObj tmp_obj;
// int64_t array_binding_count = plan_ctx->get_bind_array_count();
// ObIAllocator &allocator = ctx.get_allocator();
// for (int64_t i = 0; OB_SUCC(ret) && i < onetime_exprs_.count(); ++i) {
// ObSqlExpression *sql_expr = onetime_exprs_.at(i).first;
// int64_t param_idx = onetime_exprs_.at(i).second;
// pl::ObPLNestedTable *binding_array = NULL;
// void *ba_buf = NULL;
// void *data_buf = NULL;
// if (OB_ISNULL(sql_expr)) {
// ret = OB_ERR_UNEXPECTED;
// LOG_WARN("sql_expr is null");
// } else {
// int64_t ba_buf_size = static_cast<int64_t>(sizeof(pl::ObPLNestedTable));
// int64_t data_buf_size = static_cast<int64_t>(sizeof(ObObj) * array_binding_count);
// ObObj tmp_obj;
// if (OB_ISNULL(ba_buf = allocator.alloc(ba_buf_size))) {
// ret = OB_ALLOCATE_MEMORY_FAILED;
// LOG_WARN("allocate nested table failed", K(ret), K(ba_buf_size));
// } else if (OB_ISNULL(data_buf = allocator.alloc(data_buf_size))) {
// ret = OB_ALLOCATE_MEMORY_FAILED;
// LOG_WARN("allocate data buffer failed", K(ret), K(data_buf_size));
// } else {
// binding_array = new(ba_buf) pl::ObPLNestedTable();
// binding_array->set_data(data_buf);
// binding_array->set_count(array_binding_count);
// }
// }
// expr_ctx.cur_array_index_ = 0;
// while (OB_SUCC(ret)) {
// tmp_obj.reset();
// if (OB_FAIL(sql_expr->calc(expr_ctx, empty_row, tmp_obj))) {
// LOG_WARN("calc sql expression failed", K(ret));
// } else if (OB_FAIL(ob_write_obj(allocator, tmp_obj,
// static_cast<ObObj*>(data_buf)[expr_ctx.cur_array_index_]))) {
// LOG_WARN("write obj failed", K(ret), K(tmp_obj));
// } else if (OB_FAIL(get_child(static_cast<int32_t>(i + 1))->switch_iterator(ctx))) {
// if (OB_ITER_END != ret) {
// LOG_WARN("swtich child operator iterator failed", K(ret));
// }
// } else {
// ++expr_ctx.cur_array_index_;
// }
// }
// if (OB_ITER_END == ret) {
// ret = OB_SUCCESS;
// if (OB_UNLIKELY(0 == expr_ctx.cur_array_index_)) {
// //subquery does not contain bind array param
// plan_ctx->get_param_store_for_update().at(param_idx) =
// static_cast<ObObj*>(data_buf)[expr_ctx.cur_array_index_];
// } else if (OB_UNLIKELY(expr_ctx.cur_array_index_ != plan_ctx->get_bind_array_count() - 1)) {
// ret = OB_ERR_UNEXPECTED;
// LOG_WARN("current array index is invalid", K(ret), K_(expr_ctx.cur_array_index),
// K(plan_ctx->get_bind_array_count()));
// } else {
// plan_ctx->get_param_store_for_update().at(param_idx).set_ext(reinterpret_cast<int64_t>(binding_array));
// }
// }
// }
// }
// return ret;
// }
int ObSubPlanFilterOp::prepare_onetime_exprs()
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
if (OB_LIKELY(plan_ctx->get_bind_array_count() > 0)) {
// TODO : support array binding
// if (OB_FAIL(construct_array_params(ctx))) {
// LOG_WARN("construct array params failed", K(ret));
// }
} else {
ObDatum copyed;
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.onetime_exprs_.count(); ++i) {
const ObDynamicParamSetter& setter = MY_SPEC.onetime_exprs_.at(i);
ObDatum* datum = NULL;
if (OB_FAIL(setter.src_->eval(eval_ctx_, datum))) {
LOG_WARN("expression evaluate failed", K(ret));
} else if (OB_FAIL(copyed.deep_copy(*datum, ctx_.get_allocator()))) {
LOG_WARN("datum deep copy failed", K(ret));
} else if (OB_FAIL(setter.update_dynamic_param(eval_ctx_, copyed))) {
LOG_WARN("update dynamic param store failed", K(ret));
}
}
}
return ret;
}
int ObSubPlanFilterOp::handle_update_set()
{
int ret = OB_SUCCESS;
bool no_row = false;
const int64_t extra_size = 0;
if (NULL == update_set_mem_) {
lib::ContextParam param;
param.set_mem_attr(ctx_.get_my_session()->get_effective_tenant_id(), ObModIds::OB_SQL_EXECUTOR, ObCtxIds::WORK_AREA)
.set_properties(lib::USE_TL_PAGE_OPTIONAL);
if (OB_FAIL(CURRENT_CONTEXT.CREATE_CONTEXT(update_set_mem_, param))) {
LOG_WARN("create memory entity failed", K(ret));
}
} else {
update_set_mem_->get_arena_allocator().reuse();
}
if (OB_FAIL(ret)) {
} else if (1 != subplan_iters_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("too many subplan unexpected", K(ret), K(subplan_iters_.count()));
} else {
ObChunkDatumStore::LastStoredRow<> row_val(update_set_mem_->get_arena_allocator());
Iterator* iter = subplan_iters_.at(0);
if (OB_ISNULL(iter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("null iterator", K(ret));
} else if (OB_FAIL(iter->get_next_row())) {
if (OB_LIKELY(OB_ITER_END == ret)) {
ret = OB_SUCCESS;
no_row = true;
}
} else if (OB_FAIL(row_val.save_store_row(MY_SPEC.update_set_, eval_ctx_, extra_size))) {
LOG_WARN("deep copy row failed", K(ret));
} else {
if (OB_UNLIKELY(OB_SUCCESS == (ret = iter->get_next_row()))) {
ret = OB_ERR_MORE_THAN_ONE_ROW;
LOG_WARN("subquery too many rows", K(ret));
} else if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
}
if (OB_SUCC(ret)) {
if (no_row) {
FOREACH_CNT(expr, MY_SPEC.update_set_)
{
(*expr)->locate_expr_datum(eval_ctx_).set_null();
(*expr)->get_eval_info(eval_ctx_).evaluated_ = true;
}
} else {
OZ(row_val.store_row_->to_expr(MY_SPEC.update_set_, eval_ctx_));
}
}
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,135 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SUBQUERY_OB_SUBPLAN_FILTER_OP_H_
#define OCEANBASE_SUBQUERY_OB_SUBPLAN_FILTER_OP_H_
#include "sql/engine/ob_operator.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
namespace oceanbase {
namespace sql {
// iterator subquery rows
class ObSubQueryIterator {
public:
explicit ObSubQueryIterator(ObOperator& op);
~ObSubQueryIterator()
{}
void set_onetime_plan()
{
onetime_plan_ = true;
}
void set_init_plan()
{
init_plan_ = true;
}
const ExprFixedArray& get_output() const
{
return op_.get_spec().output_;
}
ObOperator& get_op() const
{
return op_;
}
// Call start() before get_next_row().
// We need this because the subquery() may be common subexpression, need be rewinded to the
// first row when iterate again.
int start();
int get_next_row();
int prepare_init_plan();
void reuse();
void reset(bool reset_onetime_plan = false);
TO_STRING_KV(K(onetime_plan_), K(init_plan_), K(inited_));
private:
ObOperator& op_;
bool onetime_plan_;
bool init_plan_;
bool inited_;
bool iterated_;
ObChunkDatumStore store_;
ObChunkDatumStore::Iterator store_it_;
};
class ObSubPlanFilterSpec : public ObOpSpec {
OB_UNIS_VERSION_V(1);
public:
ObSubPlanFilterSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
DECLARE_VIRTUAL_TO_STRING;
// row from driver table is the rescan params
common::ObFixedArray<ObDynamicParamSetter, common::ObIAllocator> rescan_params_;
// only compute once exprs
common::ObFixedArray<ObDynamicParamSetter, common::ObIAllocator> onetime_exprs_;
// InitPlan idxs,InitPlan only compute once, need save result
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> init_plan_idxs_;
// One-Time idxs,One-Time only compute once, no need save result
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> one_time_idxs_;
// update set (, ,) = (subquery)
ExprFixedArray update_set_;
};
class ObSubPlanFilterOp : public ObOperator {
public:
typedef ObSubQueryIterator Iterator;
ObSubPlanFilterOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
virtual ~ObSubPlanFilterOp();
virtual int open() override;
virtual int inner_open() override;
virtual int rescan() override;
virtual int switch_iterator() override;
virtual int inner_get_next_row() override;
virtual int inner_close() override;
virtual void destroy() override;
const common::ObIArray<Iterator*>& get_subplan_iters() const
{
return subplan_iters_;
}
private:
int set_param_null();
void destroy_subplan_iters();
void destroy_update_set_mem()
{
if (NULL != update_set_mem_) {
DESTROY_CONTEXT(update_set_mem_);
update_set_mem_ = NULL;
}
}
int prepare_rescan_params();
int prepare_onetime_exprs();
int handle_update_set();
private:
common::ObSEArray<Iterator*, 16> subplan_iters_;
lib::MemoryContext* update_set_mem_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_FILTER_OP_H_

View File

@ -0,0 +1,135 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/subquery/ob_subplan_scan.h"
#include "sql/engine/ob_phy_operator_type.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
class ObSubPlanScan::ObSubPlanScanCtx : public ObPhyOperatorCtx {
public:
explicit ObSubPlanScanCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx)
{}
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
friend class ObSubPlanScan;
};
ObSubPlanScan::ObSubPlanScan(ObIAllocator& alloc) : ObSingleChildPhyOperator(alloc), output_indexs_(alloc)
{}
ObSubPlanScan::~ObSubPlanScan()
{}
OB_SERIALIZE_MEMBER((ObSubPlanScan, ObSingleChildPhyOperator), output_indexs_);
void ObSubPlanScan::reset()
{
output_indexs_.reset();
ObSingleChildPhyOperator::reset();
}
int ObSubPlanScan::add_output_index(int64_t index)
{
return output_indexs_.push_back(index);
}
int ObSubPlanScan::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObSubPlanScanCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("failed to create SubQueryCtx", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op_ctx is null");
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
LOG_WARN("create current row failed", K(ret));
}
return ret;
}
int ObSubPlanScan::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("init subquery scan context failed", K(ret));
}
return ret;
}
int ObSubPlanScan::rescan(ObExecContext& ctx) const
{
// do noting, just rescan child operator
return ObSingleChildPhyOperator::rescan(ctx);
}
int ObSubPlanScan::inner_close(ObExecContext& ctx) const
{
UNUSED(ctx);
return OB_SUCCESS;
}
int ObSubPlanScan::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
// The role of ObSubQueryScan is similar to TableScan,
// which generates new rows from the results of the iteration
// in the subquery according to the data required by the upper query
int ret = OB_SUCCESS;
const ObNewRow* input_row = NULL;
ObSubPlanScanCtx* subquery_ctx = NULL;
if (OB_ISNULL(subquery_ctx = GET_PHY_OPERATOR_CTX(ObSubPlanScanCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get subquery scan context failed", K(ret));
} else if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from child operator failed", K(ret));
} else { /*do nothing*/
}
} else if (OB_ISNULL(input_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("input_row is null");
} else {
// generate new row in subquery scan
ObNewRow& output_row = subquery_ctx->get_cur_row();
if (OB_ISNULL(output_row.cells_) || OB_UNLIKELY(output_row.count_ <= 0)) {
ret = OB_NOT_INIT;
LOG_WARN("current row not init");
}
for (int64_t i = 0; OB_SUCC(ret) && i < output_indexs_.count(); ++i) {
int64_t col_idx = output_indexs_.at(i);
if (OB_UNLIKELY(i >= output_row.count_) || OB_UNLIKELY(col_idx >= input_row->get_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row not match", K(i), K(output_row), K(*input_row));
} else {
output_row.cells_[i] = input_row->get_cell(col_idx);
}
}
if (OB_SUCC(ret)) {
row = &output_row;
}
}
return ret;
}
int64_t ObSubPlanScan::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(K_(output_indexs));
return pos;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,80 @@
/**
* 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_SRC_SQL_ENGINE_SUBQUERY_OB_SUBQUERY_SCAN_H_
#define OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_SUBQUERY_SCAN_H_
#include "sql/engine/ob_single_child_phy_operator.h"
#include "lib/container/ob_fixed_array.h"
namespace oceanbase {
namespace sql {
class ObSubPlanScan : public ObSingleChildPhyOperator {
OB_UNIS_VERSION(1);
private:
class ObSubPlanScanCtx;
public:
explicit ObSubPlanScan(common::ObIAllocator& alloc);
virtual ~ObSubPlanScan();
virtual void reset();
virtual int rescan(ObExecContext& ctx) const;
int add_output_index(int64_t index);
int init_output_index(int64_t count)
{
return init_array_size<>(output_indexs_, count);
}
private:
/**
* @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
*/
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;
/**
* @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;
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObSubPlanScan);
private:
// The information saved by output_indexs_ is the index of the column iterated by the lower operator that needs to be
// output
common::ObFixedArray<int64_t, common::ObIAllocator> output_indexs_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_SUBQUERY_SCAN_H_ */

View File

@ -0,0 +1,74 @@
/**
* 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_subplan_scan_op.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObSubPlanScanSpec::ObSubPlanScanSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type), projector_(alloc)
{}
OB_SERIALIZE_MEMBER((ObSubPlanScanSpec, ObOpSpec), projector_);
ObSubPlanScanOp::ObSubPlanScanOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input)
{}
int ObSubPlanScanOp::inner_open()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(child_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("no child", K(ret));
} else if (OB_UNLIKELY(MY_SPEC.projector_.count() % 2 != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("projector array size should be multiples of 2", K(ret));
}
return ret;
}
int ObSubPlanScanOp::rescan()
{
return ObOperator::rescan();
}
int ObSubPlanScanOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_FAIL(child_->get_next_row())) {
if (OB_ITER_END != ret) {
LOG_WARN("get row from child failed", K(ret));
}
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.projector_.count(); i += 2) {
ObExpr* from = MY_SPEC.projector_[i];
ObExpr* to = MY_SPEC.projector_[i + 1];
ObDatum* datum = NULL;
if (OB_FAIL(from->eval(eval_ctx_, datum))) {
LOG_WARN("expr evaluate failed", K(ret), K(*from));
} else {
to->locate_expr_datum(eval_ctx_) = *datum;
to->get_eval_info(eval_ctx_).evaluated_ = true;
}
}
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,52 @@
/**
* 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_SUBQUERY_OB_SUBPLAN_SCAN_OP_H_
#define OCEANBASE_SUBQUERY_OB_SUBPLAN_SCAN_OP_H_
#include "sql/engine/ob_operator.h"
namespace oceanbase {
namespace sql {
class ObSubPlanScanSpec : public ObOpSpec {
OB_UNIS_VERSION_V(1);
public:
ObSubPlanScanSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
// project child output to subplan scan column.
// projector_is filled with [child output, scan column] paires, even index is child output,
// odd is scan column. e.g.:
// [child output, scan column, child output, scan column, ...]
ExprFixedArray projector_;
};
class ObSubPlanScanOp : public ObOperator {
public:
ObSubPlanScanOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
virtual int inner_open() override;
virtual int rescan() override;
virtual int inner_get_next_row() override;
virtual void destroy() override
{
ObOperator::destroy();
}
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_SCAN_OP_H_

View File

@ -0,0 +1,237 @@
/**
* 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/subquery/ob_unpivot.h"
#include "sql/engine/ob_phy_operator_type.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
class ObUnpivot::ObUnpivotCtx : public ObPhyOperatorCtx {
public:
explicit ObUnpivotCtx(ObExecContext& ctx)
: ObPhyOperatorCtx(ctx), last_input_row_(NULL), is_end_(false), curr_part_idx_(common::OB_INVALID_INDEX)
{}
void reset()
{
last_input_row_ = NULL;
is_end_ = false;
curr_part_idx_ = common::OB_INVALID_INDEX;
}
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
friend class ObUnpivot;
public:
const ObNewRow* last_input_row_;
bool is_end_;
int64_t curr_part_idx_;
};
ObUnpivot::ObUnpivot(ObIAllocator& alloc) : ObSingleChildPhyOperator(alloc), output_indexs_(alloc), unpivot_info_()
{}
ObUnpivot::~ObUnpivot()
{}
OB_SERIALIZE_MEMBER((ObUnpivot, ObSingleChildPhyOperator), output_indexs_, unpivot_info_);
void ObUnpivot::reset()
{
unpivot_info_.reset();
output_indexs_.reset();
ObSingleChildPhyOperator::reset();
}
int ObUnpivot::add_output_index(int64_t index)
{
return output_indexs_.push_back(index);
}
int ObUnpivot::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObUnpivotCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("failed to create SubQueryCtx", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op_ctx is null");
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
LOG_WARN("create current row failed", K(ret));
} else {
static_cast<ObUnpivotCtx*>(op_ctx)->reset();
}
return ret;
}
int ObUnpivot::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("init subquery scan context failed", K(ret));
}
return ret;
}
int ObUnpivot::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObUnpivotCtx* unpivot_ctx = NULL;
if (OB_ISNULL(unpivot_ctx = GET_PHY_OPERATOR_CTX(ObUnpivotCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get subquery scan context failed", K(ret));
} else {
unpivot_ctx->reset();
ret = ObSingleChildPhyOperator::rescan(ctx);
}
// do noting, just rescan child operator
return ret;
}
int ObUnpivot::inner_close(ObExecContext& ctx) const
{
UNUSED(ctx);
return OB_SUCCESS;
}
int ObUnpivot::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
// The role of ObUnpivot is similar to that of SubQueryScan, which generates new rows from
// the results of the iteration in the subquery according to the data required by the upper query
// but need transpose unpivot_mocked_column to new row
int ret = OB_SUCCESS;
ObUnpivotCtx* unpivot_ctx = NULL;
if (OB_ISNULL(unpivot_ctx = GET_PHY_OPERATOR_CTX(ObUnpivotCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get subquery scan context failed", K(ret));
} else {
const ObNewRow* input_row = NULL;
ObNewRow& output_row = unpivot_ctx->get_cur_row();
bool got_next_row = false;
const int64_t max_part_count =
(output_indexs_.count() - unpivot_info_.old_column_count_) / unpivot_info_.get_new_column_count();
if (unpivot_ctx->curr_part_idx_ <= 0 || unpivot_ctx->curr_part_idx_ >= max_part_count) {
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from child operator failed", K(ret));
}
} else if (OB_ISNULL(input_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("input_row is null");
} else {
unpivot_ctx->curr_part_idx_ = 0;
unpivot_ctx->last_input_row_ = input_row;
got_next_row = true;
}
} else {
input_row = unpivot_ctx->last_input_row_;
}
LOG_DEBUG("arrive unpivot",
K(ret),
KPC(input_row),
K(output_row),
K(output_indexs_),
K(unpivot_ctx->curr_part_idx_),
K(max_part_count),
K(unpivot_info_));
if (!unpivot_info_.is_include_null_) {
bool need_try_next_part = true;
while (need_try_next_part && OB_SUCC(ret)) {
if (unpivot_ctx->curr_part_idx_ >= max_part_count) {
if (got_next_row) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("has already get next row once, maybe filter does not work", K(ret), KPC(input_row));
} else if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from child operator failed", K(ret));
}
} else if (OB_ISNULL(input_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("input_row is null");
} else {
unpivot_ctx->curr_part_idx_ = 0;
unpivot_ctx->last_input_row_ = input_row;
got_next_row = true;
}
}
int64_t null_count = 0;
const int64_t base_idx = unpivot_ctx->curr_part_idx_ * unpivot_info_.get_new_column_count() +
unpivot_info_.old_column_count_ + unpivot_info_.for_column_count_;
for (int64_t i = 0; OB_SUCC(ret) && i < unpivot_info_.unpivot_column_count_; ++i) {
const int64_t col_idx = output_indexs_.at(i + base_idx);
if (OB_UNLIKELY(col_idx >= input_row->get_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row not match", K(ret), K(i), K(base_idx), K(col_idx), K(input_row->get_count()), KPC(input_row));
} else {
null_count += input_row->get_cell(col_idx).is_null_oracle();
}
}
if (OB_SUCC(ret)) {
if (null_count == unpivot_info_.unpivot_column_count_) {
LOG_DEBUG(
"is null, try next row", K(ret), K(unpivot_ctx->curr_part_idx_), K(unpivot_info_), KPC(input_row));
unpivot_ctx->curr_part_idx_++;
} else {
need_try_next_part = false;
}
}
} // end of while
} // end of exclude null
if (OB_SUCC(ret)) {
int64_t output_idx = 0;
for (; OB_SUCC(ret) && output_idx < unpivot_info_.old_column_count_; ++output_idx) {
const int64_t col_idx = output_indexs_.at(output_idx);
if (OB_UNLIKELY(col_idx >= input_row->get_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row not match", K(output_idx), K(col_idx), K(output_row), KPC(input_row));
} else {
output_row.cells_[output_idx] = input_row->get_cell(col_idx);
}
}
const int64_t base_idx = unpivot_ctx->curr_part_idx_ * unpivot_info_.get_new_column_count();
for (; OB_SUCC(ret) && output_idx < output_row.get_count(); ++output_idx) {
int64_t col_idx = output_indexs_.at(output_idx + base_idx);
if (OB_UNLIKELY(col_idx >= input_row->get_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row not match", K(ret), K(col_idx), K(output_row), KPC(input_row));
} else {
output_row.cells_[output_idx] = input_row->get_cell(col_idx);
}
}
}
if (OB_SUCC(ret)) {
row = &output_row;
LOG_DEBUG("output next row", K(unpivot_ctx->curr_part_idx_), K(unpivot_info_), K(output_row), KPC(input_row));
++unpivot_ctx->curr_part_idx_;
}
}
return ret;
}
int64_t ObUnpivot::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(K_(output_indexs), K_(unpivot_info));
return pos;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,80 @@
/**
* 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_SRC_SQL_ENGINE_SUBQUERY_OB_UNPIVOT_H_
#define OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_UNPIVOT_H_
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/resolver/dml/ob_dml_stmt.h"
namespace oceanbase {
namespace sql {
class ObUnpivot : public ObSingleChildPhyOperator {
OB_UNIS_VERSION(1);
private:
class ObUnpivotCtx;
public:
explicit ObUnpivot(common::ObIAllocator& alloc);
virtual ~ObUnpivot();
virtual void reset();
virtual int rescan(ObExecContext& ctx) const;
int add_output_index(int64_t index);
int init_output_index(int64_t count)
{
return init_array_size<>(output_indexs_, count);
}
private:
/**
* @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
*/
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;
/**
* @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;
public:
common::ObFixedArray<int64_t, common::ObIAllocator> output_indexs_;
ObUnpivotInfo unpivot_info_;
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObUnpivot);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_UNPIVOT_H_ */