MySQL 8.3.0
Source Code Documentation
anonymous_namespace{join_optimizer.cc} Namespace Reference

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  PossibleIndexSkipScan
 Represents a candidate index skip scan, i.e. More...
 
struct  PossibleRangeScan
 
struct  PossibleRORScan
 

Functions

string PrintAccessPath (const AccessPath &path, const JoinHypergraph &graph, const char *description_for_trace)
 
void PrintJoinOrder (const AccessPath *path, string *join_order)
 Used by optimizer trace to print join order of join paths. More...
 
AccessPathCreateMaterializationPath (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...
 
AccessPathGetSafePathToSort (THD *thd, JOIN *join, AccessPath *path, bool need_rowid, bool force_materialization=false)
 
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...
 
secondary_engine_check_optimizer_request_t SecondaryEngineStateCheckHook (const THD *thd)
 Gets the secondary storage engine hypergraph state hook function, if any. More...
 
bool IsClusteredPrimaryKey (unsigned key_index, const TABLE &table)
 
Item_func_matchGetSargableFullTextPredicate (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 &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. More...
 
AccessPathFindCheapestIndexRangeScan (THD *thd, SEL_TREE *tree, RANGE_OPT_PARAM *param, bool prefer_clustered_primary_key_scan, bool *inexact, bool need_rowid_ordered_rows)
 From a collection of index scans, find the single cheapest one and generate an AccessPath for it. More...
 
void UpdateAppliedAndSubsumedPredicates (const uint idx, const Mem_root_array< PossibleRORScan > &possible_ror_scans, const RANGE_OPT_PARAM *param, OverflowBitset *applied_predicates, OverflowBitset *subsumed_predicates)
 
int GetRowIdOrdering (const TABLE *table, const LogicalOrderings *orderings, const Mem_root_array< ActiveIndexInfo > *active_indexes)
 
int WasPushedDownToRef (Item *condition, const KeypartForRef *keyparts, unsigned num_keyparts)
 
bool ContainsSubqueries (Item *item_arg)
 
bool HasConstantEqualityForField (const Mem_root_array< SargablePredicate > &sargable_predicates, const Field *field)
 Do we have a sargable predicate which checks if "field" is equal to a constant? More...
 
bool IsSubsumableFullTextPredicate (Item_func *condition)
 
bool IsLimitHintPushableToFullTextSearch (const Item_func_match *match, const JoinHypergraph &graph, uint64_t fulltext_predicates)
 
bool LateralDependenciesAreSatisfied (int node_idx, NodeMap tables, const JoinHypergraph &graph)
 Checks if the table given by "node_idx" has all its lateral dependencies satisfied by the set of tables given by "tables". More...
 
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...
 
bool IsEmptyJoin (const RelationalExpression::Type join_type, bool left_is_empty, bool right_is_empty)
 Checks if the result of a join is empty, given that it is known that one or both of the join legs always produces an empty result. More...
 
void MoveDegenerateJoinConditionToFilter (THD *thd, Query_block *query_block, const JoinPredicate **edge, AccessPath **right_path)
 If the ON clause of a left join only references tables on the right side of the join, pushing the condition into the right side is a valid thing to do. More...
 
AccessPathDeduplicateForSemijoin (THD *thd, AccessPath *path, Item **semijoin_group, int semijoin_group_size, std::string *trace)
 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)
 
AccessPathCreateMaterializationOrStreamingPath (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)
 
bool IsImmediateDeleteCandidate (const Table_ref *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_ref *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)
 
bool InjectCastNodes (JoinHypergraph *graph)
 
void EnableFullTextCoveringIndexes (const Query_block *query_block)
 
AccessPathCreateZeroRowsForEmptyJoin (JOIN *join, const char *cause)
 Creates a ZERO_ROWS access path for an always empty join result, or a ZERO_ROWS_AGGREGATED in case of an implicitly grouped query. More...
 
AccessPath CreateStreamingAggregationPath (THD *thd, AccessPath *path, JOIN *join, olap_type olap, double row_estimate, string *trace)
 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 ApplyHavingOrQualifyCondition (THD *thd, Item *having_cond, Query_block *query_block, const char *description_for_trace, string *trace, Prealloced_array< AccessPath *, 4 > *root_candidates, CostingReceiver *receiver)
 
AccessPath MakeSortPathForDistinct (THD *thd, AccessPath *root_path, int ordering_idx, bool aggregation_is_unordered, const LogicalOrderings &orderings, LogicalOrderings::StateIndex ordering_state)
 
JoinHypergraph::NodeFindNodeWithTable (JoinHypergraph *graph, TABLE *table)
 
Prealloced_array< AccessPath *, 4 > ApplyDistinctAndOrder (THD *thd, const CostingReceiver &receiver, const LogicalOrderings &orderings, bool aggregation_is_unordered, int order_by_ordering_idx, int distinct_ordering_idx, const Mem_root_array< SortAheadOrdering > &sort_ahead_orderings, FunctionalDependencySet fd_set, Query_block *query_block, bool need_rowid, bool force_sort_rowids, Prealloced_array< AccessPath *, 4 > root_candidates, string *trace)
 
static AccessPathApplyWindow (THD *thd, AccessPath *root_path, Window *window, JOIN *join, bool need_rowid_for_window)
 
static int FindBestOrderingForWindow (JOIN *join, const LogicalOrderings &orderings, FunctionalDependencySet fd_set, const Mem_root_array< SortAheadOrdering > &sort_ahead_orderings, Bounds_checked_array< bool > finished_windows, Bounds_checked_array< bool > tmp_buffer, int first_ordering_idx, int second_ordering_idx, Bounds_checked_array< bool > included_windows)
 Find the ordering that allows us to process the most unprocessed windows. More...
 
AccessPathMakeSortPathAndApplyWindows (THD *thd, JOIN *join, AccessPath *root_path, int ordering_idx, ORDER *order, const LogicalOrderings &orderings, Bounds_checked_array< bool > windows_this_iteration, FunctionalDependencySet fd_set, int num_where_predicates, bool need_rowid_for_window, int single_window_idx, Bounds_checked_array< bool > finished_windows, int *num_windows_left)
 

Function Documentation

◆ AddFieldsToTmpSet()

void anonymous_namespace{join_optimizer.cc}::AddFieldsToTmpSet ( Item item,
TABLE table 
)

Adds all fields of "table" that are referenced from "item" to table->tmp_set.

◆ AddFlag()

uint32_t anonymous_namespace{join_optimizer.cc}::AddFlag ( uint32_t  flags,
FuzzyComparisonResult  flag 
)

◆ ApplyDistinctAndOrder()

Prealloced_array< AccessPath *, 4 > anonymous_namespace{join_optimizer.cc}::ApplyDistinctAndOrder ( THD thd,
const CostingReceiver receiver,
const LogicalOrderings orderings,
bool  aggregation_is_unordered,
int  order_by_ordering_idx,
int  distinct_ordering_idx,
const Mem_root_array< SortAheadOrdering > &  sort_ahead_orderings,
FunctionalDependencySet  fd_set,
Query_block query_block,
bool  need_rowid,
bool  force_sort_rowids,
Prealloced_array< AccessPath *, 4 >  root_candidates,
string *  trace 
)

◆ ApplyHavingOrQualifyCondition()

void anonymous_namespace{join_optimizer.cc}::ApplyHavingOrQualifyCondition ( THD thd,
Item having_cond,
Query_block query_block,
const char *  description_for_trace,
string *  trace,
Prealloced_array< AccessPath *, 4 > *  root_candidates,
CostingReceiver receiver 
)

◆ ApplyWindow()

static AccessPath * anonymous_namespace{join_optimizer.cc}::ApplyWindow ( THD thd,
AccessPath root_path,
Window window,
JOIN join,
bool  need_rowid_for_window 
)
static

◆ CheckSupportedQuery()

bool anonymous_namespace{join_optimizer.cc}::CheckSupportedQuery ( THD thd)

◆ CollectPossibleRangeScans()

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 
)

