[FEAT MERGE] implement values statement
Co-authored-by: wangt1xiuyi <13547954130@163.com>
This commit is contained in:
parent
5a62e3cee9
commit
8015a958d0
93
deps/oblib/src/common/object/ob_object.cpp
vendored
93
deps/oblib/src/common/object/ob_object.cpp
vendored
@ -2193,7 +2193,98 @@ DEFINE_GET_SERIALIZE_SIZE(ObObj)
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER_INHERIT(ObObjParam, ObObj, accuracy_, res_flags_);
|
||||
DEFINE_SERIALIZE(ObObjParam)
|
||||
{
|
||||
int ret = ObObj::serialize(buf, buf_len, pos);
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_ENCODE(accuracy_);
|
||||
OB_UNIS_ENCODE(res_flags_);
|
||||
if (OB_SUCC(ret) && is_ext_sql_array()) {
|
||||
const ObSqlArrayObj *array_obj = reinterpret_cast<const ObSqlArrayObj*>(get_ext());
|
||||
int64_t n = sizeof(ObSqlArrayObj);
|
||||
if (OB_ISNULL(array_obj)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected NULL ptr", K(ret), KP(array_obj));
|
||||
} else if (buf_len - pos < n) {
|
||||
ret = OB_BUF_NOT_ENOUGH;
|
||||
LOG_WARN("serialize buf not enough", K(ret), "remain", buf_len - pos, "needed", n);
|
||||
} else {
|
||||
MEMCPY(buf + pos, array_obj, n);
|
||||
pos += n;
|
||||
if (array_obj->count_ == 0) {
|
||||
/* do nothing */
|
||||
} else if (OB_ISNULL(array_obj->data_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("data is NULL ptr", K(ret), KP(array_obj->data_));
|
||||
} else {
|
||||
n = sizeof(array_obj->data_[0]) * array_obj->count_;
|
||||
if (buf_len - pos < n) {
|
||||
ret = OB_BUF_NOT_ENOUGH;
|
||||
LOG_WARN("serialize buf not enough", K(ret), "remain", buf_len - pos, "needed", n);
|
||||
} else {
|
||||
MEMCPY(buf + pos, static_cast<const void*>(array_obj->data_), n);
|
||||
pos += n;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEFINE_DESERIALIZE(ObObjParam)
|
||||
{
|
||||
int ret = ObObj::deserialize(buf, data_len, pos);
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_DECODE(accuracy_);
|
||||
OB_UNIS_DECODE(res_flags_);
|
||||
if (OB_SUCC(ret) && is_ext_sql_array()) {
|
||||
ObSqlArrayObj *array_obj = NULL;
|
||||
int64_t n = sizeof(ObSqlArrayObj);
|
||||
if (data_len - pos < n) {
|
||||
ret = OB_BUF_NOT_ENOUGH;
|
||||
LOG_WARN("deserialize buf not enough", K(ret), "remain", data_len - pos, "needed", n);
|
||||
} else {
|
||||
array_obj = reinterpret_cast<ObSqlArrayObj *>(const_cast<char *>(buf + pos));
|
||||
pos += n;
|
||||
}
|
||||
if (OB_SUCC(ret) && array_obj->count_ > 0) {
|
||||
n = sizeof(ObObjParam) * array_obj->count_;
|
||||
if (data_len - pos < n) {
|
||||
ret = OB_BUF_NOT_ENOUGH;
|
||||
LOG_WARN("deserialize buf not enough", K(ret), "remain", data_len - pos, "needed", n);
|
||||
} else {
|
||||
array_obj->data_ = reinterpret_cast<ObObjParam *>(const_cast<char *>(buf + pos));
|
||||
pos += n;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
set_extend(reinterpret_cast<int64_t>(array_obj), T_EXT_SQL_ARRAY);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
DEFINE_GET_SERIALIZE_SIZE(ObObjParam)
|
||||
{
|
||||
int64_t len = ObObj::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(accuracy_);
|
||||
OB_UNIS_ADD_LEN(res_flags_);
|
||||
if (is_ext_sql_array()) {
|
||||
len += sizeof(ObSqlArrayObj);
|
||||
const ObSqlArrayObj *array_obj = reinterpret_cast<const ObSqlArrayObj*>(get_ext());
|
||||
if (NULL != array_obj) {
|
||||
len += sizeof(ObSqlArrayObj);
|
||||
if (array_obj->count_ == 0) {
|
||||
/* do nothing */
|
||||
} else if (NULL != array_obj->data_) {
|
||||
len += sizeof(array_obj->data_[0]) * array_obj->count_;
|
||||
}
|
||||
}
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER(ParamFlag, flag_);
|
||||
|
||||
|
32
deps/oblib/src/common/object/ob_object.h
vendored
32
deps/oblib/src/common/object/ob_object.h
vendored
@ -4047,6 +4047,38 @@ public:
|
||||
bool is_zero_fill_;
|
||||
};
|
||||
|
||||
|
||||
struct ObSqlArrayObj
|
||||
{
|
||||
ObSqlArrayObj()
|
||||
: data_(nullptr),
|
||||
count_(0),
|
||||
element_()
|
||||
{
|
||||
}
|
||||
typedef common::ObArrayWrap<common::ObObjParam> DataArray;
|
||||
static ObSqlArrayObj *alloc(common::ObIAllocator &allocator, int64_t count);
|
||||
TO_STRING_KV("data", DataArray(data_, count_), K_(count), K_(element));
|
||||
common::ObObjParam *data_;
|
||||
int64_t count_;
|
||||
common::ObDataType element_;
|
||||
};
|
||||
|
||||
OB_INLINE ObSqlArrayObj *ObSqlArrayObj::alloc(common::ObIAllocator &allocator, int64_t count)
|
||||
{
|
||||
ObSqlArrayObj *array_obj = nullptr;
|
||||
void *array_buf = nullptr;
|
||||
void *data_buf = nullptr;
|
||||
int64_t array_size = sizeof(ObSqlArrayObj) + sizeof(common::ObObjParam) * count;
|
||||
if (OB_NOT_NULL(array_buf = allocator.alloc(array_size))) {
|
||||
array_obj = new (array_buf) ObSqlArrayObj();
|
||||
data_buf = static_cast<char*>(array_buf) + sizeof(ObSqlArrayObj);
|
||||
array_obj->data_ = new (data_buf) common::ObObjParam[count];
|
||||
array_obj->count_ = count;
|
||||
}
|
||||
return array_obj;
|
||||
}
|
||||
|
||||
OB_INLINE int64_t ObObj::get_deep_copy_size() const
|
||||
{
|
||||
int64_t ret = 0;
|
||||
|
@ -2255,6 +2255,8 @@ typedef enum ObItemType
|
||||
T_ALTER_SYSTEM_KILL, // used to support kill session in oracle
|
||||
|
||||
T_LOB_STORAGE_CLAUSE,
|
||||
T_VALUES_TABLE_EXPRESSION,//used to values statement
|
||||
T_VALUES_ROW_LIST,//used to values statement
|
||||
T_MAX //Attention: add a new type before T_MAX
|
||||
} ObItemType;
|
||||
|
||||
|
@ -916,6 +916,7 @@ ob_set_subtarget(ob_sql plan_cache
|
||||
plan_cache/ob_lib_cache_object_manager.cpp
|
||||
plan_cache/ob_lib_cache_node_factory.cpp
|
||||
plan_cache/ob_plan_match_helper.cpp
|
||||
plan_cache/ob_values_table_compression.cpp
|
||||
)
|
||||
|
||||
ob_set_subtarget(ob_sql resolver
|
||||
|
@ -1865,7 +1865,23 @@ int ObStaticEngineCG::generate_spec(ObLogExprValues &op,
|
||||
if (spec.ins_values_batch_opt_) {
|
||||
spec.contain_ab_param_ = true;
|
||||
}
|
||||
if (OB_FAIL(spec.values_.prepare_allocate(op.get_value_exprs().count()))) {
|
||||
bool find_group = false;
|
||||
int64_t group_idx = -1;
|
||||
ObExecContext * exec_ctx = nullptr;
|
||||
if (OB_ISNULL(opt_ctx_) || OB_ISNULL(exec_ctx = opt_ctx_->get_exec_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get exec context", K(ret), KP(opt_ctx_));
|
||||
} else if (exec_ctx->has_dynamic_values_table()) {
|
||||
if (OB_FAIL(op.get_array_param_group_id(group_idx, find_group))) {
|
||||
LOG_WARN("failed to get_array_param_group_id", K(ret));
|
||||
} else if (find_group) {
|
||||
spec.array_group_idx_ = group_idx;
|
||||
spec.contain_ab_param_ = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) { /* do nothing */
|
||||
} else if (OB_FAIL(spec.values_.prepare_allocate(op.get_value_exprs().count()))) {
|
||||
LOG_WARN("init fixed array failed", K(ret), K(op.get_value_exprs().count()));
|
||||
} else if (OB_FAIL(spec.column_names_.prepare_allocate(op.get_value_desc().count()))) {
|
||||
LOG_WARN("init fixed array failed", K(ret), K(op.get_value_desc().count()));
|
||||
|
@ -83,6 +83,7 @@ int ObExprValuesSpec::serialize(char *buf,
|
||||
OB_UNIS_ENCODE(contain_ab_param_);
|
||||
OB_UNIS_ENCODE(ins_values_batch_opt_);
|
||||
OB_UNIS_ENCODE(column_names_);
|
||||
OB_UNIS_ENCODE(array_group_idx_);
|
||||
}
|
||||
}
|
||||
|
||||
@ -107,6 +108,7 @@ OB_DEF_SERIALIZE_SIZE(ObExprValuesSpec)
|
||||
OB_UNIS_ADD_LEN(contain_ab_param_);
|
||||
OB_UNIS_ADD_LEN(ins_values_batch_opt_);
|
||||
OB_UNIS_ADD_LEN(column_names_);
|
||||
OB_UNIS_ADD_LEN(array_group_idx_);
|
||||
return len;
|
||||
}
|
||||
|
||||
@ -120,6 +122,7 @@ OB_DEF_SERIALIZE(ObExprValuesSpec)
|
||||
OB_UNIS_ENCODE(contain_ab_param_);
|
||||
OB_UNIS_ENCODE(ins_values_batch_opt_);
|
||||
OB_UNIS_ENCODE(column_names_);
|
||||
OB_UNIS_ENCODE(array_group_idx_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -133,6 +136,7 @@ OB_DEF_DESERIALIZE(ObExprValuesSpec)
|
||||
OB_UNIS_DECODE(contain_ab_param_);
|
||||
OB_UNIS_DECODE(ins_values_batch_opt_);
|
||||
OB_UNIS_DECODE(column_names_);
|
||||
OB_UNIS_DECODE(array_group_idx_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -180,6 +184,7 @@ int64_t ObExprValuesSpec::get_serialize_size_(const ObPhyOpSeriCtx &seri_ctx) co
|
||||
OB_UNIS_ADD_LEN(contain_ab_param_);
|
||||
OB_UNIS_ADD_LEN(ins_values_batch_opt_);
|
||||
OB_UNIS_ADD_LEN(column_names_);
|
||||
OB_UNIS_ADD_LEN(array_group_idx_);
|
||||
return len;
|
||||
}
|
||||
|
||||
@ -193,7 +198,10 @@ ObExprValuesOp::ObExprValuesOp(ObExecContext &exec_ctx,
|
||||
cm_(CM_NONE),
|
||||
err_log_service_(get_eval_ctx()),
|
||||
err_log_rt_def_(),
|
||||
has_sequence_(false)
|
||||
has_sequence_(false),
|
||||
real_value_cnt_(0),
|
||||
param_idx_(0),
|
||||
param_cnt_(0)
|
||||
{
|
||||
}
|
||||
|
||||
@ -203,14 +211,17 @@ int ObExprValuesOp::inner_open()
|
||||
node_idx_ = 0;
|
||||
const bool is_explicit_cast = false;
|
||||
const int32_t result_flag = 0;
|
||||
if (OB_FAIL(datum_caster_.init(eval_ctx_.exec_ctx_))) {
|
||||
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
if (OB_ISNULL(plan_ctx) || OB_ISNULL(ctx_.get_sql_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected NULL ptr", K(ret), KP(plan_ctx), KP(ctx_.get_sql_ctx()));
|
||||
} else if (OB_FAIL(datum_caster_.init(eval_ctx_.exec_ctx_))) {
|
||||
LOG_WARN("fail to init datum_caster", K(ret));
|
||||
} else if (OB_FAIL(ObSQLUtils::get_default_cast_mode(is_explicit_cast, result_flag,
|
||||
ctx_.get_my_session(), cm_))) {
|
||||
LOG_WARN("fail to get_default_cast_mode", K(ret));
|
||||
} else {
|
||||
// see ObSQLUtils::wrap_column_convert_ctx(), add CM_WARN_ON_FAIL for INSERT IGNORE.
|
||||
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
cm_ = cm_ | CM_COLUMN_CONVERT;
|
||||
if (plan_ctx->is_ignore_stmt() || !is_strict_mode(ctx_.get_my_session()->get_sql_mode())) {
|
||||
// CM_CHARSET_CONVERT_IGNORE_ERR is will give '?' when do string_string convert.
|
||||
@ -226,6 +237,29 @@ int ObExprValuesOp::inner_open()
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected child cnt", K(child_cnt_), K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (MY_SPEC.contain_ab_param_ && !ctx_.has_dynamic_values_table()) {
|
||||
int64_t value_group = MY_SPEC.contain_ab_param_ ?
|
||||
ctx_.get_sql_ctx()->get_batch_params_count() : 1;
|
||||
real_value_cnt_ = MY_SPEC.get_value_count() * value_group;
|
||||
param_idx_ = 0;
|
||||
param_cnt_ = plan_ctx->get_datum_param_store().count();
|
||||
} else if (MY_SPEC.contain_ab_param_ && ctx_.has_dynamic_values_table() &&
|
||||
MY_SPEC.array_group_idx_ >= 0) {
|
||||
if (OB_UNLIKELY(MY_SPEC.array_group_idx_ >= plan_ctx->get_array_param_groups().count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected idx", K(ret), K(MY_SPEC.array_group_idx_));
|
||||
} else {
|
||||
ObArrayParamGroup &array_param_group = plan_ctx->get_array_param_groups().at(MY_SPEC.array_group_idx_);
|
||||
real_value_cnt_ = MY_SPEC.get_value_count() * array_param_group.row_count_;
|
||||
param_cnt_ = array_param_group.column_count_;
|
||||
param_idx_ = array_param_group.start_param_idx_;
|
||||
}
|
||||
} else {
|
||||
real_value_cnt_ = MY_SPEC.get_value_count();
|
||||
}
|
||||
LOG_TRACE("init expr values op", K(real_value_cnt_), K(param_cnt_), K(param_idx_));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -314,11 +348,9 @@ int ObExprValuesOp::get_real_batch_obj_type(ObDatumMeta &src_meta,
|
||||
int64_t group_idx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_SPEC.ins_values_batch_opt_
|
||||
&& T_QUESTIONMARK == src_expr->type_
|
||||
&& (src_expr->frame_idx_
|
||||
< spec_.plan_->get_expr_frame_info().const_frame_.count()
|
||||
+ spec_.plan_->get_expr_frame_info().param_frame_.count())) {
|
||||
if (T_QUESTIONMARK == src_expr->type_ &&
|
||||
src_expr->frame_idx_ < spec_.plan_->get_expr_frame_info().const_frame_.count() +
|
||||
spec_.plan_->get_expr_frame_info().param_frame_.count()) {
|
||||
int64_t param_idx = src_expr->extra_;
|
||||
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
const ObSqlArrayObj *array_obj = NULL;
|
||||
@ -426,20 +458,16 @@ OB_INLINE int ObExprValuesOp::calc_next_row()
|
||||
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
int64_t col_num = MY_SPEC.get_output_count();
|
||||
int64_t col_idx = 0;
|
||||
int64_t batch_cnt = ctx_.get_sql_ctx()->get_batch_params_count();
|
||||
int64_t value_group = (MY_SPEC.contain_ab_param_ ? batch_cnt : 1);
|
||||
int64_t real_value_cnt = MY_SPEC.get_value_count() * value_group;
|
||||
if (node_idx_ == real_value_cnt) {
|
||||
if (node_idx_ == real_value_cnt_) {
|
||||
// there is no values any more
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
bool is_break = false;
|
||||
ObDatum *datum = NULL;
|
||||
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
int64_t group_idx = 0;
|
||||
if (MY_SPEC.contain_ab_param_) {
|
||||
group_idx = node_idx_ / MY_SPEC.get_value_count();
|
||||
if (OB_FAIL(plan_ctx->replace_batch_param_datum(group_idx))) {
|
||||
if (OB_FAIL(plan_ctx->replace_batch_param_datum(group_idx, param_idx_, param_cnt_))) {
|
||||
LOG_WARN("replace batch param datum failed", K(ret), K(group_idx));
|
||||
}
|
||||
}
|
||||
@ -450,7 +478,7 @@ OB_INLINE int ObExprValuesOp::calc_next_row()
|
||||
}
|
||||
}
|
||||
}
|
||||
while (OB_SUCC(ret) && node_idx_ < real_value_cnt && !is_break) {
|
||||
while (OB_SUCC(ret) && node_idx_ < real_value_cnt_ && !is_break) {
|
||||
int64_t real_node_idx = node_idx_ % MY_SPEC.get_value_count();
|
||||
int64_t row_num = real_node_idx / col_num + 1;
|
||||
ObExpr *src_expr = MY_SPEC.values_.at(real_node_idx);
|
||||
|
@ -36,9 +36,10 @@ public:
|
||||
str_values_array_(alloc),
|
||||
err_log_ct_def_(alloc),
|
||||
contain_ab_param_(0),
|
||||
ins_values_batch_opt_(false)
|
||||
ins_values_batch_opt_(false),
|
||||
array_group_idx_(-1)
|
||||
{ }
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K(array_group_idx_));
|
||||
int64_t get_value_count() const { return values_.count(); }
|
||||
int64_t get_is_strict_json_desc_count() const { return is_strict_json_desc_.count(); }
|
||||
virtual int serialize(char *buf,
|
||||
@ -57,6 +58,7 @@ public:
|
||||
ObErrLogCtDef err_log_ct_def_;
|
||||
int64_t contain_ab_param_;
|
||||
bool ins_values_batch_opt_;
|
||||
int64_t array_group_idx_;
|
||||
};
|
||||
|
||||
class ObExprValuesOp : public ObOperator
|
||||
@ -94,6 +96,9 @@ private:
|
||||
ObErrLogService err_log_service_;
|
||||
ObErrLogRtDef err_log_rt_def_;
|
||||
bool has_sequence_;
|
||||
int64_t real_value_cnt_;
|
||||
int64_t param_idx_;
|
||||
int64_t param_cnt_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
|
@ -539,7 +539,8 @@ OB_NOINLINE int ObPreCalcExprFrameInfo::do_batch_stmt_eval(ObExecContext &exec_c
|
||||
} // for end
|
||||
//replace array param to the real param and eval the pre calc expr
|
||||
for (int64_t group_id = 0; OB_SUCC(ret) && group_id < group_cnt; ++group_id) {
|
||||
if (OB_FAIL(exec_ctx.get_physical_plan_ctx()->replace_batch_param_datum(group_id))) {
|
||||
if (OB_FAIL(exec_ctx.get_physical_plan_ctx()->replace_batch_param_datum(group_id, 0,
|
||||
exec_ctx.get_physical_plan_ctx()->get_datum_param_store().count()))) {
|
||||
LOG_WARN("replace batch param frame failed", K(ret));
|
||||
}
|
||||
//params of each group need to clear the datum evaluted flags before calc the pre_calc_expr
|
||||
|
@ -470,6 +470,13 @@ public:
|
||||
void set_cur_rownum(int64_t cur_rownum) { cur_row_num_ = cur_rownum; }
|
||||
int64_t get_cur_rownum() { return cur_row_num_; }
|
||||
bool use_temp_expr_ctx_cache() const { return use_temp_expr_ctx_cache_; }
|
||||
bool has_dynamic_values_table() const {
|
||||
bool ret = false;
|
||||
if (NULL != phy_plan_ctx_) {
|
||||
ret = phy_plan_ctx_->get_array_param_groups().count() > 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
private:
|
||||
int build_temp_expr_ctx(const ObTempExpr &temp_expr, ObTempExprCtx *&temp_expr_ctx);
|
||||
int set_phy_op_ctx_ptr(uint64_t index, void *phy_op);
|
||||
|
@ -77,6 +77,7 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator)
|
||||
tenant_schema_version_(OB_INVALID_VERSION),
|
||||
orig_question_mark_cnt_(0),
|
||||
tenant_srs_version_(OB_INVALID_VERSION),
|
||||
array_param_groups_(),
|
||||
affected_rows_(0),
|
||||
is_affect_found_row_(false),
|
||||
found_rows_(0),
|
||||
@ -555,30 +556,65 @@ OB_INLINE void ObPhysicalPlanCtx::get_param_frame_info(int64_t param_idx,
|
||||
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 ObPhysicalPlanCtx::replace_batch_param_datum(const int64_t cur_group_id,
|
||||
const int64_t start_param,
|
||||
const int64_t param_cnt)
|
||||
{
|
||||
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));
|
||||
const int64_t datum_cnt = datum_param_store_.count();
|
||||
if (OB_UNLIKELY(start_param < 0 || start_param + param_cnt > datum_cnt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected params", K(ret), K(start_param), K(param_cnt), K(datum_cnt));
|
||||
} else {
|
||||
for (int64_t i = start_param; OB_SUCC(ret) && i < start_param + param_cnt; 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;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObArrayParamGroup)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_UNIS_ENCODE(row_count_);
|
||||
OB_UNIS_ENCODE(column_count_);
|
||||
OB_UNIS_ENCODE(start_param_idx_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObArrayParamGroup)
|
||||
{
|
||||
int64_t len = 0;
|
||||
OB_UNIS_ADD_LEN(row_count_);
|
||||
OB_UNIS_ADD_LEN(column_count_);
|
||||
OB_UNIS_ADD_LEN(start_param_idx_);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObArrayParamGroup)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_UNIS_DECODE(row_count_);
|
||||
OB_UNIS_DECODE(column_count_);
|
||||
OB_UNIS_DECODE(start_param_idx_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -686,6 +722,13 @@ OB_DEF_SERIALIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ENCODE(plan_start_time_);
|
||||
OB_UNIS_ENCODE(last_trace_id_);
|
||||
OB_UNIS_ENCODE(tenant_srs_version_);
|
||||
OB_UNIS_ENCODE(original_param_cnt_);
|
||||
OB_UNIS_ENCODE(array_param_groups_.count());
|
||||
if (OB_SUCC(ret) && array_param_groups_.count() > 0) {
|
||||
for (int64_t i = 0 ; OB_SUCC(ret) && i < array_param_groups_.count(); i++) {
|
||||
OB_UNIS_ENCODE(array_param_groups_.at(i));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -769,6 +812,13 @@ OB_DEF_SERIALIZE_SIZE(ObPhysicalPlanCtx)
|
||||
OB_UNIS_ADD_LEN(plan_start_time_);
|
||||
OB_UNIS_ADD_LEN(last_trace_id_);
|
||||
OB_UNIS_ADD_LEN(tenant_srs_version_);
|
||||
OB_UNIS_ADD_LEN(original_param_cnt_);
|
||||
OB_UNIS_ADD_LEN(array_param_groups_.count());
|
||||
if (array_param_groups_.count() > 0) {
|
||||
for (int64_t i = 0 ; i < array_param_groups_.count(); i++) {
|
||||
OB_UNIS_ADD_LEN(array_param_groups_.at(i));
|
||||
}
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
@ -851,6 +901,33 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx)
|
||||
}
|
||||
OB_UNIS_DECODE(last_trace_id_);
|
||||
OB_UNIS_DECODE(tenant_srs_version_);
|
||||
OB_UNIS_DECODE(original_param_cnt_);
|
||||
int64_t array_group_count = 0;
|
||||
OB_UNIS_DECODE(array_group_count);
|
||||
if (OB_SUCC(ret) && array_group_count > 0) {
|
||||
for (int64_t i = 0 ; OB_SUCC(ret) && i < array_group_count; i++) {
|
||||
ObArrayParamGroup array_p_group;
|
||||
OB_UNIS_DECODE(array_p_group);
|
||||
if (OB_FAIL(array_param_groups_.push_back(array_p_group))) {
|
||||
LOG_WARN("failed to push back");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && array_group_count > 0 &&
|
||||
datum_param_store_.count() == 0 &&
|
||||
datum_param_store_.count() != param_store_.count()) {
|
||||
if (OB_FAIL(datum_param_store_.prepare_allocate(param_store_.count()))) {
|
||||
LOG_WARN("fail to prepare allocate", K(ret), K(param_store_.count()));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_store_.count(); 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)));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
@ -72,6 +72,21 @@ struct ObRemoteSqlInfo
|
||||
bool sql_from_pl_;
|
||||
};
|
||||
|
||||
/* refer to a group of array params
|
||||
* values clause using now
|
||||
*/
|
||||
struct ObArrayParamGroup {
|
||||
OB_UNIS_VERSION(1);
|
||||
public:
|
||||
ObArrayParamGroup() : row_count_(0), column_count_(0), start_param_idx_(0) {}
|
||||
ObArrayParamGroup(const int64_t row_cnt, const int64_t param_cnt, const int64_t start_param_idx)
|
||||
: row_count_(row_cnt), column_count_(param_cnt), start_param_idx_(start_param_idx) {}
|
||||
int64_t row_count_;
|
||||
int64_t column_count_;
|
||||
int64_t start_param_idx_; // in param store
|
||||
TO_STRING_KV(K(row_count_), K(column_count_), K(start_param_idx_));
|
||||
};
|
||||
|
||||
class ObPhysicalPlanCtx
|
||||
{
|
||||
OB_UNIS_VERSION(1);
|
||||
@ -435,7 +450,7 @@ public:
|
||||
bool get_is_ps_rewrite_sql() const { return is_ps_rewrite_sql_; }
|
||||
void set_plan_start_time(int64_t t) { plan_start_time_ = t; }
|
||||
int64_t get_plan_start_time() const { return plan_start_time_; }
|
||||
int replace_batch_param_datum(int64_t cur_group_id);
|
||||
int replace_batch_param_datum(const int64_t cur_group_id, const int64_t start_param, const int64_t param_cnt);
|
||||
void set_last_trace_id(const common::ObCurTraceId::TraceId &trace_id)
|
||||
{
|
||||
last_trace_id_ = trace_id;
|
||||
@ -443,7 +458,8 @@ public:
|
||||
const common::ObCurTraceId::TraceId &get_last_trace_id() const { return last_trace_id_; }
|
||||
common::ObCurTraceId::TraceId &get_last_trace_id() { return last_trace_id_; }
|
||||
void set_spm_timeout_timestamp(const int64_t timeout) { spm_ts_timeout_us_ = timeout; }
|
||||
|
||||
const ObIArray<ObArrayParamGroup> &get_array_param_groups() const { return array_param_groups_; }
|
||||
ObIArray<ObArrayParamGroup> &get_array_param_groups() { return array_param_groups_; }
|
||||
private:
|
||||
void reset_datum_frame(char *frame, int64_t expr_cnt);
|
||||
int extend_param_frame(const int64_t old_size);
|
||||
@ -507,6 +523,7 @@ private:
|
||||
int64_t orig_question_mark_cnt_;
|
||||
common::ObCurTraceId::TraceId last_trace_id_;
|
||||
int64_t tenant_srs_version_;
|
||||
ObSEArray<ObArrayParamGroup, 2> array_param_groups_;
|
||||
|
||||
private:
|
||||
/**
|
||||
|
@ -359,6 +359,34 @@ int ObDMLStmtPrinter::print_table(const TableItem *table_item,
|
||||
}
|
||||
break;
|
||||
}
|
||||
case TableItem::VALUES_TABLE: {
|
||||
int64_t column_cnt = stmt_->get_column_size(table_item->table_id_);
|
||||
const ObIArray<ObRawExpr *> &values = table_item->table_values_;
|
||||
if (OB_UNLIKELY(column_cnt <= 0 || values.empty() || values.count() % column_cnt != 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(column_cnt), K(values));
|
||||
} else {
|
||||
DATA_PRINTF("(VALUES ");
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values.count(); ++i) {
|
||||
if (i % column_cnt == 0) {
|
||||
if (i == 0) {
|
||||
DATA_PRINTF("ROW(");
|
||||
} else {
|
||||
DATA_PRINTF("), ROW(");
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
OZ (expr_printer_.do_print(values.at(i), T_FROM_SCOPE));
|
||||
if (OB_SUCC(ret) && (i + 1) % column_cnt != 0) {
|
||||
DATA_PRINTF(", ");
|
||||
}
|
||||
}
|
||||
}
|
||||
DATA_PRINTF("))");
|
||||
DATA_PRINTF(" %.*s", LEN_AND_PTR(table_item->alias_name_));
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unknown table type", K(ret), K(table_item->type_));
|
||||
|
@ -92,6 +92,7 @@
|
||||
#include "sql/monitor/ob_sql_plan.h"
|
||||
#include "sql/optimizer/ob_explain_log_plan.h"
|
||||
#include "sql/dblink/ob_dblink_utils.h"
|
||||
#include "sql/plan_cache/ob_values_table_compression.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -4473,7 +4474,8 @@ int ObSql::after_get_plan(ObPlanCacheCtx &pc_ctx,
|
||||
pctx->get_remote_sql_info().ps_param_cnt_ = static_cast<int32_t>(param_store.count());
|
||||
} else if (phy_plan->temp_sql_can_prepare()
|
||||
&& pc_ctx.neg_param_index_.is_empty()
|
||||
&& !pc_ctx.sql_ctx_.is_batch_params_execute()) {
|
||||
&& !pc_ctx.sql_ctx_.is_batch_params_execute()
|
||||
&& !pc_ctx.exec_ctx_.has_dynamic_values_table()) {
|
||||
//本地是文本协议的SQL,并且缓存在plan中,走ps协议
|
||||
//@TODO:yuchen.wyc 文本协议中如果出现不能参数化的参数,由于param store里的值可能不是参数化对应的值
|
||||
//例如select a, b-1 from t1; 这里会参数化成select a, b-? from t1;但param store里对应的是-1
|
||||
@ -4680,8 +4682,8 @@ OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt,
|
||||
// for batched multi stmt, we only parse and optimize the first statement
|
||||
// only in multi_query, need do this
|
||||
if (!(PC_PS_MODE == mode || PC_PL_MODE == mode) &&
|
||||
context.is_batch_params_execute() &&
|
||||
OB_FAIL(get_first_batched_multi_stmt(pc_ctx, context.multi_stmt_item_, outlined_stmt))) {
|
||||
(context.is_batch_params_execute() || pc_ctx.exec_ctx_.has_dynamic_values_table()) &&
|
||||
OB_FAIL(get_reconstructed_batch_stmt(pc_ctx, outlined_stmt))) {
|
||||
LOG_WARN("failed to get first batched stmt item", K(ret));
|
||||
} else if (OB_FAIL(ObUDRUtils::match_udr_and_refill_ctx(outlined_stmt,
|
||||
context,
|
||||
@ -4893,9 +4895,16 @@ int ObSql::handle_parser(const ObString &sql,
|
||||
LOG_WARN("fail to parser normal query", K(sql), K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
pctx->set_original_param_cnt(pctx->get_param_store().count());
|
||||
if (OB_FAIL(pctx->init_datum_param_store())) {
|
||||
LOG_WARN("fail to init datum param store", K(ret));
|
||||
if (exec_ctx.has_dynamic_values_table()) {
|
||||
if (OB_FAIL(ObValuesTableCompression::resolve_params_for_values_clause(pc_ctx))) {
|
||||
LOG_WARN("failed to resolve batch param store for values table", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
pctx->set_original_param_cnt(pctx->get_param_store().count());
|
||||
if (OB_FAIL(pctx->init_datum_param_store())) {
|
||||
LOG_WARN("fail to init datum param store", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("SQL MEM USAGE", K(parser_mem_usage), K(last_mem_usage));
|
||||
@ -5494,7 +5503,7 @@ int ObSql::check_need_reroute(ObPlanCacheCtx &pc_ctx, ObSQLSessionInfo &session,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSql::get_first_batched_multi_stmt(ObPlanCacheCtx &pc_ctx, ObMultiStmtItem &multi_stmt_item, ObString &stmt_sql)
|
||||
int ObSql::get_reconstructed_batch_stmt(ObPlanCacheCtx &pc_ctx, ObString& stmt_sql)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (pc_ctx.sql_ctx_.is_do_insert_batch_opt()) {
|
||||
@ -5510,14 +5519,18 @@ int ObSql::get_first_batched_multi_stmt(ObPlanCacheCtx &pc_ctx, ObMultiStmtItem
|
||||
LOG_TRACE("print new_reconstruct_sql",
|
||||
K(pc_ctx.fp_result_.pc_key_.name_), K(pc_ctx.insert_batch_opt_info_.new_reconstruct_sql_));
|
||||
}
|
||||
} else if (OB_ISNULL(multi_stmt_item.get_queries())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(multi_stmt_item));
|
||||
} else if (OB_UNLIKELY(multi_stmt_item.get_queries()->empty())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected array count", K(ret));
|
||||
} else {
|
||||
stmt_sql = multi_stmt_item.get_queries()->at(0);
|
||||
} else if (pc_ctx.sql_ctx_.handle_batched_multi_stmt()) {
|
||||
if (OB_ISNULL(pc_ctx.sql_ctx_.multi_stmt_item_.get_queries())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(pc_ctx.sql_ctx_.multi_stmt_item_));
|
||||
} else if (OB_UNLIKELY(pc_ctx.sql_ctx_.multi_stmt_item_.get_queries()->empty())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected array count", K(ret));
|
||||
} else {
|
||||
stmt_sql = pc_ctx.sql_ctx_.multi_stmt_item_.get_queries()->at(0);
|
||||
}
|
||||
} else if (pc_ctx.exec_ctx_.has_dynamic_values_table()) {
|
||||
stmt_sql = pc_ctx.new_raw_sql_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -492,7 +492,7 @@ private:
|
||||
ObExecContext &exec_ctx,
|
||||
ObSchemaGetterGuard &schema_guard,
|
||||
ObSqlTraits &sql_traits);
|
||||
int get_first_batched_multi_stmt(ObPlanCacheCtx &pc_ctx, ObMultiStmtItem& multi_stmt_item, ObString& sql);
|
||||
int get_reconstructed_batch_stmt(ObPlanCacheCtx &pc_ctx, ObString& stmt_sql);
|
||||
static int add_param_to_param_store(const ObObjParam ¶m,
|
||||
ParamStore ¶m_store);
|
||||
|
||||
|
@ -108,7 +108,8 @@ enum PathType
|
||||
FAKE_CTE_TABLE_ACCESS,
|
||||
FUNCTION_TABLE_ACCESS,
|
||||
TEMP_TABLE_ACCESS,
|
||||
JSON_TABLE_ACCESS
|
||||
JSON_TABLE_ACCESS,
|
||||
VALUES_TABLE_ACCESS
|
||||
};
|
||||
|
||||
enum JtColType {
|
||||
@ -478,25 +479,6 @@ enum class ObParamOption {
|
||||
MAX_VALUE
|
||||
};
|
||||
|
||||
struct ObSqlArrayObj
|
||||
{
|
||||
ObSqlArrayObj()
|
||||
: data_(nullptr),
|
||||
count_(0),
|
||||
element_()
|
||||
{
|
||||
}
|
||||
typedef common::ObArrayWrap<common::ObObjParam> DataArray;
|
||||
static ObSqlArrayObj *alloc(common::ObIAllocator &allocator, int64_t count);
|
||||
TO_STRING_KV("data", DataArray(data_, count_),
|
||||
K_(count),
|
||||
K_(element));
|
||||
common::ObObjParam *data_;
|
||||
// ObObjParam *data_1_;
|
||||
int64_t count_;
|
||||
common::ObDataType element_;
|
||||
};
|
||||
|
||||
enum DominateRelation
|
||||
{
|
||||
OBJ_LEFT_DOMINATE = 0,
|
||||
@ -573,21 +555,6 @@ enum ObIDPAbortType
|
||||
IDP_NO_ABORT = 4
|
||||
};
|
||||
|
||||
OB_INLINE ObSqlArrayObj *ObSqlArrayObj::alloc(common::ObIAllocator &allocator, int64_t count)
|
||||
{
|
||||
ObSqlArrayObj *array_obj = nullptr;
|
||||
void *array_buf = nullptr;
|
||||
void *data_buf = nullptr;
|
||||
int64_t array_size = sizeof(ObSqlArrayObj) + sizeof(common::ObObjParam) * count;
|
||||
if (OB_NOT_NULL(array_buf = allocator.alloc(array_size))) {
|
||||
array_obj = new (array_buf) ObSqlArrayObj();
|
||||
data_buf = static_cast<char*>(array_buf) + sizeof(ObSqlArrayObj);
|
||||
array_obj->data_ = new (data_buf) common::ObObjParam[count];
|
||||
array_obj->count_ = count;
|
||||
}
|
||||
return array_obj;
|
||||
}
|
||||
|
||||
struct ObSqlDatumArray
|
||||
{
|
||||
ObSqlDatumArray()
|
||||
|
@ -4679,6 +4679,11 @@ bool oceanbase::sql::Path::is_access_path() const
|
||||
return NULL != parent_ && parent_->get_type() == ACCESS;
|
||||
}
|
||||
|
||||
bool oceanbase::sql::Path::is_values_table_path() const
|
||||
{
|
||||
return NULL != parent_ && parent_->get_type() == VALUES_TABLE_ACCESS;
|
||||
}
|
||||
|
||||
bool oceanbase::sql::Path::is_join_path() const
|
||||
{
|
||||
return NULL != parent_ && parent_->get_type() == JOIN;
|
||||
@ -6937,6 +6942,8 @@ int ObJoinOrder::init_base_join_order(const TableItem *table_item)
|
||||
set_type(FUNCTION_TABLE_ACCESS);
|
||||
} else if (table_item->is_json_table()) {
|
||||
set_type(JSON_TABLE_ACCESS);
|
||||
} else if (table_item->is_values_table()) {
|
||||
set_type(VALUES_TABLE_ACCESS);
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid type of table item", K(table_item->type_), K(ret));
|
||||
@ -6962,6 +6969,8 @@ int ObJoinOrder::generate_base_paths()
|
||||
OPT_TRACE_TITLE("begin generate subplan");
|
||||
ret = generate_normal_subquery_paths();
|
||||
OPT_TRACE_TITLE("end generate subplan");
|
||||
} else if (VALUES_TABLE_ACCESS == get_type()) {
|
||||
ret = generate_values_table_paths();
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected base path type", K(get_type()), K(ret));
|
||||
@ -13483,6 +13492,20 @@ int ObJoinOrder::copy_path(const Path& src_path, Path* &dst_path)
|
||||
dst_path = new_cte_path;
|
||||
}
|
||||
}
|
||||
} else if (src_path.is_values_table_path()) {
|
||||
const ValuesTablePath &values_table_path = static_cast<const ValuesTablePath&>(src_path);
|
||||
ValuesTablePath *new_values_table_path = NULL;
|
||||
if (OB_ISNULL(new_values_table_path = reinterpret_cast<ValuesTablePath*>(allocator_->alloc(sizeof(ValuesTablePath))))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("failed to allocate an ValuesTablePath", K(ret));
|
||||
} else {
|
||||
new_values_table_path = new(new_values_table_path) ValuesTablePath();
|
||||
if (OB_FAIL(new_values_table_path->assign(values_table_path, allocator_))) {
|
||||
LOG_WARN("failed to assign access path", K(ret));
|
||||
} else {
|
||||
dst_path = new_values_table_path;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected path type", K(ret));
|
||||
@ -14456,3 +14479,116 @@ bool ObJoinOrder::virtual_table_index_can_range_scan(uint64_t table_id) {
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ValuesTablePath::assign(const ValuesTablePath &other, common::ObIAllocator *allocator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(Path::assign(other, allocator))) {
|
||||
LOG_WARN("failed to assgin", K(ret));
|
||||
} else {
|
||||
table_id_ = other.table_id_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ValuesTablePath::estimate_cost()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(parent_) || OB_ISNULL(parent_->get_plan())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("get unexpected null", K(parent_), K(ret));
|
||||
} else {
|
||||
ObOptimizerContext &opt_ctx = parent_->get_plan()->get_optimizer_context();
|
||||
cost_ = ObOptEstCost::cost_get_rows(get_path_output_rows(), opt_ctx.get_cost_model_type());
|
||||
op_cost_ = cost_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObJoinOrder::generate_values_table_paths()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ValuesTablePath *values_path = NULL;
|
||||
const ObDMLStmt *stmt = NULL;
|
||||
TableItem *table_item = NULL;
|
||||
if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(allocator_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("get unexpected null", K(get_plan()), K(stmt), K(allocator_), K(ret));
|
||||
} else if (OB_ISNULL(table_item = stmt->get_table_item_by_id(table_id_)) ||
|
||||
OB_UNLIKELY(!table_item->is_values_table() ||
|
||||
stmt->get_column_size(table_id_) == 0 ||
|
||||
table_item->table_values_.empty() ||
|
||||
table_item->table_values_.count() % stmt->get_column_size(table_id_) != 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(table_id_), KPC(stmt), K(ret));
|
||||
} else if (OB_ISNULL(values_path = reinterpret_cast<ValuesTablePath*>(
|
||||
allocator_->alloc(sizeof(ValuesTablePath))))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocate values path", K(ret));
|
||||
} else {
|
||||
values_path = new(values_path) ValuesTablePath();
|
||||
values_path->table_id_ = table_id_;
|
||||
values_path->parent_ = this;
|
||||
ObSEArray<ObExecParamRawExpr *, 4> nl_params;
|
||||
output_rows_ = table_item->table_values_.count() / stmt->get_column_size(table_id_);
|
||||
values_path->strong_sharding_ = get_plan()->get_optimizer_context().get_match_all_sharding();
|
||||
if (OB_FAIL(values_path->set_parallel_and_server_info_for_match_all())) {
|
||||
LOG_WARN("failed set parallel and server info for match all", K(ret));
|
||||
} else if (OB_FAIL(append(values_path->filter_, get_restrict_infos()))) {
|
||||
LOG_WARN("failed to append filter", K(ret));
|
||||
} else if (OB_FAIL(ObOptEstCost::estimate_width_for_table(get_plan()->get_basic_table_metas(),
|
||||
get_plan()->get_selectivity_ctx(),
|
||||
stmt->get_column_items(),
|
||||
table_id_,
|
||||
output_row_size_))) {
|
||||
LOG_WARN("estimate width of row failed", K(table_id_), K(ret));
|
||||
} else if (OB_FAIL(param_values_table_expr(table_item->table_values_,
|
||||
nl_params,
|
||||
values_path->subquery_exprs_))) {
|
||||
LOG_WARN("failed to extract param for values table expr", K(ret));
|
||||
} else if (OB_FAIL(values_path->nl_params_.assign(nl_params))) {
|
||||
LOG_WARN("failed to assign nl params", K(ret));
|
||||
} else if (OB_FAIL(values_path->estimate_cost())) {
|
||||
LOG_WARN("failed to estimate cost", K(ret));
|
||||
} else if (OB_FAIL(values_path->compute_pipeline_info())) {
|
||||
LOG_WARN("failed to compute pipelined path", K(ret));
|
||||
} else if (OB_FAIL(add_path(values_path))) {
|
||||
LOG_WARN("failed to add path", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObJoinOrder::param_values_table_expr(ObIArray<ObRawExpr*> &values_vector,
|
||||
ObIArray<ObExecParamRawExpr *> &nl_params,
|
||||
ObIArray<ObRawExpr*> &subquery_exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObDMLStmt *stmt = NULL;
|
||||
ObLogPlan *plan = get_plan();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values_vector.count(); ++i) {
|
||||
ObSEArray<ObRawExpr *, 1> old_values_exprs;
|
||||
ObSEArray<ObRawExpr *, 1> new_values_exprs;
|
||||
ObSEArray<ObExecParamRawExpr *, 4> tmp_nl_params;
|
||||
if (OB_ISNULL(plan = get_plan()) || OB_ISNULL(stmt = plan->get_stmt())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("NULL pointer error", K(plan), K(ret));
|
||||
} else if (OB_FAIL(old_values_exprs.push_back(values_vector.at(i)))) {
|
||||
LOG_WARN("failed to push back function table expr", K(ret));
|
||||
} else if (OB_FAIL(extract_params_for_inner_path(values_vector.at(i)->get_relation_ids(),
|
||||
tmp_nl_params,
|
||||
subquery_exprs,
|
||||
old_values_exprs,
|
||||
new_values_exprs))) {
|
||||
LOG_WARN("failed to extract params", K(ret));
|
||||
} else if (OB_UNLIKELY(new_values_exprs.count() != 1) || OB_ISNULL(new_values_exprs.at(0))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("new values table expr is invalid", K(ret), K(new_values_exprs));
|
||||
} else if (OB_FAIL(append(nl_params, tmp_nl_params))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else {
|
||||
values_vector.at(i) = new_values_exprs.at(0);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -399,6 +399,7 @@ struct EstimateCostInfo {
|
||||
bool is_access_path() const;
|
||||
bool is_join_path() const;
|
||||
bool is_subquery_path() const;
|
||||
bool is_values_table_path() const;
|
||||
int check_is_base_table(bool &is_base_table);
|
||||
inline const common::ObIArray<OrderItem> &get_ordering() const { return ordering_; }
|
||||
inline common::ObIArray<OrderItem> &get_ordering() { return ordering_; }
|
||||
@ -1175,6 +1176,29 @@ struct EstimateCostInfo {
|
||||
DISALLOW_COPY_AND_ASSIGN(CteTablePath);
|
||||
};
|
||||
|
||||
class ValuesTablePath : public Path
|
||||
{
|
||||
public:
|
||||
ValuesTablePath()
|
||||
: Path(NULL),
|
||||
table_id_(OB_INVALID_ID) {}
|
||||
virtual ~ValuesTablePath() { }
|
||||
int assign(const ValuesTablePath &other, common::ObIAllocator *allocator);
|
||||
virtual int estimate_cost() override;
|
||||
virtual int get_name_internal(char *buf, const int64_t buf_len, int64_t &pos) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(BUF_PRINTF("@values_"))) {
|
||||
} else if (OB_FAIL(BUF_PRINTF("%lu", table_id_))) {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
public:
|
||||
uint64_t table_id_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ValuesTablePath);
|
||||
};
|
||||
|
||||
struct ObRowCountEstTask
|
||||
{
|
||||
ObRowCountEstTask() : est_arg_(NULL)
|
||||
@ -1380,6 +1404,10 @@ struct NullAwareAntiJoinInfo {
|
||||
ObIArray<ObExecParamRawExpr *> &nl_params,
|
||||
ObIArray<ObRawExpr*> &subquery_exprs);
|
||||
|
||||
int param_values_table_expr(ObIArray<ObRawExpr*> &values_vector,
|
||||
ObIArray<ObExecParamRawExpr *> &nl_params,
|
||||
ObIArray<ObRawExpr*> &subquery_exprs);
|
||||
|
||||
int param_json_table_expr(ObRawExpr* &json_table_expr,
|
||||
ObExecParamRawExpr*& nl_params,
|
||||
ObIArray<ObRawExpr*> &subquery_exprs);
|
||||
@ -1735,10 +1763,7 @@ struct NullAwareAntiJoinInfo {
|
||||
int generate_cte_table_paths();
|
||||
int generate_function_table_paths();
|
||||
int generate_json_table_paths();
|
||||
|
||||
int generate_subquery_for_function_table(ObRawExpr *function_table_expr,
|
||||
ObLogicalOperator *&function_table_root);
|
||||
|
||||
int generate_values_table_paths();
|
||||
int generate_temp_table_paths();
|
||||
|
||||
int compute_sharding_info_for_base_paths(ObIArray<AccessPath *> &access_paths);
|
||||
|
@ -213,10 +213,10 @@ int ObLogExprValues::do_re_est_cost(EstimateCostInfo ¶m, double &card, doubl
|
||||
OB_ISNULL(get_stmt())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (get_stmt()->is_insert_stmt()) {
|
||||
} else if (get_stmt()->is_insert_stmt() || is_values_table_) {
|
||||
ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context();
|
||||
const ObInsertStmt *insert_stmt = static_cast<const ObInsertStmt*>(get_stmt());
|
||||
card = insert_stmt->get_insert_row_count();
|
||||
card = get_stmt()->is_insert_stmt() ? static_cast<const ObInsertStmt*>(get_stmt())->get_insert_row_count() :
|
||||
get_values_row_count();
|
||||
op_cost = ObOptEstCost::cost_get_rows(get_card(), opt_ctx.get_cost_model_type());
|
||||
cost = op_cost;
|
||||
} else {
|
||||
@ -247,7 +247,7 @@ int ObLogExprValues::compute_one_row_info()
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (!get_stmt()->is_insert_stmt()) {
|
||||
is_at_most_one_row_ = true;
|
||||
is_at_most_one_row_ = get_values_row_count() <= 1;
|
||||
} else { /*do nothing*/ }
|
||||
|
||||
return ret;
|
||||
@ -268,6 +268,10 @@ int ObLogExprValues::get_op_exprs(ObIArray<ObRawExpr*> &all_exprs)
|
||||
if (OB_FAIL(append(all_exprs, insert_stmt->get_values_desc()))) {
|
||||
LOG_WARN("failed to append exprs", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
} else if (is_values_table_) {
|
||||
if (OB_FAIL(append(all_exprs, value_desc_))) {
|
||||
LOG_WARN("failed to append exprs", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
} else { /*do nothing*/ }
|
||||
return ret;
|
||||
}
|
||||
@ -278,9 +282,9 @@ int ObLogExprValues::allocate_expr_post(ObAllocExprContext &ctx)
|
||||
if (OB_ISNULL(get_stmt())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(get_stmt()), K(ret));
|
||||
} else if (get_stmt()->is_insert_stmt()) {
|
||||
const ObInsertStmt *insert_stmt = static_cast<const ObInsertStmt*>(get_stmt());
|
||||
const ObIArray<ObColumnRefRawExpr*> &values_desc = insert_stmt->get_values_desc();
|
||||
} else if (get_stmt()->is_insert_stmt() || is_values_table_) {
|
||||
const ObIArray<ObColumnRefRawExpr*> &values_desc = get_stmt()->is_insert_stmt() ?
|
||||
static_cast<const ObInsertStmt*>(get_stmt())->get_values_desc() : value_desc_;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values_desc.count(); ++i) {
|
||||
ObColumnRefRawExpr *value_col = values_desc.at(i);
|
||||
if (OB_FAIL(mark_expr_produced(value_col, branch_id_, id_, ctx))) {
|
||||
@ -305,6 +309,7 @@ int ObLogExprValues::allocate_expr_post(ObAllocExprContext &ctx)
|
||||
} else if (OB_FAIL(mark_probably_local_exprs())) {
|
||||
LOG_WARN("failed to mark local exprs", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -423,6 +428,13 @@ int ObLogExprValues::get_plan_item_info(PlanText &plan_text,
|
||||
EXPLAIN_PRINT_INSERT_VALUES(values, output_exprs_.count(), type);
|
||||
END_BUF_PRINT(plan_item.special_predicates_,
|
||||
plan_item.special_predicates_len_);
|
||||
if (OB_SUCC(ret) && is_values_table_) {
|
||||
const ObString &name = get_table_name();
|
||||
BUF_PRINT_OB_STR(name.ptr(),
|
||||
name.length(),
|
||||
plan_item.object_alias_,
|
||||
plan_item.object_alias_len_);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -480,6 +492,33 @@ bool ObLogExprValues::is_ins_values_batch_opt() const
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObLogExprValues::get_array_param_group_id(int64_t &group_id, bool &find)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObExecContext *exec_ctx = NULL;
|
||||
find = false;
|
||||
group_id = -1;
|
||||
if (OB_ISNULL(my_plan_) ||
|
||||
OB_ISNULL(exec_ctx = my_plan_->get_optimizer_context().get_exec_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid argument", K(ret), KP(my_plan_), KP(exec_ctx));
|
||||
} else if (exec_ctx->has_dynamic_values_table()) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !find && i < value_exprs_.count(); i++) {
|
||||
if (OB_ISNULL(value_exprs_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("raw_expr is null", K(ret));
|
||||
} else if (value_exprs_.at(i)->is_const_raw_expr()) {
|
||||
const ObConstRawExpr *const_expr = static_cast<const ObConstRawExpr *>(value_exprs_.at(i));
|
||||
if (const_expr->get_array_param_group_id() >= 0) {
|
||||
find = true;
|
||||
group_id = const_expr->get_array_param_group_id();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObLogExprValues::contain_array_binding_param() const
|
||||
{
|
||||
bool bret = false;
|
||||
|
@ -23,7 +23,9 @@ class ObLogExprValues : public ObLogicalOperator
|
||||
public:
|
||||
ObLogExprValues(ObLogPlan &plan)
|
||||
: ObLogicalOperator(plan),
|
||||
err_log_define_()
|
||||
err_log_define_(),
|
||||
is_values_table_(false),
|
||||
table_name_()
|
||||
|
||||
{}
|
||||
virtual ~ObLogExprValues() {}
|
||||
@ -72,6 +74,14 @@ class ObLogExprValues : public ObLogicalOperator
|
||||
|
||||
virtual int get_plan_item_info(PlanText &plan_text,
|
||||
ObSqlPlanItem &plan_item) override;
|
||||
int get_array_param_group_id(int64_t &group_id, bool &find);
|
||||
void set_is_values_table(bool is_values_table) { is_values_table_ = is_values_table; }
|
||||
int64_t get_values_row_count() const {
|
||||
return is_values_table_ && value_desc_.count() > 0 ? value_exprs_.count() / value_desc_.count() : 1;
|
||||
}
|
||||
inline common::ObString &get_table_name() { return table_name_; }
|
||||
inline const common::ObString &get_table_name() const { return table_name_; }
|
||||
inline void set_table_name(const common::ObString &table_name) { table_name_ = table_name; }
|
||||
private:
|
||||
int construct_array_binding_values();
|
||||
int construct_sequence_values();
|
||||
@ -80,6 +90,9 @@ class ObLogExprValues : public ObLogicalOperator
|
||||
common::ObSEArray<ObColumnRefRawExpr*, 4, common::ModulePageAllocator, true> value_desc_;
|
||||
//add for error_logging
|
||||
ObErrLogDefine err_log_define_;
|
||||
//for values table
|
||||
bool is_values_table_;
|
||||
common::ObString table_name_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObLogExprValues);
|
||||
};
|
||||
|
@ -5411,6 +5411,11 @@ int ObLogPlan::create_plan_tree_from_path(Path *path,
|
||||
if (OB_FAIL(allocate_subquery_path(subquery_path, op))) {
|
||||
LOG_WARN("failed to allocate subquery path", K(ret));
|
||||
} else { /* Do nothing */ }
|
||||
} else if (path->is_values_table_path()) {
|
||||
ValuesTablePath *values_table_path = static_cast<ValuesTablePath *>(path);
|
||||
if (OB_FAIL(allocate_values_table_path(values_table_path, op))) {
|
||||
LOG_WARN("failed to allocate values table path", K(ret));
|
||||
} else { /* Do nothing */ }
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected path type");
|
||||
@ -13902,3 +13907,38 @@ int ObLogPlan::perform_gather_stat_replace(ObLogicalOperator *op)
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLogPlan::allocate_values_table_path(ValuesTablePath *values_table_path,
|
||||
ObLogicalOperator *&out_access_path_op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLogExprValues *values_op = NULL;
|
||||
const TableItem *table_item = NULL;
|
||||
if (OB_ISNULL(values_table_path) || OB_ISNULL(get_stmt()) ||
|
||||
OB_ISNULL(table_item = get_stmt()->get_table_item_by_id(values_table_path->table_id_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(values_table_path), K(get_stmt()), K(ret));
|
||||
} else if (OB_ISNULL(values_op = static_cast<ObLogExprValues*>(get_log_op_factory().
|
||||
allocate(*this, LOG_EXPR_VALUES)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocate values op", K(ret));
|
||||
} else {
|
||||
values_op->set_table_name(table_item->get_table_name());
|
||||
values_op->set_is_values_table(true);
|
||||
ObSEArray<ObColumnRefRawExpr *, 4> values_desc;
|
||||
if (OB_FAIL(values_op->add_values_expr(table_item->table_values_))) {
|
||||
LOG_WARN("failed to add values expr", K(ret));
|
||||
} else if (OB_FAIL(get_stmt()->get_column_exprs(values_table_path->table_id_, values_desc))) {
|
||||
LOG_WARN("failed to get column exprs");
|
||||
} else if (OB_FAIL(values_op->add_values_desc(values_desc))) {
|
||||
LOG_WARN("failed to add values desc", K(ret));
|
||||
} else if (OB_FAIL(append(values_op->get_filter_exprs(), values_table_path->filter_))) {
|
||||
LOG_WARN("failed to append expr", K(ret));
|
||||
} else if (OB_FAIL(values_op->compute_property(values_table_path))) {
|
||||
LOG_WARN("failed to compute propery", K(ret));
|
||||
} else {
|
||||
out_access_path_op = values_op;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -84,6 +84,7 @@ class ObDelUpdStmt;
|
||||
class ObExchangeInfo;
|
||||
class ObDmlTableInfo;
|
||||
struct IndexDMLInfo;
|
||||
class ValuesTablePath;
|
||||
|
||||
struct FunctionTableDependInfo {
|
||||
TO_STRING_KV(
|
||||
@ -564,6 +565,9 @@ public:
|
||||
int allocate_function_table_path(FunctionTablePath *func_table_path,
|
||||
ObLogicalOperator *&out_access_path_op);
|
||||
|
||||
int allocate_values_table_path(ValuesTablePath *values_table_path,
|
||||
ObLogicalOperator *&out_access_path_op);
|
||||
|
||||
int get_has_global_index_filters(const ObIArray<ObRawExpr*> &filter_exprs,
|
||||
const ObIArray<uint64_t> &index_columns,
|
||||
bool &has_index_scan_filter,
|
||||
|
@ -62,7 +62,7 @@ int ObFastParser::parse(const common::ObString &stmt,
|
||||
int64_t &no_param_sql_len,
|
||||
ParamList *¶m_list,
|
||||
int64_t ¶m_num,
|
||||
ObQuestionMarkCtx &ctx,
|
||||
ObFastParserResult &fp_result,
|
||||
int64_t &values_token_pos)
|
||||
{
|
||||
ACTIVE_SESSION_FLAG_SETTER_GUARD(in_parse);
|
||||
@ -72,36 +72,27 @@ int ObFastParser::parse(const common::ObString &stmt,
|
||||
if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num, values_token_pos))) {
|
||||
LOG_WARN("failed to fast parser", K(stmt));
|
||||
} else {
|
||||
ctx = fp.get_question_mark_ctx();
|
||||
fp_result.question_mark_ctx_ = fp.get_question_mark_ctx();
|
||||
fp_result.values_tokens_.set_capacity(fp.get_values_tokens().count());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < fp.get_values_tokens().count(); ++i) {
|
||||
if (OB_FAIL(fp_result.values_tokens_.push_back(ObValuesTokenPos(
|
||||
fp.get_values_tokens().at(i).no_param_sql_pos_,
|
||||
fp.get_values_tokens().at(i).param_idx_)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ObFastParserOracle fp(allocator, fp_ctx);
|
||||
if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num, values_token_pos))) {
|
||||
LOG_WARN("failed to fast parser", K(stmt));
|
||||
} else {
|
||||
ctx = fp.get_question_mark_ctx();
|
||||
fp_result.question_mark_ctx_ = fp.get_question_mark_ctx();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int64_t ObFastParserBase::ObRawSql::strncasecmp(
|
||||
int64_t pos, const char *str, const int64_t size)
|
||||
{
|
||||
// It is not necessary to check if str is nullptr
|
||||
char ch = char_at(pos);
|
||||
for (int64_t i = 0; i < size; i++) {
|
||||
if (ch >= 'A' && ch <= 'Z') {
|
||||
ch += 32;
|
||||
}
|
||||
if (ch != str[i]) {
|
||||
return -1;
|
||||
}
|
||||
ch = char_at(++pos);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
ObFastParserBase::ObFastParserBase(
|
||||
ObIAllocator &allocator,
|
||||
const FPContext fp_ctx) :
|
||||
@ -787,31 +778,6 @@ int ObFastParserBase::process_insert_or_replace(const char *str, const int64_t s
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFastParserBase::process_values(const char *str)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (get_insert_) {
|
||||
if (is_oracle_mode_) {
|
||||
if (CHECK_EQ_STRNCASECMP("alues", 5)) {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(5);
|
||||
}
|
||||
} else {
|
||||
// mysql support: insert ... values / value (xx, ...);
|
||||
if (CHECK_EQ_STRNCASECMP("alues", 5)) {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(5);
|
||||
} else if (CHECK_EQ_STRNCASECMP("alue", 4)) {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(4);
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObFastParserBase::skip_space(ObRawSql &raw_sql)
|
||||
{
|
||||
bool b_ret = false;
|
||||
@ -2606,6 +2572,42 @@ int ObFastParserMysql::process_identifier_begin_with_n()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFastParserMysql::process_values(const char *str)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (get_insert_) {
|
||||
if (!is_oracle_mode_) {
|
||||
// mysql support: insert ... values / value (xx, ...);
|
||||
if (CHECK_EQ_STRNCASECMP("alues", 5)) {
|
||||
if (OB_FAIL(values_tokens_.push_back(ObValuesTokenPos(no_param_sql_len_ +
|
||||
cur_token_begin_pos_ - copy_begin_pos_, param_num_)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(5);
|
||||
}
|
||||
} else if (CHECK_EQ_STRNCASECMP("alue", 4)) {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(4);
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (!is_oracle_mode_) {
|
||||
if (CHECK_EQ_STRNCASECMP("alues", 5)) {
|
||||
if (OB_FAIL(values_tokens_.push_back(ObValuesTokenPos(no_param_sql_len_ +
|
||||
cur_token_begin_pos_ - copy_begin_pos_, param_num_)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
raw_sql_.scan(5);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFastParserMysql::process_identifier(bool is_number_begin)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -2665,7 +2667,7 @@ int ObFastParserMysql::process_identifier(bool is_number_begin)
|
||||
case 'v':
|
||||
case 'V':
|
||||
// 是不是values;
|
||||
process_values("alues");
|
||||
OZ (process_values("alues"));
|
||||
break;
|
||||
|
||||
case 'r': // replace
|
||||
@ -3011,6 +3013,20 @@ int ObFastParserOracle::process_identifier_begin_with_n()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFastParserOracle::process_values(const char *str)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (get_insert_) {
|
||||
if (is_oracle_mode_) {
|
||||
if (CHECK_EQ_STRNCASECMP("alues", 5)) {
|
||||
values_token_pos_ = raw_sql_.cur_pos_;
|
||||
raw_sql_.scan(5);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFastParserOracle::process_identifier(bool is_number_begin)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -3096,7 +3112,7 @@ int ObFastParserOracle::process_identifier(bool is_number_begin)
|
||||
}
|
||||
case 'v':
|
||||
case 'V':
|
||||
process_values("alues");
|
||||
OZ (process_values("alues"));
|
||||
break;
|
||||
default: {
|
||||
break;
|
||||
|
@ -68,10 +68,102 @@ public:
|
||||
int64_t &no_param_sql_len,
|
||||
ParamList *¶m_list,
|
||||
int64_t ¶m_num,
|
||||
ObQuestionMarkCtx &ctx,
|
||||
ObFastParserResult &fp_result,
|
||||
int64_t &values_token_pos);
|
||||
};
|
||||
|
||||
static const char INVALID_CHAR = -1;
|
||||
struct ObRawSql {
|
||||
explicit ObRawSql() :
|
||||
raw_sql_(nullptr), raw_sql_len_(0),
|
||||
cur_pos_(0), search_end_(false) {}
|
||||
|
||||
inline void init(const char *raw_sql, const int64_t len)
|
||||
{
|
||||
cur_pos_ = 0;
|
||||
raw_sql_ = raw_sql;
|
||||
raw_sql_len_ = len;
|
||||
}
|
||||
inline bool is_search_end()
|
||||
{
|
||||
return search_end_ || cur_pos_ > raw_sql_len_ - 1;
|
||||
}
|
||||
inline char peek()
|
||||
{
|
||||
if (cur_pos_ >= raw_sql_len_ - 1) {
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[cur_pos_ + 1];
|
||||
}
|
||||
inline char scan(const int64_t offset)
|
||||
{
|
||||
if (cur_pos_ + offset >= raw_sql_len_) {
|
||||
search_end_ = true;
|
||||
cur_pos_ = raw_sql_len_;
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
cur_pos_ += offset;
|
||||
return raw_sql_[cur_pos_];
|
||||
}
|
||||
inline char scan() { return scan(1); }
|
||||
inline char reverse_scan()
|
||||
{
|
||||
if (cur_pos_ <= 0 || cur_pos_ >= raw_sql_len_ + 1) {
|
||||
search_end_ = true;
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[--cur_pos_];
|
||||
}
|
||||
inline char char_at(int64_t idx)
|
||||
{
|
||||
if (idx < 0 || idx >= raw_sql_len_) {
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[idx];
|
||||
}
|
||||
inline const char *ptr(const int64_t pos)
|
||||
{
|
||||
if (pos < 0 || pos >= raw_sql_len_) {
|
||||
return nullptr;
|
||||
}
|
||||
return &(raw_sql_[pos]);
|
||||
}
|
||||
inline int64_t strncasecmp(int64_t pos, const char *str, const int64_t size)
|
||||
{
|
||||
// It is not necessary to check if str is nullptr
|
||||
char ch = char_at(pos);
|
||||
for (int64_t i = 0; i < size; i++) {
|
||||
if (ch >= 'A' && ch <= 'Z') {
|
||||
ch += 32;
|
||||
}
|
||||
if (ch != str[i]) {
|
||||
return -1;
|
||||
}
|
||||
ch = char_at(++pos);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
inline int64_t strncasecmp(const char *str, const int64_t size)
|
||||
{
|
||||
return strncasecmp(cur_pos_, str, size);
|
||||
}
|
||||
// Access a character (no bounds check)
|
||||
// char operator[] (const int64_t idx) const { return raw_sql_[idx]; }
|
||||
// For debug
|
||||
common::ObString to_string() const
|
||||
{
|
||||
if (OB_UNLIKELY(nullptr == raw_sql_ || 0 == raw_sql_len_)) {
|
||||
return common::ObString(0, nullptr);
|
||||
}
|
||||
return common::ObString(raw_sql_len_, raw_sql_);
|
||||
}
|
||||
|
||||
const char *raw_sql_;
|
||||
int64_t raw_sql_len_;
|
||||
int64_t cur_pos_;
|
||||
bool search_end_;
|
||||
};
|
||||
|
||||
class ObFastParserBase
|
||||
{
|
||||
public:
|
||||
@ -145,87 +237,9 @@ protected:
|
||||
// There are too many such branches, making our code look very ugly. therefore, we use a special
|
||||
// value of -1. when the allowed length is exceeded, peek, scan, reverse_scan, char_at return -1
|
||||
// this will not affect any correctness issues, and will make the code look better
|
||||
static const char INVALID_CHAR = -1;
|
||||
static const int64_t PARSER_NODE_SIZE = sizeof(ParseNode);
|
||||
static const int64_t FIEXED_PARAM_NODE_SIZE = PARSER_NODE_SIZE + sizeof(ParamList);
|
||||
|
||||
struct ObRawSql {
|
||||
explicit ObRawSql() :
|
||||
raw_sql_(nullptr), raw_sql_len_(0),
|
||||
cur_pos_(0), search_end_(false) {}
|
||||
|
||||
inline void init(const char *raw_sql, const int64_t len)
|
||||
{
|
||||
cur_pos_ = 0;
|
||||
raw_sql_ = raw_sql;
|
||||
raw_sql_len_ = len;
|
||||
}
|
||||
inline bool is_search_end()
|
||||
{
|
||||
return search_end_ || cur_pos_ > raw_sql_len_ - 1;
|
||||
}
|
||||
inline char peek()
|
||||
{
|
||||
if (cur_pos_ >= raw_sql_len_ - 1) {
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[cur_pos_ + 1];
|
||||
}
|
||||
inline char scan(const int64_t offset)
|
||||
{
|
||||
if (cur_pos_ + offset >= raw_sql_len_) {
|
||||
search_end_ = true;
|
||||
cur_pos_ = raw_sql_len_;
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
cur_pos_ += offset;
|
||||
return raw_sql_[cur_pos_];
|
||||
}
|
||||
inline char scan() { return scan(1); }
|
||||
inline char reverse_scan()
|
||||
{
|
||||
if (cur_pos_ <= 0 || cur_pos_ >= raw_sql_len_ + 1) {
|
||||
search_end_ = true;
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[--cur_pos_];
|
||||
}
|
||||
inline char char_at(int64_t idx)
|
||||
{
|
||||
if (idx < 0 || idx >= raw_sql_len_) {
|
||||
return INVALID_CHAR;
|
||||
}
|
||||
return raw_sql_[idx];
|
||||
}
|
||||
inline const char *ptr(const int64_t pos)
|
||||
{
|
||||
if (pos < 0 || pos >= raw_sql_len_) {
|
||||
return nullptr;
|
||||
}
|
||||
return &(raw_sql_[pos]);
|
||||
}
|
||||
int64_t strncasecmp(int64_t pos, const char *str, const int64_t size);
|
||||
inline int64_t strncasecmp(const char *str, const int64_t size)
|
||||
{
|
||||
return strncasecmp(cur_pos_, str, size);
|
||||
}
|
||||
// Access a character (no bounds check)
|
||||
// char operator[] (const int64_t idx) const { return raw_sql_[idx]; }
|
||||
// For debug
|
||||
common::ObString to_string() const
|
||||
{
|
||||
if (OB_UNLIKELY(nullptr == raw_sql_ || 0 == raw_sql_len_)) {
|
||||
return common::ObString(0, nullptr);
|
||||
}
|
||||
return common::ObString(raw_sql_len_, raw_sql_);
|
||||
}
|
||||
|
||||
const char *raw_sql_;
|
||||
int64_t raw_sql_len_;
|
||||
int64_t cur_pos_;
|
||||
bool search_end_;
|
||||
};
|
||||
|
||||
protected:
|
||||
/**
|
||||
* Check whether it is a specify character and whether it is a multi byte specify
|
||||
@ -572,7 +586,7 @@ protected:
|
||||
int process_interval();
|
||||
|
||||
int process_insert_or_replace(const char *str, const int64_t size);
|
||||
int process_values(const char *str);
|
||||
virtual int process_values(const char *str) = 0;
|
||||
|
||||
int get_one_insert_row_str(ObRawSql &raw_sql,
|
||||
ObString &str,
|
||||
@ -640,7 +654,8 @@ public:
|
||||
static_cast<ProcessIdfFunc>(&ObFastParserMysql::process_identifier));
|
||||
}
|
||||
~ObFastParserMysql() {}
|
||||
|
||||
virtual int process_values(const char *str) override;
|
||||
ObIArray<ObValuesTokenPos> &get_values_tokens() { return values_tokens_; }
|
||||
private:
|
||||
ObSQLMode sql_mode_;
|
||||
int parse_next_token();
|
||||
@ -656,8 +671,8 @@ private:
|
||||
int process_string(const char quote);
|
||||
int process_zero_identifier();
|
||||
int process_identifier_begin_with_n();
|
||||
|
||||
private:
|
||||
ObSEArray<ObValuesTokenPos, 4> values_tokens_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObFastParserMysql);
|
||||
};
|
||||
|
||||
@ -675,7 +690,7 @@ public:
|
||||
static_cast<ProcessIdfFunc>(&ObFastParserOracle::process_identifier));
|
||||
}
|
||||
~ObFastParserOracle() {}
|
||||
|
||||
virtual int process_values(const char *str) override;
|
||||
private:
|
||||
int parse_next_token();
|
||||
int process_identifier(bool is_number_begin);
|
||||
|
@ -111,7 +111,7 @@ public:
|
||||
PreParseResult &res);
|
||||
|
||||
enum State {
|
||||
S_START,
|
||||
S_START = 0,
|
||||
S_COMMENT,
|
||||
S_C_COMMENT,
|
||||
S_NORMAL,
|
||||
@ -146,7 +146,12 @@ enum State {
|
||||
S_EXPLAIN_EXTENDED,
|
||||
S_EXPLAIN_EXTENDED_NOADDR,
|
||||
S_EXPLAIN_PARTITIONS,
|
||||
|
||||
S_SELECT,
|
||||
S_INSERT,
|
||||
S_DELETE,
|
||||
S_VALUES,
|
||||
S_TABLE,
|
||||
S_INTO,
|
||||
// add new states above me
|
||||
S_MAX
|
||||
};
|
||||
@ -161,6 +166,11 @@ enum State {
|
||||
bool *is_call_procedure = NULL);
|
||||
static bool is_explain_stmt(const common::ObString &stmt,
|
||||
const char *&p_normal_start);
|
||||
static bool is_comment(const char *&p,
|
||||
const char *&p_end,
|
||||
State &save_state,
|
||||
State &state,
|
||||
State error_state);
|
||||
private:
|
||||
static int scan_trace_id(const char *ptr,
|
||||
int64_t len,
|
||||
@ -180,11 +190,6 @@ private:
|
||||
State &save_state,
|
||||
State &state,
|
||||
State next_state);
|
||||
static bool is_comment(const char *&p,
|
||||
const char *&p_end,
|
||||
State &save_state,
|
||||
State &state,
|
||||
State error_state);
|
||||
// types and constants
|
||||
private:
|
||||
// disallow copy
|
||||
|
@ -1135,4 +1135,55 @@ do {\
|
||||
} \
|
||||
} while(0); \
|
||||
|
||||
#define malloc_select_values_stmt(node, result, values_node, order_by_node, limit_node)\
|
||||
do {\
|
||||
/*gen select list*/\
|
||||
ParseNode *star_node = NULL;\
|
||||
malloc_terminal_node(star_node, result->malloc_pool_, T_STAR);\
|
||||
dup_string_to_node(star_node, result->malloc_pool_, "*");\
|
||||
ParseNode *project_node = NULL;\
|
||||
malloc_non_terminal_node(project_node, result->malloc_pool_, T_PROJECT_STRING, 1, star_node);\
|
||||
ParseNode *project_list_node = NULL;\
|
||||
merge_nodes(project_list_node, result, T_PROJECT_LIST, project_node);\
|
||||
/*gen from list*/\
|
||||
ParseNode *alias_node = NULL;\
|
||||
malloc_non_terminal_node(alias_node, result->malloc_pool_, T_ALIAS, 2, values_node, NULL);\
|
||||
ParseNode *from_list = NULL;\
|
||||
merge_nodes(from_list, result, T_FROM_LIST, alias_node);\
|
||||
/*malloc select node*/\
|
||||
malloc_select_node(node, result->malloc_pool_);\
|
||||
node->children_[PARSE_SELECT_SELECT] = project_list_node;\
|
||||
node->children_[PARSE_SELECT_FROM] = from_list;\
|
||||
node->children_[PARSE_SELECT_ORDER] = order_by_node;\
|
||||
node->children_[PARSE_SELECT_LIMIT] = limit_node;\
|
||||
} while(0);\
|
||||
|
||||
#define refine_insert_values_table(node)\
|
||||
do {\
|
||||
if (node != NULL && node->type_ == T_SELECT && node->children_[PARSE_SELECT_FROM] != NULL &&\
|
||||
node->children_[PARSE_SELECT_FROM]->type_ == T_FROM_LIST &&\
|
||||
node->children_[PARSE_SELECT_FROM]->num_child_ == 1 &&\
|
||||
node->children_[PARSE_SELECT_FROM]->children_ != NULL &&\
|
||||
node->children_[PARSE_SELECT_FROM]->children_[0] != NULL &&\
|
||||
node->children_[PARSE_SELECT_FROM]->children_[0]->type_ == T_ALIAS) {\
|
||||
ParseNode *alias_node = node->children_[PARSE_SELECT_FROM]->children_[0];\
|
||||
if (alias_node->children_ != NULL &&\
|
||||
alias_node->num_child_ == 2 && \
|
||||
alias_node->children_[0] != NULL &&\
|
||||
alias_node->children_[0]->type_ == T_VALUES_TABLE_EXPRESSION) {\
|
||||
ParseNode *values_table_node = alias_node->children_[0];\
|
||||
if (values_table_node->children_ != NULL &&\
|
||||
values_table_node->num_child_ == 1 && \
|
||||
values_table_node->children_[0] != NULL &&\
|
||||
values_table_node->children_[0]->type_ == T_VALUES_ROW_LIST) {\
|
||||
values_table_node->children_[0]->type_ = T_VALUE_LIST;\
|
||||
if (node->children_[PARSE_SELECT_ORDER] == NULL &&\
|
||||
node->children_[PARSE_SELECT_LIMIT] == NULL) {\
|
||||
node = values_table_node->children_[0]; \
|
||||
}\
|
||||
}\
|
||||
}\
|
||||
}\
|
||||
} while(0);\
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_PARSER_SQL_PARSER_BASE_H_ */
|
||||
|
@ -405,7 +405,7 @@ END_P SET_VAR DELIMITER
|
||||
%type <node> opt_query_expression_option_list query_expression_option_list query_expression_option opt_distinct opt_distinct_or_all opt_separator projection
|
||||
%type <node> from_list table_references table_reference table_factor normal_relation_factor dot_relation_factor relation_factor
|
||||
%type <node> relation_factor_in_hint relation_factor_in_hint_list relation_factor_in_pq_hint opt_relation_factor_in_hint_list relation_factor_in_use_join_hint_list
|
||||
%type <node> relation_factor_in_leading_hint_list joined_table tbl_name table_subquery
|
||||
%type <node> relation_factor_in_leading_hint_list joined_table tbl_name table_subquery table_subquery_alias
|
||||
%type <node> relation_factor_with_star relation_with_star_list opt_with_star
|
||||
%type <node> index_hint_type key_or_index index_hint_scope index_element index_list opt_index_list
|
||||
%type <node> add_key_or_index_opt add_key_or_index add_unique_key_opt add_unique_key add_constraint_uniq_key_opt add_constraint_uniq_key add_constraint_pri_key_opt add_constraint_pri_key add_primary_key_opt add_primary_key add_spatial_index_opt add_spatial_index
|
||||
@ -514,6 +514,7 @@ END_P SET_VAR DELIMITER
|
||||
%type <node> json_table_expr mock_jt_on_error_on_empty jt_column_list json_table_column_def
|
||||
%type <node> json_table_ordinality_column_def json_table_exists_column_def json_table_value_column_def json_table_nested_column_def
|
||||
%type <node> opt_value_on_empty_or_error_or_mismatch opt_on_mismatch
|
||||
%type <node> table_values_caluse table_values_caluse_with_order_by_and_limit values_row_list row_value
|
||||
|
||||
%start sql_stmt
|
||||
%%
|
||||
@ -7879,6 +7880,7 @@ dml_table_name values_clause
|
||||
{
|
||||
ParseNode *into_node = NULL;
|
||||
malloc_non_terminal_node(into_node, result->malloc_pool_, T_INSERT_INTO_CLAUSE, 2, $1, NULL);
|
||||
refine_insert_values_table($2);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_SINGLE_TABLE_INSERT, 4,
|
||||
into_node, /*insert_into_clause*/
|
||||
$2, /*values_clause*/
|
||||
@ -7889,6 +7891,7 @@ dml_table_name values_clause
|
||||
{
|
||||
ParseNode *into_node = NULL;
|
||||
malloc_non_terminal_node(into_node, result->malloc_pool_, T_INSERT_INTO_CLAUSE, 2, $1, NULL);
|
||||
refine_insert_values_table($4);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_SINGLE_TABLE_INSERT, 4,
|
||||
into_node, /*insert_into_clause*/
|
||||
$4, /*values_clause*/
|
||||
@ -7901,6 +7904,7 @@ dml_table_name values_clause
|
||||
ParseNode *column_list = NULL;
|
||||
merge_nodes(column_list, result, T_COLUMN_LIST, $3);
|
||||
malloc_non_terminal_node(into_node, result->malloc_pool_, T_INSERT_INTO_CLAUSE, 2, $1, column_list);
|
||||
refine_insert_values_table($5);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_SINGLE_TABLE_INSERT, 4,
|
||||
into_node, /*insert_into_clause*/
|
||||
$5, /*values_clause*/
|
||||
@ -8309,6 +8313,14 @@ no_table_select
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
| table_values_caluse
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
| table_values_caluse_with_order_by_and_limit
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
;
|
||||
|
||||
select_clause_set_with_order_and_limit:
|
||||
@ -8366,6 +8378,10 @@ no_table_select
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
| table_values_caluse
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
;
|
||||
|
||||
select_clause_set_left:
|
||||
@ -10590,28 +10606,40 @@ BLOCK
|
||||
|
||||
|
||||
table_subquery:
|
||||
select_with_parens relation_name
|
||||
select_with_parens table_subquery_alias
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 2, $1, $2);
|
||||
$$->sql_str_off_ = @1.first_column;
|
||||
}
|
||||
| select_with_parens AS relation_name
|
||||
| select_with_parens AS table_subquery_alias
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 2, $1, $3);
|
||||
$$->sql_str_off_ = @1.first_column;
|
||||
}
|
||||
| select_with_parens use_flashback relation_name
|
||||
| select_with_parens use_flashback table_subquery_alias
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $3, NULL, NULL, NULL, $2);
|
||||
$$->sql_str_off_ = @1.first_column;
|
||||
}
|
||||
| select_with_parens use_flashback AS relation_name
|
||||
| select_with_parens use_flashback AS table_subquery_alias
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $4, NULL, NULL, NULL, $2);
|
||||
$$->sql_str_off_ = @1.first_column;
|
||||
}
|
||||
;
|
||||
|
||||
table_subquery_alias:
|
||||
relation_name
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
| relation_name '(' alias_name_list ')'
|
||||
{
|
||||
ParseNode *col_alias_list = NULL;
|
||||
merge_nodes(col_alias_list, result, T_COLUMN_LIST, $3);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, col_alias_list);
|
||||
}
|
||||
|
||||
/*
|
||||
table PARTITION(list of partitions)
|
||||
*/
|
||||
@ -11302,6 +11330,60 @@ column_name opt_asc_desc
|
||||
}
|
||||
;*/
|
||||
|
||||
/*****************************************************************************
|
||||
*
|
||||
* Values statement clause (Mysql8.0 values statement grammer implement)
|
||||
* https://dev.mysql.com/doc/refman/8.0/en/values.html
|
||||
*
|
||||
*****************************************************************************/
|
||||
table_values_caluse:
|
||||
VALUES values_row_list
|
||||
{
|
||||
ParseNode *values_node = NULL;
|
||||
ParseNode *value_list = NULL;
|
||||
merge_nodes(value_list, result, T_VALUES_ROW_LIST, $2);
|
||||
malloc_non_terminal_node(values_node, result->malloc_pool_, T_VALUES_TABLE_EXPRESSION, 1, value_list);
|
||||
malloc_select_values_stmt($$, result, values_node, NULL, NULL);
|
||||
}
|
||||
;
|
||||
|
||||
table_values_caluse_with_order_by_and_limit:
|
||||
VALUES values_row_list order_by
|
||||
{
|
||||
ParseNode *values_node = NULL;
|
||||
ParseNode *value_list = NULL;
|
||||
merge_nodes(value_list, result, T_VALUES_ROW_LIST, $2);
|
||||
malloc_non_terminal_node(values_node, result->malloc_pool_, T_VALUES_TABLE_EXPRESSION, 1, value_list);
|
||||
malloc_select_values_stmt($$, result, values_node, $3, NULL);
|
||||
}
|
||||
| VALUES values_row_list opt_order_by limit_clause
|
||||
{
|
||||
ParseNode *values_node = NULL;
|
||||
ParseNode *value_list = NULL;
|
||||
merge_nodes(value_list, result, T_VALUES_ROW_LIST, $2);
|
||||
malloc_non_terminal_node(values_node, result->malloc_pool_, T_VALUES_TABLE_EXPRESSION, 1, value_list);
|
||||
malloc_select_values_stmt($$, result, values_node, $3, $4);
|
||||
}
|
||||
;
|
||||
|
||||
values_row_list:
|
||||
values_row_list ',' row_value
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, $3);
|
||||
}
|
||||
| row_value
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
;
|
||||
|
||||
row_value:
|
||||
ROW '(' insert_vals ')'
|
||||
{
|
||||
merge_nodes($$, result, T_VALUE_VECTOR, $3);
|
||||
}
|
||||
;
|
||||
|
||||
/*****************************************************************************
|
||||
*
|
||||
* analyze clause (mysql compatible)
|
||||
|
@ -92,7 +92,7 @@ int ObPlanCacheObject::set_params_info(const ParamStore ¶ms)
|
||||
LOG_WARN("fail to get ext obj data type", K(ret));
|
||||
} else {
|
||||
param_info.ext_real_type_ = data_type.get_obj_type();
|
||||
param_info.scale_ = data_type.get_meta_type().get_scale();
|
||||
param_info.scale_ = data_type.get_scale();
|
||||
}
|
||||
LOG_DEBUG("ext params info", K(data_type), K(param_info), K(params.at(i)));
|
||||
} else {
|
||||
|
@ -43,6 +43,7 @@
|
||||
#include "sql/spm/ob_spm_evolution_plan.h"
|
||||
#endif
|
||||
#include "pl/pl_cache/ob_pl_cache_mgr.h"
|
||||
#include "sql/plan_cache/ob_values_table_compression.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::common::hash;
|
||||
@ -718,32 +719,35 @@ int ObPlanCache::construct_fast_parser_result(common::ObIAllocator &allocator,
|
||||
batch_count,
|
||||
first_truncated_sql))) {
|
||||
LOG_WARN("fail to do insert optimization", K(ret));
|
||||
} else if (!can_do_batch_insert) {
|
||||
// can't do batch insert
|
||||
} else if (OB_FAIL(rebuild_raw_params(allocator,
|
||||
pc_ctx,
|
||||
fp_result,
|
||||
batch_count))) {
|
||||
LOG_WARN("fail to rebuild raw_param", K(ret), K(batch_count));
|
||||
} else if (pc_ctx.insert_batch_opt_info_.multi_raw_params_.empty()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected multi_raw_params, can't do batch insert opt, but not need to return error",
|
||||
K(batch_count), K(first_truncated_sql), K(pc_ctx.raw_sql_), K(fp_result));
|
||||
} else if (OB_ISNULL(pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null ptr, can't do batch insert opt, but not need to return error",
|
||||
K(batch_count), K(first_truncated_sql), K(pc_ctx.raw_sql_), K(fp_result));
|
||||
} else {
|
||||
fp_result.raw_params_.reset();
|
||||
fp_result.raw_params_.set_allocator(&allocator);
|
||||
fp_result.raw_params_.set_capacity(pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)->count());
|
||||
if (OB_FAIL(fp_result.raw_params_.assign(*pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)))) {
|
||||
LOG_WARN("fail to assign raw_param", K(ret));
|
||||
} else if (can_do_batch_insert) {
|
||||
if (OB_FAIL(rebuild_raw_params(allocator,
|
||||
pc_ctx,
|
||||
fp_result,
|
||||
batch_count))) {
|
||||
LOG_WARN("fail to rebuild raw_param", K(ret), K(batch_count));
|
||||
} else if (pc_ctx.insert_batch_opt_info_.multi_raw_params_.empty()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected multi_raw_params, can't do batch insert opt, but not need to return error",
|
||||
K(batch_count), K(first_truncated_sql), K(pc_ctx.raw_sql_), K(fp_result));
|
||||
} else if (OB_ISNULL(pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null ptr, can't do batch insert opt, but not need to return error",
|
||||
K(batch_count), K(first_truncated_sql), K(pc_ctx.raw_sql_), K(fp_result));
|
||||
} else {
|
||||
pc_ctx.sql_ctx_.set_is_do_insert_batch_opt(batch_count);
|
||||
fp_result.pc_key_.name_.assign_ptr(first_truncated_sql.ptr(), first_truncated_sql.length());
|
||||
LOG_DEBUG("print new fp_result.pc_key_.name_", K(fp_result.pc_key_.name_));
|
||||
fp_result.raw_params_.reset();
|
||||
fp_result.raw_params_.set_allocator(&allocator);
|
||||
fp_result.raw_params_.set_capacity(pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)->count());
|
||||
if (OB_FAIL(fp_result.raw_params_.assign(*pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)))) {
|
||||
LOG_WARN("fail to assign raw_param", K(ret));
|
||||
} else {
|
||||
pc_ctx.sql_ctx_.set_is_do_insert_batch_opt(batch_count);
|
||||
fp_result.pc_key_.name_.assign_ptr(first_truncated_sql.ptr(), first_truncated_sql.length());
|
||||
LOG_DEBUG("print new fp_result.pc_key_.name_", K(fp_result.pc_key_.name_));
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(ObValuesTableCompression::try_batch_exec_params(allocator, pc_ctx,
|
||||
*pc_ctx.sql_ctx_.session_info_, fp_result))) {
|
||||
LOG_WARN("failed to check fold params valid", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -192,6 +192,17 @@ struct PsNotParamInfo
|
||||
TO_STRING_KV(K_(idx), K_(ps_param));
|
||||
};
|
||||
|
||||
struct ObValuesTokenPos {
|
||||
ObValuesTokenPos() : no_param_sql_pos_(0), param_idx_(0) {}
|
||||
ObValuesTokenPos(const int64_t no_param_pos, const int64_t param_idx)
|
||||
: no_param_sql_pos_(no_param_pos), param_idx_(param_idx) {}
|
||||
int64_t no_param_sql_pos_;
|
||||
int64_t param_idx_;
|
||||
TO_STRING_KV(K(no_param_sql_pos_), K(param_idx_));
|
||||
};
|
||||
|
||||
typedef common::ObFixedArray<ObPCParam *, common::ObIAllocator> ObArrayPCParam;
|
||||
|
||||
struct ObFastParserResult
|
||||
{
|
||||
private:
|
||||
@ -203,7 +214,8 @@ public:
|
||||
raw_params_(&inner_alloc_),
|
||||
parameterized_params_(&inner_alloc_),
|
||||
cache_params_(NULL),
|
||||
values_token_pos_(0)
|
||||
values_token_pos_(0),
|
||||
values_tokens_(&inner_alloc_)
|
||||
{
|
||||
reset_question_mark_ctx();
|
||||
}
|
||||
@ -212,7 +224,9 @@ public:
|
||||
common::ObFixedArray<const common::ObObjParam *, common::ObIAllocator> parameterized_params_;
|
||||
ParamStore *cache_params_;
|
||||
ObQuestionMarkCtx question_mark_ctx_;
|
||||
int64_t values_token_pos_;
|
||||
int64_t values_token_pos_; // for insert values
|
||||
common::ObFixedArray<ObValuesTokenPos, common::ObIAllocator> values_tokens_; // for values table
|
||||
common::ObSEArray<ObArrayPCParam *, 4, common::ModulePageAllocator, true> array_params_;
|
||||
|
||||
void reset() {
|
||||
pc_key_.reset();
|
||||
@ -220,6 +234,8 @@ public:
|
||||
parameterized_params_.reuse();
|
||||
cache_params_ = NULL;
|
||||
values_token_pos_ = 0;
|
||||
values_tokens_.reuse();
|
||||
array_params_.reuse();
|
||||
}
|
||||
void reset_question_mark_ctx()
|
||||
{
|
||||
@ -230,7 +246,28 @@ public:
|
||||
question_mark_ctx_.by_name_ = false;
|
||||
question_mark_ctx_.by_defined_name_ = false;
|
||||
}
|
||||
TO_STRING_KV(K(pc_key_), K(raw_params_), K(parameterized_params_), K(cache_params_), K(values_token_pos_));
|
||||
int assign(const ObFastParserResult &other)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
pc_key_ = other.pc_key_;
|
||||
raw_params_.set_allocator(&inner_alloc_);
|
||||
parameterized_params_.set_allocator(&inner_alloc_);
|
||||
cache_params_ = other.cache_params_;
|
||||
question_mark_ctx_ = other.question_mark_ctx_;
|
||||
values_tokens_.set_allocator(&inner_alloc_);
|
||||
if (OB_FAIL(raw_params_.assign(other.raw_params_))) {
|
||||
SQL_PC_LOG(WARN, "failed to assign fix array", K(ret));
|
||||
} else if (OB_FAIL(parameterized_params_.assign(other.parameterized_params_))) {
|
||||
SQL_PC_LOG(WARN, "failed to assign fix array", K(ret));
|
||||
} else if (OB_FAIL(values_tokens_.assign(other.values_tokens_))) {
|
||||
SQL_PC_LOG(WARN, "failed to assign fix array", K(ret));
|
||||
} else if (OB_FAIL(array_params_.assign(other.array_params_))) {
|
||||
SQL_PC_LOG(WARN, "failed to assign array", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
TO_STRING_KV(K(pc_key_), K(raw_params_), K(parameterized_params_), K(cache_params_), K(values_token_pos_),
|
||||
K(values_tokens_), K(array_params_));
|
||||
};
|
||||
|
||||
enum WayToGenPlan {
|
||||
@ -341,6 +378,7 @@ struct ObPlanCacheCtx : public ObILibCacheCtx
|
||||
is_inner_sql_(false),
|
||||
is_original_ps_mode_(false),
|
||||
ab_params_(NULL),
|
||||
new_raw_sql_(),
|
||||
is_rewrite_sql_(false),
|
||||
rule_name_(),
|
||||
def_name_ctx_(NULL),
|
||||
@ -419,6 +457,7 @@ struct ObPlanCacheCtx : public ObILibCacheCtx
|
||||
K(dynamic_param_info_list_),
|
||||
K(tpl_sql_const_cons_),
|
||||
K(is_original_ps_mode_),
|
||||
K(new_raw_sql_),
|
||||
K(need_retry_add_plan_),
|
||||
K(insert_batch_opt_info_)
|
||||
);
|
||||
@ -469,6 +508,7 @@ struct ObPlanCacheCtx : public ObILibCacheCtx
|
||||
bool is_inner_sql_;
|
||||
bool is_original_ps_mode_;
|
||||
ParamStore *ab_params_; // arraybinding batch parameters,
|
||||
ObString new_raw_sql_; // values clause rebuild raw sql
|
||||
|
||||
// ********** for rewrite rule **********
|
||||
bool is_rewrite_sql_;
|
||||
|
@ -29,6 +29,8 @@
|
||||
#include "share/ob_duplicate_scope_define.h"
|
||||
#include "pl/ob_pl_stmt.h"
|
||||
#include "share/resource_manager/ob_resource_manager.h"
|
||||
#include "sql/plan_cache/ob_values_table_compression.h"
|
||||
|
||||
using namespace oceanbase::share::schema;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::pl;
|
||||
@ -175,6 +177,7 @@ ObPlanCacheValue::ObPlanCacheValue()
|
||||
need_param_(true),
|
||||
is_nested_sql_(false),
|
||||
is_batch_execute_(false),
|
||||
has_dynamic_values_table_(false),
|
||||
stored_schema_objs_(pc_alloc_),
|
||||
stmt_type_(stmt::T_MAX)
|
||||
{
|
||||
@ -266,6 +269,7 @@ int ObPlanCacheValue::init(ObPCVSet *pcv_set, const ObILibCacheObject *cache_obj
|
||||
need_param_ = plan->need_param();
|
||||
is_nested_sql_ = ObSQLUtils::is_nested_sql(&pc_ctx.exec_ctx_);
|
||||
is_batch_execute_ = pc_ctx.sql_ctx_.is_batch_params_execute();
|
||||
has_dynamic_values_table_ = pc_ctx.exec_ctx_.has_dynamic_values_table();
|
||||
MEMCPY(sql_id_, pc_ctx.sql_ctx_.sql_id_, sizeof(pc_ctx.sql_ctx_.sql_id_));
|
||||
if (OB_FAIL(not_param_index_.add_members2(pc_ctx.not_param_index_))) {
|
||||
LOG_WARN("fail to add not param index members", K(ret));
|
||||
@ -520,6 +524,12 @@ int ObPlanCacheValue::choose_plan(ObPlanCacheCtx &pc_ctx,
|
||||
LOG_WARN("failed to resolver row params", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (OB_UNLIKELY(pc_ctx.exec_ctx_.has_dynamic_values_table())) {
|
||||
if (OB_FAIL(ObValuesTableCompression::resolve_params_for_values_clause(pc_ctx, stmt_type_,
|
||||
not_param_info_, param_charset_type_, neg_param_index_, not_param_index_,
|
||||
must_be_positive_idx_, params))) {
|
||||
LOG_WARN("failed to resolve_params_for_values_clause ", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(resolver_params(pc_ctx,
|
||||
stmt_type_,
|
||||
param_charset_type_,
|
||||
@ -669,124 +679,39 @@ int ObPlanCacheValue::resolver_params(ObPlanCacheCtx &pc_ctx,
|
||||
const stmt::StmtType stmt_type,
|
||||
const ObIArray<ObCharsetType> ¶m_charset_type,
|
||||
const ObBitSet<> &neg_param_index,
|
||||
const ObBitSet<> ¬_param_index_,
|
||||
const ObBitSet<> ¬_param_index,
|
||||
const ObBitSet<> &must_be_positive_idx,
|
||||
ObIArray<ObPCParam *> &raw_params,
|
||||
ParamStore *obj_params)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSQLSessionInfo *session = pc_ctx.exec_ctx_.get_my_session();
|
||||
ParseNode *raw_param = NULL;
|
||||
ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx();
|
||||
const int64_t raw_param_cnt = raw_params.count();
|
||||
ObObjParam value;
|
||||
if (OB_ISNULL(session)) {
|
||||
if (OB_ISNULL(session) || OB_ISNULL(phy_ctx)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
SQL_PC_LOG(WARN, "invalid argument", K(ret), KP(session));
|
||||
} else if (obj_params != NULL && PC_PS_MODE != pc_ctx.mode_ && PC_PL_MODE != pc_ctx.mode_) {
|
||||
ObCollationType collation_connection = static_cast<ObCollationType>(
|
||||
session->get_local_collation_connection());
|
||||
int64_t N = raw_params.count();
|
||||
(void)obj_params->reserve(N);
|
||||
if (N != param_charset_type.count()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("raw_params and param_charset_type count is different",
|
||||
K(N), K(param_charset_type.count()),
|
||||
K(pc_ctx.raw_sql_), K(ret));
|
||||
SQL_PC_LOG(WARN, "invalid argument", K(ret), KP(session), KP(phy_ctx));
|
||||
} else if (obj_params == NULL || PC_PS_MODE == pc_ctx.mode_ || PC_PL_MODE == pc_ctx.mode_) {
|
||||
/* do nothing */
|
||||
} else if (OB_UNLIKELY(raw_param_cnt != param_charset_type.count())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
SQL_PC_LOG(WARN, "raw_params and param_charset_type count is different", K(ret),
|
||||
K(raw_param_cnt), K(param_charset_type.count()), K(pc_ctx.raw_sql_));
|
||||
} else {
|
||||
CHECK_COMPATIBILITY_MODE(session);
|
||||
ObCollationType collation_connection = static_cast<ObCollationType>(session->get_local_collation_connection());
|
||||
(void)obj_params->reserve(raw_param_cnt);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < raw_param_cnt; i++) {
|
||||
bool is_param = false;
|
||||
if (OB_FAIL(ObResolverUtils::resolver_param(pc_ctx, *session, phy_ctx->get_param_store_for_update(), stmt_type,
|
||||
param_charset_type.at(i), neg_param_index, not_param_index, must_be_positive_idx,
|
||||
raw_params.at(i), i, value, is_param))) {
|
||||
SQL_PC_LOG(WARN, "failed to resolver param", K(ret), K(i));
|
||||
} else if (is_param && OB_FAIL(obj_params->push_back(value))) {
|
||||
SQL_PC_LOG(WARN, "fail to push item to array", K(ret));
|
||||
} else {/* do nothing */}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) {
|
||||
value.reset();
|
||||
if (OB_ISNULL(raw_params.at(i))) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(raw_params.at(i)));
|
||||
} else if (NULL == (raw_param = raw_params.at(i)->node_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(raw_param));
|
||||
} else if (!not_param_index_.has_member(i)) { //not param
|
||||
if (neg_param_index.has_member(i)) {
|
||||
// select - 1.2 from dual
|
||||
// "- 1.2" will be treated as a const node with neg sign
|
||||
// however, ObNumber::from("- 1.2") will throw a error, for there are spaces between neg sign and num
|
||||
// so remove spaces before resolve_const is called
|
||||
if (OB_FAIL(rm_space_for_neg_num(raw_param, pc_ctx.allocator_))) {
|
||||
SQL_PC_LOG(WARN, "fail to remove spaces for neg node", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(handle_varchar_charset(param_charset_type.at(i),
|
||||
pc_ctx.allocator_,
|
||||
raw_param))) {
|
||||
SQL_PC_LOG(WARN, "fail to handle varchar charset");
|
||||
}
|
||||
ObString literal_prefix;
|
||||
const bool is_paramlize = false;
|
||||
CHECK_COMPATIBILITY_MODE(session);
|
||||
int64_t server_collation = CS_TYPE_INVALID;
|
||||
if (OB_SUCC(ret) && T_QUESTIONMARK == raw_param->type_) {
|
||||
ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx();
|
||||
int64_t idx = raw_param->value_;
|
||||
CK (nullptr != phy_ctx);
|
||||
CK (idx >= 0 && idx < phy_ctx->get_param_store_for_update().count());
|
||||
OX (value.set_is_boolean(phy_ctx->get_param_store_for_update().at(idx).is_boolean()));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (lib::is_oracle_mode() &&
|
||||
OB_FAIL(session->get_sys_variable(share::SYS_VAR_COLLATION_SERVER, server_collation))) {
|
||||
LOG_WARN("get sys variable failed", K(ret));
|
||||
} else if (OB_FAIL(ObResolverUtils::resolve_const(raw_param,
|
||||
stmt_type,
|
||||
pc_ctx.allocator_,
|
||||
collation_connection,
|
||||
session->get_nls_collation_nation(),
|
||||
session->get_timezone_info(),
|
||||
value,
|
||||
is_paramlize,
|
||||
literal_prefix,
|
||||
session->get_actual_nls_length_semantics(),
|
||||
static_cast<ObCollationType>(server_collation),
|
||||
NULL, session->get_sql_mode()))) {
|
||||
SQL_PC_LOG(WARN, "fail to resolve const", K(ret));
|
||||
} else if (FALSE_IT(value.set_raw_text_info(
|
||||
static_cast<int32_t>(raw_param->raw_sql_offset_),
|
||||
static_cast<int32_t>(raw_param->text_len_)))) {
|
||||
// nothing.
|
||||
} else if (OB_FAIL(obj_params->push_back(value))) {
|
||||
SQL_PC_LOG(WARN, "fail to push item to array", K(ret));
|
||||
} else if (ob_is_numeric_type(value.get_type())) {
|
||||
if (must_be_positive_idx.has_member(i)) {
|
||||
if (value.is_boolean()) {
|
||||
// boolean will skip this check
|
||||
} else if (lib::is_oracle_mode()
|
||||
&& (value.is_negative_number()
|
||||
|| (value.is_zero_number() && '-' == raw_param->str_value_[0]))) { // -0 is also counted as negative
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_TRACE("param must be positive", K(ret), K(i), K(value));
|
||||
pc_ctx.should_add_plan_ = false; // 内部主动抛出not supported时候需要设置这个标志,以免新计划add plan导致锁冲突
|
||||
} else if (lib::is_mysql_mode()
|
||||
&& value.is_integer_type()
|
||||
&& (value.get_int() < 0
|
||||
|| (0 == value.get_int() && '-' == raw_param->str_value_[0]))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_TRACE("param must be positive", K(ret), K(i), K(value));
|
||||
pc_ctx.should_add_plan_ = false; // 内部主动抛出not supported时候需要设置这个标志,以免新计划add plan导致锁冲突
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_PC_LOG(TRACE, "is_param",
|
||||
K(i),
|
||||
K(value),
|
||||
K(raw_param->type_),
|
||||
K(raw_param->value_),
|
||||
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
|
||||
} else {
|
||||
SQL_PC_LOG(TRACE, "not_param",
|
||||
K(i),
|
||||
K(value),
|
||||
K(raw_param->type_),
|
||||
K(raw_param->value_),
|
||||
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
|
||||
}
|
||||
} // for end
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1493,6 +1418,7 @@ void ObPlanCacheValue::reset()
|
||||
contain_sys_name_table_ = false;
|
||||
is_nested_sql_ = false;
|
||||
is_batch_execute_ = false;
|
||||
has_dynamic_values_table_ = false;
|
||||
for (int64_t i = 0; i < stored_schema_objs_.count(); i++) {
|
||||
if (OB_ISNULL(stored_schema_objs_.at(i)) || OB_ISNULL(pc_alloc_)) {
|
||||
// do nothing
|
||||
@ -1697,7 +1623,8 @@ int ObPlanCacheValue::match(ObPlanCacheCtx &pc_ctx,
|
||||
//because nested sql's plan be forced to use DAS plan
|
||||
//but the general sql's plan has no this constraint
|
||||
is_same = false;
|
||||
} else if (is_batch_execute_ != pc_ctx.sql_ctx_.is_batch_params_execute()) {
|
||||
} else if (is_batch_execute_ != pc_ctx.sql_ctx_.is_batch_params_execute()||
|
||||
has_dynamic_values_table_ != pc_ctx.exec_ctx_.has_dynamic_values_table()) {
|
||||
// the plan of batch execute sql can't match with the plan of general sql
|
||||
is_same = false;
|
||||
} else if (!need_param_) {
|
||||
@ -1742,43 +1669,6 @@ int ObPlanCacheValue::match(ObPlanCacheCtx &pc_ctx,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPlanCacheValue::handle_varchar_charset(ObCharsetType charset_type,
|
||||
ObIAllocator &allocator,
|
||||
ParseNode *&node)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if ((T_HEX_STRING == node->type_ || T_VARCHAR == node->type_)
|
||||
&& CHARSET_INVALID != charset_type) {
|
||||
ParseNode *charset_node = new_node(&allocator, T_CHARSET, 0);
|
||||
ParseNode *varchar_node = NULL;
|
||||
if (T_HEX_STRING == node->type_) {
|
||||
varchar_node = new_non_terminal_node(&allocator, T_VARCHAR, 1, charset_node);
|
||||
} else if (T_VARCHAR == node->type_) {
|
||||
varchar_node = new_non_terminal_node(&allocator, T_VARCHAR, 2, charset_node, node);
|
||||
}
|
||||
|
||||
if (OB_ISNULL(charset_node) || OB_ISNULL(varchar_node)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
const char *name = ObCharset::charset_name(charset_type);
|
||||
charset_node->str_value_ = parse_strdup(name, &allocator, &(charset_node->str_len_));
|
||||
if (NULL == charset_node->str_value_) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
varchar_node->str_value_ = node->str_value_;
|
||||
varchar_node->str_len_ = node->str_len_;
|
||||
varchar_node->raw_text_ = node->raw_text_;
|
||||
varchar_node->text_len_ = node->text_len_;
|
||||
varchar_node->type_ = T_VARCHAR;
|
||||
|
||||
node = varchar_node;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObPlanCacheValue::is_contain_tmp_tbl() const
|
||||
{
|
||||
bool is_contain = false;
|
||||
@ -2305,39 +2195,6 @@ int ObPlanCacheValue::lift_tenant_schema_version(int64_t new_schema_version)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPlanCacheValue::rm_space_for_neg_num(ParseNode *param_node, ObIAllocator &allocator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *buf = NULL;
|
||||
int64_t pos = 0;
|
||||
int64_t idx = 0;
|
||||
if (param_node->str_len_ <= 0) {
|
||||
// do nothing
|
||||
} else if ('-' != param_node->str_value_[idx]) {
|
||||
// 'select - 1.2 from dual' and 'select 1.2 from dual' will hit the same plan, the key is
|
||||
// select ? from dual, so '- 1.2' and '1.2' will all go here, if '-' is not presented,
|
||||
// do nothing
|
||||
LOG_TRACE("rm space for neg num", K(idx), K(ObString(param_node->str_len_, param_node->str_value_)));
|
||||
} else if (OB_ISNULL(buf = (char *)allocator.alloc(param_node->str_len_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocator memory", K(ret), K(param_node->str_len_));
|
||||
} else {
|
||||
buf[pos++] = '-';
|
||||
idx += 1;
|
||||
for (; idx < param_node->str_len_ && isspace(param_node->str_value_[idx]); idx++);
|
||||
int32_t len = (int32_t)(param_node->str_len_ - idx);
|
||||
if (len > 0) {
|
||||
MEMCPY(buf + pos, param_node->str_value_ + idx, len);
|
||||
}
|
||||
pos += len;
|
||||
param_node->str_value_ = buf;
|
||||
param_node->str_len_ = pos;
|
||||
|
||||
LOG_DEBUG("rm space for neg num", K(idx), K(ObString(param_node->str_len_, param_node->str_value_)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPlanCacheValue::check_contains_table(uint64_t db_id, common::ObString tab_name, bool &contains)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
@ -300,11 +300,6 @@ private:
|
||||
share::schema::ObSchemaObjVersion &local_outline_version);
|
||||
|
||||
int get_outline_param_index(ObExecContext &exec_ctx, int64_t ¶m_idx) const;
|
||||
|
||||
static int handle_varchar_charset(ObCharsetType charset_type,
|
||||
ObIAllocator &allocator,
|
||||
ParseNode *&node);
|
||||
|
||||
/**
|
||||
* @brief if there is a temporary table in dependency tables
|
||||
* @retval is_contain: true for containing temporary table
|
||||
@ -355,9 +350,6 @@ private:
|
||||
int need_check_schema_version(ObPlanCacheCtx &pc_ctx,
|
||||
int64_t &new_schema_version,
|
||||
bool &need_check);
|
||||
|
||||
static int rm_space_for_neg_num(ParseNode *param_node, ObIAllocator &allocator);
|
||||
|
||||
int assign_udr_infos(ObPlanCacheCtx &pc_ctx);
|
||||
void reset_tpl_sql_const_cons();
|
||||
int check_tpl_sql_const_cons(const ObFastParserResult &fp_result,
|
||||
@ -420,6 +412,7 @@ private:
|
||||
//Therefore, it is necessary to distinguish whether it is nested SQL or not.
|
||||
bool is_nested_sql_;
|
||||
bool is_batch_execute_;
|
||||
bool has_dynamic_values_table_;
|
||||
// only when not need to param, this feild will be used.
|
||||
common::ObString raw_sql_;
|
||||
common::ObFixedArray<PCVSchemaObj *, common::ObIAllocator> stored_schema_objs_;
|
||||
|
@ -1034,10 +1034,15 @@ int ObSqlParameterization::parameterize_syntax_tree(common::ObIAllocator &alloca
|
||||
fp_ctx.sql_mode_ = session->get_sql_mode();
|
||||
fp_ctx.is_udr_mode_ = pc_ctx.is_rewrite_sql_;
|
||||
fp_ctx.def_name_ctx_ = pc_ctx.def_name_ctx_;
|
||||
ObString raw_sql = pc_ctx.raw_sql_;
|
||||
if (pc_ctx.sql_ctx_.is_do_insert_batch_opt()) {
|
||||
raw_sql = pc_ctx.insert_batch_opt_info_.new_reconstruct_sql_;
|
||||
} else if (pc_ctx.exec_ctx_.has_dynamic_values_table()) {
|
||||
raw_sql = pc_ctx.new_raw_sql_;
|
||||
}
|
||||
if (OB_FAIL(fast_parser(allocator,
|
||||
fp_ctx,
|
||||
pc_ctx.sql_ctx_.is_do_insert_batch_opt() ?
|
||||
pc_ctx.insert_batch_opt_info_.new_reconstruct_sql_ : pc_ctx.raw_sql_,
|
||||
raw_sql,
|
||||
pc_ctx.fp_result_))) {
|
||||
SQL_PC_LOG(WARN, "fail to fast parser", K(ret));
|
||||
}
|
||||
@ -1474,8 +1479,8 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator,
|
||||
|| (ObParser::is_pl_stmt(sql, nullptr, &is_call_procedure) && !is_call_procedure))) {
|
||||
(void)fp_result.pc_key_.name_.assign_ptr(sql.ptr(), sql.length());
|
||||
} else if (GCONF._ob_enable_fast_parser) {
|
||||
if (OB_FAIL(ObFastParser::parse(sql, fp_ctx, allocator, no_param_sql_ptr,
|
||||
no_param_sql_len, p_list, param_num, fp_result.question_mark_ctx_, fp_result.values_token_pos_))) {
|
||||
if (OB_FAIL(ObFastParser::parse(sql, fp_ctx, allocator, no_param_sql_ptr, no_param_sql_len,
|
||||
p_list, param_num, fp_result, fp_result.values_token_pos_))) {
|
||||
LOG_WARN("fast parse error", K(param_num),
|
||||
K(ObString(no_param_sql_len, no_param_sql_ptr)), K(sql));
|
||||
}
|
||||
|
824
src/sql/plan_cache/ob_values_table_compression.cpp
Normal file
824
src/sql/plan_cache/ob_values_table_compression.cpp
Normal file
@ -0,0 +1,824 @@
|
||||
#define USING_LOG_PREFIX SQL_PC
|
||||
#include "sql/plan_cache/ob_values_table_compression.h"
|
||||
#include "sql/plan_cache/ob_plan_cache_struct.h"
|
||||
#include "sql/parser/ob_fast_parser.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/resolver/ob_resolver_utils.h"
|
||||
#include "sql/engine/expr/ob_expr_version.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
|
||||
namespace sql
|
||||
{
|
||||
// based in ob_parser.h
|
||||
const char *ObValuesTableCompression::lower_[ObParser::S_MAX] = {
|
||||
"", "", "", "", "", "", "", "", "", "", /* 0 ~9 */
|
||||
"", "", "", "update", "", "", "", "", "", "", /* 10 ~19 */
|
||||
"", "", "", "", "", "", "", "", "", "", /* 20 ~29 */
|
||||
"", "", "", "select", "insert", "delete", "values", "table", "into" /* 30 ~38 */
|
||||
};
|
||||
|
||||
const char *ObValuesTableCompression::upper_[ObParser::S_MAX] = {
|
||||
"", "", "", "", "", "", "", "", "", "", /* 0 ~9 */
|
||||
"", "", "", "UPDATE", "", "", "", "", "", "", /* 10 ~19 */
|
||||
"", "", "", "", "", "", "", "", "", "", /* 20 ~29 */
|
||||
"", "", "", "SELECT", "INSERT", "DELETE", "VALUES", "TABLE", "INTO" /* 30 ~38 */
|
||||
};
|
||||
|
||||
#define ISSPACE(c) ((c) == ' ' || (c) == '\n' || (c) == '\r' || (c) == '\t' || (c) == '\f' || (c) == '\v')
|
||||
|
||||
void ObValuesTableCompression::match_one_state(const char *&p,
|
||||
const char *p_end,
|
||||
const ObParser::State next_state,
|
||||
ObParser::State &state)
|
||||
{
|
||||
int compare_len = strlen(lower_[state]);
|
||||
if (p_end - p < compare_len) {
|
||||
state = ObParser::S_INVALID;
|
||||
} else if (0 == strncasecmp(p, lower_[state], compare_len)) {
|
||||
p += compare_len;
|
||||
state = next_state;
|
||||
} else {
|
||||
state = ObParser::S_INVALID;
|
||||
}
|
||||
}
|
||||
|
||||
bool ObValuesTableCompression::is_support_compress_values_table(const ObString &stmt)
|
||||
{
|
||||
ObParser::State state = ObParser::S_START;
|
||||
ObParser::State save_state = state;
|
||||
const char *p = stmt.ptr();
|
||||
const char *p_start = p;
|
||||
const char *p_end = p + stmt.length();
|
||||
bool is_dml_stmt = false;
|
||||
bool has_error = false;
|
||||
while (p < p_end && !has_error && !is_dml_stmt) {
|
||||
switch (state) {
|
||||
case ObParser::S_START: {
|
||||
if (ISSPACE(*p)) {
|
||||
p++;
|
||||
} else {
|
||||
if (!ObParser::is_comment(p, p_end, save_state, state, ObParser::S_INVALID) &&
|
||||
state != ObParser::S_INVALID) {
|
||||
if (ObParser::S_START == state) {
|
||||
save_state = state;
|
||||
if (*p == lower_[ObParser::S_SELECT][0] || *p == upper_[ObParser::S_SELECT][0]) {
|
||||
state = ObParser::S_SELECT;
|
||||
} else if (*p == lower_[ObParser::S_INSERT][0] || *p == upper_[ObParser::S_INSERT][0]) {
|
||||
state = ObParser::S_INSERT;
|
||||
} else if (*p == lower_[ObParser::S_UPDATE][0] || *p == upper_[ObParser::S_UPDATE][0]) {
|
||||
state = ObParser::S_UPDATE;
|
||||
} else if (*p == lower_[ObParser::S_DELETE][0] || *p == upper_[ObParser::S_DELETE][0]) {
|
||||
state = ObParser::S_DELETE;
|
||||
} else if (*p == lower_[ObParser::S_VALUES][0] || *p == upper_[ObParser::S_VALUES][0]) {
|
||||
state = ObParser::S_VALUES;
|
||||
} else {
|
||||
state = ObParser::S_INVALID;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} break;
|
||||
case ObParser::S_SELECT:
|
||||
case ObParser::S_DELETE:
|
||||
case ObParser::S_UPDATE:
|
||||
case ObParser::S_VALUES: {
|
||||
match_one_state(p, p_end, ObParser::S_NORMAL, state);
|
||||
} break;
|
||||
case ObParser::S_INSERT: {
|
||||
match_one_state(p, p_end, ObParser::S_INTO, state);
|
||||
} break;
|
||||
case ObParser::S_INTO: {
|
||||
if (ISSPACE(*p)) {
|
||||
p++;
|
||||
} else {
|
||||
if (!ObParser::is_comment(p, p_end, save_state, state, ObParser::S_INVALID) &&
|
||||
state != ObParser::S_INVALID) {
|
||||
match_one_state(p, p_end, ObParser::S_NORMAL, state);
|
||||
}
|
||||
}
|
||||
} break;
|
||||
case ObParser::S_NORMAL: {
|
||||
is_dml_stmt = true;
|
||||
has_error = false;
|
||||
} break;
|
||||
case ObParser::S_COMMENT: {
|
||||
if (*p == '\n') {
|
||||
// end of '--' comments
|
||||
state = save_state;
|
||||
}
|
||||
p++;
|
||||
} break;
|
||||
case ObParser::S_C_COMMENT: {
|
||||
if (*p == '*') {
|
||||
if ((p + 1 < p_end) && '/' == *(p + 1)) {
|
||||
// end of '/**/' comments
|
||||
state = save_state;
|
||||
p++;
|
||||
}
|
||||
}
|
||||
p++;
|
||||
} break;
|
||||
case ObParser::S_INVALID:
|
||||
default: {
|
||||
is_dml_stmt = false;
|
||||
has_error = true;
|
||||
} break;
|
||||
}
|
||||
}
|
||||
return is_dml_stmt && !has_error;
|
||||
}
|
||||
|
||||
int ObValuesTableCompression::add_raw_array_params(ObIAllocator &allocator,
|
||||
ObPlanCacheCtx &pc_ctx,
|
||||
const ObFastParserResult &fp_result,
|
||||
const int64_t begin_param,
|
||||
const int64_t row_count,
|
||||
const int64_t param_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (begin_param + row_count * param_count > fp_result.raw_params_.count() ||
|
||||
row_count <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected raw_params", K(ret), K(begin_param), K(row_count), K(param_count),
|
||||
K(fp_result.raw_params_.count()));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_count; ++i) {
|
||||
void *buf = nullptr;
|
||||
ObArrayPCParam *params_array = nullptr;
|
||||
if (OB_ISNULL(buf = allocator.alloc(sizeof(ObArrayPCParam)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to alloc memory", K(ret), K(sizeof(ObArrayPCParam)));
|
||||
} else {
|
||||
params_array = new(buf) ObArrayPCParam(allocator);
|
||||
params_array->set_capacity(row_count);
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < row_count; ++j) {
|
||||
int64_t param_idx = begin_param + j * param_count + i;
|
||||
if (OB_FAIL(params_array->push_back(fp_result.raw_params_.at(param_idx)))) {
|
||||
LOG_WARN("fail to push back", K(ret), K(i), K(j));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(pc_ctx.fp_result_.array_params_.push_back(params_array))) {
|
||||
LOG_WARN("fail to push params array", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObValuesTableCompression::rebuild_new_raw_sql(ObPlanCacheCtx &pc_ctx,
|
||||
const ObIArray<ObPCParam*> &raw_params,
|
||||
const int64_t begin_idx,
|
||||
const int64_t param_cnt,
|
||||
const int64_t delta_length,
|
||||
const ObString &no_param_sql,
|
||||
ObString &new_raw_sql,
|
||||
int64_t &no_param_sql_pos,
|
||||
int64_t &new_raw_pos)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *buff = new_raw_sql.ptr();
|
||||
int64_t buff_len = pc_ctx.raw_sql_.length();
|
||||
int64_t len = 0;
|
||||
if (OB_ISNULL(buff)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("buff is null", K(ret), KP(buff));
|
||||
} else if (begin_idx < 0 || raw_params.count() < begin_idx + param_cnt) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("param is wrong", K(ret), K(begin_idx), K(param_cnt));
|
||||
} else {
|
||||
for (int64_t i = begin_idx; OB_SUCC(ret) && i < begin_idx + param_cnt; i++) {
|
||||
const ObPCParam *pc_param = raw_params.at(i);
|
||||
if (OB_ISNULL(pc_param) || OB_ISNULL(pc_param->node_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected NULL ptr", K(ret), KP(pc_param));
|
||||
} else {
|
||||
int64_t param_pos = pc_param->node_->pos_ - delta_length; // get pos is in new no param sql
|
||||
int64_t param_len = pc_param->node_->text_len_;
|
||||
len = param_pos - no_param_sql_pos;
|
||||
if (OB_UNLIKELY(len < 0) || OB_UNLIKELY(new_raw_pos + len + param_len > buff_len)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected params", K(ret), K(len), K(param_len), K(new_raw_pos));
|
||||
} else {
|
||||
if (len > 0) {
|
||||
//copy text
|
||||
MEMCPY(buff + new_raw_pos, no_param_sql.ptr() + no_param_sql_pos, len);
|
||||
new_raw_pos += len;
|
||||
no_param_sql_pos += len;
|
||||
}
|
||||
if (param_pos == no_param_sql_pos) {
|
||||
//copy raw param
|
||||
MEMCPY(buff + new_raw_pos, pc_param->node_->raw_text_, param_len);
|
||||
new_raw_pos += param_len;
|
||||
no_param_sql_pos += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
new_raw_sql.assign_ptr(buff, new_raw_pos);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObValuesTableCompression::try_batch_exec_params(ObIAllocator &allocator,
|
||||
ObPlanCacheCtx &pc_ctx,
|
||||
ObSQLSessionInfo &session_info,
|
||||
ObFastParserResult &fp_result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool can_fold_params = false;
|
||||
// used for no_param_sql --> new_no_param_sql
|
||||
int64_t old_sql_pos = 0;
|
||||
int64_t new_sql_pos = 0;
|
||||
int64_t array_param_idx = 0;
|
||||
int64_t last_raw_param_idx = 0;
|
||||
int64_t new_raw_idx = 0;
|
||||
int64_t total_delta_len = 0;
|
||||
ObString new_no_param_sql;
|
||||
ObSEArray<ObPCParam*, 4> temp_store;
|
||||
char *buff = NULL;
|
||||
int64_t buff_len = pc_ctx.raw_sql_.length();
|
||||
int64_t no_param_sql_pos = 0;
|
||||
int64_t new_raw_sql_pos = 0;
|
||||
ObString &new_raw_sql = pc_ctx.new_raw_sql_;
|
||||
ObSEArray<int64_t, 16> raw_pos;
|
||||
ObPhysicalPlanCtx *phy_ctx = NULL;
|
||||
uint64_t data_version = 0;
|
||||
if (OB_FAIL(GET_MIN_DATA_VERSION(session_info.get_effective_tenant_id(), data_version))) {
|
||||
LOG_WARN("get tenant data version failed", K(ret), K(session_info.get_effective_tenant_id()));
|
||||
} else if (pc_ctx.sql_ctx_.handle_batched_multi_stmt() ||
|
||||
lib::is_oracle_mode() ||
|
||||
session_info.is_inner() ||
|
||||
session_info.get_is_in_retry() ||
|
||||
fp_result.values_tokens_.empty() ||
|
||||
data_version < DATA_VERSION_4_2_1_0 ||
|
||||
!is_support_compress_values_table(pc_ctx.raw_sql_)) {
|
||||
/* do nothing */
|
||||
} else if (OB_ISNULL(phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected NULL ptr", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < fp_result.values_tokens_.count(); ++i) {
|
||||
bool is_valid = false;
|
||||
int64_t values_token_pos = fp_result.values_tokens_.at(i).no_param_sql_pos_;
|
||||
int64_t param_idx = fp_result.values_tokens_.at(i).param_idx_; // idx in fp_result.raw_params_
|
||||
int64_t batch_count = 0;
|
||||
int64_t param_count = 0;
|
||||
int64_t delta_len = 0;
|
||||
if (OB_FAIL(ObValuesTableCompression::parser_values_row_str(allocator, fp_result.pc_key_.name_,
|
||||
values_token_pos, new_no_param_sql, old_sql_pos,
|
||||
new_sql_pos, batch_count, param_count, delta_len,
|
||||
is_valid))) {
|
||||
LOG_WARN("fail to parser insert string", K(ret), K(fp_result.pc_key_.name_));
|
||||
} else if (!is_valid || param_count <= 0 || batch_count <= 1 || delta_len <= 0) {
|
||||
LOG_TRACE("can not do batch opt", K(ret), K(is_valid), K(param_count), K(batch_count), K(delta_len));
|
||||
} else if (OB_FAIL(add_raw_array_params(allocator, pc_ctx, fp_result, param_idx, batch_count,
|
||||
param_count))) {
|
||||
LOG_WARN("fail to rebuild raw_param", K(ret));
|
||||
} else {
|
||||
if (!can_fold_params) {
|
||||
if (OB_ISNULL(buff = (char *)allocator.alloc(buff_len))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("buff is null", K(ret), KP(buff));
|
||||
} else {
|
||||
new_raw_sql.assign_ptr(buff, buff_len);
|
||||
}
|
||||
}
|
||||
for (int64_t j = last_raw_param_idx; OB_SUCC(ret) && j < param_idx + param_count; j++) {
|
||||
ObPCParam *pc_param = pc_ctx.fp_result_.raw_params_.at(j);
|
||||
if (OB_ISNULL(pc_param) || OB_ISNULL(pc_param->node_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("pc_param is null", K(ret), KP(pc_param));
|
||||
} else if (OB_FAIL(temp_store.push_back(pc_param))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(raw_pos.push_back(pc_param->node_->pos_ - total_delta_len))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(rebuild_new_raw_sql(pc_ctx, temp_store, new_raw_idx,
|
||||
temp_store.count() - new_raw_idx, total_delta_len, new_no_param_sql,
|
||||
new_raw_sql, no_param_sql_pos, new_raw_sql_pos))) {
|
||||
LOG_WARN("failed to rebuild new raw sql", K(ret));
|
||||
} else {
|
||||
int64_t batch_begin_idx = new_raw_idx + param_idx - last_raw_param_idx;
|
||||
if (OB_FAIL(phy_ctx->get_array_param_groups().push_back(ObArrayParamGroup(batch_count,
|
||||
param_count, batch_begin_idx)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
total_delta_len += delta_len;
|
||||
can_fold_params = true;
|
||||
last_raw_param_idx = param_idx + param_count * batch_count;
|
||||
new_raw_idx = temp_store.count();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && can_fold_params) {
|
||||
for (int64_t j = last_raw_param_idx; OB_SUCC(ret) && j < pc_ctx.fp_result_.raw_params_.count(); j++) {
|
||||
ObPCParam *pc_param = pc_ctx.fp_result_.raw_params_.at(j);
|
||||
if (OB_ISNULL(pc_param) || OB_ISNULL(pc_param->node_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("pc_param is null", K(ret), KP(pc_param));
|
||||
} else if (OB_FAIL(temp_store.push_back(pc_param))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(raw_pos.push_back(pc_param->node_->pos_ - total_delta_len))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(rebuild_new_raw_sql(pc_ctx, temp_store, new_raw_idx,
|
||||
temp_store.count() - new_raw_idx, total_delta_len, new_no_param_sql,
|
||||
new_raw_sql, no_param_sql_pos, new_raw_sql_pos))) {
|
||||
LOG_WARN("failed to rebuild new raw sql", K(ret));
|
||||
} else if (OB_UNLIKELY(raw_pos.count() != temp_store.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("param is invalid", K(ret));
|
||||
} else {
|
||||
int64_t len = new_no_param_sql.length() - no_param_sql_pos;
|
||||
if (OB_UNLIKELY(len < 0) || OB_UNLIKELY(new_raw_sql_pos + len > buff_len)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected params", K(ret), K(len), K(new_raw_sql_pos));
|
||||
} else if (len > 0) {
|
||||
MEMCPY(buff + new_raw_sql_pos, new_no_param_sql.ptr() + no_param_sql_pos, len);
|
||||
new_raw_sql_pos += len;
|
||||
no_param_sql_pos += len;
|
||||
new_raw_sql.assign_ptr(buff, new_raw_sql_pos);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// handle error.
|
||||
if (ret != OB_SUCCESS) {
|
||||
// 这里边的无论什么报错,都可以被吞掉,只是报错后就不能再做batch优化
|
||||
LOG_TRACE("failed to try fold params for values table", K(ret));
|
||||
phy_ctx->get_array_param_groups().reset();
|
||||
pc_ctx.fp_result_.array_params_.reset();
|
||||
pc_ctx.new_raw_sql_.reset();
|
||||
can_fold_params = false;
|
||||
ret = OB_SUCCESS;
|
||||
} else if (can_fold_params) {
|
||||
fp_result.pc_key_.name_.assign_ptr(new_no_param_sql.ptr(), new_no_param_sql.length());
|
||||
fp_result.raw_params_.reset();
|
||||
fp_result.raw_params_.set_allocator(&allocator);
|
||||
fp_result.raw_params_.set_capacity(temp_store.count());
|
||||
for (int64_t i = 0; i < temp_store.count(); i++) {
|
||||
// checked null before
|
||||
temp_store.at(i)->node_->pos_ = raw_pos.at(i);
|
||||
}
|
||||
if (OB_FAIL(fp_result.raw_params_.assign(temp_store))) {
|
||||
LOG_WARN("fail to assign raw_param", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/* for choose plan, will resolve all params */
|
||||
int ObValuesTableCompression::resolve_params_for_values_clause(ObPlanCacheCtx &pc_ctx,
|
||||
const stmt::StmtType stmt_type,
|
||||
const ObIArray<NotParamInfo> ¬_param_info,
|
||||
const ObIArray<ObCharsetType> ¶m_charset_type,
|
||||
const ObBitSet<> &neg_param_index,
|
||||
const ObBitSet<> ¬_param_index,
|
||||
const ObBitSet<> &must_be_positive_idx,
|
||||
ParamStore *&ab_params)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSQLSessionInfo *session = pc_ctx.exec_ctx_.get_my_session();
|
||||
ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx();
|
||||
ObObjParam obj_param;
|
||||
ObObjParam array_param;
|
||||
void *ptr = NULL;
|
||||
int64_t raw_param_cnt = pc_ctx.fp_result_.raw_params_.count();
|
||||
int64_t raw_idx = 0; // idx in pc_ctx.fp_result_.raw_params_
|
||||
int64_t array_param_idx = 0; // idx in pc_ctx.fp_result_.array_params_
|
||||
int64_t not_param_cnt = 0;
|
||||
bool is_param = false;
|
||||
if (OB_UNLIKELY(!pc_ctx.exec_ctx_.has_dynamic_values_table()) || OB_ISNULL(session) ||
|
||||
OB_ISNULL(phy_ctx) || OB_UNLIKELY(param_charset_type.count() != raw_param_cnt) ||
|
||||
OB_ISNULL(ab_params)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sql should be mutil stmt", K(ret), KP(session), KP(phy_ctx), K(raw_param_cnt),
|
||||
K(param_charset_type.count()), KP(ab_params));
|
||||
} else if (OB_FAIL(ab_params->reserve(raw_param_cnt))) {
|
||||
LOG_WARN("failed to reserve param num", K(ret));
|
||||
} else {
|
||||
ParamStore &phy_param_store = phy_ctx->get_param_store_for_update();
|
||||
ObIArray<ObArrayParamGroup> &array_param_groups = phy_ctx->get_array_param_groups();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < array_param_groups.count(); ++i) {
|
||||
int64_t param_num = array_param_groups.at(i).column_count_;
|
||||
int64_t batch_num = array_param_groups.at(i).row_count_;
|
||||
int64_t array_idx = array_param_groups.at(i).start_param_idx_;
|
||||
if (OB_UNLIKELY(array_idx + param_num > raw_param_cnt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sql should be mutil stmt", K(ret));
|
||||
}
|
||||
// 1.1 build params before batch group
|
||||
for (; OB_SUCC(ret) && raw_idx < array_idx; raw_idx++) {
|
||||
if (OB_FAIL(ObResolverUtils::resolver_param(pc_ctx, *session, phy_param_store, stmt_type,
|
||||
param_charset_type.at(raw_idx), neg_param_index, not_param_index,
|
||||
must_be_positive_idx, pc_ctx.fp_result_.raw_params_.at(raw_idx), raw_idx,
|
||||
obj_param, is_param))) {
|
||||
LOG_WARN("failed to resolver param", K(ret), K(raw_idx));
|
||||
} else if (!is_param) {
|
||||
not_param_cnt++; // in value clause, which wonn't happen actually
|
||||
} else if (OB_FAIL(ab_params->push_back(obj_param))) {
|
||||
LOG_WARN("fail to push item to array", K(ret), K(raw_idx));
|
||||
}
|
||||
}
|
||||
|
||||
// 1.2 build array_param in batch group
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < param_num; j++, raw_idx++, array_param_idx++) {
|
||||
ObArrayPCParam *raw_array_param = pc_ctx.fp_result_.array_params_.at(array_param_idx);
|
||||
ObSEArray<ObExprResType, 4> res_types;
|
||||
ObSqlArrayObj *array_param_ptr = ObSqlArrayObj::alloc(pc_ctx.allocator_, batch_num);
|
||||
bool is_same = true;
|
||||
ObExprResType new_res_type;
|
||||
if (OB_ISNULL(array_param_ptr) || OB_ISNULL(raw_array_param)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocate memory", K(ret));
|
||||
} else {
|
||||
for (int64_t k = 0; OB_SUCC(ret) && k < batch_num; k++) {
|
||||
if (OB_FAIL(ObResolverUtils::resolver_param(pc_ctx, *session, phy_param_store, stmt_type,
|
||||
param_charset_type.at(raw_idx), neg_param_index, not_param_index,
|
||||
must_be_positive_idx, raw_array_param->at(k), raw_idx,
|
||||
array_param_ptr->data_[k], is_param))) {
|
||||
LOG_WARN("failed to resolver param", K(ret), K(k), K(raw_idx), K(j));
|
||||
} else {
|
||||
const ObObjParam ¶m = array_param_ptr->data_[k];
|
||||
ObExprResType res_type;
|
||||
res_type.set_meta(ObSQLUtils::is_oracle_empty_string(param) ? param.get_param_meta() : param.get_meta());
|
||||
res_type.set_accuracy(param.get_accuracy());
|
||||
res_type.set_result_flag(param.get_result_flag());
|
||||
if (res_type.get_length() == -1) {
|
||||
if (res_type.is_varchar() || res_type.is_nvarchar2()) {
|
||||
res_type.set_length(OB_MAX_ORACLE_VARCHAR_LENGTH);
|
||||
} else if (res_type.is_char() || res_type.is_nchar()) {
|
||||
res_type.set_length(OB_MAX_ORACLE_CHAR_LENGTH_BYTE);
|
||||
}
|
||||
}
|
||||
if (k == 0) {
|
||||
new_res_type = res_type;
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
} else if (k > 0 && is_same) {
|
||||
is_same = ObSQLUtils::is_same_type(res_type, res_types.at(0));
|
||||
if (!is_same) {
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
/* 推导类型 */
|
||||
if (OB_SUCC(ret) && !is_same) {
|
||||
new_res_type.reset();
|
||||
ObExprVersion dummy_op(pc_ctx.allocator_);
|
||||
const ObLengthSemantics length_semantics = session->get_actual_nls_length_semantics();
|
||||
ObCollationType coll_type = CS_TYPE_INVALID;
|
||||
if (OB_FAIL(session->get_collation_connection(coll_type))) {
|
||||
LOG_WARN("fail to get_collation_connection", K(ret));
|
||||
} else if (OB_FAIL(dummy_op.aggregate_result_type_for_merge(new_res_type,
|
||||
&res_types.at(0),
|
||||
res_types.count(),
|
||||
coll_type,
|
||||
false,
|
||||
length_semantics,
|
||||
session))) {
|
||||
LOG_WARN("failed to aggregate result type for merge", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
array_param.reset();
|
||||
array_param_ptr->element_.set_meta_type(new_res_type.get_obj_meta());
|
||||
array_param_ptr->element_.set_accuracy(new_res_type.get_accuracy());
|
||||
array_param.set_extend(reinterpret_cast<int64_t>(array_param_ptr), T_EXT_SQL_ARRAY);
|
||||
array_param.set_param_meta();
|
||||
array_param.get_param_flag().is_batch_parameter_ = true;
|
||||
if (OB_FAIL(ab_params->push_back(array_param))) {
|
||||
LOG_WARN("failed to push back param", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
for (; OB_SUCC(ret) && raw_idx < raw_param_cnt; raw_idx++) {
|
||||
if (OB_FAIL(ObResolverUtils::resolver_param(pc_ctx, *session, phy_param_store, stmt_type,
|
||||
param_charset_type.at(raw_idx), neg_param_index, not_param_index,
|
||||
must_be_positive_idx, pc_ctx.fp_result_.raw_params_.at(raw_idx), raw_idx,
|
||||
obj_param, is_param))) {
|
||||
LOG_WARN("failed to resolver param", K(ret), K(raw_idx));
|
||||
} else if (!is_param) {
|
||||
not_param_cnt++;
|
||||
} else if (OB_FAIL(ab_params->push_back(obj_param))) {
|
||||
LOG_WARN("fail to push item to array", K(ret), K(raw_idx));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/* after handle parser, only resolve array params */
|
||||
int ObValuesTableCompression::resolve_params_for_values_clause(ObPlanCacheCtx &pc_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObBitSet<> bit_set_dummy;
|
||||
ObSQLSessionInfo *session = pc_ctx.exec_ctx_.get_my_session();
|
||||
ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx();
|
||||
ObObjParam obj_param;
|
||||
ObObjParam array_param;
|
||||
int64_t raw_param_cnt = pc_ctx.fp_result_.raw_params_.count();
|
||||
int64_t raw_idx = 0; // idx in pc_ctx.fp_result_.raw_params_
|
||||
int64_t array_param_idx = 0; // idx in pc_ctx.fp_result_.array_params_
|
||||
bool is_param = false;
|
||||
const ObIArray<ObCharsetType> ¶m_charset_type = pc_ctx.param_charset_type_;
|
||||
if (OB_UNLIKELY(!pc_ctx.exec_ctx_.has_dynamic_values_table()) || OB_ISNULL(session) ||
|
||||
OB_ISNULL(phy_ctx) || OB_UNLIKELY(param_charset_type.count() != raw_param_cnt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sql should be mutil stmt", K(ret), KP(session), KP(phy_ctx), K(raw_param_cnt),
|
||||
K(param_charset_type.count()));
|
||||
} else {
|
||||
ParamStore &phy_param_store = phy_ctx->get_param_store_for_update();
|
||||
ObIArray<ObArrayParamGroup> &array_param_groups = phy_ctx->get_array_param_groups();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < array_param_groups.count(); ++i) {
|
||||
int64_t param_num = array_param_groups.at(i).column_count_;
|
||||
int64_t batch_num = array_param_groups.at(i).row_count_;
|
||||
int64_t raw_idx = array_param_groups.at(i).start_param_idx_;
|
||||
if (OB_UNLIKELY(raw_idx + param_num > raw_param_cnt) ||
|
||||
OB_UNLIKELY(raw_idx + param_num > phy_param_store.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sql should be mutil stmt", K(ret));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < param_num; j++, raw_idx++, array_param_idx++) {
|
||||
ObArrayPCParam *raw_array_param = pc_ctx.fp_result_.array_params_.at(array_param_idx);
|
||||
ObSEArray<ObExprResType, 4> res_types;
|
||||
ObSqlArrayObj *array_param_ptr = ObSqlArrayObj::alloc(pc_ctx.allocator_, batch_num);
|
||||
bool is_same = true;
|
||||
ObExprResType new_res_type;
|
||||
if (OB_ISNULL(array_param_ptr) || OB_ISNULL(raw_array_param)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocate memory", K(ret));
|
||||
} else {
|
||||
for (int64_t k = 0; OB_SUCC(ret) && k < batch_num; k++) {
|
||||
if (OB_FAIL(ObResolverUtils::resolver_param(pc_ctx, *session, phy_param_store, stmt::T_SELECT,
|
||||
param_charset_type.at(raw_idx), bit_set_dummy, bit_set_dummy,
|
||||
bit_set_dummy, raw_array_param->at(k), raw_idx,
|
||||
array_param_ptr->data_[k], is_param))) {
|
||||
LOG_WARN("failed to resolver param", K(ret), K(k), K(raw_idx), K(j));
|
||||
} else {
|
||||
const ObObjParam ¶m = array_param_ptr->data_[k];
|
||||
ObExprResType res_type;
|
||||
res_type.set_meta(ObSQLUtils::is_oracle_empty_string(param) ? param.get_param_meta() : param.get_meta());
|
||||
res_type.set_accuracy(param.get_accuracy());
|
||||
res_type.set_result_flag(param.get_result_flag());
|
||||
if (res_type.get_length() == -1) {
|
||||
if (res_type.is_varchar() || res_type.is_nvarchar2()) {
|
||||
res_type.set_length(OB_MAX_ORACLE_VARCHAR_LENGTH);
|
||||
} else if (res_type.is_char() || res_type.is_nchar()) {
|
||||
res_type.set_length(OB_MAX_ORACLE_CHAR_LENGTH_BYTE);
|
||||
}
|
||||
}
|
||||
if (k == 0) {
|
||||
new_res_type = res_type;
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
} else if (k > 0 && is_same) {
|
||||
is_same = ObSQLUtils::is_same_type(res_type, res_types.at(0));
|
||||
if (!is_same) {
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(res_types.push_back(res_type))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
/* 推导类型 */
|
||||
if (OB_SUCC(ret) && !is_same) {
|
||||
new_res_type.reset();
|
||||
ObExprVersion dummy_op(pc_ctx.allocator_);
|
||||
const ObLengthSemantics length_semantics = session->get_actual_nls_length_semantics();
|
||||
ObCollationType coll_type = CS_TYPE_INVALID;
|
||||
if (OB_FAIL(session->get_collation_connection(coll_type))) {
|
||||
LOG_WARN("fail to get_collation_connection", K(ret));
|
||||
} else if (OB_FAIL(dummy_op.aggregate_result_type_for_merge(new_res_type,
|
||||
&res_types.at(0),
|
||||
res_types.count(),
|
||||
coll_type,
|
||||
false,
|
||||
length_semantics,
|
||||
session))) {
|
||||
LOG_WARN("failed to aggregate result type for merge", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("get result type", K(new_res_type), K(res_types));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
array_param.reset();
|
||||
array_param_ptr->element_.set_meta_type(new_res_type.get_obj_meta());
|
||||
array_param_ptr->element_.set_accuracy(new_res_type.get_accuracy());
|
||||
array_param.set_extend(reinterpret_cast<int64_t>(array_param_ptr), T_EXT_SQL_ARRAY);
|
||||
array_param.set_param_meta();
|
||||
array_param.get_param_flag().is_batch_parameter_ = true;
|
||||
phy_param_store.at(raw_idx) = array_param;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObValuesTableCompression::skip_space(ObRawSql &raw_sql)
|
||||
{
|
||||
int64_t space_len = 0;
|
||||
bool is_space = true;
|
||||
while (!raw_sql.search_end_ && is_space && raw_sql.cur_pos_ < raw_sql.raw_sql_len_) {
|
||||
if (is_mysql_space(raw_sql.raw_sql_[raw_sql.cur_pos_])) {
|
||||
raw_sql.scan(1);
|
||||
} else {
|
||||
is_space = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool ObValuesTableCompression::skip_row_constructor(ObRawSql &raw_sql)
|
||||
{
|
||||
bool b_ret = false;
|
||||
int64_t space_len = 0;
|
||||
if (!raw_sql.search_end_ && 0 == raw_sql.strncasecmp(raw_sql.cur_pos_, "row", 3)) {
|
||||
raw_sql.scan(3);
|
||||
skip_space(raw_sql);
|
||||
b_ret = true;
|
||||
}
|
||||
return b_ret;
|
||||
}
|
||||
|
||||
/* only row(?,?,?) is valid */
|
||||
void ObValuesTableCompression::get_one_row_str(ObRawSql &no_param_sql,
|
||||
int64_t ¶m_count,
|
||||
int64_t &end_pos,
|
||||
bool &is_valid)
|
||||
{
|
||||
enum ROW_STATE {
|
||||
START_STATE = 0,
|
||||
LEFT_PAR_STATE, // "("
|
||||
PARS_MATCH, // ")"
|
||||
UNEXPECTED_STATE
|
||||
};
|
||||
ROW_STATE row_state = START_STATE;
|
||||
int left_count = 0;
|
||||
int comma_count = 0;
|
||||
bool need_break = false;
|
||||
int64_t curr_pos = 0;
|
||||
is_valid = false;
|
||||
param_count = 0;
|
||||
end_pos = 0;
|
||||
skip_space(no_param_sql);
|
||||
if (no_param_sql.is_search_end() || !skip_row_constructor(no_param_sql)) {
|
||||
/* do nothing */
|
||||
} else {
|
||||
while (!need_break && !no_param_sql.is_search_end()) {
|
||||
skip_space(no_param_sql);
|
||||
char ch = no_param_sql.char_at(no_param_sql.cur_pos_);
|
||||
curr_pos = no_param_sql.cur_pos_;
|
||||
no_param_sql.scan(1);
|
||||
/* state machine */
|
||||
switch (row_state) {
|
||||
case START_STATE:
|
||||
if ('(' == ch) {
|
||||
row_state = LEFT_PAR_STATE;
|
||||
left_count++;
|
||||
} else {
|
||||
row_state = UNEXPECTED_STATE;
|
||||
}
|
||||
break;
|
||||
case LEFT_PAR_STATE:
|
||||
if (')' == ch) {
|
||||
left_count--;
|
||||
if (0 == left_count) {
|
||||
row_state = PARS_MATCH;
|
||||
end_pos = curr_pos;
|
||||
}
|
||||
} else if ('(' == ch) {
|
||||
left_count++;
|
||||
} else if ('?' == ch) {
|
||||
param_count++;
|
||||
} else if (',' == ch && comma_count + 1 == param_count) {
|
||||
comma_count++;
|
||||
} else {
|
||||
row_state = UNEXPECTED_STATE;
|
||||
}
|
||||
break;
|
||||
case PARS_MATCH:
|
||||
if (',' != ch) {
|
||||
no_param_sql.search_end_ = true;
|
||||
}
|
||||
need_break = true;
|
||||
break;
|
||||
case UNEXPECTED_STATE:
|
||||
need_break = true;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (PARS_MATCH == row_state) {
|
||||
if (param_count > 0 && param_count == comma_count + 1) {
|
||||
is_valid = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int ObValuesTableCompression::parser_values_row_str(ObIAllocator &allocator,
|
||||
const ObString &no_param_sql,
|
||||
const int64_t values_token_pos,
|
||||
ObString &new_no_param_sql,
|
||||
int64_t &old_pos,
|
||||
int64_t &new_pos,
|
||||
int64_t &row_count,
|
||||
int64_t ¶m_count,
|
||||
int64_t &delta_length,
|
||||
bool &can_batch_opt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t no_param_sql_len = no_param_sql.length();
|
||||
ObRawSql raw_sql;
|
||||
raw_sql.init(no_param_sql.ptr(), no_param_sql_len);
|
||||
raw_sql.cur_pos_ = values_token_pos;
|
||||
int64_t new_sql_len = 0;
|
||||
int64_t cur_param_count = 0;
|
||||
int64_t end_pos = 0;
|
||||
int64_t first_sql_end_pos = 0;
|
||||
bool is_first = true;
|
||||
bool is_valid = true;
|
||||
row_count = 0;
|
||||
param_count = 0;
|
||||
delta_length = 0;
|
||||
can_batch_opt = false;
|
||||
if (0 == raw_sql.strncasecmp(raw_sql.cur_pos_, "values", 6)) {
|
||||
raw_sql.scan(6);
|
||||
while (OB_SUCC(ret) && is_valid && !raw_sql.is_search_end()) {
|
||||
get_one_row_str(raw_sql, cur_param_count, end_pos, is_valid);
|
||||
if (!is_valid) {
|
||||
} else if (is_first) {
|
||||
param_count = cur_param_count;
|
||||
first_sql_end_pos = end_pos;
|
||||
is_first = false;
|
||||
} else if (cur_param_count != param_count) {
|
||||
LOG_WARN("should not be here", K(ret), K(cur_param_count), K(param_count));
|
||||
}
|
||||
row_count++;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && is_valid && row_count > 1) {
|
||||
char *buffer = NULL;
|
||||
if (OB_ISNULL(buffer = static_cast<char*>(allocator.alloc(no_param_sql_len)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to alloc memory", K(ret), K(no_param_sql_len));
|
||||
} else {
|
||||
// init
|
||||
int64_t length = new_pos;
|
||||
MEMCPY(buffer, new_no_param_sql.ptr(), length);
|
||||
new_sql_len += length;
|
||||
|
||||
length = first_sql_end_pos + 1 - old_pos;
|
||||
MEMCPY(buffer + new_pos, no_param_sql.ptr() + old_pos, length);
|
||||
new_sql_len += length;
|
||||
new_pos += length;
|
||||
old_pos = end_pos + 1;
|
||||
|
||||
length = no_param_sql_len - old_pos;
|
||||
MEMCPY(buffer + new_pos, no_param_sql.ptr() + old_pos, length);
|
||||
new_sql_len += length;
|
||||
|
||||
delta_length = end_pos - first_sql_end_pos;
|
||||
new_no_param_sql.assign_ptr(buffer, new_sql_len);
|
||||
can_batch_opt = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
92
src/sql/plan_cache/ob_values_table_compression.h
Normal file
92
src/sql/plan_cache/ob_values_table_compression.h
Normal file
@ -0,0 +1,92 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_PLAN_CACHE_OB_VALUES_TABLE_COMPRESSION_
|
||||
#define OCEANBASE_SQL_PLAN_CACHE_OB_VALUES_TABLE_COMPRESSION_
|
||||
|
||||
#include "sql/parser/ob_parser.h"
|
||||
#include "lib/string/ob_string.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
class ObPlanCacheCtx;
|
||||
struct ObRawSql;
|
||||
class ObFastParserResult;
|
||||
|
||||
// values clause folding params utils class
|
||||
class ObValuesTableCompression {
|
||||
public:
|
||||
static int try_batch_exec_params(common::ObIAllocator &allocator,
|
||||
ObPlanCacheCtx &pc_ctx,
|
||||
ObSQLSessionInfo &session_info,
|
||||
ObFastParserResult &fp_result);
|
||||
// for resolve params in fast parser to match plan set
|
||||
static int resolve_params_for_values_clause(ObPlanCacheCtx &pc_ctx,
|
||||
const stmt::StmtType stmt_type,
|
||||
const common::ObIArray<NotParamInfo> ¬_param_info,
|
||||
const common::ObIArray<ObCharsetType> ¶m_charset_type,
|
||||
const ObBitSet<> &neg_param_index,
|
||||
const ObBitSet<> ¬_param_index,
|
||||
const ObBitSet<> &must_be_positive_idx,
|
||||
ParamStore *&ab_params);
|
||||
// for resolve array params after handler_parser
|
||||
static int resolve_params_for_values_clause(ObPlanCacheCtx &pc_ctx);
|
||||
static int parser_values_row_str(common::ObIAllocator &allocator,
|
||||
const common::ObString &no_param_sql,
|
||||
const int64_t values_token_pos,
|
||||
common::ObString &new_no_param_sql,
|
||||
int64_t &old_end_pos,
|
||||
int64_t &new_end_pos,
|
||||
int64_t &row_count,
|
||||
int64_t ¶m_count,
|
||||
int64_t &delta_length,
|
||||
bool &can_batch_opt);
|
||||
private:
|
||||
static void match_one_state(const char *&p,
|
||||
const char *p_end,
|
||||
const ObParser::State next_state,
|
||||
ObParser::State &state);
|
||||
static bool is_support_compress_values_table(const common::ObString &stmt);
|
||||
static int add_raw_array_params(common::ObIAllocator &allocator,
|
||||
ObPlanCacheCtx &pc_ctx,
|
||||
const ObFastParserResult &fp_result,
|
||||
const int64_t begin_param,
|
||||
const int64_t row_count,
|
||||
const int64_t param_count);
|
||||
static int rebuild_new_raw_sql(ObPlanCacheCtx &pc_ctx,
|
||||
const common::ObIArray<ObPCParam*> &raw_params,
|
||||
const int64_t begin_idx,
|
||||
const int64_t param_cnt,
|
||||
const int64_t delta_length,
|
||||
const common::ObString &no_param_sql,
|
||||
common::ObString &new_raw_sql,
|
||||
int64_t &no_param_sql_pos,
|
||||
int64_t &new_raw_pos);
|
||||
static bool is_mysql_space(char ch) {
|
||||
return ch != INVALID_CHAR && SPACE_FLAGS[static_cast<uint8_t>(ch)];
|
||||
}
|
||||
static void skip_space(ObRawSql &raw_sql);
|
||||
static bool skip_row_constructor(ObRawSql &raw_sql);
|
||||
static void get_one_row_str(ObRawSql &no_param_sql,
|
||||
int64_t ¶m_count,
|
||||
int64_t &end_pos,
|
||||
bool &is_valid);
|
||||
private:
|
||||
static const char *lower_[ObParser::S_MAX];
|
||||
static const char *upper_[ObParser::S_MAX];
|
||||
};
|
||||
|
||||
}
|
||||
}
|
||||
#endif /* _OB_VALUES_TABLE_COMPRESSION_H */
|
@ -337,6 +337,23 @@ int ObColumnNamespaceChecker::check_column_exists(const TableItem &table_item, c
|
||||
} else {
|
||||
is_exist = false;
|
||||
}
|
||||
} else if (table_item.is_values_table()) {
|
||||
ObSEArray<ObColumnRefRawExpr *, 4> values_desc;
|
||||
if (OB_ISNULL(dml_stmt_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(dml_stmt_));
|
||||
} else if (OB_FAIL(dml_stmt_->get_column_exprs(table_item.table_id_, values_desc))) {
|
||||
LOG_WARN("failed to get column exprs");
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_exist && i < values_desc.count(); ++i) {
|
||||
if (OB_ISNULL(values_desc.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(values_desc.at(i)));
|
||||
} else {
|
||||
is_exist = ObCharset::case_compat_mode_equal(values_desc.at(i)->get_column_name(), col_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected table type", K_(table_item.type));
|
||||
|
@ -47,7 +47,9 @@ public:
|
||||
equal_columns_(),
|
||||
cur_joined_table_(NULL),
|
||||
check_unique_(true),
|
||||
join_infos_(NULL) {}
|
||||
join_infos_(NULL),
|
||||
dml_stmt_(NULL)
|
||||
{}
|
||||
|
||||
~ObColumnNamespaceChecker() {};
|
||||
/**
|
||||
@ -101,6 +103,7 @@ public:
|
||||
bool skip_join_dup = false);
|
||||
int set_equal_columns(const common::ObIArray<common::ObString> &columns);
|
||||
void clear_equal_columns();
|
||||
void set_dml_stmt(const ObDMLStmt *dml_stmt) { dml_stmt_ = dml_stmt; }
|
||||
private:
|
||||
int find_column_in_single_table(const TableItem &table_item,
|
||||
const ObQualifiedName &q_name,
|
||||
@ -133,6 +136,7 @@ private:
|
||||
const TableItem *cur_joined_table_;
|
||||
bool check_unique_;
|
||||
common::ObIArray<ResolverJoinInfo> *join_infos_;
|
||||
const ObDMLStmt *dml_stmt_;
|
||||
friend class ObTableItemIterator;
|
||||
};
|
||||
} // namespace sql
|
||||
|
@ -910,6 +910,9 @@ int ObDelUpdResolver::set_base_table_for_updatable_view(TableItem &table_item,
|
||||
} else if (new_table_item->is_fake_cte_table()) {
|
||||
ret = OB_ERR_ILLEGAL_VIEW_UPDATE;
|
||||
LOG_WARN("illegal view update", K(ret));
|
||||
} else if (new_table_item->is_values_table()) {
|
||||
ret = dml->is_insert_stmt() ? OB_ERR_NON_INSERTABLE_TABLE : OB_ERR_NON_UPDATABLE_TABLE;
|
||||
LOG_WARN("view is not updatable", K(ret));
|
||||
} else if (new_table_item->is_json_table()) {
|
||||
ret = OB_ERR_NON_INSERTABLE_TABLE;
|
||||
LOG_WARN("json table can not be insert", K(ret));
|
||||
@ -1010,6 +1013,9 @@ int ObDelUpdResolver::set_base_table_for_view(TableItem &table_item, const bool
|
||||
if (OB_FAIL(SMART_CALL(set_base_table_for_view(*base, inner_log_error)))) {
|
||||
LOG_WARN("set base table for view failed", K(ret));
|
||||
}
|
||||
} else if (base->is_values_table()) {
|
||||
ret = OB_ERR_NON_UPDATABLE_TABLE;
|
||||
LOG_WARN("non update table", K(ret));
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected table type in view", K(ret), K(*base));
|
||||
@ -3263,6 +3269,10 @@ int ObDelUpdResolver::resolve_insert_values(const ParseNode *node,
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
//move generated columns behind basic columns before resolve values
|
||||
OZ (adjust_values_desc_position(table_info, value_idxs));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; i++) {
|
||||
ParseNode *vector_node = node->children_[i];
|
||||
if (OB_ISNULL(vector_node) || OB_ISNULL(vector_node->children_)) {
|
||||
|
@ -61,6 +61,7 @@
|
||||
#include "share/ob_lob_access_utils.h"
|
||||
#include "share/resource_manager/ob_resource_manager.h"
|
||||
#include "share/stat/ob_opt_ds_stat.h"
|
||||
#include "sql/resolver/dml/ob_insert_resolver.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -95,7 +96,8 @@ ObDMLResolver::ObDMLResolver(ObResolverParams ¶ms)
|
||||
parent_cte_tables_(),
|
||||
current_cte_tables_(),
|
||||
expr_resv_ctx_(params.query_ctx_),
|
||||
cte_ctx_()
|
||||
cte_ctx_(),
|
||||
upper_insert_resolver_(NULL)
|
||||
{
|
||||
column_namespace_checker_.set_joininfos(&join_infos_);
|
||||
}
|
||||
@ -2444,6 +2446,7 @@ int ObDMLResolver::resolve_basic_column_ref(const ObQualifiedName &q_name, ObRaw
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("stmt is null", K(ret));
|
||||
} else {
|
||||
column_namespace_checker_.set_dml_stmt(get_stmt());
|
||||
if (OB_FAIL(column_namespace_checker_.check_table_column_namespace(q_name, table_item,
|
||||
stmt->is_insert_all_stmt()))) {
|
||||
LOG_WARN_IGNORE_COL_NOTFOUND(ret, "check basic column namespace failed", K(ret), K(q_name));
|
||||
@ -2459,6 +2462,9 @@ int ObDMLResolver::resolve_basic_column_ref(const ObQualifiedName &q_name, ObRaw
|
||||
if (OB_FAIL(resolve_generated_table_column_item(*table_item, q_name.col_name_, column_item))) {
|
||||
LOG_WARN("resolve column item failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(table_item->type_));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
real_ref_expr = column_item->expr_;
|
||||
@ -3713,8 +3719,7 @@ int ObDMLResolver::resolve_table(const ParseNode &parse_tree,
|
||||
} else {
|
||||
bool tmp_have_same_table = params_.have_same_table_name_;
|
||||
params_.have_same_table_name_ = false;
|
||||
ObString alias_name(alias_node->str_len_, alias_node->str_value_);
|
||||
if (OB_FAIL(resolve_generate_table(*table_node, alias_name, table_item))) {
|
||||
if (OB_FAIL(resolve_generate_table(*table_node, alias_node, table_item))) {
|
||||
LOG_WARN("resolve generate table failed", K(ret));
|
||||
} else if (OB_FAIL(resolve_transpose_table(transpose_node, table_item))) {
|
||||
LOG_WARN("resolve_transpose_table failed", K(ret));
|
||||
@ -3777,6 +3782,12 @@ int ObDMLResolver::resolve_table(const ParseNode &parse_tree,
|
||||
OZ (resolve_json_table_item(*table_node, table_item));
|
||||
break;
|
||||
}
|
||||
case T_VALUES_TABLE_EXPRESSION: {
|
||||
if (OB_FAIL(resolve_values_table_item(*table_node, table_item))) {
|
||||
LOG_WARN("failed to resolve values table item", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
/* won't be here */
|
||||
ret = OB_ERR_PARSER_SYNTAX;
|
||||
@ -4177,23 +4188,26 @@ int ObDMLResolver::resolve_table_column_expr(const ObQualifiedName &q_name, ObRa
|
||||
real_ref_expr = cast_expr;
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(column_namespace_checker_.check_table_column_namespace(q_name, table_item,
|
||||
get_stmt()->is_insert_all_stmt()))) {
|
||||
LOG_WARN_IGNORE_COL_NOTFOUND(ret, "column not found in table", K(ret), K(q_name));
|
||||
} else if (table_item->is_joined_table()) {
|
||||
const JoinedTable &joined_table = static_cast<const JoinedTable&>(*table_item);
|
||||
if (OB_FAIL(resolve_join_table_column_item(joined_table, q_name.col_name_, real_ref_expr))) {
|
||||
LOG_WARN("resolve join table column item failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
ColumnItem *col_item = NULL;
|
||||
if (OB_FAIL(resolve_single_table_column_item(*table_item, q_name.col_name_, false, col_item))) {
|
||||
LOG_WARN("resolve single table column item failed", K(ret));
|
||||
} else if (OB_ISNULL(col_item)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("col item is null", K(ret), K(lbt()));
|
||||
column_namespace_checker_.set_dml_stmt(get_stmt());
|
||||
if (OB_FAIL(column_namespace_checker_.check_table_column_namespace(q_name, table_item,
|
||||
get_stmt()->is_insert_all_stmt()))) {
|
||||
LOG_WARN_IGNORE_COL_NOTFOUND(ret, "column not found in table", K(ret), K(q_name));
|
||||
} else if (table_item->is_joined_table()) {
|
||||
const JoinedTable &joined_table = static_cast<const JoinedTable&>(*table_item);
|
||||
if (OB_FAIL(resolve_join_table_column_item(joined_table, q_name.col_name_, real_ref_expr))) {
|
||||
LOG_WARN("resolve join table column item failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
real_ref_expr = col_item->expr_;
|
||||
ColumnItem *col_item = NULL;
|
||||
if (OB_FAIL(resolve_single_table_column_item(*table_item, q_name.col_name_, false, col_item))) {
|
||||
LOG_WARN("resolve single table column item failed", K(ret));
|
||||
} else if (OB_ISNULL(col_item)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("col item is null", K(ret), K(lbt()));
|
||||
} else {
|
||||
real_ref_expr = col_item->expr_;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -4226,6 +4240,11 @@ int ObDMLResolver::resolve_single_table_column_item(const TableItem &table_item,
|
||||
if (OB_FAIL(resolve_json_table_column_item(table_item, column_name, col_item))) {
|
||||
LOG_WARN("resolve function table column failed", K(ret), K(column_name));
|
||||
}
|
||||
} else if (table_item.is_values_table()) {
|
||||
if (OB_ISNULL(col_item = stmt->get_column_item(table_item.table_id_, column_name))) {
|
||||
ret = OB_ERR_BAD_FIELD_ERROR;
|
||||
LOG_WARN("not found column in table values", K(ret), K(column_name));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -4381,7 +4400,7 @@ int ObDMLResolver::check_special_join_table(const TableItem &join_table, bool is
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
TableItem *&table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -4409,19 +4428,21 @@ int ObDMLResolver::resolve_generate_table(const ParseNode &table_node,
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(select_resolver.add_parent_gen_col_exprs(gen_col_exprs_))) {
|
||||
LOG_WARN("failed to add gen col exprs", K(ret));
|
||||
} else if (OB_FAIL(do_resolve_generate_table(table_node, alias_name, select_resolver, table_item))) {
|
||||
} else if (OB_FAIL(do_resolve_generate_table(table_node, alias_node, select_resolver, table_item))) {
|
||||
LOG_WARN("do resolve generated table failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::do_resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
ObChildStmtResolver &child_resolver,
|
||||
TableItem *&table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSelectStmt *ref_stmt = NULL;
|
||||
ObString alias_name;
|
||||
const ParseNode *column_alias_node = NULL;
|
||||
/*oracle模式允许sel/upd/del stmt中的generated table含有重复列,只要外层没有引用到重复列就行,同时对于外层引用
|
||||
* 到的列是否为重复列会在检查column时进行检测,eg: select 1 from (select c1,c1 from t1);
|
||||
* 因此对于oracle模式下sel/upd/del stmt进行检测时,检测到重复列时只需skip,但是仍然需要添加相关plan cache约束
|
||||
@ -4436,6 +4457,29 @@ int ObDMLResolver::do_resolve_generate_table(const ParseNode &table_node,
|
||||
} else if (lib::is_oracle_mode() && ref_stmt->has_for_update()) {
|
||||
ret = OB_ERR_PARSER_SYNTAX;
|
||||
LOG_WARN("for update not allowed in from clause", K(ret));
|
||||
} else if (OB_ISNULL(alias_node)) {
|
||||
//do nothing
|
||||
} else if (alias_node->type_ == T_IDENT) {
|
||||
alias_name.assign_ptr(alias_node->str_value_, alias_node->str_len_);
|
||||
} else if (alias_node->type_ == T_LINK_NODE) {
|
||||
if (OB_UNLIKELY(alias_node->num_child_ != 2) ||
|
||||
OB_ISNULL(alias_node->children_) ||
|
||||
OB_UNLIKELY(alias_node->children_[0]->type_ != T_IDENT ||
|
||||
alias_node->children_[1]->type_ != T_COLUMN_LIST)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret));
|
||||
} else {
|
||||
alias_name.assign_ptr(alias_node->children_[0]->str_value_, alias_node->children_[0]->str_len_);
|
||||
column_alias_node = alias_node->children_[1];
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(alias_node->type_), K(ret));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (column_alias_node != NULL &&
|
||||
OB_FAIL(refine_generate_table_column_name(*column_alias_node, *ref_stmt))) {
|
||||
LOG_WARN("failed to to refine generate table column name", K(ret));
|
||||
} else if (OB_FAIL(ObResolverUtils::check_duplicated_column(*ref_stmt, can_skip))) {
|
||||
// check duplicate column name for genereated table
|
||||
LOG_WARN("check duplicated column failed", K(ret));
|
||||
@ -10075,10 +10119,14 @@ int ObDMLResolver::expand_transpose(const ObSqlString &transpose_def,
|
||||
select_resolver.set_parent_namespace_resolver(parent_namespace_resolver_);
|
||||
select_resolver.set_current_view_level(current_view_level_);
|
||||
select_resolver.set_transpose_item(&transpose_item);
|
||||
ParseNode tmp_node;
|
||||
tmp_node.type_ = T_IDENT;
|
||||
tmp_node.str_len_ = transpose_item.alias_name_.length();
|
||||
tmp_node.str_value_ = transpose_item.alias_name_.ptr();
|
||||
if (OB_FAIL(add_cte_table_to_children(select_resolver))) {
|
||||
LOG_WARN("add_cte_table_to_children failed", K(ret));
|
||||
} else if (OB_FAIL(do_resolve_generate_table(*transpose_result.result_tree_->children_[0],
|
||||
transpose_item.alias_name_,
|
||||
&tmp_node,
|
||||
select_resolver,
|
||||
table_item))) {
|
||||
LOG_WARN("do_resolve_generate_table failed", K(ret));
|
||||
@ -14994,5 +15042,594 @@ int ObDMLResolver::resolve_table_dynamic_sampling_hint(const ParseNode &hint_nod
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_values_table_item(const ParseNode &table_node, TableItem *&table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_with_cte = false;
|
||||
ObDMLStmt *dml_stmt = get_stmt();
|
||||
TableItem *new_table_item = NULL;
|
||||
ParseNode *alias_node = NULL;
|
||||
ObString alias_name;
|
||||
uint64_t data_version = 0;
|
||||
if (OB_ISNULL(dml_stmt) || OB_ISNULL(allocator_) || OB_ISNULL(session_info_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret));
|
||||
} else if (OB_FAIL(GET_MIN_DATA_VERSION(session_info_->get_effective_tenant_id(), data_version))) {
|
||||
LOG_WARN("get tenant data version failed", K(ret), K(session_info_->get_effective_tenant_id()));
|
||||
} else if (data_version < DATA_VERSION_4_2_1_0) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("VALUES STATEMENT is not supported", K(ret), K(data_version));
|
||||
LOG_USER_ERROR(OB_NOT_SUPPORTED, "VALUES STATEMENT");
|
||||
} else if (OB_ISNULL(new_table_item = dml_stmt->create_table_item(*allocator_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("create table item failed");
|
||||
} else if (NULL != alias_node) {
|
||||
alias_name.assign_ptr((char *)(alias_node->str_value_), static_cast<int32_t>(alias_node->str_len_));
|
||||
} else if (NULL == alias_node) {
|
||||
if (OB_FAIL(dml_stmt->generate_values_table_name(*allocator_, alias_name))) {
|
||||
LOG_WARN("failed to generate func table name", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t column_cnt = 0;
|
||||
//common values table: values row(...), row(...),...
|
||||
if (upper_insert_resolver_ == NULL &&
|
||||
OB_FAIL(resolve_table_values_for_select(table_node,
|
||||
new_table_item->table_values_,
|
||||
column_cnt))) {
|
||||
LOG_WARN("failed to resolve table values for select", K(ret));
|
||||
//insert values table: insert into ....values row(...), row(...),...
|
||||
} else if (upper_insert_resolver_ != NULL &&
|
||||
OB_FAIL(resolve_table_values_for_insert(table_node,
|
||||
new_table_item->table_values_,
|
||||
column_cnt))) {
|
||||
LOG_WARN("failed to resolve table values for insert", K(ret));
|
||||
} else {
|
||||
new_table_item->table_id_ = generate_table_id();
|
||||
new_table_item->table_name_ = alias_name;
|
||||
new_table_item->alias_name_ = alias_name;
|
||||
new_table_item->type_ = TableItem::VALUES_TABLE;
|
||||
new_table_item->is_view_table_ = false;
|
||||
if (OB_FAIL(dml_stmt->add_table_item(session_info_, new_table_item))) {
|
||||
LOG_WARN("add table item failed", K(ret));
|
||||
} else if (OB_FAIL(gen_values_table_column_items(column_cnt, *new_table_item))) {
|
||||
LOG_WARN("failed to gen values table column items", K(ret));
|
||||
} else {
|
||||
table_item = new_table_item;
|
||||
LOG_TRACE("succeed to resolve values table item", KPC(table_item));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_table_values_for_select(const ParseNode &table_node,
|
||||
ObIArray<ObRawExpr*> &table_values,
|
||||
int64_t &column_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ParseNode *values_node = NULL;
|
||||
ObSEArray<int64_t, 8> value_idxs;
|
||||
column_cnt = 0;
|
||||
if (OB_UNLIKELY(T_VALUES_TABLE_EXPRESSION != table_node.type_ || 1 != table_node.num_child_) ||
|
||||
OB_ISNULL(table_node.children_) || OB_ISNULL(values_node = table_node.children_[0]) ||
|
||||
OB_UNLIKELY(T_VALUES_ROW_LIST != values_node->type_) ||
|
||||
OB_ISNULL(params_.expr_factory_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(values_node), K(table_node.type_),
|
||||
K(table_node.num_child_), K(params_.expr_factory_));
|
||||
} else {
|
||||
ObSEArray<ObExprResType, 8> res_types;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values_node->num_child_; i++) {
|
||||
ParseNode *vector_node = values_node->children_[i];
|
||||
if (OB_ISNULL(vector_node) ||
|
||||
OB_ISNULL(vector_node->children_) ||
|
||||
OB_UNLIKELY(T_VALUE_VECTOR != vector_node->type_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("invalid node children", K(i), K(vector_node), K(ret));
|
||||
} else if (OB_UNLIKELY(vector_node->num_child_ > common::OB_USER_ROW_MAX_COLUMNS_COUNT)) {
|
||||
ret = OB_ERR_TOO_MANY_COLUMNS;
|
||||
LOG_WARN("too many columns", K(ret));
|
||||
} else {
|
||||
column_cnt = (i == 0 ? vector_node->num_child_ : column_cnt);
|
||||
if (OB_UNLIKELY(vector_node->num_child_ != column_cnt)) {
|
||||
ret = OB_ERR_COULUMN_VALUE_NOT_MATCH;
|
||||
LOG_USER_ERROR(OB_ERR_COULUMN_VALUE_NOT_MATCH, column_cnt);
|
||||
LOG_WARN("Column count doesn't match value count",
|
||||
"num_child", vector_node->num_child_,
|
||||
"values desc count", column_cnt);
|
||||
} else {
|
||||
ObSEArray<ObExprResType, 8> cur_values_types;
|
||||
ObSEArray<ObRawExpr*, 8> cur_values_vector;
|
||||
for (int32_t j = 0; OB_SUCC(ret) && j < vector_node->num_child_; j++) {
|
||||
ObRawExpr *expr = NULL;
|
||||
const ParseNode *value_node = vector_node->children_[j];
|
||||
if (OB_ISNULL(value_node)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("inalid children node", K(j), K(vector_node));
|
||||
} else if (T_EMPTY == value_node->type_) {
|
||||
ret = OB_ERR_VALUES_CLAUSE_NEED_HAVE_COLUMN;
|
||||
LOG_WARN("Each row of a VALUES clause must have at least one column, unless when used as source in an INSERT statement.", K(ret));
|
||||
} else if (OB_FAIL(resolve_sql_expr(*value_node, expr))) {
|
||||
LOG_WARN("resolve sql expr failed", K(ret));
|
||||
} else if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to resolve sql expr", K(ret), K(expr));
|
||||
} else if (expr->get_expr_type() == T_DEFAULT) {
|
||||
ret = OB_ERR_VALUES_CLAUSE_CANNOT_USE_DEFAULT_VALUES;
|
||||
LOG_WARN("A VALUES clause cannot use DEFAULT values, unless used as a source in an INSERT statement.", K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(expr));
|
||||
} else if (OB_FAIL(expr->formalize(params_.session_info_))) {
|
||||
LOG_WARN("failed to formailize expr", K(ret));
|
||||
} else if (OB_FAIL(cur_values_vector.push_back(expr))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(cur_values_types.push_back(expr->get_result_type()))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to resolve one table values", KPC(expr));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(append(table_values, cur_values_vector))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else if (i == 0) {
|
||||
if (OB_FAIL(append(res_types, cur_values_types))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else {/*do nothing*/}
|
||||
} else {
|
||||
if (OB_FAIL(get_values_res_types(cur_values_types, res_types))) {
|
||||
LOG_WARN("failed to get values res types", K(ret));
|
||||
}
|
||||
}
|
||||
LOG_TRACE("succeed to resolve table values", K(table_values), K(res_types),
|
||||
K(cur_values_vector), K(cur_values_types));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(try_add_cast_to_values(res_types, table_values))) {
|
||||
LOG_WARN("failed to try add cast to values", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to resolve table values", K(table_values), K(res_types));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_table_values_for_insert(const ParseNode &table_node,
|
||||
ObIArray<ObRawExpr*> &table_values,
|
||||
int64_t &column_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ParseNode *values_node = NULL;
|
||||
ObSEArray<int64_t, 8> value_idxs;
|
||||
ObInsertStmt *insert_stmt = NULL;
|
||||
bool is_all_default = false;
|
||||
column_cnt = 0;
|
||||
if (OB_UNLIKELY(T_VALUES_TABLE_EXPRESSION != table_node.type_ || 1 != table_node.num_child_) ||
|
||||
OB_ISNULL(table_node.children_) || OB_ISNULL(values_node = table_node.children_[0]) ||
|
||||
OB_UNLIKELY(T_VALUE_LIST != values_node->type_) ||
|
||||
OB_ISNULL(params_.expr_factory_) ||
|
||||
OB_ISNULL(upper_insert_resolver_) ||
|
||||
OB_ISNULL(insert_stmt = upper_insert_resolver_->get_insert_stmt())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(values_node), K(table_node.type_),
|
||||
K(table_node.num_child_), K(params_.expr_factory_),
|
||||
K(upper_insert_resolver_), K(insert_stmt));
|
||||
} else if (OB_FAIL(adjust_values_desc_position(insert_stmt->get_insert_table_info(),
|
||||
value_idxs))) {
|
||||
LOG_WARN("failed to adjust values desc position", K(ret));
|
||||
} else {
|
||||
bool is_all_default = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values_node->num_child_; i++) {
|
||||
ParseNode *vector_node = values_node->children_[i];
|
||||
if (OB_ISNULL(vector_node) ||
|
||||
OB_ISNULL(vector_node->children_) ||
|
||||
OB_UNLIKELY(T_VALUE_VECTOR != vector_node->type_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("invalid node children", K(i), K(vector_node), K(ret));
|
||||
} else if (OB_UNLIKELY(vector_node->num_child_ > common::OB_USER_ROW_MAX_COLUMNS_COUNT)) {
|
||||
ret = OB_ERR_TOO_MANY_COLUMNS;
|
||||
LOG_WARN("too many columns", K(ret));
|
||||
} else {
|
||||
//insert into table values row(), row()...
|
||||
bool tmp_is_all_default = vector_node->num_child_ == 1 &&
|
||||
vector_node->children_[0] != NULL &&
|
||||
T_EMPTY == vector_node->children_[0]->type_;
|
||||
ObSEArray<ObRawExpr*, 8> cur_values_vector;
|
||||
ObInsertTableInfo &table_info = insert_stmt->get_insert_table_info();
|
||||
is_all_default = (i == 0 ? tmp_is_all_default : is_all_default);
|
||||
column_cnt = ((i == 0 || is_all_default) ? vector_node->num_child_ : column_cnt);
|
||||
if (OB_UNLIKELY(vector_node->num_child_ != column_cnt ||
|
||||
tmp_is_all_default != is_all_default)) {
|
||||
ret = OB_ERR_COULUMN_VALUE_NOT_MATCH;
|
||||
LOG_USER_ERROR(OB_ERR_COULUMN_VALUE_NOT_MATCH, column_cnt);
|
||||
LOG_WARN("Column count doesn't match value count",
|
||||
"num_child", vector_node->num_child_,
|
||||
"values desc count", column_cnt);
|
||||
} else if (is_all_default) {
|
||||
if (OB_FAIL(build_row_for_empty_values(cur_values_vector))) {
|
||||
LOG_WARN( "fail to build row for empty values", K(ret));
|
||||
} else {
|
||||
column_cnt = cur_values_vector.count();//set the real values desc count
|
||||
LOG_TRACE("succeed to resolve empty values", K(cur_values_vector));
|
||||
}
|
||||
} else if (OB_UNLIKELY(table_info.values_desc_.count() != value_idxs.count() ||
|
||||
value_idxs.count() != vector_node->num_child_)) {
|
||||
ret = OB_ERR_COULUMN_VALUE_NOT_MATCH;
|
||||
LOG_USER_ERROR(OB_ERR_COULUMN_VALUE_NOT_MATCH, table_info.values_desc_.count());
|
||||
LOG_WARN("Column count doesn't match value count",
|
||||
"num_child", vector_node->num_child_,
|
||||
"values desc count", table_info.values_desc_.count());
|
||||
} else {
|
||||
for (int32_t j = 0; OB_SUCC(ret) && j < vector_node->num_child_; j++) {
|
||||
ObRawExpr *expr = NULL;
|
||||
const ParseNode *value_node = vector_node->children_[value_idxs.at(j)];
|
||||
if (OB_ISNULL(value_node)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("inalid children node", K(j), K(vector_node));
|
||||
} else if (OB_FAIL(resolve_sql_expr(*value_node, expr))) {
|
||||
LOG_WARN("resolve sql expr failed", K(ret));
|
||||
} else if (OB_ISNULL(expr) || OB_ISNULL(table_info.values_desc_.at(j))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to resolve sql expr", K(ret), KP(expr), KP(table_info.values_desc_.at(j)));
|
||||
} else if (table_info.values_desc_.at(j)->is_generated_column() &&
|
||||
expr->get_expr_type() != T_DEFAULT) {
|
||||
ret = OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN;
|
||||
LOG_WARN("non-default value for generated column is not allowed", K(ret));
|
||||
ColumnItem *orig_col_item = NULL;
|
||||
uint64_t column_id = table_info.values_desc_.at(j)->get_column_id();
|
||||
if (NULL != (orig_col_item = insert_stmt->get_column_item_by_id(table_info.table_id_,
|
||||
column_id))
|
||||
&& orig_col_item->expr_ != NULL) {
|
||||
const ObString &column_name = orig_col_item->expr_->get_column_name();
|
||||
const ObString &table_name = orig_col_item->expr_->get_table_name();
|
||||
LOG_USER_ERROR(OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN,
|
||||
column_name.length(), column_name.ptr(),
|
||||
table_name.length(), table_name.ptr());
|
||||
}
|
||||
} else if (expr->get_expr_type() == T_DEFAULT) {
|
||||
ColumnItem *column_item = NULL;
|
||||
uint64_t column_id = table_info.values_desc_.at(j)->get_column_id();
|
||||
if (OB_ISNULL(column_item = insert_stmt->get_column_item_by_id(table_info.table_id_,
|
||||
column_id))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get column item by id failed", K(table_info.table_id_), K(column_id), K(ret));
|
||||
} else if (table_info.values_desc_.at(j)->is_generated_column()) {
|
||||
if (OB_FAIL(copy_schema_expr(*params_.expr_factory_,
|
||||
column_item->expr_->get_dependant_expr(),
|
||||
expr))) {
|
||||
LOG_WARN("copy expr failed", K(ret));
|
||||
} else if (expr->has_flag(CNT_COLUMN)) {
|
||||
if (OB_FAIL(replace_column_ref(cur_values_vector, table_info.values_desc_, expr))) {
|
||||
LOG_WARN("replace column reference failed", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to replace column ref", KPC(expr), K(cur_values_vector), K(table_info.values_desc_));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ObDefaultValueUtils utils(insert_stmt, ¶ms_, upper_insert_resolver_);
|
||||
if (OB_FAIL(utils.resolve_default_expr(*column_item, expr, T_INSERT_SCOPE))) {
|
||||
LOG_WARN("fail to resolve default value", "table_id", table_info.table_id_, K(column_id), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(expr));
|
||||
} else if (OB_FAIL(expr->formalize(params_.session_info_))) {
|
||||
LOG_WARN("failed to formailize expr", K(ret));
|
||||
} else if (OB_FAIL(cur_values_vector.push_back(expr))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to resolve one table values", KPC(expr));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(append(table_values, cur_values_vector))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to resolve one row", K(cur_values_vector), K(table_values));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::get_values_res_types(const ObIArray<ObExprResType> &cur_values_types,
|
||||
ObIArray<ObExprResType> &res_types)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(session_info_) || OB_ISNULL(allocator_) ||
|
||||
OB_UNLIKELY(res_types.count() != cur_values_types.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(res_types), K(cur_values_types),
|
||||
K(session_info_), K(allocator_), K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < res_types.count(); ++i) {
|
||||
ObExprVersion dummy_op(*allocator_);
|
||||
ObExprResType new_res_type;
|
||||
ObSEArray<ObExprResType, 2> tmp_types;
|
||||
const ObLengthSemantics length_semantics = session_info_->get_actual_nls_length_semantics();
|
||||
ObCollationType coll_type = CS_TYPE_INVALID;
|
||||
if (OB_FAIL(tmp_types.push_back(res_types.at(i))) ||
|
||||
OB_FAIL(tmp_types.push_back(cur_values_types.at(i)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(session_info_->get_collation_connection(coll_type))) {
|
||||
LOG_WARN("fail to get_collation_connection", K(ret));
|
||||
} else if (OB_FAIL(dummy_op.aggregate_result_type_for_merge(new_res_type, &tmp_types.at(0),
|
||||
tmp_types.count(), coll_type, false,
|
||||
length_semantics, session_info_))) {
|
||||
LOG_WARN("failed to aggregate result type for merge", K(ret));
|
||||
} else {
|
||||
res_types.at(i) = new_res_type;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::try_add_cast_to_values(const ObIArray<ObExprResType> &res_types,
|
||||
ObIArray<ObRawExpr*> &values_vector)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(res_types.empty() || values_vector.count() % res_types.count() != 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(res_types), K(values_vector), K(ret));
|
||||
} else {
|
||||
int64_t res_cnt = res_types.count();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < values_vector.count(); ++i) {
|
||||
ObRawExpr *new_expr = NULL;
|
||||
int64_t j = i % res_cnt;
|
||||
if (OB_ISNULL(values_vector.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(values_vector.at(i)));
|
||||
} else if (values_vector.at(i)->get_result_type() == res_types.at(j)) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(ObRawExprUtils::try_add_cast_expr_above(params_.expr_factory_,
|
||||
session_info_,
|
||||
*values_vector.at(i),
|
||||
res_types.at(j),
|
||||
new_expr))) {
|
||||
LOG_WARN("create cast expr for stmt failed", K(ret));
|
||||
} else if (values_vector.at(i) == new_expr) {
|
||||
/*do nothing*/
|
||||
} else if (OB_ISNULL(new_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(new_expr));
|
||||
} else if (OB_FAIL(new_expr->add_flag(IS_INNER_ADDED_EXPR))) {
|
||||
LOG_WARN("failed to add flag", K(ret));
|
||||
} else {
|
||||
values_vector.at(i) = new_expr;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::gen_values_table_column_items(const int64_t column_cnt, TableItem &table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(params_.expr_factory_) || OB_ISNULL(allocator_) || OB_ISNULL(get_stmt()) ||
|
||||
OB_UNLIKELY(column_cnt <= 0 || table_item.table_values_.empty() ||
|
||||
table_item.table_values_.count() % column_cnt != 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(column_cnt), K(params_.expr_factory_),
|
||||
K(table_item.table_values_), K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < column_cnt; ++i) {
|
||||
ObColumnRefRawExpr *column_expr = NULL;
|
||||
if (OB_FAIL(params_.expr_factory_->create_raw_expr(T_REF_COLUMN, column_expr))) {
|
||||
LOG_WARN("create column ref raw expr failed", K(ret));
|
||||
} else if (OB_ISNULL(column_expr) || OB_ISNULL(table_item.table_values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN(("value desc is null"));
|
||||
} else {
|
||||
column_expr->set_result_type(table_item.table_values_.at(i)->get_result_type());
|
||||
column_expr->set_result_flag(table_item.table_values_.at(i)->get_result_flag());
|
||||
column_expr->set_ref_id(table_item.table_id_, i + OB_APP_MIN_COLUMN_ID);
|
||||
// compatible Mysql8.0, column name is column_0, column_1, ...
|
||||
ObSqlString tmp_col_name;
|
||||
char *buf = NULL;
|
||||
if (OB_FAIL(tmp_col_name.append_fmt("column_%ld", i))) {
|
||||
LOG_WARN("failed to append fmt", K(ret));
|
||||
} else if (OB_ISNULL(buf = static_cast<char*>(allocator_->alloc(tmp_col_name.length())))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocate memory", K(ret), K(buf));
|
||||
} else {
|
||||
MEMCPY(buf, tmp_col_name.ptr(), tmp_col_name.length());
|
||||
ObString column_name(tmp_col_name.length(), buf);
|
||||
column_expr->set_column_attr(table_item.table_name_, column_name);
|
||||
if (ob_is_enumset_tc(table_item.table_values_.at(i)->get_result_type().get_type())
|
||||
&& OB_FAIL(column_expr->set_enum_set_values(table_item.table_values_.at(i)->get_enum_set_values()))) {
|
||||
LOG_WARN("failed to set_enum_set_values", K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(column_expr->add_flag(IS_COLUMN))) {
|
||||
LOG_WARN("failed to add flag IS_COLUMN", K(ret));
|
||||
} else {
|
||||
ColumnItem column_item;
|
||||
column_item.expr_ = column_expr;
|
||||
column_item.table_id_ = column_expr->get_table_id();
|
||||
column_item.column_id_ = column_expr->get_column_id();
|
||||
column_item.column_name_ = column_expr->get_column_name();
|
||||
if (OB_FAIL(get_stmt()->add_column_item(column_item))) {
|
||||
LOG_WARN("failed to add column item", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("succeed to gen table values desc", K(column_name), KPC(column_expr));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::refine_generate_table_column_name(const ParseNode &column_alias_node,
|
||||
ObSelectStmt &select_stmt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(column_alias_node.type_ != T_COLUMN_LIST) ||
|
||||
OB_ISNULL(column_alias_node.children_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(column_alias_node.type_), K(ret));
|
||||
} else if (OB_UNLIKELY(column_alias_node.num_child_ != select_stmt.get_select_item_size())) {
|
||||
ret = OB_ERR_COULUMN_VALUE_NOT_MATCH;
|
||||
LOG_WARN("column count does not match value count", K(column_alias_node.num_child_), K(ret),
|
||||
K(select_stmt.get_select_item_size()));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < column_alias_node.num_child_; ++i) {
|
||||
if (OB_ISNULL(column_alias_node.children_[i]) ||
|
||||
OB_UNLIKELY(column_alias_node.children_[i]->type_ != T_IDENT)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret));
|
||||
} else {
|
||||
SelectItem &select_item = select_stmt.get_select_item(i);
|
||||
select_item.alias_name_.assign_ptr(column_alias_node.children_[i]->str_value_,
|
||||
column_alias_node.children_[i]->str_len_);
|
||||
select_item.is_real_alias_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::replace_column_ref(ObIArray<ObRawExpr*> &values_vector,
|
||||
ObIArray<ObColumnRefRawExpr*> &values_desc,
|
||||
ObRawExpr *&expr)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(expr), K(ret));
|
||||
} else if (!expr->has_flag(CNT_COLUMN)) {
|
||||
//do nothing
|
||||
} else if (expr->get_param_count() > 0) {
|
||||
for (int i = 0; OB_SUCC(ret) && i < expr->get_param_count(); i++) {
|
||||
if (OB_FAIL(SMART_CALL(replace_column_ref(values_vector, values_desc, expr->get_param_expr(i))))) {
|
||||
LOG_WARN("fail to postorder_spread", K(ret), K(expr->get_param_expr(i)));
|
||||
}
|
||||
}
|
||||
} else if (expr->is_column_ref_expr()) {
|
||||
int64_t value_index = -1;
|
||||
if (values_desc.count() < values_vector.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid array size", K(values_vector), K(values_desc));
|
||||
} else {
|
||||
uint64_t column_id = static_cast<ObColumnRefRawExpr*>(expr)->get_column_id();
|
||||
bool found_it = false;
|
||||
for(int64_t i = 0; !found_it && i < values_vector.count(); i++) {
|
||||
if (values_desc.at(i)->get_column_id() == column_id) {
|
||||
expr = values_vector.at(i);
|
||||
found_it = true;
|
||||
}
|
||||
}
|
||||
if (!found_it) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(values_vector), K(values_desc), KPC(expr));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::build_row_for_empty_values(ObIArray<ObRawExpr*> &values_vector)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObInsertStmt *insert_stmt = NULL;
|
||||
if (OB_ISNULL(upper_insert_resolver_) ||
|
||||
OB_ISNULL(insert_stmt = upper_insert_resolver_->get_insert_stmt())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get null stmt", K(insert_stmt), K(ret));
|
||||
} else {
|
||||
ColumnItem *item = NULL;
|
||||
ObDefaultValueUtils utils(insert_stmt, ¶ms_, static_cast<ObDMLResolver*>(upper_insert_resolver_));
|
||||
ObInsertTableInfo &table_info = insert_stmt->get_insert_table_info();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < table_info.values_desc_.count(); ++i) {
|
||||
ObRawExpr *expr = NULL;
|
||||
int64_t column_id = table_info.values_desc_.at(i)->get_column_id();
|
||||
if (OB_ISNULL(item = insert_stmt->get_column_item_by_id(table_info.table_id_, column_id))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get column item", K(column_id));
|
||||
} else if (OB_UNLIKELY(item->expr_->is_generated_column())) {
|
||||
if (OB_FAIL(copy_schema_expr(*params_.expr_factory_,
|
||||
item->expr_->get_dependant_expr(),
|
||||
expr))) {
|
||||
LOG_WARN("copy generated column dependant expr failed", K(ret));
|
||||
} else if (expr->has_flag(CNT_COLUMN)) {
|
||||
if (OB_FAIL(replace_column_ref(values_vector, table_info.values_desc_, expr))) {
|
||||
LOG_WARN("replace column reference failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(values_vector.push_back(expr))) {
|
||||
LOG_WARN("fail to push back value expr", K(ret));
|
||||
}
|
||||
} else if (item->is_auto_increment()) {
|
||||
if (OB_FAIL(ObRawExprUtils::build_null_expr(*params_.expr_factory_, expr))) {
|
||||
LOG_WARN("failed to build next_val expr as null", K(ret));
|
||||
} else if (OB_FAIL(values_vector.push_back(expr))) {
|
||||
LOG_WARN("fail to push back value expr", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(utils.generate_insert_value(item, expr))) {
|
||||
LOG_WARN("fail to generate insert values", K(ret), K(column_id));
|
||||
} else if (OB_FAIL(values_vector.push_back(expr))) {
|
||||
LOG_WARN("fail to push back value expr", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
//move generated columns behind basic columns before resolve values
|
||||
int ObDMLResolver::adjust_values_desc_position(ObInsertTableInfo& table_info,
|
||||
ObIArray<int64_t> &value_idxs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObArray<ObColumnRefRawExpr*> tmp_values_desc;
|
||||
if (OB_FAIL(value_idxs.reserve(table_info.values_desc_.count()))) {
|
||||
LOG_WARN("fail to reserve memory", K(ret));
|
||||
} else if (OB_FAIL(tmp_values_desc.reserve(table_info.values_desc_.count()))) {
|
||||
LOG_WARN("fail to reserve memory", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < 2; ++i) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < table_info.values_desc_.count(); ++j) {
|
||||
if (OB_ISNULL(table_info.values_desc_.at(j))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("inalid value desc", K(j), K(table_info.values_desc_));
|
||||
} else if ((i == 0 && !table_info.values_desc_.at(j)->is_generated_column())
|
||||
|| (i == 1 && table_info.values_desc_.at(j)->is_generated_column())) {
|
||||
if (OB_FAIL(tmp_values_desc.push_back(table_info.values_desc_.at(j)))) {
|
||||
LOG_WARN("fail to push back values_desc_", K(ret));
|
||||
} else if (OB_FAIL(value_idxs.push_back(j))) {
|
||||
LOG_WARN("fail to push back value index", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
table_info.values_desc_.reuse();
|
||||
if (OB_FAIL(append(table_info.values_desc_, tmp_values_desc))) {
|
||||
LOG_WARN("fail to append new values_desc");
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
@ -29,6 +29,7 @@ class ObEqualAnalysis;
|
||||
class ObChildStmtResolver;
|
||||
class ObDelUpdStmt;
|
||||
class ObSelectResolver;
|
||||
class ObInsertResolver;
|
||||
|
||||
static const char *err_log_default_columns_[] = { "ORA_ERR_NUMBER$", "ORA_ERR_MESG$", "ORA_ERR_ROWID$", "ORA_ERR_OPTYP$", "ORA_ERR_TAG$" };
|
||||
static char *str_to_lower(char *pszBuf, int64_t length);
|
||||
@ -185,7 +186,7 @@ public:
|
||||
const bool used_for_generated_column = true,
|
||||
ObDMLStmt *stmt = NULL);
|
||||
int do_resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
ObChildStmtResolver &child_resolver,
|
||||
TableItem *&table_item);
|
||||
int resolve_generate_table_item(ObSelectStmt *ref_query, const ObString &alias_name, TableItem *&tbl_item);
|
||||
@ -290,6 +291,8 @@ public:
|
||||
const ObIArray<uint64_t> &object_ids,
|
||||
const ObIArray<uint64_t> &db_ids);
|
||||
ObDMLStmt *get_stmt();
|
||||
void set_upper_insert_resolver(ObInsertResolver *insert_resolver) {
|
||||
upper_insert_resolver_ = insert_resolver; }
|
||||
protected:
|
||||
int generate_pl_data_type(ObRawExpr *expr, pl::ObPLDataType &pl_data_type);
|
||||
int resolve_into_variables(const ParseNode *node,
|
||||
@ -311,11 +314,13 @@ protected:
|
||||
bool include_hidden,
|
||||
ColumnItem *&col_item,
|
||||
ObDMLStmt *stmt = NULL);
|
||||
int adjust_values_desc_position(ObInsertTableInfo& table_info,
|
||||
ObIArray<int64_t> &value_idxs);
|
||||
public:
|
||||
virtual int resolve_table(const ParseNode &parse_tree, TableItem *&table_item);
|
||||
protected:
|
||||
virtual int resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
TableItem *&tbl_item);
|
||||
int check_stmt_has_flashback_query(ObDMLStmt *stmt, bool check_all, bool &has_fq);
|
||||
virtual int resolve_basic_table(const ParseNode &parse_tree, TableItem *&table_item);
|
||||
@ -643,7 +648,6 @@ protected:
|
||||
int32_t parent,
|
||||
int32_t& id,
|
||||
int64_t& column_id);
|
||||
|
||||
int resolve_json_table_column_item(const TableItem &table_item,
|
||||
const ObString &column_name,
|
||||
ColumnItem *&col_item);
|
||||
@ -884,12 +888,31 @@ private:
|
||||
int64_t &table_id,
|
||||
int64_t &ref_id);
|
||||
int check_cast_multiset(const ObRawExpr *expr, const ObRawExpr *parent_expr = NULL);
|
||||
|
||||
int replace_col_udt_qname(ObQualifiedName& q_name);
|
||||
int check_column_udt_type(ParseNode *root_node);
|
||||
|
||||
int replace_pl_relative_expr_to_question_mark(ObRawExpr *&real_ref_expr);
|
||||
bool check_expr_has_colref(ObRawExpr *expr);
|
||||
|
||||
int resolve_values_table_item(const ParseNode &table_node, TableItem *&table_item);
|
||||
int resolve_table_values_for_select(const ParseNode &table_node,
|
||||
ObIArray<ObRawExpr*> &table_values,
|
||||
int64_t &column_cnt);
|
||||
int resolve_table_values_for_insert(const ParseNode &table_node,
|
||||
ObIArray<ObRawExpr*> &table_values,
|
||||
int64_t &column_cnt);
|
||||
int gen_values_table_column_items(const int64_t column_cnt, TableItem &table_item);
|
||||
int get_values_res_types(const ObIArray<ObExprResType> &cur_values_types,
|
||||
ObIArray<ObExprResType> &res_types);
|
||||
int try_add_cast_to_values(const ObIArray<ObExprResType> &res_types,
|
||||
ObIArray<ObRawExpr*> &values_vector);
|
||||
int refine_generate_table_column_name(const ParseNode &column_alias_node,
|
||||
ObSelectStmt &select_stmt);
|
||||
int replace_column_ref(ObIArray<ObRawExpr*> &values_vector,
|
||||
ObIArray<ObColumnRefRawExpr*> &values_desc,
|
||||
ObRawExpr *&expr);
|
||||
int build_row_for_empty_values(ObIArray<ObRawExpr*> &values_vector);
|
||||
protected:
|
||||
struct GenColumnExprInfo {
|
||||
GenColumnExprInfo():
|
||||
@ -958,6 +981,9 @@ protected:
|
||||
common::ObSEArray<ObRawExpr*, 4, common::ModulePageAllocator, true> pseudo_external_file_col_exprs_;
|
||||
//for validity check for on-condition with (+)
|
||||
common::ObSEArray<uint64_t, 4, common::ModulePageAllocator, true> ansi_join_outer_table_id_;
|
||||
|
||||
//for values table used to insert stmt:insert into table values row()....
|
||||
ObInsertResolver *upper_insert_resolver_;
|
||||
protected:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObDMLResolver);
|
||||
};
|
||||
|
@ -282,6 +282,8 @@ int TableItem::deep_copy(ObIRawExprCopier &expr_copier,
|
||||
LOG_WARN("failed to assign part ids", K(ret));
|
||||
} else if (OB_FAIL(part_names_.assign(other.part_names_))) {
|
||||
LOG_WARN("failed to assign part names", K(ret));
|
||||
} else if (OB_FAIL(expr_copier.copy(other.table_values_, table_values_))) {
|
||||
LOG_WARN("failed to deep copy table values", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -854,6 +856,9 @@ int ObDMLStmt::iterate_stmt_expr(ObStmtExprVisitor &visitor)
|
||||
OB_FAIL(visitor.visit(table_items_.at(i)->json_table_def_->doc_expr_,
|
||||
SCOPE_FROM))) {
|
||||
LOG_WARN("failed to add json table doc expr", K(ret));
|
||||
} else if (OB_FAIL(visitor.visit(table_items_.at(i)->table_values_,
|
||||
SCOPE_FROM))) {
|
||||
LOG_WARN("failed to visit table values", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
}
|
||||
|
||||
@ -2446,6 +2451,27 @@ int ObDMLStmt::generate_json_table_name(ObIAllocator &allocator, ObString &table
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLStmt::generate_values_table_name(ObIAllocator &allocator, ObString &table_name)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t pos = 0;
|
||||
const uint64_t OB_MAX_SUBQUERY_NAME_LENGTH = 64;
|
||||
const char *SUBQUERY_VIEW = "VALUES_TABLE";
|
||||
char buf[OB_MAX_SUBQUERY_NAME_LENGTH];
|
||||
int64_t buf_len = OB_MAX_SUBQUERY_NAME_LENGTH;
|
||||
if (OB_FAIL(BUF_PRINTF("%s", SUBQUERY_VIEW))) {
|
||||
LOG_WARN("append name to buf error", K(ret));
|
||||
} else if (OB_FAIL(append_id_to_view_name(buf, OB_MAX_SUBQUERY_NAME_LENGTH, pos, false))) {
|
||||
LOG_WARN("append name to buf error", K(ret));
|
||||
} else {
|
||||
ObString generate_name(pos, buf);
|
||||
if (OB_FAIL(ob_write_string(allocator, generate_name, table_name))) {
|
||||
LOG_WARN("failed to write string", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLStmt::generate_anonymous_view_name(ObIAllocator &allocator, ObString &view_name)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -4554,6 +4580,14 @@ int ObDMLStmt::check_has_cursor_expression(bool &has_cursor_expr) const
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObDMLStmt::is_values_table_query() const
|
||||
{
|
||||
return is_select_stmt() &&
|
||||
table_items_.count() == 1 &&
|
||||
table_items_.at(0) != NULL &&
|
||||
table_items_.at(0)->is_values_table();
|
||||
}
|
||||
|
||||
ObJtColBaseInfo::ObJtColBaseInfo()
|
||||
: col_type_(0),
|
||||
truncate_(0),
|
||||
|
@ -204,7 +204,8 @@ struct TableItem
|
||||
K_(ddl_schema_version), K_(ddl_table_id),
|
||||
K_(is_view_table), K_(part_ids), K_(part_names), K_(cte_type),
|
||||
KPC_(function_table_expr),
|
||||
K_(flashback_query_type), KPC_(flashback_query_expr), K_(table_type));
|
||||
K_(flashback_query_type), KPC_(flashback_query_expr), K_(table_type),
|
||||
K(table_values_));
|
||||
|
||||
enum TableType
|
||||
{
|
||||
@ -219,6 +220,7 @@ struct TableItem
|
||||
LINK_TABLE,
|
||||
JSON_TABLE,
|
||||
EXTERNAL_TABLE,
|
||||
VALUES_TABLE
|
||||
};
|
||||
|
||||
/**
|
||||
@ -254,6 +256,7 @@ struct TableItem
|
||||
bool is_link_table() const { return OB_INVALID_ID != dblink_id_; } // why not use type_, cause type_ will be changed in dblink transform rule, but dblink id don't change
|
||||
bool is_link_type() const { return LINK_TABLE == type_; } // after dblink transformer, LINK_TABLE will be BASE_TABLE, BASE_TABLE will be LINK_TABLE
|
||||
bool is_json_table() const { return JSON_TABLE == type_; }
|
||||
bool is_values_table() const { return VALUES_TABLE == type_; }//used to mark values statement: values row(1,2), row(3,4);
|
||||
bool is_synonym() const { return !synonym_name_.empty(); }
|
||||
bool is_oracle_all_or_user_sys_view() const
|
||||
{
|
||||
@ -337,6 +340,8 @@ struct TableItem
|
||||
common::ObSEArray<ObString, 1, common::ModulePageAllocator, true> part_names_;
|
||||
// json table
|
||||
ObJsonTableDef* json_table_def_;
|
||||
// values table
|
||||
common::ObArray<ObRawExpr*, common::ModulePageAllocator, true> table_values_;
|
||||
};
|
||||
|
||||
struct ColumnItem
|
||||
@ -817,6 +822,15 @@ public:
|
||||
int64_t get_table_size() const { return table_items_.count(); }
|
||||
int64_t get_CTE_table_size() const;
|
||||
int64_t get_column_size() const { return column_items_.count(); }
|
||||
int64_t get_column_size(const uint64_t table_id) const {
|
||||
int64_t size = 0;
|
||||
for (int64_t i = 0; i < column_items_.count(); i++) {
|
||||
if (table_id == column_items_.at(i).table_id_) {
|
||||
++size;
|
||||
}
|
||||
}
|
||||
return size;
|
||||
}
|
||||
inline int64_t get_condition_size() const { return condition_exprs_.count(); }
|
||||
void reset_table_items() { table_items_.reset(); }
|
||||
const ColumnItem *get_column_item(int64_t index) const
|
||||
@ -902,6 +916,7 @@ public:
|
||||
int generate_anonymous_view_name(ObIAllocator &allocator, ObString &view_name);
|
||||
int generate_func_table_name(ObIAllocator &allocator, ObString &table_name);
|
||||
int generate_json_table_name(ObIAllocator &allocator, ObString &table_name);
|
||||
int generate_values_table_name(ObIAllocator &allocator, ObString &table_name);
|
||||
int append_id_to_view_name(char *buf,
|
||||
int64_t buf_len,
|
||||
int64_t &pos,
|
||||
@ -1138,6 +1153,7 @@ public:
|
||||
bool need_replace);
|
||||
|
||||
int check_has_cursor_expression(bool &has_cursor_expr) const;
|
||||
bool is_values_table_query() const;
|
||||
|
||||
int do_formalize_query_ref_exprs_pre();
|
||||
|
||||
|
@ -573,6 +573,8 @@ int ObInsertResolver::resolve_values(const ParseNode &value_node, ObIArray<uint6
|
||||
sub_select_resolver_->set_current_view_level(current_view_level_);
|
||||
//select层不应该看到上层的insert stmt的属性,所以upper scope stmt应该为空
|
||||
sub_select_resolver_->set_parent_namespace_resolver(NULL);
|
||||
//for values stmt: insert into table_name values row()...
|
||||
sub_select_resolver_->set_upper_insert_resolver(this);
|
||||
TableItem *sub_select_table = NULL;
|
||||
ObString view_name;
|
||||
ObSEArray<ColumnItem, 4> column_items;
|
||||
@ -630,17 +632,23 @@ int ObInsertResolver::check_insert_select_field(ObInsertStmt &insert_stmt,
|
||||
is_generated_column))) {
|
||||
LOG_WARN("check basic column generated failed", K(ret));
|
||||
} else if (is_generated_column) {
|
||||
ret = OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN;
|
||||
if (!is_oracle_mode()) {
|
||||
ColumnItem *orig_col_item = NULL;
|
||||
if (NULL != (orig_col_item = insert_stmt.get_column_item_by_id(
|
||||
insert_stmt.get_insert_table_info().table_id_, value_desc->get_column_id()))
|
||||
&& orig_col_item->expr_ != NULL) {
|
||||
const ObString &column_name = orig_col_item->expr_->get_column_name();
|
||||
const ObString &table_name = orig_col_item->expr_->get_table_name();
|
||||
LOG_USER_ERROR(OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN,
|
||||
column_name.length(), column_name.ptr(),
|
||||
table_name.length(), table_name.ptr());
|
||||
if (select_stmt.get_table_size() == 1 &&
|
||||
select_stmt.get_table_item(0) != NULL &&
|
||||
select_stmt.get_table_item(0)->is_values_table()) {
|
||||
//do nothing, already checked in advance.
|
||||
} else {
|
||||
ret = OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN;
|
||||
if (!is_oracle_mode()) {
|
||||
ColumnItem *orig_col_item = NULL;
|
||||
if (NULL != (orig_col_item = insert_stmt.get_column_item_by_id(
|
||||
insert_stmt.get_insert_table_info().table_id_, value_desc->get_column_id()))
|
||||
&& orig_col_item->expr_ != NULL) {
|
||||
const ObString &column_name = orig_col_item->expr_->get_column_name();
|
||||
const ObString &table_name = orig_col_item->expr_->get_table_name();
|
||||
LOG_USER_ERROR(OB_NON_DEFAULT_VALUE_FOR_GENERATED_COLUMN,
|
||||
column_name.length(), column_name.ptr(),
|
||||
table_name.length(), table_name.ptr());
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (!session_info_->is_in_user_scope() && value_desc->is_always_identity_column()) {
|
||||
|
@ -340,12 +340,7 @@ int ObMergeResolver::resolve_table(const ParseNode &parse_tree, TableItem *&tabl
|
||||
break;
|
||||
}
|
||||
case T_SELECT: {
|
||||
alias_node = parse_tree.children_[1];
|
||||
ObString alias_name;
|
||||
if (alias_node != NULL) {
|
||||
alias_name.assign_ptr((char *)(alias_node->str_value_), static_cast<int32_t>(alias_node->str_len_));
|
||||
}
|
||||
if (OB_FAIL(resolve_generate_table(*table_node, alias_name, table_item))) {
|
||||
if (OB_FAIL(resolve_generate_table(*table_node, parse_tree.children_[1], table_item))) {
|
||||
LOG_WARN("fail to resolve generate taable", K(ret));
|
||||
}
|
||||
break;
|
||||
@ -433,7 +428,7 @@ int ObMergeResolver::check_column_validity(ObColumnRefRawExpr *col_expr)
|
||||
}
|
||||
|
||||
int ObMergeResolver::resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
TableItem *&table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -460,7 +455,10 @@ int ObMergeResolver::resolve_generate_table(const ParseNode &table_node,
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("create table item failed", K(ret));
|
||||
} else {
|
||||
|
||||
ObString alias_name;
|
||||
if (alias_node != NULL) {
|
||||
alias_name.assign_ptr((char *)(alias_node->str_value_), static_cast<int32_t>(alias_node->str_len_));
|
||||
}
|
||||
item->ref_query_ = child_stmt;
|
||||
item->table_id_ = generate_table_id();
|
||||
item->table_name_ = alias_name;
|
||||
|
@ -61,7 +61,7 @@ protected:
|
||||
ObAssignment &assign) override;
|
||||
virtual int resolve_table(const ParseNode &parse_tree, TableItem *&table_item) override;
|
||||
virtual int resolve_generate_table(const ParseNode &table_node,
|
||||
const ObString &alias_name,
|
||||
const ParseNode *alias_node,
|
||||
TableItem *&table_item) override;
|
||||
virtual int mock_values_column_ref(const ObColumnRefRawExpr *column_ref) override;
|
||||
virtual int find_value_desc(ObInsertTableInfo &table_info, uint64_t column_id, ObRawExpr *&column_ref) override;
|
||||
|
@ -2273,6 +2273,13 @@ int ObSelectResolver::expand_target_list(
|
||||
if (OB_FAIL(resolve_all_json_table_columns(table_item, &column_items))) {
|
||||
LOG_WARN("resolve function table columns failed", K(ret));
|
||||
}
|
||||
} else if (table_item.is_values_table()) {
|
||||
if (OB_ISNULL(get_stmt()) || OB_UNLIKELY(get_stmt()->get_column_size() == 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), KPC(get_stmt()));
|
||||
} else if (OB_FAIL(append(column_items, get_stmt()->get_column_items()))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected table type", K_(table_item.type), K(ret));
|
||||
|
@ -120,7 +120,7 @@ int ObViewTableResolver::check_view_circular_reference(const TableItem &view_ite
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObViewTableResolver::resolve_generate_table(const ParseNode &table_node, const ObString &alias_name, TableItem *&table_item)
|
||||
int ObViewTableResolver::resolve_generate_table(const ParseNode &table_node, const ParseNode *alias_node, TableItem *&table_item)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObViewTableResolver view_table_resolver(params_, view_db_name_, view_name_);
|
||||
@ -134,7 +134,7 @@ int ObViewTableResolver::resolve_generate_table(const ParseNode &table_node, con
|
||||
LOG_WARN("set cte ctx to child resolver failed", K(ret));
|
||||
} else if (OB_FAIL(add_cte_table_to_children(view_table_resolver))) {
|
||||
LOG_WARN("add CTE table to children failed", K(ret));
|
||||
} else if (OB_FAIL(do_resolve_generate_table(table_node, alias_name, view_table_resolver, table_item))) {
|
||||
} else if (OB_FAIL(do_resolve_generate_table(table_node, alias_node, view_table_resolver, table_item))) {
|
||||
LOG_WARN("do resolve generate table failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
|
@ -64,7 +64,7 @@ protected:
|
||||
virtual int expand_view(TableItem &view_item);
|
||||
virtual int resolve_subquery_info(const common::ObIArray<ObSubQueryInfo> &subquery_info);
|
||||
int check_view_circular_reference(const TableItem &view_item);
|
||||
virtual int resolve_generate_table(const ParseNode &table_node, const ObString &alias_name, TableItem *&table_item);
|
||||
virtual int resolve_generate_table(const ParseNode &table_node, const ParseNode *alias_node, TableItem *&table_item);
|
||||
virtual int set_select_item(SelectItem &select_item, bool is_auto_gen);
|
||||
virtual const ObString get_view_db_name() const override { return view_db_name_; }
|
||||
virtual const ObString get_view_name() const override { return view_name_; }
|
||||
|
@ -891,6 +891,7 @@ int ObConstRawExpr::assign(const ObRawExpr &other)
|
||||
obj_meta_ = const_expr.get_expr_obj_meta();
|
||||
is_date_unit_ = const_expr.is_date_unit_;
|
||||
is_literal_bool_ = const_expr.is_literal_bool();
|
||||
array_param_group_id_ = const_expr.get_array_param_group_id();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -945,6 +946,7 @@ void ObConstRawExpr::reset()
|
||||
value_.reset();
|
||||
literal_prefix_.reset();
|
||||
is_literal_bool_ = false;
|
||||
array_param_group_id_ = -1;
|
||||
}
|
||||
|
||||
void ObConstRawExpr::set_is_date_unit()
|
||||
|
@ -2077,7 +2077,8 @@ public:
|
||||
ObConstRawExpr()
|
||||
:is_date_unit_(false),
|
||||
is_literal_bool_(false),
|
||||
is_batch_stmt_parameter_(false)/*: precalc_expr_(NULL)*/
|
||||
is_batch_stmt_parameter_(false),/*: precalc_expr_(NULL)*/
|
||||
array_param_group_id_(-1)
|
||||
{ ObRawExpr::set_expr_class(ObRawExpr::EXPR_CONST); }
|
||||
ObConstRawExpr(common::ObIAllocator &alloc)
|
||||
: ObIRawExpr(alloc),
|
||||
@ -2085,7 +2086,8 @@ public:
|
||||
ObConstExpr(),
|
||||
is_date_unit_(false),
|
||||
is_literal_bool_(false),
|
||||
is_batch_stmt_parameter_(false)
|
||||
is_batch_stmt_parameter_(false),
|
||||
array_param_group_id_(-1)
|
||||
{ ObIRawExpr::set_expr_class(ObIRawExpr::EXPR_CONST); }
|
||||
ObConstRawExpr(const oceanbase::common::ObObj &val, ObItemType expr_type = T_INVALID)
|
||||
: ObIRawExpr(expr_type),
|
||||
@ -2093,7 +2095,8 @@ public:
|
||||
ObConstExpr(),
|
||||
is_date_unit_(false),
|
||||
is_literal_bool_(false),
|
||||
is_batch_stmt_parameter_(false)
|
||||
is_batch_stmt_parameter_(false),
|
||||
array_param_group_id_(-1)
|
||||
{
|
||||
set_value(val);
|
||||
set_expr_class(ObIRawExpr::EXPR_CONST);
|
||||
@ -2123,6 +2126,8 @@ public:
|
||||
bool is_literal_bool() const { return is_literal_bool_; }
|
||||
void set_is_batch_stmt_parameter() { is_batch_stmt_parameter_ = true; }
|
||||
bool is_batch_stmt_parameter() { return is_batch_stmt_parameter_; }
|
||||
void set_array_param_group_id(int64_t id) { array_param_group_id_ = id; }
|
||||
int64_t get_array_param_group_id() const { return array_param_group_id_; }
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
|
||||
private:
|
||||
@ -2134,6 +2139,7 @@ private:
|
||||
// is_batch_stmt_parameter_ only used for array_binding batch_execution optimization
|
||||
// Indicates that the current parameter is the batch parameter
|
||||
bool is_batch_stmt_parameter_;
|
||||
int64_t array_param_group_id_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObConstRawExpr);
|
||||
};
|
||||
|
@ -2562,30 +2562,59 @@ int ObRawExprResolverImpl::process_datatype_or_questionmark(const ParseNode &nod
|
||||
// LOG_DEBUG("prepare stmt add new param", K(ctx_.prepare_param_count_));
|
||||
}
|
||||
} else {
|
||||
int64_t param_idx = val.get_unknown();
|
||||
if (OB_ISNULL(ctx_.param_list_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("context param list is null", K(ret));
|
||||
} else if (val.get_unknown() >= ctx_.param_list_->count()) {
|
||||
} else if (param_idx >= ctx_.param_list_->count()) {
|
||||
ret = OB_ERR_BIND_VARIABLE_NOT_EXIST;
|
||||
LOG_WARN("bind variable does not exist",
|
||||
K(ret), K(val.get_unknown()), K(ctx_.param_list_->count()));
|
||||
K(ret), K(param_idx), K(ctx_.param_list_->count()));
|
||||
} else {
|
||||
const ObObjParam ¶m = ctx_.param_list_->at(val.get_unknown());
|
||||
const ObObjParam ¶m = ctx_.param_list_->at(param_idx);
|
||||
c_expr->set_is_literal_bool(param.is_boolean());
|
||||
if (param.is_ext()) {
|
||||
#ifndef OB_BUILD_ORACLE_PL
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("not support array binding", K(ret));
|
||||
#else
|
||||
pl::ObPLNestedTable *param_array = reinterpret_cast<pl::ObPLNestedTable*>(param.get_ext());
|
||||
if (OB_ISNULL(param_array)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("param array is invalid", KPC(param_array));
|
||||
// values statement
|
||||
if (param.is_ext_sql_array()) {
|
||||
ObSqlArrayObj *param_array = reinterpret_cast<ObSqlArrayObj*>(param.get_ext());
|
||||
const ObExecContext *exec_ctx = NULL;
|
||||
const ObPhysicalPlanCtx *phy_ctx = NULL;
|
||||
if (OB_ISNULL(param_array) || OB_ISNULL(ctx_.session_info_) ||
|
||||
OB_ISNULL(exec_ctx = ctx_.session_info_->get_cur_exec_ctx()) ||
|
||||
OB_ISNULL(phy_ctx = ctx_.session_info_->get_cur_exec_ctx()->get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("param array is invalid", K(ret), KPC(param_array));
|
||||
} else {
|
||||
c_expr->set_meta_type(param_array->element_.get_meta_type());
|
||||
c_expr->set_expr_obj_meta(param_array->element_.get_meta_type());
|
||||
c_expr->set_accuracy(param_array->element_.get_accuracy());
|
||||
c_expr->set_param(param);
|
||||
const ObIArray<ObArrayParamGroup> &array_param_groups = phy_ctx->get_array_param_groups();
|
||||
for (int64_t i = 0; i < array_param_groups.count(); i++) {
|
||||
const ObArrayParamGroup &group = array_param_groups.at(i);
|
||||
if (param_idx >= group.start_param_idx_ &&
|
||||
param_idx < group.start_param_idx_ + group.column_count_) {
|
||||
c_expr->set_array_param_group_id(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
c_expr->set_meta_type(param_array->get_element_type().get_meta_type());
|
||||
c_expr->set_expr_obj_meta(param_array->get_element_type().get_meta_type());
|
||||
c_expr->set_accuracy(param_array->get_element_type().get_accuracy());
|
||||
c_expr->set_param(param);
|
||||
#ifndef OB_BUILD_ORACLE_PL
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("not support array binding", K(ret));
|
||||
}
|
||||
#else
|
||||
pl::ObPLNestedTable *param_array = reinterpret_cast<pl::ObPLNestedTable*>(param.get_ext());
|
||||
if (OB_ISNULL(param_array)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("param array is invalid", K(ret), KPC(param_array));
|
||||
} else {
|
||||
c_expr->set_meta_type(param_array->get_element_type().get_meta_type());
|
||||
c_expr->set_expr_obj_meta(param_array->get_element_type().get_meta_type());
|
||||
c_expr->set_accuracy(param_array->get_element_type().get_accuracy());
|
||||
c_expr->set_param(param);
|
||||
}
|
||||
}
|
||||
#endif
|
||||
} else {
|
||||
|
@ -8049,5 +8049,156 @@ int ObResolverUtils::check_encryption_name(ObString &encryption_name, bool &need
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObResolverUtils::rm_space_for_neg_num(ParseNode *param_node, ObIAllocator &allocator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *buf = NULL;
|
||||
int64_t pos = 0;
|
||||
int64_t idx = 0;
|
||||
if (param_node->str_len_ <= 0) {
|
||||
// do nothing
|
||||
} else if ('-' != param_node->str_value_[idx]) {
|
||||
// 'select - 1.2 from dual' and 'select 1.2 from dual' will hit the same plan, the key is
|
||||
// select ? from dual, so '- 1.2' and '1.2' will all go here, if '-' is not presented,
|
||||
// do nothing
|
||||
LOG_TRACE("rm space for neg num", K(idx), K(ObString(param_node->str_len_, param_node->str_value_)));
|
||||
} else if (OB_ISNULL(buf = (char *)allocator.alloc(param_node->str_len_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocator memory", K(ret), K(param_node->str_len_));
|
||||
} else {
|
||||
buf[pos++] = '-';
|
||||
idx += 1;
|
||||
for (; idx < param_node->str_len_ && isspace(param_node->str_value_[idx]); idx++);
|
||||
int32_t len = (int32_t)(param_node->str_len_ - idx);
|
||||
if (len > 0) {
|
||||
MEMCPY(buf + pos, param_node->str_value_ + idx, len);
|
||||
}
|
||||
pos += len;
|
||||
param_node->str_value_ = buf;
|
||||
param_node->str_len_ = pos;
|
||||
LOG_TRACE("rm space for neg num", K(idx), K(ObString(param_node->str_len_, param_node->str_value_)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObResolverUtils::handle_varchar_charset(ObCharsetType charset_type,
|
||||
ObIAllocator &allocator,
|
||||
ParseNode *&node)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if ((T_HEX_STRING == node->type_ || T_VARCHAR == node->type_)
|
||||
&& CHARSET_INVALID != charset_type) {
|
||||
ParseNode *charset_node = new_node(&allocator, T_CHARSET, 0);
|
||||
ParseNode *varchar_node = new_non_terminal_node(&allocator, T_VARCHAR, 2, charset_node, node);
|
||||
|
||||
if (OB_ISNULL(charset_node) || OB_ISNULL(varchar_node)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
const char *name = ObCharset::charset_name(charset_type);
|
||||
charset_node->str_value_ = parse_strdup(name, &allocator, &(charset_node->str_len_));
|
||||
if (NULL == charset_node->str_value_) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
varchar_node->str_value_ = node->str_value_;
|
||||
varchar_node->str_len_ = node->str_len_;
|
||||
varchar_node->raw_text_ = node->raw_text_;
|
||||
varchar_node->text_len_ = node->text_len_;
|
||||
varchar_node->type_ = T_VARCHAR;
|
||||
|
||||
node = varchar_node;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObResolverUtils::resolver_param(ObPlanCacheCtx &pc_ctx,
|
||||
ObSQLSessionInfo &session,
|
||||
const ParamStore &phy_ctx_params,
|
||||
const stmt::StmtType stmt_type,
|
||||
const ObCharsetType param_charset_type,
|
||||
const ObBitSet<> &neg_param_index,
|
||||
const ObBitSet<> ¬_param_index,
|
||||
const ObBitSet<> &must_be_positive_idx,
|
||||
const ObPCParam *pc_param,
|
||||
const int64_t param_idx,
|
||||
ObObjParam &obj_param,
|
||||
bool &is_param)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ParseNode *raw_param = NULL;
|
||||
ObString literal_prefix;
|
||||
const bool is_paramlize = false;
|
||||
int64_t server_collation = CS_TYPE_INVALID;
|
||||
obj_param.reset();
|
||||
if (OB_ISNULL(pc_param) || OB_ISNULL(raw_param = pc_param->node_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret));
|
||||
} else if (not_param_index.has_member(param_idx)) {
|
||||
/* do nothing */
|
||||
is_param = false;
|
||||
SQL_PC_LOG(TRACE, "not_param", K(param_idx), K(raw_param->type_), K(raw_param->value_),
|
||||
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
|
||||
} else {
|
||||
// select - 1.2 from dual
|
||||
// "- 1.2" will be treated as a const node with neg sign
|
||||
// however, ObNumber::from("- 1.2") will throw a error, for there are spaces between neg sign and num
|
||||
// so remove spaces before resolve_const is called
|
||||
if (neg_param_index.has_member(param_idx) &&
|
||||
OB_FAIL(rm_space_for_neg_num(raw_param, pc_ctx.allocator_))) {
|
||||
SQL_PC_LOG(WARN, "fail to remove spaces for neg node", K(ret));
|
||||
} else if (OB_FAIL(handle_varchar_charset(param_charset_type, pc_ctx.allocator_, raw_param))) {
|
||||
SQL_PC_LOG(WARN, "fail to handle varchar charset");
|
||||
} else if (T_QUESTIONMARK == raw_param->type_) {
|
||||
int64_t idx = raw_param->value_;
|
||||
CK (idx >= 0 && idx < phy_ctx_params.count());
|
||||
OX (obj_param.set_is_boolean(phy_ctx_params.at(idx).is_boolean()));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (lib::is_oracle_mode() &&
|
||||
OB_FAIL(session.get_sys_variable(share::SYS_VAR_COLLATION_SERVER, server_collation))) {
|
||||
LOG_WARN("get sys variable failed", K(ret));
|
||||
} else if (OB_FAIL(ObResolverUtils::resolve_const(raw_param, stmt_type, pc_ctx.allocator_,
|
||||
static_cast<ObCollationType>(session.get_local_collation_connection()),
|
||||
session.get_nls_collation_nation(), session.get_timezone_info(),
|
||||
obj_param, is_paramlize, literal_prefix,
|
||||
session.get_actual_nls_length_semantics(),
|
||||
static_cast<ObCollationType>(server_collation), NULL,
|
||||
session.get_sql_mode()))) {
|
||||
SQL_PC_LOG(WARN, "fail to resolve const", K(ret));
|
||||
} else if (FALSE_IT(obj_param.set_raw_text_info(static_cast<int32_t>(raw_param->raw_sql_offset_),
|
||||
static_cast<int32_t>(raw_param->text_len_)))) {
|
||||
/* nothing */
|
||||
} else if (ob_is_numeric_type(obj_param.get_type())) {
|
||||
// -0 is also counted as negative
|
||||
if (must_be_positive_idx.has_member(param_idx)) {
|
||||
if (obj_param.is_boolean()) {
|
||||
// boolean will skip this check
|
||||
} else if (lib::is_oracle_mode()) {
|
||||
if (obj_param.is_negative_number() ||
|
||||
(obj_param.is_zero_number() && '-' == raw_param->str_value_[0])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
pc_ctx.should_add_plan_ = false; // 内部主动抛出not supported时候需要设置这个标志,以免新计划add plan导致锁冲突
|
||||
LOG_TRACE("param must be positive", K(ret), K(param_idx), K(obj_param));
|
||||
}
|
||||
} else if (lib::is_mysql_mode()) {
|
||||
if (obj_param.is_integer_type() &&
|
||||
(obj_param.get_int() < 0 || (0 == obj_param.get_int() && '-' == raw_param->str_value_[0]))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
pc_ctx.should_add_plan_ = false;
|
||||
LOG_TRACE("param must be positive", K(ret), K(param_idx), K(obj_param));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
is_param = true;
|
||||
LOG_TRACE("is_param", K(param_idx), K(obj_param), K(raw_param->type_), K(raw_param->value_),
|
||||
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
@ -754,6 +754,18 @@ public:
|
||||
ObSQLSessionInfo &session_info,
|
||||
ObDataTypeCastParams &dtc_params);
|
||||
static bool is_expr_can_be_used_in_table_function(const ObRawExpr &expr);
|
||||
static int resolver_param(ObPlanCacheCtx &pc_ctx,
|
||||
ObSQLSessionInfo &session,
|
||||
const ParamStore &phy_ctx_params,
|
||||
const stmt::StmtType stmt_type,
|
||||
const ObCharsetType param_charset_type,
|
||||
const ObBitSet<> &neg_param_index,
|
||||
const ObBitSet<> ¬_param_index,
|
||||
const ObBitSet<> &must_be_positive_idx,
|
||||
const ObPCParam *pc_param,
|
||||
const int64_t param_idx,
|
||||
ObObjParam &obj_param,
|
||||
bool &is_param);
|
||||
static int check_keystore_status(const uint64_t tenant_id, ObSchemaChecker &schema_checker);
|
||||
static int check_encryption_name(common::ObString &encryption_name, bool &need_encrypt);
|
||||
private:
|
||||
@ -783,6 +795,10 @@ private:
|
||||
static int check_partition_range_value_result_type(const share::schema::ObPartitionFuncType part_type,
|
||||
const ObColumnRefRawExpr &part_column_expr,
|
||||
ObRawExpr &part_value_expr);
|
||||
static int rm_space_for_neg_num(ParseNode *param_node, ObIAllocator &allocator);
|
||||
static int handle_varchar_charset(ObCharsetType charset_type,
|
||||
ObIAllocator &allocator,
|
||||
ParseNode *&node);
|
||||
static const common::ObString stmt_type_string[];
|
||||
|
||||
// disallow construct
|
||||
|
@ -1217,6 +1217,24 @@ int ObStmtComparer::compare_table_item(const ObDMLStmt *first,
|
||||
relation))) {
|
||||
LOG_WARN("failed to compare joined table item", K(ret));
|
||||
}
|
||||
} else if (first_table->is_values_table() && second_table->is_values_table()) {
|
||||
if (OB_FAIL(compare_values_table_item(first,
|
||||
first_table,
|
||||
second,
|
||||
second_table,
|
||||
map_info,
|
||||
relation))) {
|
||||
LOG_WARN("compare values table failed",K(ret), K(first_table), K(second_table));
|
||||
} else if (QueryRelation::QUERY_UNCOMPARABLE != relation) {
|
||||
const int32_t first_table_index = first->get_table_bit_index(first_table->table_id_);
|
||||
const int32_t second_table_index = second->get_table_bit_index(second_table->table_id_);
|
||||
if (first_table_index < 1 || first_table_index > first->get_table_size()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect table bit index", K(ret), K(first_table_index), K(first->get_table_size()));
|
||||
} else {
|
||||
map_info.table_map_.at(first_table_index - 1) = second_table_index - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1345,3 +1363,58 @@ int ObStmtComparer::compare_set_stmt(const ObSelectStmt *first,
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObStmtComparer::compare_values_table_item(const ObDMLStmt *first,
|
||||
const TableItem *first_table,
|
||||
const ObDMLStmt *second,
|
||||
const TableItem *second_table,
|
||||
ObStmtMapInfo &map_info,
|
||||
QueryRelation &relation)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObStmtCompareContext context(first, second, map_info, &first->get_query_ctx()->calculable_items_);
|
||||
relation = QueryRelation::QUERY_UNCOMPARABLE;
|
||||
if (OB_ISNULL(first) || OB_ISNULL(first_table)
|
||||
|| OB_ISNULL(second) || OB_ISNULL(second_table)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("param has null", K(first), K(first_table), K(second), K(second_table));
|
||||
} else if (first_table->is_values_table() &&
|
||||
second_table->is_values_table() &&
|
||||
first->get_column_size(first_table->table_id_) == second->get_column_size(second_table->table_id_) &&
|
||||
first_table->table_values_.count() % first->get_column_size(first_table->table_id_) == 0 &&
|
||||
second_table->table_values_.count() % second->get_column_size(second_table->table_id_) == 0) {
|
||||
//Perhaps in the future, the comparison of different row orders can be considered
|
||||
int64_t match_count = 0;
|
||||
bool is_match = true;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_match && i < first_table->table_values_.count(); ++i) {
|
||||
bool is_match = false;
|
||||
if (i >= second_table->table_values_.count()) {
|
||||
break;
|
||||
} else if (OB_FAIL(is_same_condition(first_table->table_values_.at(i),
|
||||
second_table->table_values_.at(i),
|
||||
context,
|
||||
is_match))) {
|
||||
LOG_WARN("failed to check is condition equal", K(ret));
|
||||
} else if (!is_match) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(append(map_info.equal_param_map_, context.equal_param_info_))) {
|
||||
LOG_WARN("failed to append exprs", K(ret));
|
||||
} else {
|
||||
++match_count;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && is_match) {
|
||||
if (match_count == first_table->table_values_.count() &&
|
||||
match_count == second_table->table_values_.count()) {//first table is equal second table
|
||||
relation = QueryRelation::QUERY_EQUAL;
|
||||
} else if (match_count == first_table->table_values_.count()) {//first table is subset second table
|
||||
relation = QueryRelation::QUERY_LEFT_SUBSET;
|
||||
} else if (match_count == second_table->table_values_.count()) {////second table is subset first table
|
||||
relation = QueryRelation::QUERY_RIGHT_SUBSET;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
/*do nothing*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -317,6 +317,13 @@ public:
|
||||
ObStmtMapInfo &map_info,
|
||||
QueryRelation &relation);
|
||||
|
||||
static int compare_values_table_item(const ObDMLStmt *first,
|
||||
const TableItem *first_table,
|
||||
const ObDMLStmt *second,
|
||||
const TableItem *second_table,
|
||||
ObStmtMapInfo &map_info,
|
||||
QueryRelation &relation);
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -254,7 +254,8 @@ int ObTransformQueryPushDown::check_transform_validity(ObSelectStmt *select_stmt
|
||||
(view_stmt->is_recursive_union() && !select_stmt->is_spj()) ||
|
||||
select_stmt->is_set_stmt() ||
|
||||
(select_stmt->has_sequence() && !view_stmt->is_spj()) ||
|
||||
view_stmt->has_ora_rowscn()) {//判断1, 2
|
||||
view_stmt->has_ora_rowscn() ||
|
||||
view_stmt->is_values_table_query()) {//判断1, 2
|
||||
can_transform = false;
|
||||
OPT_TRACE("stmt is not spj");
|
||||
} else if (OB_FAIL(check_rownum_push_down(select_stmt, view_stmt, check_status))) {//判断3
|
||||
|
@ -609,7 +609,8 @@ int ObTransformRule::adjust_transformed_stmt(ObIArray<ObParentDMLStmt> &parent_s
|
||||
bool ObTransformRule::is_normal_disabled_transform(const ObDMLStmt &stmt)
|
||||
{
|
||||
return (stmt.is_hierarchical_query() && transform_method_ != TransMethod::ROOT_ONLY) ||
|
||||
stmt.is_insert_all_stmt();
|
||||
stmt.is_insert_all_stmt() ||
|
||||
stmt.is_values_table_query();
|
||||
}
|
||||
|
||||
int ObTransformRule::need_transform(const common::ObIArray<ObParentDMLStmt> &parent_stmts,
|
||||
|
@ -2203,6 +2203,28 @@ int ObTransformUtils::is_column_expr_not_null(ObNotNullContext &ctx,
|
||||
constraints))) {
|
||||
LOG_WARN("failed to check expr not null", K(ret));
|
||||
}
|
||||
} else if (table->is_values_table()) {
|
||||
int64_t idx = expr->get_column_id() - OB_APP_MIN_COLUMN_ID;
|
||||
int64_t column_cnt = ctx.stmt_->get_column_size(table->table_id_);
|
||||
if (OB_UNLIKELY(idx >= column_cnt || column_cnt == 0 || table->table_values_.empty() ||
|
||||
table->table_values_.count() % column_cnt != 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(idx), KPC(table), K(column_cnt));
|
||||
} else {
|
||||
is_not_null = true;
|
||||
int64_t row_count = table->table_values_.count() / column_cnt;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && is_not_null && i < row_count; ++i) {
|
||||
if (OB_UNLIKELY(column_cnt * i + idx >= table->table_values_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(i), K(idx), KPC(table), K(column_cnt));
|
||||
} else if (OB_FAIL(is_expr_not_null(ctx,
|
||||
table->table_values_.at(column_cnt * i + idx),
|
||||
is_not_null,
|
||||
constraints))) {
|
||||
LOG_WARN("failed to check expr not null", K(ret));
|
||||
} else {/*do nothing*/}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
@ -10528,7 +10550,8 @@ int ObTransformUtils::check_project_pruning_validity(ObSelectStmt &stmt, bool &i
|
||||
bool is_const = false;
|
||||
if (stmt.has_distinct() || stmt.is_recursive_union() ||
|
||||
(stmt.is_set_stmt() && stmt.is_set_distinct()) ||
|
||||
stmt.is_hierarchical_query() || stmt.is_contains_assignment()) {
|
||||
stmt.is_hierarchical_query() || stmt.is_contains_assignment() ||
|
||||
stmt.is_values_table_query()) {
|
||||
// do nothing
|
||||
OPT_TRACE("stmt has distinct/assignment or is set stmt");
|
||||
} else if (stmt.get_select_item_size() == 1
|
||||
@ -11021,6 +11044,12 @@ int ObTransformUtils::is_table_item_correlated(
|
||||
} else if (OB_FAIL(is_correlated_expr(query_ref.get_exec_params(), table->json_table_def_->doc_expr_, contains))) {
|
||||
LOG_WARN("failed to check function table expr correlated", K(ret));
|
||||
}
|
||||
} else if (table->is_values_table()) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && !contains && j < table->table_values_.count(); ++j) {
|
||||
if (OB_FAIL(is_correlated_expr(query_ref.get_exec_params(), table->table_values_.at(j), contains))) {
|
||||
LOG_WARN("failed to check values table expr correlated", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
@ -526,7 +526,8 @@ int ObTransformViewMerge::check_basic_validity(ObDMLStmt *parent_stmt,
|
||||
|| child_stmt->get_aggr_item_size() != 0
|
||||
|| child_stmt->has_window_function()
|
||||
|| child_stmt->has_sequence()
|
||||
|| child_stmt->has_ora_rowscn()) {
|
||||
|| child_stmt->has_ora_rowscn()
|
||||
|| child_stmt->is_values_table_query()) {
|
||||
can_be = false;
|
||||
OPT_TRACE("not a valid view");
|
||||
} else if (OB_FAIL(ObTransformUtils::check_has_assignment(*child_stmt, has_assignment))) {
|
||||
|
@ -985,7 +985,7 @@ public:
|
||||
int restore_sql_session(StmtSavedValue &saved_value);
|
||||
int restore_session(StmtSavedValue &saved_value);
|
||||
ObExecContext *get_cur_exec_ctx() { return cur_exec_ctx_; }
|
||||
|
||||
const ObExecContext *get_cur_exec_ctx() const { return cur_exec_ctx_; }
|
||||
int begin_nested_session(StmtSavedValue &saved_value, bool skip_cur_stmt_tables = false);
|
||||
int end_nested_session(StmtSavedValue &saved_value);
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user