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 km...@apache.org on 2009/06/23 02:36:34 UTC

svn commit: r787505 - in /db/derby/code/branches/10.2/java: engine/org/apache/derby/iapi/sql/compile/ engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apache/derbyTesting/functionTests/tests/lang/

Author: kmarsden
Date: Tue Jun 23 00:36:33 2009
New Revision: 787505

URL: http://svn.apache.org/viewvc?rev=787505&view=rev
Log:
DERBY-3926 Incorrect ORDER BY caused by index

Contributed by Mamta Satoor (msatoor at gmail dot com)


Modified:
    db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RequiredRowOrdering.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RowOrdering.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
    db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/RowOrderingImpl.java
    db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/master/wisconsin.out
    db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/tests/lang/wisc_setup.sql

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java Tue Jun 23 00:36:33 2009
@@ -122,6 +122,23 @@
 	 * hash table on a next are qualifiers.
 	 */
 	public void markAllPredicatesQualifiers();
+	
+	/**
+	 * Check into the predicate list if the passed column has an equijoin 
+	 * predicate on it.
+	 * 
+	 * @param optTable
+	 * @param columnNumber
+	 * @param isNullOkay
+	 * @return the position of the predicate in the list which corresponds to 
+	 *   the equijoin. If no quijoin predicate found, then the return value 
+	 *   will be -1
+	 * @throws StandardException
+	 */
+	public int hasEqualityPredicateOnOrderedColumn(Optimizable optTable,
+			  int columnNumber,
+			  boolean isNullOkay)
+	throws StandardException;
 
 	/**
 	 * Is there an optimizable equality predicate on the specified column?

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RequiredRowOrdering.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RequiredRowOrdering.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RequiredRowOrdering.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RequiredRowOrdering.java Tue Jun 23 00:36:33 2009
@@ -24,6 +24,7 @@
 import org.apache.derby.iapi.error.StandardException;
 
 import org.apache.derby.iapi.util.JBitSet;
+import org.apache.derby.impl.sql.compile.PredicateList;
 
 /**
  * This interface provides a representation of the required ordering of rows
@@ -43,6 +44,11 @@
 	 * given a RowOrdering.
 	 *
 	 * @param rowOrdering	The order of rows in question
+	 * @param optimizableList	The current join order being considered by 
+	 *    the optimizer. We need to look into this to determine if the outer
+	 *    optimizables are single row resultset if the order by column is
+	 *    on an inner optimizable and that inner optimizable is not a one
+	 *    row resultset. DERBY-3926
 	 *
 	 * @return	SORT_REQUIRED if sorting is required,
 	 *			ELIMINATE_DUPS if no sorting is required but duplicates
@@ -52,7 +58,8 @@
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
-	int sortRequired(RowOrdering rowOrdering) throws StandardException;
+	int sortRequired(RowOrdering rowOrdering, OptimizableList optimizableList) 
+	throws StandardException;
 
 	/**
 	 * Tell whether sorting is required for this RequiredRowOrdering,
@@ -63,6 +70,11 @@
 	 *
 	 * @param rowOrdering	The order of rows in the partial join order
 	 * @param tableMap		A bit map of the tables in the partial join order
+	 * @param optimizableList	The current join order being considered by 
+	 *    the optimizer. We need to look into this to determine if the outer
+	 *    optimizables are single row resultset if the order by column is
+	 *    on an inner optimizable and that inner optimizable is not a one
+	 *    row resultset. DERBY-3926
 	 *
 	 * @return	SORT_REQUIRED if sorting is required,
 	 *			ELIMINATE_DUPS if no sorting is required by duplicates
@@ -72,7 +84,8 @@
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
-	int sortRequired(RowOrdering rowOrdering, JBitSet tableMap)
+	int sortRequired(RowOrdering rowOrdering, JBitSet tableMap, 
+			OptimizableList optimizableList)
 			throws StandardException;
 
 	/**

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RowOrdering.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RowOrdering.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RowOrdering.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/iapi/sql/compile/RowOrdering.java Tue Jun 23 00:36:33 2009
@@ -130,13 +130,27 @@
 	/**
 	 * Tell this RowOrdering that it is always ordered on the given column
 	 * of the given optimizable.  This is useful when a column in the
-	 * optimizable has an equals comparison with a constant expression.
-	 * This is reset when the optimizable is removed from this RowOrdering.
+	 * optimizable has an equals comparison with a constant expression or
+	 * it is involved in a equijoin with an optimizable which is always 
+	 * ordered on the column on which the equijoin is happening. This is 
+	 * reset when the optimizable is removed from this RowOrdering.
 	 *
 	 * @param optimizable	The table in question
 	 * @param columnNumber	The number of the column in question.
 	 */
 	void columnAlwaysOrdered(Optimizable optimizable, int columnNumber);
