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 2007/03/14 21:53:17 UTC

svn commit: r518322 - in /db/derby/code/trunk/java: engine/org/apache/derby/impl/sql/compile/ testing/org/apache/derbyTesting/functionTests/master/

Author: abrown
Date: Wed Mar 14 13:53:16 2007
New Revision: 518322

URL: http://svn.apache.org/viewvc?view=rev&rev=518322
Log:
DERBY-47 (partial): Patch that adds logic to create IN-list "probe predicates"
during preprocessing, thus allowing the code changes in all previous patches
for this issue to take effect. With this patch Derby will now re-write IN
lists as probe predicates and, if the optimizer thinks it is best to do so,
will do index "multi-probing" at execution time to avoid excessive scanning.
The changes in this patch affect "preprocessing" logic as follow:

  1. Replaces "A" with "B", where "A" is existing logic that creates a BETWEEN
     node for IN-lists containing all constants, and "B" is new logic that
     creates a "probe predicate" for IN-lists containing all constants *and/or*
     parameter nodes. The probe predicates are then used throughout optimization,
     modification of access paths, code generation, and execution time (as
     appropriate) in the manner described by previous patches for this issue.

  2. Adds some additional logic to OrNode preprocessing to allow the conversion
     of queries like:

        select ... from T1 where i in (2, 3) or i in (7, 10)

     into queries that look like:

        select ... from T1 where i in (2, 3, 7, 10)

     This is really just an extension of the existing logic to transform a
     chain of OR nodes into an IN-list.

  3. Adds logic to PredicateList.pushExpressionsIntoSelect() to correctly
     copy "probe predicates" so that the left operand (column reference)
     is pointing to the correct place when we do static pushing of one-
     sided predicates (which is what a "probe predicate" is).

  4. Adds a new method to ValueNodeList that is used for checking to see if
     a list of IN values consists solely of constant and/or parameter nodes
     (there are no other expressions or column references).

This patch also incorporates all of the test master updates required as a
result of the new multi-probing functionality.

Modified:
    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/OrNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.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/ValueNodeList.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatePushdown.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/readlocks.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subquery.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subqueryFlattening.out

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?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- 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 Wed Mar 14 13:53:16 2007
@@ -31,6 +31,7 @@
 import org.apache.derby.iapi.reference.ClassName;
 
 import org.apache.derby.iapi.types.TypeId;
+import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.DataValueDescriptor;
 
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
@@ -89,6 +90,27 @@
 	}
 
 	/**
+	 * Create a shallow copy of this InListOperatorNode whose operands are
+	 * the same as this node's operands.  Copy over all other necessary
+	 * state, as well.
+	 */
+	protected InListOperatorNode shallowCopy() throws StandardException
+	{
+		InListOperatorNode ilon =
+			 (InListOperatorNode)getNodeFactory().getNode(
+				C_NodeTypes.IN_LIST_OPERATOR_NODE,
+				leftOperand,
+				rightOperandList,
+				getContextManager());
+
+		ilon.copyFields(this);
+		if (isOrdered)
+			ilon.markAsOrdered();
+
+		return ilon;
+	}
+
+	/**
 	 * Preprocess an expression tree.  We do a number of transformations
 	 * here (including subqueries, IN lists, LIKE and BETWEEN) plus
 	 * subquery flattening.
@@ -129,95 +151,190 @@
 			return equal;
 		}
 		else if ((leftOperand instanceof ColumnReference) &&
-				 rightOperandList.containsAllConstantNodes())
+				 rightOperandList.containsOnlyConstantAndParamNodes())
 		{
-			/* When sorting or choosing min/max in the list, if types are not an exact
-			 * match, we use the left operand's type as the "judge", assuming that they
-			 * are compatible, as also the case with DB2.
-			 */
-			TypeId judgeTypeId = leftOperand.getTypeServices().getTypeId();
-			DataValueDescriptor judgeODV = null;  //no judge, no argument
-			if (! rightOperandList.allSamePrecendence(judgeTypeId.typePrecedence()))
-				judgeODV = (DataValueDescriptor) judgeTypeId.getNull();
-
-			//Sort the list in ascending order
-			rightOperandList.sortInAscendingOrder(judgeODV);
-			isOrdered = true;
-
-			/* If the leftOperand is a ColumnReference
-			 * and the IN list is all constants, then we generate
-			 * an additional BETWEEN clause of the form:
-			 *	CRClone BETWEEN minValue and maxValue
+			/* At this point we have an IN-list made up of constant and/or
+			 * parameter values.  Ex.:
+			 *
+			 *  select id, name from emp where id in (34, 28, ?)
+			 *
+			 * Since the optimizer does not recognize InListOperatorNodes
+			 * as potential start/stop keys for indexes, it (the optimizer)
+			 * may estimate that the cost of using any of the indexes would
+			 * be too high.  So we could--and probably would--end up doing
+			 * a table scan on the underlying base table. But if the number
+			 * of rows in the base table is significantly greater than the
+			 * number of values in the IN-list, scanning the base table can
+			 * be overkill and can lead to poor performance.  And further,
+			 * choosing to use an index but then scanning the entire index
+			 * can be slow, too. DERBY-47.
+			 *
+			 * What we do, then, is create an "IN-list probe predicate",
+			 * which is an internally generated equality predicate with a
+			 * parameter value on the right.  So for the query shown above
+			 * the probe predicate would be "id = ?".  We then replace
+			 * this InListOperatorNode with the probe predicate during
+			 * optimization.  The optimizer in turn recognizes the probe
+			 * predicate, which is disguised to look like a typical binary
+			 * equality, as a potential start/stop key for any indexes.
+			 * This start/stop key potential then factors into the estimated
+			 * cost of probing the indexes, which leads to a more reasonable
+			 * estimate and thus makes it more likely that the optimizer
+			 * will choose to use an index vs a table scan.  That done, we
+			 * then use the probe predicate to perform multiple execution-
+			 * time "probes" on the index--instead of doing a range index
+			 * scan--which eliminates unnecessary scanning. For more see
+			 * execute/MultiProbeTableScanResultSet.java.
+			 *
+			 * With this approach we know that regardless of how large the
+			 * base table is, we'll only have to probe the index a max of
+			 * N times, where "N" is the size of the IN-list. If N is
+			 * significantly less than the number of rows in the table, or
+			 * is significantly less than the number of rows between the
+			 * min value and the max value in the IN-list, this selective
+			 * probing can save us a lot of time.
+			 *
+			 * Note: We will do fewer than N probes if there are duplicates
+			 * in the list.
+			 *
+			 * Note also that, depending on the relative size of the IN-list
+			 * verses the number of rows in the table, it may actually be
+			 * better to just do a table scan--especially if there are fewer
+			 * rows in the table than there are in the IN-list.  So even though
+			 * we create a "probe predicate" and pass it to the optimizer, it
+			 * (the optimizer) may still choose to do a table scan.  If that
+			 * happens then we'll "revert" the probe predicate back to its
+			 * original form (i.e. to this InListOperatorNode) during code
+			 * generation, and then we'll use it as a regular IN-list
+			 * restriction when it comes time to execute.
 			 */
-			ValueNode leftClone = leftOperand.getClone();
-			ValueNode minValue = (ValueNode) rightOperandList.elementAt(0);  //already sorted
-			ValueNode maxValue = (ValueNode) rightOperandList.elementAt(rightOperandList.size() - 1);
 
