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/05/02 14:34:13 UTC

svn commit: r1478347 - in /db/derby/code/trunk/java/engine/org/apache/derby: iapi/sql/compile/ impl/sql/compile/

Author: rhillegas
Date: Thu May  2 12:34:13 2013
New Revision: 1478347

URL: http://svn.apache.org/r1478347
Log:
DERBY-6211: Make it possible to plug alternative trace logic into the Optimizer.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/Optimizer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromTable.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/JoinNode.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

Added: 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=1478347&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptTrace.java Thu May  2 12:34:13 2013
@@ -0,0 +1,212 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.sql.compile.OptTrace
+
+   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.iapi.sql.compile;
+
+import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
+import org.apache.derby.iapi.util.JBitSet;
+
+/**
+ * Interface for optimizer tracing.
+ */
+public  interface   OptTrace
+{
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTANTS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	BEHAVIOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /** Start optimizer tracing. */
+    public  void    traceStart( long timeOptimizationStarted );
+
+    /** Say that the optimizer ran out of time. */
+    public  void    traceTimeout( long currentTime, CostEstimate bestCost );
+
+    /** Say that there's nothing to optimizer. */
+    public  void    traceVacuous();
+
+    /** Say that we have a complete join order. */
+    public  void    traceCompleteJoinOrder();
+
+    /** Report the sort cost. */
+    public  void    traceSortCost( CostEstimate sortCost, CostEstimate currentCost );
+
+    /** Say that we couldn't find a best plan. */
+    public  void    traceNoBestPlan();
+
+    /** Say that we're modifying access paths. */
+    public  void    traceModifyingAccessPaths();
+
+    /** Say that we short-circuited a join order. */
+    public  void    traceShortCircuiting( boolean timeExceeded, Optimizable thisOpt, int joinPosition );
+
+    /** Say that we're skipping the join order starting with the next optimizable. */
+    public  void    traceSkippingJoinOrder( int nextOptimizable, int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap );
+
+    /** Say that the user specified an impossible join order. */
+    public  void    traceIllegalUserJoinOrder();
+
+    /** Say that we have optimized the user-specified join order. */
+    public  void    traceUserJoinOrderOptimized();
+
+    /** Say that we're considering a join order. */
+    public  void    traceJoinOrderConsideration( int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap );
+
+    /** Report the cost without a sort. */
+    public  void    traceCostWithoutSortAvoidance( CostEstimate currentCost );
+
+    /** Report the cost with sort avoidance.*/
+    public  void    traceCostWithSortAvoidance( CostEstimate currentSortAvoidanceCost );
+
+    /** Say that the current plan avoids a sort. */
+    public  void    traceCurrentPlanAvoidsSort( CostEstimate bestCost, CostEstimate currentSortAvoidanceCost );
+
+    /** Report the cost of the cheapest plan so far. */
+    public  void    traceCheapestPlanSoFar( int planType, CostEstimate currentCost );
+
+    /** Report that this plan needs a sort */
+    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 );
+
+    /** Say that we are skipping a plan because it consumes too much memory. */
+    public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable );
+
+    /** Report the cost of scanning a table a certain number of times */
+    public  void    traceCostOfNScans( int tableNumber, double rowCount, CostEstimate cost );
+
+    /** Say that we won't consider a hash join because the result can't be materialized */
+    public  void    traceSkipUnmaterializableHashJoin();
+
+    /** Say we won't consider a hash join because there are no hash key columns. */
+    public  void    traceSkipHashJoinNoHashKeys();
+
+    /** Report the columns being traced */
+    public  void    traceHashKeyColumns( int[] hashKeyColumns );
+
+    /** Say that we're starting to optimize a join node */
+    public  void    traceOptimizingJoinNode();
+
+    /** Say that we're considering a particular join strategy on a particular table. */
+    public  void    traceConsideringJoinStrategy( JoinStrategy js, int tableNumber );
+
+    /** Report that we've found a best access path. */
+    public  void    traceRememberingBestAccessPath( AccessPath accessPath, int tableNumber, int planType );
+
+    /** Say that we have exhausted the conglomerate possibilities for a table. */
+    public  void    traceNoMoreConglomerates( int tableNumber );
+
+    /** Report that we are considering a conglomerate for a table. */
+    public  void    traceConsideringConglomerate( ConglomerateDescriptor cd, int tableNumber );
+
+    /** Say that we're considering scanning a heap even though we have a unique key match. */
+    public  void    traceScanningHeapWithUniqueKey();
+
+    /** Say that we're adding an unordered optimizable. */
+    public  void    traceAddingUnorderedOptimizable( int predicateCount );
+
+    /** Say that we're considering a different access path for a table. */
+    public  void    traceChangingAccessPathForTable( int tableNumber );
+
+    /** Say that we're setting the lock mode to MODE_TABLE because there is no start/stop position. */
+    public  void    traceNoStartStopPosition();
+
+    /** Say that we're considering a non-covering index. */
+    public  void    traceNonCoveringIndexCost( double cost, int tableNumber );
+
+    /** Say that we're setting the lock mode to MODE_RECORD because the start and stop positions are all constant. */
+    public  void    traceConstantStartStopPositions();
+
+    /** Report the cost of using a particular conglomerate to scan a table. */
+    public  void    traceEstimatingCostOfConglomerate( ConglomerateDescriptor cd, int tableNumber );
+
+    /** Say that we're looking for an index specified by optimizer hints. */
+    public  void    traceLookingForSpecifiedIndex( String indexName, int tableNumber );
+
+    /** Report the cost of a scan which will match exactly one row. */
+    public  void    traceSingleMatchedRowCost( double cost, int tableNumber );
+
+    /** Report the cost of a scan given the selectivity of an extra first column. */
+    public  void    traceCostIncludingExtra1stColumnSelectivity( CostEstimate cost, int tableNumber );
+
+    /** Report that we are advancing to the next access path for the table. */
+    public  void    traceNextAccessPath( String baseTable, int predicateCount );
+
+    /** Report the cost if we include an extra start/stop. */
+    public  void    traceCostIncludingExtraStartStop( CostEstimate cost, int tableNumber );
+
+    /**  Report the cost if we include an extra qualifier. */
+    public  void    traceCostIncludingExtraQualifierSelectivity( CostEstimate cost, int tableNumber );
+
+    /** Report the cost if we include an extra non-qualifier. */
+    public  void    traceCostIncludingExtraNonQualifierSelectivity( CostEstimate cost, int tableNumber );
+
+    /** Report the cost of using a non-covering index. */
+    public  void    traceCostOfNoncoveringIndex( CostEstimate cost, int tableNumber );
+
+    /** Say that we've found a new best join strategy for the table. */
+    public  void    traceRememberingJoinStrategy( JoinStrategy joinStrategy, int tableNumber );
+
+    /** Report the best access path for the table so far. */
+    public  void    traceRememberingBestAccessPathSubstring( AccessPath ap, int tableNumber );
+
+    /** Report the best sort-avoiding access path for this table so far. */
+    public  void    traceRememberingBestSortAvoidanceAccessPathSubstring( AccessPath ap, int tableNumber );
+
+    /** Report an optimizer failure, e.g., while optimizing an outer join */
+    public  void    traceRememberingBestUnknownAccessPathSubstring( AccessPath ap, int tableNumber );
+
+    /** Report the cost of a conglomerate scan. */
+    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
+         );
+
+    /** Report the cost based on selectivities coming out of SYSSTATISTICS. */
+    public  void    traceCostIncludingCompositeSelectivityFromStats( CostEstimate cost, int tableNumber );
+
+    /** Report the selectivity calculated from SYSSTATISTICS. */
+    public  void    traceCompositeSelectivityFromStatistics( double statCompositeSelectivity );
+
+    /** Report the cost based on index statistics. */
+    public  void    traceCostIncludingStatsForIndex( CostEstimate cost, int tableNumber );
+    
+}

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/Optimizer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/Optimizer.java?rev=1478347&r1=1478346&r2=1478347&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/Optimizer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/Optimizer.java Thu May  2 12:34:13 2013
@@ -37,7 +37,8 @@ import org.apache.derby.iapi.util.JBitSe
  *	   caller.
  */
 