◆ ContainsSubqueries()

bool anonymous_namespace{join_optimizer.cc}::ContainsSubqueries ( Item item_arg)

◆ CreateMaterializationOrStreamingPath()

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.

◆ CreateMaterializationPath()

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.

◆ CreateStreamingAggregationPath()

AccessPath anonymous_namespace{join_optimizer.cc}::CreateStreamingAggregationPath ( THD thd,
AccessPath path,
JOIN join,
olap_type  olap,
double  row_estimate,
string *  trace 
)

Creates an AGGREGATE AccessPath, possibly with an intermediary STREAM node if one is needed.

The creation of the temporary table does not happen here, but is left for FinalizePlanForQueryBlock().

Parameters
thdThe current thread.
joinThe join to which 'path' belongs.
olapcontains the GROUP BY modifier type
row_estimateestimated number of output rows, so that we do not need to recalculate it, or kUnknownRowCount if unknown.
traceOptimizer trace.
Returns
The AGGREGATE AccessPath.

◆ CreateZeroRowsForEmptyJoin()

AccessPath * anonymous_namespace{join_optimizer.cc}::CreateZeroRowsForEmptyJoin ( JOIN join,
const char *  cause 
)

Creates a ZERO_ROWS access path for an always empty join result, or a ZERO_ROWS_AGGREGATED in case of an implicitly grouped query.

