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 ab...@apache.org on 2008/01/29 01:58:39 UTC

svn commit: r616126 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/sql/compile/ engine/org/apache/derby/iapi/sql/execute/ engine/org/apache/derby/impl/sql/compile/ engine/org/apache/derby/impl/sql/execute/ testing/org/apache/derbyTesting/...

Author: abrown
Date: Mon Jan 28 16:58:37 2008
New Revision: 616126

URL: http://svn.apache.org/viewvc?rev=616126&view=rev
Log:
DERBY-3279: Add logic for execution-time sorting of IN list values
into DESCENDING order when required.  This is necessary when doing
IN-list "multi-probing" on some column C for which a DESC sort has
been eliminated (by the optimizer). In such a case the row values
will come back in the order in which they appear in the IN List,
which means the IN list values must themselves be sorted in the
correct (descending) order.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.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/InListOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultSetNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SingleChildResultSetNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/InListMultiProbeTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/OptimizablePredicateList.java Mon Jan 28 16:58:37 2008
@@ -321,4 +321,12 @@
 	 */	
 	public double selectivity(Optimizable optTable) throws StandardException;
 	
+	/**
+	 * Walk through the predicates in this list and make any adjustments
+	 * that are required to allow for proper handling of an ORDER BY
+	 * clause.
+	 */
+	public void adjustForSortElimination(RequiredRowOrdering ordering)
+		throws StandardException;
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java Mon Jan 28 16:58:37 2008
@@ -1002,8 +1002,8 @@
 
 		@param probeVals List of values with which to probe the underlying
 			table. Should not be null.
-		@param probeValsAreSorted Whether or not the values in probeVals are
-			sorted.
+		@param sortRequired Which type of sort we need for the values
+			(ascending, descending, or none).
 	 */
 	NoPutResultSet getMultiProbeTableScanResultSet(
 			                    Activation activation,
@@ -1018,7 +1018,7 @@
 								boolean sameStartStopPosition,
 								Qualifier[][] qualifiers,
 								DataValueDescriptor [] probeVals,
-								boolean probeValsAreSorted,
+								int sortRequired,
 								String tableName,
 								String userSuppliedOptimizerOverrides,
 								String indexName,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java Mon Jan 28 16:58:37 2008
@@ -41,6 +41,15 @@
 	// JBitSet to hold the table numbers that we find.
 	private JBitSet tableMap;
 
+	/* Column number of the ColumnReference or ResultColumn
+	 * for which we most recently found a base table number. 
+	 * In cases where this visitor is only expected to find
+	 * a single base table number, this field is useful for
+	 * determining what the column position w.r.t. the found
+	 * base table was.
+	 */
+	private int columnNumber;
+
 	/**
 	 * Constructor: takes a JBitSet to use as the holder for any base table
 	 * numbers found while walking the subtree.
@@ -50,6 +59,7 @@
 	public BaseTableNumbersVisitor(JBitSet tableMap)
 	{
 		this.tableMap = tableMap;
+		columnNumber = -1;
 	}
 
 	/**
@@ -63,6 +73,25 @@
 		this.tableMap = tableMap;
 	}
 
+	/**
+	 * Reset the state of this visitor.
+	 */
+	protected void reset()
+	{
+		tableMap.clearAll();
+		columnNumber = -1;
+	}
+
+	/**
+	 * Retrieve the the position of the ColumnReference or
+	 * ResultColumn for which we most recently found a base
+	 * table number.
+	 */
+	protected int getColumnNumber()
+	{
+		return columnNumber;
+	}
+
 	////////////////////////////////////////////////
 	//
 	// VISITOR INTERFACE
@@ -96,7 +125,7 @@
 			}
 		}
 		else if (node instanceof ResultColumn)
-			rc = (ResultColumn)rc;
+			rc = (ResultColumn)node;
 		else if (node instanceof SelectNode)
 		{
 			// If the node is a SelectNode we just need to look at its
@@ -143,13 +172,16 @@
 				// have pulled it from a VirtualColumnNode's source
 				// table); so just set the number.
 					tableMap.set(baseTableNumber);
+					columnNumber = rc.getColumnPosition();
 				}
 			}
 			else if (node instanceof ColumnReference) {
 			// we couldn't find any other table numbers beneath the
 			// ColumnReference, so just use the table number for
 			// that reference.
-				((ColumnReference)node).getTablesReferenced(tableMap);
+				ColumnReference cr = (ColumnReference)node;
+				cr.getTablesReferenced(tableMap);
+				columnNumber = cr.getColumnNumber();
 			}
 		}
 

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=616126&r1=616125&r2=616126&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 Mon Jan 28 16:58:37 2008
@@ -40,6 +40,7 @@
 
 import org.apache.derby.iapi.error.StandardException;
 