-public interface Optimizer {
+public interface Optimizer
+{
 	/**
 		Module name for the monitor's module locating system.
 	 */
@@ -86,69 +87,9 @@ public interface Optimizer {
 	/** Indicates a sort-avoidance plan */
 	int SORT_AVOIDANCE_PLAN = 2;
 
-	// optimizer trace
-	public static final int STARTED = 1;
-	public static final int TIME_EXCEEDED =2;
-	public static final int NO_TABLES = 3;
-	public static final int COMPLETE_JOIN_ORDER = 4;
-	public static final int COST_OF_SORTING = 5;
-	public static final int NO_BEST_PLAN = 6;
-	public static final int MODIFYING_ACCESS_PATHS = 7;
-	public static final int SHORT_CIRCUITING = 8;
-	public static final int SKIPPING_JOIN_ORDER = 9;
-	public static final int ILLEGAL_USER_JOIN_ORDER = 10;
-	public static final int USER_JOIN_ORDER_OPTIMIZED = 11;
-	public static final int CONSIDERING_JOIN_ORDER = 12;
-	public static final int TOTAL_COST_NON_SA_PLAN = 13;
-	public static final int TOTAL_COST_SA_PLAN = 14;
-	public static final int TOTAL_COST_WITH_SORTING = 15;
-	public static final int CURRENT_PLAN_IS_SA_PLAN = 16;
-	public static final int CHEAPEST_PLAN_SO_FAR = 17;
-	public static final int PLAN_TYPE = 18;
-	public static final int COST_OF_CHEAPEST_PLAN_SO_FAR = 19;
-	public static final int SORT_NEEDED_FOR_ORDERING = 20;
-	public static final int REMEMBERING_BEST_JOIN_ORDER = 21;
-	public static final int SKIPPING_DUE_TO_EXCESS_MEMORY = 22;
-	public static final int COST_OF_N_SCANS = 23;
-	public static final int HJ_SKIP_NOT_MATERIALIZABLE = 24;
-	public static final int HJ_SKIP_NO_JOIN_COLUMNS = 25;
-	public static final int HJ_HASH_KEY_COLUMNS = 26;
-	public static final int CALLING_ON_JOIN_NODE = 27;
-	public static final int CONSIDERING_JOIN_STRATEGY = 28;
-	public static final int REMEMBERING_BEST_ACCESS_PATH = 29;
-	public static final int NO_MORE_CONGLOMERATES = 30;
-	public static final int CONSIDERING_CONGLOMERATE = 31;
-	public static final int SCANNING_HEAP_FULL_MATCH_ON_UNIQUE_KEY = 32;
-	public static final int ADDING_UNORDERED_OPTIMIZABLE = 33;
-	public static final int CHANGING_ACCESS_PATH_FOR_TABLE = 34;
-	public static final int TABLE_LOCK_NO_START_STOP = 35;
-	public static final int NON_COVERING_INDEX_COST = 36;
-	public static final int ROW_LOCK_ALL_CONSTANT_START_STOP = 37;
-	public static final int ESTIMATING_COST_OF_CONGLOMERATE = 38;
-	public static final int LOOKING_FOR_SPECIFIED_INDEX = 39;
-	public static final int MATCH_SINGLE_ROW_COST = 40;
-	public static final int COST_INCLUDING_EXTRA_1ST_COL_SELECTIVITY = 41;
-	public static final int CALLING_NEXT_ACCESS_PATH = 42;
-	public static final int TABLE_LOCK_OVER_THRESHOLD = 43;
-	public static final int ROW_LOCK_UNDER_THRESHOLD = 44;
-	public static final int COST_INCLUDING_EXTRA_START_STOP = 45;
-	public static final int COST_INCLUDING_EXTRA_QUALIFIER_SELECTIVITY = 46;
-	public static final int COST_INCLUDING_EXTRA_NONQUALIFIER_SELECTIVITY = 47;
-	public static final int COST_OF_NONCOVERING_INDEX = 48;
-	public static final int REMEMBERING_JOIN_STRATEGY = 49;
-	public static final int REMEMBERING_BEST_ACCESS_PATH_SUBSTRING = 50;
-	public static final int REMEMBERING_BEST_SORT_AVOIDANCE_ACCESS_PATH_SUBSTRING = 51;
-	public static final int REMEMBERING_BEST_UNKNOWN_ACCESS_PATH_SUBSTRING = 52;
-	public static final int COST_OF_CONGLOMERATE_SCAN1 = 53;
-	public static final int COST_OF_CONGLOMERATE_SCAN2 = 54;
-	public static final int COST_OF_CONGLOMERATE_SCAN3 = 55;
-	public static final int COST_OF_CONGLOMERATE_SCAN4 = 56;
-	public static final int COST_OF_CONGLOMERATE_SCAN5 = 57;
-	public static final int COST_OF_CONGLOMERATE_SCAN6 = 58;
-	public static final int COST_OF_CONGLOMERATE_SCAN7 = 59;
-	public static final int COST_INCLUDING_COMPOSITE_SEL_FROM_STATS= 60;
-	public static final int COMPOSITE_SEL_FROM_STATS = 61;
-	public static final int COST_INCLUDING_STATS_FOR_INDEX = 62;
+    /** Return true if optimizer tracing is on */
+    public  boolean tracingIsOn();
+
 	/**
 	 * Iterate through the permutations, returning false when the permutations
 	 * are exhausted.
@@ -250,6 +191,9 @@ public interface Optimizer {
 	/** Get a new CostEstimate object */
 	public CostEstimate newCostEstimate();
 
+	/** Get the trace machinery */
+	public OptTrace tracer();
+
 	/** Get the estimated cost of the optimized query */
 	public CostEstimate getOptimizedCost();
 
@@ -305,12 +249,6 @@ public interface Optimizer {
 	JoinStrategy getJoinStrategy(String whichStrategy);
 
 	/**
-	 * Optimizer trace.
-	 */
-	public void trace(int traceFlag, int intParam1, int intParam2,
-					  double doubleParam, Object objectParam1);
-
-	/**
 	 * Get the level of this optimizer.
 	 *
 	 * @return The level of this optimizer.

Added: 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=1478347&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DefaultOptTrace.java Thu May  2 12:34:13 2013
@@ -0,0 +1,558 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.DefaultOptTrace
+
+   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 org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+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.Optimizable;
+import org.apache.derby.iapi.sql.compile.Optimizer;
+import org.apache.derby.iapi.sql.compile.OptTrace;
+import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
+import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
+import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
+import org.apache.derby.iapi.util.JBitSet;
+
+/**
+ * This is the default optimizer tracing logic for use when a custom
+ * tracer wasn't specified.
+ */
+public  class   DefaultOptTrace implements  OptTrace
+{
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTANTS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	STATE
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    private LanguageConnectionContext   _lcc;
+    private int                                     _optimizerID;
+    
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	CONSTRUCTOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /** Make a DefaultOptTrace */
+    public  DefaultOptTrace( LanguageConnectionContext lcc, int optimizerID )
+    {
+        _lcc = lcc;
+        _optimizerID = optimizerID;
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	OptTrace BEHAVIOR
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    public  void    traceStart( long timeOptimizationStarted )
+    {
+        appendTraceString
+            (
+             "Optimization started at time " + 
+             timeOptimizationStarted +
+             " using optimizer " + _optimizerID
+             );
+    }
+
+    public  void    traceTimeout( long currentTime, CostEstimate bestCost )
+    {
+        appendTraceString
+            (
+             "Optimization time exceeded at time " +
+             currentTime + "\n" + bestCost
+             );
+    }
+   
+    public  void    traceVacuous()
+    {
+        appendTraceString( "No tables to optimize." );
+    }
+
+    public  void    traceCompleteJoinOrder()
+    {
+        appendTraceString( "We have a complete join order." );
+    }
+
+    public  void    traceSortCost( CostEstimate sortCost, CostEstimate currentCost )
+    {
+        appendTraceString( "Cost of sorting is " + sortCost );
+        appendTraceString( "Total cost of non-sort-avoidance plan with sort cost added is " + currentCost );
+    }
+
+    public  void    traceNoBestPlan()
+    {
+        appendTraceString( "No best plan found." );
+    }
+
+    public  void    traceModifyingAccessPaths()
+    {
+        appendTraceString( "Modifying access paths using optimizer " + _optimizerID );
+    }
+
+    public  void    traceShortCircuiting( boolean timeExceeded, Optimizable thisOpt, int joinPosition )
+    {
+        String basis = (timeExceeded) ? "time exceeded" : "cost";
+        if ( thisOpt.getBestAccessPath().getCostEstimate() == null ) { basis = "no best plan found"; }
+        
+        appendTraceString( "Short circuiting based on " + basis + " at join position " + joinPosition );
+    }
+    
+    public  void    traceSkippingJoinOrder
+        ( int nextOptimizable, int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
+    {
+        appendTraceString
+            (
+             reportJoinOrder( "\n\nSkipping join order: ", true, nextOptimizable, joinPosition, proposedJoinOrder, assignedTableMap )
+             );
+    }
+
+    public  void    traceIllegalUserJoinOrder()
+    {
+        appendTraceString( "User specified join order is not legal." );
+    }
+    
+    public  void    traceUserJoinOrderOptimized()
+    {
+        appendTraceString( "User-specified join order has now been optimized." );
+    }
+
+    public  void    traceJoinOrderConsideration( int joinPosition, int[] proposedJoinOrder, JBitSet assignedTableMap )
+    {
+        appendTraceString
+            (
+             reportJoinOrder( "\n\nConsidering join order: ", false, 0, joinPosition, proposedJoinOrder, assignedTableMap )
+             );
+    }
+
+    public  void    traceCostWithoutSortAvoidance( CostEstimate currentCost )
+    {
+        appendTraceString( "Total cost of non-sort-avoidance plan is " + currentCost );
+    }
+
+    public  void    traceCostWithSortAvoidance( CostEstimate currentSortAvoidanceCost )
+    {
+        appendTraceString( "Total cost of sort avoidance plan is " + currentSortAvoidanceCost );
+    }
+
+    public  void    traceCurrentPlanAvoidsSort( CostEstimate bestCost, CostEstimate currentSortAvoidanceCost )
+    {
+        appendTraceString
+            (
+             "Current plan is a sort avoidance plan." + 
+             "\n\tBest cost is : " + bestCost +
+             "\n\tThis cost is : " + currentSortAvoidanceCost
+             );
+    }
+
+    public  void    traceCheapestPlanSoFar( int planType, CostEstimate currentCost )
+    {
+        appendTraceString( "This is the cheapest plan so far." );
+        appendTraceString
+            (
+             "Plan is a " +
+             (planType == Optimizer.NORMAL_PLAN ? "normal" : "sort avoidance") +
+             " plan."
+             );
+        appendTraceString( "Cost of cheapest plan is " + currentCost );
+    }
+
+    public  void    traceSortNeededForOrdering( int planType, RequiredRowOrdering requiredRowOrdering )
+    {
+        appendTraceString
+            (
+             "Sort needed for ordering: " + (planType != Optimizer.SORT_AVOIDANCE_PLAN) +
+             "\n\tRow ordering: " + requiredRowOrdering
+             );
+    }
+
+    public  void    traceRememberingBestJoinOrder( int joinPosition, int[] bestJoinOrder, JBitSet assignedTableMap )
+    {
+        appendTraceString
+            (
+             reportJoinOrder( "\n\nRemembering join order as best: ", false, 0, joinPosition, bestJoinOrder, assignedTableMap )
+             );
+    }
+
+    public  void    traceSkippingBecauseTooMuchMemory( int maxMemoryPerTable )
+    {
+        appendTraceString( "Skipping access path due to excess memory usage, maximum is " + maxMemoryPerTable );
+    }
+
+    public  void    traceCostOfNScans( int tableNumber, double rowCount, CostEstimate cost )
+    {
+        appendTraceString
+            (
+             "Cost of " + rowCount + " scans is: " + cost +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceSkipUnmaterializableHashJoin()
+    {
+        appendTraceString( "Skipping HASH JOIN because optimizable is not materializable" );
+    }
+
+    public  void    traceSkipHashJoinNoHashKeys()
+    {
+        appendTraceString( "Skipping HASH JOIN because there are no hash key columns" );
+    }
+
+    public  void    traceHashKeyColumns( int[] hashKeyColumns )
+    {
+        String  traceString = "# hash key columns = " + hashKeyColumns.length;
+        for (int index = 0; index < hashKeyColumns.length; index++)
+        {
+            traceString = 
+                "\n" + traceString + "hashKeyColumns[" + index +
+                "] = " + hashKeyColumns[index];
+        }
+
+        appendTraceString( traceString );
+    }
+
+    public  void    traceOptimizingJoinNode()
+    {
+        appendTraceString( "Calling optimizeIt() for join node" );
+    }
+
+    public  void    traceConsideringJoinStrategy( JoinStrategy js, int tableNumber )
+    {
+        appendTraceString
+            (
+             "\nConsidering join strategy " + js +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceRememberingBestAccessPath( AccessPath accessPath, int tableNumber, int planType )
+    {
+        appendTraceString
+            (
+             "Remembering access path " + accessPath +
+             " as truly the best for table " + tableNumber + 
+             " for plan type " + (planType == Optimizer.NORMAL_PLAN ? " normal " : "sort avoidance") +
+             "\n"
+             );
+    }
+
+    public  void    traceNoMoreConglomerates( int tableNumber )
+    {
+        appendTraceString( "No more conglomerates to consider for table " + tableNumber );
+    }
+
+    public  void    traceConsideringConglomerate( ConglomerateDescriptor cd, int tableNumber )
+    {
+        appendTraceString
+            (
+             "\nConsidering conglomerate " + reportConglomerateDescriptor( cd ) +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceScanningHeapWithUniqueKey()
+    {
+        appendTraceString( "Scanning heap, but we have a full match on a unique key." );
+    }
+
+    public  void    traceAddingUnorderedOptimizable( int predicateCount )
+    {
+        appendTraceString( "Adding unordered optimizable, # of predicates = " + predicateCount );
+    }
+
+    public  void    traceChangingAccessPathForTable( int tableNumber )
+    {
+        appendTraceString( "Changing access path for table " + tableNumber );
+    }
+
+    public  void    traceNoStartStopPosition()
+    {
+        appendTraceString( "Lock mode set to MODE_TABLE because no start or stop position" );
+    }
+
+    public  void    traceNonCoveringIndexCost( double cost, int tableNumber )
+    {
+        appendTraceString
+            (
+             "Index does not cover query - cost including base row fetch is: " + cost +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceConstantStartStopPositions()
+    {
+        appendTraceString( "Lock mode set to MODE_RECORD because all start and stop positions are constant" );
+    }
+
+    public  void    traceEstimatingCostOfConglomerate( ConglomerateDescriptor cd, int tableNumber )
+    {
+        String  cdString = reportConglomerateDescriptor( cd );
+        appendTraceString
+            (
+             "Estimating cost of conglomerate: " +
+             reportCostForTable( cdString, tableNumber )
+             );
+
+    }
+
+    public  void    traceLookingForSpecifiedIndex( String indexName, int tableNumber )
+    {
+        appendTraceString
+            (
+             "Looking for user-specified index: " + indexName +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceSingleMatchedRowCost( double cost, int tableNumber )
+    {
+        appendTraceString
+            (
+             "Guaranteed to match a single row - cost is: " + cost +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceCostIncludingExtra1stColumnSelectivity( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString
+            (
+             "Cost including extra first column selectivity is : " + cost +
+             " for table " + tableNumber
+             );
+    }
+
+    public  void    traceNextAccessPath( String baseTable, int predicateCount )
+    {
+        appendTraceString
+            (
+             "Calling nextAccessPath() for base table " + baseTable +
+             " with " + predicateCount + " predicates."
+             );
+    }
+
+    public  void    traceCostIncludingExtraStartStop( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString( reportCostIncluding( "start/stop", cost, tableNumber ) );
+    }
+
+    public  void    traceCostIncludingExtraQualifierSelectivity( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString( reportCostIncluding( "qualifier", cost, tableNumber ) );
+    }
+
+    public  void    traceCostIncludingExtraNonQualifierSelectivity( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString( reportCostIncluding( "non-qualifier", cost, tableNumber ) );
+    }
+
+    public  void    traceCostOfNoncoveringIndex( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString
+            (
+             "Index does not cover query: cost including row fetch is: " +
+             reportCostForTable( cost, tableNumber )
+             );
+    }
+
+    public  void    traceRememberingJoinStrategy( JoinStrategy joinStrategy, int tableNumber )
+    {
+        appendTraceString
+            (
+             "\nRemembering join strategy " + joinStrategy +
+             " as best for table " + tableNumber
+             );
+    }
+
+    public  void    traceRememberingBestAccessPathSubstring( AccessPath ap, int tableNumber )
+    {
+        appendTraceString( "in best access path" );
+    }
+
+    public  void    traceRememberingBestSortAvoidanceAccessPathSubstring( AccessPath ap, int tableNumber )
+    {
+        appendTraceString( "in best sort avoidance access path" );
+    }
+
+    public  void    traceRememberingBestUnknownAccessPathSubstring( AccessPath ap, int tableNumber )
+    {
+        appendTraceString( "in best unknown access path" );
+    }
+
+    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
+         )
+    {
+        appendTraceString
+            (
+             "Cost of conglomerate " + reportConglomerateDescriptor( cd ) +
+             " scan for table number " + tableNumber + " is : "
+             );
+        appendTraceString( costEstimate.toString() );
+        appendTraceString
+            (
+             "\tNumber of extra first column predicates is : " + numExtraFirstColumnPreds +
+             ", extra first column selectivity is : " + extraFirstColumnSelectivity
+             );
+        appendTraceString
+            (
+             "\tNumber of extra start/stop predicates is : " + numExtraStartStopPreds +
+             ", extra start/stop selectivity is : " + extraStartStopSelectivity
+             );
+        appendTraceString
+            (
+             "\tNumber of start/stop statistics predicates is : " + startStopPredCount +
+             ", statistics start/stop selectivity is : " + statStartStopSelectivity
+             );
+        appendTraceString
+            (
+             "\tNumber of extra qualifiers is : " + numExtraQualifiers +
+             ", extra qualifier selectivity is : " + extraQualifierSelectivity
+             );
+        appendTraceString
+            (
+             "\tNumber of extra non-qualifiers is : " + numExtraNonQualifiers +
+             ", extra non-qualifier selectivity is : " + extraNonQualifierSelectivity
+             );
+    }
+
+    public  void    traceCostIncludingCompositeSelectivityFromStats( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString( reportCostIncluding( "selectivity from statistics", cost, tableNumber ) );
+    }
+
+    public  void    traceCompositeSelectivityFromStatistics( double statCompositeSelectivity )
+    {
+        appendTraceString( "Selectivity from statistics found. It is " + statCompositeSelectivity );
+    }
+
+    public  void    traceCostIncludingStatsForIndex( CostEstimate cost, int tableNumber )
+    {
+        appendTraceString( reportCostIncluding( "statistics for index being considered", cost, tableNumber ) );
+    }
+
+    ////////////////////////////////////////////////////////////////////////
+    //
+    //	REPORTING MINIONS
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+	private String reportJoinOrder
+        (
+         String prefix,
+         boolean addJoinOrderNumber,
+         int joinOrderNumber,
+         int joinPosition,
+         int[] joinOrder,
+         JBitSet    assignedTableMap
+         )
+	{
+		StringBuffer joinOrderString = new StringBuffer();
+        joinOrderString.append(prefix);
+
+		for (int i = 0; i <= joinPosition; i++)
+		{
+			joinOrderString.append(" ").append(joinOrder[i]);
+		}
+		if (addJoinOrderNumber)
+		{
+			joinOrderString.append(" ").append(joinOrderNumber);
+		}
+
+        joinOrderString.append(" with assignedTableMap = ").append(assignedTableMap).append("\n\n");
+        return joinOrderString.toString();
+	}
+
+	private String reportConglomerateDescriptor( ConglomerateDescriptor cd )
+	{
+		if (SanityManager.DEBUG)
+		{
+			return cd.toString();
+		}
+
+		String		keyString = "";
+		String[]	columnNames = cd.getColumnNames();
+
+		if (cd.isIndex() && columnNames != null )
+		{
+			IndexRowGenerator irg = cd.getIndexDescriptor();
+
+			int[] keyColumns = irg.baseColumnPositions();
+
+			keyString = ", key columns = {" + columnNames[keyColumns[0] - 1];
+			for (int index = 1; index < keyColumns.length; index++)
+			{
+				keyString = keyString + ", " + columnNames[keyColumns[index] - 1];
+			}
+			keyString = keyString + "}";
+		}
+
+		return "CD: conglomerateNumber = " + cd.getConglomerateNumber() +
+			   " name = " + cd.getConglomerateName() +
+			   " uuid = " + cd.getUUID() +
+			   " indexable = " + cd.isIndex() +
+			   keyString;
+	}
+    
+	private String reportCostForTable( Object cost, int tableNumber )
+	{
+		return cost + " for table " + tableNumber;
+	}
+
+	private String reportCostIncluding( String selectivityType, CostEstimate cost, int tableNumber )
+	{
+		return
+			"Cost including extra " + selectivityType +
+			" start/stop selectivity is : " +
+			reportCostForTable( cost, tableNumber );
+	}
+
+    /** Append a string to the optimizer trace */
+    private void    appendTraceString( String traceString )
+    {
+		_lcc.appendOptimizerTraceOutput(traceString + "\n");
+    }
+
+}

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?rev=1478347&r1=1478346&r2=1478347&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Thu May  2 12:34:13 2013
@@ -281,9 +281,8 @@ public class FromBaseTable extends FromT
 		ConglomerateDescriptor currentConglomerateDescriptor =
 												ap.getConglomerateDescriptor();
 
-		optimizer.trace(Optimizer.CALLING_NEXT_ACCESS_PATH,
-					   ((predList == null) ? 0 : predList.size()),
-					   0, 0.0, getExposedName());
+        if ( optimizer.tracingIsOn() )
+        { optimizer.tracer().traceNextAccessPath( getExposedName(), ((predList == null) ? 0 : predList.size()) ); }
 
 		/*
 		** Remove the ordering of the current conglomerate descriptor,
@@ -313,8 +312,8 @@ public class FromBaseTable extends FromT
 			}
 			else
 			{
-				optimizer.trace(Optimizer.LOOKING_FOR_SPECIFIED_INDEX,
-								tableNumber, 0, 0.0, userSpecifiedIndexName);
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceLookingForSpecifiedIndex( userSpecifiedIndexName, tableNumber ); }
 
 				if (StringUtil.SQLToUpperCase(userSpecifiedIndexName).equals("NULL"))
 				{
@@ -421,13 +420,13 @@ public class FromBaseTable extends FromT
 
 		if (currentConglomerateDescriptor == null)
 		{
-			optimizer.trace(Optimizer.NO_MORE_CONGLOMERATES, tableNumber, 0, 0.0, null);
+            if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceNoMoreConglomerates( tableNumber ); }
 		}
 		else
 		{
 			currentConglomerateDescriptor.setColumnNames(columnNames);
-			optimizer.trace(Optimizer.CONSIDERING_CONGLOMERATE, tableNumber, 0, 0.0, 
-							currentConglomerateDescriptor);
+
+            if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceConsideringConglomerate( currentConglomerateDescriptor, tableNumber ); }
 		}
 
 		/*
@@ -445,16 +444,14 @@ public class FromBaseTable extends FromT
 				 */
 				if (! isOneRowResultSet(predList))
 				{
-					optimizer.trace(Optimizer.ADDING_UNORDERED_OPTIMIZABLE,
-									 ((predList == null) ? 0 : predList.size()), 
-									 0, 0.0, null);
+                    if ( optimizer.tracingIsOn() )
+                    { optimizer.tracer().traceAddingUnorderedOptimizable( ((predList == null) ? 0 : predList.size()) ); }
 
 					rowOrdering.addUnorderedOptimizable(this);
 				}
 				else
 				{
-					optimizer.trace(Optimizer.SCANNING_HEAP_FULL_MATCH_ON_UNIQUE_KEY,
-									 0, 0, 0.0, null);
+                    if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceScanningHeapWithUniqueKey(); }
 				}
 			}
 			else
@@ -958,8 +955,7 @@ public class FromBaseTable extends FromT
 		JoinStrategy currentJoinStrategy = 
 			currentAccessPath.getJoinStrategy();
 
-		optimizer.trace(Optimizer.ESTIMATING_COST_OF_CONGLOMERATE,
-						tableNumber, 0, 0.0, cd);
+        if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceEstimatingCostOfConglomerate( cd, tableNumber ); }
 
 		/* Get the uniqueness factory for later use (see below) */
 		double tableUniquenessFactor =
@@ -1000,8 +996,7 @@ public class FromBaseTable extends FromT
 										(FormatableBitSet) null,
 										0);
 
-			optimizer.trace(Optimizer.MATCH_SINGLE_ROW_COST,
-							tableNumber, 0, cost, null);
+            if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceSingleMatchedRowCost( cost, tableNumber ); }
 
 			costEstimate.setCost(cost, 1.0d, 1.0d);
 
@@ -1057,16 +1052,15 @@ public class FromBaseTable extends FromT
 				currentAccessPath.setLockMode(
 											TransactionController.MODE_RECORD);
 
-				optimizer.trace(Optimizer.ROW_LOCK_ALL_CONSTANT_START_STOP,
-								0, 0, 0.0, null);
+                if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceConstantStartStopPositions(); }
 			}
 			else
 			{
 				setLockingBasedOnThreshold(optimizer, costEstimate.rowCount());
 			}
 
-			optimizer.trace(Optimizer.COST_OF_N_SCANS, 
-							tableNumber, 0, outerCost.rowCount(), costEstimate);
+            if ( optimizer.tracingIsOn() )
+            { optimizer.tracer().traceCostOfNScans( tableNumber, outerCost.rowCount(), costEstimate ); }
 
 			/* Add in cost of fetching base row for non-covering index */
 			if (cd.isIndex() && ( ! isCoveringIndex(cd) ) )
@@ -1098,8 +1092,7 @@ public class FromBaseTable extends FromT
 				costEstimate.setEstimatedCost(
 								costEstimate.getEstimatedCost() + cost);
 
-				optimizer.trace(Optimizer.NON_COVERING_INDEX_COST,
-								tableNumber, 0, cost, null);
+                if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceNonCoveringIndexCost( cost, tableNumber ); }
 			}
 		}
 		else
@@ -1533,25 +1526,25 @@ public class FromBaseTable extends FromT
 				}
 			}
 
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN1,
-							tableNumber, 0, 0.0, cd);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN2,
-							tableNumber, 0, 0.0, costEstimate);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN3,
-							numExtraFirstColumnPreds, 0, 
-							extraFirstColumnSelectivity, null);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN4,
-							numExtraStartStopPreds, 0, 
-							extraStartStopSelectivity, null);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN7,
-							startStopPredCount, 0,
-							statStartStopSelectivity, null);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN5,
-							numExtraQualifiers, 0, 
-							extraQualifierSelectivity, null);
-			optimizer.trace(Optimizer.COST_OF_CONGLOMERATE_SCAN6,
-							numExtraNonQualifiers, 0, 
-							extraNonQualifierSelectivity, null);
+            if ( optimizer.tracingIsOn() )
+            {
+                optimizer.tracer().traceCostOfConglomerateScan
+                    (
+                     tableNumber,
+                     cd,
+                     costEstimate,
+                     numExtraFirstColumnPreds,
+                     extraFirstColumnSelectivity,
+                     numExtraStartStopPreds,
+                     extraStartStopSelectivity,
+                     startStopPredCount,
+                     statStartStopSelectivity,
+                     numExtraQualifiers,
+                     extraQualifierSelectivity,
+                     numExtraNonQualifiers,
+                     extraNonQualifierSelectivity
+                     );
+            }
 
 			/* initial row count is the row count without applying
 			   any predicates-- we use this at the end of the routine
@@ -1577,9 +1570,9 @@ public class FromBaseTable extends FromT
 							 costEstimate.rowCount() * statStartStopSelectivity,
 							 costEstimate.singleScanRowCount() *
 							 statStartStopSelectivity);
-				optimizer.trace(Optimizer.COST_INCLUDING_STATS_FOR_INDEX,
-								tableNumber, 0, 0.0, costEstimate);
-
+                
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCostIncludingStatsForIndex( costEstimate, tableNumber ); }
 			}
 			else
 			{
@@ -1598,9 +1591,9 @@ public class FromBaseTable extends FromT
 												  oneRowResultSetForSomeConglom),
 						 costEstimate.rowCount() * extraFirstColumnSelectivity,
 						 costEstimate.singleScanRowCount() * extraFirstColumnSelectivity);
-					
-					optimizer.trace(Optimizer.COST_INCLUDING_EXTRA_1ST_COL_SELECTIVITY,
-									tableNumber, 0, 0.0, costEstimate);
+
+                    if ( optimizer.tracingIsOn() )
+                    { optimizer.tracer().traceCostIncludingExtra1stColumnSelectivity( costEstimate, tableNumber ); }
 				}
 
 				/* Factor in the extra start/stop selectivity (see comment above).
@@ -1614,8 +1607,7 @@ public class FromBaseTable extends FromT
 						costEstimate.rowCount() * extraStartStopSelectivity,
 						costEstimate.singleScanRowCount() * extraStartStopSelectivity);
 
-					optimizer.trace(Optimizer.COST_INCLUDING_EXTRA_START_STOP,
-									tableNumber, 0, 0.0, costEstimate);
+                    if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceCostIncludingExtraStartStop( costEstimate, tableNumber ); }
 				}
 			}
 
@@ -1661,8 +1653,7 @@ public class FromBaseTable extends FromT
 				currentAccessPath.setLockMode(
 											TransactionController.MODE_TABLE);
 
-				optimizer.trace(Optimizer.TABLE_LOCK_NO_START_STOP,
-							    0, 0, 0.0, null);
+                if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceNoStartStopPosition(); }
 			}
 			else
 			{
@@ -1771,8 +1762,8 @@ public class FromBaseTable extends FromT
 				costEstimate.setEstimatedCost(
 								costEstimate.getEstimatedCost() + cost);
 
-				optimizer.trace(Optimizer.COST_OF_NONCOVERING_INDEX,
-								tableNumber, 0, 0.0, costEstimate);
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCostOfNoncoveringIndex( costEstimate, tableNumber ); }
 			}
 
 			/* Factor in the extra qualifier selectivity (see comment above).
@@ -1786,8 +1777,8 @@ public class FromBaseTable extends FromT
 						costEstimate.rowCount() * extraQualifierSelectivity,
 						costEstimate.singleScanRowCount() * extraQualifierSelectivity);
 
-				optimizer.trace(Optimizer.COST_INCLUDING_EXTRA_QUALIFIER_SELECTIVITY,
-								tableNumber, 0, 0.0, costEstimate);
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCostIncludingExtraQualifierSelectivity( costEstimate, tableNumber ); }
 			}
 
 			singleScanRowCount = costEstimate.singleScanRowCount();
@@ -1912,8 +1903,8 @@ public class FromBaseTable extends FromT
 				costEstimate.singleScanRowCount());
 
 
-			optimizer.trace(Optimizer.COST_OF_N_SCANS, 
-							tableNumber, 0, outerCost.rowCount(), costEstimate);
+            if ( optimizer.tracingIsOn() )
+            { optimizer.tracer().traceCostOfNScans( tableNumber, outerCost.rowCount(), costEstimate ); }
 
 			/*
 			** Now figure in the cost of the non-qualifier predicates.
@@ -1934,8 +1925,8 @@ public class FromBaseTable extends FromT
 			if (rc != -1) // changed
 			{
 				costEstimate.setCost(costEstimate.getEstimatedCost(), rc, src);
-				optimizer.trace(Optimizer.COST_INCLUDING_EXTRA_NONQUALIFIER_SELECTIVITY,
-								tableNumber, 0, 0.0, costEstimate);
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCostIncludingExtraNonQualifierSelectivity( costEstimate, tableNumber ); }
 			}
 			
 		recomputeRowCount:
@@ -1949,9 +1940,8 @@ public class FromBaseTable extends FromT
 				*/
 
 				double compositeStatRC = initialRowCount * statCompositeSelectivity;