The zero rows path is wrapped in FILTER (for HAVING) or LIMIT_OFFSET paths as needed, as well as UPDATE_ROWS/DELETE_ROWS paths for UPDATE/DELETE statements.

◆ DeduplicateForSemijoin()

AccessPath * anonymous_namespace{join_optimizer.cc}::DeduplicateForSemijoin ( THD thd,
AccessPath path,
Item **  semijoin_group,
int  semijoin_group_size,
std::string *  trace 
)

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.

◆ DisallowParameterizedJoinPath()

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.

◆ EnableFullTextCoveringIndexes()

void anonymous_namespace{join_optimizer.cc}::EnableFullTextCoveringIndexes ( const Query_block query_block)

◆ EngineFlags()

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.

◆ EstimateOutputRowsFromRangeTree()

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:

  • If the selectivity estimates are not consistent (e.g. S(a AND b) < S(a)S(b)), the algorithm will fail to converge. Extra steps are needed to correct for this.
  • The efficient algorithm (in [Hav20]) requires a linear algebra library (for performant matrix multiplication and Cholesky decomposition).
  • If we have a lot of estimates, even the efficient algorithm fails to converge in time (just the answers require 2^n space), and we would need additional logic to partition the problem.

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:

  1. Pick the most-covering index (ie., the range scan that applies the most number of predicates) that does not cover any predicates we've already accounted for. If there are multiple ones, choose the least selective.
  2. Multiply in its selectivity, and mark all the predicates it covers as accounted for. Repeat #1 and #2 for as long as possible.
  3. For any remaining predicates, multiply by their existing estimate (ie., the one not coming from the range optimizer).

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”

◆ FindAppliedAndSubsumedPredicatesForRangeScan()

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 
)

◆ FindBestOrderingForWindow()

static int anonymous_namespace{join_optimizer.cc}::FindBestOrderingForWindow ( JOIN join,
const LogicalOrderings orderings,
FunctionalDependencySet  fd_set,
const Mem_root_array< SortAheadOrdering > &  sort_ahead_orderings,
Bounds_checked_array< bool >  finished_windows,
Bounds_checked_array< bool >  tmp_buffer,
int  first_ordering_idx,
int  second_ordering_idx,
Bounds_checked_array< bool >  included_windows 
)
static

Find the ordering that allows us to process the most unprocessed windows.

If specified, we can also demand that the ordering satisfies one or two later orderings (for DISTINCT and/or ORDER BY).

Our priorities are, in strict order:

  1. Satisfying both DISTINCT and ORDER BY (if both are given; but see below).
  2. Satisfying the first operation after windowing (which is either DISTINCT or ORDER BY).
  3. Satisfying as many windows as possible.
  4. The shortest possible ordering (as a tie-breaker).

If first_ordering_idx is given, #2 is mandatory. #4 is so that we don't get strange situations where the user specifies e.g. OVER (ORDER BY i) and we choose an ordering i,j,k,l,... because it happened to be given somewhere else.

Note that normally, it is very hard to satisfy DISTINCT for a window function, because generally, it isn't constant for a given input (by nature, it also depends on other rows). But it can happen if the window frame is static; see main.window_functions_interesting_orders.

Parameters
joinContains the list of windows.
orderingsLogical orderings in the query block.
sort_ahead_orderingsCandidate orderings to consider.
fd_setActive functional dependencies.
finished_windowsWindows to ignore.
tmp_bufferTemporary space for keeping the best list of windows so far; must be as large as the number of values.
first_ordering_idxThe first ordering after the query block that we need to satisfy (-1 if none).
second_ordering_idxThe second ordering after the query block that we would like to satisfy (-1 if none).
[out]included_windowsWhich windows can be sorted using the given ordering.
Returns
An index into sort_ahead_orderings, or -1 if no ordering could be found that sorts at least one window (plus, if first_ordering_idx is set, follows that ordering).