+import org.apache.derby.iapi.sql.compile.C_NodeTypes;
 import org.apache.derby.iapi.sql.compile.CompilerContext;
 import org.apache.derby.iapi.sql.compile.OptimizablePredicateList;
 import org.apache.derby.iapi.sql.compile.Optimizer;
@@ -48,8 +49,8 @@
 import org.apache.derby.iapi.sql.compile.CostEstimate;
 import org.apache.derby.iapi.sql.compile.AccessPath;
 import org.apache.derby.iapi.sql.compile.JoinStrategy;
+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;
 import org.apache.derby.iapi.sql.compile.Visitable;
 import org.apache.derby.iapi.sql.compile.Visitor;
 
@@ -4159,6 +4160,26 @@
 		 * FBT being under a PRN, etc.
 		 * So, we just ignore this call for now.
 		 */
+	}
+
+	/**
+	 * @see ResultSetNode#adjustForSortElimination
+	 */
+	void adjustForSortElimination(RequiredRowOrdering rowOrdering)
+		throws StandardException
+	{
+		/* We may have eliminated a sort with the assumption that
+		 * the rows from this base table will naturally come back
+		 * in the correct ORDER BY order. But in the case of IN
+		 * list probing predicates (see DERBY-47) the predicate
+		 * itself may affect the order of the rows.  In that case
+		 * we need to notify the predicate so that it does the
+		 * right thing--i.e. so that it preserves the natural
+		 * ordering of the rows as expected from this base table.
+		 * DERBY-3279.
+		 */
+		if (restrictionList != null)
+			restrictionList.adjustForSortElimination(rowOrdering);
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java Mon Jan 28 16:58:37 2008
@@ -52,6 +52,7 @@
 public final class InListOperatorNode extends BinaryListOperatorNode
 {
 	private boolean isOrdered;
+	private boolean sortDescending;
 
 	/**
 	 * Initializer for a InListOperatorNode
@@ -103,6 +104,9 @@
 		if (isOrdered)
 			ilon.markAsOrdered();
 
+		if (sortDescending)
+			ilon.markSortDescending();
+
 		return ilon;
 	}
 
@@ -739,6 +743,19 @@
 	}
 
 	/**
+	 * Indicate that the IN-list values for this node must be sorted
+	 * in DESCENDING order.  This only applies to in-list "multi-probing",
+	 * where the rows are processed in the order of the IN list elements
+	 * themselves.  In that case, any requirement to sort the rows in
+	 * descending order means that the values in the IN list have to
+	 * be sorted in descending order, as well.
+	 */
+	protected void markSortDescending()
+	{
+		sortDescending = true;
+	}
+
+	/**
 	 * Return whether or not the IN-list values for this node are ordered.
 	 * This is used for determining whether or not we need to do an execution-
 	 * time sort.
@@ -746,5 +763,14 @@
 	protected boolean isOrdered()
 	{
 		return isOrdered;
+	} 
+
+	/**
+	 * Return whether or not the IN-list values for this node must be
+	 * sorted in DESCENDING order.
+	 */
+	protected boolean sortDescending()
+	{
+		return sortDescending;
 	} 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrderByList.java Mon Jan 28 16:58:37 2008
@@ -634,4 +634,118 @@
 	{
 		return sortNeeded;
 	}
+
+	/**
+	 * Determine whether or not this RequiredRowOrdering has a
+	 * DESCENDING requirement for the column referenced by the
+	 * received ColumnReference.
+	 */
+	boolean requiresDescending(ColumnReference cRef, int numOptimizables)
+		throws StandardException
+	{
+		int size = size();
+
+		/* Start by getting the table number and column position for
+		 * the table to which the ColumnReference points.
+		 */
+		JBitSet tNum = new JBitSet(numOptimizables);
+		BaseTableNumbersVisitor btnVis = new BaseTableNumbersVisitor(tNum);
+
+		cRef.accept(btnVis);
+		int crTableNumber = tNum.getFirstSetBit();
+		int crColPosition = btnVis.getColumnNumber();
+
+		if (SanityManager.DEBUG)
+		{
+			/* We assume that we only ever get here if the column
+			 * reference points to a specific column in a specific
+			 * table...
+			 */
+			if ((crTableNumber < 0) || (crColPosition < 0))
+			{
+				SanityManager.THROWASSERT(
+					"Failed to find table/column number for column '" +
+					cRef.getColumnName() + "' when checking for an " +
+					"ORDER BY requirement.");
+			}
+
+			/* Since we started with a single ColumnReference there
+			 * should be exactly one table number.
+			 */
+			if (!tNum.hasSingleBitSet())
+			{
+				SanityManager.THROWASSERT(
+					"Expected ColumnReference '" + cRef.getColumnName() +
+					"' to reference exactly one table, but tables found " +
+					"were: " + tNum);
+			}
+		}
+
+		/* Walk through the various ORDER BY elements to see if
+		 * any of them point to the same table and column that
+		 * we found above.
+		 */
+		for (int loc = 0; loc < size; loc++)
+		{
+			OrderByColumn obc = getOrderByColumn(loc);
+			ResultColumn rcOrderBy = obc.getResultColumn();
+
+			btnVis.reset();
+			rcOrderBy.accept(btnVis);
+			int obTableNumber = tNum.getFirstSetBit();
+			int obColPosition = btnVis.getColumnNumber();
+
+			/* ORDER BY target should always have a table number and
+			 * a column position.  It may not necessarily be a base
+			 * table, but there should be some FromTable for which
+			 * we have a ResultColumnList, and the ORDER BY should
+			 * reference one of the columns in that list (otherwise
+			 * we shouldn't have made it this far).
+			 */
+			if (SanityManager.DEBUG)
+			{
+				/* Since we started with a single ResultColumn there
+				 * should exactly one table number.
+				 */
+				if (!tNum.hasSingleBitSet())
+				{
+					SanityManager.THROWASSERT("Expected ResultColumn '" +
+						rcOrderBy.getColumnName() + "' to reference " +
+						"exactly one table, but found: " + tNum);
+				}
+
+				if (obColPosition < 0)
+				{
+					SanityManager.THROWASSERT(
+						"Failed to find orderBy column number " +
+						"for ORDER BY check on column '" + 
+						cRef.getColumnName() + "'.");
+				}
+			}
+
+			if (crTableNumber != obTableNumber)
+				continue;
+
+			/* They point to the same base table, so check the
+			 * column positions.
+			 */
+
+			if (crColPosition == obColPosition)
+			{
+				/* This ORDER BY element points to the same table
+				 * and column as the received ColumnReference.  So
+				 * return whether or not this ORDER BY element is
+				 * descending.
+				 */
+				return !obc.isAscending();
+			}
+		}
+
+		/* None of the ORDER BY elements referenced the same table
+		 * and column as the received ColumnReference, so there
+		 * is no descending requirement for the ColumnReference's
+		 * source (at least not from this OrderByList).
+		 */
+		return false;
+	}
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java Mon Jan 28 16:58:37 2008
@@ -36,6 +36,8 @@
 import org.apache.derby.iapi.sql.compile.OptimizablePredicate;
 import org.apache.derby.iapi.sql.compile.OptimizablePredicateList;
 import org.apache.derby.iapi.sql.compile.Optimizable;
+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.compile.C_NodeTypes;
 
@@ -2919,7 +2921,28 @@
 
 			InListOperatorNode ilon = pred.getSourceInList();
 			mb.getField(ilon.generateListAsArray(acb, mb));
-			mb.push(ilon.isOrdered());
+
+			if (ilon.sortDescending())
+				mb.push(RowOrdering.DESCENDING);
+			else if (!ilon.isOrdered())
+			{
+				/* If there is no requirement to sort descending and the
+				 * IN list values have not already been sorted, then we
+				 * sort them in ascending order at execution time.
+				 */
+				mb.push(RowOrdering.ASCENDING);
+			}
+			else
+			{
+				/* DONTCARE here means we don't have to sort the IN
+				 * values at execution time because we already did
+				 * it as part of compilation (esp. preprocessing).
+				 * This can only be the case if all values in the IN
+				 * list are literals (as opposed to parameters).
+				 */
+				mb.push(RowOrdering.DONTCARE);
+			}
+
 			return;
 		}
 
@@ -3626,6 +3649,48 @@
 		return retval;
 	}
 	