-				optimizer.trace(Optimizer.COMPOSITE_SEL_FROM_STATS,
-								0, 0, statCompositeSelectivity, null);
-
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCompositeSelectivityFromStatistics( statCompositeSelectivity ); }
 
 				if (tableUniquenessFactor > 0.0)
 				{
@@ -1980,9 +1970,9 @@ public class FromBaseTable extends FromT
 									 (existsBaseTable) ? 
 									 1 : 
 									 compositeStatRC / outerCost.rowCount());
-				
-				optimizer.trace(Optimizer.COST_INCLUDING_COMPOSITE_SEL_FROM_STATS,
-								tableNumber, 0, 0.0, costEstimate);
+
+                if ( optimizer.tracingIsOn() )
+                { optimizer.tracer().traceCostIncludingCompositeSelectivityFromStats( costEstimate, tableNumber ); }
 			}
 		}
 
@@ -2884,8 +2874,7 @@ public class FromBaseTable extends FromT
 		JoinStrategy trulyTheBestJoinStrategy = ap.getJoinStrategy();
 		Optimizer optimizer = ap.getOptimizer();
 
-		optimizer.trace(Optimizer.CHANGING_ACCESS_PATH_FOR_TABLE,
-						tableNumber, 0, 0.0, null);
+        if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceChangingAccessPathForTable( tableNumber ); }
 
 		if (SanityManager.DEBUG)
 		{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromTable.java?rev=1478347&r1=1478346&r2=1478347&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromTable.java Thu May  2 12:34:13 2013
@@ -245,8 +245,7 @@ abstract class FromTable extends ResultS
 
 			found = true;
 
-			optimizer.trace(Optimizer.CONSIDERING_JOIN_STRATEGY, tableNumber, 0, 0.0,
-							ap.getJoinStrategy());
+            if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceConsideringJoinStrategy( ap.getJoinStrategy(), tableNumber ); }
 		}
 
 		/*
@@ -307,18 +306,18 @@ abstract class FromTable extends ResultS
 
 		ap.setJoinStrategy(getCurrentAccessPath().getJoinStrategy());
 
-		optimizer.trace(Optimizer.REMEMBERING_JOIN_STRATEGY, tableNumber, 0, 0.0,
-			  getCurrentAccessPath().getJoinStrategy());
+        if ( optimizer.tracingIsOn() )
+        { optimizer.tracer().traceRememberingJoinStrategy( getCurrentAccessPath().getJoinStrategy(), tableNumber ); }
 
 		if (ap == bestAccessPath)
 		{
-			optimizer.trace(Optimizer.REMEMBERING_BEST_ACCESS_PATH_SUBSTRING, 
-							tableNumber, 0, 0.0, ap);
+            if ( optimizer.tracingIsOn() )
+            { optimizer.tracer().traceRememberingBestAccessPathSubstring( ap, tableNumber ); }
 		}
 		else if (ap == bestSortAvoidancePath)
 		{
-			optimizer.trace(Optimizer.REMEMBERING_BEST_SORT_AVOIDANCE_ACCESS_PATH_SUBSTRING, 
-							tableNumber, 0, 0.0, ap);
+            if ( optimizer.tracingIsOn() )
+            { optimizer.tracer().traceRememberingBestSortAvoidanceAccessPathSubstring( ap, tableNumber ); }
 		}
 		else
 		{
@@ -331,8 +330,8 @@ abstract class FromTable extends ResultS
 					"unknown access path type");
 			}
 			 */
