4104 lines
		
	
	
		
			137 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
			
		
		
	
	
			4104 lines
		
	
	
		
			137 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
/**
 | 
						|
 * 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_RESV
 | 
						|
 | 
						|
#include "sql/resolver/expr/ob_raw_expr.h"
 | 
						|
#include "sql/resolver/expr/ob_raw_expr_info_extractor.h"
 | 
						|
#include "sql/resolver/expr/ob_raw_expr_deduce_type.h"
 | 
						|
#include "sql/resolver/expr/ob_expr_relation_analyzer.h"
 | 
						|
#include "sql/rewrite/ob_transform_utils.h"
 | 
						|
#include "sql/optimizer/ob_raw_expr_pull_up_aggr_expr.h"
 | 
						|
#include "sql/optimizer/ob_logical_operator.h"
 | 
						|
#include "common/object/ob_object.h"
 | 
						|
#include "common/ob_smart_call.h"
 | 
						|
#include "share/config/ob_server_config.h"
 | 
						|
 | 
						|
using namespace oceanbase::sql;
 | 
						|
using namespace oceanbase::common;
 | 
						|
using namespace oceanbase::jit::expr;
 | 
						|
 | 
						|
namespace oceanbase {
 | 
						|
namespace sql {
 | 
						|
ObRawExpr* USELESS_POINTER = NULL;
 | 
						|
 | 
						|
void ObQualifiedName::format_qualified_name(ObNameCaseMode mode)
 | 
						|
{
 | 
						|
  UNUSED(mode);
 | 
						|
  bool maybe_column = !is_sys_func();
 | 
						|
  for (int64_t i = 0; maybe_column && i < access_idents_.count(); ++i) {
 | 
						|
    if (access_idents_.at(i).access_name_.empty() || access_idents_.at(i).access_index_ != OB_INVALID_INDEX) {
 | 
						|
      maybe_column = false;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (maybe_column && access_idents_.count() == 1) {
 | 
						|
    col_name_ = access_idents_.at(0).access_name_;
 | 
						|
  }
 | 
						|
  if (maybe_column && access_idents_.count() == 2) {
 | 
						|
    tbl_name_ = access_idents_.at(0).access_name_;
 | 
						|
    col_name_ = access_idents_.at(1).access_name_;
 | 
						|
  }
 | 
						|
  if (maybe_column && access_idents_.count() == 3) {
 | 
						|
    database_name_ = access_idents_.at(0).access_name_;
 | 
						|
    tbl_name_ = access_idents_.at(1).access_name_;
 | 
						|
    col_name_ = access_idents_.at(2).access_name_;
 | 
						|
  }
 | 
						|
}
 | 
						|
 | 
						|
int ObQualifiedName::replace_access_ident_params(ObRawExpr* from, ObRawExpr* to)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < access_idents_.count(); ++i) {
 | 
						|
    OZ(access_idents_.at(i).replace_params(from, to));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObObjAccessIdent::extract_params(int64_t level, common::ObIArray<ObRawExpr*>& params) const
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < params_.count(); ++i) {
 | 
						|
    if (params_.at(i).second == level && OB_FAIL(params.push_back(params_.at(i).first))) {
 | 
						|
      LOG_WARN("push back error", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObObjAccessIdent::replace_params(ObRawExpr* from, ObRawExpr* to)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < params_.count(); ++i) {
 | 
						|
    OZ(ObRawExprUtils::replace_ref_column(params_.at(i).first, from, to));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int OrderItem::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const OrderItem& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  order_type_ = other.order_type_;
 | 
						|
  if (OB_FAIL(ObRawExprUtils::copy_expr(expr_factory, other.expr_, expr_, copy_types, use_new_allocator))) {
 | 
						|
    LOG_WARN("failed to copy expr when copying OrderItem", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
OB_SERIALIZE_MEMBER(ObVarInfo, type_, name_);
 | 
						|
 | 
						|
int ObVarInfo::deep_copy(common::ObIAllocator& allocator, ObVarInfo& var_info) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObString clone_name;
 | 
						|
  if (OB_FAIL(ob_write_string(allocator, name_, clone_name))) {
 | 
						|
    LOG_WARN("fail to write string", K(name_), K(*this), K(ret));
 | 
						|
  } else {
 | 
						|
    var_info.type_ = type_;
 | 
						|
    var_info.name_ = clone_name;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr::~ObRawExpr()
 | 
						|
{
 | 
						|
  if (0x86427531 == magic_num_) {
 | 
						|
    LOG_ERROR("ObRawExpr maybe double free!");
 | 
						|
  }
 | 
						|
  magic_num_ = 0x86427531;
 | 
						|
}
 | 
						|
 | 
						|
// This function is used to determine whether the expression itself or the parameter contains a non-constant expression
 | 
						|
// The definition of a constant expression is an expression that requires a specific operator in the plan
 | 
						|
// We agree that ObBinaryRefExpr must be generated by TableScan
 | 
						|
// ObAggFunRawExpr must be generated by Groupby operator
 | 
						|
// ObSetOpRawExpr must be generated by Set operator
 | 
						|
// ObUnaryRefExpr (i.e. subquery ref) must be generated by SubPlanFilter
 | 
						|
// PSEUDO_COLUMN is generally generated by CTE or hierarchical query
 | 
						|
// SYS_CONNECT_BY_PATH,CONNECT_BY_ROOT is generated by cby nestloop join
 | 
						|
bool ObRawExpr::has_generalized_column() const
 | 
						|
{
 | 
						|
  return has_flag(CNT_COLUMN) || has_flag(CNT_AGG) || has_flag(CNT_SET_OP) || has_flag(CNT_SUB_QUERY) ||
 | 
						|
         has_flag(CNT_WINDOW_FUNC) || has_flag(CNT_ROWNUM) || has_flag(CNT_PSEUDO_COLUMN) || has_flag(CNT_SEQ_EXPR) ||
 | 
						|
         has_flag(CNT_SYS_CONNECT_BY_PATH) || has_flag(CNT_CONNECT_BY_ROOT);
 | 
						|
}
 | 
						|
 | 
						|
bool ObRawExpr::has_enum_set_column() const
 | 
						|
{
 | 
						|
  return has_flag(CNT_ENUM_OR_SET) || has_flag(CNT_SUB_QUERY);
 | 
						|
}
 | 
						|
 | 
						|
bool ObRawExpr::has_specified_pseudocolumn() const
 | 
						|
{
 | 
						|
  return has_flag(CNT_ROWNUM) || has_flag(CNT_PSEUDO_COLUMN);
 | 
						|
}
 | 
						|
 | 
						|
bool ObRawExpr::is_generalized_column() const
 | 
						|
{
 | 
						|
  return is_aggr_expr() || is_set_op_expr() || is_column_ref_expr() || is_query_ref_expr() || is_win_func_expr() ||
 | 
						|
         has_flag(IS_ROWNUM) || has_flag(IS_PSEUDO_COLUMN);
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::assign(const ObRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(&other != this)) {
 | 
						|
    type_ = other.type_;
 | 
						|
    expr_class_ = other.expr_class_;
 | 
						|
    result_type_ = other.result_type_;
 | 
						|
    info_ = other.info_;
 | 
						|
    rel_ids_ = other.rel_ids_;
 | 
						|
    expr_levels_ = other.expr_levels_;
 | 
						|
    alias_column_name_ = other.alias_column_name_;
 | 
						|
    expr_name_ = other.expr_name_;
 | 
						|
    expr_level_ = other.expr_level_;
 | 
						|
    ref_count_ = other.ref_count_;
 | 
						|
    is_explicited_reference_ = other.is_explicited_reference_;
 | 
						|
    is_for_generated_column_ = other.is_for_generated_column_;
 | 
						|
    extra_ = other.extra_;
 | 
						|
    is_calculated_ = other.is_calculated_;
 | 
						|
    is_deterministic_ = other.is_deterministic_;
 | 
						|
    if (OB_FAIL(enum_set_values_.assign(other.enum_set_values_))) {
 | 
						|
      LOG_WARN("failed to assign enum set values", K(ret));
 | 
						|
    } else { /*do nothing*/
 | 
						|
    }
 | 
						|
    orig_expr_ = other.orig_expr_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(expr_factory);
 | 
						|
  UNUSED(copy_types);
 | 
						|
  if (OB_LIKELY(&other != this)) {
 | 
						|
    if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.alias_column_name_, alias_column_name_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.alias_column_name_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.expr_name_, expr_name_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.expr_name_), K(ret));
 | 
						|
      } else {
 | 
						|
        result_type_ = other.result_type_;
 | 
						|
        ObObj param;
 | 
						|
        if (OB_FAIL(deep_copy_obj(*inner_alloc_, other.result_type_.get_param(), param))) {
 | 
						|
          LOG_WARN("deep copy error", K(param), K(ret));
 | 
						|
        } else {
 | 
						|
          result_type_.set_param(param);
 | 
						|
        }
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < enum_set_values_.count(); i++) {
 | 
						|
          ObString tmp_str;
 | 
						|
          if (OB_FAIL(ob_write_string(*inner_alloc_, other.enum_set_values_.at(i), tmp_str))) {
 | 
						|
            LOG_WARN("fail to write string", K(ret), K(other.enum_set_values_), K(i));
 | 
						|
          } else if (OB_FAIL(enum_set_values_.push_back(tmp_str))) {
 | 
						|
            LOG_WARN("fail to push back str", K(tmp_str), K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      alias_column_name_ = other.alias_column_name_;
 | 
						|
      expr_name_ = other.expr_name_;
 | 
						|
      result_type_ = other.result_type_;
 | 
						|
      if (OB_FAIL(enum_set_values_.assign(other.enum_set_values_))) {
 | 
						|
        LOG_WARN("failed to assign enum set values", K(ret));
 | 
						|
      } else { /*do nothing*/
 | 
						|
      }
 | 
						|
    }
 | 
						|
    type_ = other.type_;
 | 
						|
    expr_class_ = other.expr_class_;
 | 
						|
    info_ = other.info_;
 | 
						|
    rel_ids_ = other.rel_ids_;
 | 
						|
    expr_levels_ = other.expr_levels_;
 | 
						|
    expr_level_ = other.expr_level_;
 | 
						|
    ref_count_ = other.ref_count_;
 | 
						|
    is_explicited_reference_ = other.is_explicited_reference_;
 | 
						|
    is_for_generated_column_ = other.is_for_generated_column_;
 | 
						|
    extra_ = other.extra_;
 | 
						|
    is_calculated_ = other.is_calculated_;
 | 
						|
    is_deterministic_ = other.is_deterministic_;
 | 
						|
    if (OB_FAIL(enum_set_values_.assign(other.enum_set_values_))) {
 | 
						|
      LOG_WARN("failed to assign enum set values", K(ret));
 | 
						|
    } else if (OB_NOT_NULL(other.orig_expr_) &&
 | 
						|
               OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.orig_expr_, orig_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy orig expr", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(other_exprs);
 | 
						|
  UNUSED(new_exprs);
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObRawExpr::reset()
 | 
						|
{
 | 
						|
  type_ = T_INVALID;
 | 
						|
  info_.reset();
 | 
						|
  rel_ids_.reset();
 | 
						|
  set_data_type(ObMaxType);
 | 
						|
  expr_level_ = -1;
 | 
						|
  ref_count_ = 0;
 | 
						|
  is_explicited_reference_ = false;
 | 
						|
  is_for_generated_column_ = false;
 | 
						|
  is_calculated_ = false;
 | 
						|
  is_deterministic_ = true;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::get_name(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_NOT_NULL(orig_expr_) && EXPLAIN_DBLINK_STMT == type) {
 | 
						|
    if (OB_FAIL(orig_expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
      LOG_WARN("fail to get name for orig expr", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(get_name_internal(buf, buf_len, pos, type))) {
 | 
						|
      LOG_WARN("fail to get_name", K(buf_len), K(pos), K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::get_type_and_length(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (EXPLAIN_EXTENDED == type || EXPLAIN_EXTENDED_NOADDR == type) {
 | 
						|
    const char* type_str = common::ob_obj_type_str(get_data_type());
 | 
						|
    if (nullptr == type_str || strlen(type_str) > INT32_MAX) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("some error happend");
 | 
						|
    } else {
 | 
						|
      int str_length = (int)strlen(type_str);
 | 
						|
      int32_t out_data = get_accuracy().get_precision();
 | 
						|
      bool output_length = ob_is_accuracy_length_valid_tc(get_data_type());
 | 
						|
      if (output_length) {
 | 
						|
        out_data = get_accuracy().get_length();
 | 
						|
      }
 | 
						|
      if (OB_FAIL(BUF_PRINTF(", %.*s, %d", str_length, type_str, out_data))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    // do nothing
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::extract_info()
 | 
						|
{
 | 
						|
  // LOG_DEBUG("extract_info", "usec", ObSQLUtils::get_usec());
 | 
						|
  // scope may be NULL, do not check
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObRawExprInfoExtractor extractor;
 | 
						|
  if (OB_FAIL(extractor.analyze(*this))) {
 | 
						|
    LOG_WARN("fail to analyze", K(ret));
 | 
						|
  }
 | 
						|
  // LOG_DEBUG("extractor_info", "usec", ObSQLUtils::get_usec());
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::deduce_type(const ObSQLSessionInfo* session_info)
 | 
						|
{
 | 
						|
  // LOG_DEBUG("deduce_type", "usec", ObSQLUtils::get_usec());
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObRawExprDeduceType expr_deducer(session_info);
 | 
						|
  expr_deducer.set_expr_factory(expr_factory_);
 | 
						|
  if (OB_FAIL(expr_deducer.deduce(*this))) {
 | 
						|
    LOG_WARN("fail to deduce", K(ret));
 | 
						|
  }
 | 
						|
  // LOG_DEBUG("deduce_type", "usec", ObSQLUtils::get_usec());
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::formalize(const ObSQLSessionInfo* session_info)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(extract_info())) {
 | 
						|
    LOG_WARN("failed to extract info", K(*this));
 | 
						|
  } else if (OB_FAIL(deduce_type(session_info))) {
 | 
						|
    LOG_WARN("failed to deduce type", K(*this));
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::pull_relation_id_and_levels(int32_t cur_stmt_level)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObExprRelationAnalyzer expr_relation_analyzer;
 | 
						|
  if (OB_FAIL(expr_relation_analyzer.pull_expr_relation_id_and_levels(this, cur_stmt_level))) {
 | 
						|
    LOG_WARN("pull expr failed", K(cur_stmt_level), K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::add_child_flags(const ObExprInfo& flags)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObExprInfo tmp = flags;
 | 
						|
  if (INHERIT_MASK_BEGIN < tmp.bit_count()) {
 | 
						|
    int64_t mask_end =
 | 
						|
        INHERIT_MASK_END < tmp.bit_count() ? static_cast<int64_t>(INHERIT_MASK_END) : tmp.bit_count() - 1;
 | 
						|
    if (tmp.do_mask(INHERIT_MASK_BEGIN, mask_end)) {
 | 
						|
      LOG_WARN("failed to do mask", K(ret));
 | 
						|
    } else if (OB_FAIL(info_.add_members(tmp))) {
 | 
						|
      LOG_WARN("failed to add expr info", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int32_t i = 0; OB_SUCC(ret) && i <= IS_INFO_MASK_END; ++i) {
 | 
						|
    if (flags.has_member(i)) {
 | 
						|
      if (OB_FAIL(info_.add_member(i + CNT_INFO_MASK_BEGIN))) {
 | 
						|
        LOG_WARN("failed to add member", K(i), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::set_expr_name(const common::ObString& expr_name)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(inner_alloc_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, expr_name, expr_name_))) {
 | 
						|
    LOG_WARN("write string failed", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::preorder_accept(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(do_visit(visitor))) {
 | 
						|
    LOG_WARN("visit failed", K(ret), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
  } else {
 | 
						|
    if (!skip_visit_child() && !visitor.skip_child(*this)) {
 | 
						|
      const int64_t cnt = get_param_count();
 | 
						|
      for (int64_t i = 0; i < cnt && OB_SUCC(ret); i++) {
 | 
						|
        ObRawExpr* e = get_param_expr(i);
 | 
						|
        if (NULL == e) {
 | 
						|
          LOG_WARN("null param expr returned", K(ret), K(i), K(cnt));
 | 
						|
        } else if (OB_FAIL(SMART_CALL(e->preorder_accept(visitor)))) {
 | 
						|
          LOG_WARN("child visit failed", K(ret), K(i), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::postorder_accept(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (!skip_visit_child() && !visitor.skip_child(*this)) {
 | 
						|
    const int64_t cnt = get_param_count();
 | 
						|
    for (int64_t i = 0; i < cnt && OB_SUCC(ret); i++) {
 | 
						|
      ObRawExpr* e = get_param_expr(i);
 | 
						|
      if (NULL == e) {
 | 
						|
        LOG_WARN("null param expr returned", K(ret), K(i), K(cnt));
 | 
						|
      } else if (OB_FAIL(SMART_CALL(e->postorder_accept(visitor)))) {
 | 
						|
        LOG_WARN("child visit failed", K(ret), K(i), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(do_visit(visitor))) {
 | 
						|
      LOG_WARN("visit failed", K(ret), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::postorder_replace(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObRawExprPullUpAggrExpr& pull_up_visitor = *static_cast<ObRawExprPullUpAggrExpr*>(&visitor);
 | 
						|
  if (!skip_visit_child() && !visitor.skip_child(*this)) {
 | 
						|
    const int64_t cnt = get_param_count();
 | 
						|
    for (int64_t i = 0; i < cnt && OB_SUCC(ret); i++) {
 | 
						|
      ObRawExpr* e = get_param_expr(i);
 | 
						|
      if (NULL == e) {
 | 
						|
        LOG_WARN("null param expr returned", K(ret), K(i), K(cnt));
 | 
						|
      } else if (OB_FAIL(SMART_CALL(e->postorder_replace(visitor)))) {
 | 
						|
        LOG_WARN("child visit failed", K(ret), K(i), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
      } else {
 | 
						|
        e = pull_up_visitor.get_new_expr();
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(do_visit(visitor))) {
 | 
						|
      LOG_WARN("visit failed", K(ret), "type", get_expr_type(), "name", get_type_name(get_expr_type()));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
ObRawExpr*& ObTerminalRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  UNUSED(index);
 | 
						|
  return USELESS_POINTER;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExpr::set_enum_set_values(const common::ObIArray<common::ObString>& values)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(enum_set_values_.assign(values))) {
 | 
						|
    LOG_WARN("failed to assign array", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObConstRawExpr::assign(const ObConstRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      value_ = other.get_value();
 | 
						|
      literal_prefix_ = other.get_literal_prefix();
 | 
						|
      obj_meta_ = other.get_expr_obj_meta();
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObConstRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObConstRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator is NULL", K(ret));
 | 
						|
      } else if (OB_FAIL(deep_copy_obj(*inner_alloc_, other.value_, value_))) {
 | 
						|
        LOG_WARN("deep copy error", K(value_), K(ret));
 | 
						|
      } else {
 | 
						|
        if (other.literal_prefix_.empty()) {
 | 
						|
          literal_prefix_.reset();
 | 
						|
        } else {
 | 
						|
          char* buf = NULL;
 | 
						|
          if (OB_ISNULL(buf = static_cast<char*>(inner_alloc_->alloc(other.literal_prefix_.length())))) {
 | 
						|
            ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
						|
            LOG_ERROR("Fail to allocate memory, ", "size", other.literal_prefix_.length(), K(ret));
 | 
						|
          } else {
 | 
						|
            literal_prefix_.assign_buffer(buf, other.literal_prefix_.length());
 | 
						|
            literal_prefix_.write(other.literal_prefix_.ptr(), other.literal_prefix_.length());
 | 
						|
          }
 | 
						|
        }
 | 
						|
        obj_meta_ = other.get_expr_obj_meta();
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      value_ = other.get_value();
 | 
						|
      literal_prefix_ = other.get_literal_prefix();
 | 
						|
      obj_meta_ = other.get_expr_obj_meta();
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObConstRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObConstRawExpr::set_value(const ObObj& val)
 | 
						|
{
 | 
						|
  set_meta_type(val.get_meta());
 | 
						|
  set_expr_obj_meta(val.get_meta());
 | 
						|
  value_ = val;
 | 
						|
}
 | 
						|
 | 
						|
void ObConstRawExpr::set_literal_prefix(const ObString& name)
 | 
						|
{
 | 
						|
  literal_prefix_ = name;
 | 
						|
}
 | 
						|
 | 
						|
void ObConstRawExpr::reset()
 | 
						|
{
 | 
						|
  ObRawExpr::reset();
 | 
						|
  value_.reset();
 | 
						|
  literal_prefix_.reset();
 | 
						|
}
 | 
						|
 | 
						|
void ObConstRawExpr::set_is_date_unit()
 | 
						|
{
 | 
						|
  is_date_unit_ = true;
 | 
						|
}
 | 
						|
 | 
						|
void ObConstRawExpr::reset_is_date_unit()
 | 
						|
{
 | 
						|
  is_date_unit_ = false;
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObConstRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  return value_.hash(seed);
 | 
						|
}
 | 
						|
 | 
						|
bool ObConstRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (check_context != NULL && check_context->override_const_compare_) {
 | 
						|
    if (expr.is_const_expr()) {
 | 
						|
      bool_ret = check_context->compare_const(*this, static_cast<const ObConstRawExpr&>(expr));
 | 
						|
    }
 | 
						|
  } else if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
    // what are you doing ?
 | 
						|
    if (NULL != check_context) {
 | 
						|
      if (expr.is_const_expr()) {
 | 
						|
        if (T_QUESTIONMARK == expr.get_expr_type()) {
 | 
						|
          bool_ret = true;
 | 
						|
          const ObConstRawExpr* c_expr = static_cast<const ObConstRawExpr*>(&expr);
 | 
						|
          int64_t param_idx = -1;
 | 
						|
          int& ret = check_context->err_code_;
 | 
						|
          if (OB_FAIL(c_expr->get_value().get_unknown(param_idx))) {
 | 
						|
            LOG_WARN("Failed to get param", K(ret));
 | 
						|
          } else if (OB_FAIL(check_context->param_expr_.push_back(
 | 
						|
                         ObExprEqualCheckContext::ParamExprPair(param_idx, this)))) {
 | 
						|
            LOG_WARN("Failed to add param expr pair", K(ret));
 | 
						|
          } else {
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    const ObConstRawExpr* c_expr = static_cast<const ObConstRawExpr*>(&expr);
 | 
						|
    if (get_value().get_meta() != c_expr->get_value().get_meta()) {
 | 
						|
      bool_ret = false;
 | 
						|
    } else if (get_value().is_equal(c_expr->get_value(), CS_TYPE_BINARY)) {
 | 
						|
      bool_ret = true;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObExprEqualCheckContext::compare_const(const ObConstRawExpr& left, const ObConstRawExpr& right)
 | 
						|
{
 | 
						|
  int& ret = err_code_;
 | 
						|
  bool result = false;
 | 
						|
  if (left.get_result_type() == right.get_result_type()) {
 | 
						|
    const ObObj& this_value = left.get_value().is_unknown() ? left.get_result_type().get_param() : left.get_value();
 | 
						|
    const ObObj& other_value = right.get_value().is_unknown() ? right.get_result_type().get_param() : right.get_value();
 | 
						|
    result = this_value.is_equal(other_value, CS_TYPE_BINARY);
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && result && left.get_value().is_unknown()) {
 | 
						|
    if (OB_FAIL(add_param_pair(left.get_value().get_unknown(), NULL))) {
 | 
						|
      LOG_WARN("add param pair failed", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && result && right.get_value().is_unknown()) {
 | 
						|
    if (OB_FAIL(add_param_pair(right.get_value().get_unknown(), NULL))) {
 | 
						|
      LOG_WARN("add param pair failed", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return result;
 | 
						|
}
 | 
						|
 | 
						|
int ObConstRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObConstRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(type);
 | 
						|
  if (get_value().is_unknown()) {
 | 
						|
    if (EXPLAIN_DBLINK_STMT != type) {
 | 
						|
      if (OB_FAIL(get_value().print_plain_str_literal(buf, buf_len, pos))) {
 | 
						|
        LOG_WARN("fail to print_sql_literal", K(get_value()), K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(ObLinkStmtParam::write(buf, buf_len, pos, get_value().get_unknown()))) {
 | 
						|
        LOG_WARN("fail to write param to buf", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(get_value().print_sql_literal(buf, buf_len, pos))) {
 | 
						|
      LOG_WARN("fail to print_sql_literal", K(get_value()), K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int64_t ObConstRawExpr::to_string(char* buf, const int64_t buf_len) const
 | 
						|
{
 | 
						|
  int64_t pos = 0;
 | 
						|
  J_OBJ_START();
 | 
						|
  J_KV(N_ITEM_TYPE, type_, N_RESULT_TYPE, result_type_, N_EXPR_INFO, info_, N_REL_ID, rel_ids_, N_VALUE, value_);
 | 
						|
  if (!literal_prefix_.empty()) {
 | 
						|
    J_COMMA();
 | 
						|
    J_KV(K_(literal_prefix));
 | 
						|
  }
 | 
						|
  J_OBJ_END();
 | 
						|
  return pos;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObVarRawExpr::assign(const ObVarRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign terminal raw expr", K(ret));
 | 
						|
    } else {
 | 
						|
      result_type_assigned_ = other.result_type_assigned_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObVarRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObVarRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      result_type_assigned_ = other.result_type_assigned_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObVarRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObVarRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  UNUSED(check_context);
 | 
						|
  return expr.is_var_expr() && get_expr_type() == expr.get_expr_type() && get_result_type() == expr.get_result_type();
 | 
						|
}
 | 
						|
 | 
						|
int ObVarRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(type);
 | 
						|
  if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", get_type_name(type_)))) {
 | 
						|
    LOG_WARN("databuff print failed", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObVarRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObUserVarIdentRawExpr::assign(const ObUserVarIdentRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObConstRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign const raw expr", K(ret));
 | 
						|
    } else {
 | 
						|
      is_contain_assign_ = other.is_contain_assign_;
 | 
						|
      query_has_udf_ = other.query_has_udf_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObUserVarIdentRawExpr::deep_copy(ObRawExprFactory& expr_factory, const ObUserVarIdentRawExpr& other,
 | 
						|
    const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObConstRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      is_contain_assign_ = other.is_contain_assign_;
 | 
						|
      query_has_udf_ = other.query_has_udf_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObUserVarIdentRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObUserVarIdentRawExpr::reset()
 | 
						|
{
 | 
						|
  ObConstRawExpr::reset();
 | 
						|
  is_contain_assign_ = false;
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObUserVarIdentRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  seed = value_.hash(seed);
 | 
						|
  seed = common::do_hash(is_contain_assign_, seed);
 | 
						|
  return seed;
 | 
						|
}
 | 
						|
 | 
						|
bool ObUserVarIdentRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  UNUSED(check_context);
 | 
						|
  if (get_expr_type() == expr.get_expr_type()) {
 | 
						|
    const ObUserVarIdentRawExpr* user_var_expr = static_cast<const ObUserVarIdentRawExpr*>(&expr);
 | 
						|
    if (get_value().get_meta() != user_var_expr->get_value().get_meta()) {
 | 
						|
      bool_ret = false;
 | 
						|
    } else if (get_value().is_equal(user_var_expr->get_value(), CS_TYPE_BINARY)) {
 | 
						|
      bool_ret = (get_is_contain_assign() == user_var_expr->get_is_contain_assign());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObUserVarIdentRawExpr::is_same_variable(const ObObj& obj) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_value().get_meta() == obj.get_meta() && get_value().is_equal(obj, CS_TYPE_BINARY)) {
 | 
						|
    bool_ret = true;
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObUserVarIdentRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int64_t ObUserVarIdentRawExpr::to_string(char* buf, const int64_t buf_len) const
 | 
						|
{
 | 
						|
  int64_t pos = 0;
 | 
						|
  J_OBJ_START();
 | 
						|
  J_KV(N_ITEM_TYPE,
 | 
						|
      type_,
 | 
						|
      N_RESULT_TYPE,
 | 
						|
      result_type_,
 | 
						|
      N_EXPR_INFO,
 | 
						|
      info_,
 | 
						|
      N_REL_ID,
 | 
						|
      rel_ids_,
 | 
						|
      N_VALUE,
 | 
						|
      value_,
 | 
						|
      "is_contain_assign",
 | 
						|
      is_contain_assign_,
 | 
						|
      "query_has_udf",
 | 
						|
      query_has_udf_);
 | 
						|
  J_OBJ_END();
 | 
						|
  return pos;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObQueryRefRawExpr::assign(const ObQueryRefRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(column_types_.assign(other.column_types_))) {
 | 
						|
      LOG_WARN("failed to copy column types", K(ret));
 | 
						|
    } else {
 | 
						|
      ref_id_ = other.ref_id_;
 | 
						|
      if (OB_STMT == other.ref_type_) {
 | 
						|
        set_ref_stmt(other.ref_stmt_);
 | 
						|
      } else if (OB_LOGICAL_OPERATOR == other.ref_type_) {
 | 
						|
        set_ref_operator(other.ref_operator_);
 | 
						|
      } else { /* Do nothing */
 | 
						|
      }
 | 
						|
      output_column_ = other.output_column_;
 | 
						|
      is_set_ = other.is_set_;
 | 
						|
      column_types_ = other.column_types_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObQueryRefRawExpr::deep_copy(
 | 
						|
    ObStmtFactory& stmt_factory, ObRawExprFactory& expr_factory, const ObQueryRefRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSelectStmt* temp_ref_stmt = NULL;
 | 
						|
  if (OB_ISNULL(other.get_ref_stmt())) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("null stmt", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
    LOG_WARN("failed to copy row expr", K(ret));
 | 
						|
  } else if (OB_FAIL(stmt_factory.create_stmt(temp_ref_stmt)) || OB_ISNULL(temp_ref_stmt)) {
 | 
						|
    LOG_WARN("failed to create statement", K(ret));
 | 
						|
  } else if (OB_FAIL(temp_ref_stmt->deep_copy(stmt_factory, expr_factory, *other.get_ref_stmt()))) {
 | 
						|
    LOG_WARN("failed to deep copy stmt", K(ret));
 | 
						|
  } else if (OB_FAIL(column_types_.assign(other.column_types_))) {
 | 
						|
    LOG_WARN("failed to assign copy types", K(ret));
 | 
						|
  } else {
 | 
						|
    ref_id_ = other.ref_id_;
 | 
						|
    output_column_ = other.output_column_;
 | 
						|
    is_set_ = other.is_set_;
 | 
						|
    set_ref_stmt(temp_ref_stmt);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObQueryRefRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObSelectStmt* ObQueryRefRawExpr::get_ref_stmt()
 | 
						|
{
 | 
						|
  ObSelectStmt* stmt = NULL;
 | 
						|
  if (is_ref_stmt()) {
 | 
						|
    stmt = ref_stmt_;
 | 
						|
  } else if (is_ref_operator() && NULL != get_ref_operator() && NULL != get_ref_operator()->get_stmt() &&
 | 
						|
             get_ref_operator()->get_stmt()->is_select_stmt()) {
 | 
						|
    stmt = static_cast<ObSelectStmt*>(get_ref_operator()->get_stmt());
 | 
						|
  }
 | 
						|
  return stmt;
 | 
						|
}
 | 
						|
 | 
						|
const ObSelectStmt* ObQueryRefRawExpr::get_ref_stmt() const
 | 
						|
{
 | 
						|
  const ObSelectStmt* stmt = NULL;
 | 
						|
  if (is_ref_stmt()) {
 | 
						|
    stmt = ref_stmt_;
 | 
						|
  } else if (is_ref_operator() && NULL != get_ref_operator() && NULL != get_ref_operator()->get_stmt() &&
 | 
						|
             get_ref_operator()->get_stmt()->is_select_stmt()) {
 | 
						|
    stmt = static_cast<const ObSelectStmt*>(get_ref_operator()->get_stmt());
 | 
						|
  }
 | 
						|
  return stmt;
 | 
						|
}
 | 
						|
 | 
						|
void ObQueryRefRawExpr::reset()
 | 
						|
{
 | 
						|
  ObTerminalRawExpr::reset();
 | 
						|
  ref_id_ = OB_INVALID_ID;
 | 
						|
}
 | 
						|
 | 
						|
bool ObQueryRefRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_expr_type() == expr.get_expr_type()) {
 | 
						|
    const ObQueryRefRawExpr& u_expr = static_cast<const ObQueryRefRawExpr&>(expr);
 | 
						|
    if (check_context != NULL && check_context->override_query_compare_) {
 | 
						|
      bool_ret = check_context->compare_query(*this, u_expr);
 | 
						|
    } else {
 | 
						|
      // very tricky, check the definition of ref_stmt_ and get_ref_stmt()
 | 
						|
      bool_ret = (get_ref_id() == u_expr.get_ref_id() && ref_stmt_ == u_expr.ref_stmt_);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObExprEqualCheckContext::compare_query(const ObQueryRefRawExpr& left, const ObQueryRefRawExpr& right)
 | 
						|
{
 | 
						|
  return left.get_ref_id() == right.get_ref_id() && left.get_ref_stmt() == right.get_ref_stmt();
 | 
						|
}
 | 
						|
 | 
						|
int ObQueryRefRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObQueryRefRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(BUF_PRINTF("subquery(%lu)", ref_id_))) {
 | 
						|
    LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
  } else if (EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
void ObColumnRefRawExpr::set_column_attr(const ObString& table_name, const ObString& column_name)
 | 
						|
{
 | 
						|
  /* NOTE: we just point to the buffer without owning it. */
 | 
						|
  table_name_.assign_ptr(table_name.ptr(), table_name.length());
 | 
						|
  column_name_.assign_ptr(column_name.ptr(), column_name.length());
 | 
						|
}
 | 
						|
 | 
						|
int ObColumnRefRawExpr::assign(const ObColumnRefRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      table_id_ = other.table_id_;
 | 
						|
      column_id_ = other.column_id_;
 | 
						|
      database_name_ = other.database_name_;
 | 
						|
      table_name_ = other.table_name_;
 | 
						|
      column_name_ = other.column_name_;
 | 
						|
      expr_level_ = other.expr_level_;
 | 
						|
      column_flags_ = other.column_flags_;
 | 
						|
      dependant_expr_ = other.dependant_expr_;
 | 
						|
      real_expr_ = other.real_expr_;
 | 
						|
      is_unpivot_mocked_column_ = other.is_unpivot_mocked_column_;
 | 
						|
      is_hidden_ = other.is_hidden_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObColumnRefRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  seed = common::do_hash(table_id_, seed);
 | 
						|
  seed = common::do_hash(column_id_, seed);
 | 
						|
 | 
						|
  return seed;
 | 
						|
}
 | 
						|
 | 
						|
int ObColumnRefRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObColumnRefRawExpr& other, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::deep_copy(expr_factory, other, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      table_id_ = other.table_id_;
 | 
						|
      column_id_ = other.column_id_;
 | 
						|
      expr_level_ = other.expr_level_;
 | 
						|
      column_flags_ = other.column_flags_;
 | 
						|
      is_unpivot_mocked_column_ = other.is_unpivot_mocked_column_;
 | 
						|
      is_hidden_ = other.is_hidden_;
 | 
						|
      is_lob_column_ = other.is_lob_column_;
 | 
						|
      if (use_new_allocator) {
 | 
						|
        if (OB_ISNULL(inner_alloc_)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
        } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.database_name_, database_name_))) {
 | 
						|
          LOG_WARN("fail to write string", K(other.database_name_), K(ret));
 | 
						|
        } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.table_name_, table_name_))) {
 | 
						|
          LOG_WARN("fail to write string", K(other.table_name_), K(ret));
 | 
						|
        } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.synonym_name_, synonym_name_))) {
 | 
						|
          LOG_WARN("fail to write string", K(other.synonym_name_), K(ret));
 | 
						|
        } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.synonym_db_name_, synonym_db_name_))) {
 | 
						|
          LOG_WARN("fail to write string", K(other.synonym_name_), K(ret));
 | 
						|
        } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.column_name_, column_name_))) {
 | 
						|
          LOG_WARN("fail to write string", K(other.column_name_), K(ret));
 | 
						|
        } else { /*do nothing*/
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        database_name_ = other.database_name_;
 | 
						|
        table_name_ = other.table_name_;
 | 
						|
        synonym_name_ = other.synonym_name_;
 | 
						|
        synonym_db_name_ = other.synonym_db_name_;
 | 
						|
        column_name_ = other.column_name_;
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret) && other.dependant_expr_ != NULL) {
 | 
						|
        ObRawExpr* temp_expr = NULL;
 | 
						|
        if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                expr_factory, other.get_dependant_expr(), temp_expr, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
          LOG_WARN("failed to deep copy expr", K(ret));
 | 
						|
        } else {
 | 
						|
          dependant_expr_ = temp_expr;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret) && other.real_expr_ != NULL) {
 | 
						|
        ObRawExpr* temp_expr = NULL;
 | 
						|
        if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                expr_factory, other.real_expr_, temp_expr, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
          LOG_WARN("failed to deep copy expr", K(ret));
 | 
						|
        } else {
 | 
						|
          real_expr_ = temp_expr;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObColumnRefRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (NULL != dependant_expr_ &&
 | 
						|
             OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, dependant_expr_))) {
 | 
						|
    LOG_WARN("failed to replace dependant exprs", K(ret));
 | 
						|
  } else if (NULL != real_expr_ && OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, real_expr_))) {
 | 
						|
    LOG_WARN("failed to replace dependant exprs", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObColumnRefRawExpr::reset()
 | 
						|
{
 | 
						|
  ObTerminalRawExpr::reset();
 | 
						|
  table_id_ = OB_INVALID_ID;
 | 
						|
  column_id_ = OB_INVALID_ID;
 | 
						|
  database_name_.reset();
 | 
						|
  table_name_.reset();
 | 
						|
  column_name_.reset();
 | 
						|
}
 | 
						|
 | 
						|
bool ObColumnRefRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  UNUSED(check_context);
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
  } else if (OB_UNLIKELY(check_context != NULL) && OB_UNLIKELY(check_context->override_column_compare_)) {
 | 
						|
    // use new column compare method
 | 
						|
    bool_ret = check_context->compare_column(*this, static_cast<const ObColumnRefRawExpr&>(expr));
 | 
						|
  } else {
 | 
						|
    const ObColumnRefRawExpr* b_expr = static_cast<const ObColumnRefRawExpr*>(&expr);
 | 
						|
    if (this->get_table_id() == b_expr->get_table_id() && this->get_column_id() == b_expr->get_column_id()) {
 | 
						|
      bool_ret = true;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObExprEqualCheckContext::compare_column(const ObColumnRefRawExpr& left, const ObColumnRefRawExpr& right)
 | 
						|
{
 | 
						|
  return left.get_result_type() == right.get_result_type();
 | 
						|
}
 | 
						|
 | 
						|
int ObColumnRefRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObColumnRefRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (EXPLAIN_DBLINK_STMT == type) {
 | 
						|
    if (!table_name_.empty() && OB_FAIL(BUF_PRINTF("%.*s.", table_name_.length(), table_name_.ptr()))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%.*s", column_name_.length(), column_name_.ptr()))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(BUF_PRINTF(
 | 
						|
            "%.*s.%.*s", table_name_.length(), table_name_.ptr(), column_name_.length(), column_name_.ptr()))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
      if (EXPLAIN_EXTENDED == type) {
 | 
						|
        if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObAliasRefRawExpr::get_ref_expr() const
 | 
						|
{
 | 
						|
  const ObRawExpr* ret = NULL;
 | 
						|
  if (project_index_ == OB_INVALID_INDEX) {
 | 
						|
    ret = ref_expr_;
 | 
						|
  } else if (ref_expr_->is_query_ref_expr()) {
 | 
						|
    ObQueryRefRawExpr* subquery = static_cast<ObQueryRefRawExpr*>(ref_expr_);
 | 
						|
    ObSelectStmt* stmt = NULL;
 | 
						|
    if (OB_ISNULL(stmt = subquery->get_ref_stmt()) ||
 | 
						|
        OB_UNLIKELY(project_index_ < 0 || project_index_ >= stmt->get_select_item_size())) {
 | 
						|
      // do nothing
 | 
						|
    } else {
 | 
						|
      ret = stmt->get_select_item(project_index_).expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr* ObAliasRefRawExpr::get_ref_expr()
 | 
						|
{
 | 
						|
  ObRawExpr* ret = NULL;
 | 
						|
  if (project_index_ == OB_INVALID_INDEX) {
 | 
						|
    ret = ref_expr_;
 | 
						|
  } else if (ref_expr_->is_query_ref_expr()) {
 | 
						|
    ObQueryRefRawExpr* subquery = static_cast<ObQueryRefRawExpr*>(ref_expr_);
 | 
						|
    ObSelectStmt* stmt = NULL;
 | 
						|
    if (OB_ISNULL(stmt = subquery->get_ref_stmt()) ||
 | 
						|
        OB_UNLIKELY(project_index_ < 0 || project_index_ >= stmt->get_select_item_size())) {
 | 
						|
      // do nothing
 | 
						|
    } else {
 | 
						|
      ret = stmt->get_select_item(project_index_).expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObAliasRefRawExpr::assign(const ObAliasRefRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      ref_expr_ = other.ref_expr_;
 | 
						|
      project_index_ = other.project_index_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObAliasRefRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObAliasRefRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.ref_expr_, ref_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy ref expr", K(ret));
 | 
						|
    } else {
 | 
						|
      project_index_ = other.project_index_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObAliasRefRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_ERR_UNEXPECTED;
 | 
						|
  if (OB_FAIL(ObRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, ref_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObAliasRefRawExpr::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  ObRawExpr* expr = NULL;
 | 
						|
  if (index < 0 || index > 1) {
 | 
						|
    LOG_WARN("index out of range", K(index));
 | 
						|
  } else {
 | 
						|
    expr = ref_expr_;
 | 
						|
  }
 | 
						|
  return expr;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr*& ObAliasRefRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  ObRawExpr** expr = &USELESS_POINTER;
 | 
						|
  if (index < 0 || index >= 1) {
 | 
						|
    LOG_WARN("index out of range", K(index));
 | 
						|
  } else {
 | 
						|
    expr = &ref_expr_;
 | 
						|
  }
 | 
						|
  return *expr;
 | 
						|
}
 | 
						|
 | 
						|
int ObAliasRefRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObAliasRefRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  uint64_t hash_code = 0;
 | 
						|
  if (ref_expr_ != NULL) {
 | 
						|
    hash_code = ref_expr_->hash(seed);
 | 
						|
  }
 | 
						|
  return hash_code;
 | 
						|
}
 | 
						|
 | 
						|
int ObAliasRefRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(ref_expr_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("Ref expr is NULL", K(ret));
 | 
						|
  } else if (OB_FAIL(ref_expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
  } else {
 | 
						|
  }  // do nothing
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObAliasRefRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  UNUSED(check_context);
 | 
						|
  bool bret = false;
 | 
						|
  if (OB_ISNULL(ref_expr_)) {
 | 
						|
    LOG_WARN("ref_expr_ is null");
 | 
						|
  } else if (expr.get_expr_type() == get_expr_type()) {
 | 
						|
    const ObAliasRefRawExpr& alias_ref = static_cast<const ObAliasRefRawExpr&>(expr);
 | 
						|
    if (alias_ref.get_expr_level() == get_expr_level()) {
 | 
						|
      bret = (alias_ref.get_ref_expr() == get_ref_expr());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
ObExprOperator* ObNonTerminalRawExpr::get_op()
 | 
						|
{
 | 
						|
  if (NULL == op_) {
 | 
						|
    if (OB_ISNULL(inner_alloc_)) {
 | 
						|
      LOG_WARN("invalid inner alloc", K(inner_alloc_));
 | 
						|
    } else if (type_ != T_OP_ROW) {
 | 
						|
      ObExprOperatorFactory factory(*inner_alloc_);
 | 
						|
      if (OB_SUCCESS != factory.alloc(type_, op_)) {
 | 
						|
        LOG_WARN("Can not malloc expression operator", "expr_type", get_type_name(type_), K(lbt()));
 | 
						|
      } else if (OB_ISNULL(op_)) {
 | 
						|
        LOG_ERROR("alloc a null expr_op", K(op_));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return op_;
 | 
						|
}
 | 
						|
 | 
						|
void ObNonTerminalRawExpr::free_op()
 | 
						|
{
 | 
						|
  if (NULL != op_) {
 | 
						|
    op_ = NULL;
 | 
						|
  }
 | 
						|
}
 | 
						|
 | 
						|
int ObNonTerminalRawExpr::assign(const ObNonTerminalRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(this->input_types_.assign(other.input_types_))) {
 | 
						|
      LOG_WARN("copy input types failed", K(ret));
 | 
						|
    } else if (other.op_ != NULL) {
 | 
						|
      ObExprOperator* this_op = get_op();
 | 
						|
      if (OB_ISNULL(this_op)) {
 | 
						|
        ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
						|
        LOG_WARN("allocated memory for expr operator failed", K_(type));
 | 
						|
      } else if (OB_FAIL(this_op->assign(*other.op_))) {
 | 
						|
        LOG_WARN("assign this operator failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNonTerminalRawExpr::deep_copy(ObRawExprFactory& expr_factory, const ObNonTerminalRawExpr& other,
 | 
						|
    const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(this->input_types_.assign(other.input_types_))) {
 | 
						|
      LOG_WARN("copy input types failed", K(ret));
 | 
						|
    } else if (other.op_ != NULL) {
 | 
						|
      ObExprOperator* this_op = get_op();
 | 
						|
      if (OB_ISNULL(this_op)) {
 | 
						|
        ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
						|
        LOG_WARN("allocated memory for expr operator failed", K_(type));
 | 
						|
      } else if (OB_FAIL(this_op->assign(*other.op_))) {
 | 
						|
        LOG_WARN("assign this operator failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNonTerminalRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr for base class ObRawExpr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
ObOpRawExpr::ObOpRawExpr(ObRawExpr* first_expr, ObRawExpr* second_expr, ObItemType type)
 | 
						|
    : ObExpr(),
 | 
						|
      ObNonTerminalRawExpr(),
 | 
						|
      jit::expr::ObOpExpr(),
 | 
						|
      subquery_key_(T_WITH_NONE),
 | 
						|
      deduce_type_adding_implicit_cast_(true)
 | 
						|
{
 | 
						|
  set_param_exprs(first_expr, second_expr);
 | 
						|
  set_expr_type(type);
 | 
						|
  set_expr_class(EXPR_OPERATOR);
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::set_param_expr(ObRawExpr* expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(0 == exprs_.count())) {
 | 
						|
    if (OB_FAIL(exprs_.push_back(expr))) {
 | 
						|
      LOG_WARN("failed to set param", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("can not be set twice", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::set_param_exprs(ObRawExpr* first_expr, ObRawExpr* second_expr, ObRawExpr* third_expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(0 == exprs_.count())) {
 | 
						|
    if (OB_FAIL(exprs_.push_back(first_expr))) {
 | 
						|
      LOG_WARN("failed to set param", K(ret));
 | 
						|
    } else if (OB_FAIL(exprs_.push_back(second_expr))) {
 | 
						|
      LOG_WARN("failed to set param", K(ret));
 | 
						|
    } else if (OB_FAIL(exprs_.push_back(third_expr))) {
 | 
						|
      LOG_WARN("failed to set param", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("can not be set twice", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::set_param_exprs(ObRawExpr* first_expr, ObRawExpr* second_expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObItemType exchange_type = T_MIN_OP;
 | 
						|
  switch (get_expr_type()) {
 | 
						|
    case T_OP_LE:
 | 
						|
      exchange_type = T_OP_GE;
 | 
						|
      break;
 | 
						|
    case T_OP_LT:
 | 
						|
      exchange_type = T_OP_GT;
 | 
						|
      break;
 | 
						|
    case T_OP_GE:
 | 
						|
      exchange_type = T_OP_LE;
 | 
						|
      break;
 | 
						|
    case T_OP_GT:
 | 
						|
      exchange_type = T_OP_LT;
 | 
						|
      break;
 | 
						|
    case T_OP_EQ:
 | 
						|
    case T_OP_NSEQ:
 | 
						|
    case T_OP_NE:
 | 
						|
      exchange_type = get_expr_type();
 | 
						|
      break;
 | 
						|
    default:
 | 
						|
      exchange_type = T_MIN_OP;
 | 
						|
      break;
 | 
						|
  }
 | 
						|
  if (T_MIN_OP != exchange_type && first_expr && first_expr->has_flag(IS_CONST) && second_expr &&
 | 
						|
      second_expr->has_flag(IS_COLUMN)) {
 | 
						|
    set_expr_type(exchange_type);
 | 
						|
    ObRawExpr* tmp_expr = second_expr;
 | 
						|
    second_expr = first_expr;
 | 
						|
    first_expr = tmp_expr;
 | 
						|
  }
 | 
						|
 | 
						|
  if (0 == exprs_.count()) {
 | 
						|
    if (OB_ISNULL(first_expr) || OB_ISNULL(second_expr)) {
 | 
						|
      ret = OB_INVALID_ARGUMENT;
 | 
						|
      LOG_WARN("invalid argument", K(first_expr), K(second_expr), K(ret));
 | 
						|
    } else if (OB_FAIL(exprs_.push_back(first_expr))) {
 | 
						|
      LOG_WARN("failed to push back first_expr", K(ret));
 | 
						|
    } else if (OB_FAIL(exprs_.push_back(second_expr))) {
 | 
						|
      LOG_WARN("failed to push back gc_expr", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("can not be set twice", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObOpRawExpr::reset()
 | 
						|
{
 | 
						|
  ObNonTerminalRawExpr::reset();
 | 
						|
  clear_child();
 | 
						|
  deduce_type_adding_implicit_cast_ = true;
 | 
						|
}
 | 
						|
 | 
						|
// used for jit expr
 | 
						|
// Initialize when jit_exprs_ is obtained, not when initializing exprs_.Reason:
 | 
						|
// Resolving ObColumnRefRawExpr in the resolver is just an uninitialized ObColumnRefRawExpr
 | 
						|
// Push the object into exprs_, and the object will be replaced later, so exprs_ is not initialized
 | 
						|
// together here.
 | 
						|
 | 
						|
int ObOpRawExpr::get_children(ExprArray& jit_exprs) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  for (int i = 0; OB_SUCCESS == ret && i < exprs_.count(); i++) {
 | 
						|
    if (OB_FAIL(jit_exprs.push_back(static_cast<ObExpr*>(exprs_.at(i))))) {
 | 
						|
      LOG_WARN("fail to gen ObExpr Array", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::assign(const ObOpRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObNonTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObNonTerminalRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(exprs_.assign(other.exprs_))) {
 | 
						|
      LOG_WARN("failed to assign exprs", K(ret));
 | 
						|
    } else {
 | 
						|
      subquery_key_ = other.subquery_key_;
 | 
						|
      deduce_type_adding_implicit_cast_ = other.deduce_type_adding_implicit_cast_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObOpRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObNonTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObNonTerminalRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(expr_factory, other.exprs_, exprs_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to assign exprs", K(ret));
 | 
						|
    } else {
 | 
						|
      subquery_key_ = other.subquery_key_;
 | 
						|
      deduce_type_adding_implicit_cast_ = other.deduce_type_adding_implicit_cast_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObNonTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, exprs_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObOpRawExpr::clear_child()
 | 
						|
{
 | 
						|
  exprs_.reset();
 | 
						|
}
 | 
						|
 | 
						|
bool ObOpRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  bool need_cmp = true;
 | 
						|
 | 
						|
  enum { REGULAR_CMP = 1, REVERSE_CMP = 2, BOTH_CMP = 3 } cmp_type;
 | 
						|
 | 
						|
  cmp_type = REGULAR_CMP;
 | 
						|
 | 
						|
  if (T_OP_EQ == get_expr_type() || T_OP_NSEQ == get_expr_type() || T_OP_NE == get_expr_type()) {
 | 
						|
    if (expr.get_expr_type() == get_expr_type()) {
 | 
						|
      cmp_type = BOTH_CMP;
 | 
						|
    } else {
 | 
						|
      need_cmp = false;
 | 
						|
    }
 | 
						|
  } else if (IS_COMMON_COMPARISON_OP(get_expr_type())) {
 | 
						|
    // GE, GT, LE, LT
 | 
						|
    if (expr.get_expr_type() == get_expr_type()) {
 | 
						|
      cmp_type = REGULAR_CMP;
 | 
						|
    } else if ((T_OP_GE == expr.get_expr_type() && T_OP_LE == get_expr_type()) ||
 | 
						|
               (T_OP_GT == expr.get_expr_type() && T_OP_LT == get_expr_type()) ||
 | 
						|
               (T_OP_GE == get_expr_type() && T_OP_LE == expr.get_expr_type()) ||
 | 
						|
               (T_OP_GT == get_expr_type() && T_OP_LT == expr.get_expr_type())) {
 | 
						|
      cmp_type = REVERSE_CMP;
 | 
						|
    } else {
 | 
						|
      need_cmp = false;
 | 
						|
    }
 | 
						|
  } else if (expr.get_expr_type() != get_expr_type()) {
 | 
						|
    need_cmp = false;
 | 
						|
  }
 | 
						|
 | 
						|
  if (need_cmp) {
 | 
						|
    const ObOpRawExpr* m_expr = static_cast<const ObOpRawExpr*>(&expr);
 | 
						|
    if (BOTH_CMP == cmp_type || REGULAR_CMP == cmp_type) {
 | 
						|
      if (this->get_param_count() == m_expr->get_param_count()) {
 | 
						|
        bool_ret = true;
 | 
						|
        for (int64_t i = 0; bool_ret && i < m_expr->get_param_count(); ++i) {
 | 
						|
          if (NULL == this->get_param_expr(i) || NULL == m_expr->get_param_expr(i) ||
 | 
						|
              !(this->get_param_expr(i)->same_as(*m_expr->get_param_expr(i), check_context))) {
 | 
						|
            bool_ret = false;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (!bool_ret && (BOTH_CMP == cmp_type || REVERSE_CMP == cmp_type)) {
 | 
						|
      if (NULL == this->get_param_expr(0) || NULL == m_expr->get_param_expr(0) || NULL == this->get_param_expr(1) ||
 | 
						|
          NULL == m_expr->get_param_expr(1)) {
 | 
						|
        /* bool_ret = false; */
 | 
						|
      } else {
 | 
						|
        bool_ret = this->get_param_expr(0)->same_as(*m_expr->get_param_expr(1), check_context) &&
 | 
						|
                   this->get_param_expr(1)->same_as(*m_expr->get_param_expr(0), check_context);
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObString symbol;
 | 
						|
  switch (get_expr_type()) {
 | 
						|
    case T_OP_ADD:
 | 
						|
    case T_OP_AGG_ADD:
 | 
						|
      symbol = "+";
 | 
						|
      break;
 | 
						|
    case T_OP_MINUS:
 | 
						|
    case T_OP_AGG_MINUS:
 | 
						|
      symbol = "-";
 | 
						|
      break;
 | 
						|
    case T_OP_MUL:
 | 
						|
    case T_OP_AGG_MUL:
 | 
						|
      symbol = "*";
 | 
						|
      break;
 | 
						|
    case T_OP_DIV:
 | 
						|
    case T_OP_AGG_DIV:
 | 
						|
      symbol = "/";
 | 
						|
      break;
 | 
						|
    case T_OP_MOD:
 | 
						|
      symbol = (EXPLAIN_DBLINK_STMT == type) ? "MOD" : "%";
 | 
						|
      break;
 | 
						|
    case T_OP_INT_DIV:
 | 
						|
      symbol = "DIV";
 | 
						|
      break;
 | 
						|
    case T_OP_LE:
 | 
						|
    case T_OP_SQ_LE:
 | 
						|
      symbol = "<=";
 | 
						|
      break;
 | 
						|
    case T_OP_LT:
 | 
						|
    case T_OP_SQ_LT:
 | 
						|
      symbol = "<";
 | 
						|
      break;
 | 
						|
    case T_OP_EQ:
 | 
						|
    case T_OP_SQ_EQ:
 | 
						|
      symbol = "=";
 | 
						|
      break;
 | 
						|
    case T_OP_NSEQ:
 | 
						|
    case T_OP_SQ_NSEQ:
 | 
						|
      symbol = "<=>";
 | 
						|
      break;
 | 
						|
    case T_OP_GE:
 | 
						|
    case T_OP_SQ_GE:
 | 
						|
      symbol = ">=";
 | 
						|
      break;
 | 
						|
    case T_OP_GT:
 | 
						|
    case T_OP_SQ_GT:
 | 
						|
      symbol = ">";
 | 
						|
      break;
 | 
						|
    case T_OP_NE:
 | 
						|
    case T_OP_SQ_NE:
 | 
						|
      symbol = "!=";
 | 
						|
      break;
 | 
						|
    case T_OP_AND:
 | 
						|
      symbol = "AND";
 | 
						|
      break;
 | 
						|
    case T_OP_OR:
 | 
						|
      symbol = "OR";
 | 
						|
      break;
 | 
						|
    case T_OP_IN:
 | 
						|
      symbol = "IN";
 | 
						|
      break;
 | 
						|
    case T_OP_POW:
 | 
						|
      symbol = "POW";
 | 
						|
      break;
 | 
						|
    case T_OP_BIT_XOR:
 | 
						|
      symbol = "^";
 | 
						|
      break;
 | 
						|
    case T_OP_XOR:
 | 
						|
      symbol = "XOR";
 | 
						|
      break;
 | 
						|
    case T_OP_MULTISET:
 | 
						|
      symbol = "MULTISET";
 | 
						|
      break;
 | 
						|
    case T_OP_COLL_PRED:
 | 
						|
      symbol = "COLL_PRED";
 | 
						|
      break;
 | 
						|
    case T_OP_IS:
 | 
						|
      symbol = "IS";
 | 
						|
      break;
 | 
						|
    case T_OP_IS_NOT:
 | 
						|
      symbol = "IS NOT";
 | 
						|
      break;
 | 
						|
    default:
 | 
						|
      break;
 | 
						|
  }
 | 
						|
  if (IS_SUBQUERY_COMPARISON_OP(get_expr_type())) {
 | 
						|
    if (OB_FAIL(get_subquery_comparison_name(symbol, buf, buf_len, pos, type))) {
 | 
						|
      LOG_WARN("get subquery comparison name failed", K(ret));
 | 
						|
    }
 | 
						|
  } else if ((!symbol.empty() && 2 == get_param_count()) ||
 | 
						|
             (EXPLAIN_DBLINK_STMT == type && (T_OP_IS == get_expr_type() || T_OP_IS_NOT == get_expr_type()))) {
 | 
						|
    if (OB_ISNULL(get_param_expr(0))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("first param expr is NULL", K(ret));
 | 
						|
    } else if (OB_ISNULL(get_param_expr(1))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("second param expr is NULL", K(ret));
 | 
						|
    } else if (EXPLAIN_DBLINK_STMT == type && T_OP_MOD == get_expr_type()) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("%.*s", symbol.length(), symbol.ptr()))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(get_param_expr(0)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(get_param_expr(1)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      if (EXPLAIN_DBLINK_STMT == type && OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(get_param_expr(0)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("%.*s", symbol.length(), symbol.ptr()))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(get_param_expr(1)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (EXPLAIN_DBLINK_STMT == type && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (T_OP_ROW == get_expr_type()) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < get_param_count(); ++i) {
 | 
						|
        if (OB_ISNULL(get_param_expr(i))) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("param expr is NULL", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
        } else if (i < get_param_count() - 1) {
 | 
						|
          if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCCESS == ret && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (T_OP_ORACLE_OUTER_JOIN_SYMBOL == get_expr_type()) {
 | 
						|
    if (OB_UNLIKELY(1 != get_param_count()) || OB_ISNULL(get_param_expr(0))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("first param expr is NULL", K(ret), K(get_param_count()));
 | 
						|
    } else if (OB_FAIL(get_param_expr(0)->get_name(buf, buf_len, pos, type))) {
 | 
						|
      LOG_WARN("fail to get_name", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("(+)"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  } else if (has_flag(IS_INNER_ADDED_EXPR) && (EXPLAIN_EXTENDED != type && EXPLAIN_EXTENDED_NOADDR != type) &&
 | 
						|
             T_OP_BOOL == get_expr_type()) {
 | 
						|
    CK(1 == get_param_count());
 | 
						|
    OZ(get_param_expr(0)->get_name(buf, buf_len, pos, type));
 | 
						|
  } else if (T_OP_BOOL == get_expr_type()) {
 | 
						|
    if (OB_UNLIKELY(1 != get_param_count())) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("param_count of bool expr is invalid", K(ret), K(get_param_count()));
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("BOOL("))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_ISNULL(get_param_expr(0))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("param expr is NULL", K(ret));
 | 
						|
      } else if (OB_FAIL(get_param_expr(0)->get_name_internal(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name_internal", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (T_OP_TO_OUTFILE_ROW == get_expr_type()) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("SYS_OP_TO_OUTFILE_ROW("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < get_param_count(); ++i) {
 | 
						|
        if (OB_ISNULL(get_param_expr(i))) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("param_expr is NULL", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
        } else if (i < get_param_count() - 1) {
 | 
						|
          if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
          }
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("(%s", get_type_name(get_expr_type())))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < get_param_count(); ++i) {
 | 
						|
        if (OB_ISNULL(get_param_expr(i))) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("param_expr is NULL", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCCESS == ret && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (EXPLAIN_EXTENDED == type) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObOpRawExpr::get_subquery_comparison_name(
 | 
						|
    const ObString& symbol, char* buf, int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const ObRawExpr* param1 = NULL;
 | 
						|
  const ObRawExpr* param2 = NULL;
 | 
						|
  ObString subquery_keyname = "";
 | 
						|
  if (has_flag(IS_WITH_ANY)) {
 | 
						|
    subquery_keyname = "ANY";
 | 
						|
  } else if (has_flag(IS_WITH_ALL)) {
 | 
						|
    subquery_keyname = "ALL";
 | 
						|
  }
 | 
						|
  if (OB_UNLIKELY(symbol.empty()) || OB_ISNULL(buf) || OB_UNLIKELY(buf_len <= 0) || OB_UNLIKELY(pos < 0)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(symbol), K(buf), K(buf_len), K(pos));
 | 
						|
  } else if (OB_UNLIKELY(get_param_count() != 2)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("expr param count is unexpected", K(get_param_count()));
 | 
						|
  } else if (OB_ISNULL(param1 = get_param_expr(0)) || OB_ISNULL(param2 = get_param_expr(1))) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("param is null", K(param1), K(param2));
 | 
						|
  } else if (OB_FAIL(param1->get_name(buf, buf_len, pos, type))) {
 | 
						|
    LOG_WARN("get param name failed", K(ret));
 | 
						|
  } else if (OB_FAIL(BUF_PRINTF(
 | 
						|
                 " %.*s %.*s", symbol.length(), symbol.ptr(), subquery_keyname.length(), subquery_keyname.ptr()))) {
 | 
						|
    LOG_WARN("print symbol name failed", K(ret));
 | 
						|
  } else if (!subquery_keyname.empty() && OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
    LOG_WARN("print paren failed", K(ret));
 | 
						|
  } else if (OB_FAIL(param2->get_name(buf, buf_len, pos, type))) {
 | 
						|
    LOG_WARN("get param2 name failed", K(ret));
 | 
						|
  } else if (!subquery_keyname.empty() && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
    LOG_WARN("print paren failed", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObOpRawExpr::set_expr_type(ObItemType type)
 | 
						|
{
 | 
						|
  type_ = type;
 | 
						|
  if ((T_OP_IN == type_ || T_OP_NOT_IN == type_) && GCONF.enable_static_engine_for_query()) {
 | 
						|
    set_deduce_type_adding_implicit_cast(false);
 | 
						|
  }
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
 | 
						|
int ObObjAccessRawExpr::assign(const ObObjAccessRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObOpRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObOpRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(append(var_indexs_, other.var_indexs_))) {
 | 
						|
      LOG_WARN("append error", K(ret));
 | 
						|
    } else {
 | 
						|
      get_attr_func_ = other.get_attr_func_;
 | 
						|
      func_name_ = other.func_name_;
 | 
						|
      for_write_ = other.for_write_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObObjAccessRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObObjAccessRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObOpRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObOpRawExpr failed", K(ret));
 | 
						|
    } else if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator is NULL", K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.func_name_, func_name_))) {
 | 
						|
        LOG_WARN("Failed to write string", K(other.func_name_), K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      func_name_ = other.func_name_;
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(append(var_indexs_, other.var_indexs_))) {
 | 
						|
        LOG_WARN("append error", K(ret));
 | 
						|
      } else {
 | 
						|
        get_attr_func_ = other.get_attr_func_;
 | 
						|
        for_write_ = other.for_write_;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObObjAccessRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
  } else if (ObOpRawExpr::same_as(expr, check_context)) {
 | 
						|
    const ObObjAccessRawExpr& obj_access_expr = static_cast<const ObObjAccessRawExpr&>(expr);
 | 
						|
    bool_ret = get_attr_func_ == obj_access_expr.get_attr_func_ &&
 | 
						|
               0 == func_name_.case_compare(obj_access_expr.func_name_) &&
 | 
						|
               is_array_equal(var_indexs_, obj_access_expr.var_indexs_) && for_write_ == obj_access_expr.for_write_;
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNonTerminalRawExpr::set_input_types(const ObIExprResTypes& input_types)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  input_types_.reset();
 | 
						|
  if (OB_ISNULL(inner_alloc_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("inner allocator is NULL", K(ret));
 | 
						|
  } else if (OB_FAIL(input_types_.prepare_allocate(input_types.count()))) {
 | 
						|
    LOG_WARN("fail to prepare allocator", K(ret));
 | 
						|
  }
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < input_types.count(); ++i) {
 | 
						|
    input_types_.at(i).set_allocator(inner_alloc_);
 | 
						|
    if (OB_FAIL(input_types_.at(i).assign(input_types.at(i)))) {
 | 
						|
      LOG_WARN("fail to assign input types", K(input_types), K(input_types_), K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObCaseOpRawExpr::assign(const ObCaseOpRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObNonTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObNonTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      arg_expr_ = other.arg_expr_;
 | 
						|
      when_exprs_ = other.when_exprs_;
 | 
						|
      then_exprs_ = other.then_exprs_;
 | 
						|
      default_expr_ = other.default_expr_;
 | 
						|
      is_decode_func_ = other.is_decode_func_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObCaseOpRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObCaseOpRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObNonTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObNonTerminalRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.arg_expr_, arg_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy argument expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(
 | 
						|
                   expr_factory, other.when_exprs_, when_exprs_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy when expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(
 | 
						|
                   expr_factory, other.then_exprs_, then_exprs_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy then exprs", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.default_expr_, default_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy default expr", K(ret));
 | 
						|
    } else {
 | 
						|
      is_decode_func_ = other.is_decode_func_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObCaseOpRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObNonTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, arg_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, default_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, when_exprs_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, then_exprs_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// jit exprs gen order:
 | 
						|
//[arg_expr], when_expr, then_expr, [when_expr, then_expr,]...[default_expr]
 | 
						|
int ObCaseOpRawExpr::get_children(ExprArray& jit_exprs) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (arg_expr_ != NULL) {
 | 
						|
    if (OB_FAIL(jit_exprs.push_back(static_cast<ObExpr*>(arg_expr_)))) {
 | 
						|
      LOG_WARN("fail to push arg_expr_", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (when_exprs_.count() != then_exprs_.count()) {
 | 
						|
      LOG_WARN("when_exprs count is not equal to then_exprs count", K(when_exprs_.count()), K(then_exprs_.count()));
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCCESS == ret && i < when_exprs_.count(); i++) {
 | 
						|
      if (OB_FAIL(jit_exprs.push_back(static_cast<ObExpr*>(when_exprs_.at(i))))) {
 | 
						|
        LOG_WARN("fail to push when_expr_", K(ret));
 | 
						|
      } else if (OB_FAIL(jit_exprs.push_back(static_cast<ObExpr*>(then_exprs_.at(i))))) {
 | 
						|
        LOG_WARN("fail to push then_expr_", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret) && NULL != default_expr_) {
 | 
						|
    if (OB_FAIL(jit_exprs.push_back(static_cast<ObExpr*>(default_expr_)))) {
 | 
						|
      LOG_WARN("fail to push arg_expr_", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObCaseOpRawExpr::clear_child()
 | 
						|
{
 | 
						|
  arg_expr_ = NULL;
 | 
						|
  when_exprs_.reset();
 | 
						|
  then_exprs_.reset();
 | 
						|
  default_expr_ = NULL;
 | 
						|
}
 | 
						|
 | 
						|
void ObCaseOpRawExpr::reset()
 | 
						|
{
 | 
						|
  ObNonTerminalRawExpr::reset();
 | 
						|
  clear_child();
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObCaseOpRawExpr::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  ObRawExpr* expr = const_cast<ObCaseOpRawExpr*>(this)->get_param_expr(index);
 | 
						|
  return &expr == &USELESS_POINTER ? NULL : expr;
 | 
						|
}
 | 
						|
 | 
						|
int ObCaseOpRawExpr::replace_param_expr(int64_t index, ObRawExpr* new_expr)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  ObRawExpr*& old_expr = this->get_param_expr(index);
 | 
						|
  if (OB_ISNULL(old_expr)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN(
 | 
						|
        "replace_param_expr for case expr failed", K(ret), K(index), K(get_when_expr_size()), K(get_then_expr_size()));
 | 
						|
  } else {
 | 
						|
    old_expr = new_expr;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// return child expr in visitor order:
 | 
						|
//    arg expr, when expr, then expr, when expr, then expr ..., default expr
 | 
						|
ObRawExpr*& ObCaseOpRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObRawExpr** expr = &USELESS_POINTER;
 | 
						|
  if (index < 0 || index >= get_param_count()) {
 | 
						|
    LOG_WARN("invalid index", K(index), K(get_param_count()));
 | 
						|
  } else if (when_exprs_.count() != then_exprs_.count()) {
 | 
						|
    LOG_WARN("when and then expr count mismatch", K(ret), K(when_exprs_.count()), K(then_exprs_.count()));
 | 
						|
  } else {
 | 
						|
    if (NULL != arg_expr_ && 0 == index) {
 | 
						|
      expr = &arg_expr_;
 | 
						|
    } else {
 | 
						|
      index -= (NULL != arg_expr_ ? 1 : 0);
 | 
						|
      if (index >= when_exprs_.count() * 2) {
 | 
						|
        if (NULL != default_expr_) {
 | 
						|
          expr = &default_expr_;
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        expr = &(index % 2 == 0 ? when_exprs_ : then_exprs_).at(index / 2);
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return *expr;
 | 
						|
}
 | 
						|
 | 
						|
bool ObCaseOpRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
  } else {
 | 
						|
    const ObCaseOpRawExpr* c_expr = static_cast<const ObCaseOpRawExpr*>(&expr);
 | 
						|
    if (((NULL == arg_expr_ && NULL == c_expr->arg_expr_) ||
 | 
						|
            (NULL != arg_expr_ && c_expr->arg_expr_ && arg_expr_->same_as(*c_expr->arg_expr_, check_context))) &&
 | 
						|
        ((NULL == default_expr_ && NULL == c_expr->default_expr_) ||
 | 
						|
            (NULL != default_expr_ && c_expr->default_expr_ &&
 | 
						|
                default_expr_->same_as(*c_expr->default_expr_, check_context))) &&
 | 
						|
        this->get_when_expr_size() == c_expr->get_when_expr_size()) {
 | 
						|
      bool_ret = true;
 | 
						|
      for (int64_t i = 0; bool_ret && i < c_expr->get_when_expr_size(); ++i) {
 | 
						|
        if (OB_ISNULL(this->get_when_param_expr(i)) || OB_ISNULL(c_expr->get_when_param_expr(i)) ||
 | 
						|
            OB_ISNULL(this->get_then_param_expr(i)) || OB_ISNULL(c_expr->get_then_param_expr(i)) ||
 | 
						|
            !this->get_when_param_expr(i)->same_as(*c_expr->get_when_param_expr(i), check_context) ||
 | 
						|
            !this->get_then_param_expr(i)->same_as(*c_expr->get_then_param_expr(i), check_context)) {
 | 
						|
          bool_ret = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObCaseOpRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObCaseOpRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(BUF_PRINTF("CASE"))) {
 | 
						|
    LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
  } else if (NULL != arg_expr_) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(arg_expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
      LOG_WARN("fail to get_name", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < get_when_expr_size(); ++i) {
 | 
						|
      if (OB_ISNULL(get_when_param_expr(i))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("when_param_expr is NULL", K(i), K(ret));
 | 
						|
      } else if (OB_ISNULL(get_then_param_expr(i))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("then_param_expr is NULL", K(i), K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("WHEN"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(i), K(ret));
 | 
						|
      } else if (OB_FAIL(get_when_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("THEN"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(get_then_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("ELSE"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else {
 | 
						|
        if (NULL != default_expr_) {
 | 
						|
          if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          } else if (OB_FAIL(default_expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
            LOG_WARN("fail to get_name", K(ret));
 | 
						|
          } else {
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF(" "))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("END"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCCESS == ret && EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObAggFunRawExpr::assign(const ObAggFunRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      real_param_exprs_ = other.real_param_exprs_;
 | 
						|
      push_down_sum_expr_ = other.push_down_sum_expr_;
 | 
						|
      push_down_count_expr_ = other.push_down_count_expr_;
 | 
						|
      push_down_synopsis_expr_ = other.push_down_synopsis_expr_;
 | 
						|
      distinct_ = other.distinct_;
 | 
						|
      order_items_ = other.order_items_;
 | 
						|
      separator_param_expr_ = other.separator_param_expr_;
 | 
						|
      linear_inter_expr_ = other.linear_inter_expr_;
 | 
						|
      is_nested_aggr_ = other.is_nested_aggr_;
 | 
						|
      udf_meta_.assign(other.udf_meta_);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObAggFunRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObAggFunRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObRawExpr* temp_expr = NULL;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.push_down_sum_expr_, push_down_sum_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy push down sum expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.push_down_count_expr_, push_down_count_expr_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy push down count expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(expr_factory,
 | 
						|
                   other.push_down_synopsis_expr_,
 | 
						|
                   push_down_synopsis_expr_,
 | 
						|
                   copy_types,
 | 
						|
                   use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy push down sysnopsis expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.separator_param_expr_, temp_expr, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy separator param expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(
 | 
						|
                   expr_factory, other.real_param_exprs_, real_param_exprs_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy real param exprs", K(ret));
 | 
						|
    } else if (OB_FAIL(ObTransformUtils::deep_copy_order_items(
 | 
						|
                   expr_factory, other.order_items_, order_items_, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy order item", K(ret));
 | 
						|
    } else {
 | 
						|
      distinct_ = other.distinct_;
 | 
						|
      udf_meta_ = other.udf_meta_;
 | 
						|
      is_nested_aggr_ = other.is_nested_aggr_;
 | 
						|
      linear_inter_expr_ = other.linear_inter_expr_;
 | 
						|
      if (NULL != temp_expr) {
 | 
						|
        separator_param_expr_ = temp_expr;
 | 
						|
      }
 | 
						|
      if (OB_FAIL(common::ob_write_string(expr_factory.get_allocator(), other.udf_meta_.name_, udf_meta_.name_))) {
 | 
						|
        LOG_WARN("failed to write string", K(ret));
 | 
						|
      } else if (OB_FAIL(common::ob_write_string(expr_factory.get_allocator(), other.udf_meta_.dl_, udf_meta_.dl_))) {
 | 
						|
        LOG_WARN("failed to write string", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObAggFunRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, push_down_count_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, push_down_sum_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, push_down_synopsis_expr_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, real_param_exprs_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr_for_order_item(other_exprs, new_exprs, order_items_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObAggFunRawExpr::clear_child()
 | 
						|
{
 | 
						|
  real_param_exprs_.reset();
 | 
						|
}
 | 
						|
 | 
						|
void ObAggFunRawExpr::reset()
 | 
						|
{
 | 
						|
  ObRawExpr::reset();
 | 
						|
  distinct_ = false;
 | 
						|
  clear_child();
 | 
						|
  order_items_.reset();
 | 
						|
  separator_param_expr_ = NULL;
 | 
						|
  linear_inter_expr_ = NULL;
 | 
						|
  is_nested_aggr_ = false;
 | 
						|
}
 | 
						|
 | 
						|
bool ObAggFunRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
  } else {
 | 
						|
    const ObAggFunRawExpr* a_expr = static_cast<const ObAggFunRawExpr*>(&expr);
 | 
						|
    if (is_nested_aggr_ != a_expr->is_nested_aggr()) {
 | 
						|
      // do nothing.
 | 
						|
    } else if (distinct_ == a_expr->is_param_distinct()) {
 | 
						|
      if ((NULL == separator_param_expr_ && NULL == a_expr->separator_param_expr_) ||
 | 
						|
          (NULL != separator_param_expr_ && NULL != a_expr->separator_param_expr_ &&
 | 
						|
              separator_param_expr_->same_as(*(a_expr->separator_param_expr_), check_context))) {
 | 
						|
        if (real_param_exprs_.count() == a_expr->real_param_exprs_.count()) {
 | 
						|
          bool_ret = true;
 | 
						|
          for (int64_t i = 0; bool_ret && i < real_param_exprs_.count(); ++i) {
 | 
						|
            if (OB_ISNULL(real_param_exprs_.at(i)) || OB_ISNULL(a_expr->real_param_exprs_.at(i)) ||
 | 
						|
                !real_param_exprs_.at(i)->same_as(*(a_expr->real_param_exprs_.at(i)), check_context)) {
 | 
						|
              bool_ret = false;
 | 
						|
            }
 | 
						|
          }
 | 
						|
 | 
						|
          if (bool_ret) {
 | 
						|
            if (order_items_.count() == a_expr->order_items_.count()) {
 | 
						|
              for (int64_t i = 0; bool_ret && i < order_items_.count(); ++i) {
 | 
						|
                if (OB_ISNULL(order_items_.at(i).expr_) || OB_ISNULL(a_expr->order_items_.at(i).expr_) ||
 | 
						|
                    !order_items_.at(i).expr_->same_as(*(a_expr->order_items_.at(i).expr_), check_context)) {
 | 
						|
                  bool_ret = false;
 | 
						|
                }
 | 
						|
              }
 | 
						|
            } else {
 | 
						|
              bool_ret = false;
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (bool_ret) {
 | 
						|
            if ((NULL == linear_inter_expr_ && NULL == a_expr->linear_inter_expr_) ||
 | 
						|
                (NULL != linear_inter_expr_ && NULL != a_expr->linear_inter_expr_ &&
 | 
						|
                    linear_inter_expr_->same_as(*(a_expr->linear_inter_expr_), check_context))) {
 | 
						|
              bool_ret = true;
 | 
						|
            } else {
 | 
						|
              bool_ret = false;
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObAggFunRawExpr::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  const ObRawExpr* ptr_ret = NULL;
 | 
						|
  if (0 <= index && index < real_param_exprs_.count()) {
 | 
						|
    ptr_ret = real_param_exprs_.at(index);
 | 
						|
  } else if ((0 <= index - real_param_exprs_.count()) && (index - real_param_exprs_.count() < order_items_.count())) {
 | 
						|
    ptr_ret = order_items_.at(index - real_param_exprs_.count()).expr_;
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
  return ptr_ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr*& ObAggFunRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  if (0 <= index && index < real_param_exprs_.count()) {
 | 
						|
    return real_param_exprs_.at(index);
 | 
						|
  } else if ((0 <= index - real_param_exprs_.count()) && (index - real_param_exprs_.count() < order_items_.count())) {
 | 
						|
    return order_items_.at(index - real_param_exprs_.count()).expr_;
 | 
						|
  } else {
 | 
						|
    return USELESS_POINTER;
 | 
						|
  }
 | 
						|
}
 | 
						|
 | 
						|
int ObAggFunRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObAggFunRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (T_FUN_AGG_UDF == get_expr_type()) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("%.*s(", udf_meta_.name_.length(), udf_meta_.name_.ptr()))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  } else if (EXPLAIN_DBLINK_STMT == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("%s(", get_name_dblink(get_expr_type())))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("%s(", get_type_name(get_expr_type())))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (distinct_) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("distinct "))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    int64_t i = 0;
 | 
						|
    if (T_FUN_MEDIAN == get_expr_type() || T_FUN_GROUP_PERCENTILE_CONT == get_expr_type()) {
 | 
						|
      if (1 > get_real_param_count()) {
 | 
						|
        ret = OB_ERR_PARAM_SIZE;
 | 
						|
        LOG_WARN("invalid number of arguments", K(ret), K(get_expr_type()));
 | 
						|
      } else if (OB_FAIL(get_real_param_exprs().at(0)->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      for (; OB_SUCC(ret) && i < get_real_param_count() - 1; ++i) {
 | 
						|
        if (OB_ISNULL(get_real_param_exprs().at(i))) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("param expr is NULL", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(get_real_param_exprs().at(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (0 == get_real_param_count()) {
 | 
						|
          if (OB_FAIL(BUF_PRINTF("*"))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          }
 | 
						|
        } else if (OB_ISNULL(get_real_param_exprs().at(i))) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("param expr is NULL", K(i), K(ret));
 | 
						|
        } else if (OB_FAIL(get_real_param_exprs().at(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCCESS == ret && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
    if (OB_SUCCESS == ret &&
 | 
						|
        (T_FUN_GROUP_CONCAT == get_expr_type() || T_FUN_GROUP_RANK == get_expr_type() ||
 | 
						|
            T_FUN_GROUP_DENSE_RANK == get_expr_type() || T_FUN_GROUP_PERCENT_RANK == get_expr_type() ||
 | 
						|
            T_FUN_GROUP_CUME_DIST == get_expr_type() || T_FUN_GROUP_PERCENTILE_CONT == get_expr_type() ||
 | 
						|
            T_FUN_GROUP_PERCENTILE_DISC == get_expr_type() || T_FUN_KEEP_MAX == get_expr_type() ||
 | 
						|
            T_FUN_KEEP_MIN == get_expr_type() || T_FUN_KEEP_SUM == get_expr_type() ||
 | 
						|
            T_FUN_KEEP_COUNT == get_expr_type() || T_FUN_KEEP_WM_CONCAT == get_expr_type())) {
 | 
						|
      if (order_items_.count() > 0) {
 | 
						|
        if (OB_FAIL(BUF_PRINTF(" order_items("))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        }
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < order_items_.count(); ++i) {
 | 
						|
          if (i > 0) {
 | 
						|
            if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
              LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (OB_FAIL(ret)) {
 | 
						|
            // do nothing
 | 
						|
          } else if (OB_ISNULL(order_items_.at(i).expr_)) {
 | 
						|
            LOG_WARN("expr is NULL", K(i), K(ret));
 | 
						|
          } else if (OB_FAIL(order_items_.at(i).expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
            LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
          } else {
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (OB_SUCCESS == ret && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCCESS == ret && NULL != separator_param_expr_) {
 | 
						|
        if (OB_FAIL(BUF_PRINTF(" separator_param_expr("))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else if (OB_FAIL(separator_param_expr_->get_name(buf, buf_len, pos, type))) {
 | 
						|
          LOG_WARN("fail to get_name", K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCCESS == ret && EXPLAIN_EXTENDED == type) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
const char* ObAggFunRawExpr::get_name_dblink(ObItemType expr_type) const
 | 
						|
{
 | 
						|
  const char* name = NULL;
 | 
						|
  switch (expr_type) {
 | 
						|
    case T_FUN_MAX:
 | 
						|
      name = "max";
 | 
						|
      break;
 | 
						|
    case T_FUN_MIN:
 | 
						|
      name = "min";
 | 
						|
      break;
 | 
						|
    case T_FUN_SUM:
 | 
						|
      name = "sum";
 | 
						|
      break;
 | 
						|
    case T_FUN_COUNT:
 | 
						|
      name = "count";
 | 
						|
      break;
 | 
						|
    case T_FUN_AVG:
 | 
						|
      name = "avg";
 | 
						|
      break;
 | 
						|
    default:
 | 
						|
      name = "UNKNOWN";
 | 
						|
      break;
 | 
						|
  }
 | 
						|
  return name;
 | 
						|
}
 | 
						|
 | 
						|
int ObAggFunRawExpr::set_udf_meta(const share::schema::ObUDF& udf)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  udf_meta_.tenant_id_ = udf.get_tenant_id();
 | 
						|
  udf_meta_.ret_ = udf.get_ret();
 | 
						|
  udf_meta_.type_ = udf.get_type();
 | 
						|
 | 
						|
  if (OB_ISNULL(inner_alloc_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, udf.get_name_str(), udf_meta_.name_))) {
 | 
						|
    LOG_WARN("fail to write string", K(udf.get_name_str()), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, udf.get_dl_str(), udf_meta_.dl_))) {
 | 
						|
    LOG_WARN("fail to write string", K(udf.get_name_str()), K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
////////////////////////////////////////////////////////////////
 | 
						|
int ObSysFunRawExpr::assign(const ObSysFunRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObOpRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObOpRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      func_name_ = other.func_name_;
 | 
						|
      operator_id_ = other.operator_id_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObSysFunRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObOpRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObOpRawExpr failed", K(ret));
 | 
						|
    } else if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.func_name_, func_name_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.func_name_), K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      func_name_ = other.func_name_;
 | 
						|
      operator_id_ = other.operator_id_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObSysFunRawExpr::clear_child()
 | 
						|
{
 | 
						|
  exprs_.reset();
 | 
						|
}
 | 
						|
 | 
						|
void ObSysFunRawExpr::reset()
 | 
						|
{
 | 
						|
  ObNonTerminalRawExpr::reset();
 | 
						|
  func_name_.reset();
 | 
						|
  clear_child();
 | 
						|
}
 | 
						|
 | 
						|
bool ObSysFunRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  const ObSysFunRawExpr* s_expr = static_cast<const ObSysFunRawExpr*>(&expr);
 | 
						|
  if (get_expr_type() != expr.get_expr_type()) {
 | 
						|
  } else if (T_FUN_SYS_RAND == get_expr_type() || T_FUN_SYS_GUID == get_expr_type()) {
 | 
						|
  } else {
 | 
						|
    if (ObCharset::case_insensitive_equal(func_name_, s_expr->get_func_name()) &&
 | 
						|
        this->get_param_count() == s_expr->get_param_count()) {
 | 
						|
      bool_ret = true;
 | 
						|
      for (int64_t i = 0; bool_ret && i < s_expr->get_param_count(); i++) {
 | 
						|
        if (OB_ISNULL(get_param_expr(i)) || OB_ISNULL(s_expr->get_param_expr(i)) ||
 | 
						|
            !this->get_param_expr(i)->same_as(*s_expr->get_param_expr(i), check_context)) {
 | 
						|
          bool_ret = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
ObExprOperator* ObSysFunRawExpr::get_op()
 | 
						|
{
 | 
						|
  ObExprOperator* op = NULL;
 | 
						|
  if (get_expr_type() == T_FUN_SYS) {
 | 
						|
    free_op();
 | 
						|
    ObExprOperatorType type;
 | 
						|
    if (T_INVALID != (type = ObExprOperatorFactory::get_type_by_name(func_name_))) {
 | 
						|
      set_expr_type(type);
 | 
						|
    } else {
 | 
						|
      LOG_ERROR("invalid func name", K_(func_name));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_UNLIKELY(NULL == (op = ObOpRawExpr::get_op()))) {
 | 
						|
    LOG_ERROR("make function failed", K_(func_name));
 | 
						|
  }
 | 
						|
  return op;
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::check_param_num()
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObExprOperator* op = NULL;
 | 
						|
  ObExprOperatorType type;
 | 
						|
  if (OB_UNLIKELY(T_INVALID == (type = ObExprOperatorFactory::get_type_by_name(func_name_)))) {
 | 
						|
    ret = OB_ERR_FUNCTION_UNKNOWN;
 | 
						|
    LOG_WARN("system function not exists, maybe a user define function", K(func_name_), K(ret));
 | 
						|
  } else if (OB_UNLIKELY(NULL == (op = get_op()))) {
 | 
						|
    ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
						|
    LOG_ERROR("fail to make function", K(func_name_), K(ret));
 | 
						|
  } else {
 | 
						|
    int32_t param_num = op->get_param_num();
 | 
						|
    switch (param_num) {
 | 
						|
      case ObExprOperator::MORE_THAN_ZERO: {
 | 
						|
        if (get_param_count() <= 0) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function can not be 0", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::MORE_THAN_ONE: {
 | 
						|
        if (get_param_count() <= 1) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be more than 1", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::MORE_THAN_TWO: {
 | 
						|
        if (get_param_count() <= 2) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be more than 2", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::ZERO_OR_ONE: {
 | 
						|
        if (0 != get_param_count() && 1 != get_param_count()) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be 0 or 1", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::ONE_OR_TWO: {
 | 
						|
        if (get_param_count() != 1 && get_param_count() != 2) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be 1 or 2", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::TWO_OR_THREE: {
 | 
						|
        if (get_param_count() != 2 && get_param_count() != 3) {
 | 
						|
          if (share::is_oracle_mode() && (T_FUN_SYS_RPAD == type || T_FUN_SYS_LPAD == type)) {
 | 
						|
            if (get_param_count() > 3) {
 | 
						|
              ret = OB_ERR_TOO_MANY_ARGS_FOR_FUN;
 | 
						|
              LOG_WARN("param count larger than 3", K(ret), K(func_name_), K(get_param_count()));
 | 
						|
            } else {
 | 
						|
              ret = OB_ERR_NOT_ENOUGH_ARGS_FOR_FUN;
 | 
						|
              LOG_WARN("param count less than 2", K(ret), K(func_name_), K(get_param_count()));
 | 
						|
            }
 | 
						|
          } else {
 | 
						|
            ret = OB_ERR_PARAM_SIZE;
 | 
						|
            LOG_WARN("Param num of function should be 2 or 3", K(func_name_), K(ret), K(get_param_count()));
 | 
						|
          }
 | 
						|
        } else if (!share::is_oracle_mode() && T_FUN_SYS_REPLACE == type && get_param_count() != 3) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be 3", K(func_name_), K(ret), K(share::is_oracle_mode()));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::OCCUR_AS_PAIR: {
 | 
						|
        if (get_param_count() % 2 != 0) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("Param num of function should be even", K(func_name_), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case ObExprOperator::PARAM_NUM_UNKNOWN: {
 | 
						|
        // nothing
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      default: {
 | 
						|
        if (get_param_count() != param_num) {
 | 
						|
          ret = OB_ERR_PARAM_SIZE;
 | 
						|
          LOG_WARN("invalid Param num of function", K(func_name_), K(param_num), K(get_param_count()), K(ret));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_UNLIKELY(OB_ERR_PARAM_SIZE == ret)) {
 | 
						|
      LOG_USER_ERROR(OB_ERR_PARAM_SIZE, func_name_.length(), func_name_.ptr());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (has_flag(IS_INNER_ADDED_EXPR) && (EXPLAIN_EXTENDED != type && EXPLAIN_EXTENDED_NOADDR != type) &&
 | 
						|
      T_FUN_SYS_REMOVE_CONST == get_expr_type()) {
 | 
						|
    CK(1 == get_param_count());
 | 
						|
    OZ(get_param_expr(0)->get_name(buf, buf_len, pos, type));
 | 
						|
  } else if (OB_FAIL(BUF_PRINTF("%.*s(", get_func_name().length(), get_func_name().ptr()))) {
 | 
						|
    LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
  } else {
 | 
						|
    if (T_FUN_COLUMN_CONV == get_expr_type()) {
 | 
						|
      if (OB_FAIL(get_column_conv_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to get_column_conv_name", K(ret));
 | 
						|
      }
 | 
						|
    } else if (T_FUN_SYS_PART_ID == get_expr_type()) {
 | 
						|
      // ignore the print of T_FUN_SYS_PART_ID expr
 | 
						|
    } else {
 | 
						|
      int64_t i = 0;
 | 
						|
      if (get_param_count() > 1) {
 | 
						|
        int64_t real_param_count = get_param_count();
 | 
						|
        if (T_FUN_SYS_FROM_UNIX_TIME == get_expr_type() && 3 == real_param_count) {
 | 
						|
          real_param_count = 2;
 | 
						|
        } else if ((T_FUN_SYS_LEAST_INNER == get_expr_type() || T_FUN_SYS_GREATEST_INNER == get_expr_type())) {
 | 
						|
          real_param_count /= 3;
 | 
						|
        } else if (T_FUN_SYS_NULLIF == get_expr_type() && 6 == real_param_count) {
 | 
						|
          real_param_count = 2;
 | 
						|
        }
 | 
						|
        for (; OB_SUCC(ret) && i < real_param_count - 1; ++i) {
 | 
						|
          if (OB_ISNULL(get_param_expr(i))) {
 | 
						|
            ret = OB_ERR_UNEXPECTED;
 | 
						|
            LOG_WARN("param_expr is NULL", K(i), K(ret));
 | 
						|
          } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
            LOG_WARN("fail to get_name", K(i), K(ret));
 | 
						|
          } else if (OB_FAIL(BUF_PRINTF(", "))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          } else {
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (T_FUN_SYS_CAST != get_expr_type()) {
 | 
						|
          if (get_param_count() >= 1) {
 | 
						|
            if (OB_ISNULL(get_param_expr(i))) {
 | 
						|
              ret = OB_ERR_UNEXPECTED;
 | 
						|
              LOG_WARN("poram expr is NULL", K(i), K(ret));
 | 
						|
            } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) {
 | 
						|
              LOG_WARN("fail to get_name", K(ret));
 | 
						|
            } else {
 | 
						|
            }
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(get_cast_type_name(buf, buf_len, pos))) {
 | 
						|
            LOG_WARN("fail to get_cast_type_name", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      } else if (EXPLAIN_EXTENDED == type) {
 | 
						|
        if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::get_cast_type_name(char* buf, int64_t buf_len, int64_t& pos) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(get_param_expr(1))) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("second param expr is NULL", K(ret));
 | 
						|
  } else if (!get_param_expr(1)->is_const_expr()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("second param expr is not const expr", K(ret));
 | 
						|
  } else {
 | 
						|
    const ObConstRawExpr* const_expr = static_cast<const ObConstRawExpr*>(get_param_expr(1));
 | 
						|
    ObObj value = const_expr->get_value();
 | 
						|
    if (!value.is_int()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("value of second param expr is not int", K(value), K(ret));
 | 
						|
    } else {
 | 
						|
      ParseNode node;
 | 
						|
      node.value_ = value.get_int();
 | 
						|
      ObObjType dest_type = static_cast<ObObjType>(node.int16_values_[0]);
 | 
						|
      ObCollationType coll_type = static_cast<ObCollationType>(node.int16_values_[1]);
 | 
						|
      int32_t length = 0;
 | 
						|
      int16_t precision = 0;
 | 
						|
      int16_t scale = 0;
 | 
						|
      const char* type_str = ob_obj_type_str(dest_type);
 | 
						|
      if (ob_is_string_tc(dest_type)) {
 | 
						|
        length = node.int32_values_[1] < 0 ? static_cast<int32_t>(OB_MAX_VARCHAR_LENGTH) : node.int32_values_[1];
 | 
						|
        if (share::is_oracle_mode()) {
 | 
						|
          ObLengthSemantics length_semantics = const_expr->get_accuracy().get_length_semantics();
 | 
						|
          if (OB_FAIL(BUF_PRINTF("%s(%d %s)", type_str, length, get_length_semantics_str(length_semantics)))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(BUF_PRINTF("%s(%d)", type_str, length))) {
 | 
						|
            LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      } else if (ob_is_text_tc(dest_type)) {
 | 
						|
        // TODO texttc should use default length
 | 
						|
        length = ObAccuracy::DDL_DEFAULT_ACCURACY[dest_type].get_length();
 | 
						|
        if (OB_FAIL(BUF_PRINTF("%s(%d)", type_str, length))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        }
 | 
						|
      } else if (ob_is_lob_tc(dest_type)) {
 | 
						|
        const char* dest_type_str = CS_TYPE_BINARY == coll_type ? "blob" : "clob";
 | 
						|
        if (OB_FAIL(BUF_PRINTF(dest_type_str, type_str, length))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        precision = node.int16_values_[2];
 | 
						|
        scale = node.int16_values_[3];
 | 
						|
        if (OB_FAIL(BUF_PRINTF("%s(%d, %d)", type_str, precision, scale))) {
 | 
						|
          LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSysFunRawExpr::get_column_conv_name(char* buf, int64_t buf_len, int64_t& pos, ExplainType explain_type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(buf) || T_FUN_COLUMN_CONV != get_expr_type()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(buf), K(get_expr_type()), K(ret));
 | 
						|
  } else if (OB_ISNULL(get_param_expr(0)) || OB_ISNULL(get_param_expr(1)) || OB_ISNULL(get_param_expr(2)) ||
 | 
						|
             OB_ISNULL(get_param_expr(3)) || OB_ISNULL(get_param_expr(4))) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("parm expr is NULL",
 | 
						|
        K(get_param_expr(0)),
 | 
						|
        K(get_param_expr(1)),
 | 
						|
        K(get_param_expr(2)),
 | 
						|
        K(get_param_expr(3)),
 | 
						|
        K(get_param_expr(4)),
 | 
						|
        K(ret));
 | 
						|
  } else {
 | 
						|
    const ObConstRawExpr* type_expr = static_cast<const ObConstRawExpr*>(get_param_expr(0));
 | 
						|
    ObObjType type = static_cast<ObObjType>(type_expr->get_value().get_int());
 | 
						|
    const ObConstRawExpr* collation_expr = static_cast<const ObConstRawExpr*>(get_param_expr(1));
 | 
						|
    common::ObCollationType cs_type = static_cast<common::ObCollationType>(collation_expr->get_value().get_int());
 | 
						|
    const ObConstRawExpr* accuracy_expr = static_cast<const ObConstRawExpr*>(get_param_expr(2));
 | 
						|
    int64_t accuray_value = accuracy_expr->get_value().get_int();
 | 
						|
    ObAccuracy accuracy;
 | 
						|
    accuracy.set_accuracy(accuray_value);
 | 
						|
    const ObConstRawExpr* bool_expr = static_cast<const ObConstRawExpr*>(get_param_expr(3));
 | 
						|
    bool is_nullable = bool_expr->get_value().get_bool();
 | 
						|
    const char* type_str = ob_obj_type_str(type);
 | 
						|
    if (ob_is_string_type(type)) {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("%s,%s,length:%d,%s,",
 | 
						|
              type_str,
 | 
						|
              ObCharset::collation_name(cs_type),
 | 
						|
              accuracy.get_length(),
 | 
						|
              is_nullable ? "NULL" : "NOT NULL"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(BUF_PRINTF("%s,PS:(%d,%d),%s,",
 | 
						|
              type_str,
 | 
						|
              accuracy.get_precision(),
 | 
						|
              accuracy.get_scale(),
 | 
						|
              is_nullable ? "NULL" : "NOT NULL"))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && OB_FAIL(get_param_expr(4)->get_name(buf, buf_len, pos, explain_type))) {
 | 
						|
      LOG_WARN("fail to get_name", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSequenceRawExpr::assign(const ObSequenceRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObSysFunRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign sys raw expr");
 | 
						|
    } else {
 | 
						|
      name_ = other.name_;
 | 
						|
      action_ = other.action_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSequenceRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObSequenceRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObSysFunRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObSysRawExpr failed", K(ret));
 | 
						|
    } else if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.name_, name_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.name_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.action_, action_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.action_), K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      name_ = other.name_;
 | 
						|
      action_ = other.action_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSequenceRawExpr::set_sequence_meta(
 | 
						|
    const common::ObString& name, const common::ObString& action, uint64_t sequence_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ob_write_string(*inner_alloc_, name, name_))) {
 | 
						|
    LOG_WARN("fail to write string", K(name), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, action, action_))) {
 | 
						|
    LOG_WARN("fail to write string", K(action), K(ret));
 | 
						|
  } else {
 | 
						|
    sequence_id_ = sequence_id;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObSequenceRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  UNUSED(expr);
 | 
						|
  UNUSED(check_context);
 | 
						|
  return false;
 | 
						|
}
 | 
						|
 | 
						|
int ObSequenceRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(BUF_PRINTF("%.*s.%.*s", name_.length(), name_.ptr(), action_.length(), action_.ptr()))) {
 | 
						|
    LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
  } else if (EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNormalDllUdfRawExpr::set_udf_meta(const share::schema::ObUDF& udf)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  udf_meta_.tenant_id_ = udf.get_tenant_id();
 | 
						|
  udf_meta_.ret_ = udf.get_ret();
 | 
						|
  udf_meta_.type_ = udf.get_type();
 | 
						|
 | 
						|
  /* data from schame, deep copy maybe a better choices */
 | 
						|
  if (OB_ISNULL(inner_alloc_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, udf.get_name_str(), udf_meta_.name_))) {
 | 
						|
    LOG_WARN("fail to write string", K(udf.get_name_str()), K(ret));
 | 
						|
  } else if (OB_FAIL(ob_write_string(*inner_alloc_, udf.get_dl_str(), udf_meta_.dl_))) {
 | 
						|
    LOG_WARN("fail to write string", K(udf.get_name_str()), K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNormalDllUdfRawExpr::add_udf_attribute_name(const common::ObString& name)
 | 
						|
{
 | 
						|
  return udf_attributes_.push_back(name);
 | 
						|
}
 | 
						|
 | 
						|
int ObNormalDllUdfRawExpr::add_udf_attribute(const ObRawExpr* expr, const ParseNode* node)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(node) || OB_ISNULL(expr)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("the expr is null", K(ret), K(expr), K(node));
 | 
						|
  } else if (expr->is_column_ref_expr()) {
 | 
						|
    const ObColumnRefRawExpr* col_expr = static_cast<const ObColumnRefRawExpr*>(expr);
 | 
						|
    const ObString& column_name = col_expr->get_column_name();
 | 
						|
    if (OB_FAIL(add_udf_attribute_name(column_name))) {
 | 
						|
      LOG_WARN("failed to udf's attribute name", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ObString name(node->str_len_, node->str_value_);
 | 
						|
    if (OB_FAIL(add_udf_attribute_name(name))) {
 | 
						|
      LOG_WARN("failed to udf's attribute name", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObNormalDllUdfRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  UNUSED(expr);
 | 
						|
  UNUSED(check_context);
 | 
						|
  return false;
 | 
						|
}
 | 
						|
 | 
						|
int ObNormalDllUdfRawExpr::assign(const ObNormalDllUdfRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObSysFunRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign sys raw expr");
 | 
						|
    } else {
 | 
						|
      IGNORE_RETURN udf_meta_.assign(other.get_udf_meta());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObNormalDllUdfRawExpr::deep_copy(ObRawExprFactory& expr_factory, const ObNormalDllUdfRawExpr& other,
 | 
						|
    const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObSysFunRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObSysRawExpr failed", K(ret));
 | 
						|
    } else if (use_new_allocator) {
 | 
						|
      if (OB_ISNULL(inner_alloc_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("inner allocator or expr factory is NULL", K(inner_alloc_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.udf_meta_.name_, udf_meta_.name_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.udf_meta_.name_), K(ret));
 | 
						|
      } else if (OB_FAIL(ob_write_string(*inner_alloc_, other.udf_meta_.dl_, udf_meta_.dl_))) {
 | 
						|
        LOG_WARN("fail to write string", K(other.udf_meta_.name_), K(ret));
 | 
						|
      } else {
 | 
						|
        udf_meta_.ret_ = other.udf_meta_.ret_;
 | 
						|
        udf_meta_.tenant_id_ = other.udf_meta_.tenant_id_;
 | 
						|
        udf_meta_.type_ = other.udf_meta_.type_;
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      IGNORE_RETURN udf_meta_.assign(other.get_udf_meta());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObPLSQLCodeSQLErrmRawExpr::assign(const ObPLSQLCodeSQLErrmRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    OZ(ObSysFunRawExpr::assign(other));
 | 
						|
    OX(is_sqlcode_ = other.is_sqlcode_);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObPLSQLCodeSQLErrmRawExpr::deep_copy(ObRawExprFactory& expr_factory, const ObPLSQLCodeSQLErrmRawExpr& other,
 | 
						|
    const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    OZ(ObSysFunRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator));
 | 
						|
    OX(is_sqlcode_ = other.is_sqlcode_);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObExprOperator* ObPLSQLCodeSQLErrmRawExpr::get_op()
 | 
						|
{
 | 
						|
  return NULL;
 | 
						|
}
 | 
						|
 | 
						|
int ObSetOpRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObSetOpRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObSetOpRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bool_ret = false;
 | 
						|
  UNUSED(check_context);
 | 
						|
  if (&expr == this) {
 | 
						|
    bool_ret = true;
 | 
						|
  }
 | 
						|
  return bool_ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSetOpRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObSetOpRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("copy in Base class ObTerminalRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      idx_ = other.idx_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSetOpRawExpr::assign(const ObSetOpRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("copy in Base class ObRawExpr failed", K(ret));
 | 
						|
    } else {
 | 
						|
      idx_ = other.idx_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSetOpRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const char* op_name = "UNKNOWN";
 | 
						|
  switch (get_expr_type()) {
 | 
						|
    case T_OP_UNION:
 | 
						|
      op_name = "UNION";
 | 
						|
      break;
 | 
						|
    case T_OP_INTERSECT:
 | 
						|
      op_name = "INTERSECT";
 | 
						|
      break;
 | 
						|
    case T_OP_EXCEPT:
 | 
						|
      if (lib::is_oracle_mode()) {
 | 
						|
        op_name = "MINUS";
 | 
						|
      } else {
 | 
						|
        op_name = "EXCEPT";
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    default:
 | 
						|
      break;
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
    // do nothing
 | 
						|
  } else if (OB_FAIL(BUF_PRINTF("%s([%ld])", op_name, idx_ + 1))) {
 | 
						|
    LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
  if (OB_SUCCESS == ret && EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObFunMatchAgainst::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  const ObRawExpr* ret = NULL;
 | 
						|
  if (index < 0 || index >= 2) {
 | 
						|
    // do nothing
 | 
						|
  } else if (0 == index) {
 | 
						|
    ret = match_columns_;
 | 
						|
  } else if (1 == index) {
 | 
						|
    ret = search_key_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr*& ObFunMatchAgainst::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  ObRawExpr** ret = &USELESS_POINTER;
 | 
						|
  if (index < 0 || index >= 2) {
 | 
						|
    // do nothing
 | 
						|
  } else if (0 == index) {
 | 
						|
    ret = &match_columns_;
 | 
						|
  } else if (1 == index) {
 | 
						|
    ret = &search_key_;
 | 
						|
  }
 | 
						|
  return *ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObFunMatchAgainst::clear_child()
 | 
						|
{
 | 
						|
  match_columns_ = NULL;
 | 
						|
  search_key_ = NULL;
 | 
						|
  real_column_ = NULL;
 | 
						|
  //  search_tree_ = NULL;
 | 
						|
  mode_flag_ = NATURAL_LANGUAGE_MODE;
 | 
						|
}
 | 
						|
 | 
						|
int ObFunMatchAgainst::replace_expr(
 | 
						|
    const common::ObIArray<ObRawExpr*>& other_exprs, const common::ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, match_columns_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, search_key_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObFunMatchAgainst::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
bool ObFunMatchAgainst::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bret = false;
 | 
						|
  if (expr.is_domain_index_func()) {
 | 
						|
    bret = true;
 | 
						|
    const ObFunMatchAgainst& other_ma = static_cast<const ObFunMatchAgainst&>(expr);
 | 
						|
    for (int64_t i = 0; bret && i < other_ma.get_param_count(); ++i) {
 | 
						|
      if (OB_ISNULL(other_ma.get_param_expr(i)) || OB_ISNULL(get_param_expr(i))) {
 | 
						|
        bret = false;
 | 
						|
      } else {
 | 
						|
        bret = get_param_expr(i)->same_as(*(other_ma.get_param_expr(i)), check_context);
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (bret) {
 | 
						|
      bret = (mode_flag_ == other_ma.mode_flag_);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bret;
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObFunMatchAgainst::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  uint64_t hash_value = seed;
 | 
						|
  for (int64_t i = 0; i < get_param_count(); ++i) {
 | 
						|
    if (NULL != get_param_expr(i)) {
 | 
						|
      hash_value = do_hash(*get_param_expr(i), hash_value);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return hash_value;
 | 
						|
}
 | 
						|
 | 
						|
int ObFunMatchAgainst::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  UNUSED(buf);
 | 
						|
  UNUSED(buf_len);
 | 
						|
  UNUSED(pos);
 | 
						|
  UNUSED(type);
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObSetIterRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObSetIterRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  UNUSED(buf);
 | 
						|
  UNUSED(buf_len);
 | 
						|
  UNUSED(pos);
 | 
						|
  UNUSED(type);
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr*& ObSetIterRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  ObRawExpr** expr = &USELESS_POINTER;
 | 
						|
  if (0 == index) {
 | 
						|
    expr = &left_iter_;
 | 
						|
  } else if (1 == index) {
 | 
						|
    expr = &right_iter_;
 | 
						|
  }
 | 
						|
  return *expr;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObSetIterRawExpr::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  const ObRawExpr* ret = NULL;
 | 
						|
  if (0 == index) {
 | 
						|
    ret = left_iter_;
 | 
						|
  } else if (1 == index) {
 | 
						|
    ret = right_iter_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRowIterRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
int ObRowIterRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  UNUSED(buf);
 | 
						|
  UNUSED(buf_len);
 | 
						|
  UNUSED(pos);
 | 
						|
  UNUSED(type);
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int Bound::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const Bound& other, const uint64_t copy_types, bool use_new_allocator /*= false*/)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  type_ = other.type_;
 | 
						|
  is_preceding_ = other.is_preceding_;
 | 
						|
  is_nmb_literal_ = other.is_nmb_literal_;
 | 
						|
  if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
          expr_factory, other.interval_expr_, interval_expr_, copy_types, use_new_allocator))) {
 | 
						|
    LOG_WARN("failed to copy bound", K(ret));
 | 
						|
  } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                 expr_factory, other.date_unit_expr_, date_unit_expr_, copy_types, use_new_allocator))) {
 | 
						|
    LOG_WARN("failed to copy bound", K(ret));
 | 
						|
  }
 | 
						|
  for (int i = 0; OB_SUCC(ret) && i < BOUND_EXPR_MAX; ++i) {
 | 
						|
    if (OB_ISNULL(other.exprs_[i])) {
 | 
						|
      // do nothing
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(
 | 
						|
                   expr_factory, other.exprs_[i], exprs_[i], copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy bound expr", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int Bound::replace_expr(const common::ObIArray<ObRawExpr*>& other_exprs, const common::ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, interval_expr_))) {
 | 
						|
    LOG_WARN("failed to repalce exprs", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, date_unit_expr_))) {
 | 
						|
    LOG_WARN("failed to repalce exprs", K(ret));
 | 
						|
  }
 | 
						|
  for (int i = 0; OB_SUCC(ret) && i < BOUND_EXPR_MAX; ++i) {
 | 
						|
    if (OB_ISNULL(exprs_[i])) {
 | 
						|
      // do nothing
 | 
						|
    } else if (OB_FAIL(ObTransformUtils::replace_expr(other_exprs, new_exprs, exprs_[i]))) {
 | 
						|
      LOG_WARN("failed to replace bound expr", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObFrame::assign(const ObFrame& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    win_type_ = other.win_type_;
 | 
						|
    is_between_ = other.is_between_;
 | 
						|
    upper_ = other.upper_;
 | 
						|
    lower_ = other.lower_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObWindow::assign(const ObWindow& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(partition_exprs_.assign(other.partition_exprs_))) {
 | 
						|
      LOG_WARN("failed to assign partition exprs", K(ret));
 | 
						|
    } else if (OB_FAIL(order_items_.assign(other.order_items_))) {
 | 
						|
      LOG_WARN("failed to assign order items", K(ret));
 | 
						|
    } else if (OB_FAIL(ObFrame::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign frame", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
void ObWinFunRawExpr::clear_child()
 | 
						|
{
 | 
						|
  func_type_ = T_MAX;
 | 
						|
  is_distinct_ = false;
 | 
						|
  func_params_.reset();
 | 
						|
  partition_exprs_.reset();
 | 
						|
  order_items_.reset();
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::assign(const ObWinFunRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign raw expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObWindow::assign(other))) {
 | 
						|
      LOG_WARN("failed to assign window", K(ret));
 | 
						|
    } else if (OB_FAIL(func_params_.assign(other.func_params_))) {
 | 
						|
      LOG_WARN("failed to assign func params", K(ret));
 | 
						|
    } else {
 | 
						|
      func_type_ = other.func_type_;
 | 
						|
      is_distinct_ = other.is_distinct_;
 | 
						|
      is_ignore_null_ = other.is_ignore_null_;
 | 
						|
      is_from_first_ = other.is_from_first_;
 | 
						|
      agg_expr_ = other.agg_expr_;
 | 
						|
      sort_str_ = other.sort_str_;
 | 
						|
      MEMCPY(upper_.exprs_, other.upper_.exprs_, BOUND_EXPR_MAX);
 | 
						|
      MEMCPY(lower_.exprs_, other.lower_.exprs_, BOUND_EXPR_MAX);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObWinFunRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to assign raw expr", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_expr(expr_factory,
 | 
						|
                   other.agg_expr_,
 | 
						|
                   reinterpret_cast<ObRawExpr*&>(agg_expr_),
 | 
						|
                   COPY_REF_SHARED,
 | 
						|
                   use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy win func", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(
 | 
						|
                   expr_factory, other.func_params_, func_params_, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy func params", K(ret));
 | 
						|
    } else if (OB_FAIL(ObRawExprUtils::copy_exprs(
 | 
						|
                   expr_factory, other.partition_exprs_, partition_exprs_, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy partition exprs", K(ret));
 | 
						|
    } else if (OB_FAIL(ObTransformUtils::deep_copy_order_items(
 | 
						|
                   expr_factory, other.order_items_, order_items_, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy order item", K(ret));
 | 
						|
    } else if (OB_FAIL(upper_.deep_copy(expr_factory, other.upper_, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy upper bound", K(ret));
 | 
						|
    } else if (OB_FAIL(lower_.deep_copy(expr_factory, other.lower_, COPY_REF_DEFAULT, use_new_allocator))) {
 | 
						|
      LOG_WARN("failed to copy lower bound", K(ret));
 | 
						|
    } else {
 | 
						|
      func_type_ = other.func_type_;
 | 
						|
      is_distinct_ = other.is_distinct_;
 | 
						|
      is_from_first_ = other.is_from_first_;
 | 
						|
      is_ignore_null_ = other.is_ignore_null_;
 | 
						|
      win_type_ = other.win_type_;
 | 
						|
      is_between_ = other.is_between_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::replace_param_expr(int64_t index, ObRawExpr* expr)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  int64_t count = agg_expr_ != NULL ? agg_expr_->get_param_count() : 0;
 | 
						|
  count += func_params_.count();
 | 
						|
  if (index < count || index >= count + partition_exprs_.count()) {
 | 
						|
    ret = common::OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("only replace partition expr supported", K(ret), K(index), K(*this));
 | 
						|
  } else {
 | 
						|
    ObRawExpr*& target_expr = partition_exprs_.at(index - count);
 | 
						|
    if (OB_NOT_NULL(expr) && expr != target_expr) {
 | 
						|
      expr->set_orig_expr(target_expr);
 | 
						|
    }
 | 
						|
    target_expr = expr;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(
 | 
						|
                 ObTransformUtils::replace_expr(other_exprs, new_exprs, reinterpret_cast<ObRawExpr*&>(agg_expr_)))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, func_params_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_exprs(other_exprs, new_exprs, partition_exprs_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(ObTransformUtils::replace_expr_for_order_item(other_exprs, new_exprs, order_items_))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else if (OB_FAIL(upper_.replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr in upper bound", K(ret));
 | 
						|
  } else if (OB_FAIL(lower_.replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr in lower bound", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObWinFunRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  bool bret = false;
 | 
						|
  if (expr.is_win_func_expr()) {
 | 
						|
    bret = true;
 | 
						|
    const ObWinFunRawExpr& other_ma = static_cast<const ObWinFunRawExpr&>(expr);
 | 
						|
    if (other_ma.get_func_type() != get_func_type()) {
 | 
						|
      bret = false;
 | 
						|
      // Because name window will construct a window function of count(1) over,
 | 
						|
      // here we need to compare the name of name Windows
 | 
						|
      // If it is in mysql mode, the name is sensitive,
 | 
						|
      // and there is no need to de-duplicate when generating count(1) over.
 | 
						|
      // The oracle mode is true.
 | 
						|
    } else if (0 != other_ma.win_name_.case_compare(win_name_)) {
 | 
						|
      bret = false;
 | 
						|
    } else if (T_WIN_FUN_NTH_VALUE == other_ma.get_func_type() &&
 | 
						|
               (other_ma.is_from_first_ != is_from_first_ || other_ma.is_ignore_null_ != is_ignore_null_)) {
 | 
						|
      bret = false;
 | 
						|
    } else if ((T_WIN_FUN_LAG == other_ma.get_func_type() || T_WIN_FUN_LEAD == other_ma.get_func_type()) &&
 | 
						|
               other_ma.is_ignore_null_ != is_ignore_null_) {
 | 
						|
      bret = false;
 | 
						|
    } else if (agg_expr_ != NULL && other_ma.agg_expr_ != NULL) {
 | 
						|
      bret = agg_expr_->same_as(*(other_ma.agg_expr_), check_context) ? true : false;
 | 
						|
    } else { /* do nothing. */
 | 
						|
    }
 | 
						|
 | 
						|
    if (bret &&
 | 
						|
        (other_ma.get_param_count() != get_param_count() || other_ma.func_params_.count() != func_params_.count() ||
 | 
						|
            other_ma.partition_exprs_.count() != partition_exprs_.count() ||
 | 
						|
            other_ma.order_items_.count() != order_items_.count())) {
 | 
						|
      bret = false;
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; bret && i < other_ma.get_param_count(); ++i) {
 | 
						|
        if (OB_ISNULL(other_ma.get_param_expr(i)) || OB_ISNULL(get_param_expr(i))) {
 | 
						|
          bret = false;
 | 
						|
        } else {
 | 
						|
          bret = get_param_expr(i)->same_as(*(other_ma.get_param_expr(i)), check_context);
 | 
						|
        }
 | 
						|
      }
 | 
						|
      for (int64_t i = 0; bret && i < other_ma.order_items_.count(); ++i) {
 | 
						|
        if (other_ma.order_items_.at(i).order_type_ != order_items_.at(i).order_type_) {
 | 
						|
          bret = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bret;
 | 
						|
}
 | 
						|
 | 
						|
const ObRawExpr* ObWinFunRawExpr::get_param_expr(int64_t index) const
 | 
						|
{
 | 
						|
  int i = 0;
 | 
						|
  ObRawExpr* expr = NULL;
 | 
						|
  for (int64_t j = 0; NULL == expr && agg_expr_ != NULL && j < agg_expr_->get_param_count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      expr = agg_expr_->get_param_expr(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; NULL == expr && j < func_params_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      expr = func_params_.at(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; NULL == expr && j < partition_exprs_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      expr = partition_exprs_.at(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; NULL == expr && j < order_items_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      expr = order_items_.at(j).expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (NULL == expr) {
 | 
						|
    if (upper_.interval_expr_ != NULL) {
 | 
						|
      if (i++ == index) {
 | 
						|
        expr = upper_.interval_expr_;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (NULL == expr) {
 | 
						|
    if (lower_.interval_expr_ != NULL) {
 | 
						|
      if (i++ == index) {
 | 
						|
        expr = lower_.interval_expr_;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; j < 2 && NULL == expr; ++j) {
 | 
						|
    const Bound* bound = 0 == j ? &upper_ : &lower_;
 | 
						|
    for (int64_t k = 0; k < BOUND_EXPR_MAX && NULL == expr; ++k) {
 | 
						|
      if (NULL != bound->exprs_[k]) {
 | 
						|
        if (i++ == index) {
 | 
						|
          expr = bound->exprs_[k];
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return expr;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExpr*& ObWinFunRawExpr::get_param_expr(int64_t index)
 | 
						|
{
 | 
						|
  int i = 0;
 | 
						|
  for (int64_t j = 0; agg_expr_ != NULL && j < agg_expr_->get_param_count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return agg_expr_->get_param_expr(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; j < func_params_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return func_params_.at(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; j < partition_exprs_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return partition_exprs_.at(j);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; j < order_items_.count(); ++j) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return order_items_.at(j).expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (upper_.interval_expr_ != NULL) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return upper_.interval_expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (lower_.interval_expr_ != NULL) {
 | 
						|
    if (i++ == index) {
 | 
						|
      return lower_.interval_expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  for (int64_t j = 0; j < 2; ++j) {
 | 
						|
    Bound* bound = 0 == j ? &upper_ : &lower_;
 | 
						|
    for (int64_t k = 0; k < BOUND_EXPR_MAX; ++k) {
 | 
						|
      if (NULL != bound->exprs_[k]) {
 | 
						|
        if (i++ == index) {
 | 
						|
          return bound->exprs_[k];
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return USELESS_POINTER;
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObWinFunRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  uint64_t hash_value = seed;
 | 
						|
  for (int64_t i = 0; i < get_param_count(); ++i) {
 | 
						|
    if (NULL != get_param_expr(i)) {
 | 
						|
      hash_value = do_hash(*get_param_expr(i), hash_value);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return hash_value;
 | 
						|
}
 | 
						|
 | 
						|
int ObWinFunRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
 | 
						|
  if (agg_expr_ != NULL) {
 | 
						|
    ret = get_agg_expr()->get_name(buf, buf_len, pos, type);
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("%s(", get_type_name(func_type_)))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
    // partition_by && order_by will be printed in ObLogWindowFunction operator
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < func_params_.count(); ++i) {
 | 
						|
      ObRawExpr* func_param;
 | 
						|
      if (0 != i) {
 | 
						|
        if (OB_FAIL(BUF_PRINTF(","))) {
 | 
						|
          LOG_WARN("Failed to add comma", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret) && OB_ISNULL(func_param = func_params_.at(i))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("func param is NULL", K(ret));
 | 
						|
      } else if (OB_SUCC(ret) && OB_FAIL(func_param->get_name(buf, buf_len, pos, type))) {
 | 
						|
        LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCCESS == ret && EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
OB_SERIALIZE_MEMBER(Bound, type_, is_preceding_, is_nmb_literal_);
 | 
						|
 | 
						|
int ObPseudoColumnRawExpr::assign(const ObPseudoColumnRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::assign(other))) {
 | 
						|
      LOG_WARN("fail to assign", K(ret));
 | 
						|
    } else {
 | 
						|
      cte_cycle_value_ = other.cte_cycle_value_;
 | 
						|
      cte_cycle_default_value_ = other.cte_cycle_default_value_;
 | 
						|
      table_id_ = other.table_id_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObPseudoColumnRawExpr::deep_copy(ObRawExprFactory& expr_factory, const ObPseudoColumnRawExpr& other,
 | 
						|
    const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_LIKELY(this != &other)) {
 | 
						|
    if (OB_FAIL(ObTerminalRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
      LOG_WARN("fail to assign", K(ret));
 | 
						|
    } else {
 | 
						|
      cte_cycle_value_ = other.cte_cycle_value_;
 | 
						|
      cte_cycle_default_value_ = other.cte_cycle_default_value_;
 | 
						|
      table_id_ = other.table_id_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObPseudoColumnRawExpr::replace_expr(const ObIArray<ObRawExpr*>& other_exprs, const ObIArray<ObRawExpr*>& new_exprs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObTerminalRawExpr::replace_expr(other_exprs, new_exprs))) {
 | 
						|
    LOG_WARN("failed to replace expr", K(ret));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObPseudoColumnRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context /* = NULL*/) const
 | 
						|
{
 | 
						|
  UNUSED(check_context);
 | 
						|
  return type_ == expr.get_expr_type();
 | 
						|
}
 | 
						|
 | 
						|
int ObPseudoColumnRawExpr::do_visit(ObRawExprVisitor& visitor)
 | 
						|
{
 | 
						|
  return visitor.visit(*this);
 | 
						|
}
 | 
						|
 | 
						|
uint64_t ObPseudoColumnRawExpr::hash_internal(uint64_t seed) const
 | 
						|
{
 | 
						|
  return do_hash(get_expr_type(), seed);
 | 
						|
}
 | 
						|
 | 
						|
int ObPseudoColumnRawExpr::get_name_internal(char* buf, const int64_t buf_len, int64_t& pos, ExplainType type) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(type);
 | 
						|
  switch (get_expr_type()) {
 | 
						|
    case T_LEVEL:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("LEVEL"))) {
 | 
						|
        LOG_WARN("fail to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_CONNECT_BY_ISCYCLE:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("CONNECT_BY_ISCYCLE"))) {
 | 
						|
        LOG_WARN("fail to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_CONNECT_BY_ISLEAF:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("CONNECT_BY_ISLEAF"))) {
 | 
						|
        LOG_WARN("fail to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_CTE_SEARCH_COLUMN:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("T_CTE_SEARCH_COLUMN"))) {
 | 
						|
        LOG_WARN("fail to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_CTE_CYCLE_COLUMN:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("T_CTE_CYCLE_COLUMN"))) {
 | 
						|
        LOG_WARN("fail to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_ORA_ROWSCN:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("ORA_ROWSCN"))) {
 | 
						|
        LOG_WARN("failed to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    case T_PDML_PARTITION_ID:
 | 
						|
      if (OB_FAIL(BUF_PRINTF("PARTITION_ID"))) {
 | 
						|
        LOG_WARN("failed to print", K(ret));
 | 
						|
      }
 | 
						|
      break;
 | 
						|
    default:
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("invalid expr type", K(get_expr_type()));
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && EXPLAIN_EXTENDED == type) {
 | 
						|
    if (OB_FAIL(BUF_PRINTF("("))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF("%p", this))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else if (OB_FAIL(BUF_PRINTF(")"))) {
 | 
						|
      LOG_WARN("fail to BUF_PRINTF", K(ret));
 | 
						|
    } else {
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
ObRawExprPointer::ObRawExprPointer() : expr_group_()
 | 
						|
{}
 | 
						|
 | 
						|
ObRawExprPointer::~ObRawExprPointer()
 | 
						|
{}
 | 
						|
 | 
						|
int ObRawExprPointer::get(ObRawExpr*& expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (expr_group_.count() <= 0 || OB_ISNULL(expr_group_.at(0))) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("expr pointer is not set", K(ret));
 | 
						|
  } else {
 | 
						|
    expr = *(expr_group_.at(0));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExprPointer::set(ObRawExpr* expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < expr_group_.count(); ++i) {
 | 
						|
    if (OB_ISNULL(expr_group_.at(i))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("expr pointer is null", K(ret));
 | 
						|
    } else if (*expr_group_.at(i) == expr) {
 | 
						|
      // not changed
 | 
						|
      break;
 | 
						|
    } else {
 | 
						|
      *expr_group_.at(i) = expr;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObRawExprPointer::add_ref(ObRawExpr** expr)
 | 
						|
{
 | 
						|
  return expr_group_.push_back(expr);
 | 
						|
}
 | 
						|
 | 
						|
int ObMultiSetRawExpr::assign(const ObMultiSetRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObOpRawExpr::assign(other))) {
 | 
						|
    LOG_WARN("failed to assign multiset expr", K(ret));
 | 
						|
  } else {
 | 
						|
    ms_type_ = other.ms_type_;
 | 
						|
    ms_modifier_ = other.ms_modifier_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObMultiSetRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObMultiSetRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObOpRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
    LOG_WARN("failed to deep copy multiset op", K(ret));
 | 
						|
  } else {
 | 
						|
    ms_type_ = other.ms_type_;
 | 
						|
    ms_modifier_ = other.ms_modifier_;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObMultiSetRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  return ObOpRawExpr::same_as(expr, check_context);
 | 
						|
}
 | 
						|
 | 
						|
int ObCollPredRawExpr::assign(const ObCollPredRawExpr& other)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObMultiSetRawExpr::assign(other))) {
 | 
						|
    LOG_WARN("failed to assign multiset expr", K(ret));
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObCollPredRawExpr::deep_copy(
 | 
						|
    ObRawExprFactory& expr_factory, const ObCollPredRawExpr& other, const uint64_t copy_types, bool use_new_allocator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ObMultiSetRawExpr::deep_copy(expr_factory, other, copy_types, use_new_allocator))) {
 | 
						|
    LOG_WARN("failed to deep copy multiset op", K(ret));
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObCollPredRawExpr::same_as(const ObRawExpr& expr, ObExprEqualCheckContext* check_context) const
 | 
						|
{
 | 
						|
  return ObMultiSetRawExpr::same_as(expr, check_context);
 | 
						|
}
 | 
						|
 | 
						|
int ObExprParamCheckContext::init(const ObQueryCtx* context)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(context)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("get unexpected null", K(ret));
 | 
						|
  } else {
 | 
						|
    context_ = context;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObExprParamCheckContext::compare_const(const ObConstRawExpr& left, const ObConstRawExpr& right)
 | 
						|
{
 | 
						|
  int& ret = err_code_;
 | 
						|
  bool bret = false;
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
    /*do nothing*/
 | 
						|
  } else if (OB_ISNULL(context_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("context is null", K(ret));
 | 
						|
  } else if (&left == &right) {
 | 
						|
    bret = true;
 | 
						|
  } else if (left.get_result_type() != right.get_result_type()) {
 | 
						|
    /*do thing*/
 | 
						|
  } else if (left.has_flag(IS_PARAM) && right.has_flag(IS_PARAM)) {
 | 
						|
    bool is_left_calc_item = false;
 | 
						|
    bool is_right_calc_item = false;
 | 
						|
    if (OB_FAIL(is_pre_calc_item(left, is_left_calc_item))) {
 | 
						|
      LOG_WARN("failed to is pre calc item", K(ret));
 | 
						|
    } else if (OB_FAIL(is_pre_calc_item(right, is_right_calc_item))) {
 | 
						|
      LOG_WARN("failed to is pre calc item", K(ret));
 | 
						|
    } else if (is_left_calc_item && is_right_calc_item) {
 | 
						|
      const ObRawExpr* left_param = NULL;
 | 
						|
      const ObRawExpr* right_param = NULL;
 | 
						|
      if (OB_FAIL(get_calc_expr(left.get_value().get_unknown(), left_param))) {
 | 
						|
        LOG_WARN("faield to get calculable expr", K(ret));
 | 
						|
      } else if (OB_FAIL(get_calc_expr(right.get_value().get_unknown(), right_param))) {
 | 
						|
        LOG_WARN("failed to get calculable expr", K(ret));
 | 
						|
      } else if (OB_ISNULL(left_param) || OB_ISNULL(right_param)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("param exprs are null", K(ret), K(left_param), K(right_param));
 | 
						|
      } else {
 | 
						|
        bret = left_param->same_as(*right_param, this);
 | 
						|
      }
 | 
						|
    } else if (is_left_calc_item || is_right_calc_item) {
 | 
						|
      bret = false;
 | 
						|
    } else {
 | 
						|
      ObPCParamEqualInfo info;
 | 
						|
      info.first_param_idx_ = left.get_value().get_unknown();
 | 
						|
      info.second_param_idx_ = right.get_value().get_unknown();
 | 
						|
      if (info.first_param_idx_ == info.second_param_idx_) {
 | 
						|
        bret = true;
 | 
						|
      } else {
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && !bret && i < context_->all_equal_param_constraints_.count(); ++i) {
 | 
						|
          if (context_->all_equal_param_constraints_.at(i).first_param_idx_ == info.first_param_idx_ &&
 | 
						|
              context_->all_equal_param_constraints_.at(i).second_param_idx_ == info.second_param_idx_) {
 | 
						|
            bret = true;
 | 
						|
          } else if (context_->all_equal_param_constraints_.at(i).first_param_idx_ == info.second_param_idx_ &&
 | 
						|
                     context_->all_equal_param_constraints_.at(i).second_param_idx_ == info.first_param_idx_) {
 | 
						|
            bret = true;
 | 
						|
          } else { /*do nothing*/
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (left.has_flag(IS_PARAM) || right.has_flag(IS_PARAM)) {
 | 
						|
    /*do nothing*/
 | 
						|
  } else {
 | 
						|
    bret = left.get_value().is_equal(right.get_value(), CS_TYPE_BINARY);
 | 
						|
  }
 | 
						|
  return bret;
 | 
						|
}
 | 
						|
 | 
						|
int ObExprParamCheckContext::is_pre_calc_item(const ObConstRawExpr& const_expr, bool& is_calc)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t calc_count = 0;
 | 
						|
  is_calc = false;
 | 
						|
  if (OB_ISNULL(context_) || OB_UNLIKELY((calc_count = context_->calculable_items_.count()) < 0 ||
 | 
						|
                                         const_expr.get_expr_type() != T_QUESTIONMARK)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("get unexpected error", K(ret), K(context_), K(const_expr.get_expr_type()), K(calc_count));
 | 
						|
  } else if (const_expr.has_flag(IS_EXEC_PARAM)) {
 | 
						|
    is_calc = true;
 | 
						|
  } else if (calc_count > 0) {
 | 
						|
    int64_t q_idx = const_expr.get_value().get_unknown();
 | 
						|
    int64_t min_calc_index = context_->calculable_items_.at(0).hidden_idx_;
 | 
						|
    if (OB_UNLIKELY(q_idx < 0 || min_calc_index < 0)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("get invalid argument", K(q_idx), K(min_calc_index));
 | 
						|
    } else if (q_idx - min_calc_index >= 0 && q_idx - min_calc_index < calc_count) {
 | 
						|
      is_calc = true;
 | 
						|
    } else { /*do nothing*/
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObExprParamCheckContext::get_calc_expr(const int64_t param_idx, const ObRawExpr*& expr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(context_) || context_->calculable_items_.count() <= 0) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("query context is null", K(ret));
 | 
						|
  } else {
 | 
						|
    int64_t offset = param_idx - context_->calculable_items_.at(0).hidden_idx_;
 | 
						|
    if (offset < 0 || offset >= context_->calculable_items_.count() ||
 | 
						|
        param_idx != context_->calculable_items_.at(offset).hidden_idx_) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("invalid param index", K(ret), K(param_idx), K(offset));
 | 
						|
    } else {
 | 
						|
      expr = context_->calculable_items_.at(offset).expr_;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
}  // namespace sql
 | 
						|
}  // namespace oceanbase
 |