◆ FindCheapestIndexRangeScan()

AccessPath * anonymous_namespace{join_optimizer.cc}::FindCheapestIndexRangeScan ( THD thd,
SEL_TREE tree,
RANGE_OPT_PARAM param,
bool  prefer_clustered_primary_key_scan,
bool *  inexact,
bool  need_rowid_ordered_rows 
)

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.

◆ FindFullTextSearchedTables()

NodeMap anonymous_namespace{join_optimizer.cc}::FindFullTextSearchedTables ( const JoinHypergraph graph)

◆ FindImmediateUpdateDeleteCandidates()

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.

◆ FindNodeWithTable()

JoinHypergraph::Node * anonymous_namespace{join_optimizer.cc}::FindNodeWithTable ( JoinHypergraph graph,
TABLE table 
)

◆ FindReachableTablesFrom()

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.

◆ FindSargableFullTextPredicates()

uint64_t anonymous_namespace{join_optimizer.cc}::FindSargableFullTextPredicates ( const JoinHypergraph graph)

◆ FindUpdateDeleteTargetTables()

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.

◆ GetRowIdOrdering()

int anonymous_namespace{join_optimizer.cc}::GetRowIdOrdering ( const TABLE table,
const LogicalOrderings orderings,
const Mem_root_array< ActiveIndexInfo > *  active_indexes 
)

◆ GetSafePathToSort()

AccessPath * anonymous_namespace{join_optimizer.cc}::GetSafePathToSort ( THD thd,
JOIN join,
AccessPath path,
bool  need_rowid,
bool  force_materialization = false 
)

◆ GetSargableFullTextPredicate()

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.

◆ HasConstantEqualityForField()

bool anonymous_namespace{join_optimizer.cc}::HasConstantEqualityForField ( const Mem_root_array< SargablePredicate > &  sargable_predicates,
const Field field 
)

Do we have a sargable predicate which checks if "field" is equal to a constant?

◆ HasFlag()

bool anonymous_namespace{join_optimizer.cc}::HasFlag ( uint32_t  flags,
FuzzyComparisonResult  flag 
)

◆ InjectCastNodes()

bool anonymous_namespace{join_optimizer.cc}::InjectCastNodes ( JoinHypergraph graph)

◆ IsClusteredPrimaryKey()

bool anonymous_namespace{join_optimizer.cc}::IsClusteredPrimaryKey ( unsigned  key_index,
const TABLE table 
)

◆ IsDeleteStatement()

bool anonymous_namespace{join_optimizer.cc}::IsDeleteStatement ( const THD thd)

Is the current statement a DELETE statement?

◆ IsEmptyJoin()

bool anonymous_namespace{join_optimizer.cc}::IsEmptyJoin ( const RelationalExpression::Type  join_type,
bool  left_is_empty,
bool  right_is_empty 
)

Checks if the result of a join is empty, given that it is known that one or both of the join legs always produces an empty result.

◆ IsImmediateDeleteCandidate()