-			optimizer.trace(Optimizer.REMEMBERING_BEST_UNKNOWN_ACCESS_PATH_SUBSTRING, 
-							tableNumber, 0, 0.0, ap);
+            if ( optimizer.tracingIsOn() )
+            { optimizer.tracer().traceRememberingBestUnknownAccessPathSubstring( ap, tableNumber ); }
 		}
 	}
 
@@ -635,8 +634,8 @@ abstract class FromTable extends ResultS
 
 		setCostEstimate(bestPath.getCostEstimate());
 
-		bestPath.getOptimizer().trace(Optimizer.REMEMBERING_BEST_ACCESS_PATH,
-							tableNumber, planType, 0.0, bestPath);
+        if ( bestPath.getOptimizer().tracingIsOn() )
+        { bestPath.getOptimizer().tracer().traceRememberingBestAccessPath( bestPath, tableNumber, planType ); }
 	}
 
 	/** @see Optimizable#startOptimizing */

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=1478347&r1=1478346&r2=1478347&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 Thu May  2 12:34:13 2013
@@ -37,6 +37,7 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.TransactionController;
 
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
+import org.apache.derby.iapi.services.io.ArrayUtil;
 
 import org.apache.derby.iapi.error.StandardException;
 
@@ -79,9 +80,7 @@ public class HashJoinStrategy extends Ba
 		 */
 		if (! innerTable.isMaterializable())
 		{
-
-			optimizer.trace(Optimizer.HJ_SKIP_NOT_MATERIALIZABLE, 0, 0, 0.0,
-							null);
+            if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceSkipUnmaterializableHashJoin(); }
 			return false;
 		}
 
