/** * Copyright (c) 2021 OceanBase * OceanBase CE is licensed under Mulan PubL v2. * You can use this software according to the terms and conditions of the Mulan PubL v2. * You may obtain a copy of Mulan PubL v2 at: * http://license.coscl.org.cn/MulanPubL-2.0 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. * See the Mulan PubL v2 for more details. */ #define USING_LOG_PREFIX SQL_ENG #include "sql/engine/expr/ob_expr_minus.h" #include "lib/oblog/ob_log.h" #include "lib/utility/ob_macro_utils.h" #include "sql/engine/expr/ob_expr_result_type_util.h" #include "sql/ob_sql_utils.h" #include "sql/session/ob_sql_session_info.h" #include "sql/engine/ob_exec_context.h" #include "sql/code_generator/ob_static_engine_expr_cg.h" #include "sql/engine/expr/ob_batch_eval_util.h" #include "sql/engine/expr/ob_rt_datum_arith.h" namespace oceanbase { using namespace common; using namespace common::number; using namespace oceanbase::lib; namespace sql { ObExprMinus::ObExprMinus(ObIAllocator &alloc, ObExprOperatorType type) : ObArithExprOperator(alloc, type, N_MINUS, 2, NOT_ROW_DIMENSION, ObExprResultTypeUtil::get_minus_result_type, ObExprResultTypeUtil::get_minus_calc_type, minus_funcs_) { param_lazy_eval_ = lib::is_oracle_mode(); } int ObExprMinus::calc_result_type2(ObExprResType &type, ObExprResType &type1, ObExprResType &type2, ObExprTypeCtx &type_ctx) const { int ret = OB_SUCCESS; static const int64_t CARRY_OFFSET = 1; ObScale scale = SCALE_UNKNOWN_YET; ObPrecision precision = PRECISION_UNKNOWN_YET; const ObSQLSessionInfo *session = nullptr; bool is_oracle = lib::is_oracle_mode(); if (OB_ISNULL(session = type_ctx.get_session())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get mysession", K(ret)); } else if (OB_FAIL(ObArithExprOperator::calc_result_type2(type, type1, type2, type_ctx))) { LOG_WARN("fail to calc result type", K(ret), K(type), K(type1), K(type2)); } else if (is_oracle && type.is_oracle_decimal()) { type.set_scale(ORA_NUMBER_SCALE_UNKNOWN_YET); type.set_precision(PRECISION_UNKNOWN_YET); } else if (OB_UNLIKELY(SCALE_UNKNOWN_YET == type1.get_scale() || SCALE_UNKNOWN_YET == type2.get_scale())) { type.set_scale(SCALE_UNKNOWN_YET); type.set_precision(PRECISION_UNKNOWN_YET); } else if (type1.get_type_class() == ObIntervalTC || type2.get_type_class() == ObIntervalTC) { type.set_scale(ObAccuracy::MAX_ACCURACY2[ORACLE_MODE][type.get_type()].get_scale()); type.set_precision(ObAccuracy::MAX_ACCURACY2[ORACLE_MODE][type.get_type()].get_precision()); } else if (ob_is_oracle_datetime_tc(type1.get_type()) && ob_is_oracle_datetime_tc(type2.get_type()) && type.get_type_class() == ObIntervalTC) { type.set_scale(ObIntervalScaleUtil::interval_ds_scale_to_ob_scale( MAX_SCALE_FOR_ORACLE_TEMPORAL, std::max(type1.get_scale(), type2.get_scale()))); type.set_precision(ObAccuracy::MAX_ACCURACY2[ORACLE_MODE][type.get_type()].get_precision()); } else { if (OB_UNLIKELY(is_oracle && type.is_datetime())) { scale = OB_MAX_DATE_PRECISION; } else { ObScale scale1 = static_cast(MAX(type1.get_scale(), 0)); ObScale scale2 = static_cast(MAX(type2.get_scale(), 0)); scale = MAX(scale1, scale2); if (lib::is_mysql_mode() && type.is_double()) { precision = ObMySQLUtil::float_length(scale); } else { int64_t inter_part_length1 = type1.get_precision() - type1.get_scale(); int64_t inter_part_length2 = type2.get_precision() - type2.get_scale(); precision = static_cast(MAX(inter_part_length1, inter_part_length2) + CARRY_OFFSET + scale); } } type.set_scale(scale); if (OB_UNLIKELY(PRECISION_UNKNOWN_YET == type1.get_precision()) || OB_UNLIKELY(PRECISION_UNKNOWN_YET == type2.get_precision())) { type.set_precision(PRECISION_UNKNOWN_YET); } else { type.set_precision(precision); } if (lib::is_mysql_mode() && is_no_unsigned_subtraction(session->get_sql_mode())) { ObObjType convert_type = type.get_type(); convert_unsigned_type_to_signed(convert_type); type.set_type(convert_type); } LOG_DEBUG("calc_result_type2", K(scale), K(type1), K(type2), K(type), K(precision)); } return ret; } int ObExprMinus::calc_datetime_minus(common::ObObj &result, const common::ObObj &left, const common::ObObj &right, common::ObExprCtx &expr_ctx, common::ObScale calc_scale) { int ret = OB_SUCCESS; if (OB_ISNULL(expr_ctx.calc_buf_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("calc_buf_ should not be null", K(ret), K(expr_ctx.calc_buf_)); } else { const ObObj *res_left = &left; const ObObj *res_right = &right; ObObj tmp_left_obj; ObObj tmp_right_obj; EXPR_DEFINE_CAST_CTX(expr_ctx, CM_NO_CAST_INT_UINT); if (OB_FAIL(ObObjCaster::to_type(ObDateTimeType, cast_ctx, left, tmp_left_obj, res_left))) { LOG_WARN("cast failed.", K(ret), K(left)); } else if (OB_FAIL(ObObjCaster::to_type(ObDateTimeType, cast_ctx, right, tmp_right_obj, res_right))) { LOG_WARN("cast failed.", K(ret), K(right)); } else { const int64_t LOCAL_BUF_SIZE = ObNumber::MAX_CALC_BYTE_LEN * 5; char local_buf[LOCAL_BUF_SIZE]; ObDataBuffer local_alloc(local_buf, LOCAL_BUF_SIZE); ObNumber left_datetime; ObNumber right_datetime; ObNumber usecs_per_day; ObNumber left_date; ObNumber right_date; if (OB_FAIL(left_datetime.from(res_left->get_datetime(), local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret), K(res_left)); } else if (OB_FAIL(right_datetime.from(res_right->get_datetime(), local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret), K(res_right)); } else if (OB_FAIL(usecs_per_day.from(USECS_PER_DAY, local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret)); } else if (OB_FAIL(left_datetime.div_v3(usecs_per_day, left_date, local_alloc, ObNumber::OB_MAX_DECIMAL_DIGIT, false))) { LOG_WARN("calc left date number failed", K(ret)); } else if (OB_FAIL(right_datetime.div_v3(usecs_per_day, right_date, local_alloc, ObNumber::OB_MAX_DECIMAL_DIGIT, false))) { LOG_WARN("calc left date number failed", K(ret)); } else if (FALSE_IT(tmp_left_obj.set_number(left_date))) { } else if (FALSE_IT(tmp_right_obj.set_number(right_date))) { } else if (OB_FAIL(minus_number(result, tmp_left_obj, tmp_right_obj, expr_ctx.calc_buf_, calc_scale))) { LOG_WARN("minus_number failed.", K(ret), K(tmp_left_obj), K(tmp_right_obj)); } else { LOG_DEBUG("succ to calc_datetime_minus", K(ret), K(result), K(left), K(right), K(left_datetime), K(right_datetime), "left_double", res_left->get_datetime(), K(left_date), K(right_date), K(tmp_left_obj), K(tmp_right_obj)); } } } return ret; } int ObExprMinus::calc_timestamp_minus(common::ObObj &result, const common::ObObj &left, const common::ObObj &right, const ObTimeZoneInfo *tz_info) { int ret = OB_SUCCESS; ObIntervalDSValue res_interval; ObOTimestampData left_v; ObOTimestampData right_v; if (OB_UNLIKELY(!ob_is_oracle_datetime_tc(left.get_type()) || !ob_is_oracle_datetime_tc(right.get_type()))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret)); } if (OB_SUCC(ret)) { if (left.is_datetime()) { ret = ObTimeConverter::odate_to_otimestamp(left.get_datetime(), tz_info, ObTimestampTZType, left_v); } else if (left.is_timestamp_nano()) { ret = ObTimeConverter::odate_to_otimestamp(left.get_otimestamp_value().time_us_, tz_info, ObTimestampTZType, left_v); left_v.time_ctx_.tail_nsec_ = left.get_otimestamp_value().time_ctx_.tail_nsec_; } else { ret = ObTimeConverter::otimestamp_to_otimestamp(left.get_type(), left.get_otimestamp_value(), tz_info, ObTimestampTZType, left_v); } if (OB_FAIL(ret)) { LOG_WARN("fail to convert left to timestamp tz", K(ret), K(left)); } } if (OB_SUCC(ret)) { if (right.is_datetime()) { ret = ObTimeConverter::odate_to_otimestamp(right.get_datetime(), tz_info, ObTimestampTZType, right_v); } else if (right.is_timestamp_nano()) { ret = ObTimeConverter::odate_to_otimestamp(right.get_otimestamp_value().time_us_, tz_info, ObTimestampTZType, right_v); right_v.time_ctx_.tail_nsec_ = right.get_otimestamp_value().time_ctx_.tail_nsec_; } else { ret = ObTimeConverter::otimestamp_to_otimestamp(right.get_type(), right.get_otimestamp_value(), tz_info, ObTimestampTZType, right_v); } if (OB_FAIL(ret)) { LOG_WARN("fail to convert right to timestamp tz", K(ret), K(right)); } } if (OB_SUCC(ret)) { ObTimeConverter::calc_oracle_temporal_minus(left_v, right_v, res_interval); ObScale res_scale = ObIntervalScaleUtil::interval_ds_scale_to_ob_scale( MAX_SCALE_FOR_ORACLE_TEMPORAL, std::max(left.get_scale(), right.get_scale()) ); result.set_interval_ds(res_interval); result.set_scale(res_scale); } LOG_DEBUG("succ to calc_timestamp_minus", K(ret), K(result), K(left_v), K(right_v), K(left), K(right), K(res_interval)); return ret; } int ObExprMinus::calc(ObDatum &res, const ObDatum &left, const ObDatum &right, const ObDatumMeta &left_meta, const ObDatumMeta &right_meta, ObIAllocator *allocator, uint64_t tenant_id) { int ret = OB_SUCCESS; if (OB_ISNULL(allocator)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null allocator", K(ret)); } else { SMART_VARS_2((ObSQLSessionInfo, default_session), (ObExecContext, exec_ctx, *allocator)) { const ObTenantSchema *tenant_schema = NULL; ObSchemaGetterGuard schema_guard; if (OB_FAIL(default_session.init(0, 0, allocator))) { LOG_WARN("init empty session failed", K(ret)); } else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("get tenant_schema failed", K(ret)); } else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) { LOG_WARN("init tenant failed", K(ret)); } else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) { LOG_WARN("session load system variable failed", K(ret)); } else if (OB_FAIL(default_session.load_default_configs_in_pc())) { LOG_WARN("session load default configs failed", K(ret)); } else { // ObExecContext exec_ctx(*allocator); ObPhysicalPlanCtx plan_ctx(*allocator); ObRTDatumArith *arith = NULL; OZ (plan_ctx.init_datum_param_store()); OX (exec_ctx.set_physical_plan_ctx(&plan_ctx)); OX (exec_ctx.set_my_session(&default_session)); OX (arith = OB_NEWx(ObRTDatumArith, allocator, exec_ctx, default_session)); if (OB_FAIL(ret)) { if (NULL != arith) { arith->~ObRTDatumArith(); arith = NULL; } } else if (NULL == arith) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("allocate memory failed", K(ret)); } else if (OB_FAIL(arith->setup_datum_metas(left_meta, right_meta))) { LOG_WARN("setup datum metas failed", K(ret)); } else { auto left_item = arith->ref(0); auto right_item = arith->ref(1); if (OB_FAIL(arith->generate(left_item - right_item))) { LOG_WARN("generate arithmetic expression failed", K(ret)); } else if (NULL == arith->get_expr()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("no runtime expr generated", K(ret)); } if (OB_SUCC(ret)) { ObDatum *eval_res = NULL; if (OB_FAIL(arith->eval(eval_res, left, right))) { LOG_WARN("runtime datum arithmetic evaluate failed", K(ret)); } else if (OB_ISNULL(eval_res)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("eval result is NULL", K(ret)); } else if (OB_FAIL(res.deep_copy(*eval_res, *allocator))){ LOG_WARN("failed to deep copy", K(ret)); } } } if (NULL != arith) { arith->~ObRTDatumArith(); } } } } return ret; } ObArithFunc ObExprMinus::minus_funcs_[ObMaxTC] = { NULL, ObExprMinus::minus_int, ObExprMinus::minus_uint, ObExprMinus::minus_float, ObExprMinus::minus_double, ObExprMinus::minus_number, ObExprMinus::minus_datetime,//datetime NULL,//date NULL,//time NULL,//year NULL,//string NULL,//extend NULL,//unknown NULL,//text NULL,//bit NULL,//enumset NULL,//enumsetInner }; ObArithFunc ObExprMinus::agg_minus_funcs_[ObMaxTC] = { NULL, ObExprMinus::minus_int, ObExprMinus::minus_uint, ObExprMinus::minus_float, ObExprMinus::minus_double_no_overflow, ObExprMinus::minus_number, ObExprMinus::minus_datetime,//datetime NULL,//date NULL,//time NULL,//year NULL,//string NULL,//extend NULL,//unknown NULL,//text NULL,//bit NULL,//enumset NULL,//enumsetInner }; int ObExprMinus::minus_int(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; int64_t left_i = left.get_int(); int64_t right_i = right.get_int(); char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; if (left.get_type_class() == right.get_type_class()) { res.set_int(left_i - right_i); if (OB_UNLIKELY(is_int_int_out_of_range(left_i, right_i, res.get_int()))) { ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %ld)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT", expr_str); } } else if (OB_UNLIKELY(ObUIntTC != right.get_type_class())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid types", K(ret), K(left), K(right)); } else { res.set_uint64(left_i - right_i); if (OB_UNLIKELY(is_int_uint_out_of_range(left_i, right_i, res.get_uint64()))) { ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %lu)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } } UNUSED(allocator); UNUSED(scale); return ret; } int ObExprMinus::minus_uint(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; uint64_t left_i = left.get_uint64(); uint64_t right_i = right.get_uint64(); res.set_uint64(left_i - right_i); char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; if (left.get_type_class() == right.get_type_class()) { if (OB_UNLIKELY(is_uint_uint_out_of_range(left_i, right_i, res.get_uint64()))) { ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%lu - %lu)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } } else if (OB_UNLIKELY(ObIntTC != right.get_type_class())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid types", K(ret), K(left), K(right)); } else { if (OB_UNLIKELY(is_uint_int_out_of_range(right_i, left_i, res.get_uint64()))) { ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%lu - %ld)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } } UNUSED(allocator); UNUSED(scale); return ret; } int ObExprMinus::minus_float(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!lib::is_oracle_mode())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("only oracle mode arrive here", K(ret), K(left), K(right)); } else if (OB_UNLIKELY(left.get_type_class() != right.get_type_class())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid types", K(ret), K(left), K(right)); } else { float left_f = left.get_float(); float right_f = right.get_float(); res.set_float(left_f - right_f); if (OB_UNLIKELY(is_float_out_of_range(res.get_float())) && !lib::is_oracle_mode()) { ret = OB_OPERATE_OVERFLOW; char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%e - %e)'", left_f, right_f); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BINARY_FLOAT", expr_str); LOG_WARN("float out of range", K(res), K(left), K(right), K(res)); } } LOG_DEBUG("succ to minus float", K(res), K(left), K(right)); UNUSED(allocator); UNUSED(scale); return ret; } int ObExprMinus::minus_double(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; if (OB_UNLIKELY(left.get_type_class() != right.get_type_class())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid types", K(ret), K(left), K(right)); } else { double left_d = left.get_double(); double right_d = right.get_double(); res.set_double(left_d - right_d); if (OB_UNLIKELY(is_double_out_of_range(res.get_double())) && !lib::is_oracle_mode()) { ret = OB_OPERATE_OVERFLOW; char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%e - %e)'", left_d, right_d); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "DOUBLE", expr_str); LOG_WARN("double out of range", K(res), K(left), K(right), K(res)); res.set_null(); } LOG_DEBUG("succ to minus double", K(res), K(left), K(right)); } UNUSED(allocator); UNUSED(scale); return ret; } int ObExprMinus::minus_double_no_overflow(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *, ObScale) { int ret = OB_SUCCESS; if (OB_UNLIKELY(left.get_type_class() != right.get_type_class())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid types", K(ret), K(left), K(right)); } else { double left_d = left.get_double(); double right_d = right.get_double(); res.set_double(left_d - right_d); } return ret; } int ObExprMinus::minus_number(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; number::ObNumber res_nmb; if (OB_UNLIKELY(NULL == allocator)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("allocator is null", K(ret)); } else if (OB_FAIL(left.get_number().sub_v3(right.get_number(), res_nmb, *allocator))) { LOG_WARN("failed to sub numbers", K(ret), K(left), K(right)); } else { if (ObUNumberType == res.get_type()) { res.set_unumber(res_nmb); } else { res.set_number(res_nmb); } } UNUSED(scale); return ret; } int ObExprMinus::minus_datetime(ObObj &res, const ObObj &left, const ObObj &right, ObIAllocator *allocator, ObScale scale) { int ret = OB_SUCCESS; const int64_t left_i = left.get_datetime(); const int64_t right_i = right.get_datetime(); ObTime ob_time; if (OB_LIKELY(left.get_type_class() == right.get_type_class())) { int64_t round_value = left_i - right_i; ObTimeConverter::round_datetime(OB_MAX_DATE_PRECISION, round_value); res.set_datetime(round_value); res.set_scale(OB_MAX_DATE_PRECISION); if (OB_UNLIKELY(res.get_datetime() > DATETIME_MAX_VAL || res.get_datetime() < DATETIME_MIN_VAL) || (OB_FAIL(ObTimeConverter::datetime_to_ob_time(res.get_datetime(), NULL, ob_time))) || (OB_FAIL(ObTimeConverter::validate_oracle_date(ob_time)))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %ld)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "DATE", expr_str); } } LOG_DEBUG("minus datetime", K(left), K(right), K(ob_time), K(res)); UNUSED(allocator); UNUSED(scale); return ret; } int ObExprMinus::cg_expr(ObExprCGCtx &op_cg_ctx, const ObRawExpr &raw_expr, ObExpr &rt_expr) const { #define SET_MINUS_FUNC_PTR(v) \ rt_expr.eval_func_ = ObExprMinus::v; \ rt_expr.eval_batch_func_ = ObExprMinus::v##_batch; int ret = OB_SUCCESS; UNUSED(raw_expr); UNUSED(op_cg_ctx); if (rt_expr.arg_cnt_ != 2 || OB_ISNULL(rt_expr.args_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("count of children is not 2 or children is null", K(ret), K(rt_expr.arg_cnt_), K(rt_expr.args_)); } else if (OB_ISNULL(rt_expr.args_[0]) || OB_ISNULL(rt_expr.args_[1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("child is null", K(ret), K(rt_expr.args_[0]), K(rt_expr.args_[1])); } else { rt_expr.eval_func_ = NULL; const ObObjType left_type = rt_expr.args_[0]->datum_meta_.type_; const ObObjType right_type = rt_expr.args_[1]->datum_meta_.type_; const ObObjType result_type = rt_expr.datum_meta_.type_; const ObObjTypeClass left_tc = ob_obj_type_class(left_type); const ObObjTypeClass right_tc = ob_obj_type_class(right_type); switch (result_type) { case ObIntType: SET_MINUS_FUNC_PTR(minus_int_int); break; case ObUInt64Type: if (ObIntTC == left_tc && ObUIntTC == right_tc) { SET_MINUS_FUNC_PTR(minus_int_uint); } else if (ObUIntTC == left_tc && ObIntTC == right_tc) { SET_MINUS_FUNC_PTR(minus_uint_int); } else if (ObUIntTC == left_tc && ObUIntTC == right_tc) { SET_MINUS_FUNC_PTR(minus_uint_uint); } break; case ObIntervalYMType: SET_MINUS_FUNC_PTR(minus_intervalym_intervalym); break; case ObIntervalDSType: if (ObIntervalDSType == left_type) { SET_MINUS_FUNC_PTR(minus_intervalds_intervalds); } else { SET_MINUS_FUNC_PTR(minus_timestamp_timestamp); } break; case ObDateTimeType: if (ObIntervalYMType == right_type) { SET_MINUS_FUNC_PTR(minus_datetime_intervalym); } else if (ObIntervalDSType == right_type) { SET_MINUS_FUNC_PTR(minus_datetime_intervalds); } else if (ObDateTimeType == left_type && ObNumberType == right_type) { SET_MINUS_FUNC_PTR(minus_datetime_number); } else { SET_MINUS_FUNC_PTR(minus_datetime_datetime); } break; case ObTimestampTZType: if (ObIntervalYMType == right_type) { SET_MINUS_FUNC_PTR(minus_timestamptz_intervalym); } else if (ObIntervalDSType == right_type) { SET_MINUS_FUNC_PTR(minus_timestamptz_intervalds); } break; case ObTimestampLTZType: if (ObIntervalYMType == right_type) { SET_MINUS_FUNC_PTR(minus_timestampltz_intervalym); } else if (ObIntervalDSType == right_type) { SET_MINUS_FUNC_PTR(minus_timestamp_tiny_intervalds); } break; case ObTimestampNanoType: if (ObIntervalYMType == right_type) { SET_MINUS_FUNC_PTR(minus_timestampnano_intervalym); } else if (ObIntervalDSType == right_type) { SET_MINUS_FUNC_PTR(minus_timestamp_tiny_intervalds); } break; case ObFloatType: SET_MINUS_FUNC_PTR(minus_float_float); break; case ObDoubleType: SET_MINUS_FUNC_PTR(minus_double_double); break; case ObUNumberType: case ObNumberType: if (ObDateTimeType == left_type) { SET_MINUS_FUNC_PTR(minus_datetime_datetime_oracle); } else { SET_MINUS_FUNC_PTR(minus_number_number); } break; default: break; } if (OB_ISNULL(rt_expr.eval_func_)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("unexpected params type.", K(ret), K(left_type), K(right_type), K(result_type)); } } return ret; #undef SET_MINUS_FUNC_PTR } struct ObIntIntBatchMinusRaw : public ObArithOpRawType { static void raw_op(int64_t &res, const int64_t l, const int64_t r) { res = l - r; } static int raw_check(const int64_t res, const int64_t l, const int64_t r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_int_int_out_of_range(l, r, res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %ld)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT", expr_str); } return ret; } }; //calc_type is IntTC left and right has same TC int ObExprMinus::minus_int_int(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_int_int_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObIntUIntBatchMinusRaw : public ObArithOpRawType { static void raw_op(uint64_t &res, const int64_t l, const uint64_t r) { res = l - r; } static int raw_check(const uint64_t res, const int64_t l, const uint64_t r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_int_uint_out_of_range(l, r, res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %lu)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } return ret; } }; // calc_type/left_type is IntTC, right is ObUIntTC, only mysql mode int ObExprMinus::minus_int_uint(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_int_uint_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObUIntUIntBatchMinusRaw : public ObArithOpRawType { static void raw_op(uint64_t &res, const uint64_t l, const uint64_t r) { res = l - r; } static int raw_check(const uint64_t res, const uint64_t l, const uint64_t r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_uint_uint_out_of_range(l, r, res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%lu - %lu)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } return ret; } }; //calc_type is UIntTC left and right has same TC int ObExprMinus::minus_uint_uint(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_uint_uint_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObUIntIntBatchMinusRaw : public ObArithOpRawType { static void raw_op(uint64_t &res, const uint64_t l, const int64_t r) { res = l - r; } static int raw_check(const uint64_t res, const uint64_t l, const int64_t r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_uint_int_out_of_range(r, l, res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%lu - %ld)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "BIGINT UNSIGNED", expr_str); } return ret; } }; // calc_type/left_tpee is UIntTC , right is intTC. only mysql mode int ObExprMinus::minus_uint_int(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_uint_int_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObFloatBatchMinusRawNoCheck : public ObArithOpRawType { static void raw_op(float &res, const float l, const float r) { res = l - r; } static int raw_check(const float, const float, const float) { return OB_SUCCESS; } }; struct ObFloatBatchMinusRawWithCheck: public ObFloatBatchMinusRawNoCheck { static int raw_check(const float res, const float l, const float r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_float_out_of_range(res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%e - %e)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "FLOAT", expr_str); LOG_WARN("float out of range", K(l), K(r), K(res)); } return ret; } }; //calc type is floatTC, left and right has same TC, only oracle mode int ObExprMinus::minus_float_float(EVAL_FUNC_ARG_DECL) { return lib::is_oracle_mode() ? def_arith_eval_func>(EVAL_FUNC_ARG_LIST) : def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_float_float_batch(BATCH_EVAL_FUNC_ARG_DECL) { return lib::is_oracle_mode() ? def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST) : def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObDoubleBatchMinusRawNoCheck : public ObArithOpRawType { static void raw_op(double &res, const double l, const double r) { res = l - r; } static int raw_check(const double , const double , const double) { return OB_SUCCESS; } }; struct ObDoubleBatchMinusRawWithCheck: public ObDoubleBatchMinusRawNoCheck { static int raw_check(const double res, const double l, const double r) { int ret = OB_SUCCESS; if (OB_UNLIKELY(ObExprMinus::is_double_out_of_range(res))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; ret = OB_OPERATE_OVERFLOW; int64_t pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%e - %e)'", l, r); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "DOUBLE", expr_str); LOG_WARN("double out of range", K(l), K(r), K(res)); } return ret; } }; //calc type is doubleTC, left and right has same TC int ObExprMinus::minus_double_double(EVAL_FUNC_ARG_DECL) { return lib::is_oracle_mode() || T_OP_AGG_MINUS == expr.type_ ? def_arith_eval_func>(EVAL_FUNC_ARG_LIST) : def_arith_eval_func>(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_double_double_batch(BATCH_EVAL_FUNC_ARG_DECL) { return lib::is_oracle_mode() || T_OP_AGG_MINUS == expr.type_ ? def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST) : def_batch_arith_op>(BATCH_EVAL_FUNC_ARG_LIST); } struct ObNumberMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; char local_buff[ObNumber::MAX_BYTE_LEN]; ObDataBuffer local_alloc(local_buff, ObNumber::MAX_BYTE_LEN); number::ObNumber l_num(l.get_number()); number::ObNumber r_num(r.get_number()); number::ObNumber res_num; if (OB_FAIL(l_num.sub_v3(r_num, res_num, local_alloc))) { LOG_WARN("minus num failed", K(ret), K(l_num), K(r_num)); } else { res.set_number(res_num); } return ret; } }; //calc type TC is ObNumberTC int ObExprMinus::minus_number_number(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_number_number_batch(BATCH_EVAL_FUNC_ARG_DECL) { LOG_DEBUG("minus_number_number_batch begin"); int ret = OB_SUCCESS; ObDatumVector l_datums; ObDatumVector r_datums; const ObExpr &left = *expr.args_[0]; const ObExpr &right = *expr.args_[1]; if (OB_FAIL(binary_operand_batch_eval(expr, ctx, skip, size, lib::is_oracle_mode()))) { LOG_WARN("number minus batch evaluation failure", K(ret)); } else { l_datums = left.locate_expr_datumvector(ctx); r_datums = right.locate_expr_datumvector(ctx); } if (OB_SUCC(ret)) { char local_buff[ObNumber::MAX_BYTE_LEN]; ObDataBuffer local_alloc(local_buff, ObNumber::MAX_BYTE_LEN); ObDatumVector results = expr.locate_expr_datumvector(ctx); ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); for (auto i = 0; OB_SUCC(ret) && i < size; i++) { if (eval_flags.at(i) || skip.at(i)) { continue; } if (l_datums.at(i)->is_null() || r_datums.at(i)->is_null()) { results.at(i)->set_null(); eval_flags.set(i); continue; } ObNumber l_num(l_datums.at(i)->get_number()); ObNumber r_num(r_datums.at(i)->get_number()); uint32_t *res_digits = const_cast (results.at(i)->get_number_digits()); ObNumber::Desc &desc_buf = const_cast (results.at(i)->get_number_desc()); // Notice that, space of desc_buf is allocated in frame but without memset operation, which causes random memory content. // And the reserved in storage layer should be 0, thus you must replacement new here to avoid checksum error, etc. ObNumber::Desc *res_desc = new (&desc_buf) ObNumberDesc(); // speedup detection if (ObNumber::try_fast_minus(l_num, r_num, res_digits, *res_desc)) { results.at(i)->set_pack(sizeof(number::ObCompactNumber) + res_desc->len_ * sizeof(*res_digits)); eval_flags.set(i); // LOG_INFO("mul speedup done", K(l_num.format()), K(r_num.format())); } else { // normal path: no speedup ObNumber res_num; if (OB_FAIL(l_num.sub_v3(r_num, res_num, local_alloc))) { LOG_WARN("mul num failed", K(ret), K(l_num), K(r_num)); } else { results.at(i)->set_number(res_num); eval_flags.set(i); } local_alloc.free(); } } } LOG_DEBUG("minus_number_number_batch done"); return ret; } struct ObIntervalYMIntervalYMMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; ObIntervalYMValue value = l.get_interval_nmonth() - r.get_interval_nmonth(); if (OB_FAIL(value.validate())) { LOG_WARN("value validate failed", K(ret), K(value)); } else { res.set_interval_nmonth(value.get_nmonth()); } return ret; } }; //interval can calc with different types such as date, timestamp, interval. //the params do not need to do cast //left and right have the same type. both IntervalYM. int ObExprMinus::minus_intervalym_intervalym(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_intervalym_intervalym_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObIntervalDSIntervalDSMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; ObIntervalDSValue value = l.get_interval_ds() - r.get_interval_ds(); if (OB_FAIL(value.validate())) { LOG_WARN("value validate failed", K(ret), K(value)); } else { res.set_interval_ds(value); } return ret; } }; //left and right must have the same type. both IntervalDS int ObExprMinus::minus_intervalds_intervalds(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_intervalds_intervalds_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObDatetimeIntervalYMMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; int64_t result_v = 0; ret = ObTimeConverter::date_add_nmonth(l.get_datetime(), -r.get_interval_nmonth(), result_v); if (OB_FAIL(ret)) { LOG_WARN("minus value failed", K(ret), K(l), K(r)); } else { res.set_datetime(result_v); } return ret; } }; //Left is datetime TC. Right is intervalYM type. int ObExprMinus::minus_datetime_intervalym(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_datetime_intervalym_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObDatetimeIntervalDSMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; int64_t result_v = 0; ret = ObTimeConverter::date_add_nsecond(l.get_datetime(), - r.get_interval_ds().get_nsecond(), - r.get_interval_ds().get_fs(), result_v); if (OB_FAIL(ret)) { LOG_WARN("minus value failed", K(ret), K(l), K(r)); } else { res.set_datetime(result_v); } return ret; } }; //Left is datetime TC. Right is intervalDS type. int ObExprMinus::minus_datetime_intervalds(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_datetime_intervalds_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObTimestampTZIntervalYMMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r, ObEvalCtx &ctx) const { int ret = OB_SUCCESS; ObOTimestampData result_v; ret = ObTimeConverter::otimestamp_add_nmonth( ObTimestampTZType, l.get_otimestamp_tz(), get_timezone_info(ctx.exec_ctx_.get_my_session()), - r.get_interval_nmonth(), result_v); if (OB_FAIL(ret)) { LOG_WARN("minus value failed", K(ret), K(l), K(r)); } else { res.set_otimestamp_tz(result_v); } return ret; } }; //Left is timestampTZ type. Right is intervalYM type. int ObExprMinus::minus_timestamptz_intervalym(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST, ctx); } int ObExprMinus::minus_timestamptz_intervalym_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST, ctx); } struct ObTimestampLTZIntervalYMMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r, ObEvalCtx &ctx) const { int ret = OB_SUCCESS; ObOTimestampData result_v; ret = ObTimeConverter::otimestamp_add_nmonth( ObTimestampLTZType, l.get_otimestamp_tiny(), get_timezone_info(ctx.exec_ctx_.get_my_session()), - r.get_interval_nmonth(), result_v); if (OB_FAIL(ret)) { LOG_WARN("minus value failed", K(ret), K(l), K(r)); } else { res.set_otimestamp_tiny(result_v); } return ret; } }; //Left is timestampLTZ type. Right is intervalYM type. int ObExprMinus::minus_timestampltz_intervalym(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST, ctx); } int ObExprMinus::minus_timestampltz_intervalym_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST, ctx); } struct ObTimestampNanoIntervalYMMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r, ObEvalCtx &ctx) const { int ret = OB_SUCCESS; ObOTimestampData result_v; ret = ObTimeConverter::otimestamp_add_nmonth( ObTimestampNanoType, l.get_otimestamp_tiny(), get_timezone_info(ctx.exec_ctx_.get_my_session()), - r.get_interval_nmonth(), result_v); if (OB_FAIL(ret)) { LOG_WARN("calc with timestamp value failed", K(ret), K(l), K(r)); } else { res.set_otimestamp_tiny(result_v); } return ret; } }; //Left is timestampNano type. Right is intervalYM type. int ObExprMinus::minus_timestampnano_intervalym(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST, ctx); } int ObExprMinus::minus_timestampnano_intervalym_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST, ctx); } struct ObTimestampTZIntervalDSMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; ObOTimestampData result_v; ret = ObTimeConverter::otimestamp_add_nsecond(l.get_otimestamp_tz(), - r.get_interval_ds().get_nsecond(), - r.get_interval_ds().get_fs(), result_v); if (OB_FAIL(ret)) { LOG_WARN("calc with timestamp value failed", K(ret), K(l), K(r)); } else { res.set_otimestamp_tz(result_v); } return ret; } }; //Left is timestamp TZ. Right is intervalDS type. int ObExprMinus::minus_timestamptz_intervalds(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_timestamptz_intervalds_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObTimestampLTZIntervalDSMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; ObOTimestampData result_v; ret = ObTimeConverter::otimestamp_add_nsecond(l.get_otimestamp_tiny(), - r.get_interval_ds().get_nsecond(), - r.get_interval_ds().get_fs(), result_v); if (OB_FAIL(ret)) { LOG_WARN("calc with timestamp value failed", K(ret), K(l), K(r)); } else { res.set_otimestamp_tiny(result_v); } return ret; } }; //Left is timestamp LTZ or Nano. Right is intervalDS type. int ObExprMinus::minus_timestamp_tiny_intervalds(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_timestamp_tiny_intervalds_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObTimestampTimestampMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r, ObEvalCtx &ctx, const ObObjType &left_type, const ObObjType &right_type) const { int ret = OB_SUCCESS; ObOTimestampData left_v, right_v; ObIntervalDSValue result_intervalds; const ObTimeZoneInfo *tz_info = get_timezone_info(ctx.exec_ctx_.get_my_session()); if (ObDateTimeType == left_type) { ret = ObTimeConverter::odate_to_otimestamp(l.get_datetime(), tz_info, ObTimestampTZType, left_v); } else if (ObTimestampNanoType == left_type) { ret = ObTimeConverter::odate_to_otimestamp(l.get_otimestamp_tiny().time_us_, tz_info, ObTimestampTZType, left_v); left_v.time_ctx_.tail_nsec_ = l.get_otimestamp_tiny().time_ctx_.tail_nsec_; } else if (ObTimestampTZType == left_type) { ret = ObTimeConverter::otimestamp_to_otimestamp(ObTimestampTZType, l.get_otimestamp_tz(), tz_info, ObTimestampTZType, left_v); } else { ret = ObTimeConverter::otimestamp_to_otimestamp(ObTimestampLTZType, l.get_otimestamp_tiny(), tz_info, ObTimestampTZType, left_v); } if (OB_FAIL(ret)) { LOG_WARN("fail to convert left to timestamp tz", K(ret), K(l)); } if (OB_SUCC(ret)) { if (ObDateTimeType == right_type) { ret = ObTimeConverter::odate_to_otimestamp(r.get_datetime(), tz_info, ObTimestampTZType, right_v); } else if (ObTimestampNanoType == right_type) { ret = ObTimeConverter::odate_to_otimestamp(r.get_otimestamp_tiny().time_us_, tz_info, ObTimestampTZType, right_v); right_v.time_ctx_.tail_nsec_ = r.get_otimestamp_tiny().time_ctx_.tail_nsec_; } else if (ObTimestampTZType == right_type) { ret = ObTimeConverter::otimestamp_to_otimestamp(ObTimestampTZType, r.get_otimestamp_tz(), tz_info, ObTimestampTZType, right_v); } else { ret = ObTimeConverter::otimestamp_to_otimestamp(ObTimestampLTZType, r.get_otimestamp_tiny(), tz_info, ObTimestampTZType, right_v); } if (OB_FAIL(ret)) { LOG_WARN("fail to convert right to timestamp tz", K(ret), K(r)); } } if (OB_SUCC(ret)) { ObTimeConverter::calc_oracle_temporal_minus(left_v, right_v, result_intervalds); res.set_interval_ds(result_intervalds); } return ret; } }; // only oracle mode. //both left and right are datetimeTC or otimestampTC. //cast left and right to ObTimestampTZType first. int ObExprMinus::minus_timestamp_timestamp(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func( EVAL_FUNC_ARG_LIST, ctx, expr.args_[0]->datum_meta_.type_, expr.args_[1]->datum_meta_.type_); } int ObExprMinus::minus_timestamp_timestamp_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST, ctx, expr.args_[0]->datum_meta_.type_, expr.args_[1]->datum_meta_.type_); } struct ObDatetimeNumberMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; number::ObNumber right_nmb(r.get_number()); const int64_t left_i = l.get_datetime(); int64_t int_part = 0; int64_t dec_part = 0; if (!right_nmb.is_int_parts_valid_int64(int_part,dec_part)) { ret = OB_INVALID_DATE_FORMAT; LOG_WARN("invalid date format", K(ret), K(right_nmb)); } else { const int64_t right_i = static_cast(int_part * USECS_PER_DAY) + (right_nmb.is_negative() ? -1 : 1 ) * static_cast(static_cast(dec_part) / NSECS_PER_SEC * static_cast(USECS_PER_DAY)); int64_t round_value = left_i - right_i; ObTimeConverter::round_datetime(OB_MAX_DATE_PRECISION, round_value); res.set_datetime(round_value); ObTime ob_time; if (OB_UNLIKELY(res.get_datetime() > DATETIME_MAX_VAL || res.get_datetime() < DATETIME_MIN_VAL) || (OB_FAIL(ObTimeConverter::datetime_to_ob_time(res.get_datetime(), NULL, ob_time))) || (OB_FAIL(ObTimeConverter::validate_oracle_date(ob_time)))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %ld)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "DATE", expr_str); } } return ret; } }; int ObExprMinus::minus_datetime_number(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_datetime_number_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObDatetimeDatetimeOralceMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; const int64_t LOCAL_BUF_SIZE = ObNumber::MAX_CALC_BYTE_LEN * 5; char local_buf[LOCAL_BUF_SIZE]; ObDataBuffer local_alloc(local_buf, LOCAL_BUF_SIZE); ObNumber left_datetime; ObNumber right_datetime; ObNumber usecs_per_day; ObNumber sub_datetime; ObNumber sub_date; if (OB_FAIL(left_datetime.from(l.get_datetime(), local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret), K(l.get_datetime())); } else if (OB_FAIL(right_datetime.from(r.get_datetime(), local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret), K(r.get_datetime())); } else if (OB_FAIL(usecs_per_day.from(USECS_PER_DAY, local_alloc))) { LOG_WARN("convert int64 to number failed", K(ret)); } else if (OB_FAIL(left_datetime.sub_v3(right_datetime, sub_datetime, local_alloc))) { LOG_WARN("sub failed", K(ret), K(left_datetime), K(right_datetime)); } else if (OB_FAIL(sub_datetime.div_v3(usecs_per_day, sub_date, local_alloc))) { LOG_WARN("calc left date number failed", K(ret)); } else { res.set_number(sub_date); } return ret; } }; //left and right are both ObDateTimeTC. calc_type is ObNumberType. only oracle mode. int ObExprMinus::minus_datetime_datetime_oracle(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_datetime_datetime_oracle_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } struct ObDatetimeDatetimeMinusFunc { int operator()(ObDatum &res, const ObDatum &l, const ObDatum &r) const { int ret = OB_SUCCESS; const int64_t left_i = l.get_datetime(); const int64_t right_i = r.get_datetime(); ObTime ob_time; int64_t round_value = left_i - right_i; ObTimeConverter::round_datetime(OB_MAX_DATE_PRECISION, round_value); res.set_datetime(round_value); if (OB_UNLIKELY(res.get_datetime() > DATETIME_MAX_VAL || res.get_datetime() < DATETIME_MIN_VAL) || (OB_FAIL(ObTimeConverter::datetime_to_ob_time(res.get_datetime(), NULL, ob_time))) || (OB_FAIL(ObTimeConverter::validate_oracle_date(ob_time)))) { char expr_str[OB_MAX_TWO_OPERATOR_EXPR_LENGTH]; int64_t pos = 0; ret = OB_OPERATE_OVERFLOW; pos = 0; databuff_printf(expr_str, OB_MAX_TWO_OPERATOR_EXPR_LENGTH, pos, "'(%ld - %ld)'", left_i, right_i); LOG_USER_ERROR(OB_OPERATE_OVERFLOW, "DATE", expr_str); } return ret; } }; //calc type is datetimeTC. cast left and right to calc_type. int ObExprMinus::minus_datetime_datetime(EVAL_FUNC_ARG_DECL) { return def_arith_eval_func(EVAL_FUNC_ARG_LIST); } int ObExprMinus::minus_datetime_datetime_batch(BATCH_EVAL_FUNC_ARG_DECL) { return def_batch_arith_op_by_datum_func( BATCH_EVAL_FUNC_ARG_LIST); } } }