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/06/11 20:29:47 UTC

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

Author: rhillegas
Date: Tue Jun 11 18:29:46 2013
New Revision: 1491893

URL: http://svn.apache.org/r1491893
Log:
DERBY-6211: Committed derby-6211-05-aa-xmlOptimizerTracer, which adds xml-based optimizer tracing.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/Java5SystemProcedures.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.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/HashJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Level2OptimizerImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.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/OptimizerTracer.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/DummyOptTrace.java
    db/derby/code/trunk/tools/jar/extraDBMSclasses.properties

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/Java5SystemProcedures.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/catalog/Java5SystemProcedures.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/Java5SystemProcedures.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/Java5SystemProcedures.java Tue Jun 11 18:29:46 2013
@@ -65,6 +65,7 @@ public  class   Java5SystemProcedures
         { "databaseMetaData", "org.apache.derby.impl.tools.optional.DBMDWrapper" },
         { "foreignViews", "org.apache.derby.impl.tools.optional.ForeignDBViews" },
         { "optimizerTracing", "org.apache.derby.impl.sql.compile.OptimizerTracer" },
+        { "optimizerTracingViews", "org.apache.derby.impl.sql.compile.OptTraceViewer" },
     };
 
     ///////////////////////////////////////////////////////////////////////////////////

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java Tue Jun 11 18:29:46 2013
@@ -176,6 +176,9 @@ public interface JoinStrategy {
 	/** Get the costing type, for use with StoreCostController.getScanCost */
 	int scanCostType();
 
+    /** Get the operator symbol used to represent this join strategy in optimizer traces */
+    String  getOperatorSymbol();
+
 	/**
 	 * Get the name of the result set method for base table scans
 	 *

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=1491893&r1=1491892&r2=1491893&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 Tue Jun 11 18:29:46 2013
@@ -47,7 +47,7 @@ public  interface   OptTrace
     public  void    traceStartStatement( String statementText );
 
     /** Start optimizer tracing. */
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID );
+    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList );
 
     /** Say that the optimizer ran out of time. */
     public  void    traceTimeout( long currentTime, CostEstimate bestCost );
@@ -98,7 +98,8 @@ public  interface   OptTrace
     public  void    traceSortNeededForOrdering( int planType, RequiredRowOrdering requiredRowOrdering );
 
     /** Say that we are remembering the current plan as the best join order so far. */
-    public  void    traceRememberingBestJoinOrder( int joinPosition, int[] bestJoinOrder, JBitSet assignedTableMap );
+    public  void    traceRememberingBestJoinOrder
+        ( int joinPosition, int[] bestJoinOrder, int planType, CostEstimate planCost, JBitSet assignedTableMap );
 
     /** Say that we are skipping a plan because it consumes too much memory. */
     public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable );

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=1491893&r1=1491892&r2=1491893&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 Tue Jun 11 18:29:46 2013
@@ -28,6 +28,7 @@ import org.apache.derby.iapi.sql.compile
 import org.apache.derby.iapi.sql.compile.CostEstimate;
 import org.apache.derby.iapi.sql.compile.JoinStrategy;
 import org.apache.derby.iapi.sql.compile.Optimizable;
+import org.apache.derby.iapi.sql.compile.OptimizableList;
 import org.apache.derby.iapi.sql.compile.Optimizer;
 import org.apache.derby.iapi.sql.compile.OptTrace;
 import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
@@ -78,7 +79,7 @@ public  class   DefaultOptTrace implemen
         appendTraceString( statementText );
     }
     
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID )
+    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList )
     {
         appendTraceString
             (
@@ -199,7 +200,8 @@ public  class   DefaultOptTrace implemen
              );
     }
 
-    public  void    traceRememberingBestJoinOrder( int joinPosition, int[] bestJoinOrder, JBitSet assignedTableMap )
+    public  void    traceRememberingBestJoinOrder
+        ( int joinPosition, int[] bestJoinOrder, int planType, CostEstimate planCost, JBitSet assignedTableMap )
     {
         appendTraceString
             (

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java Tue Jun 11 18:29:46 2013
@@ -298,6 +298,10 @@ public class HashJoinStrategy extends Ba
 		return StoreCostController.STORECOST_SCAN_SET;
 	}
 
+	/** @see JoinStrategy#getOperatorSymbol */
+    public  String  getOperatorSymbol() { return "#"; }
+
+
 	/** @see JoinStrategy#resultSetMethodName */
 	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
 		return "getHashScanResultSet";

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Level2OptimizerImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Level2OptimizerImpl.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Level2OptimizerImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Level2OptimizerImpl.java Tue Jun 11 18:29:46 2013
@@ -66,7 +66,7 @@ public class Level2OptimizerImpl extends
 		this.lcc = lcc;
 
 		// Optimization started
-		if (tracingIsOn()) { tracer().traceStart( timeOptimizationStarted, hashCode() ); }
+		if (tracingIsOn()) { tracer().traceStart( timeOptimizationStarted, hashCode(), optimizableList ); }
 	}
 
 	/** @see Optimizer#getLevel */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java Tue Jun 11 18:29:46 2013