@@ -163,14 +162,17 @@ public class HashJoinStrategy extends Ba
 
 		if (SanityManager.DEBUG)
 		{
-			if (hashKeyColumns == null)
-			{
-				optimizer.trace(Optimizer.HJ_SKIP_NO_JOIN_COLUMNS, 0, 0, 0.0, null);
-			}
-			else
-			{
-				optimizer.trace(Optimizer.HJ_HASH_KEY_COLUMNS, 0, 0, 0.0, hashKeyColumns);
-			}
+            if ( optimizer.tracingIsOn() )
+            {
+                if (hashKeyColumns == null)
+                {
+                    optimizer.tracer().traceSkipHashJoinNoHashKeys();
+                }
+                else
+                {
+                    optimizer.tracer().traceHashKeyColumns( ArrayUtil.copy( hashKeyColumns ) );
+                }
+            }
 		}
 
 		if (hashKeyColumns == null)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java?rev=1478347&r1=1478346&r2=1478347&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/JoinNode.java Thu May  2 12:34:13 2013
@@ -170,7 +170,7 @@ public class JoinNode extends TableOpera
 							RowOrdering rowOrdering)
 			throws StandardException
 	{
-		optimizer.trace(Optimizer.CALLING_ON_JOIN_NODE, 0, 0, 0.0, null);
+        if ( optimizer.tracingIsOn() ) { optimizer.tracer().traceOptimizingJoinNode(); }
 
 		// It's possible that a call to optimize the left/right will cause
 		// a new "truly the best" plan to be stored in the underlying base

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=1478347&r1=1478346&r2=1478347&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 Thu May  2 12:34:13 2013
@@ -37,7 +37,6 @@ import org.apache.derby.iapi.sql.conn.La
 
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 
 /**
  * This is the Level 2 Optimizer.
@@ -45,8 +44,6 @@ import org.apache.derby.iapi.sql.diction
 
 public class Level2OptimizerImpl extends OptimizerImpl
 {
-	private LanguageConnectionContext lcc;
-
 	Level2OptimizerImpl(OptimizableList optimizableList, 
 				  OptimizablePredicateList predicateList,
 				  DataDictionary dDictionary,
@@ -72,10 +69,7 @@ public class Level2OptimizerImpl extends
 		this.lcc = lcc;
 
 		// Optimization started
-		if (optimizerTrace)
-		{
-			trace(STARTED, 0, 0, 0.0, null);
-		}
+		if (optimizerTrace) { tracer().traceStart( timeOptimizationStarted ); }
 	}
 
 	/** @see Optimizer#getLevel */
@@ -97,442 +91,11 @@ public class Level2OptimizerImpl extends
 		return new Level2CostEstimateImpl(theCost, theRowCount, theSingleScanRowCount);
 	}
 
