diff --git a/.gitignore b/.gitignore index 59275ebb1d..3b9540bc14 100644 --- a/.gitignore +++ b/.gitignore @@ -345,3 +345,6 @@ deps/3rd ############# test ############# test/tmp test/var + +single_ob2 +store diff --git a/src/sql/optimizer/ob_join_order.cpp b/src/sql/optimizer/ob_join_order.cpp index 1e51a86930..92b404e09f 100644 --- a/src/sql/optimizer/ob_join_order.cpp +++ b/src/sql/optimizer/ob_join_order.cpp @@ -3064,7 +3064,7 @@ int ObJoinOrder::check_expr_overlap_index(const ObRawExpr* qual, return ret; } -/* 拿到quals中涉及的 column的列的id 这个函数在抽取不出query range和 intersting order的情况下调用 */ +/* 拿到quals中涉及的 column的列的id 这个函数在抽取不出query range和 interesting order的情况下调用 */ int ObJoinOrder::extract_filter_column_ids(const ObIArray &quals, const bool is_data_table, const ObTableSchema &index_schema, @@ -3307,7 +3307,7 @@ int ObJoinOrder::get_candi_range_expr(const ObIArray &range_columns, std::sort(sorted_predicates.begin(), sorted_predicates.end(), compare_op); LOG_TRACE("sort predicates and calc cost", K(min_cost), K(sorted_predicates)); } - //for earch candi range expr, check scan cost + //for each candi range expr, check scan cost for (int64_t i = 0; OB_SUCC(ret) && has_in_pred && i < sorted_predicates.count(); ++i) { CandiRangeExprs *candi_exprs = sorted_predicates.at(i); ObRawExpr *min_cost_in_expr = NULL; @@ -4279,7 +4279,7 @@ int ObJoinOrder::add_path(Path* path) path->contain_match_all_fake_cte() && !path->is_remote()) { should_add = false; - OPT_TRACE("containt match all fake cte, but not remote path, will not add path"); + OPT_TRACE("contain match all fake cte, but not remote path, will not add path"); } for (int64_t i = interesting_paths_.count() - 1; OB_SUCC(ret) && should_add && i >= 0; --i) { Path *cur_path = interesting_paths_.at(i); @@ -4423,13 +4423,13 @@ int ObJoinOrder::compute_path_relationship(const Path &first_path, OPT_TRACE("the sharding of the two paths is equal"); } else if (temp_relation == DominateRelation::OBJ_LEFT_DOMINATE) { left_dominated_count++; - OPT_TRACE("left path dominate right path beacuse of sharding"); + OPT_TRACE("left path dominate right path because of sharding"); if (right_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; } } else if (temp_relation == DominateRelation::OBJ_RIGHT_DOMINATE) { right_dominated_count++; - OPT_TRACE("right path dominate left path beacuse of sharding"); + OPT_TRACE("right path dominate left path because of sharding"); if (left_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; } @@ -4450,12 +4450,12 @@ int ObJoinOrder::compute_path_relationship(const Path &first_path, OPT_TRACE("both path is pipeline"); } else if (temp_relation == DominateRelation::OBJ_LEFT_DOMINATE) { left_dominated_count++; - OPT_TRACE("left path dominate right path beacuse of pipeline"); + OPT_TRACE("left path dominate right path because of pipeline"); if (right_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; } } else if (temp_relation == DominateRelation::OBJ_RIGHT_DOMINATE) { - OPT_TRACE("right path dominate left path beacuse of pipeline"); + OPT_TRACE("right path dominate left path because of pipeline"); right_dominated_count++; if (left_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; @@ -4483,12 +4483,12 @@ int ObJoinOrder::compute_path_relationship(const Path &first_path, OPT_TRACE("the interesting order of the two paths is equal"); } else if (temp_relation == DominateRelation::OBJ_LEFT_DOMINATE) { left_dominated_count++; - OPT_TRACE("left path dominate right path beacuse of interesting order"); + OPT_TRACE("left path dominate right path because of interesting order"); if (right_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; } } else if (temp_relation == DominateRelation::OBJ_RIGHT_DOMINATE) { - OPT_TRACE("right path dominate left path beacuse of interesting order"); + OPT_TRACE("right path dominate left path because of interesting order"); right_dominated_count++; if (left_dominated_count > 0) { relation = DominateRelation::OBJ_UNCOMPARABLE; @@ -5610,7 +5610,7 @@ int JoinPath::compute_join_path_ordering() if (!is_left_need_sort()) { set_interesting_order_info(left_path_->get_interesting_order_info()); if(OB_FAIL(append(ordering_, left_path_->ordering_))) { - LOG_WARN("failed to append join orderign", K(ret)); + LOG_WARN("failed to append join ordering", K(ret)); } else if (OB_FAIL(parent_->check_join_interesting_order(this))) { LOG_WARN("failed to update join interesting order info", K(ret)); } else { @@ -6664,7 +6664,7 @@ int JoinPath::cost_hash_join(int64_t join_parallel, right_rows /= in_parallel; } else if (DistAlgo::DIST_BC2HOST_NONE == join_dist_algo_) { // only for shared hash join - // right_rows is same as the impelentation of DIST_BROADCAST_NONE + // right_rows is same as the implementation of DIST_BROADCAST_NONE // left_rows is left_rows / parallel * server right_rows /= in_parallel; left_rows = left_rows / in_parallel * server_cnt_; @@ -9363,7 +9363,7 @@ int ObJoinOrder::find_minimal_cost_merge_path(const Path &left_path, DistAlgo::DIST_BASIC_METHOD == join_dist_algo) && left_merge_key.need_sort_ && right_need_sort && prune_mj) { // do nothing - OPT_TRACE("prune merge join,beacuse both left and right path need sort"); + OPT_TRACE("prune merge join,because both left and right path need sort"); } else if (OB_FAIL(JoinPath::compute_join_path_parallel_and_server_info(opt_ctx.get_local_server_addr(), &left_path, right_path, @@ -10274,7 +10274,7 @@ int ObJoinOrder::remove_invalid_join_filter_infos(ObIArray &join for (int i = 0; OB_SUCC(ret) && i < join_filter_infos.count(); ++i) { if (!join_filter_infos.at(i).can_use_join_filter_ && !join_filter_infos.at(i).need_partition_join_filter_) { - //do nothong + //do nothing } else if (OB_FAIL(new_infos.push_back(join_filter_infos.at(i)))) { LOG_WARN("failed to push back join filter info", K(ret)); } @@ -10351,7 +10351,7 @@ int ObJoinOrder::create_and_add_mj_path(const Path *left_path, } else if (OB_FAIL(append(join_path->right_sort_keys_, right_sort_keys))) { LOG_WARN("failed to append right expected ordering", K(ret)); } else if (OB_FAIL(append(join_path->merge_directions_, merge_directions))) { - LOG_WARN("faield to append merge directions", K(ret)); + LOG_WARN("failed to append merge directions", K(ret)); } else if (OB_FAIL(join_path->compute_join_path_property())) { LOG_WARN("failed to compute join path property", K(ret)); } else if (OB_FAIL(create_subplan_filter_for_join_path(join_path, @@ -10461,7 +10461,7 @@ int ObJoinOrder::classify_hashjoin_conditions(const ObJoinOrder &left_tree, other_join_conditions, join_type, naaj_info))) { - LOG_WARN("failed to extract hash join condtiions and filters", K(join_type), K(ret)); + LOG_WARN("failed to extract hash join conditions and filters", K(join_type), K(ret)); } else if (IS_OUTER_OR_CONNECT_BY_JOIN(join_type) && OB_FAIL(append(filters, where_filters))) { LOG_WARN("failed to append join quals", K(ret)); @@ -10684,7 +10684,7 @@ int ObJoinOrder::is_onetime_expr(const ObRelIds &ignore_relids,ObRawExpr* expr, ret = OB_ERR_UNEXPECTED; LOG_WARN("expr is null", K(ret)); } else { - // if a expr contain psedu column, hierachical expr, any column + // if a expr contain psedu column, hierarchical expr, any column is_valid = !ObOptimizerUtil::has_psedu_column(*expr) && !ObOptimizerUtil::has_hierarchical_expr(*expr) && @@ -10735,7 +10735,7 @@ int ObJoinOrder::create_onetime_expr(const ObRelIds &ignore_relids, ObRawExpr* & ObExecParamRawExpr *new_expr = NULL; ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory(); if (OB_FAIL(expr_factory.create_raw_expr(T_QUESTIONMARK, new_expr))) { - LOG_WARN("faield to create exec param expr", K(ret)); + LOG_WARN("failed to create exec param expr", K(ret)); } else if (OB_ISNULL(new_expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("new expr is null", K(ret), K(new_expr)); @@ -11201,7 +11201,7 @@ int ObJoinOrder::get_simple_index_info(const uint64_t table_id, /** * prefix_filters: 影响query_range range范围的filter * pushdown prefix filters: push down filters that can contribute query range - * posfix_filters: filters that can be evaluated on index + * postfix_filters: filters that can be evaluated on index * table_filters: filters that can be evaluated after index back */ int ObJoinOrder::fill_filters(const ObIArray &all_filters, @@ -11529,7 +11529,7 @@ int ObJoinOrder::fill_path_index_meta_info(const uint64_t table_id, } else if (OB_ISNULL(get_plan()) || OB_ISNULL(schema_guard = OPT_CTX.get_sql_schema_guard())) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("nulll point error", K(schema_guard), K(get_plan()), K(ret)); + LOG_WARN("null point error", K(schema_guard), K(get_plan()), K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < access_paths.count(); ++i) { AccessPath *ap = access_paths.at(i); @@ -12289,7 +12289,7 @@ int ObJoinOrder::calc_join_output_rows(ObLogPlan *plan, } } if (OB_SUCC(ret)) { - // although we compute join row count as anti join, but here selectivity is trated as join + // although we compute join row count as anti join, but here selectivity is treated as join // selectivity. So refine selectivity as output_row / (left_row * right_row) selectivity = new_rows / (left_output_rows * right_output_rows); } @@ -13534,7 +13534,7 @@ int ObJoinOrder::extract_pushdown_quals(const ObIArray &quals, ObRawExpr *qual = quals.at(i); if (OB_ISNULL(qual)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexplected null", K(qual), K(ret)); + LOG_WARN("get unexpected null", K(qual), K(ret)); // can not push down expr with subquery } else if (qual->has_flag(CNT_PSEUDO_COLUMN) || qual->has_flag(CNT_PRIOR) || @@ -13743,7 +13743,7 @@ int ObJoinOrder::deduce_prefix_str_idx_exprs(ObRawExpr *expr, type, new_expr, helper))) { - LOG_WARN("get_prefix str idx exprs faield", K(ret)); + LOG_WARN("get_prefix str idx exprs failed", K(ret)); } else { //do nothing } diff --git a/src/sql/optimizer/ob_opt_est_cost_model.h b/src/sql/optimizer/ob_opt_est_cost_model.h index 39fe8b54c4..79cd376bcd 100644 --- a/src/sql/optimizer/ob_opt_est_cost_model.h +++ b/src/sql/optimizer/ob_opt_est_cost_model.h @@ -117,7 +117,7 @@ struct ObIndexMetaInfo uint64_t index_id_; // index id int64_t index_micro_block_size_; //index micro block size, same as main table when path is primary uint64_t index_part_count_; - double index_part_size_; //index table partitoin(0) data size, same as main table when path is primary + double index_part_size_; //index table partition(0) data size, same as main table when path is primary int64_t index_column_count_; //index column count bool is_index_back_; // is index back bool is_unique_index_; // is unique index diff --git a/src/sql/optimizer/ob_route_policy.cpp b/src/sql/optimizer/ob_route_policy.cpp index c41eb5a7db..97510f5a6d 100644 --- a/src/sql/optimizer/ob_route_policy.cpp +++ b/src/sql/optimizer/ob_route_policy.cpp @@ -209,7 +209,7 @@ int ObRoutePolicy::get_server_locality(const ObAddr &addr, const ObServerLocality &cur_locality = server_locality_array.at(i); if (addr == cur_locality.get_addr()) { if (OB_FAIL(svr_locality.assign(cur_locality))) { - LOG_WARN("fail to assgin locality", K(addr), K(cur_locality), K(server_locality_array), K(ret)); + LOG_WARN("fail to assign locality", K(addr), K(cur_locality), K(server_locality_array), K(ret)); } else { is_found = true; } diff --git a/src/sql/optimizer/ob_select_log_plan.cpp b/src/sql/optimizer/ob_select_log_plan.cpp index a7f4345ff2..19b10dcbc9 100644 --- a/src/sql/optimizer/ob_select_log_plan.cpp +++ b/src/sql/optimizer/ob_select_log_plan.cpp @@ -203,7 +203,7 @@ int ObSelectLogPlan::get_groupby_rollup_exprs(const ObLogicalOperator *top, top->get_output_equal_sets(), group_directions))) { } else if (OB_FAIL(rollup_exprs.assign(stmt->get_rollup_exprs()))) { - LOG_WARN("failed to assign to rollop exprs.", K(ret)); + LOG_WARN("failed to assign to rollup exprs.", K(ret)); } else if (rollup_exprs.count() > 0) { bool has_rollup_dir = stmt->has_rollup_dir(); if (OB_UNLIKELY(has_rollup_dir && (stmt->get_rollup_dir_size() != rollup_exprs.count()))) { @@ -1065,10 +1065,10 @@ int ObSelectLogPlan::inner_create_merge_group_plan(const ObIArray &r } else if (top->is_distributed() && OB_FAIL(top->check_sharding_compatible_with_reduce_expr(reduce_exprs, is_partition_wise))) { - LOG_WARN("failed to check if sharding compatiable with reduce expr", K(ret)); + LOG_WARN("failed to check if sharding compatible with reduce expr", K(ret)); } else if (!top->is_distributed() || is_partition_wise) { if (OB_FAIL(try_allocate_sort_as_top(top, sort_keys, need_sort, prefix_pos, part_cnt))) { - LOG_WARN("failed to allcoate sort as top", K(ret)); + LOG_WARN("failed to allocate sort as top", K(ret)); } else if (OB_FAIL(allocate_group_by_as_top(top, MERGE_AGGREGATE, adjusted_group_by_exprs, @@ -1116,7 +1116,7 @@ int ObSelectLogPlan::inner_create_merge_group_plan(const ObIArray &r nullptr, is_fetch_with_ties, use_part_sort ? &hash_sortkey : NULL))) { - LOG_WARN("failed to allcoate sort as top", K(ret)); + LOG_WARN("failed to allocate sort as top", K(ret)); } else if (OB_FAIL(allocate_group_by_as_top(top, MERGE_AGGREGATE, sort_exprs, @@ -1614,7 +1614,7 @@ int ObSelectLogPlan::create_merge_distinct_plan(ObLogicalOperator *&top, } else if (top->is_distributed() && OB_FAIL(top->check_sharding_compatible_with_reduce_expr(reduce_exprs, is_partition_wise))) { - LOG_WARN("failed to check sharding compatiable with reduce exprs", K(ret)); + LOG_WARN("failed to check sharding compatible with reduce exprs", K(ret)); } else if (!top->is_distributed() || is_partition_wise) { OPT_TRACE("is basic distinct:", !top->is_distributed()); OPT_TRACE("is partition wise distinct", is_partition_wise); @@ -2118,7 +2118,7 @@ int ObSelectLogPlan::create_union_all_plan(const ObIArray &c //for union all to keep child branches execute serially from left to right set_dist_methods &= (DistAlgo::DIST_PULL_TO_LOCAL | DistAlgo::DIST_BASIC_METHOD); } - OPT_TRACE("start create unoin all plan"); + OPT_TRACE("start create union all plan"); if (OB_SUCC(ret) && (set_dist_methods & DistAlgo::DIST_BASIC_METHOD)) { bool is_basic = false; OPT_TRACE("check match basic method"); @@ -4380,7 +4380,7 @@ int ObSelectLogPlan::allocate_plan_top() // step. allocate 'group-by' if needed if (OB_SUCC(ret) && (select_stmt->has_group_by() || select_stmt->has_rollup())) { // group-by or rollup both allocate group by logical operator. - // mysql mode for update need allocate before group by becauese group by isn't pk preserving. + // mysql mode for update need allocate before group by because group by isn't pk preserving. if (lib::is_mysql_mode() && select_stmt->has_for_update()) { if (OB_FAIL(candi_allocate_for_update())) { LOG_WARN("failed to allocate for update operator", K(ret)); @@ -4393,7 +4393,7 @@ int ObSelectLogPlan::allocate_plan_top() if (OB_FAIL(candi_allocate_group_by())) { LOG_WARN("failed to allocate group-by operator", K(ret)); } else { - LOG_TRACE("succeed to allocate group-by opeartor", + LOG_TRACE("succeed to allocate group-by operator", K(candidates_.candidate_plans_.count())); } } @@ -4411,7 +4411,7 @@ int ObSelectLogPlan::allocate_plan_top() // step. allocate 'distinct' if needed if (OB_SUCC(ret) && select_stmt->has_distinct()) { - // mysql mode for update need allocate before distinct becauese distinct isn't pk preserving. + // mysql mode for update need allocate before distinct because distinct isn't pk preserving. if (lib::is_mysql_mode() && select_stmt->has_for_update() && !for_update_is_allocated) { if (OB_FAIL(candi_allocate_for_update())) { LOG_WARN("failed to allocate for update operator", K(ret)); @@ -4507,7 +4507,7 @@ int ObSelectLogPlan::allocate_plan_top() if (OB_FAIL(candi_allocate_select_into())) { LOG_WARN("failed to allocate select into operator", K(ret)); } else { - LOG_TRACE("succeed to allocate select into cluase", + LOG_TRACE("succeed to allocate select into clause", K(candidates_.candidate_plans_.count())); } } @@ -4590,7 +4590,7 @@ int ObSelectLogPlan::generate_raw_plan_for_expr_values() if (OB_FAIL(allocate_plan_top())) { LOG_WARN("failed to allocate top operators for expr select", K(ret)); } else { - LOG_TRACE("succeed to allcoate top operators for expr select", K(ret)); + LOG_TRACE("succeed to allocate top operators for expr select", K(ret)); } } } @@ -4638,7 +4638,7 @@ int ObSelectLogPlan::generate_child_plan_for_set(const ObDMLStmt *sub_stmt, (optimizer_context_.get_log_plan_factory().create(optimizer_context_, *sub_stmt)))) { ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_ERROR("Failed to create logcial plan", K(sub_plan), K(ret)); + LOG_ERROR("Failed to create logical plan", K(sub_plan), K(ret)); } else if (FALSE_IT(sub_plan->set_is_parent_set_distinct(is_set_distinct))) { // do nothing } else if (OB_FAIL(sub_plan->add_pushdown_filters(pushdown_filters))) { @@ -5585,7 +5585,7 @@ int ObSelectLogPlan::get_sort_keys_for_window_function(const ObFdItemSet &fd_ite if (OB_FAIL(ret)) { } else if (OB_FAIL(append(output_sort_keys, win_expr->get_order_items()))) { - LOG_WARN("faield to append order items", K(ret)); + LOG_WARN("failed to append order items", K(ret)); } else if (OB_FAIL(ObOptimizerUtil::simplify_ordered_exprs(fd_item_set, equal_sets, const_exprs, @@ -6304,7 +6304,7 @@ int ObSelectLogPlan::create_pushdown_hash_dist_win_func(ObLogicalOperator *&top, NULL, false, // is_fetch_with_ties hash_sortkey))) { - LOG_WARN("failed to allcoate sort as top", K(ret)); + LOG_WARN("failed to allocate sort as top", K(ret)); } else if (OB_FAIL(allocate_window_function_as_top(WinDistAlgo::WIN_DIST_HASH, win_func_exprs, false, /* match_parallel */ @@ -6403,7 +6403,7 @@ int ObSelectLogPlan::sort_window_functions(const ObFdItemSet &fd_item_set, } } if (OB_SUCC(ret) && OB_FAIL(expr_entries.push_back(std::pair(-non_const_exprs, i)))) { - LOG_WARN("faield to push back expr entry", K(ret)); + LOG_WARN("failed to push back expr entry", K(ret)); } } if (OB_SUCC(ret)) { @@ -6736,7 +6736,7 @@ int ObSelectLogPlan::convert_project_columns(ObSelectStmt *stmt, expr->set_table_name(project_table_item->get_table_name()); if (expr->is_virtual_generated_column()) { if (item->is_geo_ == true && expr->get_srs_id() != SPATIAL_COLUMN_SRID_MASK) { - // spatial index generated column, cannot projet from main table + // spatial index generated column, cannot project from main table if (OB_FAIL(new_col_items.push_back(*item))) { LOG_WARN("failed to push back column item", K(ret)); } @@ -6910,7 +6910,7 @@ int ObSelectLogPlan::adjust_late_materialization_plan_structure(ObLogicalOperato right_expr, left_expr, equal_expr))) { - LOG_WARN("failed to crerate equal expr", K(ret)); + LOG_WARN("failed to create equal expr", K(ret)); } else if (OB_ISNULL(equal_expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); @@ -7033,7 +7033,7 @@ int ObSelectLogPlan::generate_late_materialization_table_get(ObLogTableScan *ind LOG_WARN("failed to allocate table partition info", K(ret)); } else if (FALSE_IT(table_scan_part_info = new (table_scan_part_info) ObTablePartitionInfo(allocator))) { } else if (OB_FAIL(table_scan_part_info->assign(*index_scan_part_info))) { - LOG_WARN("failed to assigin table partition info", K(ret)); + LOG_WARN("failed to assign table partition info", K(ret)); } else { table_scan->set_index_back(false); table_scan->set_table_id(table_id);