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 rh...@apache.org on 2013/08/07 15:50:14 UTC

svn commit: r1511315 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/sql/compile/ engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: rhillegas
Date: Wed Aug  7 13:50:13 2013
New Revision: 1511315

URL: http://svn.apache.org/r1511315
Log:
DERBY-6211: Trace the end of optimizing a query block so that the xml-based optimizer tracer can associate trace information with the correct query block; commits derby-6211-12-aa-traceEndOfQueryBlock.diff.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizerPlan.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java Wed Aug  7 13:50:13 2013
@@ -46,8 +46,11 @@ public  interface   OptTrace
     /** Start the start of tracing a statement. */
     public  void    traceStartStatement( String statementText );
 
-    /** Start optimizer tracing. */
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList );
+    /** Start optimizer tracing for a query block. */
+    public  void    traceStartQueryBlock( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList );
+
+    /** End tracing the optimization of a query block. */
+    public  void    traceEndQueryBlock();
 
     /** Say that the optimizer ran out of time. */
     public  void    traceTimeout( long currentTime, CostEstimate bestCost );

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizerPlan.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizerPlan.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizerPlan.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizerPlan.java Wed Aug  7 13:50:13 2013
@@ -116,7 +116,7 @@ public abstract class OptimizerPlan
      * Get the leftmost leaf node in this plan.
      * </p>
      */
-    public abstract    RowSource    leftmostLeaf();
+    public abstract    OptimizerPlan    leftmostLeaf();
     
     /**
      * <p>
@@ -179,7 +179,7 @@ public abstract class OptimizerPlan
             return _leafNodeCount;
         }
 
-        public RowSource    leftmostLeaf()   { return leftChild.leftmostLeaf(); }
+        public OptimizerPlan    leftmostLeaf()   { return leftChild.leftmostLeaf(); }
         
         public boolean  isLeftPrefixOf( OptimizerPlan other )
         {
@@ -218,6 +218,39 @@ public abstract class OptimizerPlan
         }
     }
 
+    /** Generic plan for row sources we don't understand */
+    public static  class    DeadEnd extends OptimizerPlan
+    {
+        private String  _name;
+
+        public DeadEnd( String name )
+        {
+            _name = name;
+        }
+
+        public void    bind
+            (
+             DataDictionary dataDictionary,
+             LanguageConnectionContext lcc,
+             CompilerContext cc
+             )
+            throws StandardException
+        {}
+        
+        public boolean isBound() { return true; }
+
+        public int countLeafNodes()    { return 1; }
+
+        public OptimizerPlan    leftmostLeaf()   { return this; }
+        
+        public boolean  isLeftPrefixOf( OptimizerPlan that )
+        {
+            return this.equals( that.leftmostLeaf() );
+        }
+        
+        public  String  toString()  { return _name; }
+    }
+
     public abstract    static  class   RowSource<D extends UniqueTupleDescriptor>   extends OptimizerPlan
     {
         protected   String  _schemaName;
@@ -255,7 +288,7 @@ public abstract class OptimizerPlan
 
         public int countLeafNodes()    { return 1; }
 
-        public RowSource    leftmostLeaf()   { return this; }
+        public OptimizerPlan    leftmostLeaf()   { return this; }
         
         public boolean  isLeftPrefixOf( OptimizerPlan that )
         {

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java Wed Aug  7 13:50:13 2013
@@ -78,7 +78,7 @@ public  class   DefaultOptTrace implemen
         appendTraceString( statementText );
     }
     
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList )
+    public  void    traceStartQueryBlock( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList )
     {
         appendTraceString
             (
@@ -88,6 +88,8 @@ public  class   DefaultOptTrace implemen
              );
     }
 