+	/**
+	 * @see OptimizablePredicateList#adjustForSortElimination
+	 *
+	 * Currently this method only accounts for IN list multi-probing
+	 * predicates (DERBY-3279).
+	 */
+	public void adjustForSortElimination(
+		RequiredRowOrdering ordering) throws StandardException
+	{
+		// Nothing to do if there's no required ordering. 
+		if (ordering == null)
+			return;
+
+		/* Walk through the predicate list and search for any
+		 * multi-probing predicates.  If we find any which
+		 * operate on a column that is part of the received
+		 * ORDER BY, then check to see if the ORDER BY requires
+		 * a DESCENDING sort.  If so, then we must take note
+		 * of this requirement so that the IN list values for
+		 * the probe predicate are sorted in DESCENDING order
+		 * at execution time.
+		 */
+		int size = size();
+		OrderByList orderBy = (OrderByList)ordering;
+		for (int index = 0; index < size; index++)
+		{
+			Predicate pred = (Predicate) elementAt(index);
+			if (!pred.isInListProbePredicate())
+				continue;
+
+			BinaryRelationalOperatorNode bron =
+				(BinaryRelationalOperatorNode)pred.getRelop();
+
+			if (orderBy.requiresDescending(
+				(ColumnReference)bron.getLeftOperand(),
+				pred.getReferencedSet().size()))
+			{
+				pred.getSourceInList(true).markSortDescending();
+			}
+		}
+	}
+		
 	/** 
 	 * @see OptimizablePredicateList#selectivity
 	 */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultSetNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultSetNode.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultSetNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultSetNode.java Mon Jan 28 16:58:37 2008