-	// Optimzer trace
-	public void trace(int traceFlag, int intParam1, int intParam2,
-					  double doubleParam, Object objectParam1)
-	{
-		ConglomerateDescriptor cd;
-		String cdString;
-		String traceString = null;
-
-		// We can get called from outside optimizer when tracing is off
-		if (!optimizerTrace)
-		{
-			return;
-		}
-
-		switch (traceFlag)
-		{
-			case STARTED:
-				traceString = 
-					"Optimization started at time " +
-					timeOptimizationStarted +
-					" using optimizer " + this.hashCode();
-				break;
-
-			case TIME_EXCEEDED:
-				traceString =
-					"Optimization time exceeded at time " +
-							currentTime + "\n" + bestCost();
-				break;
-
-			case NO_TABLES:
-				traceString = "No tables to optimize.";
-				break;
-
-			case COMPLETE_JOIN_ORDER:
-				traceString = "We have a complete join order.";
-				break;
-
-			case COST_OF_SORTING:
-				traceString = 
-					"Cost of sorting is " + sortCost;
-				break;
-
-			case NO_BEST_PLAN:
-				traceString =
-					"No best plan found.";
-				break;
-
-			case MODIFYING_ACCESS_PATHS:
-				traceString = 
-					"Modifying access paths using optimizer " + this.hashCode();
-				break;
-
-			case SHORT_CIRCUITING:
-				String basis = (timeExceeded) ? "time exceeded" : "cost";
-				Optimizable thisOpt =
-					optimizableList.getOptimizable(
-										proposedJoinOrder[joinPosition]);
-				if (thisOpt.getBestAccessPath().getCostEstimate() == null)
-					basis = "no best plan found";
-				traceString = 
-					"Short circuiting based on " + basis +
-					" at join position " + joinPosition;
-				break;
-
-			case SKIPPING_JOIN_ORDER:
-				traceString = buildJoinOrder("\n\nSkipping join order: ", true, intParam1,
-											 proposedJoinOrder);
-				break;
-
-			case ILLEGAL_USER_JOIN_ORDER:
-				traceString =
-					"User specified join order is not legal.";
-				break;
-
-			case USER_JOIN_ORDER_OPTIMIZED:
-				traceString =
-					"User-specified join order has now been optimized.";
-				break;
-
-			case CONSIDERING_JOIN_ORDER:
-				traceString = buildJoinOrder("\n\nConsidering join order: ", false, intParam1,
-											 proposedJoinOrder);
-				break;
-
-			case TOTAL_COST_NON_SA_PLAN:
-				traceString =
-					"Total cost of non-sort-avoidance plan is " +
-						currentCost;
-				break;
-
-			case TOTAL_COST_SA_PLAN:
-				traceString =
-					"Total cost of sort avoidance plan is " +
-						currentSortAvoidanceCost;
-				break;
-
-			case TOTAL_COST_WITH_SORTING:
-				traceString = 
-					"Total cost of non-sort-avoidance plan with sort cost added is " + currentCost;
-				break;
-
-			case CURRENT_PLAN_IS_SA_PLAN:
-				traceString = 
-					"Current plan is a sort avoidance plan." + 
-					"\n\tBest cost is : " + bestCost +
-					"\n\tThis cost is : " + currentSortAvoidanceCost;
-				break;
-
-			case CHEAPEST_PLAN_SO_FAR:
-				traceString = 
-					"This is the cheapest plan so far.";
-				break;
-
-			case PLAN_TYPE:
-				traceString = 
-					"Plan is a " +
-					(intParam1 == Optimizer.NORMAL_PLAN ?
-						"normal" : "sort avoidance") +
-					" plan.";
-				break;
-
-			case COST_OF_CHEAPEST_PLAN_SO_FAR:
-				traceString = 
-					"Cost of cheapest plan is " + currentCost;
-				break;
-
-			case SORT_NEEDED_FOR_ORDERING:
-				traceString = 
-					"Sort needed for ordering: " + 
-						(intParam1 != Optimizer.SORT_AVOIDANCE_PLAN) +
-						"\n\tRow ordering: " +
-						requiredRowOrdering;
-				break;
-
-			case REMEMBERING_BEST_JOIN_ORDER:
-				traceString = buildJoinOrder("\n\nRemembering join order as best: ", false, intParam1,
-											 bestJoinOrder);
-				break;
-
-			case SKIPPING_DUE_TO_EXCESS_MEMORY:
-				traceString = 
-					"Skipping access path due to excess memory usage, maximum is " +
-					maxMemoryPerTable;
-				break;
-
-			case COST_OF_N_SCANS:
-				traceString =
-						"Cost of " + doubleParam +
-						" scans is: " +
-						objectParam1 +
-						" for table " +
-						intParam1;
-				break;
-
-			case HJ_SKIP_NOT_MATERIALIZABLE:
-				traceString = "Skipping HASH JOIN because optimizable is not materializable";
-				break;
-
-			case HJ_SKIP_NO_JOIN_COLUMNS:
-				traceString = "Skipping HASH JOIN because there are no hash key columns";
-				break;
-
-			case HJ_HASH_KEY_COLUMNS:
-				int[] hashKeyColumns = (int []) objectParam1;
-				traceString = "# hash key columns = " + hashKeyColumns.length;
-				for (int index = 0; index < hashKeyColumns.length; index++)
-				{
-					traceString = 
-						"\n" + traceString + "hashKeyColumns[" + index +
-						"] = " + hashKeyColumns[index];
-				}
-				break;
-
-			case CALLING_ON_JOIN_NODE:
-				traceString = "Calling optimizeIt() for join node";
-				break;
-
-			case CONSIDERING_JOIN_STRATEGY:
-				JoinStrategy js = (JoinStrategy) objectParam1;
-				traceString = 
-					"\nConsidering join strategy " +
-					js + " for table " + intParam1;
-				break;
-
-			case REMEMBERING_BEST_ACCESS_PATH:
-				traceString = 
-					"Remembering access path " +
-					objectParam1 +
-					" as truly the best for table " +
-					intParam1 + 
-					" for plan type " +
-					(intParam2 == Optimizer.NORMAL_PLAN ?
-										" normal " : "sort avoidance") +
-					"\n";
-				break;
-
-			case NO_MORE_CONGLOMERATES:
-				traceString =
-					"No more conglomerates to consider for table " +
-					intParam1;
-				break;
-
-			case CONSIDERING_CONGLOMERATE:
-				cd = (ConglomerateDescriptor) objectParam1;
-				cdString = dumpConglomerateDescriptor(cd);
-				traceString =
-					"\nConsidering conglomerate " +
-					cdString +
-					" for table " +
-					intParam1;
-				break;
-
-			case SCANNING_HEAP_FULL_MATCH_ON_UNIQUE_KEY:
-				traceString = "Scanning heap, but we have a full match on a unique key.";
-				break;
-
-			case ADDING_UNORDERED_OPTIMIZABLE:
-				traceString = "Adding unordered optimizable, # of predicates = " + intParam1;
-				break;
-
-			case CHANGING_ACCESS_PATH_FOR_TABLE:
-				traceString = "Changing access path for table " + intParam1;
-				break;
-
-			case TABLE_LOCK_NO_START_STOP:
-				traceString = "Lock mode set to MODE_TABLE because no start or stop position";
-				break;
-
-			case NON_COVERING_INDEX_COST:
-				traceString = 
-					"Index does not cover query - cost including base row fetch is: " +
-					doubleParam +
-					" for table " + intParam1;
-				break;
-
-			case ROW_LOCK_ALL_CONSTANT_START_STOP:
-				traceString = 
-					"Lock mode set to MODE_RECORD because all start and stop positions are constant";
-				break;
-
-			case ESTIMATING_COST_OF_CONGLOMERATE:
-				cd = (ConglomerateDescriptor) objectParam1;
-				cdString = dumpConglomerateDescriptor(cd);
-				traceString =
-					"Estimating cost of conglomerate: " +
-					costForTable(cdString, intParam1);
-				break;
-				
-			case LOOKING_FOR_SPECIFIED_INDEX:
-				traceString = 
-					"Looking for user-specified index: " +
-					objectParam1 + " for table " +
-					intParam1;
-				break;
-
-			case MATCH_SINGLE_ROW_COST:
-				traceString =
-					"Guaranteed to match a single row - cost is: " +
-					doubleParam + " for table " + intParam1;
-				break;
-
-			case COST_INCLUDING_EXTRA_1ST_COL_SELECTIVITY:
-				traceString = costIncluding(
-								"1st column", objectParam1, intParam1);
-				traceString =
-					"Cost including extra first column selectivity is : " +
-					objectParam1 + " for table " + intParam1;
-				break;
-
-			case CALLING_NEXT_ACCESS_PATH:
-				traceString =
-					"Calling nextAccessPath() for base table " +
-					objectParam1 + " with " + intParam1 + " predicates.";
-				break;
-
-			case TABLE_LOCK_OVER_THRESHOLD:
-				traceString = lockModeThreshold("MODE_TABLE", "greater",
-												doubleParam, intParam1);
-				break;
-
-			case ROW_LOCK_UNDER_THRESHOLD:
-				traceString = lockModeThreshold("MODE_RECORD", "less",
-												doubleParam, intParam1);
-				break;
-
-			case COST_INCLUDING_EXTRA_START_STOP:
-				traceString = costIncluding(
-								"start/stop", objectParam1, intParam1);
-				break;
-
-			case COST_INCLUDING_EXTRA_QUALIFIER_SELECTIVITY:
-				traceString = costIncluding(
-								"qualifier", objectParam1, intParam1);
-				break;
-
-			case COST_INCLUDING_EXTRA_NONQUALIFIER_SELECTIVITY:
-				traceString = costIncluding(
-								"non-qualifier", objectParam1, intParam1);
-				break;
-
-   		    case COST_INCLUDING_COMPOSITE_SEL_FROM_STATS:
-				traceString = costIncluding("selectivity from statistics",
-											objectParam1, intParam1);
-				break;
-
-			case COST_INCLUDING_STATS_FOR_INDEX:
-				traceString = costIncluding("statistics for index being considered", 
-											objectParam1, intParam1);
-				break;
-		    case COMPOSITE_SEL_FROM_STATS:
-				traceString = "Selectivity from statistics found. It is " +
-					doubleParam;
-				break;
-
-			case COST_OF_NONCOVERING_INDEX:
-				traceString =
-					"Index does not cover query: cost including row fetch is: " +
-					costForTable(objectParam1, intParam1);
-				break;
-
-			case REMEMBERING_JOIN_STRATEGY:
-				traceString =
-					"\nRemembering join strategy " + objectParam1 +
-					" as best for table " + intParam1;
-				break;
-
-			case REMEMBERING_BEST_ACCESS_PATH_SUBSTRING:
-				traceString =
-					"in best access path";
-				break;
-
-			case REMEMBERING_BEST_SORT_AVOIDANCE_ACCESS_PATH_SUBSTRING:
-				traceString =
-					"in best sort avoidance access path";
-				break;
-
-			case REMEMBERING_BEST_UNKNOWN_ACCESS_PATH_SUBSTRING:
-				traceString =
-					"in best unknown access path";
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN1:
-				cd = (ConglomerateDescriptor) objectParam1;
-				cdString = dumpConglomerateDescriptor(cd);
-				traceString =
-					"Cost of conglomerate " +
-					cdString +
-					" scan for table number " +
-					intParam1 + " is : ";
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN2:
-				traceString =
-					objectParam1.toString();
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN3:
-				traceString =
-					"\tNumber of extra first column predicates is : " +
-					intParam1 +
-					", extra first column selectivity is : " +
-					doubleParam;
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN4:
-				traceString =
-					"\tNumber of extra start/stop predicates is : " +
-					intParam1 +
-					", extra start/stop selectivity is : " +
-					doubleParam;
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN5:
-				traceString =
-					"\tNumber of extra qualifiers is : " +
-					intParam1 +
-					", extra qualifier selectivity is : " +
-					doubleParam;
-				break;
-
-			case COST_OF_CONGLOMERATE_SCAN6:
-				traceString =
-					"\tNumber of extra non-qualifiers is : " +
-					intParam1 +
-					", extra non-qualifier selectivity is : " +
-					doubleParam;
-				break;
-
-		    case COST_OF_CONGLOMERATE_SCAN7:
-				traceString = 
-					"\tNumber of start/stop statistics predicates is : " +
-					intParam1 + 
-					", statistics start/stop selectivity is : " +
-					doubleParam;
-				break;
-		}
-		if (SanityManager.DEBUG)
-		{
-			if (traceString == null)
-			{
-				SanityManager.THROWASSERT(
-					"traceString expected to be non-null");
-			}
-		}
-		lcc.appendOptimizerTraceOutput(traceString + "\n");
-	}
-
-	private String costForTable(Object cost, int tableNumber)
-	{
-		return cost + " for table " + tableNumber;
-	}
-
 	private String bestCost()
 	{
 		return "Best cost = " + bestCost + "\n";
 	}
 