-			/* Handle the degenerate case where 
-			 * the min and the max are the same value.
+			boolean allConstants = rightOperandList.containsAllConstantNodes();
+
+			/* If we have all constants then sort them now.  This allows us to
+			 * skip the sort at execution time (we have to sort them so that
+			 * we can eliminate duplicate IN-list values).  If we have one
+			 * or more parameter nodes then we do *not* sort the values here
+			 * because we do not (and cannot) know what values the parameter(s)
+			 * will have.  In that case we'll sort the values at execution
+			 * time. 
 			 */
-			DataValueDescriptor minODV =
-				 ((ConstantNode) minValue).getValue();
-			DataValueDescriptor maxODV =
-				 ((ConstantNode) maxValue).getValue();
-			if ((judgeODV == null && minODV.compare(maxODV) == 0) ||
-				(judgeODV != null && judgeODV.equals(minODV, maxODV).equals(true)))
+			if (allConstants)
 			{
-				BinaryComparisonOperatorNode equal = 
-					(BinaryComparisonOperatorNode) getNodeFactory().getNode(
-						C_NodeTypes.BINARY_EQUALS_OPERATOR_NODE,
-						leftOperand, 
-						minValue,
-						getContextManager());
-				/* Set type info for the operator node */
-				equal.bindComparisonOperator();
-				return equal;
+				/* When sorting or choosing min/max in the list, if types
+				 * are not an exact match, we use the left operand's type
+				 * as the "judge", assuming that they are compatible, as
+				 * also the case with DB2.
+				 */
+				TypeId judgeTypeId = leftOperand.getTypeServices().getTypeId();
+				DataValueDescriptor judgeODV = null;  //no judge, no argument
+				if (!rightOperandList.allSamePrecendence(
+					judgeTypeId.typePrecedence()))
+				{
+					judgeODV = (DataValueDescriptor) judgeTypeId.getNull();
+				}
+ 
+				// Sort the list in ascending order
+				rightOperandList.sortInAscendingOrder(judgeODV);
+				isOrdered = true;
+
+				ValueNode minValue = (ValueNode)rightOperandList.elementAt(0);
+				ValueNode maxValue =
+					(ValueNode)rightOperandList.elementAt(
+						rightOperandList.size() - 1);
+
+				/* Handle the degenerate case where the min and the max
+				 * are the same value.
+				 */
+				DataValueDescriptor minODV =
+					((ConstantNode) minValue).getValue();
+				DataValueDescriptor maxODV =
+					 ((ConstantNode) maxValue).getValue();
+
+				if (((judgeODV == null) && (minODV.compare(maxODV) == 0)) ||
+					((judgeODV != null)
+						&& judgeODV.equals(minODV, maxODV).equals(true)))
+				{
+					BinaryComparisonOperatorNode equal = 
+						(BinaryComparisonOperatorNode)getNodeFactory().getNode(
+							C_NodeTypes.BINARY_EQUALS_OPERATOR_NODE,
+							leftOperand, 
+							minValue,
+							getContextManager());
+					/* Set type info for the operator node */
+					equal.bindComparisonOperator();
+					return equal;
+				}
 			}
 
-			// Build the Between
-			ValueNodeList vnl = (ValueNodeList) getNodeFactory().getNode(
-													C_NodeTypes.VALUE_NODE_LIST,
-													getContextManager());
-			vnl.addValueNode(minValue);
-			vnl.addValueNode(maxValue);
-
-			BetweenOperatorNode bon = 
-				(BetweenOperatorNode) getNodeFactory().getNode(
-									C_NodeTypes.BETWEEN_OPERATOR_NODE,
-									leftClone,
-									vnl,
-									getContextManager());
-
-			/* The transformed tree has to be normalized:
-			 *				AND
-			 *			   /   \
-			 *		IN LIST    AND
-			 *				   /   \
-			 *				  >=	AND
-			 *						/   \
-			 *					   <=	TRUE
+			/* Create a parameter node to serve as the right operand of
+			 * the probe predicate.  We intentionally use a parameter node
+			 * instead of a constant node because the IN-list has more than
+			 * one value (some of which may be unknown at compile time, i.e.
+			 * if they are parameters), so we don't want an estimate based
+			 * on any single literal.  Instead we want a generic estimate
+			 * of the cost to retrieve the rows matching some _unspecified_
+			 * value (namely, one of the values in the IN-list, but we
+			 * don't know which one).  That's exactly what a parameter
+			 * node gives us.
+			 *
+			 * Note: If the IN-list only had a single value then we would
+			 * have taken the "if (rightOperandList.size() == 1)" branch
+			 * above and thus would not be here.
+			 *
+			 * We create the parameter node based on the first value in
+			 * the list.  This is arbitrary and should not matter in the
+			 * big picture.
 			 */
+			ValueNode srcVal = (ValueNode) rightOperandList.elementAt(0);
+			ParameterNode pNode =
+				(ParameterNode) getNodeFactory().getNode(
+					C_NodeTypes.PARAMETER_NODE,
+					new Integer(0),
+					null, // default value
+					getContextManager());
+
+			DataTypeDescriptor pType = srcVal.getTypeServices();
+			pNode.setDescriptors(new DataTypeDescriptor [] { pType });
+			pNode.setType(pType);
+
+			/* If we choose to use the new predicate for execution-time
+			 * probing then the right operand will function as a start-key
+			 * "place-holder" into which we'll store the different IN-list
+			 * values as we iterate through them.  This means we have to
+			 * generate a valid value for the parameter node--i.e. for the
+			 * right side of the probe predicate--in order to have a valid
+			 * execution-time placeholder.  To do that we pass the source
+			 * value from which we found the type down to the new, "fake"
+			 * parameter node.  Then, when it comes time to generate the
+			 * parameter node, we'll just generate the source value as our
+			 * place-holder.  See ParameterNode.generateExpression().
+			 *
+			 * Note: the actual value of the "place-holder" does not matter
+			 * because it will be clobbered by the various IN-list values
+			 * (which includes "srcVal" itself) as we iterate through them
+			 * during execution.
+			 */
+			pNode.setValueToGenerate(srcVal);
 
-			/* Create the AND */
-			AndNode newAnd;
-
-			newAnd = (AndNode) getNodeFactory().getNode(
-									C_NodeTypes.AND_NODE,
-									this,
-									bon.preprocess(numTables,
-												   outerFromList,
-												   outerSubqueryList,
-												   outerPredicateList),
-									getContextManager());
-			newAnd.postBindFixup();
-
-			/* Mark this node as transformed so that we don't get
-			 * calculated into the selectivity mulitple times.
+			/* Finally, create the "column = ?" equality that serves as the
+			 * basis for the probe predicate.  We store a reference to "this"
+			 * node inside the probe predicate so that, if we later decide
+			 * *not* to use the probe predicate for execution time index
+			 * probing, we can revert it back to its original form (i.e.
+			 * to "this").
 			 */
-			setTransformed();
+			BinaryComparisonOperatorNode equal = 
+				(BinaryComparisonOperatorNode) getNodeFactory().getNode(
+					C_NodeTypes.BINARY_EQUALS_OPERATOR_NODE,
+					leftOperand, 
+					pNode,
+					this,
+					getContextManager());
 
