You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by Apache Wiki <wi...@apache.org> on 2006/10/27 18:53:39 UTC

[Db-derby Wiki] Update of "LanguageOptimize" by Army

Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Db-derby Wiki" for change notification.

The following page has been changed by Army:
http://wiki.apache.org/db-derby/LanguageOptimize

------------------------------------------------------------------------------
  = Derby SQL Optimization =
  
- Two sub-phases, pre-process and then optimization.
+ Three sub-phases: preprocessing, optimization, and modification of access paths.  Each of these three sub-phases begins with a corresponding call from the following code in the "optimize()" method of DML!StatementNode.java, namely:
+ 
+ {{{
+   resultSet = resultSet.preprocess(getCompilerContext().getNumTables(),
+                   null,
+                   (FromList) null);
+  
+   resultSet = resultSet.optimize(getDataDictionary(), null, 1.0d);
+  
+   resultSet = resultSet.modifyAccessPaths();
+ }}}
+ 
+ Upon completion of these three calls "resultSet" will be the root of a complete query tree that represents the best access path for the query. At execution time we will then walk the tree and make calls to generate the necessary byte-code for each node in the tree.
  
  == Pre-process ==
  
- === Transformations ===
+ === In Brief ===
  
  Preparation, obvious rewrite/optimization
  
@@ -41, +53 @@

   * Union (without ALL) eliminates ORDER BY (prefix columns).
   * And of course, a lot more…
  