-	private String buildJoinOrder(String prefix, boolean addJoinOrderNumber,
-								  int joinOrderNumber, int[] joinOrder)
-	{
-		StringBuffer joinOrderString = new StringBuffer();
-        joinOrderString.append(prefix);
-
-		for (int i = 0; i <= joinPosition; i++)
-		{
-			joinOrderString.append(" ").append(joinOrder[i]);
-		}
-		if (addJoinOrderNumber)
-		{
-			joinOrderString.append(" ").append(joinOrderNumber);
-		}
-
-        joinOrderString.append(" with assignedTableMap = ").append(assignedTableMap).append("\n\n");
-        return joinOrderString.toString();
-	}
-
 	private String lockModeThreshold(
 						String lockMode, String relop,
 						double rowCount, int threshold)
@@ -543,43 +106,5 @@ public class Level2OptimizerImpl extends
 			" " + relop + " than threshold of " + threshold;
 	}
 
-	private String costIncluding(
-					String selectivityType, Object objectParam1, int intParam1)
-	{
-		return
-			"Cost including extra " + selectivityType +
-			" start/stop selectivity is : " +
-			costForTable(objectParam1, intParam1);
-	}
 
-	private String dumpConglomerateDescriptor(ConglomerateDescriptor cd)
-	{
-		if (SanityManager.DEBUG)
-		{
-			return cd.toString();
-		}
-
-		String		keyString = "";
-		String[]	columnNames = cd.getColumnNames();
-
-		if (cd.isIndex() && columnNames != null )
-		{
-			IndexRowGenerator irg = cd.getIndexDescriptor();
-
-			int[] keyColumns = irg.baseColumnPositions();
-
-			keyString = ", key columns = {" + columnNames[keyColumns[0] - 1];
-			for (int index = 1; index < keyColumns.length; index++)
-			{
-				keyString = keyString + ", " + columnNames[keyColumns[index] - 1];
-			}
-			keyString = keyString + "}";
-		}
-
-		return "CD: conglomerateNumber = " + cd.getConglomerateNumber() +
-			   " name = " + cd.getConglomerateName() +
-			   " uuid = " + cd.getUUID() +
-			   " indexable = " + cd.isIndex() +
-			   keyString;
-	}
 }

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=1478347&r1=1478346&r2=1478347&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 Thu May  2 12:34:13 2013
@@ -147,8 +147,8 @@ public class NestedLoopJoinStrategy exte
 							 CostEstimate costEstimate) {
 		costEstimate.multiply(outerCost.rowCount(), costEstimate);
 
-		optimizer.trace(Optimizer.COST_OF_N_SCANS, innerTable.getTableNumber(), 0, outerCost.rowCount(),
-						costEstimate);
+        if ( optimizer.tracingIsOn() )
+        { optimizer.tracer().traceCostOfNScans( innerTable.getTableNumber(), outerCost.rowCount(), costEstimate ); }
 	}
 
 	/** @see JoinStrategy#maxCapacity */

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=1478347&r1=1478346&r2=1478347&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 Thu May  2 12:34:13 2013
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.sql.compile;
 
+import org.apache.derby.iapi.services.io.ArrayUtil;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 
 import org.apache.derby.iapi.error.StandardException;
@@ -31,14 +32,17 @@ import org.apache.derby.iapi.sql.compile
 import org.apache.derby.iapi.sql.compile.OptimizablePredicate;
 import org.apache.derby.iapi.sql.compile.OptimizablePredicateList;
 import org.apache.derby.iapi.sql.compile.Optimizer;
+import org.apache.derby.iapi.sql.compile.OptTrace;
 import org.apache.derby.iapi.sql.compile.CostEstimate;
 import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
 import org.apache.derby.iapi.sql.compile.RowOrdering;
 import org.apache.derby.iapi.sql.compile.AccessPath;
 
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 
 import org.apache.derby.iapi.reference.SQLState;
@@ -68,6 +72,7 @@ import java.util.HashMap;
 
 public class OptimizerImpl implements Optimizer 
 {
+	protected LanguageConnectionContext lcc;
 
 	DataDictionary			 dDictionary;
 	/* The number of tables in the query as a whole.  (Size of bit maps.) */
@@ -133,6 +138,7 @@ public class OptimizerImpl implements Op
 	// optimizer trace
 	protected boolean optimizerTrace;
 	protected boolean optimizerTraceHtml;
+    private OptTrace    _tracer;
 
 	// max memory use per table
 	protected int maxMemoryPerTable;
@@ -353,6 +359,8 @@ public class OptimizerImpl implements Op
 		desiredJoinOrderFound = false;
 	}
 