+	
+	/**
+	 * Return true if the column is always ordered. That will be true if the
+	 * column has a constant comparison predicate on it or it is involved in
+	 * a equijoin with an optimizable which is always ordered on the column
+	 * on which the equijoin is happening.
+	 *
+	 * @param tableNumber	The table in question
+	 * @param columnNumber	The number of the column in question.
+	 * @return	true means this column is always ordered
+	*/
+	public boolean isColumnAlwaysOrdered(int tableNumber, int columnNumber);
 
 	/**
 	 * Ask whether the given table is always ordered.

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Tue Jun 23 00:36:33 2009
@@ -456,6 +456,13 @@
 
 				for (int i = 0; i < baseColumnPositions.length; i++)
 				{
+					//Check if the order by column has equijoin on another 
+					//column which is already identified as an ordered column
+					if (doesOrderByColumnHaveEquiJoin(
+							irg, predList, rowOrdering))
+						rowOrdering.columnAlwaysOrdered(this, 
+								baseColumnPositions[i]);
+
 					/*
 					** Don't add the column to the ordering if it's already
 					** an ordered column.  This can happen in the following
@@ -4263,6 +4270,86 @@
 		return true;
 	}
 
+	//Check if the columns in the index have an equijoin on them
+	//with other already ordered columns from the other optimizables. This
+	//is done by going through the columns in the index and checking the
+	//predicate list for equijoins on the index columns. 
+	private boolean doesOrderByColumnHaveEquiJoin(IndexRowGenerator irg,
+			OptimizablePredicateList predList, RowOrdering ro)
+	throws StandardException
+	{
+		if (predList == null)
+		{
+			return false;
+		}
+
+		// is this a unique index. 
+		if (! irg.isUnique())
+		{
+			return false;
+		}
+
+		PredicateList restrictionList = (PredicateList) predList;
+
+		int[] baseColumnPositions = irg.baseColumnPositions();
+
+		for (int index = 0; index < baseColumnPositions.length; index++)
+		{
+			// get the column number at this position
+			int curCol = baseColumnPositions[index];
+
+			//Check if this column from the index has an equi join predicate
+			//on them.
+			int j = restrictionList.hasEqualityPredicateOnOrderedColumn(
+					this, curCol, true);
+			if (j == -1)
+				return false;
+
+			//We have found a predicate which has an equi join involving the
+			//index column. Now ensure that the equi join is with a column
+			//which is already identified as always ordered.
+			Predicate		predicate;
+			predicate = (Predicate) restrictionList.elementAt(j);
+			ValueNode vn = predicate.getAndNode().getLeftOperand();
+			ColumnReference cr;
+  			if (vn instanceof BinaryRelationalOperatorNode)
+  			{
+  				BinaryRelationalOperatorNode bon = 
+  					(BinaryRelationalOperatorNode) vn;
+  				cr = null;
+				if (bon.columnOnOneSide(this) == 
+					BinaryRelationalOperatorNode.LEFT)
+				{
+	  				//If the index column is on left side, then look for the 
+					//operand on the other side to see if it is of type 
+					//ColumnReference. If it is, then check if that column 
+					//is identified as always ordered
+					if (bon.getRightOperand() instanceof ColumnReference)
+						cr = (ColumnReference)bon.getRightOperand();
+				} else
+				{
+	  				//If the index column is on right side, then look for the 
+					//operand on the other side to see if it is of type 
+					//ColumnReference. If it is, then check if that column 
+					//is identified as always ordered
+					if (bon.getLeftOperand() instanceof ColumnReference)
+						cr = (ColumnReference)bon.getLeftOperand();
+				}
+				if (cr!=null)
+				{
+					//We have found that the index column is involved in an
+					//equijoin with another column. Now check if that other
+					//column is always ordered
+					if (ro.orderedOnColumn(1, cr.getTableNumber(), 
+							cr.getColumnNumber()))
+						return true;
+				}
+  			}
+		}
+
+		return false;
+	}
+
 	/**
 	 * Is this a one-row result set with the given conglomerate descriptor?
 	 */

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OptimizerImpl.java Tue Jun 23 00:36:33 2009
@@ -1665,8 +1665,9 @@
 				if (requiredRowOrdering != null &&
 					curOpt.considerSortAvoidancePath())
 				{
-					if (requiredRowOrdering.sortRequired(bestRowOrdering) ==
-									RequiredRowOrdering.NOTHING_REQUIRED)
+					if (requiredRowOrdering.sortRequired(
+							bestRowOrdering, optimizableList) == 
+								RequiredRowOrdering.NOTHING_REQUIRED)
 					{
 						if (optimizerTrace)
 						{
@@ -2122,8 +2123,9 @@
 				** path avoid a sort?
 				*/
 				if (requiredRowOrdering.sortRequired(currentRowOrdering,
-														assignedTableMap)
-										== RequiredRowOrdering.NOTHING_REQUIRED)
+														assignedTableMap,
+														optimizableList)
+										==RequiredRowOrdering.NOTHING_REQUIRED)
 				{
 					ap = optimizable.getBestSortAvoidancePath();
 					bestCostEstimate = ap.getCostEstimate();
@@ -2253,7 +2255,8 @@
 				** path avoid a sort?
 				*/
 				if (requiredRowOrdering.sortRequired(currentRowOrdering,
-														assignedTableMap)
+														assignedTableMap,
+														optimizableList)
 										== RequiredRowOrdering.NOTHING_REQUIRED)
 				{
 					ap = optimizable.getBestSortAvoidancePath();

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java Tue Jun 23 00:36:33 2009
@@ -22,7 +22,9 @@
 package	org.apache.derby.impl.sql.compile;
 
 import org.apache.derby.iapi.sql.compile.CompilerContext;
+import org.apache.derby.iapi.sql.compile.Optimizable;
 import org.apache.derby.iapi.sql.compile.CostEstimate;
+import org.apache.derby.iapi.sql.compile.OptimizableList;
 import org.apache.derby.iapi.sql.compile.RequiredRowOrdering;
 import org.apache.derby.iapi.sql.compile.RowOrdering;
 import org.apache.derby.iapi.sql.compile.C_NodeTypes;
@@ -413,9 +415,10 @@
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
-	public int sortRequired(RowOrdering rowOrdering) throws StandardException
+	public int sortRequired(RowOrdering rowOrdering,
+			OptimizableList optimizableList) throws StandardException
 	{
-		return sortRequired(rowOrdering, (JBitSet) null);
+		return sortRequired(rowOrdering, (JBitSet) null, optimizableList);
 	}
 
 	/**
@@ -423,7 +426,9 @@
 	 *
 	 * @exception StandardException		Thrown on error
 	 */
-	public int sortRequired(RowOrdering rowOrdering, JBitSet tableMap)
+	public int sortRequired(RowOrdering rowOrdering, 
+			JBitSet tableMap,
+			OptimizableList optimizableList)
 				throws StandardException
 	{
 		/*
@@ -499,7 +504,92 @@
 					return RequiredRowOrdering.NOTHING_REQUIRED;
 				}
 			}
-
+			/*
+			 * Does this order by column belong to the outermost optimizable in
+			 * the current join order?
+			 * 
+			 * If yes, then we do not need worry about the ordering of the rows
+			 * feeding into it. Because the order by column is associated with 
+			 * the outermost optimizable, optimizer will not have to deal with 
+			 * the order of any rows coming in from the previous optimizables. 
+			 * 
+			 * But if the current order by column belongs to an inner 
+			 * optimizable in the join order, then go through the following
+			 * if condition logic.
+			 */
+
+			/* If the following boolean is true, then it means that the join 
+			 * order being considered has more than one table 
+			 */
+			boolean moreThanOneTableInJoinOrder = tableMap!=null?
+					(!tableMap.hasSingleBitSet()) : false;
+			if (moreThanOneTableInJoinOrder) 
+			{
+				/*
+				 * First check if the order by column has a constant comparison
+				 * predicate on it or it belongs to an optimizable which is 
+				 * always ordered(that means it is a single row table) or the 
+				 * column is involved in an equijoin with an optimizable which 
+				 * is always ordered on the column on which the equijoin is 
+				 * happening. If yes, then we know that the rows will always be 
+				 * sorted and hence we do not need to worry if (any) prior 
+				 * optimizables in join order are one-row resultsets or not. 
+				 */
+				if ((!rowOrdering.alwaysOrdered(cr.getTableNumber())) &&
+						(!rowOrdering.isColumnAlwaysOrdered(
+								cr.getTableNumber(), cr.getColumnNumber())))
+				{
+					/*
+					 * The current order by column is not always ordered which 
+					 * means that the rows from it will not necessarily be in 
+					 * the sorted order on that column. Because of this, we 
+					 * need to make sure that the outer optimizables (outer to 
+					 * the order by columns's optimizable) in the join order 
+					 * are all one row optimizables, meaning that they can at 
+					 * the most return only one row. If they return more than 
+					 * one row, then it will require multiple scans of the 
+					 * order by column's optimizable and the rows returned 
+					 * from those multiple scans may not be ordered correctly.
+					 */
+					for (int i=0; i < optimizableList.size(); i++)
+					{
+						//Get one outer optimizable at a time from the join
+						//order
+						Optimizable considerOptimizable = 
+							optimizableList.getOptimizable(i);
+						//If we have come across the optimizable for the order 
+						//by column in the join order, then we do not need to 
+						//look at the inner optimizables in the join order. As
+						//long as the outer optimizables are one row resultset,
+						//we are fine to consider sort avoidance.
+						if (considerOptimizable.getTableNumber() == 
+							cr.getTableNumber())
+							break;
+						/*
+						 * The following if condition is checking if the
+						 * outer optimizable to the order by column's 
+						 * optimizable is one row resultset or not. 
+						 * 
+						 * If the outer optimizable is one row resultset, 
+						 * then move on to the next optimizable in the join 
+						 * order and do the same check on that optimizable. 
+						 * Continue this  until we are done checking that all 
+						 * the outer optimizables in the join order are single 
+						 * row resultsets. If we run into an outer optimizable 
+						 * which is not one row resultset, then we can not 
+						 * consider sort avoidance for the query.
+						 */
+						if (rowOrdering.alwaysOrdered(
+								considerOptimizable.getTableNumber()))
+							continue;
+						else
+							//This outer optimizable can return more than 
+							//one row. Because of this, we can't avoid the
+							//sorting for this query.
+							return RequiredRowOrdering.SORT_REQUIRED;
+					}
+				}
+			}
 			if ( ! rowOrdering.alwaysOrdered(cr.getTableNumber()))
 			{
 				/*

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java Tue Jun 23 00:36:33 2009
@@ -329,6 +329,45 @@
 	}
 
 	/**
+	 * @see OptimizablePredicateList#hasEqualityPredicateOnOrderedColumn
+	 *
+	 * @exception StandardException		Thrown on error
+	 */
+	public int hasEqualityPredicateOnOrderedColumn(Optimizable optTable,
+													  int columnNumber,
+													  boolean isNullOkay)
+							throws StandardException
+	{
+		ValueNode opNode = null;
+		int size = size();
+		for (int index = 0; index < size; index++)
+		{
+			AndNode			andNode;
+			Predicate		predicate;
+			predicate = (Predicate) elementAt(index);
+			//We are not looking at constant comparison predicate.
+			if (predicate.getReferencedMap().hasSingleBitSet())
+			{
+				continue;
+			}
+
+			andNode = (AndNode) predicate.getAndNode();
+
+			// skip non-equality predicates
+			opNode = andNode.getLeftOperand();
+
+			if (opNode.optimizableEqualityNode(optTable,
+											   columnNumber,
+											   isNullOkay))
+			{
+				return index;
+			}
+		}
+
+		return -1;
+	}
+
+	/**
 	 * @see OptimizablePredicateList#hasOptimizableEqualityPredicate
 	 *
 	 * @exception StandardException		Thrown on error

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java Tue Jun 23 00:36:33 2009
@@ -4064,4 +4064,21 @@
 	protected void markInitialSize() {
 		initialListSize = size();
 	}
+	
+	/**
+	 * Remove any columns that may have been added for an order by clause.
+	 * In a query like:
+	 * <pre>select a from t order by b</pre> b is added to the select list
+	 * However in the final projection, after the sort is complete, b will have
+	 * to be removed. 
+	 *
+	 */
+	public void removeOrderByColumns() 
+	{
+		int idx = size() - 1;
+		for (int i = 0; i < orderBySelect; i++, idx--) {
+			removeElementAt(idx);
+		}
+		orderBySelect = 0;
+	}
 }

Modified: db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/RowOrderingImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/RowOrderingImpl.java?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/RowOrderingImpl.java (original)
+++ db/derby/code/branches/10.2/java/engine/org/apache/derby/impl/sql/compile/RowOrderingImpl.java Tue Jun 23 00:36:33 2009
@@ -39,7 +39,9 @@
 	** This ColumnOrdering represents the columns that can be considered
 	** ordered no matter what.  For example, columns that are compared to
 	** constants with = are always ordered.  Also, all columns in a one-row
-	** result set are ordered.
+	** result set are ordered. Another instance of always ordered is when
+	** the column is involved in an equijoin with an optimizable which is 
+	** always ordered on the column on which the equijoin is happening.
 	*/
 	ColumnOrdering	columnsAlwaysOrdered;
 
@@ -60,6 +62,11 @@
 		columnsAlwaysOrdered = new ColumnOrdering(RowOrdering.DONTCARE);
 		alwaysOrderedOptimizables = new Vector();
 	}
+	
+	/** @see RowOrdering#isColumnAlwaysOrdered */
+	public boolean isColumnAlwaysOrdered(int tableNumber, int columnNumber){
+		return (columnsAlwaysOrdered.contains(tableNumber, columnNumber)); 
+	}
 
 	/**
 	 * @see RowOrdering#orderedOnColumn

Modified: db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/master/wisconsin.out
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/master/wisconsin.out?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/master/wisconsin.out (original)
+++ db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/master/wisconsin.out Tue Jun 23 00:36:33 2009
@@ -21828,97 +21828,112 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Hash Exists Join ResultSet:
+Project-Restrict ResultSet (6):
 <filtered number of opens>
-<filtered rows seen from the left>
-<filtered rows seen from the right>
+<filtered rows seen>
 Rows filtered = 0
-<filtered rows returned>
+restriction = false
+projection = true
 	constructor time (milliseconds) = 0
 	open time (milliseconds) = 0
 	next time (milliseconds) = 0
 	close time (milliseconds) = 0
-Left result set:
-	Index Row to Base Row ResultSet for TENKTUP1:
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Hash Exists Join ResultSet:
 	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+	<filtered rows seen from the left>
+	<filtered rows seen from the right>
+	Rows filtered = 0
+	<filtered rows returned>
 		constructor time (milliseconds) = 0
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Index Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+	Left result set:
+		Index Row to Base Row ResultSet for TENKTUP2:
 		<filtered number of opens>
 		<filtered rows seen>
-		Rows filtered = 0
-		Fetch Size = 1
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+			<filtered number of opens>
+			<filtered rows seen>
+			Rows filtered = 0
+			Fetch Size = 1
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			qualifiers:
+				qualifiers:
 None
-Right result set:
-	Index Row to Base Row ResultSet for TENKTUP2:
-	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
-		constructor time (milliseconds) = 0
-		open time (milliseconds) = 0
-		next time (milliseconds) = 0
-		close time (milliseconds) = 0
-		Hash Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking: 
+	Right result set:
+		Index Row to Base Row ResultSet for TENKTUP1:
 		<filtered number of opens>
-		Hash table size = 2500
-		Hash key is column number 0
 		<filtered rows seen>
-		Rows filtered = 0
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
+			<filtered number of opens>
+			Hash table size = 2500
+			Hash key is column number 0
+			<filtered rows seen>
+			Rows filtered = 0
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			scan qualifiers:
+				scan qualifiers:
 None
-			next qualifiers:
+				next qualifiers:
 Column[0][0] Id: 0
 Operator: =
 Ordered nulls: false
 Unknown return value: false
 Negate comparison result: false
 ij> commit;
-ij> -- 25% of rows from joining table
+ij> -- force TENKTUP1 as the outermost join table to make sure 
+-- that no sorting is necessary. DERBY-3926
 get cursor c as
-	'select * from TENKTUP1, TENKTUP2
+	'select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
 	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1';
@@ -21929,7 +21944,8 @@
 Statement Name: 
 	C
 Statement Text: 
-	select * from TENKTUP1, TENKTUP2
+	select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
 	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1
@@ -21944,6 +21960,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
+User supplied optimizer overrides for join are { joinOrder=FIXED }
 Hash Exists Join ResultSet:
 <filtered number of opens>
 <filtered rows seen from the left>
@@ -22032,11 +22049,13 @@
 Unknown return value: false
 Negate comparison result: false
 ij> commit;
-ij> -- 10% of rows from joining table
+ij> -- This time, force TENKTUP2 as the outermost join table to make sure 
+-- that still no sorting is necessary. DERBY-3926
 get cursor c as
-	'select * from TENKTUP1, TENKTUP2
+	'select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP2, TENKTUP1
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 1000
+	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1';
 ij> close c;
 ij> values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
@@ -22045,9 +22064,10 @@
 Statement Name: 
 	C
 Statement Text: 
-	select * from TENKTUP1, TENKTUP2
+	select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP2, TENKTUP1
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 1000
+	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1
 Parse Time: 0
 Bind Time: 0
@@ -22060,6 +22080,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
+User supplied optimizer overrides for join are { joinOrder=FIXED }
 Hash Exists Join ResultSet:
 <filtered number of opens>
 <filtered rows seen from the left>
@@ -22071,7 +22092,7 @@
 	next time (milliseconds) = 0
 	close time (milliseconds) = 0
 Left result set:
-	Index Row to Base Row ResultSet for TENKTUP1:
+	Index Row to Base Row ResultSet for TENKTUP2:
 	<filtered number of opens>
 	<filtered rows seen>
 	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
@@ -22079,7 +22100,7 @@
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Index Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+		Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
 		<filtered number of opens>
 		<filtered rows seen>
 		Rows filtered = 0
@@ -22106,7 +22127,7 @@
 			qualifiers:
 None
 Right result set:
-	Index Row to Base Row ResultSet for TENKTUP2:
+	Index Row to Base Row ResultSet for TENKTUP1:
 	<filtered number of opens>
 	<filtered rows seen>
 	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
@@ -22114,9 +22135,9 @@
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Hash Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking: 
+		Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
 		<filtered number of opens>
-		Hash table size = 1000
+		Hash table size = 2500
 		Hash key is column number 0
 		<filtered rows seen>
 		Rows filtered = 0
@@ -22148,11 +22169,11 @@
 Unknown return value: false
 Negate comparison result: false
 ij> commit;
-ij> -- 5% of rows from joining table
+ij> -- 25% of rows from joining table
 get cursor c as
 	'select * from TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 500
+	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1';
 ij> close c;
 ij> values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
@@ -22163,7 +22184,7 @@
 Statement Text: 
 	select * from TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 500
+	 and TENKTUP2.unique1 < 2500
 	 order by TENKTUP1.unique1
 Parse Time: 0
 Bind Time: 0
@@ -22176,99 +22197,112 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Hash Exists Join ResultSet:
+Project-Restrict ResultSet (6):
 <filtered number of opens>
-<filtered rows seen from the left>
-<filtered rows seen from the right>
+<filtered rows seen>
 Rows filtered = 0
-<filtered rows returned>
+restriction = false
+projection = true
 	constructor time (milliseconds) = 0
 	open time (milliseconds) = 0
 	next time (milliseconds) = 0
 	close time (milliseconds) = 0
-Left result set:
-	Index Row to Base Row ResultSet for TENKTUP1:
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Hash Exists Join ResultSet:
 	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+	<filtered rows seen from the left>
+	<filtered rows seen from the right>
+	Rows filtered = 0
+	<filtered rows returned>
 		constructor time (milliseconds) = 0
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Index Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+	Left result set:
+		Index Row to Base Row ResultSet for TENKTUP2:
 		<filtered number of opens>
 		<filtered rows seen>
-		Rows filtered = 0
-		Fetch Size = 1
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+			<filtered number of opens>
+			<filtered rows seen>
+			Rows filtered = 0
+			Fetch Size = 1
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			qualifiers:
+				qualifiers:
 None
-Right result set:
-	Index Row to Base Row ResultSet for TENKTUP2:
-	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
-		constructor time (milliseconds) = 0
-		open time (milliseconds) = 0
-		next time (milliseconds) = 0
-		close time (milliseconds) = 0
-		Hash Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking: 
+	Right result set:
+		Index Row to Base Row ResultSet for TENKTUP1:
 		<filtered number of opens>
-		Hash table size = 500
-		Hash key is column number 0
 		<filtered rows seen>
-		Rows filtered = 0
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
+			<filtered number of opens>
+			Hash table size = 2500
+			Hash key is column number 0
+			<filtered rows seen>
+			Rows filtered = 0
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			scan qualifiers:
+				scan qualifiers:
 None
-			next qualifiers:
+				next qualifiers:
 Column[0][0] Id: 0
 Operator: =
 Ordered nulls: false
 Unknown return value: false
 Negate comparison result: false
 ij> commit;
-ij> -- 1% of rows from joining table
+ij> -- 10% of rows from joining table
 get cursor c as
 	'select * from TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 100
+	 and TENKTUP2.unique1 < 1000
 	 order by TENKTUP1.unique1';
 ij> close c;
 ij> values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
@@ -22279,7 +22313,7 @@
 Statement Text: 
 	select * from TENKTUP1, TENKTUP2
 	 where TENKTUP1.unique1 = TENKTUP2.unique1
-	 and TENKTUP2.unique1 < 100
+	 and TENKTUP2.unique1 < 1000
 	 order by TENKTUP1.unique1
 Parse Time: 0
 Bind Time: 0
@@ -22292,88 +22326,359 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Hash Exists Join ResultSet:
+Project-Restrict ResultSet (6):
 <filtered number of opens>
-<filtered rows seen from the left>
-<filtered rows seen from the right>
+<filtered rows seen>
 Rows filtered = 0
-<filtered rows returned>
+restriction = false
+projection = true
 	constructor time (milliseconds) = 0
 	open time (milliseconds) = 0
 	next time (milliseconds) = 0
 	close time (milliseconds) = 0
-Left result set:
-	Index Row to Base Row ResultSet for TENKTUP1:
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Hash Exists Join ResultSet:
 	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
-		constructor time (milliseconds) = 0
+	<filtered rows seen from the left>
+	<filtered rows seen from the right>
+	Rows filtered = 0
+	<filtered rows returned>
+		constructor time (milliseconds) = 0
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Index Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+	Left result set:
+		Index Row to Base Row ResultSet for TENKTUP2:
 		<filtered number of opens>
 		<filtered rows seen>
-		Rows filtered = 0
-		Fetch Size = 1
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+			<filtered number of opens>
+			<filtered rows seen>
+			Rows filtered = 0
+			Fetch Size = 1
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			qualifiers:
+				qualifiers:
 None
-Right result set:
-	Index Row to Base Row ResultSet for TENKTUP2:
+	Right result set:
+		Index Row to Base Row ResultSet for TENKTUP1:
+		<filtered number of opens>
+		<filtered rows seen>
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
+			<filtered number of opens>
+			Hash table size = 1000
+			Hash key is column number 0
+			<filtered rows seen>
+			Rows filtered = 0
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
+	None
+				stop position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+				scan qualifiers:
+None
+				next qualifiers:
+Column[0][0] Id: 0
+Operator: =
+Ordered nulls: false
+Unknown return value: false
+Negate comparison result: false
+ij> commit;
+ij> -- 5% of rows from joining table
+get cursor c as
+	'select * from TENKTUP1, TENKTUP2
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 500
+	 order by TENKTUP1.unique1';
+ij> close c;
+ij> values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	C
+Statement Text: 
+	select * from TENKTUP1, TENKTUP2
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 500
+	 order by TENKTUP1.unique1
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (6):
+<filtered number of opens>
+<filtered rows seen>
+Rows filtered = 0
+restriction = false
+projection = true
+	constructor time (milliseconds) = 0
+	open time (milliseconds) = 0
+	next time (milliseconds) = 0
+	close time (milliseconds) = 0
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Hash Exists Join ResultSet:
 	<filtered number of opens>
-	<filtered rows seen>
-	Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+	<filtered rows seen from the left>
+	<filtered rows seen from the right>
+	Rows filtered = 0
+	<filtered rows returned>
 		constructor time (milliseconds) = 0
 		open time (milliseconds) = 0
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
-		Hash Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking: 
+	Left result set:
+		Index Row to Base Row ResultSet for TENKTUP2:
 		<filtered number of opens>
-		Hash table size = 100
-		Hash key is column number 0
 		<filtered rows seen>
-		Rows filtered = 0
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
 			constructor time (milliseconds) = 0
 			open time (milliseconds) = 0
 			next time (milliseconds) = 0
 			close time (milliseconds) = 0
-		scan information: 
-			Bit set of columns fetched=All
-			Number of columns fetched=2
-			Number of deleted rows visited=0
-			<filtered number of pages visited>
-			<filtered number of rows qualified>
-			<filtered number of rows visited>
-			Scan type=btree
-			Tree height=2
-			start position: 
+			Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+			<filtered number of opens>
+			<filtered rows seen>
+			Rows filtered = 0
+			Fetch Size = 1
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
 	None
-			stop position: 
+				stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
 0 
-			scan qualifiers:
+				qualifiers:
 None
-			next qualifiers:
+	Right result set:
+		Index Row to Base Row ResultSet for TENKTUP1:
+		<filtered number of opens>
+		<filtered rows seen>
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
+			<filtered number of opens>
+			Hash table size = 500
+			Hash key is column number 0
+			<filtered rows seen>
+			Rows filtered = 0
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
+	None
+				stop position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+				scan qualifiers:
+None
+				next qualifiers:
+Column[0][0] Id: 0
+Operator: =
+Ordered nulls: false
+Unknown return value: false
+Negate comparison result: false
+ij> commit;
+ij> -- 1% of rows from joining table
+get cursor c as
+	'select * from TENKTUP1, TENKTUP2
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 100
+	 order by TENKTUP1.unique1';
+ij> close c;
+ij> values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	C
+Statement Text: 
+	select * from TENKTUP1, TENKTUP2
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 100
+	 order by TENKTUP1.unique1
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (6):
+<filtered number of opens>
+<filtered rows seen>
+Rows filtered = 0
+restriction = false
+projection = true
+	constructor time (milliseconds) = 0
+	open time (milliseconds) = 0
+	next time (milliseconds) = 0
+	close time (milliseconds) = 0
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Hash Exists Join ResultSet:
+	<filtered number of opens>
+	<filtered rows seen from the left>
+	<filtered rows seen from the right>
+	Rows filtered = 0
+	<filtered rows returned>
+		constructor time (milliseconds) = 0
+		open time (milliseconds) = 0
+		next time (milliseconds) = 0
+		close time (milliseconds) = 0
+	Left result set:
+		Index Row to Base Row ResultSet for TENKTUP2:
+		<filtered number of opens>
+		<filtered rows seen>
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			Index Scan ResultSet for TENKTUP2 using index TK2UNIQUE1 at serializable isolation level using share row locking chosen by the optimizer
+			<filtered number of opens>
+			<filtered rows seen>
+			Rows filtered = 0
+			Fetch Size = 1
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
+	None
+				stop position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+				qualifiers:
+None
+	Right result set:
+		Index Row to Base Row ResultSet for TENKTUP1:
+		<filtered number of opens>
+		<filtered rows seen>
+		Columns accessed from heap = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			Hash Scan ResultSet for TENKTUP1 using index TK1UNIQUE1 at serializable isolation level using share row locking: 
+			<filtered number of opens>
+			Hash table size = 100
+			Hash key is column number 0
+			<filtered rows seen>
+			Rows filtered = 0
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				<filtered number of pages visited>
+				<filtered number of rows qualified>
+				<filtered number of rows visited>
+				Scan type=btree
+				Tree height=2
+				start position: 
+	None
+				stop position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+				scan qualifiers:
+None
+				next qualifiers:
 Column[0][0] Id: 0
 Operator: =
 Ordered nulls: false

Modified: db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/tests/lang/wisc_setup.sql
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/tests/lang/wisc_setup.sql?rev=787505&r1=787504&r2=787505&view=diff
==============================================================================
--- db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/tests/lang/wisc_setup.sql (original)
+++ db/derby/code/branches/10.2/java/testing/org/apache/derbyTesting/functionTests/tests/lang/wisc_setup.sql Tue Jun 23 00:36:33 2009
@@ -2702,6 +2702,34 @@
 
 commit;
 
+-- force TENKTUP1 as the outermost join table to make sure 
+-- that no sorting is necessary. DERBY-3926
+get cursor c as
+	'select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP1, TENKTUP2
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 2500
+	 order by TENKTUP1.unique1';
+close c;
+
+values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+
+commit;
+
+-- This time, force TENKTUP2 as the outermost join table to make sure 
+-- that still no sorting is necessary. DERBY-3926
+get cursor c as
+	'select * from --DERBY-PROPERTIES joinOrder=FIXED
+	 TENKTUP2, TENKTUP1
+	 where TENKTUP1.unique1 = TENKTUP2.unique1
+	 and TENKTUP2.unique1 < 2500
+	 order by TENKTUP1.unique1';
+close c;
+
+values SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+
+commit;
+
 -- 25% of rows from joining table
 get cursor c as
 	'select * from TENKTUP1, TENKTUP2