@@ -168,6 +168,9 @@ public class NestedLoopJoinStrategy exte
 		return StoreCostController.STORECOST_SCAN_NORMAL;
 	}
 
+	/** @see JoinStrategy#getOperatorSymbol */
+    public  String  getOperatorSymbol() { return "*"; }
+
 	/** @see JoinStrategy#resultSetMethodName */
 	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
 		if (bulkFetch)

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java?rev=1491893&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java Tue Jun 11 18:29:46 2013
@@ -0,0 +1,171 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.OptTraceViewer
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derby.impl.sql.compile;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.i18n.MessageService;
+import org.apache.derby.iapi.sql.dictionary.OptionalTool;
+
+/**
+ * <p>
+ * OptionalTool for viewing the output created when you xml-trace the optimizer.
+ * </p>
+ */
+public	class   OptTraceViewer  implements OptionalTool
+{
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTANTS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	STATE
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTRUCTOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /** 0-arg constructor required by the OptionalTool contract */
+    public  OptTraceViewer() {}
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // OptionalTool BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Takes the following arguments:
+     * </p>
+     *
+     * <ul>
+     * <li>fileURL - The url of the file holding the xml trace. E.g.: "file:///Users/me/mainline/z.txt"</li>
+     * </ul>
+     *
+     * <p>
+     * ...and creates the following schema objects for viewing the xml trace of the optimizer:
+     * </p>
+     *
+     * <ul>
+     * <li>ArrayList - A user-defined type wrapping java.util.ArrayList.</li>
+     * <li>asList - A factory function for creating ArrayLists.</li>
+     * <li>planCost - An XmlVTI for viewing xml traces.</li>
+     * <li>planCost - A view which passes the file name to the XmlVTI.</li>
+     * </ul>
+     */
+    public  void    loadTool( String... configurationParameters )
+        throws SQLException
+    {
+        if ( (configurationParameters == null) || (configurationParameters.length != 1) )
+        { throw wrap( MessageService.getTextMessage( SQLState.LANG_BAD_OPTIONAL_TOOL_ARGS ) ); }
+
+        String  fileURL = configurationParameters[ 0 ];
+        String  createView = XMLOptTrace.PLAN_COST_VIEW.replace( "FILE_URL", fileURL );
+        
+        Connection          conn = getDerbyConnection();
+
+        executeDDL
+            (
+             conn,
+             "create type ArrayList external name 'java.util.ArrayList' language java"
+             );
+        executeDDL
+            (
+             conn,
+             "create function asList( cell varchar( 32672 ) ... ) returns ArrayList\n" +
+             "language java parameter style derby no sql\n" +
+             "external name 'org.apache.derby.vti.XmlVTI.asList'\n"
+             );
+        executeDDL( conn, XMLOptTrace.PLAN_COST_VTI );
+        executeDDL( conn, createView );
+    }
+
+    /**
+     * <p>
+     * Drop the schema objects which were created for viewing the xml file
+     * containing the optimizer trace.
+     * </p>
+     */
+    public  void    unloadTool( String... configurationParameters )
+        throws SQLException
+    {
+        Connection          conn = getDerbyConnection();
+
+        executeDDL( conn, "drop view planCost" );
+        executeDDL( conn, "drop function planCost" );
+        executeDDL( conn, "drop function asList" );
+        executeDDL( conn, "drop type ArrayList restrict" );
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	MINIONS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    private Connection  getDerbyConnection() throws SQLException
+    {
+        return DriverManager.getConnection( "jdbc:default:connection" );
+    }
+
+    private void    executeDDL
+        ( Connection conn, String text )
+        throws SQLException
+    {
+        PreparedStatement   ddl = prepareStatement( conn, text );
+        ddl.execute();
+        ddl.close();
+    }
+    
+    private PreparedStatement   prepareStatement
+        ( Connection conn, String text )
+        throws SQLException
+    {
+        return conn.prepareStatement( text );
+    }
+
+    /** Wrap an exception in a SQLException */
+    private SQLException    wrap( Throwable t )
+    {
+        return new SQLException( t.getMessage(), t );
+    }
+    
+    private SQLException    wrap( String errorMessage )
+    {
+        String  sqlState = org.apache.derby.shared.common.reference.SQLState.JAVA_EXCEPTION.substring( 0, 5 );
+
+        return new SQLException( errorMessage, sqlState );
+    }
+}
+

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptTraceViewer.java
------------------------------------------------------------------------------
    svn:eol-style = native

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=1491893&r1=1491892&r2=1491893&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 Tue Jun 11 18:29:46 2013
@@ -1864,7 +1864,8 @@ public class OptimizerImpl implements Op
 		if (tracingIsOn())
 		{
 			if (requiredRowOrdering != null)    { tracer().traceSortNeededForOrdering( planType, requiredRowOrdering ); }
-            tracer().traceRememberingBestJoinOrder( joinPosition, ArrayUtil.copy( bestJoinOrder ), (JBitSet) assignedTableMap.clone() );
+            tracer().traceRememberingBestJoinOrder
+                ( joinPosition, ArrayUtil.copy( bestJoinOrder ), planType, currentCost, (JBitSet) assignedTableMap.clone() );
 		}
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerTracer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerTracer.java?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerTracer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OptimizerTracer.java Tue Jun 11 18:29:46 2013
@@ -73,6 +73,8 @@ public	class   OptimizerTracer  implemen
      * </p>
      *
      * <ul>
+     * <li>xml - If the first arg is the "xml" literal, then trace output will be
+     * formatted as xml.</li>
      * <li>custom, $class - If the first arg is the "custom" literal, then the next arg must be
      * the name of a class which implements org.apache.derby.iapi.sql.compile.OptTrace
      * and which has a 0-arg constructor. The 0-arg constructor is called and the resulting
@@ -88,6 +90,13 @@ public	class   OptimizerTracer  implemen
         {
             tracer = new DefaultOptTrace();
         }
+        else if ( "xml".equals( configurationParameters[ 0 ] ) )
+        {
+            try {
+                tracer = new XMLOptTrace();
+            }
+            catch (Throwable t) { throw wrap( t ); }
+        }
         else if ( "custom".equals( configurationParameters[ 0 ] ) )
         {
             if ( configurationParameters.length != 2 )

Added: 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=1491893&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java Tue Jun 11 18:29:46 2013
@@ -0,0 +1,715 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.XMLOptTrace
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+package org.apache.derby.impl.sql.compile;
+
+import java.io.PrintWriter;
+import java.util.Date;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import org.w3c.dom.Attr;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+
+import org.apache.derby.iapi.error.StandardException;
+
+import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
+import org.apache.derby.iapi.util.JBitSet;
+import org.apache.derby.iapi.services.monitor.Monitor;
+import org.apache.derby.iapi.sql.compile.AccessPath;
+import org.apache.derby.iapi.sql.compile.CostEstimate;
+import org.apache.derby.iapi.sql.compile.JoinStrategy;
+import org.apache.derby.iapi.sql.compile.OptTrace;
+import org.apache.derby.iapi.sql.compile.Optimizable;
+import org.apache.derby.iapi.sql.compile.OptimizableList;
+import org.apache.derby.iapi.sql.compile.Optimizer;
+import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
+
+/**
+ * Optimizer tracer which produces output in an xml format.
+ */
+public  class   XMLOptTrace implements  OptTrace
+{
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTANTS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    // statement tags
+    private static  final   String  STMT = "statement";
+    private static  final   String  STMT_ID = "stmtID";
+    private static  final   String  STMT_TEXT = "stmtText";
+
+    // query block tags
+    private static  final   String  QBLOCK = "queryBlock";
+    private static  final   String  QBLOCK_OPTIMIZER_ID = "qbOptimizerID";
+    private static  final   String  QBLOCK_START_TIME = "qbStartTime";
+    private static  final   String  QBLOCK_ID = "qbID";
+    private static  final   String  QBLOCK_OPTIMIZABLE = "qbOptimizable";
+    private static  final   String  QBLOCK_OPT_TABLE_NUMBER = "qboTableNumber";
+    private static  final   String  QBLOCK_TIMEOUT = "qbTimeout";
+    private static  final   String  QBLOCK_VACUOUS = "qbVacuous";
+    private static  final   String  QBLOCK_SORT_COST = "qbSortCost";
+    private static  final   String  QBLOCK_TOTAL_COST = "qbTotalCost";
+    private static  final   String  QBLOCK_NO_BEST_PLAN = "qbNoBestPlan";
+    private static  final   String  QBLOCK_SKIP = "qbSkip";
+
+    // join order tags
+    private static  final   String  JO = "joinOrder";
+    private static  final   String  JO_COMPLETE = "joComplete";
+    private static  final   String  JO_SLOT = "joSlot";
+
+    // decoration tags
+    private static  final   String  DECORATION = "decoration";
+    private static  final   String  DECORATION_CONGLOM_NAME = "decConglomerateName";
+    private static  final   String  DECORATION_KEY = "decKey";
+    private static  final   String  DECORATION_TABLE_NAME = "decTableName";
+    private static  final   String  DECORATION_JOIN_STRATEGY = "decJoinStrategy";
+    private static  final   String  DECORATION_SKIP = "decSkip";
+    private static  final   String  DECORATION_CONGLOM_COST = "decConglomerateCost";
+    private static  final   String  DECORATION_FIRST_COLUMN_SELECTIVITY = "decExtraFirstColumnPreds";
+    private static  final   String  DECORATION_EXTRA_START_STOP_SELECTIVITY = "decExtraFirstStartStopPreds";
+    private static  final   String  DECORATION_START_STOP_SELECTIVITY = "decStartStopPred";
+    private static  final   String  DECORATION_EXTRA_QUALIFIERS = "decExtraQualifiers";
+    private static  final   String  DECORATION_EXTRA_NON_QUALIFIERS = "decExtraNonQualifiers";
+
+    // skip tags
+    private static  final   String  SKIP_REASON = "skipReason";
+
+    // plan cost tags
+    private static  final   String  PC = "planCost";
+    private static  final   String  PC_TYPE = "pcType";
+    private static  final   String  PC_COMPLETE = "pcComplete";
+    private static  final   String  PC_AVOID_SORT= "pcAvoidSort";
+    private static  final   String  PC_SUMMARY= "pcSummary";
+    private static  final   String  PC_VERBOSE= "pcVerbose";
+
+    // CostEstimate tags
+    private static  final   String  CE_ESTIMATED_COST = "ceEstimatedCost";
+    private static  final   String  CE_ROW_COUNT = "ceEstimatedRowCount";
+    private static  final   String  CE_SINGLE_SCAN_ROW_COUNT = "ceSingleScanRowCount";
+
+    // selectivity tags
+    private static  final   String  SEL_COUNT = "selCount";
+    private static  final   String  SEL_SELECTIVITY = "selSelectivity";
+
+    //
+    // Statement and view for declaring a table function which reads the planCost element.
+    // This table function is an instance of the XmlVTI and assumes that you have
+    // already declared an ArrayList user-type and an asList factory function for it.
+    //
+    static  final   String  PLAN_COST_VTI =
+        "create function planCost\n" +
+        "(\n" +
+        "    xmlResourceName varchar( 32672 ),\n" +
+        "    rowTag varchar( 32672 ),\n" +
+        "    parentTags ArrayList,\n" +
+        "    childTags ArrayList\n" +
+        ")\n" +
+        "returns table\n" +
+        "(\n" +
+        "    text varchar( 32672 ),\n" +
+        "    stmtID    int,\n" +
+        "    qbID   int,\n" +
+        "    complete  boolean,\n" +
+        "    summary   varchar( 32672 ),\n" +
+        "    verbose   varchar( 32672 ),\n" +
+        "    type        varchar( 50 ),\n" +
+        "    estimatedCost        double,\n" +
+        "    estimatedRowCount    bigint\n" +
+        ")\n" +
+        "language java parameter style derby_jdbc_result_set no sql\n" +
+        "external name 'org.apache.derby.vti.XmlVTI.xmlVTI'\n";
+
+    static  final   String  PLAN_COST_VIEW =
+        "create view planCost as\n" +
+        "select *\n" +
+        "from table\n" +
+        "(\n" +
+        "    planCost\n" +
+        "    (\n" +
+        "        'FILE_URL',\n" +
+        "        'planCost',\n" +
+        "        asList( '" + STMT_TEXT + "', '" + STMT_ID + "', '" + QBLOCK_ID + "' ),\n" +
+        "        asList( '" + PC_COMPLETE + "', '" + PC_SUMMARY + "', '" + PC_VERBOSE + "', '" + PC_TYPE + "', '" +
+        CE_ESTIMATED_COST + "', '" + CE_ROW_COUNT + "' )\n" +
+        "     )\n" +
+        ") v\n";
+        
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	STATE
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    private Document    _doc;
+    private Element         _root;
+    
+    private Element         _currentStatement;
+    private int                 _currentStatementID;
+
+    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;
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTRUCTOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /** 0-arg constructor required by OptTrace contract */
+    public  XMLOptTrace()
+        throws ParserConfigurationException
+    {
+        _doc = DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+        _root = createElement( null, "optimizerTrace", null );
+        _doc.appendChild( _root );
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	BEHAVIOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    public  void    traceStartStatement( String statementText )
+    {
+        _currentStatementID++;
+        _currentQueryID = 0;
+        
+        _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 )
+    {
+        _currentQueryID++;
+        _currentOptimizableList = optimizableList;
+        _currentJoinOrder = null;
+
+        _currentDecorationStrategy = null;
+        _currentDecoration = null;
+
+        _currentBestPlan = null;
+
+        _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 ) );
+
+        if ( _currentOptimizableList != null )
+        {
+            for ( int i = 0; i < _currentOptimizableList.size(); i++ )
+            {
+                Optimizable opt = _currentOptimizableList.getOptimizable( i );
+                Element optElement = createElement( _currentQuery, QBLOCK_OPTIMIZABLE, getOptimizableName( opt ) );
+                optElement.setAttribute( QBLOCK_OPT_TABLE_NUMBER, Integer.toString( opt.getTableNumber() ) );
+            }
+        }
+    }
+    
+    public  void    traceTimeout( long currentTime, CostEstimate bestCost )
+    {
+        Element timeout = createElement( _currentQuery, QBLOCK_TIMEOUT, null );
+        formatCost( timeout, bestCost );
+    }
+    
+    public  void    traceVacuous()
+    {
+        createElement( _currentQuery, QBLOCK_VACUOUS, null );
+    }
+    
+    public  void    traceCompleteJoinOrder()
+    {
+        if ( _currentJoinsElement != null )    { _currentJoinsElement.setAttribute( JO_COMPLETE, "true" ); }
+    }
+    
+    public  void    traceSortCost( CostEstimate sortCost, CostEstimate currentCost )
+    {
+        Element sc = createElement( _currentQuery, QBLOCK_SORT_COST, null );
+        formatCost( sc, sortCost );
+            
+        Element tcis = createElement( _currentQuery, QBLOCK_TOTAL_COST, null );
+        formatCost( tcis, currentCost );
+    }
+    
+    public  void    traceNoBestPlan()
+    {
+        createElement( _currentQuery, QBLOCK_NO_BEST_PLAN, null );
+    }
+    
+    public  void    traceModifyingAccessPaths( int optimizerID ) {}
+    
+    public  void    traceShortCircuiting( boolean timeExceeded, Optimizable thisOpt, int joinPosition ) {}
+    
+    public  void    traceSkippingJoinOrder( int nextOptimizable, int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
+    {
+        Optimizable opt = _currentOptimizableList.getOptimizable( nextOptimizable );
+
+        Element skip = formatSkip
+            (
+             _currentQuery, QBLOCK_SKIP,
+             "Useless join order. " + getOptimizableName( opt ) + " depends on tables after it in the join order"
+             );
+        formatJoinOrder( skip, proposedJoinOrder );
+    }
+    
+    public  void    traceIllegalUserJoinOrder() {}
+    public  void    traceUserJoinOrderOptimized() {}
+    
+    public  void    traceJoinOrderConsideration( int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
+    {
+        _currentJoinsElement = createElement( _currentQuery, JO, null );
+        _currentJoinOrder = proposedJoinOrder;
+
+        _currentDecorationStrategy = null;
+        _currentDecoration = null;
+
+        formatJoinOrder( _currentJoinsElement, proposedJoinOrder );
+    }
+
+    public  void    traceCostWithoutSortAvoidance( CostEstimate currentCost )
+    {
+        formatPlanCost
+            (
+             _currentJoinsElement, "withoutSortAvoidance",
+             _currentJoinOrder, Optimizer.NORMAL_PLAN, currentCost
+             );
+    }
+    
+    public  void    traceCostWithSortAvoidance( CostEstimate currentSortAvoidanceCost )
+    {
+        formatPlanCost
+            (
+             _currentJoinsElement, "withSortAvoidance",
+             _currentJoinOrder, Optimizer.SORT_AVOIDANCE_PLAN, currentSortAvoidanceCost
+             );
+    }
+    
+    public  void    traceCurrentPlanAvoidsSort( CostEstimate bestCost, CostEstimate currentSortAvoidanceCost ) {}
+    public  void    traceCheapestPlanSoFar( int planType, CostEstimate currentCost ) {}
+    public  void    traceSortNeededForOrdering( int planType, RequiredRowOrdering requiredRowOrdering ) {}
+    
+    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 );
+    }
+    
+    public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable )
+    {
+        formatSkip( _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" );
+    }
+    
+    public  void    traceSkipHashJoinNoHashKeys()
+    {
+        formatSkip( _currentDecoration, DECORATION_SKIP, "No hash keys" );
+    }
+    
+    public  void    traceHashKeyColumns( int[] hashKeyColumns ) {}
+    public  void    traceOptimizingJoinNode() {}
+    
+    public  void    traceConsideringJoinStrategy( JoinStrategy js, int tableNumber )
+    {
+        _currentDecorationStrategy = js;
+    }
+    
+    public  void    traceRememberingBestAccessPath( AccessPath accessPath, int tableNumber, int planType ) {}
+    public  void    traceNoMoreConglomerates( int tableNumber ) {}
+    
+    public  void    traceConsideringConglomerate( ConglomerateDescriptor cd, int tableNumber )
+    {
+        Optimizable opt = getOptimizable( tableNumber );
+        
+        _currentDecoration = createElement( _currentJoinsElement, DECORATION, null );
+
+        _currentDecoration.setAttribute( DECORATION_CONGLOM_NAME, cd.getConglomerateName() );
+        _currentDecoration.setAttribute( DECORATION_TABLE_NAME, getOptimizableName( opt ) );
+        _currentDecoration.setAttribute( DECORATION_JOIN_STRATEGY, _currentDecorationStrategy.getName() );
+        
+		String[]	columnNames = cd.getColumnNames();
+
+		if ( cd.isIndex() && (columnNames != null) )
+		{
+			int[]   keyColumns = cd.getIndexDescriptor().baseColumnPositions();
+
+            for ( int i = 0; i < keyColumns.length; i++ )
+            {
+                createElement( _currentDecoration, DECORATION_KEY, columnNames[ keyColumns[ i ] - 1 ] );
+            }
+		}
+    }
+    
+    public  void    traceScanningHeapWithUniqueKey() {}
+    public  void    traceAddingUnorderedOptimizable( int predicateCount ) {}
+    public  void    traceChangingAccessPathForTable( int tableNumber ) {}
+    public  void    traceNoStartStopPosition() {}
+    public  void    traceNonCoveringIndexCost( double cost, int tableNumber ) {}
+    public  void    traceConstantStartStopPositions() {}
+    public  void    traceEstimatingCostOfConglomerate( ConglomerateDescriptor cd, int tableNumber ) {}
+    public  void    traceLookingForSpecifiedIndex( String indexName, int tableNumber ) {}
+    public  void    traceSingleMatchedRowCost( double cost, int tableNumber ) {}
+    public  void    traceCostIncludingExtra1stColumnSelectivity( CostEstimate cost, int tableNumber ) {}
+    public  void    traceNextAccessPath( String baseTable, int predicateCount ) {}
+    public  void    traceCostIncludingExtraStartStop( CostEstimate cost, int tableNumber ) {}
+    public  void    traceCostIncludingExtraQualifierSelectivity( CostEstimate cost, int tableNumber ) {}
+    public  void    traceCostIncludingExtraNonQualifierSelectivity( CostEstimate cost, int tableNumber ) {}
+    public  void    traceCostOfNoncoveringIndex( CostEstimate cost, int tableNumber ) {}
+    public  void    traceRememberingJoinStrategy( JoinStrategy joinStrategy, int tableNumber ) {}
+    public  void    traceRememberingBestAccessPathSubstring( AccessPath ap, int tableNumber ) {}
+    public  void    traceRememberingBestSortAvoidanceAccessPathSubstring( AccessPath ap, int tableNumber ) {}
+    public  void    traceRememberingBestUnknownAccessPathSubstring( AccessPath ap, int tableNumber ) {}
+    
+    public  void    traceCostOfConglomerateScan
+        (
+         int    tableNumber,
+         ConglomerateDescriptor cd,
+         CostEstimate   costEstimate,
+         int    numExtraFirstColumnPreds,
+         double    extraFirstColumnSelectivity,
+         int    numExtraStartStopPreds,
+         double    extraStartStopSelectivity,
+         int    startStopPredCount,
+         double    statStartStopSelectivity,
+         int    numExtraQualifiers,
+         double    extraQualifierSelectivity,
+         int    numExtraNonQualifiers,
+         double    extraNonQualifierSelectivity
+         )
+    {
+        Element cost = createElement( _currentDecoration, DECORATION_CONGLOM_COST, null );
+        cost.setAttribute( "name", cd.getConglomerateName() );
+
+        formatCost( cost, costEstimate );
+        formatSelectivity( cost, DECORATION_FIRST_COLUMN_SELECTIVITY, numExtraFirstColumnPreds, extraFirstColumnSelectivity );
+        formatSelectivity( cost, DECORATION_EXTRA_START_STOP_SELECTIVITY, numExtraStartStopPreds, extraStartStopSelectivity );
+        formatSelectivity( cost, DECORATION_START_STOP_SELECTIVITY, startStopPredCount, statStartStopSelectivity );
+        formatSelectivity( cost, DECORATION_EXTRA_QUALIFIERS, numExtraQualifiers, extraQualifierSelectivity );
+        formatSelectivity( cost, DECORATION_EXTRA_NON_QUALIFIERS, numExtraNonQualifiers, extraNonQualifierSelectivity );
+    }
+    
+    public  void    traceCostIncludingCompositeSelectivityFromStats( CostEstimate cost, int tableNumber ) {}
+    public  void    traceCompositeSelectivityFromStatistics( double statCompositeSelectivity ) {}
+    public  void    traceCostIncludingStatsForIndex( CostEstimate cost, int tableNumber ) {}
+
+    public  void    printToWriter( PrintWriter out )
+    {
+        try {
+            TransformerFactory transformerFactory = TransformerFactory.newInstance();
+            Transformer transformer = transformerFactory.newTransformer();
+            DOMSource source = new DOMSource( _doc );
+            StreamResult result = new StreamResult( out );
+
+            // pretty-print
+            transformer.setOutputProperty( OutputKeys.OMIT_XML_DECLARATION, "no" );
+            transformer.setOutputProperty( OutputKeys.METHOD, "xml" );
+            transformer.setOutputProperty( OutputKeys.INDENT, "yes" );
+            transformer.setOutputProperty( OutputKeys.ENCODING, "UTF-8" );
+            transformer.setOutputProperty( "{http://xml.apache.org/xslt}indent-amount", "4" );
+            
+            transformer.transform( source, result );
+            
+        }   catch (Throwable t) { printThrowable( t ); }
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	MINIONS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /** Get the Optimizable with the given tableNumber */
+    private Optimizable getOptimizable( int tableNumber )
+    {
+        for ( int i = 0; i < _currentOptimizableList.size(); i++ )
+        {
+            Optimizable candidate = _currentOptimizableList.getOptimizable( i );
+            
+            if ( tableNumber == candidate.getTableNumber() )    { return candidate; }
+        }
+
+        return null;
+    }
+
+    /** Get the name of a optimizables */
+    private String    getOptimizableName( Optimizable optimizable )
+    {
+        String  name = null;
+        
+        if ( optimizable instanceof ProjectRestrictNode )
+        {
+            ProjectRestrictNode prn = (ProjectRestrictNode) optimizable;
+            ResultSetNode   rsn = prn.getChildResult();
+            if ( rsn instanceof FromBaseTable )
+            {
+                try {
+                    name = ((FromBaseTable) rsn).getTableName().getFullTableName();
+                }
+                catch (StandardException e)
+                {
+                    // Technically, an exception could occur here if the table name
+                    // was not previously bound and if an error occured while binding it.
+                    // But the FromBaseTable should have been bound long before optimization,
+                    // so this should not be a problem.
+                }
+            }
+        }
+
+        // fallback
+        if ( name == null )
+        {
+            String  nodeClass = optimizable.getClass().getName();
+            name = nodeClass.substring( nodeClass.lastIndexOf( "." ) + 1 );
+        }
+
+        return name;
+    }
+
+    /** Print an exception to the log file */
+    private void    printThrowable( Throwable t )
+    {
+        t.printStackTrace( Monitor.getStream().getPrintWriter() );
+    }
+
+    /** Create an element and add it to a parent */
+    private Element createElement( Element parent, String tag, String content )
+    {
+        Element child = null;
+        
+        try {
+            child = _doc.createElement( tag );
+            if ( parent != null) { parent.appendChild( child ); }
+            if ( content != null ) { child.setTextContent( content ); }
+        }
+        catch (Throwable t) { printThrowable( t ); }
+
+        return child;
+    }
+
+    /** Turn a timestamp into a human-readable string */
+    private String  formatTimestamp( long timestamp ) { return (new Date( timestamp )).toString(); }
+
+    /** Create an element explaining that we're skipping some processing */
+    private Element formatSkip( Element parent, String skipTag, String reason )
+    {
+        Element skip = createElement( parent, skipTag, null );
+        skip.setAttribute( SKIP_REASON, reason );
+
+        return skip;
+    }
+    
+    /** Turn a CostEstimate for a join order into a human-readable element */
+    private Element formatPlanCost( Element parent, String type, int[] planOrder, int planType, CostEstimate raw )
+    {
+        Element cost = createElement( parent, PC, null );
+
+        cost.setAttribute( PC_TYPE, type );
+        if ( isComplete( planOrder ) ) { cost.setAttribute( PC_COMPLETE, "true" ); }
+        if ( planType == Optimizer.SORT_AVOIDANCE_PLAN ) { cost.setAttribute( PC_AVOID_SORT, "true" ); }
+
+        createElement( cost, PC_SUMMARY, formatPlanSummary( planOrder, planType, false ) );
+        createElement( cost, PC_VERBOSE, formatPlanSummary( planOrder, planType, true ) );
+        formatCost( cost, raw );
+
+        return cost;
+    }
+
+    /** Return true if the join order has been completely filled in */
+    private boolean isComplete( int[] joinOrder )
+    {
+        if ( joinOrder == null ) { return false; }
+        if ( joinOrder.length < _currentOptimizableList.size() ) { return false; }
+
+        for ( int i = 0; i < joinOrder.length; i++ )
+        {
+            if ( joinOrder[ i ] < 0 ) { return false; }
+        }
+
+        return true;
+    }
+
+    /** Format a CostEstimate as subelements of a parent */
+    private void    formatCost( Element costElement, CostEstimate raw )
+    {
+        createElement( costElement, CE_ESTIMATED_COST, Double.toString( raw.getEstimatedCost() ) );
+        createElement( costElement, CE_ROW_COUNT, Long.toString( raw.getEstimatedRowCount() ) );
+        createElement( costElement, CE_SINGLE_SCAN_ROW_COUNT, Double.toString( raw.singleScanRowCount() ) );
+    }
+
+    /** Format selectivity subelement */
+    private void    formatSelectivity( Element parent, String tag, int count, double selectivity )
+    {
+        Element child = createElement( parent, tag, null );
+        child.setAttribute( SEL_COUNT, Integer.toString( count ) );
+        child.setAttribute( SEL_SELECTIVITY, Double.toString( selectivity ) );
+    }
+
+    /** Format a join order list */
+    private void    formatJoinOrder( Element parent, int[] proposedJoinOrder )
+    {
+        if ( proposedJoinOrder != null )
+        {
+            for ( int idx = 0; idx < proposedJoinOrder.length; idx++ )
+            {
+                int     optimizableNumber = proposedJoinOrder[ idx ];
+                if ( optimizableNumber >= 0 )
+                {
+                    Optimizable optimizable = _currentOptimizableList.getOptimizable( optimizableNumber );
+                    createElement( parent, JO_SLOT, getOptimizableName( optimizable ) );
+                }
+            }
+        }
+    }
+
+
+    /**
+     * <p>
+     * Produce a string representation of the plan being considered now.
+     * The string has the following grammar:
+     * </p>
+     *
+     * <pre>
+     * join :== factor OP factor
+     *
+     * OP :== "*" | "#"
+     *
+     * factor :== factor | conglomerateName
+     * </pre>
+     */
+    private String  formatPlanSummary( int[] planOrder, int planType, boolean verbose )
+    {
+        StringBuilder   buffer = new StringBuilder();
+        boolean     avoidSort = (planType == Optimizer.SORT_AVOIDANCE_PLAN);
+
+        // a negative optimizable number indicates the end of the plan
+        int planLength = 0;
+        for ( ; planLength < planOrder.length; planLength++ )
+        {
+            if ( planOrder[ planLength ] < 0 ) { break; }
+        }
+
+        // only add parentheses if there are more than 2 slots in the join order
+        int     dontNeedParentheses = 2;
+        int     lastParenthesizedIndex = planLength - dontNeedParentheses;
+        for ( int i = 0; i < lastParenthesizedIndex; i++ ) { buffer.append( "(" ); }
+        
+        for ( int i = 0; i < planLength; i++ )
+        {
+            int     listIndex = planOrder[ i ];
+
+            if ( listIndex >= _currentOptimizableList.size() )
+            {
+                // should never happen!
+                buffer.append( "{ UNKNOWN LIST INDEX " + listIndex + " } " );
+                continue;
+            }
+
+            Optimizable optimizable = _currentOptimizableList.getOptimizable( listIndex );
+            
+            AccessPath  ap = avoidSort ?
+                optimizable.getBestSortAvoidancePath() : optimizable.getBestAccessPath();
+            ConglomerateDescriptor  cd = ap.getConglomerateDescriptor();
+            String  conglomerateName = getConglomerateName( optimizable, cd, verbose );
+            JoinStrategy    js = ap.getJoinStrategy();
+
+            //
+            // The very first optimizable in the join order obiously doesn't join
+            // to anything before it. For that reason, its join strategy is always
+            // NESTED_LOOP. We can just assume that and not clutter up the
+            // representation with vacuous information.
+            //
+            if ( i > 0 ) { buffer.append( " " + js.getOperatorSymbol() + " " ); }
+            
+            buffer.append( conglomerateName );
+            if ( (i > 0) && (i <= lastParenthesizedIndex) ) { buffer.append( ")" ); }
+        }
+
+        return buffer.toString();
+    }
+
+    /**
+     * <p>
+     * Get a human-readable name for a conglomerate.
+     * </p>
+     */
+    private String  getConglomerateName( Optimizable optimizable, ConglomerateDescriptor cd, boolean verbose )
+    {
+        if ( !verbose ) { return cd.getConglomerateName(); }
+
+        StringBuilder   buffer = new StringBuilder();
+        buffer.append( getOptimizableName( optimizable ) );
+        
+        if ( cd.isIndex() )
+        {
+            buffer.append( "{" );
+            String[]	columnNames = cd.getColumnNames();
+            
+            if ( columnNames != null )
+            {
+                int[]   keyColumns = cd.getIndexDescriptor().baseColumnPositions();
+                
+                for ( int i = 0; i < keyColumns.length; i++ )
+                {
+                    if ( i > 0 ) { buffer.append( "," ); }
+                    buffer.append( columnNames[ keyColumns[ i ] - 1 ] );
+                }
+            }
+            buffer.append( "}" );
+        }
+
+        return buffer.toString();
+    }
+    
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/XMLOptTrace.java
------------------------------------------------------------------------------
    svn:eol-style = native

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=1491893&r1=1491892&r2=1491893&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 Tue Jun 11 18:29:46 2013
@@ -30,6 +30,7 @@ import org.apache.derby.iapi.sql.compile
 import org.apache.derby.iapi.sql.compile.JoinStrategy;
 import org.apache.derby.iapi.sql.compile.OptTrace;
 import org.apache.derby.iapi.sql.compile.Optimizable;
+import org.apache.derby.iapi.sql.compile.OptimizableList;
 import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
 
 /**
@@ -88,7 +89,7 @@ public  class   DummyOptTrace   implemen
     //
     // Don't need to bother implementing the rest of the behavior.
     //
-    public  void    traceStart( long timeOptimizationStarted, int optimizerID ) {}
+    public  void    traceStart( long timeOptimizationStarted, int optimizerID, OptimizableList optimizableList ) {}
     public  void    traceTimeout( long currentTime, CostEstimate bestCost ) {}
     public  void    traceVacuous() {}
     public  void    traceCompleteJoinOrder() {}
@@ -105,7 +106,8 @@ public  class   DummyOptTrace   implemen
     public  void    traceCurrentPlanAvoidsSort( CostEstimate bestCost, CostEstimate currentSortAvoidanceCost ) {}
     public  void    traceCheapestPlanSoFar( int planType, CostEstimate currentCost ) {}
     public  void    traceSortNeededForOrdering( int planType, RequiredRowOrdering requiredRowOrdering ) {}
-    public  void    traceRememberingBestJoinOrder( int joinPosition, int[] bestJoinOrder, JBitSet assignedTableMap ) {}
+    public  void    traceRememberingBestJoinOrder
+        ( int joinPosition, int[] bestJoinOrder, int planType, CostEstimate planCost, JBitSet assignedTableMap ) {}
     public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable ) {}
     public  void    traceCostOfNScans( int tableNumber, double rowCount, CostEstimate cost ) {}
     public  void    traceSkipUnmaterializableHashJoin() {}

Modified: db/derby/code/trunk/tools/jar/extraDBMSclasses.properties
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/tools/jar/extraDBMSclasses.properties?rev=1491893&r1=1491892&r2=1491893&view=diff
==============================================================================
--- db/derby/code/trunk/tools/jar/extraDBMSclasses.properties (original)
+++ db/derby/code/trunk/tools/jar/extraDBMSclasses.properties Tue Jun 11 18:29:46 2013
@@ -109,3 +109,4 @@ derby.module.engine.threaddump=org.apach
 
 # optional tools
 derby.module.opttrace=org.apache.derby.impl.sql.compile.OptimizerTracer
+derby.module.opttraceviewer=org.apache.derby.impl.sql.compile.OptTraceViewer