-			// Return new AndNode
-			return newAnd;
+			/* Set type info for the operator node */
+			equal.bindComparisonOperator();
+			return equal;
 		}
 		else
 		{
@@ -591,6 +708,15 @@
 			mb.callMethod(VMOpcode.INVOKESTATIC, ClassName.BaseExpressionActivation, methodName, ClassName.DataValueDescriptor, 6);
 
 		}
+	}
+
+	/**
+	 * Indicate that the IN-list values for this node are ordered (i.e. they
+	 * are all constants and they have been sorted).
+	 */
+	protected void markAsOrdered()
+	{
+		isOrdered = true;
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrNode.java?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/OrNode.java Wed Mar 14 13:53:16 2007
@@ -132,8 +132,38 @@
 				// Is the operator an =
 				if (!left.isRelationalOperator())
 				{
-					convert = false;
-					break;
+					/* If the operator is an IN-list disguised as a relational
+					 * operator then we can still convert it--we'll just
+					 * combine the existing IN-list ("left") with the new IN-
+					 * list values.  So check for that case now.
+					 */ 
+
+					if (SanityManager.DEBUG)
+					{
+						/* At the time of writing the only way a call to
+						 * left.isRelationalOperator() would return false for
+						 * a BinaryRelationalOperatorNode was if that node
+						 * was for an IN-list probe predicate.  That's why we
+						 * we can get by with the simple "instanceof" check
+						 * below.  But if we're running in SANE mode, do a
+						 * quick check to make sure that's still valid.
+					 	 */
+						BinaryRelationalOperatorNode bron = null;
+						if (left instanceof BinaryRelationalOperatorNode)
+						{
+ 							bron = (BinaryRelationalOperatorNode)left;
+							if (bron.getInListOp() == null)
+							{
+								SanityManager.THROWASSERT(
+								"isRelationalOperator() unexpectedly returned "
+								+ "false for a BinaryRelationalOperatorNode.");
+							}
+						}
+					}
+
+					convert = (left instanceof BinaryRelationalOperatorNode);
+					if (!convert)
+						break;
 				}
 
 				if (!(((RelationalOperator)left).getOperator() == RelationalOperator.EQUALS_RELOP))
@@ -142,11 +172,11 @@
 					break;
 				}
 
-				BinaryRelationalOperatorNode beon = (BinaryRelationalOperatorNode)left;
+				BinaryRelationalOperatorNode bron = (BinaryRelationalOperatorNode)left;
 
-				if (beon.getLeftOperand() instanceof ColumnReference)
+				if (bron.getLeftOperand() instanceof ColumnReference)
 				{
-					cr = (ColumnReference) beon.getLeftOperand();
+					cr = (ColumnReference) bron.getLeftOperand();
 					if (tableNumber == -1)
 					{
 						tableNumber = cr.getTableNumber();
@@ -159,9 +189,9 @@
 						break;
 					}
 				}