+    public  void    traceEndQueryBlock() {}
+
     public  void    traceTimeout( long currentTime, CostEstimate bestCost )
     {
         appendTraceString

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java Wed Aug  7 13:50:13 2013
@@ -276,7 +276,7 @@ class OptimizerImpl implements Optimizer
 		bestJoinOrderUsedPredsFromAbove = false;
 
 		// Optimization started
-		if (tracingIsOn()) { tracer().traceStart( timeOptimizationStarted, hashCode(), optimizableList ); }
+		if (tracingIsOn()) { tracer().traceStartQueryBlock( timeOptimizationStarted, hashCode(), optimizableList ); }
 
         // make sure that optimizer overrides are bound and left-deep
         if ( overridingPlan != null )

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java Wed Aug  7 13:50:13 2013
@@ -847,6 +847,7 @@ abstract class TableOperatorNode extends
 			if (predList == null)
                 predList = new PredicateList(getContextManager());
 
+            // recursively create a new optimizer
 			LanguageConnectionContext lcc = getLanguageConnectionContext();
 			OptimizerFactory optimizerFactory = lcc.getOptimizerFactory();
 			optimizer = optimizerFactory.getOptimizer(optList,
@@ -888,11 +889,14 @@ abstract class TableOperatorNode extends
 			}
 
 			retval = sourceResultSet;
+
+            // dispose of the recursively created optimizer
+            if ( optimizerTracingIsOn() ) { getOptimizerTracer().traceEndQueryBlock(); }
 		}
 		else
 		{
 			retval = sourceResultSet.optimize(
-										optimizer.getDataDictionary(),
+										getDataDictionary(),
 										predList,
 										outerCost.rowCount());
 		}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java Wed Aug  7 13:50:13 2013
@@ -22,6 +22,7 @@ package org.apache.derby.impl.sql.compil
 
 import java.io.PrintWriter;
 import java.util.Date;
+import java.util.Stack;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.transform.OutputKeys;
@@ -166,6 +167,39 @@ class   XMLOptTrace implements  OptTrace
         
     ////////////////////////////////////////////////////////////////////////
     //