+ === In the Code ===
+ 
+ Preprocessing of a query begins with the following call in DML!StatementNode.optimize():
+ 
+ {{{
+   resultSet = resultSet.preprocess(getCompilerContext().getNumTables(),
+                   null,
+                   (FromList) null);
+ }}}
+ 
+ By the time we get to this line of code we have [http://wiki.apache.org/db-derby/LanguageParse parsed] and [http://wiki.apache.org/db-derby/LanguageBind bound] the query into a tree of Nodes that represents the query from a syntactical standpoint.  "resultSet" is the root of that tree.
+ 
+ Each node in the tree defines a "preprocess" method, either directly or via inheritance.  What we do, then, is start at the root of the tree and do a depth-first traversal, calling the "preprocess()" method of every node in the tree.  Depending on the type of node, the call to preprocess() can perform a wide range of transformations and static optimization rewrites, including (but not limited to) the transformations listed above.
+ 
+ As an example, a quick look at the comments in !SelectNode.preprocess() suggests that the following work is done in that method:
+ 
+   - Conversion of expressions into conjunctive normal form
+ 
+   - Linearization of left outer joins
+ 
+   - Flattening of subquery and join nodes
+ 
+   - Transitive closure on the where predicates
+ 
+   - Distinct elimination
+ 
+   - Pushing of single-table predicates
+ 
+   - Creation of a referenced table map
+ 
+ In addition to doing all of that work, the "preprocess()" method of !SelectNode also makes calls to the preprocess() methods of its child nodes, as necessary for full tree traversal.  In particular this means the method makes calls to preprocess its FROM list, its result columns, its WHERE clause, and its GROUP BY list.
+ 
+ At the other end of the spectrum, the only thing the preprocess() method of a !FromBaseTable does is create a referenced table map (see ReferencedTableMaps) and generate a !ProjectRestrictNode.  So as mentioned above, the kind and amount of work that a node does for preprocessing is completely dependent on the type of node in question.
+ 
+ When preprocessing is complete the result will be a modified query tree with additional information (usually in the form of additional or rewritten nodes).  At the very least, all result set nodes in the tree will have been assigned a referenced table map, which is required for successful optimization of those nodes (esp. when pushing predicates).
  
  == Optimization ==
+ 
+ === In Brief ===
  
   * Cost based optimization.
   * Purpose: select join order, join method, access method, least operation/cost.
@@ -57, +106 @@

   * Cost index access according to start/stop keys or predicate’s estimated selectivity (or statistics for the index).
   * If optimizer timeout (too many plans), choose the best one so far
  
+ === In the Code ===
+ 
+ Cost-based optimization of a query begins with the following call in DML!StatementNode.optimize():
+ 
+ {{{
+   resultSet = resultSet.optimize(getDataDictionary(), null, 1.0d);
+ }}}
+ 
+ where "resultSet" is the root of the query tree as it exists upon completion of the "preprocessing" phase.  This call ultimately leads to the instantiation of one or more "!OptimizerImpl" objects, each of which is in turn responsible for carrying out the cost-based optimization of some part of the query tree. Where the !OptimizerImpls are instantiated and how much of the query tree each one optimizes depends on the types of nodes in the query tree.  As a general rule one !OptimizerImpl is created for the top-level query, and then an additional !OptimizerImpl is created for each non-flattened subquery.
+ 
+ Each !OptimizerImpl holds a list of "optimizable" result sets and tries to determine what the best "access path" for each of those result sets is.  An "Optimizable" result set is defined in Derby as any type of result set that could potentially appear within the FROM clause of a SELECT query. If the Optimizable is a base table, then its "access path" determines which (if any) index to use and also indicates which join strategy is most appropriate for that table.  If the Optimizable is not a base table (for example, if it's a union or a subquery) then the access path consists of the combined access paths from the Optimizable's children, plus the best join strategy for that Optimizable.
+ 
+ Each !OptimizerImpl also holds a potentially-empty list of predicates. For any given "complete" join order (see below), every predicate in the list must be pushed to one of the Optimizables.  These predicates are the means by which hash joins are made possible, and they are also heavily influential when estimating the cost of a query.
+ 
+ The !OptimizerImpl's goal is to find the best possible join order for its list of Optimizables.  A "join order" is represented as an array of the Optimizables such that the first element in the array is the leftmost table in the join tree, and the successive elements in the array are the joining tables in the left-deep tree. If, for example, our list of Optimizables consists of four base tables, T1, T2, T3, and T4, one possible join order is {T2, T1, T4, T3}.  The left-deep join tree for this join order would be:
+ 
+ {{{
+          JOIN
+         /    \
+       JOIN    T3
+      /    \
+    JOIN    T4
+   /    \
+  T2     T1
+ }}}
+ 
+ In this case we would first join T2 with T1, then we would take the result of that join and join it with T4, the result of which we would then join with T3 to get our final result set.
+ 
+ A "complete" join order is an array in which every Optimizable in the list has an assigned position.  A "best join order", then, is a complete join order such that the total estimated cost of the join order is less than any other complete join order found so far.
+ 
+ All of that said, there are three primary methods in !OptimizerImpl that encapsulate the general process of trying to find the "best join order" for a given !OptimizerImpl's list of Optimizables.  These three methods, along with the ways in which they interact, are perhaps most easily summarized by the following code in !SelectNode.optimize():
+ 
+ {{{
+ 	/* Optimize this SelectNode */
+ 	while (optimizer.getNextPermutation())
+ 	{
+ 		while (optimizer.getNextDecoratedPermutation())
+ 		{
+ 			optimizer.costPermutation();
+ 		}
+ 	}
+ }}}
+ 
+ Note that by the time we get to this code, "optimizer" is an instance of !OptimizerImpl whose list of Optimizables is the FROM list for the !SelectNode, and whose list of predicates corresponds to the !SelectNode's WHERE clause.  Note also that !SelectNode is not the only place where we find this kind of loop; we find a similar loop in !TableOperatorNode.
+ 
+ The call to "getNextPermutation()" tells the !OptimizerImpl to place an Optimizable at a position in the current join order array.  This method contains all of the logic for adding an Optimizable to the join order, for removing an Optimizable from the join order, and for selecting which Optimizable is next in line for placement/removal.  For details on how all of that works, see JoinOrderPermutations.
+ 
+ Then the call to "getNextDecoratedPermutation()" tells the optimizer to find out what the next available "decorated permutation" is for the newly-placed Optimizable.  A "decorated permutation" translates into an "access path" for the Optimizable.  Thus, as mentioned above, it is composed of index choice plus join strategy for base tables, and is simply the join strategy for non-base tables.  It is perhaps helpful to think of a decorated permutation as a join order with indexes and join strategies that "decorate" the Optimizables.  For details on how a "decoration" is chosen, see DecoratedPermutations.
+ 
+ And finally, the call to "costPermutation()" tells the optimizer to figure out how much it costs to have the Optimizable at it's current position with its current decorated permutation.  As part of this method the !OptimizerImpl figures out which predicates in its list of predicates can be evaluated at the current position in the join order. It then pushes those predicates down to the most-recently-placed Optimizable, where they can in turn be used for costing.  The determination of which predicates can be pushed--and the call to push them--is made in the "pushOptPredicates()" method of !OptimizerImpl. See PredicatePushdown for more on that method. See PermutationCosting for more on the code flow that results from a call to the costPermutation() method.
+ 
+ When we have the cost of the Optimizable at its current position in the join order with its current "decoration", the inner loop again calls getNextDecoratedPermutation().  That method checks to see if the most-recently-estimated cost is the best one for the Optimizable so far.  If so, the access path and cost are saved.  Then getNextDecoratedPermutation() will search for the next "decorated permutation" of the Optimizable and, if it finds one, we will again make the call to costPermutation().
+ 
+ This inner loop continues until we run out of decorated permutations for the most-recently-placed Optimizable.  In other words, it continues until we have tried all combinations of index and join strategy for the Optimizable at its current position in the current join order.  At that point, if the Optimizable was the final Optimizable to be placed in a complete join order, the call to getNextDecoratedPermutation() ensures that the cost of the complete join order is saved (within the !OptimizerImpl itself).  That method (getNextDecoratedPermutation()) then returns "false", which brings us back to the outer loop.
+ 
+ The outer loop then calls "getNextPermutation()" again.  That method checks to see if there is at least one remaining join order for which we have not yet determined the cost.  If so, getNextPermutation() finds the next available position in the join order and places an Optimizable at that position.  Then the inner loop repeats itself again.  This process continues until either:
+ 
+   * all possible join orders have been tried.  This means that every Optimizable has been placed at every possible position such that all possible orderings have been evaluated.
+ 
+   or 
+ 
+   * the optimizer "times out" (see OptimizerTimeout).
+ 
+ At that point getNextPermutation() returns false and terminates the outer loop, thus completing the current "round" of optimization for the !OptimizerImpl.
+ 
+ Note that if the !OptimizerImpl corresponds to a subquery within a larger, outer query, then it is possible that the !OptimizerImpl will go through this entire process multiple times.  As a simple example, assume we have the following query:
+ 
+ {{{
+  select * from
+   t1,
+   (select distinct * from t2) x2
+  where t1.i = x2.j;
+ }}}
+ 
+ In this case we'll have two different instances of !OptimizerImpl: one (OI_1) for the outer SELECT query and another (OI_2) for the subquery.  There are two possible join orders for the outer query (OI_1): either {T1, X2} or {X2, T1}.  For *each* join order we will have to figure out the cost of the subquery X2, which means the code in !SelectNode above will be executed twice for OI_2: once for each join order of OI_1.  We call each execution of that code a "round" of optimization for OI_2.
+ 
+ When all cost-based optimization has been completed, every Optimizable node in the query tree knows what its "best access path" is.  Similarly, every !OptimizerImpl knows what its "best join order" is.  The combination of these access paths and join orders together is what makes up the optimizer's final "decision" regarding what it (the Derby optimizer) thinks is the best query execution plan.
+ 
+ == Modification of Access Paths ==
+ 
+ === In Brief ===
+ 
+ Walk the query tree and modify/replace nodes so that the tree reflects a) the optimizer's "best access path" decisions and b) the functional requirements of the query.  Ex:
+ 
+  * Push predicates down the tree for the last time; should push them exactly as they were pushed when the optimizer found the "best access path" for the query.
+    
+  * Generate !IndexToBaseRowNode nodes for base tables whose access path dictates use of an index.
+ 
+  * Add functional nodes like !DistinctNode, !ProjectRestrictNode, !OrderByNode, !NormalizeResultSetNode, !HashTableNode, etc. where required by the query and/or by the optimizer decisions.
+ 
+ The end result of this phase is a complete query tree from which byte-code will be generated and executed.
+ 
+ === In the Code ===
+ 
+ The third and final phase of query "optimization" begins with the following call in DML!StatementNode.optimize():
+ 
+ {{{
+   resultSet = resultSet.modifyAccessPaths();
+ }}}
+ 
+ where "resultSet" is the root of the query tree. At this point every result set in the query tree has an associated "best access path". For this phase of optimization, then, we start at the root of the tree and do a depth-first traversal, calling the "modifyAccessPaths()" method of every result set in the tree. Depending on the type of node, the call to modifyAccessPaths() can perform a wide range of tree transformations.
+ 
+ The purpose of these transformations is to put the query tree into a state that represents the optimizer's final decisions with respect to access paths.  For example, if the optimizer decided to use an index for some base table T1, then the call to !FromBaseTable. modifyAccessPaths() will insert an !IndexToBaseRowNode above T1 in the query tree.  Or as another example, if the optimizer decided to do a hash join with a subquery as the inner result set, the call to "modifyAccessPaths()" in the !ProjectRestrictNode above the subquery will generate a !HashTableNode between the PRN and the subquery. Other kinds of nodes that can be created as part of the "modify access paths" phase include, but are not limited to, !ProjectRestrictNodes, !GroupByNodes, !OrderByNodes, !DistinctNodes, and !NormalizeResultSetNodes.  Each of these newly-added nodes has a specific functional purpose, and each node knows what byte-code it has to generate in order to achieve that functionality.  Since b
 yte-code generation is the first thing that happens when a query is executed, the goal of "modifying access paths" is to make it so that the query tree can generate all of the byte-code necessary to execute the query according the access path and join order decisions made during the "cost-based optimization" phase.
+ 
+ One other important thing that we do when modifying access paths is push predicates down the query tree for the last time.  The "best access paths" that we found in phase 2 all expect that certain predicates were pushed to certain Optimizables in the query tree. Thus if we want to ensure that we generate (and execute) the query plan that was chosen by the optimizer, we have to make sure that we push predicates to the same places they were at when the optimizer found the best overall query plan.
+ 
+ Once all nodes in the tree have been "modified" to reflect the best access path decisions made by the !OptimizerImpl(s) in phase 2, the "optimization" phase of compilation is officially "done."
+ 
+ At this point query flow moves on to the [http://wiki.apache.org/db-derby/LanguageGenerate Byte Code Generation] phase.
+ 
  == References ==
  
  http://db.apache.org/derby/papers/optimizer.html Derby Optimizer Design
   
-  
+ The following Jira issues contain descriptions of various pieces of the optimizer code (which "piece" depends on the issue) on which the "In the Code" sections above are based:
  
+ [https://issues.apache.org/jira/secure/attachment/12336229/DERBY-781_v1.html DERBY-781],
+ [https://issues.apache.org/jira/secure/attachment/12325158/DERBY-805_v5.html DERBY-805],
+ [https://issues.apache.org/jira/secure/attachment/12339226/DERBY-1633_v3.html DERBY-1633],
+ [http://issues.apache.org/jira/browse/DERBY-1777 DERBY-1777],
+ [http://issues.apache.org/jira/browse/DERBY-1866 DERBY-1866].
+