@@ -1901,6 +1901,19 @@
 		}
 	}
 
+	/**
+	 * Same goal as adjustForSortElimination above, but this version
+	 * takes a RequiredRowOrdering to allow nodes to adjust based on
+	 * the ORDER BY clause, if needed.
+	 */
+	void adjustForSortElimination(RequiredRowOrdering rowOrdering)
+		throws StandardException
+	{
+		/* Default is to ignore the row ordering; subclasses must
+		 * override if they need to use it.
+		 */
+		adjustForSortElimination();
+	}
 
 	/**
 	 * Count the number of distinct aggregates in the list.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java Mon Jan 28 16:58:37 2008
@@ -1422,7 +1422,7 @@
 		 */
 		if (eliminateSort)
 		{
-			prnRSN.adjustForSortElimination();
+			prnRSN.adjustForSortElimination(orderByList);
 		}
 
 		/* Set the cost of this node in the generated node */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SingleChildResultSetNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SingleChildResultSetNode.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SingleChildResultSetNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SingleChildResultSetNode.java Mon Jan 28 16:58:37 2008
@@ -577,6 +577,15 @@
 	}
 
 	/**
+	 * @see ResultSetNode#adjustForSortElimination
+	 */
+	void adjustForSortElimination(RequiredRowOrdering rowOrdering)
+		throws StandardException
+	{
+		childResult.adjustForSortElimination(rowOrdering);
+	}
+
+	/**
 	 * Get the final CostEstimate for this node.
 	 *
 	 * @return	The final CostEstimate for this node, which is

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TableOperatorNode.java Mon Jan 28 16:58:37 2008
@@ -938,6 +938,16 @@
 	}
 
 	/**
+	 * @see ResultSetNode#adjustForSortElimination
+	 */
+	void adjustForSortElimination(RequiredRowOrdering rowOrdering)
+		throws StandardException
+	{
+		leftResultSet.adjustForSortElimination(rowOrdering);
+		rightResultSet.adjustForSortElimination(rowOrdering);
+	}
+
+	/**
 	 * Accept a visitor, and call v.visit()
 	 * on child nodes as necessary.  
 	 * 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java Mon Jan 28 16:58:37 2008
@@ -735,7 +735,7 @@
 									boolean sameStartStopPosition,
 									Qualifier[][] qualifiers,
 									DataValueDescriptor [] probeVals,
-									boolean probeValsAreSorted,
+									int sortRequired,
 									String tableName,
 									String userSuppliedOptimizerOverrides,
 									String indexName,
@@ -767,7 +767,7 @@
 								sameStartStopPosition,
 								qualifiers,
 								probeVals,
-								probeValsAreSorted,
+								sortRequired,
 								tableName,
 								userSuppliedOptimizerOverrides,
 								indexName,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java Mon Jan 28 16:58:37 2008
@@ -31,6 +31,7 @@
 import org.apache.derby.iapi.store.access.TransactionController;
 
 import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.compile.RowOrdering;
 import org.apache.derby.iapi.sql.execute.CursorResultSet;
 import org.apache.derby.iapi.sql.execute.ExecRow;
 
@@ -81,11 +82,11 @@
     protected int probeValIndex;
 
     /**
-     * Whether or not we need to sort the values.  If all values were
-     * specified as literals (as opposed to parameters) then we did the
-     * sort at compile time and so we do not need to do it here.
+     * Indicator as to which type of sort we need: ASCENDING, DESCENDING,
+     * or NONE (NONE is represented by "RowOrdering.DONTCARE" and is used
+     * for cases where all necessary sorting occurred at compilation time).
      */