+    //	NESTED CLASSES
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    public  static  final   class   QueryBlock
+    {
+        final   int                 queryBlockID;
+        final   OptimizableList optimizableList;
+        final   Element         queryBlockElement;
+        
+        Element         currentJoinsElement;
+        int[]              currentJoinOrder;
+        Element         currentBestPlan;
+
+        // reset per join order
+        JoinStrategy    currentDecorationStrategy;
+        Element         currentDecoration;
+
+        public  QueryBlock
+            (
+             int    queryBlockID,
+             OptimizableList    optimizableList,
+             Element    queryBlockElement
+             )
+        {
+            this.queryBlockID = queryBlockID;
+            this.optimizableList = optimizableList;
+            this.queryBlockElement = queryBlockElement;
+        }
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
     //	STATE
     //
     ////////////////////////////////////////////////////////////////////////
@@ -175,17 +209,12 @@ class   XMLOptTrace implements  OptTrace
     
     private Element         _currentStatement;
     private int                 _currentStatementID;
+    private QueryBlock      _currentQueryBlock;
+    private int                 _maxQueryID;
+
 
-    private Element         _currentQuery;
-    private int                 _currentQueryID;
-    private OptimizableList _currentOptimizableList;
-    private Element         _currentJoinsElement;
-    private int[]              _currentJoinOrder;
-    private Element         _currentBestPlan;
-
-    // reset per join order
-    private JoinStrategy    _currentDecorationStrategy;
-    private Element         _currentDecoration;
+    // pushed and popped on query block boundaries
+    private Stack<QueryBlock>  _queryBlockStack;
 
     // context
     private ContextManager  _cm;
@@ -215,86 +244,84 @@ class   XMLOptTrace implements  OptTrace
     public  void    traceStartStatement( String statementText )
     {
         _currentStatementID++;
-        _currentQueryID = 0;
+        _maxQueryID = 0;
+        _currentQueryBlock = null;
+        _queryBlockStack = new Stack<QueryBlock>();
         
         _currentStatement = createElement( _root, STMT, null );
         _currentStatement .setAttribute( STMT_ID, Integer.toString( _currentStatementID ) );
 
-        _currentOptimizableList = null;
-        _currentJoinsElement = null;
-        _currentJoinOrder = null;
-
-        _currentDecorationStrategy = null;
-        _currentDecoration = null;
-
-        _currentBestPlan = null;
-        
         createElement( _currentStatement, STMT_TEXT, statementText );
     }
     
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList )
+    public  void    traceStartQueryBlock( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList )
     {
-        _currentQueryID++;
-        _currentOptimizableList = optimizableList;
-        _currentJoinOrder = null;
-
-        _currentDecorationStrategy = null;
-        _currentDecoration = null;
+        _maxQueryID++;
+        if ( _currentQueryBlock != null ) { _queryBlockStack.push( _currentQueryBlock ); }
 
-        _currentBestPlan = null;
+        Element queryElement = createElement( _currentStatement, QBLOCK, null );
+        queryElement.setAttribute( QBLOCK_OPTIMIZER_ID, Integer.toString( optimizerID ) );
+        queryElement.setAttribute( QBLOCK_START_TIME, formatTimestamp( timeOptimizationStarted ) );
+        queryElement.setAttribute( QBLOCK_ID, Integer.toString( _maxQueryID ) );
 
-        _currentQuery = createElement( _currentStatement, QBLOCK, null );
-        _currentQuery.setAttribute( QBLOCK_OPTIMIZER_ID, Integer.toString( optimizerID ) );
-        _currentQuery.setAttribute( QBLOCK_START_TIME, formatTimestamp( timeOptimizationStarted ) );
-        _currentQuery.setAttribute( QBLOCK_ID, Integer.toString( _currentQueryID ) );
+        _currentQueryBlock = new QueryBlock( _maxQueryID, optimizableList, queryElement );
 
-        if ( _currentOptimizableList != null )
+        if ( optimizableList != null )
         {
-            for ( int i = 0; i < _currentOptimizableList.size(); i++ )
+            for ( int i = 0; i < optimizableList.size(); i++ )
             {
-                Optimizable opt = _currentOptimizableList.getOptimizable( i );
+                Optimizable opt = optimizableList.getOptimizable( i );
 
-                if ( i == 0 )
+                if ( _cm == null )
                 {
                     _cm = ((QueryTreeNode) opt).getContextManager();
                     _lcc = (LanguageConnectionContext) _cm.getContext( LanguageConnectionContext.CONTEXT_ID );
                 }
                 
                 Element optElement = createElement
-                    ( _currentQuery, QBLOCK_OPTIMIZABLE, getOptimizableName( opt ).getFullSQLName() );
+                    ( queryElement, QBLOCK_OPTIMIZABLE, getOptimizableName( opt ).getFullSQLName() );
                 optElement.setAttribute( QBLOCK_OPT_TABLE_NUMBER, Integer.toString( opt.getTableNumber() ) );
             }
         }
     }
     
+    public  void    traceEndQueryBlock()
+    {
+        if ( _queryBlockStack.size() > 0 )
+        {
+            _currentQueryBlock = _queryBlockStack.pop();
+        }
+    }
+
     public  void    traceTimeout( long currentTime, CostEstimate bestCost )
     {
-        Element timeout = createElement( _currentQuery, QBLOCK_TIMEOUT, null );
+        Element timeout = createElement( _currentQueryBlock.queryBlockElement, QBLOCK_TIMEOUT, null );
         formatCost( timeout, bestCost );
     }
     
     public  void    traceVacuous()
     {
-        createElement( _currentQuery, QBLOCK_VACUOUS, null );
+        createElement( _currentQueryBlock.queryBlockElement, QBLOCK_VACUOUS, null );
     }
     
     public  void    traceCompleteJoinOrder()
     {
-        if ( _currentJoinsElement != null )    { _currentJoinsElement.setAttribute( JO_COMPLETE, "true" ); }
+        if ( _currentQueryBlock.currentJoinsElement != null )
+        { _currentQueryBlock.currentJoinsElement.setAttribute( JO_COMPLETE, "true" ); }
     }
     
     public  void    traceSortCost( CostEstimate sortCost, CostEstimate currentCost )
     {
-        Element sc = createElement( _currentQuery, QBLOCK_SORT_COST, null );
+        Element sc = createElement( _currentQueryBlock.queryBlockElement, QBLOCK_SORT_COST, null );
         formatCost( sc, sortCost );
             
-        Element tcis = createElement( _currentQuery, QBLOCK_TOTAL_COST, null );
+        Element tcis = createElement( _currentQueryBlock.queryBlockElement, QBLOCK_TOTAL_COST, null );
         formatCost( tcis, currentCost );
     }
     
     public  void    traceNoBestPlan()
     {
-        createElement( _currentQuery, QBLOCK_NO_BEST_PLAN, null );
+        createElement( _currentQueryBlock.queryBlockElement, QBLOCK_NO_BEST_PLAN, null );
     }
     
     public  void    traceModifyingAccessPaths( int optimizerID ) {}