+    public  boolean tracingIsOn() { return optimizerTrace; }
+
     public int getMaxMemoryPerTable()
     {
         return maxMemoryPerTable;
@@ -369,10 +377,7 @@ public class OptimizerImpl implements Op
 		/* Don't get any permutations if there is nothing to optimize */
 		if (numOptimizables < 1)
 		{
-			if (optimizerTrace)
-			{
-				trace(NO_TABLES, 0, 0, 0.0, null);
-			}
+			if (optimizerTrace) { tracer().traceVacuous(); }
 
 			endOfRoundCleanup();
 			return false;
@@ -402,10 +407,7 @@ public class OptimizerImpl implements Op
 			currentTime = System.currentTimeMillis();
 			timeExceeded = (currentTime - timeOptimizationStarted) > timeLimit;
 
-			if (optimizerTrace && timeExceeded)
-			{
-				trace(TIME_EXCEEDED, 0, 0, 0.0, null);
-			}
+			if (optimizerTrace && timeExceeded) { tracer().traceTimeout( currentTime, bestCost ); }
 		}
 
 		if (bestCost.isUninitialized() && foundABestPlan &&
@@ -552,18 +554,6 @@ public class OptimizerImpl implements Op
 		}
 		else
 		{
-			if (optimizerTrace)
- 			{
-				/*
-				** Not considered short-circuiting if all slots in join
-				** order are taken.
-				*/
-				if (joinPosition < (numOptimizables - 1))
-				{
-					trace(SHORT_CIRCUITING, 0, 0, 0.0, null);
-				}
-			}
-
 			// If we short-circuited the current join order then we need
 			// to make sure that, when we start pulling optimizables to find
 			// a new join order, we reload the best plans for those
@@ -572,7 +562,17 @@ public class OptimizerImpl implements Op
 			// was part of a short-circuited (and thus rejected) join
 			// order.
 			if (joinPosition < (numOptimizables - 1))
-				reloadBestPlan = true;
+            {
+                if (optimizerTrace)
+                {
+                    tracer().traceShortCircuiting
+                        (
+                         timeExceeded, optimizableList.getOptimizable( proposedJoinOrder[joinPosition] ), joinPosition
+                         );
+                }
+
+                reloadBestPlan = true;
+            }
 		}
 
 		if (permuteState == JUMPING && !joinPosAdvanced && joinPosition >= 0)
@@ -793,19 +793,17 @@ public class OptimizerImpl implements Op
 						!joinOrderMeetsDependencies(nextOptimizable))
 					{
 						if (optimizerTrace)
-						{
-							trace(SKIPPING_JOIN_ORDER, nextOptimizable, 0, 0.0, null);
-						}
+                        {
+                            tracer().traceSkippingJoinOrder
+                                ( nextOptimizable, joinPosition, ArrayUtil.copy( proposedJoinOrder ), (JBitSet) assignedTableMap.clone() );
+                        }
 
 						/*
 						** If this is a user specified join order then it is illegal.
 						*/
 						if ( ! optimizableList.optimizeJoinOrder())
 						{
-							if (optimizerTrace)
-							{
-								trace(ILLEGAL_USER_JOIN_ORDER, 0, 0, 0.0, null);
-							}
+							if (optimizerTrace) { tracer().traceIllegalUserJoinOrder(); }
 
 							throw StandardException.newException(
 								SQLState.LANG_ILLEGAL_FORCED_JOIN_ORDER);
@@ -830,18 +828,12 @@ public class OptimizerImpl implements Op
 					// Verify that the user specified a legal join order
 					if ( ! optimizableList.legalJoinOrder(numTablesInQuery))
 					{
-						if (optimizerTrace)
-						{
-							trace(ILLEGAL_USER_JOIN_ORDER, 0, 0, 0.0, null);
-						}
+						if (optimizerTrace)  { tracer().traceIllegalUserJoinOrder(); }
 
 						throw StandardException.newException(SQLState.LANG_ILLEGAL_FORCED_JOIN_ORDER);
 					}
 
-					if (optimizerTrace)
-					{
-						trace(USER_JOIN_ORDER_OPTIMIZED, 0, 0, 0.0, null);
-					}
+					if (optimizerTrace) { tracer().traceUserJoinOrderOptimized(); }
 
 					desiredJoinOrderFound = true;
 				}
@@ -962,9 +954,10 @@ public class OptimizerImpl implements Op
 				getBestAccessPath().setCostEstimate((CostEstimate) null);
 
 			if (optimizerTrace)
-			{
-				trace(CONSIDERING_JOIN_ORDER, 0, 0, 0.0, null);
-			}
+            {
+                tracer().traceJoinOrderConsideration
+                    ( joinPosition, ArrayUtil.copy( proposedJoinOrder ), (JBitSet) assignedTableMap.clone() );
+            }
 
 			Optimizable nextOpt =
 							optimizableList.getOptimizable(nextOptimizable);
@@ -1621,22 +1614,15 @@ public class OptimizerImpl implements Op
 					ce.singleScanRowCount());
 			}
 
-			if (optimizerTrace)
+			if (optimizerTrace) { tracer().traceCostWithoutSortAvoidance( currentCost ); }
 			{
-				trace(TOTAL_COST_NON_SA_PLAN, 0, 0, 0.0, null);
-				if (curOpt.considerSortAvoidancePath())
-				{
-					trace(TOTAL_COST_SA_PLAN, 0, 0, 0.0, null);
-				}
+				if (curOpt.considerSortAvoidancePath()) { tracer().traceCostWithSortAvoidance( currentSortAvoidanceCost ); }
 			}
 				
 			/* Do we have a complete join order? */
 			if ( joinPosition == (numOptimizables - 1) )
 			{
-				if (optimizerTrace)
-				{
-					trace(COMPLETE_JOIN_ORDER, 0, 0, 0.0, null);
-				}
+				if (optimizerTrace) { tracer().traceCompleteJoinOrder(); }
 
 				/* Add cost of sorting to non-sort-avoidance cost */
 				if (requiredRowOrdering != null)
@@ -1718,11 +1704,7 @@ public class OptimizerImpl implements Op
 										currentCost.singleScanRowCount()
 										);
 					
-					if (optimizerTrace)
-					{
-						trace(COST_OF_SORTING, 0, 0, 0.0, null);
-						trace(TOTAL_COST_WITH_SORTING, 0, 0, 0.0, null);
-					}
+					if (optimizerTrace) { tracer().traceSortCost( sortCost, currentCost ); }
 				}
 
 				/*
@@ -1790,10 +1772,7 @@ public class OptimizerImpl implements Op
 							bestRowOrdering, optimizableList) == 
 								RequiredRowOrdering.NOTHING_REQUIRED)
 					{
-						if (optimizerTrace)
-						{
-							trace(CURRENT_PLAN_IS_SA_PLAN, 0, 0, 0.0, null);
-						}
+						if (optimizerTrace) { tracer().traceCurrentPlanAvoidsSort( bestCost, currentSortAvoidanceCost ); }
 
 						if ((currentSortAvoidanceCost.compare(bestCost) <= 0)
 							|| bestCost.isUninitialized())
@@ -1823,12 +1802,7 @@ public class OptimizerImpl implements Op
 	{
 		foundABestPlan = true;
 
-		if (optimizerTrace)
-		{
-			trace(CHEAPEST_PLAN_SO_FAR, 0, 0, 0.0, null);
-			trace(PLAN_TYPE, planType, 0, 0.0, null);
-			trace(COST_OF_CHEAPEST_PLAN_SO_FAR, 0, 0, 0.0, null);
-		}
+		if (optimizerTrace) { tracer().traceCheapestPlanSoFar( planType, currentCost ); }
 
 		/* Remember the current cost as best */
 		bestCost.setCost(currentCost);
@@ -1871,11 +1845,8 @@ public class OptimizerImpl implements Op
 
 		if (optimizerTrace)
 		{
-			if (requiredRowOrdering != null)
-			{
-				trace(SORT_NEEDED_FOR_ORDERING, planType, 0, 0.0, null);
-			}
-			trace(REMEMBERING_BEST_JOIN_ORDER, 0, 0, 0.0, null);
+			if (requiredRowOrdering != null)    { tracer().traceSortNeededForOrdering( planType, requiredRowOrdering ); }
+            tracer().traceRememberingBestJoinOrder( joinPosition, ArrayUtil.copy( bestJoinOrder ), (JBitSet) assignedTableMap.clone() );
 		}
 	}
 
@@ -2192,10 +2163,7 @@ public class OptimizerImpl implements Op
 		// DERBY-1259.
 		if( ! optimizable.memoryUsageOK( estimatedCost.rowCount() / outerCost.rowCount(), maxMemoryPerTable))
 		{
-			if (optimizerTrace)
-			{
-				trace(SKIPPING_DUE_TO_EXCESS_MEMORY, 0, 0, 0.0, null);
-			}
+			if (optimizerTrace) { tracer().traceSkippingBecauseTooMuchMemory( maxMemoryPerTable ); }
 			return;
 		}
 
@@ -2329,10 +2297,7 @@ public class OptimizerImpl implements Op
         if( ! optimizable.memoryUsageOK( estimatedCost.rowCount() / outerCost.rowCount(),
                                          maxMemoryPerTable))
 		{
-			if (optimizerTrace)
-			{
-				trace(SKIPPING_DUE_TO_EXCESS_MEMORY, 0, 0, 0.0, null);
-			}
+			if (optimizerTrace) { tracer().traceSkippingBecauseTooMuchMemory( maxMemoryPerTable ); }
 			return;
 		}
 
@@ -2418,17 +2383,11 @@ public class OptimizerImpl implements Op
 	 */
 	public void modifyAccessPaths() throws StandardException
 	{
-		if (optimizerTrace)
-		{
-			trace(MODIFYING_ACCESS_PATHS, 0, 0, 0.0, null);
-		}
+		if (optimizerTrace) { tracer().traceModifyingAccessPaths(); }
 
 		if ( ! foundABestPlan)
 		{
-			if (optimizerTrace)
-			{
-				trace(NO_BEST_PLAN, 0, 0, 0.0, null);
-			}
+			if (optimizerTrace) { tracer().traceNoBestPlan(); }
 
 			throw StandardException.newException(SQLState.LANG_NO_BEST_PLAN_FOUND);
 		}
@@ -2643,12 +2602,6 @@ public class OptimizerImpl implements Op
 		return new CostEstimateImpl(theCost, theRowCount, theSingleScanRowCount);
 	}
 
-	// Optimzer trace
-	public void trace(int traceFlag, int intParam1, int intParam2,
-					  double doubleParam, Object objectParam1)
-	{
-	}
-	
 	/** @see Optimizer#useStatistics */
 	public boolean useStatistics() { return useStatistics && optimizableList.useStatistics(); }
 
@@ -2798,4 +2751,12 @@ public class OptimizerImpl implements Op
 		return;
 	}
 
+    /** Get the trace machinery */
+    public  OptTrace    tracer()
+    {
+        if ( _tracer == null ) { _tracer = new DefaultOptTrace( lcc,  hashCode() ); }
+
+        return _tracer;
+    }
+
 }