-    private boolean needSort;
+    private int sortRequired;
 
     /**
      * Constructor.  Just save off the relevant probing state and pass
@@ -103,7 +104,7 @@
         boolean sameStartStopPosition,
         Qualifier[][] qualifiers,
         DataValueDescriptor [] probingVals,
-        boolean probeValsAreSorted,
+        int sortRequired,
         String tableName,
         String userSuppliedOptimizerOverrides,
         String indexName,
@@ -157,7 +158,7 @@
         }
 
         this.origProbeValues = probingVals;
-        this.needSort = !probeValsAreSorted;
+        this.sortRequired = sortRequired;
     }
 
     /**
@@ -175,7 +176,14 @@
          * have to do the sort here, at execution time, because this is the
          * only point at which we know what values the parameters have.
          */
-        if (needSort)
+        if (sortRequired == RowOrdering.DONTCARE)
+        {
+            /* DONTCARE really means that the values are already sorted
+             * in ascending order, and that's good enough.
+             */
+            probeValues = origProbeValues;
+        }
+        else
         {
             /* RESOLVE: For some reason sorting the probeValues array
              * directly leads to incorrect parameter value assignment when
@@ -190,11 +198,17 @@
             for (int i = 0; i < pVals.length; i++)
                 pVals[i] = origProbeValues[i].getClone();
 
-            java.util.Arrays.sort(pVals);
+            if (sortRequired == RowOrdering.ASCENDING)
+                java.util.Arrays.sort(pVals);
+            else
+            {
+                // Sort the values in DESCENDING order.
+                java.util.Arrays.sort(
+                    pVals, java.util.Collections.reverseOrder());
+            }
+
             probeValues = pVals;
         }
-        else
-            probeValues = origProbeValues;
 
         probeValIndex = 0;
         super.openCore();
@@ -369,9 +383,10 @@
 
     /**
      * Return the next non-duplicate value from the probe list.
-     * Assumption is that the list is sorted in ascending order
-     * and that probeValIndex is the index of the next value.
-     * If we've exhausted the probe list then just return null.
+     * Assumption is that the list is sorted so that duplicates
+     * appear next to each other, and that probeValIndex is the
+     * index of the next value. If we've exhausted the probe list
+     * then just return null.
      */
     private DataValueDescriptor getNextProbeValue()
         throws StandardException

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/InListMultiProbeTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/InListMultiProbeTest.java?rev=616126&r1=616125&r2=616126&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/InListMultiProbeTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/InListMultiProbeTest.java Mon Jan 28 16:58:37 2008
@@ -546,6 +546,175 @@
     }
 
     /**
+     * When IN list multi-probing occurs, the rows from the underlying
+     * table are returned in the order of the values in the *IN list*,
+     * instead of in the order of the rows as they are returned from
+     * the index scan.  So if the index is defined as "DESC" and we
+     * eliminate an ORDER BY ... DESC sort during optimization, we
+     * have to sort the IN list values in descending order to make up
+     * for the eliminated sort.  DERBY-3279.
+     */
+    public void testInListProbingWithOrderBy() throws SQLException
+    {
+        Statement st = createStatement();
+
+        st.execute("create table CHEESE (CHEESE_CODE VARCHAR(5), " +
+            "CHEESE_NAME VARCHAR(20), CHEESE_COST DECIMAL(7,4))");
+
+        st.execute("create index cheese_index on CHEESE " +
+            "(CHEESE_CODE DESC, CHEESE_NAME DESC, CHEESE_COST DESC)");
+
+        st.execute(
+            "INSERT INTO CHEESE (CHEESE_CODE, CHEESE_NAME, CHEESE_COST) " +
+            "VALUES ('00000', 'GOUDA', 001.1234), ('00000', 'EDAM', " +
+            "002.1111), ('54321', 'EDAM', 008.5646), ('12345', " +
+            "'GORGONZOLA', 888.2309), ('AAAAA', 'EDAM', 999.8888), " +
+            "('54321', 'MUENSTER', 077.9545)");
+
+        /* ORDER BY is DESC, so we'll eliminate the ORDER BY sort for
+         * this query.  Results should still come back in descending
+         * order, though.
+         */
+
+        String [][] expRS1 =
+            new String [][] {
+                {"AAAAA", "EDAM", "999.8888"},
+                {"54321", "EDAM", "8.5646"},
+                {"00000", "EDAM", "2.1111"}
+            };
+
+        JDBC.assertFullResultSet(st.executeQuery(
+            "SELECT * FROM CHEESE " +
+            "WHERE (CHEESE_CODE='54321' OR CHEESE_CODE='00000' " +
+            "OR CHEESE_CODE='AAAAA') AND CHEESE_NAME='EDAM' " +
+            "ORDER BY CHEESE_CODE DESC, CHEESE_NAME DESC, CHEESE_COST DESC"),
+            expRS1);
+
+        /* ORDER BY is ASC so we will not eliminate the sort; make
+         * sure the rows are still correctly ordered.
+         */
+
+        String [][] expRS2 =
+            new String [][] {
+                {"00000", "EDAM", "2.1111"},
+                {"54321", "EDAM", "8.5646"},
+                {"AAAAA", "EDAM", "999.8888"}
+            };
+
+        JDBC.assertFullResultSet(st.executeQuery(
+            "SELECT * FROM CHEESE " +
+            "WHERE (CHEESE_CODE='54321' OR CHEESE_CODE='00000' " +
+            "OR CHEESE_CODE='AAAAA') AND CHEESE_NAME='EDAM' " +
+            "ORDER BY CHEESE_CODE ASC, CHEESE_NAME DESC, CHEESE_COST DESC"),
+            expRS2);
+
+        /* Simple join where the ORDER BY is based on position in
+         * the RCL and the probe predicate is w.r.t. the second
+         * table in the FROM list.  In this case the ORDER BY's
+         * immediate column position is "4" while the probe pred's
+         * immediate column position is "1"; but underneath we
+         * should still be able to figure out that they are pointing
+         * to the same column and thus do the correct sorting.
+         */
+
+        String [][] expRS3 =
+            new String [][] {
+                {"AAAAA", "EDAM", "999.8888", "54321", "EDAM", "8.5646"},
+                {"54321", "EDAM", "8.5646", "54321", "EDAM", "8.5646"},
+                {"00000", "EDAM", "2.1111", "54321", "EDAM", "8.5646"},
+                {"AAAAA", "EDAM", "999.8888", "00000", "EDAM", "2.1111"},
+                {"54321", "EDAM", "8.5646", "00000", "EDAM", "2.1111"},
+                {"00000", "EDAM", "2.1111", "00000", "EDAM", "2.1111"}
+            };
+
+        JDBC.assertFullResultSet(st.executeQuery(
+            "SELECT * FROM CHEESE C1, CHEESE C2 " +
+            "WHERE C1.CHEESE_NAME = C2.CHEESE_NAME AND " +
+            "(C2.CHEESE_CODE='00000' OR C2.CHEESE_CODE='54321') " +
+            "AND C1.CHEESE_NAME='EDAM' ORDER BY 4 DESC, 5 DESC, 6 DESC"),
+            expRS3);
+
+        // Same as previous query but with ASC in the ORDER BY.
+
+        String [][] expRS4 =
+            new String [][] {
+                {"00000", "EDAM", "2.1111", "00000", "EDAM", "2.1111"},
+                {"54321", "EDAM", "8.5646", "00000", "EDAM", "2.1111"},
+                {"AAAAA", "EDAM", "999.8888", "00000", "EDAM", "2.1111"},
+                {"00000", "EDAM", "2.1111", "54321", "EDAM", "8.5646"},
+                {"54321", "EDAM", "8.5646", "54321", "EDAM", "8.5646"},
+                {"AAAAA", "EDAM", "999.8888", "54321", "EDAM", "8.5646"}
+            };
+
+        JDBC.assertFullResultSet(st.executeQuery(
+            "SELECT * FROM CHEESE C1, CHEESE C2 " +
+            "WHERE C1.CHEESE_NAME = C2.CHEESE_NAME AND " +
+            "(C2.CHEESE_CODE='00000' OR C2.CHEESE_CODE='54321') " +
+            "AND C1.CHEESE_NAME='EDAM' ORDER BY 4 ASC, 5 DESC, 6 DESC"),
+            expRS4);
+
+        /* Repeat the tests with parameter markers instead of literals,
+         * and explicit IN lists instead of an OR clause that would
+         * get transformed into an IN list.
+         */
+
+        /* ORDER BY is DESC, so we'll eliminate the ORDER BY sort for
+         * this query.  Results should still come back in descending
+         * order, though.
+         */
+        PreparedStatement ps = prepareStatement("SELECT * FROM CHEESE " +
+            "WHERE CHEESE_CODE IN (?,?,?) AND CHEESE_NAME='EDAM' " +
+            "ORDER BY CHEESE_CODE DESC, CHEESE_NAME DESC, CHEESE_COST DESC");
+
+        ps.setString(1, "00000");
+        ps.setString(2, "AAAAA");
+        ps.setString(3, "54321");
+        JDBC.assertFullResultSet(ps.executeQuery(), expRS1);
+
+        /* ORDER BY is ASC so we will not eliminate the sort; make
+         * sure the rows are still correctly ordered.
+         */
+
+        ps = prepareStatement("SELECT * FROM CHEESE " +
+            "WHERE CHEESE_CODE IN (?,?,?) AND CHEESE_NAME='EDAM' " +
+            "ORDER BY CHEESE_CODE ASC, CHEESE_NAME DESC, CHEESE_COST DESC");
+
+        ps.setString(1, "00000");
+        ps.setString(2, "AAAAA");
+        ps.setString(3, "54321");
+        JDBC.assertFullResultSet(ps.executeQuery(), expRS2);
+
+        /* Simple join where the ORDER BY is based on position in
+         * the RCL and the probe predicate is w.r.t. to the second
+         * table in the FROM list.
+         */
+
+        ps = prepareStatement("SELECT * FROM CHEESE C1, CHEESE C2 " +
+            "WHERE C1.CHEESE_NAME = C2.CHEESE_NAME AND " +
+            "C2.CHEESE_CODE IN (?,?) AND C1.CHEESE_NAME='EDAM' " +
+            "ORDER BY 4 DESC, 5 DESC, 6 DESC");
+
+        ps.setString(1, "00000");
+        ps.setString(2, "54321");
+        JDBC.assertFullResultSet(ps.executeQuery(), expRS3);
+
+        // Same as previous query but with ASC in the ORDER BY.
+
+        ps = prepareStatement("SELECT * FROM CHEESE C1, CHEESE C2 " +
+            "WHERE C1.CHEESE_NAME = C2.CHEESE_NAME AND " +
+            "C2.CHEESE_CODE IN (?,?) AND C1.CHEESE_NAME='EDAM' " +
+            "ORDER BY 4 ASC, 5 ASC, 6 ASC");
+
+        ps.setString(1, "00000");
+        ps.setString(2, "54321");
+        JDBC.assertFullResultSet(ps.executeQuery(), expRS4);
+
+        ps.close();
+        st.execute("drop table cheese");
+        st.close();
+    }
+
+    /**
      * Insert the received number of rows into DATA_TABLE via
      * batch processing.
      */