patch 4.0
This commit is contained in:
@ -1,858 +0,0 @@
|
||||
/**
|
||||
* 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::reset_rescan_params(ObExecContext &ctx) const {
|
||||
int ret = OB_SUCCESS;
|
||||
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", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; 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_TRACE("prepare_rescan_params", K(ret), K(i), K(idx));
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSubPlanFilter::rescan(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_FAIL(reset_rescan_params(ctx))) {
|
||||
LOG_WARN("fail to reset rescan params", K(ret));
|
||||
}
|
||||
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));
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(reset_rescan_params(ctx))) {
|
||||
LOG_WARN("fail to reset rescan params", K(ret));
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
@ -1,121 +0,0 @@
|
||||
/**
|
||||
* 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() override;
|
||||
void reuse() override;
|
||||
int rescan(ObExecContext& ctx) const override;
|
||||
|
||||
/**
|
||||
* @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 override;
|
||||
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 override;
|
||||
/**
|
||||
* @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 override;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const override;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const override;
|
||||
/**
|
||||
* @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 override;
|
||||
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 override;
|
||||
int handle_update_set(ObSubPlanFilterCtx* subplan_ctx, const common::ObNewRow*& row) const;
|
||||
int construct_array_params(ObExecContext& ctx) const;
|
||||
int reset_rescan_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_ */
|
||||
File diff suppressed because it is too large
Load Diff
@ -16,85 +16,154 @@
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
class DatumRow
|
||||
{
|
||||
public:
|
||||
DatumRow() : elems_(NULL), cnt_(0) {}
|
||||
~DatumRow() {}
|
||||
bool operator==(const DatumRow &other) const;
|
||||
uint64_t hash(uint64_t seed=0) const;
|
||||
TO_STRING_KV(KP(elems_));
|
||||
ObDatum *elems_;
|
||||
int64_t cnt_;
|
||||
};
|
||||
|
||||
// iterator subquery rows
|
||||
class ObSubQueryIterator {
|
||||
class ObSubQueryIterator
|
||||
{
|
||||
public:
|
||||
explicit ObSubQueryIterator(ObOperator& op);
|
||||
explicit ObSubQueryIterator(ObOperator &op);
|
||||
~ObSubQueryIterator()
|
||||
{}
|
||||
void set_onetime_plan()
|
||||
{
|
||||
onetime_plan_ = true;
|
||||
if (hashmap_.created()) {
|
||||
hashmap_.destroy();
|
||||
}
|
||||
}
|
||||
void set_init_plan()
|
||||
enum RescanStatus
|
||||
{
|
||||
init_plan_ = true;
|
||||
}
|
||||
INVALID_STATUS = 0,
|
||||
SWITCH_BATCH,
|
||||
NOT_SWITCH_BATCH,
|
||||
NORMAL
|
||||
};
|
||||
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_;
|
||||
}
|
||||
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);
|
||||
int rewind(bool reset_onetime_plan = false);
|
||||
//int rescan_chlid(int64_t child_idx);
|
||||
int init_mem_entity();
|
||||
int init_hashmap(const int64_t param_num)
|
||||
{
|
||||
return hashmap_.create(param_num * 2, common::ObModIds::OB_HASH_BUCKET);
|
||||
}
|
||||
bool has_hashmap() const { return hashmap_.created(); }
|
||||
int init_probe_row(const int64_t cnt);
|
||||
int get_arena_allocator(common::ObIAllocator *&alloc);
|
||||
//fill curr exec param into probe_row_
|
||||
int get_curr_probe_row();
|
||||
void set_iter_id(const int64_t id) { id_ = id; }
|
||||
int64_t get_iter_id() const { return id_; }
|
||||
//use curr probe_row_ to probe hashmap
|
||||
int get_refactored(common::ObDatum &out);
|
||||
//set row into hashmap
|
||||
int set_refactored(const DatumRow &row, const ObDatum &result, const int64_t deep_copy_size);
|
||||
void set_parent(const ObSubPlanFilterOp *filter) { parent_ = filter; }
|
||||
int reset_hash_map();
|
||||
|
||||
bool check_can_insert(const int64_t deep_copy_size)
|
||||
{
|
||||
return deep_copy_size + memory_used_ < HASH_MAP_MEMORY_LIMIT;
|
||||
}
|
||||
|
||||
ObEvalCtx &get_eval_ctx() { return eval_ctx_; }
|
||||
|
||||
//for vectorized
|
||||
int get_next_batch(const int64_t max_row_cnt, const ObBatchRows *&batch_rows);
|
||||
//for vectorized end
|
||||
bool is_onetime_plan() const { return onetime_plan_; }
|
||||
TO_STRING_KV(K(onetime_plan_), K(init_plan_), K(inited_));
|
||||
|
||||
//a row cache for hash optimizer to use
|
||||
DatumRow probe_row_;
|
||||
//hard core, 1M limit for each hashmap
|
||||
const static int HASH_MAP_MEMORY_LIMIT = 1024 * 1024;
|
||||
|
||||
private:
|
||||
ObOperator& op_;
|
||||
|
||||
ObOperator &op_;
|
||||
bool onetime_plan_;
|
||||
bool init_plan_;
|
||||
bool inited_;
|
||||
bool iterated_;
|
||||
|
||||
ObChunkDatumStore store_;
|
||||
ObChunkDatumStore::Iterator store_it_;
|
||||
|
||||
//cache optimizer for spf, the same exec_param into queryref_expr will return directly
|
||||
common::hash::ObHashMap<DatumRow, common::ObDatum, common::hash::NoPthreadDefendMode> hashmap_;
|
||||
lib::MemoryContext mem_entity_;
|
||||
int64_t id_; // curr op_id in spf
|
||||
const ObSubPlanFilterOp *parent_; //needs to get exec_param_idxs_ from op
|
||||
int64_t memory_used_;
|
||||
ObEvalCtx &eval_ctx_;
|
||||
|
||||
// for vectorized
|
||||
const ObBatchRows *iter_brs_;
|
||||
int64_t batch_size_;
|
||||
int64_t batch_row_pos_;
|
||||
bool iter_end_;
|
||||
// for vectorized end
|
||||
};
|
||||
|
||||
class ObSubPlanFilterSpec : public ObOpSpec {
|
||||
class ObSubPlanFilterSpec : public ObOpSpec
|
||||
{
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObSubPlanFilterSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
ObSubPlanFilterSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type);
|
||||
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
int init_px_batch_rescan_flags(int64_t count)
|
||||
{ return enable_px_batch_rescans_.init(count); }
|
||||
|
||||
// row from driver table is the rescan params
|
||||
//在主表的每次迭代生成的行数据对于subquery来说都是驱动其进行数据迭代的参数
|
||||
common::ObFixedArray<ObDynamicParamSetter, common::ObIAllocator> rescan_params_;
|
||||
// only compute once exprs
|
||||
//只计算一次subquery条件
|
||||
common::ObFixedArray<ObDynamicParamSetter, common::ObIAllocator> onetime_exprs_;
|
||||
// InitPlan idxs,InitPlan only compute once, need save result
|
||||
//InitPlan idxs,InitPlan只算一次,需要存储结果
|
||||
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> init_plan_idxs_;
|
||||
// One-Time idxs,One-Time only compute once, no need save result
|
||||
//One-Time idxs,One-Time只算一次,不用存储结果
|
||||
common::ObBitSet<common::OB_DEFAULT_BITSET_SIZE, common::ModulePageAllocator> one_time_idxs_;
|
||||
|
||||
// update set (, ,) = (subquery)
|
||||
ExprFixedArray update_set_;
|
||||
common::ObFixedArray<ObFixedArray<ObExpr *, common::ObIAllocator>, common::ObIAllocator> exec_param_array_;
|
||||
bool exec_param_idxs_inited_;
|
||||
// 标记每个子查询是否可以做px batch rescan
|
||||
common::ObFixedArray<bool, common::ObIAllocator> enable_px_batch_rescans_;
|
||||
bool enable_das_batch_rescans_;
|
||||
ExprFixedArray filter_exprs_;
|
||||
ExprFixedArray output_exprs_;
|
||||
};
|
||||
|
||||
class ObSubPlanFilterOp : public ObOperator {
|
||||
class ObSubPlanFilterOp : public ObOperator
|
||||
{
|
||||
public:
|
||||
typedef ObSubQueryIterator Iterator;
|
||||
|
||||
ObSubPlanFilterOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
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;
|
||||
@ -104,14 +173,29 @@ public:
|
||||
|
||||
virtual void destroy() override;
|
||||
|
||||
const common::ObIArray<Iterator*>& get_subplan_iters() const
|
||||
const common::ObIArray<Iterator *> &get_subplan_iters() const { return subplan_iters_; }
|
||||
int reset_batch_rescan_param()
|
||||
{
|
||||
return subplan_iters_;
|
||||
rescan_batch_params_.reset();
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
int handle_next_row();
|
||||
bool enable_px_batch_rescan() { return enable_left_px_batch_; }
|
||||
bool enable_das_batch_rescan() { return enable_left_das_batch_; }
|
||||
//for vectorized
|
||||
int inner_get_next_batch(const int64_t max_row_cnt);
|
||||
// for vectorized end
|
||||
|
||||
int init_left_cur_row(const int64_t column_cnt, ObExecContext &ctx);
|
||||
int fill_cur_row_rescan_param();
|
||||
public:
|
||||
ObBatchRescanCtl &get_batch_rescan_ctl() { return batch_rescan_ctl_; }
|
||||
static const int64_t PX_RESCAN_BATCH_ROW_COUNT = 8192;
|
||||
int handle_next_batch_with_px_rescan(const int64_t op_max_batch_size);
|
||||
private:
|
||||
int set_param_null();
|
||||
void set_param_null() { set_pushdown_param_null(MY_SPEC.rescan_params_); };
|
||||
void destroy_subplan_iters();
|
||||
void destroy_px_batch_rescan_status();
|
||||
void destroy_update_set_mem()
|
||||
{
|
||||
if (NULL != update_set_mem_) {
|
||||
@ -120,16 +204,40 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
int prepare_rescan_params();
|
||||
int prepare_rescan_params(bool save);
|
||||
int prepare_onetime_exprs();
|
||||
int prepare_onetime_exprs_inner();
|
||||
int handle_update_set();
|
||||
bool continue_fetching(uint64_t left_rows_total_cnt, bool stop)
|
||||
{
|
||||
return (!stop && (left_rows_total_cnt < PX_RESCAN_BATCH_ROW_COUNT));
|
||||
}
|
||||
|
||||
private:
|
||||
common::ObSEArray<Iterator*, 16> subplan_iters_;
|
||||
common::ObSEArray<Iterator *, 16> subplan_iters_;
|
||||
lib::MemoryContext update_set_mem_;
|
||||
bool iter_end_;
|
||||
// for px batch rescan
|
||||
bool enable_left_px_batch_;
|
||||
// for das batch rescan
|
||||
bool enable_left_das_batch_;
|
||||
ObChunkDatumStore left_rows_;
|
||||
ObChunkDatumStore::Iterator left_rows_iter_;
|
||||
ObChunkDatumStore::ShadowStoredRow last_store_row_;
|
||||
bool save_last_row_;
|
||||
bool is_left_end_;
|
||||
ObBatchRescanParams rescan_batch_params_;
|
||||
int64_t left_row_idx_;
|
||||
ObBatchRescanCtl batch_rescan_ctl_;
|
||||
common::ObSEArray<common::ObObjParam, 8> cur_params_;
|
||||
common::ObSArray<int64_t> cur_param_idxs_;
|
||||
common::ObSArray<int64_t> cur_param_expr_idxs_;
|
||||
common::ObSEArray<Iterator*, 8> subplan_iters_to_check_;
|
||||
lib::MemoryContext last_store_row_mem_;
|
||||
ObBatchResultHolder brs_holder_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_FILTER_OP_H_
|
||||
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_FILTER_OP_H_
|
||||
|
||||
@ -1,135 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
@ -1,80 +0,0 @@
|
||||
/**
|
||||
* 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_ */
|
||||
@ -14,19 +14,25 @@
|
||||
|
||||
#include "ob_subplan_scan_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace oceanbase
|
||||
{
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
namespace sql
|
||||
{
|
||||
|
||||
ObSubPlanScanSpec::ObSubPlanScanSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
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)
|
||||
{}
|
||||
|
||||
ObSubPlanScanOp::ObSubPlanScanOp(
|
||||
ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input)
|
||||
: ObOperator(exec_ctx, spec, input)
|
||||
{
|
||||
}
|
||||
|
||||
int ObSubPlanScanOp::inner_open()
|
||||
{
|
||||
@ -41,9 +47,9 @@ int ObSubPlanScanOp::inner_open()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSubPlanScanOp::rescan()
|
||||
int ObSubPlanScanOp::inner_rescan()
|
||||
{
|
||||
return ObOperator::rescan();
|
||||
return ObOperator::inner_rescan();
|
||||
}
|
||||
|
||||
int ObSubPlanScanOp::inner_get_next_row()
|
||||
@ -67,20 +73,65 @@ int ObSubPlanScanOp::inner_get_next_row()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
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;
|
||||
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;
|
||||
to->set_evaluated_projected(eval_ctx_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
int ObSubPlanScanOp::inner_get_next_batch(const int64_t max_row_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
const ObBatchRows *child_brs = nullptr;
|
||||
if (OB_FAIL(child_->get_next_batch(max_row_cnt, child_brs))) {
|
||||
LOG_WARN("get child next batch failed", K(ret));
|
||||
} else if (child_brs->end_ && 0 == child_brs->size_) {
|
||||
brs_.copy(child_brs);
|
||||
} else {
|
||||
brs_.copy(child_brs);
|
||||
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];
|
||||
if (OB_FAIL(from->eval_batch(eval_ctx_, *brs_.skip_, brs_.size_))) {
|
||||
LOG_WARN("eval batch failed", K(ret));
|
||||
} else {
|
||||
ObDatum *from_datums = from->locate_batch_datums(eval_ctx_);
|
||||
ObDatum *to_datums = to->locate_batch_datums(eval_ctx_);
|
||||
const ObEvalInfo &from_info = from->get_eval_info(eval_ctx_);
|
||||
ObEvalInfo &to_info = to->get_eval_info(eval_ctx_);
|
||||
if (OB_UNLIKELY(!to->is_batch_result())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("output of subplan scan should be batch result", K(ret), KPC(to));
|
||||
} else if (from->is_batch_result()) {
|
||||
MEMCPY(to_datums, from_datums, brs_.size_ * sizeof(ObDatum));
|
||||
to_info = from_info;
|
||||
to_info.projected_ = true;
|
||||
to_info.point_to_frame_ = false;
|
||||
} else {
|
||||
for (int64_t j = 0; j < brs_.size_; j++) {
|
||||
to_datums[j] = *from_datums;
|
||||
}
|
||||
to_info = from_info;
|
||||
to_info.projected_ = true;
|
||||
to_info.point_to_frame_ = false;
|
||||
to_info.cnt_ = brs_.size_;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
@ -15,14 +15,16 @@
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
class ObSubPlanScanSpec : public ObOpSpec {
|
||||
class ObSubPlanScanSpec : public ObOpSpec
|
||||
{
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObSubPlanScanSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
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.:
|
||||
@ -30,23 +32,23 @@ public:
|
||||
ExprFixedArray projector_;
|
||||
};
|
||||
|
||||
class ObSubPlanScanOp : public ObOperator {
|
||||
class ObSubPlanScanOp : public ObOperator
|
||||
{
|
||||
public:
|
||||
ObSubPlanScanOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
ObSubPlanScanOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
|
||||
virtual int rescan() override;
|
||||
virtual int inner_rescan() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
virtual int inner_get_next_batch(const int64_t max_row_cnt) override;
|
||||
|
||||
virtual void destroy() override { ObOperator::destroy(); }
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_SCAN_OP_H_
|
||||
#endif // OCEANBASE_SUBQUERY_OB_SUBPLAN_SCAN_OP_H_
|
||||
|
||||
@ -1,237 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
@ -1,80 +0,0 @@
|
||||
/**
|
||||
* 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_ */
|
||||
286
src/sql/engine/subquery/ob_unpivot_op.cpp
Normal file
286
src/sql/engine/subquery/ob_unpivot_op.cpp
Normal file
@ -0,0 +1,286 @@
|
||||
/**
|
||||
* 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_op.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
ObUnpivotSpec::ObUnpivotSpec(ObIAllocator &alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), max_part_count_(-1), unpivot_info_(false, 0, 0, 0) {}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObUnpivotSpec, ObOpSpec), max_part_count_, unpivot_info_);
|
||||
|
||||
ObUnpivotOp::ObUnpivotOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input)
|
||||
: ObOperator(exec_ctx, spec, input), curr_part_idx_(-1), curr_cols_idx_(-1),
|
||||
child_brs_(NULL), multiplex_(NULL) {}
|
||||
|
||||
int ObUnpivotOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::inner_open())) {
|
||||
LOG_WARN("failed to open in base class", K(ret));
|
||||
} else if (MY_SPEC.is_vectorized()) {
|
||||
int64_t size = MY_SPEC.max_batch_size_ * MY_SPEC.get_output_count() * sizeof(ObDatum);
|
||||
multiplex_ = static_cast<ObDatum *>(ctx_.get_allocator().alloc(size));
|
||||
if (size > 0 && OB_ISNULL(multiplex_)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret), K(size), K(multiplex_));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObUnpivotOp::inner_close()
|
||||
{
|
||||
return ObOperator::inner_close();
|
||||
}
|
||||
|
||||
int ObUnpivotOp::inner_rescan()
|
||||
{
|
||||
reset();
|
||||
return ObOperator::inner_rescan();
|
||||
}
|
||||
|
||||
int ObUnpivotOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool got_next_row = false;
|
||||
clear_evaluated_flag();
|
||||
if (curr_part_idx_ < 0 || curr_part_idx_ >= MY_SPEC.max_part_count_) {
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from child operator failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
curr_part_idx_ = 0;
|
||||
got_next_row = true;
|
||||
}
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
LOG_DEBUG("arrive unpivot", K(ret), K(child_->get_spec().output_), K(MY_SPEC.output_),
|
||||
K(curr_part_idx_), K(MY_SPEC.max_part_count_), K(MY_SPEC.unpivot_info_));
|
||||
|
||||
if (OB_SUCC(ret) && !MY_SPEC.unpivot_info_.is_include_null_) {
|
||||
bool need_try_next_part = true;
|
||||
while (need_try_next_part && OB_SUCC(ret)) {
|
||||
if (curr_part_idx_ >= MY_SPEC.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),
|
||||
K(child_->get_spec().output_));
|
||||
} else if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from child operator failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
curr_part_idx_ = 0;
|
||||
got_next_row = true;
|
||||
}
|
||||
}
|
||||
|
||||
int64_t null_count = 0;
|
||||
const int64_t base_idx = curr_part_idx_ * MY_SPEC.unpivot_info_.get_new_column_count()
|
||||
+ MY_SPEC.unpivot_info_.old_column_count_
|
||||
+ MY_SPEC.unpivot_info_.for_column_count_;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.unpivot_info_.unpivot_column_count_; ++i) {
|
||||
const int64_t input_idx = i + base_idx;
|
||||
ObDatum *datum = NULL;
|
||||
ObExpr *expr = child_->get_spec().output_[input_idx];
|
||||
if (OB_FAIL(expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("expr evaluate failed", K(ret), K(expr));
|
||||
} else if (datum->is_null()) {
|
||||
++null_count;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
if (null_count == MY_SPEC.unpivot_info_.unpivot_column_count_) {
|
||||
LOG_DEBUG("is null, try next row", K(ret), K(curr_part_idx_),
|
||||
K(MY_SPEC.unpivot_info_), K(child_->get_spec().output_));
|
||||
++curr_part_idx_;
|
||||
} else {
|
||||
need_try_next_part = false;
|
||||
}
|
||||
}
|
||||
} // end of while
|
||||
} // end of exclude null
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
// old_column is the same as child, pass
|
||||
int64_t output_idx = MY_SPEC.unpivot_info_.old_column_count_;
|
||||
const int64_t base_idx = curr_part_idx_ * MY_SPEC.unpivot_info_.get_new_column_count();
|
||||
for (; OB_SUCC(ret) && output_idx < MY_SPEC.get_output_count(); ++output_idx) {
|
||||
const int64_t input_idx = output_idx + base_idx;
|
||||
ObDatum *datum = NULL;
|
||||
ObExpr *expr = child_->get_spec().output_[input_idx];
|
||||
if (OB_FAIL(expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("expr evaluate failed", K(ret), K(expr));
|
||||
} else {
|
||||
MY_SPEC.output_[output_idx]->locate_expr_datum(eval_ctx_) = *datum;
|
||||
MY_SPEC.output_[output_idx]->set_evaluated_projected(eval_ctx_);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("output next row", K(curr_part_idx_), K(MY_SPEC.unpivot_info_),
|
||||
K(child_->get_spec().output_), K(MY_SPEC.output_));
|
||||
++curr_part_idx_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObUnpivotOp::inner_get_next_batch(const int64_t max_row_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObBatchRows *child_brs = nullptr;
|
||||
int64_t batch_size = common::min(max_row_cnt, MY_SPEC.max_batch_size_);
|
||||
clear_evaluated_flag();
|
||||
|
||||
if (curr_part_idx_ < 0 || curr_part_idx_ >= MY_SPEC.max_part_count_) {
|
||||
const ObBatchRows *child_brs = nullptr;
|
||||
if (OB_FAIL(child_->get_next_batch(batch_size, child_brs))) {
|
||||
LOG_WARN("get next batch from child operator failed", K(ret));
|
||||
} else if (child_brs->end_ && 0 == child_brs->size_) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
brs_.size_ = child_brs->size_;
|
||||
child_brs_ = child_brs;
|
||||
curr_part_idx_ = 0;
|
||||
curr_cols_idx_ = 0;
|
||||
int64_t pos = 0;
|
||||
MEMSET(multiplex_, 0, MY_SPEC.max_batch_size_ * MY_SPEC.get_output_count() * sizeof(ObDatum));
|
||||
for (int64_t cols = 0; OB_SUCC(ret) && cols < MY_SPEC.get_output_count(); ++cols) {
|
||||
ObExpr *child_expr = child_->get_spec().output_[cols];
|
||||
ObDatum *child_datum = child_expr->locate_batch_datums(eval_ctx_);
|
||||
if (child_expr->is_batch_result()) {
|
||||
MEMCPY(multiplex_ + pos, child_datum, child_brs->size_ * sizeof(ObDatum));
|
||||
pos += child_brs->size_;
|
||||
} else {
|
||||
for (int64_t rows = 0; rows < child_brs->size_; ++rows) {
|
||||
MEMCPY(multiplex_ + pos++, child_datum, sizeof(ObDatum));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
brs_.size_ = child_brs_->size_;
|
||||
}
|
||||
|
||||
LOG_DEBUG("arrive unpivot batch", K(ret), K(child_->get_spec().output_), K(MY_SPEC.output_),
|
||||
K(curr_part_idx_), K(MY_SPEC.max_part_count_), K(MY_SPEC.unpivot_info_));
|
||||
|
||||
for (int64_t read_piece = 0; OB_SUCC(ret) && read_piece < brs_.size_; ++read_piece) {
|
||||
int64_t read_cur_row = (curr_part_idx_ * brs_.size_ + read_piece) / MY_SPEC.max_part_count_;
|
||||
// check for_column whether is all null
|
||||
if (OB_SUCC(ret) && !MY_SPEC.unpivot_info_.is_include_null_) {
|
||||
int64_t null_count = 0;
|
||||
const int64_t base_idx = curr_cols_idx_ * MY_SPEC.unpivot_info_.get_new_column_count()
|
||||
+ MY_SPEC.unpivot_info_.old_column_count_
|
||||
+ MY_SPEC.unpivot_info_.for_column_count_;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.unpivot_info_.unpivot_column_count_; ++i) {
|
||||
const int64_t input_idx = i + base_idx;
|
||||
if (0 == curr_cols_idx_) {
|
||||
if (multiplex_[input_idx * brs_.size_ + read_cur_row].is_null()) {
|
||||
++null_count;
|
||||
}
|
||||
} else {
|
||||
ObExpr *child_expr = child_->get_spec().output_[input_idx];
|
||||
ObDatum &child_datum = child_expr->locate_expr_datum(eval_ctx_, read_cur_row);
|
||||
if (child_datum.is_null()) {
|
||||
++null_count;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && null_count == MY_SPEC.unpivot_info_.unpivot_column_count_) {
|
||||
brs_.skip_->set(read_piece);
|
||||
curr_cols_idx_ = (curr_cols_idx_ + 1) % MY_SPEC.max_part_count_;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
// old_column
|
||||
int64_t output_idx = 0;
|
||||
for (; OB_SUCC(ret) && output_idx < MY_SPEC.unpivot_info_.old_column_count_; ++output_idx) {
|
||||
ObExpr *father_expr = MY_SPEC.output_[output_idx];
|
||||
ObDatum *father_datum = father_expr->locate_batch_datums(eval_ctx_);
|
||||
father_datum[read_piece] = multiplex_[output_idx * brs_.size_ + read_cur_row];
|
||||
}
|
||||
// new_column
|
||||
if (0 == curr_cols_idx_) {
|
||||
for (; OB_SUCC(ret) && output_idx < MY_SPEC.get_output_count(); ++output_idx) {
|
||||
ObExpr *father_expr = MY_SPEC.output_[output_idx];
|
||||
ObDatum *father_datum = father_expr->locate_batch_datums(eval_ctx_);
|
||||
if (father_expr->is_batch_result()) {
|
||||
father_datum[read_piece] = multiplex_[output_idx * brs_.size_ + read_cur_row];
|
||||
} else {
|
||||
father_datum[0] = multiplex_[output_idx * brs_.size_ + read_cur_row];
|
||||
}
|
||||
}
|
||||
} else {
|
||||
const int64_t base_idx = curr_cols_idx_ * MY_SPEC.unpivot_info_.get_new_column_count();
|
||||
for (; OB_SUCC(ret) && output_idx < MY_SPEC.get_output_count(); ++output_idx) {
|
||||
const int64_t input_idx = output_idx + base_idx;
|
||||
ObExpr *child_expr = child_->get_spec().output_[input_idx];
|
||||
ObExpr *father_expr = MY_SPEC.output_[output_idx];
|
||||
ObDatum *child_datum = child_expr->locate_batch_datums(eval_ctx_);
|
||||
ObDatum *father_datum = father_expr->locate_batch_datums(eval_ctx_);
|
||||
if (OB_UNLIKELY(!father_expr->is_batch_result())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("output of unpivot should be batch result", K(ret), KPC(father_expr));
|
||||
} else if (child_expr->is_batch_result()) {
|
||||
father_datum[read_piece] = child_datum[read_cur_row];
|
||||
} else {
|
||||
father_datum[read_piece] = *child_datum;
|
||||
}
|
||||
}
|
||||
}
|
||||
curr_cols_idx_ = (curr_cols_idx_ + 1) % MY_SPEC.max_part_count_;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
++curr_part_idx_;
|
||||
if (curr_part_idx_ == MY_SPEC.max_part_count_) {
|
||||
for (int64_t output_idx = 0; output_idx < MY_SPEC.get_output_count(); ++output_idx) {
|
||||
ObExpr *father_expr = MY_SPEC.output_[output_idx];
|
||||
father_expr->set_evaluated_projected(eval_ctx_);
|
||||
}
|
||||
}
|
||||
} else if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
brs_.end_ = true;
|
||||
brs_.size_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObUnpivotOp::destroy()
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
void ObUnpivotOp::reset()
|
||||
{
|
||||
curr_part_idx_ = -1;
|
||||
curr_cols_idx_ = -1;
|
||||
child_brs_ = NULL;
|
||||
memset(multiplex_, 0, MY_SPEC.max_batch_size_ * MY_SPEC.get_output_count() * sizeof(ObDatum));
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
60
src/sql/engine/subquery/ob_unpivot_op.h
Normal file
60
src/sql/engine/subquery/ob_unpivot_op.h
Normal file
@ -0,0 +1,60 @@
|
||||
/**
|
||||
* 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_OP_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_UNPIVOT_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/resolver/dml/ob_dml_stmt.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
class ObUnpivotSpec : public ObOpSpec
|
||||
{
|
||||
OB_UNIS_VERSION(1);
|
||||
public:
|
||||
ObUnpivotSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(unpivot_info));
|
||||
|
||||
int64_t max_part_count_;
|
||||
ObUnpivotInfo unpivot_info_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObUnpivotSpec);
|
||||
};
|
||||
|
||||
class ObUnpivotOp : public ObOperator
|
||||
{
|
||||
public:
|
||||
ObUnpivotOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input);
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int inner_rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_get_next_batch(const int64_t max_row_cnt) override;
|
||||
virtual void destroy() override;
|
||||
|
||||
private:
|
||||
void reset();
|
||||
int64_t curr_part_idx_;
|
||||
int64_t curr_cols_idx_;
|
||||
const ObBatchRows *child_brs_;
|
||||
ObDatum *multiplex_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObUnpivotOp);
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_SUBQUERY_OB_UNPIVOT_OP_H_ */
|
||||
Reference in New Issue
Block a user