diff --git a/src/sql/code_generator/ob_expr_generator_impl.cpp b/src/sql/code_generator/ob_expr_generator_impl.cpp index 5e0a61448c..002cc63f2a 100644 --- a/src/sql/code_generator/ob_expr_generator_impl.cpp +++ b/src/sql/code_generator/ob_expr_generator_impl.cpp @@ -676,7 +676,7 @@ inline int ObExprGeneratorImpl::visit_regex_expr(ObOpRawExpr &expr, ObExprRegexp ObIArray ¶m_exprs = expr.get_param_exprs(); if (OB_ISNULL(regexp_op)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("regexpr_op is NULL"); + LOG_WARN("regexp_op is NULL"); } else if (OB_UNLIKELY(2 != param_exprs.count())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("regex op should have 2 arguments", K(param_exprs.count())); @@ -779,7 +779,7 @@ inline int ObExprGeneratorImpl::visit_in_expr(ObOpRawExpr &expr, ObExprInOrNotIn } } //for row_type in left_param of EXPR IN - //if min_cluster_version < 3.1, do not check params can use hash optimizition + //if min_cluster_version < 3.1, do not check params can use hash optimization bool param_all_const = true; bool param_all_same_type = true; bool param_all_same_cs_type = true; @@ -875,7 +875,7 @@ inline int ObExprGeneratorImpl::visit_in_expr(ObOpRawExpr &expr, ObExprInOrNotIn ? param_all_same_cs_type : (param_all_same_cs_type &= param_all_same_cs_level)); in_op->set_param_is_ext_type_oracle(param_all_is_ext); - //now only support c1 in (1,2,3,4,5...) to vecotrized + //now only support c1 in (1,2,3,4,5...) to vectorized if (param_all_can_vectorize && expr.get_param_expr(0)->is_vectorize_result()) { in_op->set_param_can_vectorized(); } @@ -893,7 +893,7 @@ int ObExprGeneratorImpl::visit_decode_expr(ObNonTerminalRawExpr &expr, ObExprOra } else { // decode(col, cond1, val1, cond2, val2, ......, condN, valN, def_val) // cmp type of decode is always equal to cond1, or varchar if cond1 is const null. - // res type of decode is always euqal to val1, or varchar if val1 is const null. + // res type of decode is always equal to val1, or varchar if val1 is const null. bool cond_all_same_meta = true; bool val_all_same_meta = true; int64_t param_count = expr.get_param_count(); @@ -1454,7 +1454,7 @@ int ObExprGeneratorImpl::visit_pl_integer_checker_expr(ObOpRawExpr &expr, ObPLIntegerCheckerRawExpr &pl_expr = static_cast(expr); if (OB_ISNULL(checker)) { ret = OB_ERR_UNEXPECTED; - LOG_ERROR("pl interger checker operator is null", K(ret)); + LOG_ERROR("pl integer checker operator is null", K(ret)); } else { checker->set_pl_integer_type(pl_expr.get_pl_integer_type()); checker->set_range(pl_expr.get_lower(), pl_expr.get_upper()); diff --git a/src/sql/code_generator/ob_static_engine_cg.cpp b/src/sql/code_generator/ob_static_engine_cg.cpp index 471df021a3..a1066abe7f 100644 --- a/src/sql/code_generator/ob_static_engine_cg.cpp +++ b/src/sql/code_generator/ob_static_engine_cg.cpp @@ -693,7 +693,7 @@ int ObStaticEngineCG::check_vectorize_supported(bool &support, support = false; stop_checking = true; } - LOG_DEBUG("check_vectorie_supported", K(disable_vectorize), K(support), K(stop_checking), + LOG_DEBUG("check_vectorize_supported", K(disable_vectorize), K(support), K(stop_checking), K(op->get_num_of_child())); // continue searching until found an operator with vectorization explicitly disabled for (int64_t i = 0; !stop_checking && OB_SUCC(ret) && i < op->get_num_of_child(); i++) { @@ -709,7 +709,7 @@ int ObStaticEngineCG::check_vectorize_supported(bool &support, // 从raw expr中获取rt_expr,并将raw expr push到cur_op_exprs_中 // -// 设置operater的rt expr, 从raw expr中获取时,均需要通过该接口, +// 设置operator的rt expr, 从raw expr中获取时,均需要通过该接口, // 其中ObStaticEngineExprCG::generate_rt_expr是ObRawExpr的友元函数, 可直接访问ObRawExpr中rt expr, // // 为什么不是ObRawExpr中直接提供访问rt expr的接口给外部使用, 而是用友元函数的方式处理? @@ -2837,7 +2837,7 @@ int ObStaticEngineCG::generate_spec(ObLogGranuleIterator &op, ObGranuleIteratorS "desc", op.desc_order(), "flags", op.get_gi_flags(), "tsc_ids", spec.pw_dml_tsc_ids_, - "repart_prunint_idx", spec.repart_pruning_tsc_idx_, + "repart_pruning_tsc_idx", spec.repart_pruning_tsc_idx_, K(pwj_gi), K(enable_repart_pruning)); return ret; } @@ -3300,7 +3300,7 @@ int ObStaticEngineCG::generate_spec(ObLogExchange &op, ObPxRepartTransmitSpec &s // repartition_exprs_ only use by null aware anti join // now just support single join key // either repart_keys or repart_sub_keys exists join key - // so we can generate from one of them driectly + // so we can generate from one of them directly if (op.get_repart_keys().count() > 0) { if (OB_FAIL(generate_rt_exprs(op.get_repart_keys(), spec.repartition_exprs_))) { LOG_WARN("failed to generate repart exprs", K(ret)); @@ -3792,7 +3792,7 @@ int ObStaticEngineCG::generate_dist_aggr_group(ObLogGroupBy &op, ObGroupBySpec & const ObDistinctAggrBatch &distinct_batch = op.get_distinct_aggr_batch().at(i); aggr_group_idx += distinct_batch.mocked_aggrs_.count(); if (OB_FAIL(spec.dist_aggr_group_idxes_.push_back(aggr_group_idx))) { - LOG_WARN("failed to push back aggr group aggr inndex", K(ret)); + LOG_WARN("failed to push back aggr group aggr index", K(ret)); } } // end for return ret; @@ -3979,7 +3979,7 @@ int ObStaticEngineCG::generate_normal_tsc(ObLogTableScan &op, ObTableScanSpec &s } root = root->and_next_; } - // TODO @baixian.zr the above optimization is overrided by ObTscCgService::generate_tsc_ctdef before this commit + // TODO @baixian.zr the above optimization is overrode by ObTscCgService::generate_tsc_ctdef before this commit // but after the deep copy of pre_query_range_ is removed in ObTscCgService::generate_tsc_ctdef, // error is returned in such sql 'set global x=y', should fix this; // spec.tsc_ctdef_.pre_query_range_.set_is_equal_and(is_equal_and); @@ -4654,7 +4654,7 @@ int ObStaticEngineCG::generate_join_spec(ObLogJoin &op, ObJoinSpec &spec) for (auto l_output_idx = 0; OB_SUCC(ret) && l_output_idx < nlj.get_left()->output_.count(); l_output_idx++) { - // check if left child expr appears in other_condtion + // check if left child expr appears in other_condition bool appears_in_cond = false; if (OB_FAIL(cond->contain_expr( nlj.get_left()->output_.at(l_output_idx), appears_in_cond))) { @@ -4814,7 +4814,7 @@ int ObStaticEngineCG::generate_join_spec(ObLogJoin &op, ObJoinSpec &spec) //} else if (exec_params.count() == 0) { //// Do nothing //} else if (exec_params.count() != 1) { - //// Only one ? expr for all level expr in connent by clause. + //// Only one ? expr for all level expr in connect by clause. //ret = OB_ERR_UNEXPECTED; //LOG_WARN("unexpected exec params count in connect by", K(exec_params.count()), K(ret)); //} else if (OB_FAIL(nlj_op->init_exec_param_count(exec_params.count()))) { @@ -5052,7 +5052,7 @@ int ObStaticEngineCG::recursive_get_column_expr(const ObColumnRefRawExpr *&colum LOG_WARN("get unexpected null", K(ret)); } else if ((table_item->is_generated_table() || table_item->is_temp_table()) && OB_FAIL(recursive_get_column_expr(inner_column, *table_item))) { - LOG_WARN("faield to recursive get column expr", K(ret)); + LOG_WARN("failed to recursive get column expr", K(ret)); } else { column = inner_column; } @@ -5383,7 +5383,7 @@ int ObStaticEngineCG::generate_spec(ObLogInsert &op, ObPxMultiPartSSTableInsertS const ObExecContext *exec_ctx = nullptr; ObLogPlan *log_plan = nullptr; if (OB_FAIL(generate_spec(op, static_cast(spec), in_root_job))) { - LOG_WARN("generge multi part sstable insert spec failed", K(ret)); + LOG_WARN("generate multi part sstable insert spec failed", K(ret)); } else if (OB_ISNULL(log_plan = op.get_plan()) || OB_ISNULL(exec_ctx = log_plan->get_optimizer_context().get_exec_ctx())) { ret = OB_INVALID_ARGUMENT; @@ -6621,7 +6621,7 @@ int ObStaticEngineCG::generate_insert_all_with_das(ObLogInsertAll &op, ObTableIn OB_ISNULL(phy_plan_) || OB_UNLIKELY(op.get_table_list().count() != op.get_insert_all_table_info()->count())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpeceted error", K(ret), K(phy_plan_), K(op.get_insert_all_table_info())); + LOG_WARN("get unexpected error", K(ret), K(phy_plan_), K(op.get_insert_all_table_info())); } else if (OB_FAIL(spec.ins_ctdefs_.allocate_array(phy_plan_->get_allocator(), op.get_table_list().count()))) { LOG_WARN("allocate insert ctdef array failed", K(ret)); @@ -6723,7 +6723,7 @@ int ObStaticEngineCG::generate_spec(ObLogStatCollector &op, } } else { ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpcted type", K(spec.type_)); + LOG_WARN("unexpected type", K(spec.type_)); } return ret; } @@ -7003,7 +7003,7 @@ int ObStaticEngineCG::set_other_properties(const ObLogPlan &log_plan, ObPhysical } if (OB_SUCC(ret) && metas.count() > 0) { if (OB_FAIL(phy_plan.get_encrypt_meta_array().assign(metas))) { - LOG_WARN("fail to assgin encrypt meta", K(ret)); + LOG_WARN("fail to assign encrypt meta", K(ret)); } } } @@ -7546,7 +7546,7 @@ int ObStaticEngineCG::add_output_datum_check_flag(ObOpSpec &spec) } else { // Because the Unpivot will affect the output datum of the SubplanScan, // which is an by designed case, we need to set the SubplanScan operator - // to not check the ouput datum. + // to not check the output datum. spec.get_child(0)->need_check_output_datum_ = false; } } else { diff --git a/src/sql/code_generator/ob_static_engine_expr_cg.cpp b/src/sql/code_generator/ob_static_engine_expr_cg.cpp index bc329f9c7d..5d34227bb5 100644 --- a/src/sql/code_generator/ob_static_engine_expr_cg.cpp +++ b/src/sql/code_generator/ob_static_engine_expr_cg.cpp @@ -229,7 +229,7 @@ int ObStaticEngineExprCG::cg_exprs(const ObIArray &raw_exprs, // cg_expr_parents must be after cg_expr_by_operator, // because cg_expr_by_operator may replace rt_expr.args_ } else if (OB_FAIL(cg_expr_parents(raw_exprs))) { - LOG_WARN("fail to init expr parenets", K(ret), K(raw_exprs)); + LOG_WARN("fail to init expr parents", K(ret), K(raw_exprs)); // init res_buf_len_, frame_idx_, datum_off_, res_buf_off_ } else if (OB_FAIL(cg_all_frame_layout(raw_exprs, expr_info))) { LOG_WARN("fail to init expr data layout", K(ret), K(raw_exprs)); @@ -701,7 +701,7 @@ int ObStaticEngineExprCG::cg_frame_layout_vector_version(const ObIArray &exprs, { int ret = OB_SUCCESS; if (continuous_datum) { - // Layout1: Frame is seperated from meta part and data part. + // Layout1: Frame is separated from meta part and data part. // Meta part(datum header) are allocated continuously. // Reserved data/buf part are allocated continuously // Frame layouts: @@ -987,7 +987,7 @@ int ObStaticEngineExprCG::arrange_datums_data(ObIArray &exprs, CK((total_header_len + expr_data_offset) == frame.frame_size_); } else { // Layout2: Frame is seperated by exprs - // All data(metas + reserved data/buf) within one expr are allocated continously + // All data(metas + reserved data/buf) within one expr are allocated continuously // Frame layouts: // +--------------------------------+ // | Datums in Expr1 | @@ -1297,7 +1297,7 @@ int ObStaticEngineExprCG::calc_exprs_res_buf_len(const ObIArray &ra rt_expr->res_buf_len_ = min(def_res_len, static_cast(rt_expr->max_length_)); } else { - // max_length may eaual -1 + // max_length may equal -1 rt_expr->res_buf_len_ = def_res_len; } } else { @@ -1336,7 +1336,7 @@ int ObStaticEngineExprCG::create_tmp_frameinfo(const common::ObIArray mod_infos_; // tenant trace FLTControlInfo tenant_info_; - // allcator + // allocator ObArenaAllocator alloc_; }; diff --git a/src/sql/monitor/ob_exec_stat.h b/src/sql/monitor/ob_exec_stat.h index 5f3020f61e..c5395a1fc6 100644 --- a/src/sql/monitor/ob_exec_stat.h +++ b/src/sql/monitor/ob_exec_stat.h @@ -367,7 +367,7 @@ struct ObAuditRecordData { int64_t plan_id_; int64_t affected_rows_;//delete,update,insert影响的行数,及select选出的行数 int64_t return_rows_; - int64_t partition_cnt_;//该请求涉及的所以parttion个数 + int64_t partition_cnt_;//该请求涉及的所以partition个数 int64_t expected_worker_cnt_; // px 预期分配线程数 int64_t used_worker_cnt_; // px 实际分配线程数 int64_t try_cnt_; //尝试执行次数 diff --git a/src/sql/monitor/ob_exec_stat_collector.cpp b/src/sql/monitor/ob_exec_stat_collector.cpp index 4122e847db..c64f2ec61a 100644 --- a/src/sql/monitor/ob_exec_stat_collector.cpp +++ b/src/sql/monitor/ob_exec_stat_collector.cpp @@ -82,7 +82,7 @@ int ObExecStatCollector::collect_plan_monitor_info(uint64_t job_id, SQL_MONITOR_LOG(WARN, "fail to get operator info by index", K(ret), K(i)); } else if (OB_ISNULL(op_info)) { ret = OB_ERR_UNEXPECTED; - SQL_MONITOR_LOG(WARN, "get invalie op_info", K(ret), K(op_info)); + SQL_MONITOR_LOG(WARN, "get invalid op_info", K(ret), K(op_info)); } else if (OB_FAIL(op_info->set_job_id(job_id))) { SQL_MONITOR_LOG(WARN, "fail to set job id", K(ret), K(job_id)); } else if (OB_FAIL(op_info->set_task_id(task_id))) { @@ -142,7 +142,7 @@ int ObExecStatDispatch::dispatch(bool need_add_monitor, } else if (need_add_monitor && OB_FAIL(monitor_info->add_operator_info(op_info))) { LOG_WARN("fail to add operator info", K(ret), K(op_info)); } else if (need_update_plan && OB_FAIL(plan->op_stats_.add_op_stat(op_info))) { - LOG_WARN("fail to add operatgor info", K(ret), K(op_info)); + LOG_WARN("fail to add operator info", K(ret), K(op_info)); } break; } diff --git a/src/sql/monitor/ob_phy_operator_stats.cpp b/src/sql/monitor/ob_phy_operator_stats.cpp index e047d8bae7..e4b3ee4f1c 100644 --- a/src/sql/monitor/ob_phy_operator_stats.cpp +++ b/src/sql/monitor/ob_phy_operator_stats.cpp @@ -69,7 +69,7 @@ int ObPhyOperatorStats::add_op_stat(ObPhyOperatorMonitorInfo &info) int64_t stat_start_index = copy_start_index + info.get_op_id() * StatId::MAX_STAT; if (stat_start_index < 0 || stat_start_index + StatId::MAX_STAT > array_size_) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("invaild array index", K(stat_start_index), K(array_size_)); + LOG_WARN("invalid array index", K(stat_start_index), K(array_size_)); } else { int64_t last_input_rows = 0; int64_t last_output_rows = 0; diff --git a/src/sql/monitor/ob_phy_plan_monitor_info.cpp b/src/sql/monitor/ob_phy_plan_monitor_info.cpp index 2438320786..4d41a10e8f 100644 --- a/src/sql/monitor/ob_phy_plan_monitor_info.cpp +++ b/src/sql/monitor/ob_phy_plan_monitor_info.cpp @@ -41,7 +41,7 @@ int ObPhyPlanMonitorInfo::get_operator_info(int64_t op_id, ObPhyOperatorMonitorI ARRAY_FOREACH_NORET(operator_infos_, idx) { if (operator_infos_.at(idx).get_op_id() == op_id) { if (OB_FAIL(info.assign(operator_infos_.at(idx)))) { - LOG_WARN("fail to assgin to phy_operator info", K(ret)); + LOG_WARN("fail to assign to phy_operator info", K(ret)); } else { ret = OB_SUCCESS; } diff --git a/src/sql/monitor/ob_sql_plan.cpp b/src/sql/monitor/ob_sql_plan.cpp index 009548fec9..988e03bdb4 100644 --- a/src/sql/monitor/ob_sql_plan.cpp +++ b/src/sql/monitor/ob_sql_plan.cpp @@ -513,7 +513,7 @@ int ObSqlPlan::escape_quotes(ObSqlPlanItem &plan_item) } /** - * escape qutotes for string value + * escape quotes for string value * oracle: ' => '' * mysql: ' => \' */ @@ -532,7 +532,7 @@ int ObSqlPlan::inner_escape_quotes(char* &ptr, int64_t &length) int64_t pos = 0; if (OB_ISNULL(buf=(char*)allocator_.alloc(buf_len))) { ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("failed to allocate mempry", K(ret)); + LOG_WARN("failed to allocate memory", K(ret)); } else { for (int64_t i = 0; i < length; ++i) { if (ptr[i] == '\'') { diff --git a/src/sql/ob_result_set.cpp b/src/sql/ob_result_set.cpp index 8cf6cc878c..63e7b5c584 100644 --- a/src/sql/ob_result_set.cpp +++ b/src/sql/ob_result_set.cpp @@ -903,7 +903,7 @@ int ObResultSet::close(int &client_ret) } else if (OB_NOT_NULL(physical_plan_)) { //Because of the async close result we need set the partition_hit flag //to the call back param, than close the result. - //But the das framwork set the patition_hit after result is closed. + //But the das framwork set the partition_hit after result is closed. //So we need to set the partition info at here. if (is_end_trans_async()) { ObCurTraceId::TraceId *cur_trace_id = NULL; @@ -1448,7 +1448,7 @@ int ObResultSet::copy_field_columns(const ObPhysicalPlan &plan) } else if (OB_FAIL(field_columns_.push_back(field))) { LOG_WARN("push back field column failed", K(ret)); } else { - LOG_DEBUG("succs to copy field", K(field)); + LOG_DEBUG("success to copy field", K(field)); } } return ret; @@ -1820,8 +1820,8 @@ int ObRemoteResultSet::copy_field_columns( LOG_WARN("failed to reserve field column array", K(ret), K(N)); } for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) { - const ObField &ofield = src_field_columns.at(i); - if (OB_FAIL(field.deep_copy(ofield, &get_mem_pool()))) { + const ObField &obfield = src_field_columns.at(i); + if (OB_FAIL(field.deep_copy(obfield, &get_mem_pool()))) { LOG_WARN("deep copy field failed", K(ret)); } else if (OB_FAIL(field_columns_.push_back(field))) { LOG_WARN("push back field column failed", K(ret)); @@ -1843,7 +1843,7 @@ int ObRemoteResultSet::setup_next_scanner() } else { ObInnerSQLTransmitResult *transmit_result= NULL; - if (!first_response_received_) { /* has not gotten the first scanner responsed */ + if (!first_response_received_) { /* has not gotten the first scanner response */ if (OB_ISNULL(transmit_result = remote_resp_handler_->get_result())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("transmit_result is NULL", K(ret)); @@ -1852,7 +1852,7 @@ int ObRemoteResultSet::setup_next_scanner() } else { scanner_ = &transmit_result->get_scanner(); scanner_iter_ = scanner_->begin(); - first_response_received_ = true; /* has gotten the first scanner responsed already */ + first_response_received_ = true; /* has gotten the first scanner response already */ found_rows_ += scanner_->get_found_rows(); stmt_type_ = transmit_result->get_stmt_type(); const common::ObSArray &src_field_columns = diff --git a/src/sql/ob_spi.h b/src/sql/ob_spi.h index 015bb9aca2..07a6ebaa97 100644 --- a/src/sql/ob_spi.h +++ b/src/sql/ob_spi.h @@ -408,7 +408,7 @@ public: const ObDataType *column_types, int64_t type_count, const bool *exprs_not_null_flag, - const int64_t *pl_integer_rangs, + const int64_t *pl_integer_ranges, bool is_bulk = false, bool is_forall = false, bool is_type_record = false, @@ -424,7 +424,7 @@ public: const ObDataType *column_types, int64_t type_count, const bool *exprs_not_null_flag, - const int64_t *pl_integer_rangs, + const int64_t *pl_integer_ranges, bool is_bulk = false, bool is_returning = false, bool is_type_record = false); @@ -504,7 +504,7 @@ public: const ObDataType *column_types, int64_t type_count, const bool *exprs_not_null_flag, - const int64_t *pl_integer_rangs, + const int64_t *pl_integer_ranges, bool is_bulk, int64_t limit, const ObDataType *return_types, @@ -527,7 +527,7 @@ public: static int spi_extend_collection(pl::ObPLExecCtx *ctx, const ObSqlExpression *collection_expr, - int64_t coluln_count, + int64_t column_count, const ObSqlExpression *n_expr, const ObSqlExpression *i_expr = NULL, uint64_t package_id = OB_INVALID_ID); @@ -791,7 +791,7 @@ private: const ObDataType *column_types, int64_t type_count, const bool *exprs_not_null_flag, - const int64_t *pl_integer_rangs, + const int64_t *pl_integer_ranges, int64_t is_bulk, bool is_forall = false, bool is_type_record = false, diff --git a/src/sql/ob_sql_utils.cpp b/src/sql/ob_sql_utils.cpp index 7148c2f961..cf59652817 100644 --- a/src/sql/ob_sql_utils.cpp +++ b/src/sql/ob_sql_utils.cpp @@ -597,7 +597,7 @@ int ObSQLUtils::calc_calculable_expr(ObSQLSessionInfo *session, SQL_LOG(WARN, "Invalid arguments", K(expr), K(allocator)); } else if (!expr->is_static_scalar_const_expr()) { ret = OB_INVALID_ARGUMENT; - SQL_LOG(WARN, "expr should be calculabe expr", K(*expr), K(ret)); + SQL_LOG(WARN, "expr should be calculable expr", K(*expr), K(ret)); } else if (OB_FAIL(calc_const_expr(session, *expr, result, @@ -4118,7 +4118,7 @@ int ObSQLUtils::create_encode_sortkey_expr( } else if (OB_FAIL(expr_factory.create_raw_expr(T_FUN_SYS_ENCODE_SORTKEY, encode_expr))) { LOG_WARN("failed to create encode_expr", K(ret)); } else { - // Assamble encode sortkey. + // Assemble encode sortkey. for (int64_t i = start_key; OB_SUCC(ret) && i < order_keys.count(); i++) { ObConstRawExpr *nulls_pos_expr = nullptr; ObConstRawExpr *order_expr = nullptr; @@ -5047,7 +5047,7 @@ int ObSQLUtils::find_synonym_ref_obj(const uint64_t database_id, obj_type = ObObjectType::FUNCTION; schema_version = routine_info->get_schema_version(); } else if (OB_FAIL(guard.get_standalone_procedure_info(tenant_id, database_id, object_name, routine_info))) { - LOG_WARN("failed to get procedore info", K(ret)); + LOG_WARN("failed to get procedure info", K(ret)); } else if (nullptr != routine_info) { exist = true; object_id = routine_info->get_object_id(); @@ -5102,7 +5102,7 @@ int ObSQLUtils::print_identifier_require_quotes(ObCollationType collation_type, require = false; if (OB_ISNULL(info)) { ret = OB_INVALID_ARGUMENT; - OB_LOG(WARN, "arguemnt is invalid", K(ret)); + OB_LOG(WARN, "argument is invalid", K(ret)); } else if (ident.length() > 0 && ident[0] == '$') { require = true; } diff --git a/src/sql/parser/ob_parser.cpp b/src/sql/parser/ob_parser.cpp index d188e37b12..b466d9442f 100644 --- a/src/sql/parser/ob_parser.cpp +++ b/src/sql/parser/ob_parser.cpp @@ -499,7 +499,7 @@ ObParser::State ObParser::transform_normal( } break; default: { is_not_pl = true; - LOG_WARN_RET(common::OB_ERR_UNEXPECTED, "unexpecte state", K(state)); + LOG_WARN_RET(common::OB_ERR_UNEXPECTED, "unexpected state", K(state)); } break; } return state; @@ -837,14 +837,14 @@ int32_t ObParser::get_well_formed_errlen(const ObCharsetInfo *charset_info, } return res_len; } -// avoid separeting sql by semicolons in quotes or comment. +// avoid separating sql by semicolons in quotes or comment. void ObParser::get_single_sql(const common::ObString &stmt, int64_t offset, int64_t remain, int64_t &str_len) { /* following two flags are used to mark wether we are in comment, if in comment, ';' can't be used to split sql*/ // in -- comment bool comment_flag = false; // in /*! comment */ or /* comment */ bool c_comment_flag = false; - /* follwing three flags are used to mark wether we are in quotes.*/ + /* following three flags are used to mark wether we are in quotes.*/ // in '', single quotes bool sq_flag = false; // in "", double quotes @@ -1240,7 +1240,7 @@ int ObParser::pre_parse(const common::ObString &stmt, PreParseResult &res) { int ret = OB_SUCCESS; - // /*tracd_id=xxx*/ + // /*trace_id=xxx*/ int64_t len = stmt.length(); if (len <= 13) { //do_nothing diff --git a/src/sql/plan_cache/ob_pc_ref_handle.h b/src/sql/plan_cache/ob_pc_ref_handle.h index ade8794122..7148f5e026 100644 --- a/src/sql/plan_cache/ob_pc_ref_handle.h +++ b/src/sql/plan_cache/ob_pc_ref_handle.h @@ -190,5 +190,5 @@ private: uint64_t tenant_id_; }; } // end namespace sql -} // end namespace oceanbases +} // end namespace oceanbase #endif // !OCEANBASE_PC_REF_HANDLE_H_ diff --git a/src/sql/plan_cache/ob_plan_cache.cpp b/src/sql/plan_cache/ob_plan_cache.cpp index 88e4ecf127..e7c917019c 100644 --- a/src/sql/plan_cache/ob_plan_cache.cpp +++ b/src/sql/plan_cache/ob_plan_cache.cpp @@ -558,7 +558,7 @@ int ObPlanCache::get_plan(common::ObIAllocator &allocator, if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_) || OB_ISNULL(pc_ctx.sql_ctx_.schema_guard_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arguement",K(ret), + LOG_WARN("invalid argument",K(ret), K(pc_ctx.sql_ctx_.schema_guard_), K(pc_ctx.exec_ctx_.get_physical_plan_ctx())); } else if (pc_ctx.sql_ctx_.multi_stmt_item_.is_batched_multi_stmt()) { if (OB_FAIL(construct_multi_stmt_fast_parser_result(allocator, @@ -1906,7 +1906,7 @@ int64_t ObPlanCache::get_label_hold(lib::ObLabel &label) const // } else if (plan->get_mem_size() >= get_mem_high()) { // // plan mem is too big to reach memory highwater, do not add plan // } else if (OB_FAIL(construct_plan_cache_key(pc_ctx, ObLibCacheNameSpace::NS_CRSR))) { -// LOG_WARN("fail to construnct plan cache key", K(ret)); +// LOG_WARN("fail to construct plan cache key", K(ret)); // } else if (OB_FAIL(add_plan_cache(pc_ctx, plan))) { // if (OB_FAIL(deal_add_ps_plan_result(ret, pc_ctx, *plan))) { // LOG_WARN("fail to deal result code", K(ret)); @@ -1946,7 +1946,7 @@ int ObPlanCache::add_ps_plan(T *plan, ObPlanCacheCtx &pc_ctx) } else if (plan->get_mem_size() >= get_mem_high()) { // plan mem is too big to reach memory highwater, do not add plan } else if (OB_FAIL(construct_plan_cache_key(pc_ctx, ObLibCacheNameSpace::NS_CRSR))) { - LOG_WARN("fail to construnct plan cache key", K(ret)); + LOG_WARN("fail to construct plan cache key", K(ret)); } else if (OB_ISNULL(pc_ctx.raw_sql_.ptr())) { ret = OB_ERR_UNEXPECTED; SQL_PC_LOG(WARN, "pc_ctx.raw_sql_.ptr() is NULL, cannot add plan to plan cache by sql", K(ret)); @@ -2022,7 +2022,7 @@ int ObPlanCache::deal_add_ps_plan_result(int add_plan_ret, } } else { pc_ctx.sql_ctx_.self_add_plan_ = true; - LOG_TRACE("Successed to add plan to ObPlanCache", K(cache_object)); + LOG_TRACE("Succeed to add plan to ObPlanCache", K(cache_object)); } return ret; @@ -2186,7 +2186,7 @@ int ObPlanCache::get_ps_plan(ObCacheObjGuard& guard, } else if (FALSE_IT(original_param_cnt = pc_ctx.fp_result_.cache_params_->count())) { // do nothing } else if (OB_FAIL(construct_plan_cache_key(pc_ctx, ObLibCacheNameSpace::NS_CRSR))) { - LOG_WARN("fail to construnct plan cache key", K(ret)); + LOG_WARN("fail to construct plan cache key", K(ret)); } else { ObPsStmtId new_stmt_id = pc_ctx.fp_result_.pc_key_.key_id_; // the remote plan uses key_id is 0 to distinguish, so if key_id is 0, it cannot be set to OB_INVALID_ID diff --git a/src/sql/plan_cache/ob_plan_cache_callback.cpp b/src/sql/plan_cache/ob_plan_cache_callback.cpp index 4d42ff5dfd..efa308f8de 100644 --- a/src/sql/plan_cache/ob_plan_cache_callback.cpp +++ b/src/sql/plan_cache/ob_plan_cache_callback.cpp @@ -57,7 +57,7 @@ from map | dec ref cnt => ref_cnt=0 | | ref plan id x. inc_ref=1 deleting plan x | - | acess plan x --> cause core! + | access plan x --> cause core! */ void ObCacheObjAtomicOp::operator()(ObjKV &entry) diff --git a/src/sql/plan_cache/ob_plan_cache_util.h b/src/sql/plan_cache/ob_plan_cache_util.h index 0ccee2993c..f33d749511 100644 --- a/src/sql/plan_cache/ob_plan_cache_util.h +++ b/src/sql/plan_cache/ob_plan_cache_util.h @@ -368,7 +368,7 @@ struct ObOperatorStat //不再记录last的执行结果 //int64_t last_input_rows_; //上次input rows //int64_t last_rescan_times_; //rescan的次数 - //int64_t last_output_rows_; //output rows in last exectuion + //int64_t last_output_rows_; //output rows in last execution //暂时不支持以下和oracle兼容的统计项 //int64_t last_cr_buffer_gets_; //上次执行逻辑读次数 //int64_t cr_buffer_gets_; //累计逻辑读次数 @@ -539,7 +539,7 @@ struct ObPlanStat int64_t delayed_large_querys_; int64_t delayed_px_querys_; // px query 被丢回队列重试的次数 int64_t expected_worker_count_; // px 预期分配线程数 - int64_t minimal_worker_count_; // minial threads required for query + int64_t minimal_worker_count_; // minimal threads required for query int64_t outline_version_; int64_t outline_id_; bool is_last_exec_succ_; // record whether last execute success @@ -976,14 +976,14 @@ public: * * update cached configs * 1. @Funciton load_influence_plan_config() will load values - * 2. @Function serialize_configs() will serialze config values to strings and plan cache will + * 2. @Function serialize_configs() will serialize config values to strings and plan cache will * compare this string so as to figure out whether configs has changed. * 3. after generate string, @Function should do @Function update_version() * * add configs has influence in execution plan. @see load_influence_plan_config(); * * NOTES: - * to add configs that will influence exectuion plan, please add to following funcs: + * to add configs that will influence execution plan, please add to following funcs: * 1. load_influence_plan_config(); * 2. serialize_configs(); * 3. adds default values to ObBasicSessionInfo::load_default_configs_in_pc() @@ -1011,7 +1011,7 @@ public: } // init tenant_id_ void init(int t_id) {tenant_id_ = t_id;} - // load configs which will influence exectuion plan + // load configs which will influence execution plan int load_influence_plan_config(); // generate config string int serialize_configs(char *buf, int buf_len, int64_t &pos); diff --git a/src/sql/plan_cache/ob_ps_cache.cpp b/src/sql/plan_cache/ob_ps_cache.cpp index 358d5d2f9c..df6f2263e4 100644 --- a/src/sql/plan_cache/ob_ps_cache.cpp +++ b/src/sql/plan_cache/ob_ps_cache.cpp @@ -294,7 +294,7 @@ int ObPsCache::get_or_add_stmt_item(uint64_t db_id, new_item_value->~ObPsStmtItem(); inner_allocator_->free(new_item_value); } else { - LOG_WARN("unexpecte error", K(ret), K(new_stmt_id)); + LOG_WARN("unexpected error", K(ret), K(new_stmt_id)); new_item_value->~ObPsStmtItem(); inner_allocator_->free(new_item_value); } @@ -339,7 +339,7 @@ int ObPsCache::ref_stmt_item(const ObPsSqlKey &ps_sql_key, if (OB_SUCCESS != (callback_ret = op.get_callback_ret())) { ret = callback_ret; if (OB_EAGAIN == ret) { - LOG_INFO("try egain", K(ret), "stmt_id", ps_sql_key, K(retry_cnt)); + LOG_INFO("try again", K(ret), "stmt_id", ps_sql_key, K(retry_cnt)); ob_usleep(static_cast(500)); //sleep 500us } } else if (OB_FAIL(op.get_value(ps_stmt_item))) { @@ -352,7 +352,7 @@ int ObPsCache::ref_stmt_item(const ObPsSqlKey &ps_sql_key, break; } case OB_EAGAIN: { - LOG_WARN("try egain", K(ret), "stmt_id", ps_sql_key, K(retry_cnt)); + LOG_WARN("try again", K(ret), "stmt_id", ps_sql_key, K(retry_cnt)); ob_usleep(static_cast(500)); //sleep 500us break; } @@ -531,7 +531,7 @@ int ObPsCache::fill_ps_stmt_info(const ObResultSet &result, LOG_WARN("invalid argument", K(params), K(columns), K(sql_ctx), K(ret)); } else if (OB_FAIL(ps_stmt_info.reserve_ps_meta_fields(params->count(), columns->count()))) { - LOG_WARN("fail to reserver ps meta field", K(ret)); + LOG_WARN("fail to reserve ps meta field", K(ret)); } for (int i = 0; OB_SUCC(ret) && i < params->count(); ++i) { if (OB_FAIL(ps_stmt_info.add_param_field(params->at(i)))) { @@ -662,7 +662,7 @@ int ObPsCache::ref_stmt_info(const ObPsStmtId stmt_id, ObPsStmtInfo *&ps_stmt_in if (OB_SUCCESS != (callback_ret = op.get_callback_ret())) { ret = callback_ret; if (OB_EAGAIN == ret) { - LOG_INFO("try egain", K(ret), K(stmt_id), K(retry_cnt)); + LOG_INFO("try again", K(ret), K(stmt_id), K(retry_cnt)); ob_usleep(static_cast(500)); //sleep 500us } } else if (OB_FAIL(op.get_value(ps_stmt_info))) { @@ -675,7 +675,7 @@ int ObPsCache::ref_stmt_info(const ObPsStmtId stmt_id, ObPsStmtInfo *&ps_stmt_in break; } case OB_EAGAIN: { - LOG_INFO("try egain", K(ret), K(stmt_id), K(retry_cnt)); + LOG_INFO("try again", K(ret), K(stmt_id), K(retry_cnt)); ob_usleep(static_cast(500)); //sleep 500us break; } diff --git a/src/sql/plan_cache/ob_sql_parameterization.cpp b/src/sql/plan_cache/ob_sql_parameterization.cpp index 377dee8e87..eb5943ff21 100644 --- a/src/sql/plan_cache/ob_sql_parameterization.cpp +++ b/src/sql/plan_cache/ob_sql_parameterization.cpp @@ -133,7 +133,7 @@ TransformTreeCtx::TransformTreeCtx() : } // replace const expr with ? in syntax tree -// seprate params from syntax tree +// separate params from syntax tree int ObSqlParameterization::transform_syntax_tree(ObIAllocator &allocator, const ObSQLSessionInfo &session, const ObIArray *raw_params, @@ -644,7 +644,7 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, p_info.raw_text_pos_ = ctx.tree_->sql_str_off_; if (ctx.tree_->sql_str_off_ == -1) { ret = OB_NOT_SUPPORTED; - LOG_WARN("invlid str off", K(lbt()), K(ctx.tree_), + LOG_WARN("invalid str off", K(lbt()), K(ctx.tree_), K(ctx.tree_->raw_param_idx_), K(get_type_name(node_type)), K(session_info.get_current_query_string()), "result_tree_", SJ(ObParserResultPrintWrapper(*ctx.top_node_))); @@ -709,12 +709,12 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, // transform `operand - const_num_val` to `operand + (-const_num_val)` if (OB_SUCC(ret) && OB_FAIL(transform_minus_op(*(ctx.allocator_), ctx.tree_, ctx.is_from_pl_))) { - LOG_WARN("failed to transform minums operation", K(ret)); + LOG_WARN("failed to transform minus operation", K(ret)); } else if (lib::is_oracle_mode()) { // in oracle mode, select +-1 from dual is prohibited, but with following orders, it can be executed successfully: - // 1. select +1 from dual; (genereted plan with key: select +? from dual) + // 1. select +1 from dual; (generated plan with key: select +? from dual) // 2. select +-1 from dual; (hit plan before, executed successfully) - // Thus, add a constraint in plan cache: the numeric value following `-` or `+` operators must be posbitive number + // Thus, add a constraint in plan cache: the numeric value following `-` or `+` operators must be positive number if (T_OP_POS == ctx.tree_->type_ || T_OP_NEG == ctx.tree_->type_ ) { if (OB_ISNULL(ctx.tree_->children_) || ctx.tree_->num_child_ != 1 @@ -731,7 +731,7 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, } } if (T_LIMIT_CLAUSE == ctx.tree_->type_) { - // limit a offset b, a and b must be postive + // limit a offset b, a and b must be positive // 0 is counted as positive, -0 is counted as negative if (OB_ISNULL(ctx.tree_->children_) || 2 != ctx.tree_->num_child_) { ret = OB_INVALID_ARGUMENT; @@ -745,7 +745,7 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, && FALSE_IT(ctx.tree_->children_[1]->is_num_must_be_pos_ = 1)) { } } else if (T_COMMA_LIMIT_CLAUSE == ctx.tree_->type_) { - // limit a, b, a and b must be postive + // limit a, b, a and b must be positive if (OB_ISNULL(ctx.tree_->children_) || 2 != ctx.tree_->num_child_ || OB_ISNULL(ctx.tree_->children_[0]) @@ -890,13 +890,13 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, p_info.raw_text_pos_ = root->sql_str_off_; if (root->sql_str_off_ == -1) { ret = OB_NOT_SUPPORTED; - LOG_WARN("invlid str off", K(lbt()), K(ctx.tree_), + LOG_WARN("invalid str off", K(lbt()), K(ctx.tree_), K(root->raw_param_idx_), K(get_type_name(root->type_)), K(session_info.get_current_query_string()), "result_tree_", SJ(ObParserResultPrintWrapper(*ctx.top_node_))); } if (OB_FAIL(ret)) { - // do nithing + // do nothing } else if (OB_FAIL(ctx.sql_info_->parse_infos_.push_back(p_info))) { SQL_PC_LOG(WARN, "fail to push parser info", K(ret)); } @@ -1679,7 +1679,7 @@ int ObSqlParameterization::add_not_param_flag(const ParseNode *node, SqlInfo &sq p_info.raw_text_pos_ = node->sql_str_off_; if (node->sql_str_off_ == -1) { ret = OB_NOT_SUPPORTED; - LOG_WARN("invlid str off", K(lbt()), K(node), + LOG_WARN("invalid str off", K(lbt()), K(node), K(node->raw_param_idx_), K(get_type_name(node->type_))); } if (OB_FAIL(ret)) { @@ -1702,7 +1702,7 @@ int ObSqlParameterization::add_not_param_flag(const ParseNode *node, SqlInfo &sq p_info.raw_text_pos_ = node->sql_str_off_; if (node->sql_str_off_ == -1) { ret = OB_NOT_SUPPORTED; - LOG_WARN("invlid str off", K(lbt()), K(node), + LOG_WARN("invalid str off", K(lbt()), K(node), K(node->raw_param_idx_), K(get_type_name(node->type_))); } if (OB_FAIL(ret)) { @@ -2111,7 +2111,7 @@ int ObSqlParameterization::get_select_item_param_info(const common::ObIArraychildren_[i])) { stack_frames.at(frame_idx).next_child_idx_ = i + 1; } else if (OB_FAIL(stack_frames.push_back(TraverseStackFrame{frame.cur_node_->children_[i], 0}))) { - LOG_WARN("failed to push back eleemnt", K(ret)); + LOG_WARN("failed to push back element", K(ret)); } else { stack_frames.at(frame_idx).next_child_idx_ = i + 1; LOG_DEBUG("after pushing frame", K(stack_frames)); @@ -2162,7 +2162,7 @@ int ObSqlParameterization::resolve_paramed_const(SelectItemTraverseCtx &ctx) SQL_PC_LOG(WARN, "invalid argument", K(ret), K(idx), K(ctx.raw_params_.count())); } else if (OB_ISNULL(ctx.raw_params_.at(idx)) || OB_ISNULL(ctx.raw_params_.at(idx)->node_)) { ret = OB_INVALID_ARGUMENT; - SQL_PC_LOG(WARN, "invalid arguemnt", K(ret)); + SQL_PC_LOG(WARN, "invalid argument", K(ret)); } else { const ParseNode *param_node = ctx.raw_params_.at(idx)->node_; int64_t tmp_len = std::min(ctx.buf_len_ - ctx.param_info_.name_len_, param_node->raw_sql_offset_ - ctx.expr_pos_);