bool anonymous_namespace{join_optimizer.cc}::IsImmediateDeleteCandidate ( const Table_ref 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.

◆ IsImmediateUpdateCandidate()

bool anonymous_namespace{join_optimizer.cc}::IsImmediateUpdateCandidate ( const Table_ref 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.

◆ IsLimitHintPushableToFullTextSearch()

bool anonymous_namespace{join_optimizer.cc}::IsLimitHintPushableToFullTextSearch ( const Item_func_match match,
const JoinHypergraph graph,
uint64_t  fulltext_predicates 
)

◆ IsMaterializationPath()

bool anonymous_namespace{join_optimizer.cc}::IsMaterializationPath ( const AccessPath path)

◆ IsSargableFullTextIndexPredicate()

bool anonymous_namespace{join_optimizer.cc}::IsSargableFullTextIndexPredicate ( Item condition)

◆ IsSubsumableFullTextPredicate()

bool anonymous_namespace{join_optimizer.cc}::IsSubsumableFullTextPredicate ( Item_func condition)

◆ IsUpdateStatement()

bool anonymous_namespace{join_optimizer.cc}::IsUpdateStatement ( const THD thd)

Is the current statement a DELETE statement?

◆ LateralDependenciesAreSatisfied()

bool anonymous_namespace{join_optimizer.cc}::LateralDependenciesAreSatisfied ( int  node_idx,
NodeMap  tables,
const JoinHypergraph graph 
)

Checks if the table given by "node_idx" has all its lateral dependencies satisfied by the set of tables given by "tables".

◆ MakeSortPathAndApplyWindows()

AccessPath * anonymous_namespace{join_optimizer.cc}::MakeSortPathAndApplyWindows ( THD thd,
JOIN join,
AccessPath root_path,
int  ordering_idx,
ORDER order,
const LogicalOrderings orderings,
Bounds_checked_array< bool >  windows_this_iteration,
FunctionalDependencySet  fd_set,
int  num_where_predicates,
bool  need_rowid_for_window,
int  single_window_idx,
Bounds_checked_array< bool >  finished_windows,
int *  num_windows_left 
)

◆ MakeSortPathForDistinct()

AccessPath anonymous_namespace{join_optimizer.cc}::MakeSortPathForDistinct ( THD thd,
AccessPath root_path,
int  ordering_idx,
bool  aggregation_is_unordered,
const LogicalOrderings orderings,
LogicalOrderings::StateIndex  ordering_state 
)

◆ MakeSortPathWithoutFilesort()

AccessPath anonymous_namespace{join_optimizer.cc}::MakeSortPathWithoutFilesort ( THD thd,
AccessPath child,
ORDER order,
int  ordering_state,
int  num_where_predicates 
)

◆ MoveDegenerateJoinConditionToFilter()

void anonymous_namespace{join_optimizer.cc}::MoveDegenerateJoinConditionToFilter ( THD thd,
Query_block query_block,
const JoinPredicate **  edge,
AccessPath **  right_path 
)

If the ON clause of a left join only references tables on the right side of the join, pushing the condition into the right side is a valid thing to do.

If such conditions are not pushed down for some reason, and are left in the ON clause, HeatWave might reject the query. This happens if the entire join condition is degenerate and only references the right side. Such conditions are most commonly seen in queries that have gone through subquery_to_derived transformation.

This limitation is worked around here by moving the degenerate join condition from the join predicate to a filter path on top of the right path. This is only done for secondary storage engines.

TODO(khatlen): If HeatWave gets capable of processing queries with such conditions, this workaround should be removed.

◆ PartiallyResolvedParameterization()

bool anonymous_namespace{join_optimizer.cc}::PartiallyResolvedParameterization ( NodeMap  parameter_tables,
NodeMap  other_side 
)

◆ PrintAccessPath()

string anonymous_namespace{join_optimizer.cc}::PrintAccessPath ( const AccessPath path,
const JoinHypergraph graph,
const char *  description_for_trace 
)

◆ PrintJoinOrder()

void anonymous_namespace join_optimizer anonymous_namespace{join_optimizer.cc}::cc::PrintJoinOrder ( const AccessPath path,
string *  join_order 
)

Used by optimizer trace to print join order of join paths.

Appends into 'join_order' a string that looks something like '(t1,(t2,t3))' where t1 is an alias of any kind of table including materialized table, and t1 is joined with (t2,t3) where (t2,t3) is another join.

◆ SecondaryEngineCostHook()

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.

◆ SecondaryEngineStateCheckHook()

secondary_engine_check_optimizer_request_t anonymous_namespace{join_optimizer.cc}::SecondaryEngineStateCheckHook ( const THD thd)

Gets the secondary storage engine hypergraph state hook function, if any.

◆ SplitHavingCondition()

void anonymous_namespace{join_optimizer.cc}::SplitHavingCondition ( THD thd,
Item cond,
Item **  having_cond,
Item **  having_cond_wf 
)

◆ UpdateAppliedAndSubsumedPredicates()

void anonymous_namespace{join_optimizer.cc}::UpdateAppliedAndSubsumedPredicates ( const uint  idx,
const Mem_root_array< PossibleRORScan > &  possible_ror_scans,
const RANGE_OPT_PARAM param,
OverflowBitset applied_predicates,
OverflowBitset subsumed_predicates 
)

◆ WasPushedDownToRef()

int anonymous_namespace{join_optimizer.cc}::WasPushedDownToRef ( Item condition,
const KeypartForRef keyparts,
unsigned  num_keyparts 
)