@@ -303,11 +330,11 @@ class   XMLOptTrace implements  OptTrace
     
     public  void    traceSkippingJoinOrder( int nextOptimizable, int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
     {
-        Optimizable opt = _currentOptimizableList.getOptimizable( nextOptimizable );
+        Optimizable opt = _currentQueryBlock.optimizableList.getOptimizable( nextOptimizable );
 
         Element skip = formatSkip
             (
-             _currentQuery, QBLOCK_SKIP,
+             _currentQueryBlock.queryBlockElement, QBLOCK_SKIP,
              "Useless join order. " + getOptimizableName( opt ).getFullSQLName() + " depends on tables after it in the join order"
              );
         formatJoinOrder( skip, proposedJoinOrder );
@@ -318,21 +345,21 @@ class   XMLOptTrace implements  OptTrace
     
     public  void    traceJoinOrderConsideration( int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
     {
-        _currentJoinsElement = createElement( _currentQuery, JO, null );
-        _currentJoinOrder = proposedJoinOrder;
+        _currentQueryBlock.currentJoinsElement = createElement( _currentQueryBlock.queryBlockElement, JO, null );
+        _currentQueryBlock.currentJoinOrder = proposedJoinOrder;
 
-        _currentDecorationStrategy = null;
-        _currentDecoration = null;
+        _currentQueryBlock.currentDecorationStrategy = null;
+        _currentQueryBlock.currentDecoration = null;
 
-        formatJoinOrder( _currentJoinsElement, proposedJoinOrder );
+        formatJoinOrder( _currentQueryBlock.currentJoinsElement, proposedJoinOrder );
     }
 
     public  void    traceCostWithoutSortAvoidance( CostEstimate currentCost )
     {
         formatPlanCost
             (
-             _currentJoinsElement, "withoutSortAvoidance",
-             _currentJoinOrder, Optimizer.NORMAL_PLAN, currentCost
+             _currentQueryBlock.currentJoinsElement, "withoutSortAvoidance",
+             _currentQueryBlock.currentJoinOrder, Optimizer.NORMAL_PLAN, currentCost
              );
     }
     
@@ -340,8 +367,8 @@ class   XMLOptTrace implements  OptTrace
     {
         formatPlanCost
             (
-             _currentJoinsElement, "withSortAvoidance",
-             _currentJoinOrder, Optimizer.SORT_AVOIDANCE_PLAN, currentSortAvoidanceCost
+             _currentQueryBlock.currentJoinsElement, "withSortAvoidance",
+             _currentQueryBlock.currentJoinOrder, Optimizer.SORT_AVOIDANCE_PLAN, currentSortAvoidanceCost
              );
     }
     
@@ -352,25 +379,29 @@ class   XMLOptTrace implements  OptTrace
     public  void    traceRememberingBestJoinOrder
         ( int joinPosition, int[] bestJoinOrder, int planType, CostEstimate planCost, JBitSet assignedTableMap )
     {
-        if ( _currentBestPlan != null ) { _currentQuery.removeChild( _currentBestPlan ); }
-        _currentBestPlan = formatPlanCost( _currentQuery, "bestPlan", bestJoinOrder, planType, planCost );
+        if ( _currentQueryBlock.currentBestPlan != null )
+        { _currentQueryBlock.queryBlockElement.removeChild( _currentQueryBlock.currentBestPlan ); }
+        _currentQueryBlock.currentBestPlan = formatPlanCost
+            ( _currentQueryBlock.queryBlockElement, "bestPlan", bestJoinOrder, planType, planCost );
     }
     
     public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable )
     {
-        formatSkip( _currentDecoration, DECORATION_SKIP, "Exceeds limit on memory per table: " + maxMemoryPerTable );
+        formatSkip
+            ( _currentQueryBlock.currentDecoration, DECORATION_SKIP, "Exceeds limit on memory per table: " + maxMemoryPerTable );
     }
     
     public  void    traceCostOfNScans( int tableNumber, double rowCount, CostEstimate cost ) {}
     
     public  void    traceSkipUnmaterializableHashJoin()
     {
-        formatSkip( _currentDecoration, DECORATION_SKIP, "Hash strategy not possible because table is not materializable" );
+        formatSkip
+            ( _currentQueryBlock.currentDecoration, DECORATION_SKIP, "Hash strategy not possible because table is not materializable" );
     }
     
     public  void    traceSkipHashJoinNoHashKeys()
     {
-        formatSkip( _currentDecoration, DECORATION_SKIP, "No hash keys" );
+        formatSkip( _currentQueryBlock.currentDecoration, DECORATION_SKIP, "No hash keys" );
     }
     
     public  void    traceHashKeyColumns( int[] hashKeyColumns ) {}
@@ -378,7 +409,7 @@ class   XMLOptTrace implements  OptTrace
     
     public  void    traceConsideringJoinStrategy( JoinStrategy js, int tableNumber )
     {
-        _currentDecorationStrategy = js;
+        _currentQueryBlock.currentDecorationStrategy = js;
     }
     
     public  void    traceRememberingBestAccessPath( AccessPath accessPath, int tableNumber, int planType ) {}
@@ -388,11 +419,12 @@ class   XMLOptTrace implements  OptTrace
     {
         Optimizable opt = getOptimizable( tableNumber );
         
-        _currentDecoration = createElement( _currentJoinsElement, DECORATION, null );
+        _currentQueryBlock.currentDecoration = createElement( _currentQueryBlock.currentJoinsElement, DECORATION, null );
 
-        _currentDecoration.setAttribute( DECORATION_CONGLOM_NAME, cd.getConglomerateName() );
-        _currentDecoration.setAttribute( DECORATION_TABLE_NAME, getOptimizableName( opt ).toString() );
-        _currentDecoration.setAttribute( DECORATION_JOIN_STRATEGY, _currentDecorationStrategy.getName() );
+        _currentQueryBlock.currentDecoration.setAttribute( DECORATION_CONGLOM_NAME, cd.getConglomerateName() );
+        _currentQueryBlock.currentDecoration.setAttribute( DECORATION_TABLE_NAME, getOptimizableName( opt ).toString() );
+        _currentQueryBlock.currentDecoration.setAttribute
+            ( DECORATION_JOIN_STRATEGY, _currentQueryBlock.currentDecorationStrategy.getName() );
         
 		String[]	columnNames = cd.getColumnNames();
 
@@ -402,7 +434,7 @@ class   XMLOptTrace implements  OptTrace
 
             for ( int i = 0; i < keyColumns.length; i++ )
             {
-                createElement( _currentDecoration, DECORATION_KEY, columnNames[ keyColumns[ i ] - 1 ] );
+                createElement( _currentQueryBlock.currentDecoration, DECORATION_KEY, columnNames[ keyColumns[ i ] - 1 ] );
             }
 		}
     }
@@ -444,7 +476,7 @@ class   XMLOptTrace implements  OptTrace
          double    extraNonQualifierSelectivity
          )
     {
-        Element cost = createElement( _currentDecoration, DECORATION_CONGLOM_COST, null );
+        Element cost = createElement( _currentQueryBlock.currentDecoration, DECORATION_CONGLOM_COST, null );
         cost.setAttribute( "name", cd.getConglomerateName() );
 
         formatCost( cost, costEstimate );
@@ -488,9 +520,9 @@ class   XMLOptTrace implements  OptTrace
     /** Get the Optimizable with the given tableNumber */
     private Optimizable getOptimizable( int tableNumber )
     {
-        for ( int i = 0; i < _currentOptimizableList.size(); i++ )
+        for ( int i = 0; i < _currentQueryBlock.optimizableList.size(); i++ )
         {
-            Optimizable candidate = _currentOptimizableList.getOptimizable( i );
+            Optimizable candidate = _currentQueryBlock.optimizableList.getOptimizable( i );
             
             if ( tableNumber == candidate.getTableNumber() )    { return candidate; }
         }
@@ -619,7 +651,7 @@ class   XMLOptTrace implements  OptTrace
     private boolean isComplete( int[] joinOrder )
     {
         if ( joinOrder == null ) { return false; }
-        if ( joinOrder.length < _currentOptimizableList.size() ) { return false; }
+        if ( joinOrder.length < _currentQueryBlock.optimizableList.size() ) { return false; }
 
         for ( int i = 0; i < joinOrder.length; i++ )
         {
@@ -655,7 +687,7 @@ class   XMLOptTrace implements  OptTrace
                 int     optimizableNumber = proposedJoinOrder[ idx ];
                 if ( optimizableNumber >= 0 )
                 {
-                    Optimizable optimizable = _currentOptimizableList.getOptimizable( optimizableNumber );
+                    Optimizable optimizable = _currentQueryBlock.optimizableList.getOptimizable( optimizableNumber );
                     createElement( parent, JO_SLOT, getOptimizableName( optimizable ).getFullSQLName() );
                 }
             }
@@ -696,14 +728,14 @@ class   XMLOptTrace implements  OptTrace
             {
                 int     listIndex = planOrder[ i ];
 
-                if ( listIndex >= _currentOptimizableList.size() )
+                if ( listIndex >= _currentQueryBlock.optimizableList.size() )
                 {
                     // should never happen!
                     buffer.append( "{ UNKNOWN LIST INDEX " + listIndex + " } " );
                     continue;
                 }
 
-                Optimizable optimizable = _currentOptimizableList.getOptimizable( listIndex );
+                Optimizable optimizable = _currentQueryBlock.optimizableList.getOptimizable( listIndex );
             
                 AccessPath  ap = avoidSort ?
                     optimizable.getBestSortAvoidancePath() : optimizable.getBestAccessPath();
@@ -712,7 +744,9 @@ class   XMLOptTrace implements  OptTrace
                     ((StaticMethodCallNode) ((FromVTI) ((ProjectRestrictNode) optimizable).getChildResult()).getMethodCall()).ad :
                     ap.getConglomerateDescriptor();
 
-                OptimizerPlan   current = OptimizerPlan.makeRowSource( utd, _lcc.getDataDictionary() );
+                OptimizerPlan   current =   (utd == null) ?
+                    new OptimizerPlan.DeadEnd( getOptimizableName( optimizable ).toString() ) :
+                    OptimizerPlan.makeRowSource( utd, _lcc.getDataDictionary() );
 
                 if ( plan != null )
                 {

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java?rev=1511315&r1=1511314&r2=1511315&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java Wed Aug  7 13:50:13 2013
@@ -89,7 +89,8 @@ public  class   DummyOptTrace   implemen
     //
     // Don't need to bother implementing the rest of the behavior.
     //
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList ) {}
+    public  void    traceStartQueryBlock( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList ) {}
+    public  void    traceEndQueryBlock() {}
     public  void    traceTimeout( long currentTime, CostEstimate bestCost ) {}
     public  void    traceVacuous() {}
     public  void    traceCompleteJoinOrder() {}