patch 4.0
This commit is contained in:
@ -22,16 +22,21 @@
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/px/ob_dfo.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
namespace oceanbase {
|
||||
namespace oceanbase
|
||||
{
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
using namespace transaction;
|
||||
namespace sql {
|
||||
namespace sql
|
||||
{
|
||||
DEF_TO_STRING(ObRemoteSqlInfo)
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_KV(K_(use_ps), K_(is_batched_stmt), K_(ps_param_cnt), K_(remote_sql));
|
||||
J_KV(K_(use_ps),
|
||||
K_(is_batched_stmt),
|
||||
K_(ps_param_cnt),
|
||||
K_(remote_sql));
|
||||
J_COMMA();
|
||||
J_NAME("ps_params");
|
||||
J_COLON();
|
||||
@ -55,14 +60,12 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator)
|
||||
: allocator_(allocator),
|
||||
tenant_id_(OB_INVALID_ID),
|
||||
tsc_snapshot_timestamp_(0),
|
||||
cur_time_(),
|
||||
merging_frozen_time_(),
|
||||
ts_timeout_us_(0),
|
||||
consistency_level_(INVALID_CONSISTENCY),
|
||||
param_store_((ObWrapperAllocator(&allocator_))),
|
||||
param_store_( (ObWrapperAllocator(&allocator_)) ),
|
||||
datum_param_store_(ObWrapperAllocator(&allocator_)),
|
||||
original_param_cnt_(0),
|
||||
param_frame_capacity_(0),
|
||||
sys_view_bigint_params_(),
|
||||
sql_mode_(SMO_DEFAULT),
|
||||
autoinc_params_(allocator),
|
||||
last_insert_id_session_(0),
|
||||
@ -71,7 +74,7 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator)
|
||||
bind_array_count_(0),
|
||||
bind_array_idx_(0),
|
||||
tenant_schema_version_(OB_INVALID_VERSION),
|
||||
start_trans_param_(),
|
||||
orig_question_mark_cnt_(0),
|
||||
affected_rows_(0),
|
||||
is_affect_found_row_(false),
|
||||
found_rows_(0),
|
||||
@ -87,13 +90,13 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator)
|
||||
row_matched_count_(0),
|
||||
row_duplicated_count_(0),
|
||||
row_deleted_count_(0),
|
||||
warning_count_(0),
|
||||
is_error_ignored_(false),
|
||||
is_select_into_(false),
|
||||
is_result_accurate_(true),
|
||||
foreign_key_checks_(true),
|
||||
unsed_worker_count_since_222rel_(0),
|
||||
exec_ctx_(NULL),
|
||||
table_scan_stat_(),
|
||||
table_row_count_list_(allocator),
|
||||
batched_stmt_param_idxs_(allocator),
|
||||
implicit_cursor_infos_(allocator),
|
||||
@ -101,15 +104,17 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator)
|
||||
is_or_expand_transformed_(false),
|
||||
is_show_seed_(false),
|
||||
is_multi_dml_(false),
|
||||
is_new_engine_(false),
|
||||
remote_sql_info_(),
|
||||
is_large_query_(false)
|
||||
is_large_query_(false),
|
||||
field_array_(nullptr),
|
||||
is_ps_protocol_(false),
|
||||
plan_start_time_(0),
|
||||
is_ps_rewrite_sql_(false)
|
||||
{
|
||||
message_[0] = '\0';
|
||||
}
|
||||
|
||||
ObPhysicalPlanCtx::~ObPhysicalPlanCtx()
|
||||
{}
|
||||
{
|
||||
}
|
||||
|
||||
void ObPhysicalPlanCtx::restore_param_store(const int64_t original_param_cnt)
|
||||
{
|
||||
@ -134,8 +139,8 @@ int ObPhysicalPlanCtx::reserve_param_space(int64_t param_count)
|
||||
return ret;
|
||||
}
|
||||
|
||||
// 1. generate datum_param_store_
|
||||
// 2. generate param frame
|
||||
// 1. 生成datum_param_store_
|
||||
// 2. 生成param frame
|
||||
int ObPhysicalPlanCtx::init_datum_param_store()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -145,26 +150,16 @@ int ObPhysicalPlanCtx::init_datum_param_store()
|
||||
if (OB_FAIL(datum_param_store_.prepare_allocate(param_store_.count()))) {
|
||||
LOG_WARN("fail to prepare allocate", K(ret), K(param_store_.count()));
|
||||
}
|
||||
// by param_store, generate datum_param_store
|
||||
// 通过param_store, 生成datum_param_store
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_store_.count(); i++) {
|
||||
ObDatumObjParam& datum_param = datum_param_store_.at(i);
|
||||
ObObjDatumMapType obj_datum_map = ObDatum::get_obj_datum_map_type(param_store_.at(i).meta_.get_type());
|
||||
if (OBJ_DATUM_NULL == obj_datum_map) {
|
||||
// do nothing
|
||||
} else {
|
||||
uint32_t def_res_len = ObDatum::get_reserved_size(obj_datum_map);
|
||||
if (NULL == (datum_param.datum_.ptr_ = static_cast<char*>(allocator_.alloc(def_res_len)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to alloc memory", K(def_res_len), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(datum_param.from_objparam(param_store_.at(i)))) {
|
||||
LOG_WARN("fail to convert obj param", K(ret), K(param_store_.at(i)));
|
||||
}
|
||||
ObDatumObjParam &datum_param = datum_param_store_.at(i);
|
||||
if (OB_FAIL(datum_param.alloc_datum_reserved_buff(param_store_.at(i).meta_, allocator_))) {
|
||||
LOG_WARN("alloc datum reserved buffer failed", K(ret));
|
||||
} else if (OB_FAIL(datum_param.from_objparam(param_store_.at(i), &allocator_))) {
|
||||
LOG_WARN("fail to convert obj param", K(ret), K(param_store_.at(i)));
|
||||
}
|
||||
}
|
||||
// alocate param frame memory and set param datum
|
||||
// 分配param frame内存, 并设置param datum
|
||||
if (OB_SUCC(ret)) {
|
||||
const int64_t old_size = 0;
|
||||
if (OB_FAIL(extend_param_frame(old_size))) {
|
||||
@ -179,24 +174,24 @@ int ObPhysicalPlanCtx::init_datum_param_store()
|
||||
int ObPhysicalPlanCtx::sync_last_value_local()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAutoincrementService& auto_service = ObAutoincrementService::get_instance();
|
||||
ObIArray<AutoincParam>& autoinc_params = get_autoinc_params();
|
||||
ObAutoincrementService &auto_service = ObAutoincrementService::get_instance();
|
||||
ObIArray<AutoincParam> &autoinc_params = get_autoinc_params();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < autoinc_params.count(); ++i) {
|
||||
AutoincParam& autoinc_param = autoinc_params.at(i);
|
||||
AutoincParam &autoinc_param = autoinc_params.at(i);
|
||||
if (OB_FAIL(auto_service.sync_insert_value_local(autoinc_param))) {
|
||||
LOG_WARN("failed to sync last insert value locally", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::sync_last_value_global()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAutoincrementService& auto_service = ObAutoincrementService::get_instance();
|
||||
ObIArray<AutoincParam>& autoinc_params = get_autoinc_params();
|
||||
ObAutoincrementService &auto_service = ObAutoincrementService::get_instance();
|
||||
ObIArray<AutoincParam> &autoinc_params = get_autoinc_params();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < autoinc_params.count(); ++i) {
|
||||
AutoincParam& autoinc_param = autoinc_params.at(i);
|
||||
AutoincParam &autoinc_param = autoinc_params.at(i);
|
||||
if (OB_FAIL(auto_service.sync_insert_value_global(autoinc_param))) {
|
||||
LOG_WARN("failed to sync last insert value globally", K(ret));
|
||||
}
|
||||
@ -204,8 +199,10 @@ int ObPhysicalPlanCtx::sync_last_value_global()
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPhysicalPlanCtx::set_cur_time(const int64_t& session_val, const ObSQLSessionInfo& session)
|
||||
void ObPhysicalPlanCtx::set_cur_time(const int64_t &session_val, const ObSQLSessionInfo &session)
|
||||
{
|
||||
//如果系统变量设置了timestamp, 则该请求内部是用设置的timestamp
|
||||
//如果没有设置, 则使用当前请求执行时的时间
|
||||
int64_t ts = session.get_local_timestamp();
|
||||
if (0 != ts) {
|
||||
cur_time_.set_timestamp(ts);
|
||||
@ -255,15 +252,12 @@ bool ObPhysicalPlanCtx::is_plain_select_stmt() const
|
||||
return phy_plan_ != NULL && phy_plan_->get_stmt_type() == stmt::T_SELECT && false == phy_plan_->has_for_update();
|
||||
}
|
||||
|
||||
void ObPhysicalPlanCtx::set_phy_plan(const ObPhysicalPlan* phy_plan)
|
||||
void ObPhysicalPlanCtx::set_phy_plan(const ObPhysicalPlan *phy_plan)
|
||||
{
|
||||
phy_plan_ = phy_plan;
|
||||
if (nullptr != phy_plan) {
|
||||
is_new_engine_ = phy_plan->is_new_engine();
|
||||
}
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::assign_batch_stmt_param_idxs(const BatchParamIdxArray& param_idxs)
|
||||
int ObPhysicalPlanCtx::assign_batch_stmt_param_idxs(const BatchParamIdxArray ¶m_idxs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(batched_stmt_param_idxs_.init(param_idxs.count()))) {
|
||||
@ -284,7 +278,10 @@ int ObPhysicalPlanCtx::assign_batch_stmt_param_idxs(const BatchParamIdxArray& pa
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::set_batched_stmt_partition_ids(ObIArray<int64_t>& partition_ids)
|
||||
//设置batch update stmt的partition id和param index的映射关系
|
||||
//batch update stmt中的参数被构造成nested table对象存储在param store上
|
||||
//partition_ids:array index是batch update stmt中参数的下标,值是param对应的partition id
|
||||
int ObPhysicalPlanCtx::set_batched_stmt_partition_ids(ObIArray<int64_t> &partition_ids)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!partition_ids.empty()) {
|
||||
@ -301,7 +298,8 @@ int ObPhysicalPlanCtx::set_batched_stmt_partition_ids(ObIArray<int64_t>& partiti
|
||||
int64_t array_idx = OB_INVALID_INDEX;
|
||||
if (has_exist_in_array(part_ids, part_id, &array_idx)) {
|
||||
if (OB_FAIL(batched_stmt_param_idxs_.at(array_idx).part_param_idxs_.push_back(i))) {
|
||||
LOG_WARN("store param index to partition ids failed", K(ret), K(array_idx), K(part_ids), K(i));
|
||||
LOG_WARN("store param index to partition ids failed",
|
||||
K(ret), K(array_idx), K(part_ids), K(i));
|
||||
}
|
||||
} else {
|
||||
int64_t last_idx = batched_stmt_param_idxs_.count();
|
||||
@ -310,7 +308,8 @@ int ObPhysicalPlanCtx::set_batched_stmt_partition_ids(ObIArray<int64_t>& partiti
|
||||
param_idxs.part_param_idxs_.set_allocator(&allocator_);
|
||||
if (OB_FAIL(batched_stmt_param_idxs_.push_back(param_idxs))) {
|
||||
LOG_WARN("store param idxs failed", K(ret), K(param_idxs));
|
||||
} else if (OB_FAIL(batched_stmt_param_idxs_.at(last_idx).part_param_idxs_.init(partition_ids.count()))) {
|
||||
} else if (OB_FAIL(batched_stmt_param_idxs_.at(last_idx).part_param_idxs_.init(
|
||||
partition_ids.count()))) {
|
||||
LOG_WARN("init param_idxs failed", K(ret), K(last_idx), K(partition_ids));
|
||||
} else if (OB_FAIL(batched_stmt_param_idxs_.at(last_idx).part_param_idxs_.push_back(i))) {
|
||||
LOG_WARN("store param index to param idxs failed", K(ret), K(i));
|
||||
@ -319,7 +318,8 @@ int ObPhysicalPlanCtx::set_batched_stmt_partition_ids(ObIArray<int64_t>& partiti
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("set batched stmt partition ids end", K(ret), K(partition_ids), K_(batched_stmt_param_idxs));
|
||||
LOG_DEBUG("set batched stmt partition ids end", K(ret),
|
||||
K(partition_ids), K_(batched_stmt_param_idxs));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -333,33 +333,35 @@ void ObPhysicalPlanCtx::reset_cursor_info()
|
||||
row_deleted_count_ = 0;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::merge_implicit_cursors(const ObIArray<ObImplicitCursorInfo>& implicit_cursor)
|
||||
int ObPhysicalPlanCtx::merge_implicit_cursors(const ObIArray<ObImplicitCursorInfo> &implicit_cursor)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ARRAY_FOREACH(implicit_cursor, idx)
|
||||
{
|
||||
ARRAY_FOREACH(implicit_cursor, idx) {
|
||||
OZ(merge_implicit_cursor_info(implicit_cursor.at(idx)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::merge_implicit_cursor_info(const ObImplicitCursorInfo& implicit_cursor)
|
||||
int ObPhysicalPlanCtx::merge_implicit_cursor_info(const ObImplicitCursorInfo &implicit_cursor)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(implicit_cursor.stmt_id_ < 0) ||
|
||||
OB_UNLIKELY(implicit_cursor.stmt_id_ >= implicit_cursor_infos_.count())) {
|
||||
if (OB_UNLIKELY(implicit_cursor.stmt_id_ < 0)
|
||||
|| OB_UNLIKELY(implicit_cursor.stmt_id_ >= implicit_cursor_infos_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("implicit cursor is invalid", K(implicit_cursor), K(implicit_cursor_infos_.count()));
|
||||
} else if (OB_FAIL(implicit_cursor_infos_.at(implicit_cursor.stmt_id_).merge_cursor(implicit_cursor))) {
|
||||
LOG_WARN("merge implicit cursor info failed", K(ret), K(implicit_cursor), K(implicit_cursor_infos_));
|
||||
LOG_WARN("implicit cursor is invalid", K(ret),
|
||||
K(implicit_cursor), K(implicit_cursor_infos_.count()));
|
||||
} else if (OB_FAIL(implicit_cursor_infos_.at(implicit_cursor.stmt_id_).
|
||||
merge_cursor(implicit_cursor))) {
|
||||
LOG_WARN("merge implicit cursor info failed", K(ret),
|
||||
K(implicit_cursor), K(implicit_cursor_infos_));
|
||||
}
|
||||
LOG_DEBUG("merge implicit cursor info", K(ret), K(implicit_cursor), K(lbt()));
|
||||
return ret;
|
||||
}
|
||||
|
||||
const ObIArray<int64_t>* ObPhysicalPlanCtx::get_part_param_idxs(int64_t part_id) const
|
||||
const ObIArray<int64_t> *ObPhysicalPlanCtx::get_part_param_idxs(int64_t part_id) const
|
||||
{
|
||||
const ObIArray<int64_t>* part_param_idxs = nullptr;
|
||||
const ObIArray<int64_t> *part_param_idxs = nullptr;
|
||||
for (int64_t i = 0; nullptr == part_param_idxs && i < batched_stmt_param_idxs_.count(); ++i) {
|
||||
if (batched_stmt_param_idxs_.at(i).part_id_ == part_id) {
|
||||
part_param_idxs = &(batched_stmt_param_idxs_.at(i).part_param_idxs_);
|
||||
@ -376,13 +378,14 @@ int ObPhysicalPlanCtx::switch_implicit_cursor()
|
||||
LOG_WARN("cur stmt id is invalid", K(ret), K_(cur_stmt_id));
|
||||
} else if (cur_stmt_id_ >= implicit_cursor_infos_.count()) {
|
||||
ret = OB_ITER_END;
|
||||
LOG_DEBUG("switch implicit cursor iter end", K(ret), K(cur_stmt_id_), K(implicit_cursor_infos_));
|
||||
} else if (implicit_cursor_infos_.at(cur_stmt_id_).stmt_id_ != cur_stmt_id_ &&
|
||||
implicit_cursor_infos_.at(cur_stmt_id_).stmt_id_ >= 0) {
|
||||
LOG_DEBUG("switch implicit cursor iter end", K(ret),
|
||||
K(cur_stmt_id_), K(implicit_cursor_infos_));
|
||||
} else if (implicit_cursor_infos_.at(cur_stmt_id_).stmt_id_ != cur_stmt_id_
|
||||
&& implicit_cursor_infos_.at(cur_stmt_id_).stmt_id_ >= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid implicit cursor", K(ret), K(cur_stmt_id_), K(implicit_cursor_infos_));
|
||||
} else {
|
||||
const ObImplicitCursorInfo& cursor_info = implicit_cursor_infos_.at(cur_stmt_id_);
|
||||
const ObImplicitCursorInfo &cursor_info = implicit_cursor_infos_.at(cur_stmt_id_);
|
||||
set_affected_rows(cursor_info.affected_rows_);
|
||||
set_found_rows(cursor_info.found_rows_);
|
||||
set_row_matched_count(cursor_info.matched_rows_);
|
||||
@ -393,7 +396,7 @@ int ObPhysicalPlanCtx::switch_implicit_cursor()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::extend_datum_param_store(DatumParamStore& ext_datum_store)
|
||||
int ObPhysicalPlanCtx::extend_datum_param_store(DatumParamStore &ext_datum_store)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (ext_datum_store.count() <= 0) {
|
||||
@ -404,9 +407,10 @@ int ObPhysicalPlanCtx::extend_datum_param_store(DatumParamStore& ext_datum_store
|
||||
if (OB_FAIL(datum_param_store_.push_back(ext_datum_store.at(i)))) {
|
||||
LOG_WARN("failed to push back element", K(ret));
|
||||
}
|
||||
} // for end
|
||||
} // for end
|
||||
|
||||
LOG_DEBUG("try to extend param frame", K(ext_datum_store), K(datum_param_store_), K(param_store_));
|
||||
LOG_DEBUG("try to extend param frame",
|
||||
K(ext_datum_store), K(datum_param_store_), K(param_store_));
|
||||
if (OB_FAIL(ret)) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(extend_param_frame(old_size))) {
|
||||
@ -415,59 +419,98 @@ int ObPhysicalPlanCtx::extend_datum_param_store(DatumParamStore& ext_datum_store
|
||||
// transform ext datums to obj params and push back to param_store_
|
||||
for (int i = 0; OB_SUCC(ret) && i < ext_datum_store.count(); i++) {
|
||||
ObObjParam tmp_obj_param;
|
||||
if (OB_FAIL(ext_datum_store.at(i).to_objparam(tmp_obj_param))) {
|
||||
if (OB_FAIL(ext_datum_store.at(i).to_objparam(tmp_obj_param, &allocator_))) {
|
||||
LOG_WARN("failed to transform expr datum to obj param", K(ret));
|
||||
} else if (OB_FAIL(param_store_.push_back(tmp_obj_param))) {
|
||||
LOG_WARN("failed to push back element", K(ret));
|
||||
}
|
||||
} // for end
|
||||
} // for end
|
||||
}
|
||||
LOG_DEBUG("after extended param frame", K(param_store_));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::reserve_param_frame(const int64_t capacity)
|
||||
void ObPhysicalPlanCtx::reset_datum_frame(char *frame, int64_t expr_cnt)
|
||||
{
|
||||
const int64_t datum_eval_info_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
for (int64_t j = 0; j < expr_cnt; ++j) {
|
||||
ObDatum *datum = reinterpret_cast<ObDatum *>(frame + j * datum_eval_info_size);
|
||||
datum->set_null();
|
||||
}
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::reserve_param_frame(const int64_t input_capacity)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (capacity > param_frame_capacity_) {
|
||||
if (input_capacity > param_frame_capacity_) {
|
||||
const int64_t cnt_per_frame = ObExprFrameInfo::EXPR_CNT_PER_FRAME;
|
||||
const int64_t item_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
int64_t frame_cnt = (capacity + cnt_per_frame - 1) / cnt_per_frame;
|
||||
int64_t last = capacity - (frame_cnt - 1) * cnt_per_frame;
|
||||
// align last frame capacity to pow of 2.
|
||||
last = std::min(cnt_per_frame, next_pow2(last));
|
||||
int64_t new_capacity = (frame_cnt - 1) * cnt_per_frame + last;
|
||||
|
||||
const int64_t old_frame_cnt = (param_frame_capacity_ + cnt_per_frame - 1) / cnt_per_frame;
|
||||
int64_t frame_idx = old_frame_cnt;
|
||||
if (param_frame_capacity_ % cnt_per_frame != 0) {
|
||||
frame_idx--;
|
||||
auto calc_frame_cnt = [&](int64_t cap) { return (cap + cnt_per_frame - 1) / cnt_per_frame; };
|
||||
// reserve original param frames first
|
||||
if (param_frame_capacity_ < original_param_cnt_) {
|
||||
if (param_frame_capacity_ > 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("original_param_cnt_ changes after init_datum_param_store",
|
||||
K(ret), K(param_frame_capacity_), K(input_capacity));
|
||||
} else {
|
||||
int64_t total = original_param_cnt_;
|
||||
while (total > 0 && OB_SUCC(ret)) {
|
||||
int64_t cnt = std::min(total, cnt_per_frame);
|
||||
char *frame = static_cast<char *>(allocator_.alloc(item_size * cnt));
|
||||
OV(NULL != frame, OB_ALLOCATE_MEMORY_FAILED);
|
||||
OX(MEMSET(frame, 0, item_size * cnt));
|
||||
OX(reset_datum_frame(frame, cnt));
|
||||
OZ(param_frame_ptrs_.push_back(frame));
|
||||
total -= cnt;
|
||||
}
|
||||
OX(param_frame_capacity_ = original_param_cnt_);
|
||||
}
|
||||
}
|
||||
|
||||
for (; OB_SUCC(ret) && frame_idx < frame_cnt; frame_idx++) {
|
||||
const int64_t cnt = frame_idx + 1 == frame_cnt ? last : cnt_per_frame;
|
||||
char* frame = static_cast<char*>(allocator_.alloc(item_size * cnt));
|
||||
if (OB_ISNULL(frame)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(frame, 0, item_size * cnt);
|
||||
if (frame_idx + 1 == param_frame_ptrs_.count()) {
|
||||
// copy last frame data
|
||||
int64_t old_last_cnt = param_frame_capacity_ % cnt_per_frame;
|
||||
MEMMOVE(frame, param_frame_ptrs_.at(frame_idx), item_size * old_last_cnt);
|
||||
param_frame_ptrs_.at(frame_idx) = frame;
|
||||
} else {
|
||||
if (OB_FAIL(param_frame_ptrs_.push_back(frame))) {
|
||||
LOG_WARN("array push back failed", K(ret));
|
||||
// reserve frames beyond original param frames, align the last frame with pow of 2.
|
||||
int64_t beyond = input_capacity - original_param_cnt_;
|
||||
if (OB_SUCC(ret) && beyond > 0) {
|
||||
|
||||
int64_t frame_cnt = calc_frame_cnt(beyond);
|
||||
int64_t last = beyond - (frame_cnt - 1) * cnt_per_frame;
|
||||
|
||||
// align last frame capacity to pow of 2.
|
||||
last = std::min(cnt_per_frame, next_pow2(last));
|
||||
beyond = (frame_cnt - 1) * cnt_per_frame + last;
|
||||
|
||||
const int64_t original_frame_cnt = original_param_cnt_ > 0
|
||||
? calc_frame_cnt(original_param_cnt_)
|
||||
: 0;
|
||||
|
||||
const int64_t old_capacity = param_frame_capacity_ - original_param_cnt_;
|
||||
const int64_t old_frame_cnt = calc_frame_cnt(old_capacity);
|
||||
int64_t frame_idx = old_frame_cnt;
|
||||
if (old_capacity % cnt_per_frame != 0) {
|
||||
frame_idx--;
|
||||
}
|
||||
|
||||
for (; OB_SUCC(ret) && frame_idx < frame_cnt; frame_idx++) {
|
||||
const int64_t cnt = frame_idx + 1 == frame_cnt ? last : cnt_per_frame;
|
||||
char *frame = static_cast<char *>(allocator_.alloc(item_size * cnt));
|
||||
OV(NULL != frame, OB_ALLOCATE_MEMORY_FAILED);
|
||||
if (OB_SUCC(ret)) {
|
||||
const int64_t array_idx = original_frame_cnt + frame_idx;
|
||||
MEMSET(frame, 0, item_size * cnt);
|
||||
if (array_idx + 1 == param_frame_ptrs_.count()) {
|
||||
// copy last frame data
|
||||
int64_t old_last = (param_frame_capacity_ - original_param_cnt_) % cnt_per_frame;
|
||||
MEMMOVE(frame, param_frame_ptrs_.at(array_idx), item_size * old_last);
|
||||
param_frame_ptrs_.at(array_idx) = frame;
|
||||
} else {
|
||||
OX(reset_datum_frame(frame, cnt));
|
||||
OZ(param_frame_ptrs_.push_back(frame));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
param_frame_capacity_ = new_capacity;
|
||||
if (OB_SUCC(ret)) {
|
||||
param_frame_capacity_ = original_param_cnt_ + beyond;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -483,14 +526,52 @@ int ObPhysicalPlanCtx::extend_param_frame(const int64_t old_size)
|
||||
} else if (OB_FAIL(reserve_param_frame(datum_param_store_.count()))) {
|
||||
LOG_WARN("reserve param frame failed", K(ret));
|
||||
} else {
|
||||
const int64_t datum_eval_info_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
for (int64_t i = old_size; i < datum_param_store_.count(); i++) {
|
||||
const int64_t idx = i / ObExprFrameInfo::EXPR_CNT_PER_FRAME;
|
||||
const int64_t off = (i % ObExprFrameInfo::EXPR_CNT_PER_FRAME) * datum_eval_info_size;
|
||||
ObDatum* datum = reinterpret_cast<ObDatum*>(param_frame_ptrs_.at(idx) + off);
|
||||
ObDatum *datum = nullptr;
|
||||
ObEvalInfo *eval_info = nullptr;
|
||||
get_param_frame_info(i, datum, eval_info);
|
||||
*datum = datum_param_store_.at(i).datum_;
|
||||
ObEvalInfo* eval_info = reinterpret_cast<ObEvalInfo*>(param_frame_ptrs_.at(idx) + off + sizeof(ObDatum));
|
||||
eval_info->evaluated_ = true;
|
||||
eval_info->evaluated_ = false;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_INLINE void ObPhysicalPlanCtx::get_param_frame_info(int64_t param_idx,
|
||||
ObDatum *&datum,
|
||||
ObEvalInfo *&eval_info)
|
||||
{
|
||||
const int64_t cnt_per_frame = ObExprFrameInfo::EXPR_CNT_PER_FRAME;
|
||||
const int64_t datum_eval_info_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
int64_t datum_idx = param_idx < original_param_cnt_ ? param_idx : param_idx - original_param_cnt_;
|
||||
int64_t idx = datum_idx / cnt_per_frame;
|
||||
int64_t off = (datum_idx % cnt_per_frame) * datum_eval_info_size;
|
||||
if (original_param_cnt_ > 0 && param_idx >= original_param_cnt_) {
|
||||
idx += (original_param_cnt_ + cnt_per_frame - 1) / cnt_per_frame;
|
||||
}
|
||||
datum = reinterpret_cast<ObDatum*>(param_frame_ptrs_.at(idx) + off);
|
||||
eval_info = reinterpret_cast<ObEvalInfo *>(param_frame_ptrs_.at(idx) + off + sizeof(ObDatum));
|
||||
}
|
||||
|
||||
int ObPhysicalPlanCtx::replace_batch_param_datum(int64_t cur_group_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < datum_param_store_.count(); i++) {
|
||||
if (datum_param_store_.at(i).meta_.is_ext_sql_array()) {
|
||||
//need to expand the real param to param frame
|
||||
ObDatum *datum = nullptr;
|
||||
ObEvalInfo *eval_info = nullptr;
|
||||
get_param_frame_info(i, datum, eval_info);
|
||||
const ObSqlDatumArray *datum_array = datum_param_store_.at(i).get_sql_datum_array();;
|
||||
if (OB_UNLIKELY(cur_group_id < 0) || OB_UNLIKELY(cur_group_id >= datum_array->count_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid group id", K(ret), K(cur_group_id), K(datum_array->count_));
|
||||
} else {
|
||||
//assign datum ptr to the real param datum
|
||||
*datum = datum_array->data_[cur_group_id];
|
||||
eval_info->evaluated_ = true;
|
||||
LOG_DEBUG("replace batch param datum", K(cur_group_id), KPC(datum), K(datum));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -500,17 +581,22 @@ OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ParamStore empty_param_store;
|
||||
const ParamStore* param_store = NULL;
|
||||
const ObRowIdListArray* row_id_list_array = NULL;
|
||||
const ParamStore *param_store = NULL;
|
||||
const ObRowIdListArray *row_id_list_array = NULL;
|
||||
int64_t cursor_count = implicit_cursor_infos_.count();
|
||||
// used for function sys_view_bigint_param(idx), @note unused anymore
|
||||
ObSEArray<common::ObObj, 1> sys_view_bigint_params_;
|
||||
char message_[1] = {'\0'}; //error msg buffer, unused anymore
|
||||
if (exec_ctx_ != NULL) {
|
||||
row_id_list_array = &exec_ctx_->get_row_id_list_array();
|
||||
}
|
||||
if ((row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL) || is_multi_dml_) {
|
||||
if ((row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL) ||
|
||||
is_multi_dml_) {
|
||||
param_store = &empty_param_store;
|
||||
} else {
|
||||
param_store = ¶m_store_;
|
||||
}
|
||||
//按老的序列方式进行
|
||||
OB_UNIS_ENCODE(tenant_id_);
|
||||
OB_UNIS_ENCODE(tsc_snapshot_timestamp_);
|
||||
OB_UNIS_ENCODE(cur_time_);
|
||||
@ -521,17 +607,25 @@ OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ENCODE(sys_view_bigint_params_);
|
||||
OB_UNIS_ENCODE(sql_mode_);
|
||||
OB_UNIS_ENCODE(autoinc_params_);
|
||||
OB_UNIS_ENCODE(tablet_autoinc_param_);
|
||||
OB_UNIS_ENCODE(last_insert_id_session_);
|
||||
OB_UNIS_ENCODE(message_);
|
||||
OB_UNIS_ENCODE(expr_op_size_);
|
||||
OB_UNIS_ENCODE(is_ignore_stmt_);
|
||||
if (OB_SUCC(ret) && row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL &&
|
||||
!(param_store_.empty())) {
|
||||
if (OB_SUCC(ret) && row_id_list_array != NULL &&
|
||||
!row_id_list_array->empty() &&
|
||||
phy_plan_ != NULL && !(param_store_.empty())) {
|
||||
//按需序列化param store
|
||||
//先序列化param store的槽位
|
||||
//需要序列化param store个数
|
||||
//序列化param index
|
||||
//序列化param value
|
||||
OB_UNIS_ENCODE(param_store_.count());
|
||||
//序列化完才知道被序列化的参数值有多少,所以先跳掉count的位置
|
||||
int64_t seri_param_cnt_pos = pos;
|
||||
int32_t real_param_cnt = 0;
|
||||
pos += serialization::encoded_length_i32(real_param_cnt);
|
||||
const ObIArray<int64_t>* param_idxs = NULL;
|
||||
const ObIArray<int64_t> *param_idxs = NULL;
|
||||
if (phy_plan_->get_row_param_map().count() <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row param map is empty");
|
||||
@ -547,6 +641,7 @@ OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
auto row_id_list = row_id_list_array->at(k);
|
||||
CK(OB_NOT_NULL(row_id_list));
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row_id_list->count(); ++i) {
|
||||
//row_param_map从1开始,因为index=0的位置用来存放公共param了
|
||||
int64_t row_idx = row_id_list->at(i) + 1;
|
||||
if (OB_UNLIKELY(row_idx >= phy_plan_->get_row_param_map().count()) || OB_UNLIKELY(row_idx < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -586,8 +681,8 @@ OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ENCODE(unsed_worker_count_since_222rel_);
|
||||
OB_UNIS_ENCODE(tenant_schema_version_);
|
||||
OB_UNIS_ENCODE(cursor_count);
|
||||
OB_UNIS_ENCODE(is_new_engine_);
|
||||
OB_UNIS_ENCODE(is_large_query_);
|
||||
OB_UNIS_ENCODE(plan_start_time_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -595,17 +690,22 @@ OB_DEF_SERIALIZE_SIZE(ObPhysicalPlanCtx)
|
||||
{
|
||||
int64_t len = 0;
|
||||
ParamStore empty_param_store;
|
||||
const ParamStore* param_store = NULL;
|
||||
const ObRowIdListArray* row_id_list_array = NULL;
|
||||
const ParamStore *param_store = NULL;
|
||||
const ObRowIdListArray *row_id_list_array = NULL;
|
||||
int64_t cursor_count = implicit_cursor_infos_.count();
|
||||
// used for function sys_view_bigint_param(idx), @note unused anymore
|
||||
ObSEArray<common::ObObj, 1> sys_view_bigint_params_;
|
||||
char message_[1] = {'\0'}; //error msg buffer, unused anymore
|
||||
if (exec_ctx_ != NULL) {
|
||||
row_id_list_array = &exec_ctx_->get_row_id_list_array();
|
||||
}
|
||||
if ((row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL) || is_multi_dml_) {
|
||||
if ((row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL) ||
|
||||
is_multi_dml_) {
|
||||
param_store = &empty_param_store;
|
||||
} else {
|
||||
param_store = ¶m_store_;
|
||||
}
|
||||
//按老的序列方式进行
|
||||
OB_UNIS_ADD_LEN(tenant_id_);
|
||||
OB_UNIS_ADD_LEN(tsc_snapshot_timestamp_);
|
||||
OB_UNIS_ADD_LEN(cur_time_);
|
||||
@ -616,16 +716,23 @@ OB_DEF_SERIALIZE_SIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ADD_LEN(sys_view_bigint_params_);
|
||||
OB_UNIS_ADD_LEN(sql_mode_);
|
||||
OB_UNIS_ADD_LEN(autoinc_params_);
|
||||
OB_UNIS_ADD_LEN(tablet_autoinc_param_);
|
||||
OB_UNIS_ADD_LEN(last_insert_id_session_);
|
||||
OB_UNIS_ADD_LEN(message_);
|
||||
OB_UNIS_ADD_LEN(expr_op_size_);
|
||||
OB_UNIS_ADD_LEN(is_ignore_stmt_);
|
||||
if (row_id_list_array != NULL && !row_id_list_array->empty() && phy_plan_ != NULL && !(param_store_.empty())) {
|
||||
if (row_id_list_array != NULL && !row_id_list_array->empty() &&
|
||||
phy_plan_ != NULL && !(param_store_.empty())) {
|
||||
//按需序列化param store
|
||||
//需要序列化param store个数
|
||||
//序列化param index
|
||||
//序列化param value
|
||||
OB_UNIS_ADD_LEN(param_store_.count());
|
||||
//序列化完才知道被序列化的参数值有多少,所以先跳掉count的位置
|
||||
int32_t real_param_cnt = 0;
|
||||
len += serialization::encoded_length_i32(real_param_cnt);
|
||||
if (phy_plan_->get_row_param_map().count() > 0) {
|
||||
const ObIArray<int64_t>& param_idxs = phy_plan_->get_row_param_map().at(0);
|
||||
const ObIArray<int64_t> ¶m_idxs = phy_plan_->get_row_param_map().at(0);
|
||||
for (int64_t i = 0; i < param_idxs.count(); ++i) {
|
||||
OB_UNIS_ADD_LEN(param_idxs.at(i));
|
||||
OB_UNIS_ADD_LEN(param_store_.at(param_idxs.at(i)));
|
||||
@ -635,9 +742,10 @@ OB_DEF_SERIALIZE_SIZE(ObPhysicalPlanCtx)
|
||||
auto row_id_list = row_id_list_array->at(k);
|
||||
if (OB_NOT_NULL(row_id_list)) {
|
||||
for (int64_t i = 0; i < row_id_list->count(); ++i) {
|
||||
//row_param_map从1开始,因为index=0的位置用来存放公共param了
|
||||
int64_t row_idx = row_id_list->at(i) + 1;
|
||||
if (row_idx < phy_plan_->get_row_param_map().count() && row_idx >= 0) {
|
||||
const ObIArray<int64_t>& param_idxs = phy_plan_->get_row_param_map().at(row_idx);
|
||||
const ObIArray<int64_t> ¶m_idxs = phy_plan_->get_row_param_map().at(row_idx);
|
||||
for (int64_t j = 0; j < param_idxs.count(); ++j) {
|
||||
OB_UNIS_ADD_LEN(param_idxs.at(j));
|
||||
OB_UNIS_ADD_LEN(param_store_.at(param_idxs.at(j)));
|
||||
@ -655,8 +763,8 @@ OB_DEF_SERIALIZE_SIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ADD_LEN(unsed_worker_count_since_222rel_);
|
||||
OB_UNIS_ADD_LEN(tenant_schema_version_);
|
||||
OB_UNIS_ADD_LEN(cursor_count);
|
||||
OB_UNIS_ADD_LEN(is_new_engine_);
|
||||
OB_UNIS_ADD_LEN(is_large_query_);
|
||||
OB_UNIS_ADD_LEN(plan_start_time_);
|
||||
return len;
|
||||
}
|
||||
|
||||
@ -668,6 +776,10 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx)
|
||||
int64_t param_idx = OB_INVALID_INDEX;
|
||||
ObObjParam param_obj;
|
||||
int64_t cursor_count = 0;
|
||||
// used for function sys_view_bigint_param(idx), @note unused anymore
|
||||
ObSEArray<common::ObObj, 1> sys_view_bigint_params_;
|
||||
char message_[1] = {'\0'}; //error msg buffer, unused anymore
|
||||
//按老的序列方式进行
|
||||
OB_UNIS_DECODE(tenant_id_);
|
||||
OB_UNIS_DECODE(tsc_snapshot_timestamp_);
|
||||
OB_UNIS_DECODE(cur_time_);
|
||||
@ -678,6 +790,7 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_DECODE(sys_view_bigint_params_);
|
||||
OB_UNIS_DECODE(sql_mode_);
|
||||
OB_UNIS_DECODE(autoinc_params_);
|
||||
OB_UNIS_DECODE(tablet_autoinc_param_);
|
||||
OB_UNIS_DECODE(last_insert_id_session_);
|
||||
OB_UNIS_DECODE(message_);
|
||||
OB_UNIS_DECODE(expr_op_size_);
|
||||
@ -705,16 +818,18 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx)
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && param_cnt <= 0) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_store_.count(); ++i) {
|
||||
const ObObjParam& objpara = param_store_.at(i);
|
||||
ObObj tmp;
|
||||
//param count <= 0不为按需序列化分区相关的param store,直接序列化所有的param
|
||||
//所以需要对param store的所有元素都执行一次深拷贝
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_store_.count(); ++i) {
|
||||
const ObObjParam &objpara = param_store_.at(i);
|
||||
ObObj tmp;
|
||||
if (OB_FAIL(deep_copy_obj(allocator_, objpara, tmp))) {
|
||||
LOG_WARN("deep copy obj failed", K(ret));
|
||||
LOG_WARN("deep copy obj failed", K(ret));
|
||||
} else {
|
||||
param_store_.at(i) = tmp;
|
||||
param_store_.at(i).set_param_meta();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
OB_UNIS_DECODE(foreign_key_checks_);
|
||||
OB_UNIS_DECODE(unsed_worker_count_since_222rel_);
|
||||
@ -725,17 +840,33 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx)
|
||||
LOG_WARN("init implicit cursor infos failed", K(ret));
|
||||
}
|
||||
}
|
||||
OB_UNIS_DECODE(is_new_engine_);
|
||||
OB_UNIS_DECODE(is_large_query_);
|
||||
OB_UNIS_DECODE(plan_start_time_);
|
||||
if (OB_SUCC(ret)) {
|
||||
(void)ObSQLUtils::adjust_time_by_ntp_offset(plan_start_time_);
|
||||
(void)ObSQLUtils::adjust_time_by_ntp_offset(ts_timeout_us_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPhysicalPlanCtx::add_px_dml_row_info(const ObPxDmlRowInfo& dml_row_info)
|
||||
void ObPhysicalPlanCtx::add_px_dml_row_info(const ObPxDmlRowInfo &dml_row_info)
|
||||
{
|
||||
add_row_matched_count(dml_row_info.row_match_count_);
|
||||
add_row_duplicated_count(dml_row_info.row_duplicated_count_);
|
||||
add_row_deleted_count(dml_row_info.row_deleted_count_);
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
int ObPhysicalPlanCtx::get_field(const int64_t idx, ObField &field)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(field_array_) || idx >= field_array_->count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("field array is not init", K(ret), K(field_array_), K(idx));
|
||||
} else {
|
||||
field = field_array_->at(idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} //sql
|
||||
} //oceanbase
|
||||
|
||||
Reference in New Issue
Block a user