![]() |
MySQL 8.0.30
Source Code Documentation
|
Classes | |
class | CostingReceiver |
CostingReceiver contains the main join planning logic, selecting access paths based on cost. More... | |
struct | KeypartForRef |
struct | PossibleIndexMerge |
Represents a candidate index merge, ie. More... | |
struct | PossibleRangeScan |
Functions | |
string | PrintAccessPath (const AccessPath &path, const JoinHypergraph &graph, const char *description_for_trace) |
AccessPath * | CreateMaterializationPath (THD *thd, JOIN *join, AccessPath *path, TABLE *temp_table, Temp_table_param *temp_table_param, bool copy_items) |
Sets up an access path for materializing the results returned from a path in a temporary table. More... | |
AccessPath * | GetSafePathToSort (THD *thd, JOIN *join, AccessPath *path, bool need_rowid) |
SecondaryEngineFlags | EngineFlags (const THD *thd) |
Lists the current secondary engine flags in use. More... | |
secondary_engine_modify_access_path_cost_t | SecondaryEngineCostHook (const THD *thd) |
Gets the secondary storage engine cost modification function, if any. More... | |
Item_func_match * | GetSargableFullTextPredicate (const Predicate &predicate) |
Returns the MATCH function of a predicate that can be pushed down to a full-text index. More... | |
bool | IsDeleteStatement (const THD *thd) |
Is the current statement a DELETE statement? More... | |
bool | IsUpdateStatement (const THD *thd) |
Is the current statement a DELETE statement? More... | |
void | FindAppliedAndSubsumedPredicatesForRangeScan (THD *thd, KEY *key, unsigned used_key_parts, unsigned num_exact_key_parts, TABLE *table, OverflowBitset tree_applied_predicates, OverflowBitset tree_subsumed_predicates, const JoinHypergraph &graph, OverflowBitset *applied_predicates_out, OverflowBitset *subsumed_predicates_out) |
bool | CollectPossibleRangeScans (THD *thd, SEL_TREE *tree, RANGE_OPT_PARAM *param, OverflowBitset tree_applied_predicates, OverflowBitset tree_subsumed_predicates, const JoinHypergraph &graph, Mem_root_array< PossibleRangeScan > *possible_scans) |
double | EstimateOutputRowsFromRangeTree (THD *thd, const RANGE_OPT_PARAM ¶m, ha_rows total_rows, const Mem_root_array< PossibleRangeScan > &possible_scans, const JoinHypergraph &graph, OverflowBitset predicates, string *trace) |
Based on estimates for all the different range scans (which cover different but potentially overlapping combinations of predicates), try to find an estimate for the number of rows scanning the given table, with all predicates applied. More... | |
AccessPath * | FindCheapestIndexRangeScan (THD *thd, SEL_TREE *tree, RANGE_OPT_PARAM *param, bool prefer_clustered_primary_key_scan, bool *inexact) |
From a collection of index scans, find the single cheapest one and generate an AccessPath for it. More... | |
int | WasPushedDownToRef (Item *condition, const KeypartForRef *keyparts, unsigned num_keyparts) |
bool | ContainsSubqueries (Item *item_arg) |
bool | IsSubsumableFullTextPredicate (Item_func *condition) |
bool | IsLimitHintPushableToFullTextSearch (const Item_func_match *match, const JoinHypergraph &graph, uint64_t fulltext_predicates) |
NodeMap | FindReachableTablesFrom (NodeMap tables, const JoinHypergraph &graph) |
Find the set of tables we can join directly against, given that we have the given set of tables on one of the sides (effectively the same concept as DPhyp's “neighborhood”). More... | |
bool | PartiallyResolvedParameterization (NodeMap parameter_tables, NodeMap other_side) |
bool | DisallowParameterizedJoinPath (AccessPath *left_path, AccessPath *right_path, NodeMap left, NodeMap right, NodeMap left_reachable, NodeMap right_reachable) |
Decide whether joining the two given paths would create a disallowed parameterized path. More... | |
AccessPath * | DeduplicateForSemijoin (THD *thd, AccessPath *path, Item **semijoin_group, int semijoin_group_size) |
Build an access path that deduplicates its input on a certain grouping. More... | |
uint32_t | AddFlag (uint32_t flags, FuzzyComparisonResult flag) |
bool | HasFlag (uint32_t flags, FuzzyComparisonResult flag) |
AccessPath | MakeSortPathWithoutFilesort (THD *thd, AccessPath *child, ORDER *order, int ordering_state, int num_where_predicates) |
bool | CheckSupportedQuery (THD *thd) |
AccessPath * | CreateMaterializationOrStreamingPath (THD *thd, JOIN *join, AccessPath *path, bool need_rowid, bool copy_items) |
Set up an access path for streaming or materializing through a temporary table. More... | |
bool | IsMaterializationPath (const AccessPath *path) |
void | CollectItemsToMaterializeForFullTextAggregation (Item *root, mem_root_deque< Item * > *items) |
Goes through an item tree and collects all the sub-items that should be materialized if full-text search is used in combination with sort-based aggregation. More... | |
bool | CreateTemporaryTableForFullTextFunctions (THD *thd, Query_block *query_block, TABLE **temp_table, Temp_table_param **temp_table_param) |
Creates a temporary table which materializes the results of all full-text functions that need to be accessible after aggregation. More... | |
bool | IsImmediateDeleteCandidate (const TABLE_LIST *table_ref, const Query_block *query_block) |
Is this DELETE target table a candidate for being deleted from immediately, while scanning the result of the join? It only checks if it is a candidate for immediate delete. More... | |
void | AddFieldsToTmpSet (Item *item, TABLE *table) |
Adds all fields of "table" that are referenced from "item" to table->tmp_set. More... | |
bool | IsImmediateUpdateCandidate (const TABLE_LIST *table_ref, int node_idx, const JoinHypergraph &graph, table_map target_tables) |
Is this UPDATE target table a candidate for being updated immediately, while scanning the result of the join? It only checks if it is a candidate for immediate update. More... | |
table_map | FindUpdateDeleteTargetTables (const Query_block *query_block) |
Finds all the target tables of an UPDATE or DELETE statement. More... | |
table_map | FindImmediateUpdateDeleteCandidates (const JoinHypergraph &graph, table_map target_tables, bool is_delete) |
Finds all of the target tables of an UPDATE or DELETE statement that are candidates from being updated or deleted from immediately while scanning the results of the join, without need to buffer the row IDs in a temporary table for delayed update/delete after the join has completed. More... | |
NodeMap | FindFullTextSearchedTables (const JoinHypergraph &graph) |
bool | IsSargableFullTextIndexPredicate (Item *condition) |
uint64_t | FindSargableFullTextPredicates (const JoinHypergraph &graph) |
void | EnableFullTextCoveringIndexes (const Query_block *query_block) |
bool | HasEqRefWithCache (AccessPath *path) |
Does this path contain an EQ_REF path which has caching enabled? More... | |
AccessPath | CreateStreamingAggregationPath (THD *thd, AccessPath *path, const Query_block *query_block, bool rollup, TABLE *table, Temp_table_param *param) |
Creates an AGGREGATE AccessPath, possibly with an intermediary STREAM node if one is needed. More... | |
void | SplitHavingCondition (THD *thd, Item *cond, Item **having_cond, Item **having_cond_wf) |
void | ApplyHavingCondition (THD *thd, Item *having_cond, Query_block *query_block, const char *description_for_trace, string *trace, Prealloced_array< AccessPath *, 4 > *root_candidates, CostingReceiver *receiver) |
Adds all fields of "table" that are referenced from "item" to table->tmp_set.
uint32_t anonymous_namespace{join_optimizer.cc}::AddFlag | ( | uint32_t | flags, |
FuzzyComparisonResult | flag | ||
) |
void anonymous_namespace{join_optimizer.cc}::ApplyHavingCondition | ( | THD * | thd, |
Item * | having_cond, | ||
Query_block * | query_block, | ||
const char * | description_for_trace, | ||
string * | trace, | ||
Prealloced_array< AccessPath *, 4 > * | root_candidates, | ||
CostingReceiver * | receiver | ||
) |
bool anonymous_namespace{join_optimizer.cc}::CheckSupportedQuery | ( | THD * | thd | ) |
void anonymous_namespace{join_optimizer.cc}::CollectItemsToMaterializeForFullTextAggregation | ( | Item * | root, |
mem_root_deque< Item * > * | items | ||
) |
Goes through an item tree and collects all the sub-items that should be materialized if full-text search is used in combination with sort-based aggregation.
This means all expressions in the SELECT list, GROUP BY clause, ORDER BY clause and HAVING clause that are possible to evaluate before aggregation.
The reason why this materialization is needed, is that Item_func_match::val_real() can only be evaluated if the underlying scan is positioned on the row for which the full-text search score is to be retrieved. In the sort-based aggregation performed by AggregateIterator, the rows are returned with the underlying scan positioned on some other row (typically one in the next group). Even though AggregateIterator restores the contents of the record buffers to what they had when the scan was positioned on that row, it is not enough; the handler needs to be repositioned to make Item_func_match give the correct result. To avoid this, we materialize the results of the MATCH functions before they are seen by AggregateIterator.
The most important thing to materialize is the MATCH function, but AggregateIterator is not currently prepared for reading some data from a materialized source and other data directly from the base tables, so we have to materialize all expressions that are to be used as input to AggregateIterator.
The old optimizer does not have special code for materializing MATCH functions. In most cases it does not need it, because it usually performs aggregation by materializing all expressions (not only MATCH) in the SELECT list, the GROUP BY clause and the ORDER BY clause anyway. It does not, however, materialize the non-aggregated expressions in the HAVING clause, so calls to the MATCH function in the HAVING clause may give wrong results with the old optimizer.
The old optimizer uses the same sort-based aggregation as the hypergraph optimizer for ROLLUP. The lack of materialization of MATCH expressions leads to wrong results also when MATCH is used in the SELECT list or the ORDER BY clause of a ROLLUP query.
The materialization performed by this function makes the hypergraph optimizer produce correct results for the above mentioned cases where the old optimizer produces wrong results.
bool anonymous_namespace{join_optimizer.cc}::CollectPossibleRangeScans | ( | THD * | thd, |
SEL_TREE * | tree, | ||
RANGE_OPT_PARAM * | param, | ||
OverflowBitset | tree_applied_predicates, | ||
OverflowBitset | tree_subsumed_predicates, | ||
const JoinHypergraph & | graph, | ||
Mem_root_array< PossibleRangeScan > * | possible_scans | ||
) |
bool anonymous_namespace{join_optimizer.cc}::ContainsSubqueries | ( | Item * | item_arg | ) |
AccessPath * anonymous_namespace{join_optimizer.cc}::CreateMaterializationOrStreamingPath | ( | THD * | thd, |
JOIN * | join, | ||
AccessPath * | path, | ||
bool | need_rowid, | ||
bool | copy_items | ||
) |
Set up an access path for streaming or materializing through a temporary table.
If none is needed (because earlier iterators already materialize what needs to be done), returns the path itself.
The actual temporary table will be created and filled out during finalization.
AccessPath *anonymous_namespace join_optimizer anonymous_namespace{join_optimizer.cc}::cc::CreateMaterializationPath | ( | THD * | thd, |
JOIN * | join, | ||
AccessPath * | path, | ||
TABLE * | temp_table, | ||
Temp_table_param * | temp_table_param, | ||
bool | copy_items | ||
) |
Sets up an access path for materializing the results returned from a path in a temporary table.
AccessPath anonymous_namespace{join_optimizer.cc}::CreateStreamingAggregationPath | ( | THD * | thd, |
AccessPath * | path, | ||
const Query_block * | query_block, | ||
bool | rollup, | ||
TABLE * | table, | ||
Temp_table_param * | param | ||
) |
Creates an AGGREGATE AccessPath, possibly with an intermediary STREAM node if one is needed.
If the caller has already determined that an intermediary STREAM node is needed, it can pass a TABLE and Temp_table_param describing what to materialize. (This is only used by full-text search, which needs a temporary table of a different shape than what we get from FinalizePlanForQueryBlock()/DelayedCreateTemporaryTable(). See CreateTemporaryTableForFullTextFunctions().)
Otherwise, we check if "path" has any other property that makes streaming necessary, and add a STREAM node if needed. The creation of the temporary table does not happen here, but is left for FinalizePlanForQueryBlock().
bool anonymous_namespace{join_optimizer.cc}::CreateTemporaryTableForFullTextFunctions | ( | THD * | thd, |
Query_block * | query_block, | ||
TABLE ** | temp_table, | ||
Temp_table_param ** | temp_table_param | ||
) |
Creates a temporary table which materializes the results of all full-text functions that need to be accessible after aggregation.
This is needed for sort-based aggregation on full-text searched tables if the full-text search score is accessed in the SELECT list, GROUP BY clause, ORDER BY clause or HAVING clause. See CollectItemsToMaterializeForFullTextAggregation() for more details.
thd | the session object | |
query_block | the query block | |
[out] | temp_table | the created temporary table, or nullptr if there are no MATCH functions that need materialization |
[out] | temp_table_param | the parameters of the created temporary table |
AccessPath * anonymous_namespace{join_optimizer.cc}::DeduplicateForSemijoin | ( | THD * | thd, |
AccessPath * | path, | ||
Item ** | semijoin_group, | ||
int | semijoin_group_size | ||
) |
Build an access path that deduplicates its input on a certain grouping.
This is used for converting semijoins to inner joins. If the grouping is empty, all rows are the same, and we make a simple LIMIT 1 instead.
bool anonymous_namespace{join_optimizer.cc}::DisallowParameterizedJoinPath | ( | AccessPath * | left_path, |
AccessPath * | right_path, | ||
NodeMap | left, | ||
NodeMap | right, | ||
NodeMap | left_reachable, | ||
NodeMap | right_reachable | ||
) |
Decide whether joining the two given paths would create a disallowed parameterized path.
Parameterized paths are disallowed if they delay joining in their parameterizations without reason (ie., they could join in a parameterization right away, but don't). This is a trick borrowed from Postgres, which essentially forces inner-join ref-lookup plans to be left-deep (since such plans never gain anything from being bushy), reducing the search space significantly without compromising plan quality.
void anonymous_namespace{join_optimizer.cc}::EnableFullTextCoveringIndexes | ( | const Query_block * | query_block | ) |
SecondaryEngineFlags anonymous_namespace{join_optimizer.cc}::EngineFlags | ( | const THD * | thd | ) |
Lists the current secondary engine flags in use.
If there is no secondary engine, will use a default set of permissive flags suitable for non-secondary engine use.
double anonymous_namespace{join_optimizer.cc}::EstimateOutputRowsFromRangeTree | ( | THD * | thd, |
const RANGE_OPT_PARAM & | param, | ||
ha_rows | total_rows, | ||
const Mem_root_array< PossibleRangeScan > & | possible_scans, | ||
const JoinHypergraph & | graph, | ||
OverflowBitset | predicates, | ||
string * | trace | ||
) |
Based on estimates for all the different range scans (which cover different but potentially overlapping combinations of predicates), try to find an estimate for the number of rows scanning the given table, with all predicates applied.
The #1 priority here is to get a single estimate for all (non-parameterized) scans over this table (including non-range scans), that we can reuse for all access paths. This makes sure they are fairly compared on cost (and ordering) alone; different estimates would be nonsensical, and cause those where we happen to have lower estimates to get preferred as they are joined higher up in the tree. Obviously, however, it is also attractive to get an estimate that is as good as possible. We only really care about the total selectivity of all predicates; we don't care to adjust each individual selectivity.
[Mar07] describes an unbiased estimator that is exactly what we want, and [Hav20] demonstrates an efficient calculation method (up to about 20–25 possible predicates) of this estimator. Long-term, implementing this would be our best choice. However, the implementation is not entirely trivial:
Thus, for the time being, we use an ad-hoc algorithm instead. The estimate will not be as good, but it will hopefully be on the pessimistic side (overestimating the number of rows). It goes as follows:
The hope is that in #1, we will usually prefer using selectivity information from indexes with more keyparts; e.g., it's better to use an index on (a,b) than on (a) alone, since it will take into account the correlation between predicates on a and predicates on b.
[Mar07]: Markl et al: “Consistent Selectivity Estimation Via Maximum Entropy” [Hav20]: Havenstein et al: “Fast Entropy Maximization for Selectivity Estimation of Conjunctive Predicates on CPUs and GPUs”
void anonymous_namespace{join_optimizer.cc}::FindAppliedAndSubsumedPredicatesForRangeScan | ( | THD * | thd, |
KEY * | key, | ||
unsigned | used_key_parts, | ||
unsigned | num_exact_key_parts, | ||
TABLE * | table, | ||
OverflowBitset | tree_applied_predicates, | ||
OverflowBitset | tree_subsumed_predicates, | ||
const JoinHypergraph & | graph, | ||
OverflowBitset * | applied_predicates_out, | ||
OverflowBitset * | subsumed_predicates_out | ||
) |
AccessPath * anonymous_namespace{join_optimizer.cc}::FindCheapestIndexRangeScan | ( | THD * | thd, |
SEL_TREE * | tree, | ||
RANGE_OPT_PARAM * | param, | ||
bool | prefer_clustered_primary_key_scan, | ||
bool * | inexact | ||
) |
From a collection of index scans, find the single cheapest one and generate an AccessPath for it.
This is similar to CollectPossibleRangeScans(), except that this is for index merge, where we don't want to enumerate all possibilities; since we don't care about the ordering of the index (we're going to sort all of the rows to deduplicate them anyway), cost is the only interesting metric, so we only need to pick out and collect ranges for one of them. (This isn't strictly true; sometimes, it can be attractive to choose a clustered primary key, so we prefer one if we allow them. See the code about is_preferred_cpk below, and the comment on the caller. Also, see about exactness below.)
This function can probably be extended to find ROR-capable scans later (just check is_ror_scan instead of is_imerge_scan).
Note that all such scans are index-only (covering), which is reflected in the cost parameters we use.
inexact is set to true if and only if the chosen path does not reflect its predicate faithfully, and needs to be rechecked. We do not currently take into account that this may affect the cost higher up, as the difference should be small enough that we don't want the combinatorial explosion.
NodeMap anonymous_namespace{join_optimizer.cc}::FindFullTextSearchedTables | ( | const JoinHypergraph & | graph | ) |
table_map anonymous_namespace{join_optimizer.cc}::FindImmediateUpdateDeleteCandidates | ( | const JoinHypergraph & | graph, |
table_map | target_tables, | ||
bool | is_delete | ||
) |
Finds all of the target tables of an UPDATE or DELETE statement that are candidates from being updated or deleted from immediately while scanning the results of the join, without need to buffer the row IDs in a temporary table for delayed update/delete after the join has completed.
These are candidates only; the actual tables to update while scanning, if any, will be chosen based on cost during planning.
NodeMap anonymous_namespace{join_optimizer.cc}::FindReachableTablesFrom | ( | NodeMap | tables, |
const JoinHypergraph & | graph | ||
) |
Find the set of tables we can join directly against, given that we have the given set of tables on one of the sides (effectively the same concept as DPhyp's “neighborhood”).
Note that having false negatives here is fine (it will only make DisallowParameterizedJoinPath() slightly less effective), but false positives is not (it may disallow valid parameterized paths, ultimately even making LATERAL queries impossible to plan). Thus, we need to check conflict rules, and our handling of hyperedges with more than one table on the other side may also be a bit too strict (this may need adjustments when we get FULL OUTER JOIN).
If this calculation turns out to be slow, we could probably cache it in AccessPathSet, or even try to build it incrementally.
uint64_t anonymous_namespace{join_optimizer.cc}::FindSargableFullTextPredicates | ( | const JoinHypergraph & | graph | ) |
table_map anonymous_namespace{join_optimizer.cc}::FindUpdateDeleteTargetTables | ( | const Query_block * | query_block | ) |
Finds all the target tables of an UPDATE or DELETE statement.
It additionally disables covering index scans on the target tables, since ha_update_row() and ha_delete_row() can only be called on scans reading the full row.
AccessPath * anonymous_namespace{join_optimizer.cc}::GetSafePathToSort | ( | THD * | thd, |
JOIN * | join, | ||
AccessPath * | path, | ||
bool | need_rowid | ||
) |
Item_func_match * anonymous_namespace{join_optimizer.cc}::GetSargableFullTextPredicate | ( | const Predicate & | predicate | ) |
Returns the MATCH function of a predicate that can be pushed down to a full-text index.
This can be done if the predicate is a MATCH function, or in some cases (see IsSargableFullTextIndexPredicate() for details) where the predicate is a comparison function which compares the result of MATCH with a constant. For example, predicates on this form could be pushed down to a full-text index:
WHERE MATCH (x) AGAINST ('search string') AND <more predicates>
WHERE MATCH (x) AGAINST ('search string') > 0.5 AND <more predicates>
Since full-text index scans return documents with positive scores only, an index scan can only be used if the predicate excludes negative or zero scores.
bool anonymous_namespace{join_optimizer.cc}::HasEqRefWithCache | ( | AccessPath * | path | ) |
Does this path contain an EQ_REF path which has caching enabled?
bool anonymous_namespace{join_optimizer.cc}::HasFlag | ( | uint32_t | flags, |
FuzzyComparisonResult | flag | ||
) |
bool anonymous_namespace{join_optimizer.cc}::IsDeleteStatement | ( | const THD * | thd | ) |
Is the current statement a DELETE statement?
bool anonymous_namespace{join_optimizer.cc}::IsImmediateDeleteCandidate | ( | const TABLE_LIST * | table_ref, |
const Query_block * | query_block | ||
) |
Is this DELETE target table a candidate for being deleted from immediately, while scanning the result of the join? It only checks if it is a candidate for immediate delete.
Whether it actually ends up being deleted from immediately, depends on the plan that is chosen.
bool anonymous_namespace{join_optimizer.cc}::IsImmediateUpdateCandidate | ( | const TABLE_LIST * | table_ref, |
int | node_idx, | ||
const JoinHypergraph & | graph, | ||
table_map | target_tables | ||
) |
Is this UPDATE target table a candidate for being updated immediately, while scanning the result of the join? It only checks if it is a candidate for immediate update.
Whether it actually ends up being updated immediately, depends on the plan that is chosen.
bool anonymous_namespace{join_optimizer.cc}::IsLimitHintPushableToFullTextSearch | ( | const Item_func_match * | match, |
const JoinHypergraph & | graph, | ||
uint64_t | fulltext_predicates | ||
) |
bool anonymous_namespace{join_optimizer.cc}::IsMaterializationPath | ( | const AccessPath * | path | ) |
bool anonymous_namespace{join_optimizer.cc}::IsSargableFullTextIndexPredicate | ( | Item * | condition | ) |
bool anonymous_namespace{join_optimizer.cc}::IsSubsumableFullTextPredicate | ( | Item_func * | condition | ) |
bool anonymous_namespace{join_optimizer.cc}::IsUpdateStatement | ( | const THD * | thd | ) |
Is the current statement a DELETE statement?
AccessPath anonymous_namespace{join_optimizer.cc}::MakeSortPathWithoutFilesort | ( | THD * | thd, |
AccessPath * | child, | ||
ORDER * | order, | ||
int | ordering_state, | ||
int | num_where_predicates | ||
) |
bool anonymous_namespace{join_optimizer.cc}::PartiallyResolvedParameterization | ( | NodeMap | parameter_tables, |
NodeMap | other_side | ||
) |
string anonymous_namespace{join_optimizer.cc}::PrintAccessPath | ( | const AccessPath & | path, |
const JoinHypergraph & | graph, | ||
const char * | description_for_trace | ||
) |
secondary_engine_modify_access_path_cost_t anonymous_namespace{join_optimizer.cc}::SecondaryEngineCostHook | ( | const THD * | thd | ) |
Gets the secondary storage engine cost modification function, if any.
void anonymous_namespace{join_optimizer.cc}::SplitHavingCondition | ( | THD * | thd, |
Item * | cond, | ||
Item ** | having_cond, | ||
Item ** | having_cond_wf | ||
) |
int anonymous_namespace{join_optimizer.cc}::WasPushedDownToRef | ( | Item * | condition, |
const KeypartForRef * | keyparts, | ||
unsigned | num_keyparts | ||
) |