-				else if (beon.getRightOperand() instanceof ColumnReference)
+				else if (bron.getRightOperand() instanceof ColumnReference)
 				{
-					cr = (ColumnReference) beon.getRightOperand();
+					cr = (ColumnReference) bron.getRightOperand();
 					if (tableNumber == -1)
 					{
 						tableNumber = cr.getTableNumber();
@@ -191,14 +221,30 @@
 				for (ValueNode vn = this; vn instanceof OrNode; vn = ((OrNode) vn).getRightOperand())
 				{
 					OrNode on = (OrNode) vn;
-					BinaryRelationalOperatorNode beon = (BinaryRelationalOperatorNode) on.getLeftOperand();
-					if (beon.getLeftOperand() instanceof ColumnReference)
+					BinaryRelationalOperatorNode bron =
+						(BinaryRelationalOperatorNode) on.getLeftOperand();
+					if (bron.getInListOp() != null)
+					{
+						/* If we have an OR between multiple IN-lists on the same
+						 * column then just combine them into a single IN-list.
+						 * Ex.
+						 *
+						 *   select ... from T1 where i in (2, 3) or i in (7, 10)
+						 *
+						 * effectively becomes:
+						 *
+						 *   select ... from T1 where i in (2, 3, 7, 10).
+						 */
+						vnl.destructiveAppend(
+							bron.getInListOp().getRightOperandList());
+					}
+					else if (bron.getLeftOperand() instanceof ColumnReference)
 					{
-						vnl.addValueNode(beon.getRightOperand());
+						vnl.addValueNode(bron.getRightOperand());
 					}
 					else
 					{
-						vnl.addValueNode(beon.getLeftOperand());
+						vnl.addValueNode(bron.getLeftOperand());
 					}
 				}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java Wed Mar 14 13:53:16 2007
@@ -337,7 +337,8 @@
 									throws StandardException
 	{
 		/* If we were given a specific ValueNode to generate then
-		 * just use that.
+		 * just use that.  See, in particular, the preprocess method
+		 * of InListOperatorNode.
 		 */
 		if (valToGenerate != null)
 		{

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?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- 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 Wed Mar 14 13:53:16 2007
@@ -1499,11 +1499,26 @@
 				// <column> <relop> <value> AND TRUE
 				if (andNode.getLeftOperand() instanceof BinaryRelationalOperatorNode)
 				{
+					/* If the operator is a binary relational operator that was
+					 * created for a probe predicate then we have to make a
+					 * copy of the underlying IN-list as well, so that we can
+					 * give it the correct left operand (i.e. the new Column
+					 * Reference node).  Then we pass that copy into the new
+					 * relational operator node.
+					 */
+					InListOperatorNode ilon = opNode.getInListOp();
+					if (ilon != null)
+					{
+						ilon = ilon.shallowCopy();
+						ilon.setLeftOperand(newCRNode);
+					}
+
 					BinaryRelationalOperatorNode newRelop = (BinaryRelationalOperatorNode)
 							getNodeFactory().getNode(
 										opNode.getNodeType(),
 										newCRNode,
 										opNode.getRightOperand(),
+										ilon,
 										getContextManager());
 					newRelop.bindComparisonOperator();
 					leftOperand = newRelop;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ValueNodeList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ValueNodeList.java?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ValueNodeList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ValueNodeList.java Wed Mar 14 13:53:16 2007
@@ -411,6 +411,29 @@
 	}
 
 	/**
+	 * Does this list *only* contain constant and/or parameter nodes?
+	 *
+	 * @return boolean	True if every node in this list is either a constant
+	 *  node or parameter node.
+	 */
+	public boolean containsOnlyConstantAndParamNodes()
+	{
+		int size = size();
+
+		for (int index = 0; index < size; index++)
+		{
+			ValueNode vNode = (ValueNode)elementAt(index);
+			if (!vNode.requiresTypeFromContext() &&
+			    !(vNode instanceof ConstantNode))
+			{
+				return false;
+			}
+		}
+
+		return true;
+	}
+
+	/**
 	 * Sort the entries in the list in ascending order.
 	 * (All values are assumed to be constants.)
 	 *

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatePushdown.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatePushdown.out?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatePushdown.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatePushdown.out Wed Mar 14 13:53:16 2007
@@ -2555,8 +2555,8 @@
 						Left result set:
 							Project-Restrict ResultSet (6):
 							Number of opens = 1
-							Rows seen = 3
-							Rows filtered = 1
+							Rows seen = 5
+							Rows filtered = 3
 							restriction = true
 							projection = false
 								constructor time (milliseconds) = 0
@@ -2568,7 +2568,7 @@
 							Source result set:
 								Table Scan ResultSet for T1 at read committed isolation level using share row locking chosen by the optimizer
 								Number of opens = 1
-								Rows seen = 3
+								Rows seen = 5
 								Rows filtered = 0
 								Fetch Size = 1
 									constructor time (milliseconds) = 0
@@ -2580,27 +2580,18 @@
 									Bit set of columns fetched=All
 									Number of columns fetched=2
 									Number of pages visited=1
-									Number of rows qualified=3
+									Number of rows qualified=5
 									Number of rows visited=5
 									Scan type=heap
 									start position: 
 null									stop position: 
 null									qualifiers:
-Column[0][0] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 						Right result set:
 							Project-Restrict ResultSet (8):
 							Number of opens = 1
-							Rows seen = 3
-							Rows filtered = 1
+							Rows seen = 5
+							Rows filtered = 3
 							restriction = true
 							projection = false
 								constructor time (milliseconds) = 0
@@ -2612,7 +2603,7 @@
 							Source result set:
 								Table Scan ResultSet for T2 at read committed isolation level using share row locking chosen by the optimizer
 								Number of opens = 1
-								Rows seen = 3
+								Rows seen = 5
 								Rows filtered = 0
 								Fetch Size = 1
 									constructor time (milliseconds) = 0
@@ -2624,22 +2615,13 @@
 									Bit set of columns fetched=All
 									Number of columns fetched=2
 									Number of pages visited=1
-									Number of rows qualified=3
+									Number of rows qualified=5
 									Number of rows visited=5
 									Scan type=heap
 									start position: 
 null									stop position: 
 null									qualifiers:
-Column[0][0] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 			Right result set:
 				Sort ResultSet:
 				Number of opens = 4
@@ -3010,8 +2992,8 @@
 			Left result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 1
-				Rows seen = 3
-				Rows filtered = 1
+				Rows seen = 5
+				Rows filtered = 3
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -3023,7 +3005,7 @@
 				Source result set:
 					Table Scan ResultSet for T1 at read committed isolation level using share row locking chosen by the optimizer
 					Number of opens = 1
-					Rows seen = 3
+					Rows seen = 5
 					Rows filtered = 0
 					Fetch Size = 1
 						constructor time (milliseconds) = 0
@@ -3035,27 +3017,18 @@
 						Bit set of columns fetched=All
 						Number of columns fetched=2
 						Number of pages visited=1
-						Number of rows qualified=3
+						Number of rows qualified=5
 						Number of rows visited=5
 						Scan type=heap
 						start position: 
 null						stop position: 
 null						qualifiers:
-Column[0][0] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 			Right result set:
 				Project-Restrict ResultSet (7):
 				Number of opens = 1
-				Rows seen = 3
-				Rows filtered = 1
+				Rows seen = 5
+				Rows filtered = 3
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -3067,7 +3040,7 @@
 				Source result set:
 					Table Scan ResultSet for T2 at read committed isolation level using share row locking chosen by the optimizer
 					Number of opens = 1
-					Rows seen = 3
+					Rows seen = 5
 					Rows filtered = 0
 					Fetch Size = 1
 						constructor time (milliseconds) = 0
@@ -3079,22 +3052,13 @@
 						Bit set of columns fetched=All
 						Number of columns fetched=2
 						Number of pages visited=1
-						Number of rows qualified=3
+						Number of rows qualified=5
 						Number of rows visited=5
 						Scan type=heap
 						start position: 
 null						stop position: 
 null						qualifiers:
-Column[0][0] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 Right result set:
 	Sort ResultSet:
 	Number of opens = 4

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out Wed Mar 14 13:53:16 2007
@@ -2385,7 +2385,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (8):
+Project-Restrict ResultSet (6):
 Number of opens = 1
 Rows seen = 0
 Rows filtered = 0
@@ -2408,100 +2408,74 @@
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
 	Left result set:
-		Project-Restrict ResultSet (4):
+		Index Row to Base Row ResultSet for TABLE1:
 		Number of opens = 1
 		Rows seen = 0
-		Rows filtered = 0
-		restriction = true
-		projection = true
+		Columns accessed from heap = {0}
 			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:
-			Index Row to Base Row ResultSet for TABLE1:
-			Number of opens = 1
+			Index Scan ResultSet for TABLE1 using index TABLE1IDX at serializable isolation level using share row locking chosen by the optimizer
+			Number of opens = 3
 			Rows seen = 0
-			Columns accessed from heap = {0}
+			Rows filtered = 0
+			Fetch Size = 1
 				constructor time (milliseconds) = 0
 				open time (milliseconds) = 0
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
-				Index Scan ResultSet for TABLE1 using index TABLE1IDX at serializable isolation level using share row locking chosen by the optimizer
-				Number of opens = 1
-				Rows seen = 0
-				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
-					Number of pages visited=1
-					Number of rows qualified=0
-					Number of rows visited=0
-					Scan type=btree
-					Tree height=1
-					start position: 
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				Number of pages visited=3
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=btree
+				Tree height=1
+				start position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					stop position: 
+				stop position: 
 	> on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					qualifiers:
+				qualifiers:
 None
 	Right result set:
-		Project-Restrict ResultSet (7):
+		Index Row to Base Row ResultSet for TABLE2:
 		Number of opens = 1
 		Rows seen = 0
-		Rows filtered = 0
-		restriction = true
-		projection = true
+		Columns accessed from heap = {0}
 			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:
-			Index Row to Base Row ResultSet for TABLE2:
-			Number of opens = 1
+			Index Scan ResultSet for TABLE2 using index TABLE2IDX at serializable isolation level using share row locking chosen by the optimizer
+			Number of opens = 3
 			Rows seen = 0
-			Columns accessed from heap = {0}
+			Rows filtered = 0
+			Fetch Size = 1
 				constructor time (milliseconds) = 0
 				open time (milliseconds) = 0
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
-				Index Scan ResultSet for TABLE2 using index TABLE2IDX at serializable isolation level using share row locking chosen by the optimizer
-				Number of opens = 1
-				Rows seen = 0
-				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
-					Number of pages visited=1
-					Number of rows qualified=0
-					Number of rows visited=0
-					Scan type=btree
-					Tree height=1
-					start position: 
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				Number of pages visited=3
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=btree
+				Tree height=1
+				start position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					stop position: 
+				stop position: 
 	> on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					qualifiers:
+				qualifiers:
 None
 ij> prepare s as 'select * from test.view0 where b in (?, ?, ?)';
 ij> execute s using 'values (1,2,3)';
@@ -2526,7 +2500,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (8):
+Project-Restrict ResultSet (6):
 Number of opens = 1
 Rows seen = 0
 Rows filtered = 0
@@ -2549,100 +2523,74 @@
 		next time (milliseconds) = 0
 		close time (milliseconds) = 0
 	Left result set:
-		Project-Restrict ResultSet (4):
+		Index Row to Base Row ResultSet for TABLE1:
 		Number of opens = 1
 		Rows seen = 0
-		Rows filtered = 0
-		restriction = true
-		projection = true
+		Columns accessed from heap = {0}
 			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:
-			Index Row to Base Row ResultSet for TABLE1:
-			Number of opens = 1
+			Index Scan ResultSet for TABLE1 using index TABLE1IDX at serializable isolation level using share row locking chosen by the optimizer
+			Number of opens = 3
 			Rows seen = 0
-			Columns accessed from heap = {0}
+			Rows filtered = 0
+			Fetch Size = 1
 				constructor time (milliseconds) = 0
 				open time (milliseconds) = 0
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
-				Index Scan ResultSet for TABLE1 using index TABLE1IDX at serializable isolation level using share row locking chosen by the optimizer
-				Number of opens = 1
-				Rows seen = 0
-				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
-					Number of pages visited=1
-					Number of rows qualified=0
-					Number of rows visited=0
-					Scan type=btree
-					Tree height=1
-					start position: 
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				Number of pages visited=3
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=btree
+				Tree height=1
+				start position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					stop position: 
+				stop position: 
 	> on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					qualifiers:
+				qualifiers:
 None
 	Right result set:
-		Project-Restrict ResultSet (7):
+		Index Row to Base Row ResultSet for TABLE2:
 		Number of opens = 1
 		Rows seen = 0
-		Rows filtered = 0
-		restriction = true
-		projection = true
+		Columns accessed from heap = {0}
 			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:
-			Index Row to Base Row ResultSet for TABLE2:
-			Number of opens = 1
+			Index Scan ResultSet for TABLE2 using index TABLE2IDX at serializable isolation level using share row locking chosen by the optimizer
+			Number of opens = 3
 			Rows seen = 0
-			Columns accessed from heap = {0}
+			Rows filtered = 0
+			Fetch Size = 1
 				constructor time (milliseconds) = 0
 				open time (milliseconds) = 0
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
-				Index Scan ResultSet for TABLE2 using index TABLE2IDX at serializable isolation level using share row locking chosen by the optimizer
-				Number of opens = 1
-				Rows seen = 0
-				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
-					Number of pages visited=1
-					Number of rows qualified=0
-					Number of rows visited=0
-					Scan type=btree
-					Tree height=1
-					start position: 
+			scan information: 
+				Bit set of columns fetched=All
+				Number of columns fetched=2
+				Number of deleted rows visited=0
+				Number of pages visited=3
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=btree
+				Tree height=1
+				start position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					stop position: 
+				stop position: 
 	> on first 1 column(s).
 	Ordered null semantics on the following columns: 
-					qualifiers:
+				qualifiers:
 None
 ij> -- This select should use index for first two selects, table scan for the third
 select a from test.view1 where b=25;

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/readlocks.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/readlocks.out?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/readlocks.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/readlocks.out Wed Mar 14 13:53:16 2007
@@ -4946,7 +4946,7 @@
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(2,1)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -8103,6 +8103,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -8112,6 +8113,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> close scan_cursor;
 ij> select * from lock_table order by tabname, type desc, mode, cnt, lockname;
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
@@ -8956,6 +8958,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -8965,6 +8968,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> close scan_cursor;
 ij> select * from lock_table order by tabname, type desc, mode, cnt, lockname;
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
@@ -11255,7 +11259,8 @@
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(2,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -11265,6 +11270,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> close scan_cursor;
 ij> select * from lock_table order by tabname, type desc, mode, cnt, lockname;
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
@@ -12101,6 +12107,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -12110,6 +12117,7 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> close scan_cursor;
 ij> select * from lock_table order by tabname, type desc, mode, cnt, lockname;
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
@@ -14746,7 +14754,6 @@
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -15752,7 +15759,6 @@
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -18421,9 +18427,8 @@
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(2,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -19421,7 +19426,6 @@
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -21779,8 +21783,6 @@
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,12)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -21791,7 +21793,6 @@
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,12)    |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> close scan_cursor;
 ij> select * from lock_table order by tabname, type desc, mode, cnt, lockname;
@@ -21799,7 +21800,6 @@
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,12)    |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> commit;
 ij> --------------------------------------------------------------------------------
@@ -22639,8 +22639,6 @@
 APP     |UserTran|ROW     |1   |S   |A           |(1,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,10)    |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(1,11)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,12)    |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(1,13)    |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -24974,11 +24972,9 @@
 USERNAME|TRANTYPE|TYPE    |CNT |MODE|TABNAME     |LOCKNAME  |STATE|STATUS  
 ---------------------------------------------------------------------------
 APP     |UserTran|TABLE   |1   |IS  |A           |Tablelock |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
+APP     |UserTran|ROW     |1   |S   |A           |(2,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(4,6)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(6,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------
@@ -25831,8 +25827,6 @@
 APP     |UserTran|ROW     |1   |S   |A           |(3,1)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(4,6)     |GRANT|ACTIVE  
 APP     |UserTran|ROW     |1   |S   |A           |(5,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(6,6)     |GRANT|ACTIVE  
-APP     |UserTran|ROW     |1   |S   |A           |(7,6)     |GRANT|ACTIVE  
 ij> next scan_cursor;
 A          
 -----------

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subquery.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subquery.out?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subquery.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subquery.out Wed Mar 14 13:53:16 2007
@@ -1189,16 +1189,7 @@
 						start position: 
 null						stop position: 
 null						qualifiers:
-Column[0][0] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+None
 			Right result set:
 				Table Scan ResultSet for T2 at read committed isolation level using share row locking chosen by the optimizer
 				Number of opens = 5
@@ -1498,16 +1489,7 @@
 						start position: 
 null						stop position: 
 null						qualifiers:
-Column[0][0] Id: 0
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 0
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 			Right result set:
 				Hash Scan ResultSet for T2 at read committed isolation level using instantaneous share row locking: 
 				Number of opens = 655

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subqueryFlattening.out
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subqueryFlattening.out?view=diff&rev=518322&r1=518321&r2=518322
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subqueryFlattening.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/subqueryFlattening.out Wed Mar 14 13:53:16 2007
@@ -3516,7 +3516,7 @@
 				restriction time (milliseconds) = 0
 				projection time (milliseconds) = 0
 			Source result set:
-				Hash Exists Join ResultSet:
+				Nested Loop Exists Join ResultSet:
 				Number of opens = 1
 				Rows seen from the left = 6
 				Rows seen from the right = 4
@@ -3555,8 +3555,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 6
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -3566,12 +3566,11 @@
 						restriction time (milliseconds) = 0
 						projection time (milliseconds) = 0
 					Source result set:
-						Hash Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking: 
+						Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 						Number of opens = 6
-						Hash table size = 3
-						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
+						Fetch Size = 1
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
 							next time (milliseconds) = 0
@@ -3581,32 +3580,21 @@
 							Bit set of columns fetched={0, 1}
 							Number of columns fetched=2
 							Number of deleted rows visited=0
-							Number of pages visited=1
+							Number of pages visited=6
 							Number of rows qualified=3
 							Number of rows visited=6
 							Scan type=btree
 							Tree height=1
 							start position: 
-	None
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
 							stop position: 
-	None
-							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-							next qualifiers:
-Column[0][0] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+							qualifiers:
+None
 ij> -- NOT EXISTS is flattened
 SELECT COUNT(*) FROM
 ( SELECT ID FROM DOCS WHERE
@@ -3687,7 +3675,7 @@
 				restriction time (milliseconds) = 0
 				projection time (milliseconds) = 0
 			Source result set:
-				Hash Exists Join ResultSet:
+				Nested Loop Exists Join ResultSet:
 				Number of opens = 1
 				Rows seen from the left = 6
 				Rows seen from the right = 4
@@ -3726,8 +3714,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 6
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -3737,12 +3725,11 @@
 						restriction time (milliseconds) = 0
 						projection time (milliseconds) = 0
 					Source result set:
-						Hash Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking: 
+						Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 						Number of opens = 6
-						Hash table size = 3
-						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
+						Fetch Size = 1
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
 							next time (milliseconds) = 0
@@ -3752,32 +3739,21 @@
 							Bit set of columns fetched={0, 1}
 							Number of columns fetched=2
 							Number of deleted rows visited=0
-							Number of pages visited=1
+							Number of pages visited=6
 							Number of rows qualified=3
 							Number of rows visited=6
 							Scan type=btree
 							Tree height=1
 							start position: 
-	None
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
 							stop position: 
-	None
-							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-							next qualifiers:
-Column[0][0] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+							qualifiers:
+None
 ij> -- EXISTS is flattened
 SELECT COUNT(*) FROM
 ( SELECT ID FROM DOCS WHERE
@@ -3858,7 +3834,7 @@
 				restriction time (milliseconds) = 0
 				projection time (milliseconds) = 0
 			Source result set:
-				Hash Exists Join ResultSet:
+				Nested Loop Exists Join ResultSet:
 				Number of opens = 1
 				Rows seen from the left = 6
 				Rows seen from the right = 2
@@ -3897,8 +3873,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 6
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -3908,12 +3884,11 @@
 						restriction time (milliseconds) = 0
 						projection time (milliseconds) = 0
 					Source result set:
-						Hash Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking: 
+						Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 						Number of opens = 6
-						Hash table size = 3
-						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
+						Fetch Size = 1
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
 							next time (milliseconds) = 0
@@ -3923,32 +3898,21 @@
 							Bit set of columns fetched={0, 1}
 							Number of columns fetched=2
 							Number of deleted rows visited=0
-							Number of pages visited=1
+							Number of pages visited=6
 							Number of rows qualified=3
 							Number of rows visited=6
 							Scan type=btree
 							Tree height=1
 							start position: 
-	None
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
 							stop position: 
-	None
-							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-							next qualifiers:
-Column[0][0] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+							qualifiers:
+None
 ij> -- IN is flattened
 SELECT count(ID) FROM DOCS WHERE ID IN (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4008,7 +3972,7 @@
 			restriction time (milliseconds) = 0
 			projection time (milliseconds) = 0
 		Source result set:
-			Hash Exists Join ResultSet:
+			Nested Loop Exists Join ResultSet:
 			Number of opens = 1
 			Rows seen from the left = 6
 			Rows seen from the right = 2
@@ -4047,8 +4011,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 6
-				Rows seen = 2
-				Rows filtered = 0
+				Rows seen = 3
+				Rows filtered = 1
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -4058,12 +4022,11 @@
 					restriction time (milliseconds) = 0
 					projection time (milliseconds) = 0
 				Source result set:
-					Hash Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking: 
+					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 					Number of opens = 6
-					Hash table size = 3
-					Hash key is column number 0
-					Rows seen = 2
+					Rows seen = 3
 					Rows filtered = 0
+					Fetch Size = 1
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
@@ -4073,32 +4036,21 @@
 						Bit set of columns fetched={0, 1}
 						Number of columns fetched=2
 						Number of deleted rows visited=0
-						Number of pages visited=1
+						Number of pages visited=6
 						Number of rows qualified=3
 						Number of rows visited=6
 						Scan type=btree
 						Tree height=1
 						start position: 
-	None
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
 						stop position: 
-	None
-						scan qualifiers:
-Column[0][0] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-						next qualifiers:
-Column[0][0] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+						qualifiers:
+None
 ij> -- ANY is flattened
 SELECT count(ID) FROM DOCS WHERE ID > ANY (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4197,8 +4149,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 6
-				Rows seen = 4
-				Rows filtered = 0
+				Rows seen = 15
+				Rows filtered = 11
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -4210,7 +4162,7 @@
 				Source result set:
 					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 					Number of opens = 6
-					Rows seen = 4
+					Rows seen = 15
 					Rows filtered = 0
 					Fetch Size = 1
 						constructor time (milliseconds) = 0
@@ -4223,7 +4175,7 @@
 						Number of columns fetched=2
 						Number of deleted rows visited=0
 						Number of pages visited=6
-						Number of rows qualified=4
+						Number of rows qualified=15
 						Number of rows visited=17
 						Scan type=btree
 						Tree height=1
@@ -4234,16 +4186,7 @@
 	Ordered null semantics on the following columns: 
 0 
 						qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 ij> -- ANY is flattened
 SELECT count(ID) FROM DOCS WHERE ID <> ANY (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4268,7 +4211,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (7):
+Project-Restrict ResultSet (8):
 Number of opens = 1
 Rows seen = 1
 Rows filtered = 0
@@ -4290,7 +4233,7 @@
 		close time (milliseconds) = 0
 	Index Key Optimization = false
 	Source result set:
-		Project-Restrict ResultSet (6):
+		Project-Restrict ResultSet (7):
 		Number of opens = 1
 		Rows seen = 6
 		Rows filtered = 0
@@ -4340,12 +4283,12 @@
 					qualifiers:
 None
 			Right result set:
-				Project-Restrict ResultSet (5):
+				Project-Restrict ResultSet (6):
 				Number of opens = 6
-				Rows seen = 6
-				Rows filtered = 0
+				Rows seen = 7
+				Rows filtered = 1
 				restriction = true
-				projection = false
+				projection = true
 					constructor time (milliseconds) = 0
 					open time (milliseconds) = 0
 					next time (milliseconds) = 0
@@ -4353,45 +4296,41 @@
 					restriction time (milliseconds) = 0
 					projection time (milliseconds) = 0
 				Source result set:
-					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking chosen by the optimizer
+					Index Row to Base Row ResultSet for COLLS:
 					Number of opens = 6
-					Rows seen = 6
-					Rows filtered = 0
-					Fetch Size = 1
+					Rows seen = 7
+					Columns accessed from heap = {0}
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
 						close time (milliseconds) = 0
-						next time in milliseconds/row = 0
-					scan information: 
-						Bit set of columns fetched={0, 1}
-						Number of columns fetched=2
-						Number of deleted rows visited=0
-						Number of pages visited=6
-						Number of rows qualified=6
-						Number of rows visited=19
-						Scan type=btree
-						Tree height=1
-						start position: 
-	None
-						stop position: 
-	None
-						qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][2] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+						Index Scan ResultSet for COLLS using index NEW_INDEX3 at serializable isolation level using share row locking chosen by the optimizer
+						Number of opens = 6
+						Rows seen = 7
+						Rows filtered = 0
+						Fetch Size = 1
+							constructor time (milliseconds) = 0
+							open time (milliseconds) = 0
+							next time (milliseconds) = 0
+							close time (milliseconds) = 0
+							next time in milliseconds/row = 0
+						scan information: 
+							Bit set of columns fetched=All
+							Number of columns fetched=2
+							Number of deleted rows visited=0
+							Number of pages visited=6
+							Number of rows qualified=7
+							Number of rows visited=7
+							Scan type=btree
+							Tree height=1
+							start position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+							stop position: 
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+							qualifiers:
+None
 ij> -- ALL is flattened, what's not?
 SELECT count(ID) FROM DOCS WHERE ID = ALL (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4416,7 +4355,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (7):
+Project-Restrict ResultSet (8):
 Number of opens = 1
 Rows seen = 1
 Rows filtered = 0
@@ -4438,7 +4377,7 @@
 		close time (milliseconds) = 0
 	Index Key Optimization = false
 	Source result set:
-		Project-Restrict ResultSet (6):
+		Project-Restrict ResultSet (7):
 		Number of opens = 1
 		Rows seen = 0
 		Rows filtered = 0
@@ -4488,12 +4427,12 @@
 					qualifiers:
 None
 			Right result set:
-				Project-Restrict ResultSet (5):
+				Project-Restrict ResultSet (6):
 				Number of opens = 6
-				Rows seen = 6
-				Rows filtered = 0
+				Rows seen = 7
+				Rows filtered = 1
 				restriction = true
-				projection = false
+				projection = true
 					constructor time (milliseconds) = 0
 					open time (milliseconds) = 0
 					next time (milliseconds) = 0
@@ -4501,45 +4440,41 @@
 					restriction time (milliseconds) = 0
 					projection time (milliseconds) = 0
 				Source result set:
-					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking chosen by the optimizer
+					Index Row to Base Row ResultSet for COLLS:
 					Number of opens = 6
-					Rows seen = 6
-					Rows filtered = 0
-					Fetch Size = 1
+					Rows seen = 7
+					Columns accessed from heap = {0}
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
 						close time (milliseconds) = 0
-						next time in milliseconds/row = 0
-					scan information: 
-						Bit set of columns fetched={0, 1}
-						Number of columns fetched=2
-						Number of deleted rows visited=0
-						Number of pages visited=6
-						Number of rows qualified=6
-						Number of rows visited=19
-						Scan type=btree
-						Tree height=1
-						start position: 
-	None
-						stop position: 
-	None
-						qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][2] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+						Index Scan ResultSet for COLLS using index NEW_INDEX3 at serializable isolation level using share row locking chosen by the optimizer
+						Number of opens = 6
+						Rows seen = 7
+						Rows filtered = 0
+						Fetch Size = 1
+							constructor time (milliseconds) = 0
+							open time (milliseconds) = 0
+							next time (milliseconds) = 0
+							close time (milliseconds) = 0
+							next time in milliseconds/row = 0
+						scan information: 
+							Bit set of columns fetched=All
+							Number of columns fetched=2
+							Number of deleted rows visited=0
+							Number of pages visited=6
+							Number of rows qualified=7
+							Number of rows visited=7
+							Scan type=btree
+							Tree height=1
+							start position: 
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+							stop position: 
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+							qualifiers:
+None
 ij> -- ALL is flattened, what's not?
 SELECT count(ID) FROM DOCS WHERE ID < ALL (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4638,8 +4573,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 6
-				Rows seen = 5
-				Rows filtered = 0
+				Rows seen = 17
+				Rows filtered = 12
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -4651,7 +4586,7 @@
 				Source result set:
 					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 					Number of opens = 6
-					Rows seen = 5
+					Rows seen = 17
 					Rows filtered = 0
 					Fetch Size = 1
 						constructor time (milliseconds) = 0
@@ -4664,27 +4599,18 @@
 						Number of columns fetched=2
 						Number of deleted rows visited=0
 						Number of pages visited=6
-						Number of rows qualified=5
+						Number of rows qualified=17
 						Number of rows visited=18
 						Scan type=btree
 						Tree height=1
 						start position: 
 	None
 						stop position: 
-	> on first 2 column(s).
+	> on first 1 column(s).
 	Ordered null semantics on the following columns: 
-0 1 
+0 
 						qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 ij> -- ALL is flattened, what's not?
 SELECT count(ID) FROM DOCS WHERE ID <> ALL (SELECT ID FROM COLLS WHERE COLLID IN (-2,1) );
 1          
@@ -4744,7 +4670,7 @@
 			restriction time (milliseconds) = 0
 			projection time (milliseconds) = 0
 		Source result set:
-			Hash Exists Join ResultSet:
+			Nested Loop Exists Join ResultSet:
 			Number of opens = 1
 			Rows seen from the left = 6
 			Rows seen from the right = 4
@@ -4783,8 +4709,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 6
-				Rows seen = 2
-				Rows filtered = 0
+				Rows seen = 3
+				Rows filtered = 1
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -4794,12 +4720,11 @@
 					restriction time (milliseconds) = 0
 					projection time (milliseconds) = 0
 				Source result set:
-					Hash Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share table locking: 
+					Index Scan ResultSet for COLLS using constraint NEW_KEY2 at serializable isolation level using share row locking chosen by the optimizer
 					Number of opens = 6
-					Hash table size = 3
-					Hash key is column number 0
-					Rows seen = 2
+					Rows seen = 3
 					Rows filtered = 0
+					Fetch Size = 1
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
@@ -4809,32 +4734,21 @@
 						Bit set of columns fetched={0, 1}
 						Number of columns fetched=2
 						Number of deleted rows visited=0
-						Number of pages visited=1
+						Number of pages visited=6
 						Number of rows qualified=3
 						Number of rows visited=6
 						Scan type=btree
 						Tree height=1
 						start position: 
-	None
+	>= on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
 						stop position: 
-	None
-						scan qualifiers:
-Column[0][0] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
-Column[0][1] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-						next qualifiers:
-Column[0][0] Id: 0
-Operator: =
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
+0 
+						qualifiers:
+None
 ij> -- Now test nullable correlated columns
 drop table colls;
 0 rows inserted/updated/deleted
@@ -4984,8 +4898,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 6
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -4997,9 +4911,9 @@
 					Source result set:
 						Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 						Number of opens = 6
-						Hash table size = 3
+						Hash table size = 6
 						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
@@ -5010,22 +4924,13 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of pages visited=1
-							Number of rows qualified=5
+							Number of rows qualified=9
 							Number of rows visited=9
 							Scan type=heap
 							start position: 
 null							stop position: 
 null							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 							next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -5151,8 +5056,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 6
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -5164,9 +5069,9 @@
 					Source result set:
 						Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 						Number of opens = 6
-						Hash table size = 3
+						Hash table size = 6
 						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
@@ -5177,22 +5082,13 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of pages visited=1
-							Number of rows qualified=5
+							Number of rows qualified=9
 							Number of rows visited=9
 							Scan type=heap
 							start position: 
 null							stop position: 
 null							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 							next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -5297,8 +5193,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 6
-				Rows seen = 2
-				Rows filtered = 0
+				Rows seen = 3
+				Rows filtered = 1
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -5310,9 +5206,9 @@
 				Source result set:
 					Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 					Number of opens = 6
-					Hash table size = 3
+					Hash table size = 6
 					Hash key is column number 0
-					Rows seen = 2
+					Rows seen = 3
 					Rows filtered = 0
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
@@ -5323,22 +5219,13 @@
 						Bit set of columns fetched=All
 						Number of columns fetched=2
 						Number of pages visited=1
-						Number of rows qualified=5
+						Number of rows qualified=9
 						Number of rows visited=9
 						Scan type=heap
 						start position: 
 null						stop position: 
 null						scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 						next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -5443,8 +5330,8 @@
 			Right result set:
 				Project-Restrict ResultSet (6):
 				Number of opens = 6
-				Rows seen = 15
-				Rows filtered = 11
+				Rows seen = 16
+				Rows filtered = 12
 				restriction = true
 				projection = true
 					constructor time (milliseconds) = 0
@@ -5456,15 +5343,15 @@
 				Source result set:
 					Index Row to Base Row ResultSet for COLLS:
 					Number of opens = 6
-					Rows seen = 15
-					Columns accessed from heap = {1}
+					Rows seen = 16
+					Columns accessed from heap = {0}
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
 						close time (milliseconds) = 0
-						Index Scan ResultSet for COLLS using index NEW_INDEX2 at serializable isolation level using share row locking chosen by the optimizer
-						Number of opens = 6
-						Rows seen = 15
+						Index Scan ResultSet for COLLS using index NEW_INDEX3 at serializable isolation level using share row locking chosen by the optimizer
+						Number of opens = 9
+						Rows seen = 16
 						Rows filtered = 0
 						Fetch Size = 1
 							constructor time (milliseconds) = 0
@@ -5476,16 +5363,17 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of deleted rows visited=0
-							Number of pages visited=6
-							Number of rows qualified=15
-							Number of rows visited=17
+							Number of pages visited=9
+							Number of rows qualified=16
+							Number of rows visited=21
 							Scan type=btree
 							Tree height=1
 							start position: 
-	None
-							stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
+							stop position: 
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
 							qualifiers:
 None
 ij> -- ALL should NOT be flattened, but subquery should be materialized
@@ -5512,7 +5400,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (10):
+Project-Restrict ResultSet (9):
 Number of opens = 1
 Rows seen = 1
 Rows filtered = 0
@@ -5544,7 +5432,7 @@
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
 			Source result set:
-				Project-Restrict ResultSet (8):
+				Project-Restrict ResultSet (7):
 				Number of opens = 6
 				Rows seen = 12
 				Rows filtered = 6
@@ -5805,8 +5693,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 7
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -5818,9 +5706,9 @@
 					Source result set:
 						Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 						Number of opens = 7
-						Hash table size = 3
+						Hash table size = 6
 						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
@@ -5831,22 +5719,13 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of pages visited=1
-							Number of rows qualified=5
+							Number of rows qualified=9
 							Number of rows visited=9
 							Scan type=heap
 							start position: 
 null							stop position: 
 null							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 							next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -5972,8 +5851,8 @@
 				Right result set:
 					Project-Restrict ResultSet (5):
 					Number of opens = 7
-					Rows seen = 2
-					Rows filtered = 0
+					Rows seen = 3
+					Rows filtered = 1
 					restriction = true
 					projection = false
 						constructor time (milliseconds) = 0
@@ -5985,9 +5864,9 @@
 					Source result set:
 						Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 						Number of opens = 7
-						Hash table size = 3
+						Hash table size = 6
 						Hash key is column number 0
-						Rows seen = 2
+						Rows seen = 3
 						Rows filtered = 0
 							constructor time (milliseconds) = 0
 							open time (milliseconds) = 0
@@ -5998,22 +5877,13 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of pages visited=1
-							Number of rows qualified=5
+							Number of rows qualified=9
 							Number of rows visited=9
 							Scan type=heap
 							start position: 
 null							stop position: 
 null							scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 							next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -6118,8 +5988,8 @@
 			Right result set:
 				Project-Restrict ResultSet (5):
 				Number of opens = 7
-				Rows seen = 2
-				Rows filtered = 0
+				Rows seen = 3
+				Rows filtered = 1
 				restriction = true
 				projection = false
 					constructor time (milliseconds) = 0
@@ -6131,9 +6001,9 @@
 				Source result set:
 					Hash Scan ResultSet for COLLS at serializable isolation level using share table locking: 
 					Number of opens = 7
-					Hash table size = 3
+					Hash table size = 6
 					Hash key is column number 0
-					Rows seen = 2
+					Rows seen = 3
 					Rows filtered = 0
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
@@ -6144,22 +6014,13 @@
 						Bit set of columns fetched=All
 						Number of columns fetched=2
 						Number of pages visited=1
-						Number of rows qualified=5
+						Number of rows qualified=9
 						Number of rows visited=9
 						Scan type=heap
 						start position: 
 null						stop position: 
 null						scan qualifiers:
-Column[0][0] Id: 1
-Operator: <=
-Ordered nulls: false
-Unknown return value: false
-Negate comparison result: false
-Column[0][1] Id: 1
-Operator: <
-Ordered nulls: false
-Unknown return value: true
-Negate comparison result: true
+None
 						next qualifiers:
 Column[0][0] Id: 0
 Operator: =
@@ -6264,8 +6125,8 @@
 			Right result set:
 				Project-Restrict ResultSet (6):
 				Number of opens = 7
-				Rows seen = 15
-				Rows filtered = 11
+				Rows seen = 19
+				Rows filtered = 15
 				restriction = true
 				projection = true
 					constructor time (milliseconds) = 0
@@ -6277,15 +6138,15 @@
 				Source result set:
 					Index Row to Base Row ResultSet for COLLS:
 					Number of opens = 7
-					Rows seen = 15
-					Columns accessed from heap = {1}
+					Rows seen = 19
+					Columns accessed from heap = {0}
 						constructor time (milliseconds) = 0
 						open time (milliseconds) = 0
 						next time (milliseconds) = 0
 						close time (milliseconds) = 0
-						Index Scan ResultSet for COLLS using index NEW_INDEX2 at serializable isolation level using share row locking chosen by the optimizer
-						Number of opens = 7
-						Rows seen = 15
+						Index Scan ResultSet for COLLS using index NEW_INDEX3 at serializable isolation level using share row locking chosen by the optimizer
+						Number of opens = 10
+						Rows seen = 19
 						Rows filtered = 0
 						Fetch Size = 1
 							constructor time (milliseconds) = 0
@@ -6297,16 +6158,17 @@
 							Bit set of columns fetched=All
 							Number of columns fetched=2
 							Number of deleted rows visited=0
-							Number of pages visited=6
-							Number of rows qualified=15
-							Number of rows visited=17
+							Number of pages visited=10
+							Number of rows qualified=19
+							Number of rows visited=25
 							Scan type=btree
 							Tree height=1
 							start position: 
-	None
-							stop position: 
 	>= on first 1 column(s).
 	Ordered null semantics on the following columns: 
+							stop position: 
+	> on first 1 column(s).
+	Ordered null semantics on the following columns: 
 							qualifiers:
 None
 ij> -- ALL should NOT be flattened, but subquery should be materialized, watch out results
@@ -6333,7 +6195,7 @@
 Begin Execution Timestamp : null
 End Execution Timestamp : null
 Statement Execution Plan Text: 
-Project-Restrict ResultSet (10):
+Project-Restrict ResultSet (9):
 Number of opens = 1
 Rows seen = 1
 Rows filtered = 0
@@ -6365,7 +6227,7 @@
 				next time (milliseconds) = 0
 				close time (milliseconds) = 0
 			Source result set:
-				Project-Restrict ResultSet (8):
+				Project-Restrict ResultSet (7):
 				Number of opens = 7
 				Rows seen = 13
 				Rows filtered = 6