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 ba...@apache.org on 2006/03/02 00:36:11 UTC

svn commit: r382200 - in /db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile: BaseTableNumbersVisitor.java BinaryRelationalOperatorNode.java ColumnReference.java Predicate.java ResultColumn.java

Author: bandaram
Date: Wed Mar  1 15:36:11 2006
New Revision: 382200

URL: http://svn.apache.org/viewcvs?rev=382200&view=rev
Log:
DERBY-805: Phase II changes that are described in detail in JIRA attachment. 

This phase essentially tries to scope join predicates that could be pushed
into table expressions. More work is needed to actually perform this pushing
and costing, which will come in follow up patches.

Submitted by Army Brown (qozinx@sbcglobal.net)

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryRelationalOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ColumnReference.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java?rev=382200&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseTableNumbersVisitor.java Wed Mar  1 15:36:11 2006
@@ -0,0 +1,193 @@
+/*
+
+   Derby - Class org.apache.derby.impl.sql.compile.BaseTableNumbersVisitor
+
+   Copyright 2006 The Apache Software Foundation or its licensors, as applicable.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package	org.apache.derby.impl.sql.compile;
+
+import org.apache.derby.iapi.sql.compile.Visitable; 
+import org.apache.derby.iapi.sql.compile.Visitor;
+import org.apache.derby.iapi.util.JBitSet;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.services.sanity.SanityManager;
+
+/**
+ * Walk through a subtree and build a list of the assigned numbers for
+ * all tables that exist in that subtree.  We do this by looking for any
+ * column references in the subtree and, for each column reference, we
+ * walk down the ColumnReference-ResultColumn chain until we find the
+ * the bottom-most table number, which should correspond to a base
+ * table.
+ */
+public class BaseTableNumbersVisitor implements Visitor
+{
+	// JBitSet to hold the table numbers that we find.
+	private JBitSet tableMap;
+
+	/**
+	 * Constructor: takes a JBitSet to use as the holder for any base table
+	 * numbers found while walking the subtree.
+	 *
+	 * @param tableMap JBitSet into which we put the table numbers we find.
+	 */
+	public BaseTableNumbersVisitor(JBitSet tableMap)
+	{
+		this.tableMap = tableMap;
+	}
+
+	/**
+	 * Set a new JBitSet to serve as the holder for base table numbers
+	 * we find while walking.
+	 *
+	 * @param tableMap JBitSet into which we put the table numbers we find.
+	 */
+	protected void setTableMap(JBitSet tableMap)
+	{
+		this.tableMap = tableMap;
+	}
+
+	////////////////////////////////////////////////
+	//
+	// VISITOR INTERFACE
+	//
+	////////////////////////////////////////////////
+
+	/**
+	 * @see org.apache.derby.iapi.sql.compile.Visitor#visit
+	 */
+	public Visitable visit(Visitable node)
+		throws StandardException
+	{
+		ResultColumn rc = null;
+		if (node instanceof ColumnReference)
+		{
+			// Start by seeing if this column reference is the
+			// bottom-most one, meaning that there are no column
+			// references beneath this one.
+			rc = ((ColumnReference)node).getSource();
+
+			if (rc == null) {
+			// this can happen if column reference is pointing to a column
+			// that is not from a base table.  For example, if we have a
+			// VALUES clause like
+			//
+			//    (values (1, 2), (3, 4)) V1 (i, j)
+			//
+			// and then a column reference to VI.i, the column reference
+			// won't have a source.
+				return node;
+			}
+		}
+		else if (node instanceof ResultColumn)
+			rc = (ResultColumn)rc;
+		else if (node instanceof SelectNode)
+		{
+			// If the node is a SelectNode we just need to look at its
+			// FROM list.
+			((SelectNode)node).getFromList().accept(this);
+		}
+		else if (node instanceof FromBaseTable) {
+		// just grab the FBT's table number.
+			tableMap.set(((FromBaseTable)node).getTableNumber());
+		}
+
+		if (rc != null)
+		{
+			// This next call will walk through the ResultColumn tree
+			// until it finds another ColumnReference, and then will
+			// return the table number for that column reference.  We
+			// can't stop there, though, because the column reference
+			// that we found might in turn have column references beneath
+			// it, and we only want the table number of the bottom-most
+			// column reference.  So once we find the column reference,
+			// we have to recurse.
+
+			int baseTableNumber = rc.getTableNumber();
+			if (baseTableNumber >= 0) {
+			// Move down to the column reference that has the table
+			// number that we just found.  There may be one or more
+			// VirtualColumnNode-to-ResultColumnNode links between
+			// the current ResultColumn and the column reference we're
+			// looking for, so we have to walk past those until we find
+			// the desired column reference.
+
+				ValueNode rcExpr = rc.getExpression();
+				while (rcExpr instanceof VirtualColumnNode) {
+					rc = ((VirtualColumnNode)rcExpr).getSourceColumn();
+					rcExpr = rc.getExpression();
+				}
+
+				if (rcExpr instanceof ColumnReference)
+				// we found our column reference; recurse using that.
+					return rcExpr.accept(this);
+
+				// Else we must have found the table number someplace
+				// other than within a ColumnReference (ex. we may
+				// have pulled it from a VirtualColumnNode's source
+				// table); so just set the number.
+				tableMap.set(baseTableNumber);
+
+			}
+			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);
+			}
+		}
+
+		return node;
+	}
+
+	/**
+	 * @see org.apache.derby.iapi.sql.compile.Visitor#skipChildren
+	 */
+	public boolean skipChildren(Visitable node)
+	{
+		/* A SelectNode's children can include a where clause in the
+		 * form of either a PredicateList or an AndNode.  In either
+		 * case we don't want to descend into the where clause because
+		 * it's possible that it references a base table that is not
+		 * in the subtree we're walking.  So we skip the children of
+		 * a SelectNode.  Similarly, any other PredicateList may contain
+		 * references to base tables that we don't want to include, so
+		 * we skip a PredicateList's children as well.  Note, though,
+		 * that if this visitor is specifically targeted for a particular
+		 * Predicate or AndNode (i.e. a call is directly made to
+		 * Predicate.accept() or AndNode.accept()) then we _will_ descend
+		 * into that predicate's operands and retrieve referenced base
+		 * table numbers.
+		 *
+		 * And finally, if we visit a FromBaseTable we can just grab
+		 * it's number and that's it--there's no need to go any further.
+		 */
+		return (node instanceof FromBaseTable) ||
+			(node instanceof SelectNode) ||
+			(node instanceof PredicateList);
+	}
+
+	/**
+	 * @see org.apache.derby.iapi.sql.compile.Visitor#stopTraversal
+	 */
+	public boolean stopTraversal()
+	{
+		return false;
+	}
+
+}	

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryRelationalOperatorNode.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryRelationalOperatorNode.java?rev=382200&r1=382199&r2=382200&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryRelationalOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryRelationalOperatorNode.java Wed Mar  1 15:36:11 2006
@@ -1123,6 +1123,190 @@
 		
 		return super.genSQLJavaSQLTree();
 	}
+
+	/**
+	 * Take a ResultSetNode and return a column reference that is scoped for
+	 * for the received ResultSetNode, where "scoped" means that the column
+	 * reference points to a specific column in the RSN.  This is used for
+	 * remapping predicates from an outer query down to a subquery. 
+	 *
+	 * For example, assume we have the following query:
+	 *
+	 *  select * from
+	 *    (select i,j from t1 union select i,j from t2) X1,
+	 *    (select a,b from t3 union select a,b from t4) X2
+	 *  where X1.j = X2.b;
+	 *
+	 * Then assume that this BinaryRelationalOperatorNode represents the
+	 * "X1.j = X2.b" predicate and that the childRSN we received as a
+	 * parameter represents one of the subqueries to which we want to push
+	 * the predicate; let's say it's:
+	 *
+	 *    select i,j from t1
+	 *
+	 * Then what we want to do in this method is map one of the operands
+	 * X1.j or X2.b (depending on the 'whichSide' parameter) to the childRSN,
+	 * if possible.  Note that in our example, "X2.b" should _NOT_ be mapped
+	 * because it doesn't apply to the childRSN for the subquery "select i,j
+	 * from t1"; thus we should leave it as it is.  "X1.j", however, _does_
+	 * need to be scoped, and so this method will return a ColumnReference
+	 * pointing to "T1.j" (or whatever the corresponding column in T1 is).
+	 *
+	 * ASSUMPTION: We should only get to this method if we know that
+	 * at least one operand in the predicate to which this operator belongs
+	 * can and should be mapped to the received childRSN. 
+	 *
+     * @param whichSide The operand are we trying to scope (LEFT or RIGHT)
+     * @param parentRSNsTables Set of all table numbers referenced by
+     *  the ResultSetNode that is _parent_ to the received childRSN.
+     *  We need this to make sure we don't scope the operand to a
+     *  ResultSetNode to which it doesn't apply.
+     * @param childRSN The result set node to which we want to create
+     *  a scoped predicate.
+     * @return A column reference scoped to the received childRSN, if possible.
+     *  If the operand is a ColumnReference that is not supposed to be scoped,
+	 *  we return a _clone_ of the reference--this is necessary because the
+	 *  reference is going to be pushed to two places (left and right children
+	 *  of the parentRSN) and if both children are referencing the same
+	 *  instance of the column reference, they'll interfere with each other
+	 *  during optimization.
+	 */
+	public ValueNode getScopedOperand(int whichSide,
+		JBitSet parentRSNsTables, ResultSetNode childRSN)
+		throws StandardException
+	{
+		ResultColumn rc = null;
+		ColumnReference cr = 
+			whichSide == LEFT
+				? (ColumnReference)leftOperand
+				: (ColumnReference)rightOperand;
+
+		// The first thing we need to do is see if this ColumnReference
+		// is supposed to be scoped for childRSN.  We do that by figuring
+		// out what underlying base table the column reference is pointing
+		// to and then seeing if that base table is included in the list of
+		// table numbers from the parentRSN.
+		JBitSet crTables = new JBitSet(parentRSNsTables.size());
+		BaseTableNumbersVisitor btnVis =
+			new BaseTableNumbersVisitor(crTables);
+		cr.accept(btnVis);
+
+		// If the column reference doesn't reference any tables,
+		// then there's no point in mapping it to the child result
+		// set; just return a clone of the operand.
+		if (crTables.getFirstSetBit() == -1)
+		{
+			return (ValueNode)cr.getClone();
+		}
+
+		/* If the column reference in question is not intended for
+		 * the received result set node, just leave the operand as
+		 * it is (i.e. return a clone).  In the example mentioned at
+		 * the start of this method, this will happen when the operand
+		 * is X2.b and childRSN is either "select i,j from t1" or
+		 * "select i,j from t2", in which case the operand does not
+		 * apply to childRSN.  When we get here and try to map the
+		 * "X1.j" operand, though, the following "contains" check will
+		 * return true and thus we can go ahead and return a scoped
+		 * version of that operand.
+		 */
+		if (!parentRSNsTables.contains(crTables))
+		{
+			return (ValueNode)cr.getClone();
+		}
+
+		// If the column reference is already pointing to the
+		// correct table, then there's no need to change it.
+		if ((childRSN.getReferencedTableMap() != null) &&
+			childRSN.getReferencedTableMap().get(cr.getTableNumber()))
+		{
+			return cr;
+		}
+
+		/* Find the target ResultColumn in the received result set.  At
+		 * this point we know that we do in fact need to scope the column
+		 * reference for childRSN, so go ahead and do it.  We get the
+		 * target column by column position instead of by name because
+		 * it's possible that the name given for the query doesn't match
+		 * the name of the actual column we're looking for.  Ex.
+		 *
+		 *  select * from
+		 *    (select i,j from t1 union select i,j from t2) X1 (x,y),
+		 *    (select a,b from t3 union select a,b from t4) X2
+		 *  where X1.x = X2.b;
+		 *
+		 * If we searched for "x" in the childRSN "select i,j from t1"
+		 * we wouldn't find it.  So we have to look based on position.
+		 */
+
+		rc = childRSN.getResultColumns().getResultColumn(cr.getColumnNumber());
+
+		// rc shouldn't be null; if there was no matching ResultColumn at all,
+		// then we shouldn't have made it this far.
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(rc != null,
+				"Failed to locate result column when trying to " +
+				"scope operand '" + cr.getTableName() + "." +
+				cr.getColumnName() + "'.");
+		}
+
+		/* If the ResultColumn we found has an expression that is a
+		 * ColumnReference, then that column reference has all of the info
+		 * we need, with one exception: the columnNumber.  Depending on our
+		 * depth in the tree, the ResultColumn's ColumnReference could be
+		 * pointing to a base column in the FromBaseTable.  In that case the
+		 * ColumnReference will hold the column position as it is with respect
+		 * to the FromBaseTable.  But when we're scoping a column reference,
+		 * we're scoping it to a ResultSetNode that sits (either directly or
+		 * indirectly) above a ProjectRestrictNode that in turn sits above the
+		 * FromBaseTable. This means that the scoped reference's columnNumber
+		 * needs to be with respect to the PRN that sits above the base table,
+		 * _not_ with respect to the FromBaseTable itself.  This is important
+		 * because column "1" in the PRN might not be column "1" in the
+		 * underlying base table. For example, if we have base table TT with
+		 * four columns (a int, b int, i int, j int) and the PRN above it only
+		 * projects out columns (i,j), then column "1" for the PRN is "i", but
+		 * column "1" for base table TT is "a".  On the flip side, column "3"
+		 * for base table TT is "i", but if we search the PRN's result columns
+		 * (which match the result columns for the ResultSetNode to which
+		 * we're scoping) for column "3", we won't find it.
+		 *
+		 * So what does all of that mean?  It means that if the ResultColumn
+		 * we found has an expression that's a ColumnReference, we can simply
+		 * return that ColumnReference IF we set it's columnNumber correctly.
+		 * Thankfully the column reference we're trying to scope ("cr") came
+		 * from further up the tree and so it knows what the correct column
+		 * position (namely, the position w.r.t the ProjectRestrictNode above
+		 * the FromBaseTable) needs to be.  So that's the column number we
+		 * use.
+		 */
+		if (rc.getExpression() instanceof ColumnReference)
+		{
+			// Make sure the ColumnReference's columnNumber is correct,
+			// then just return that reference.  Note: it's okay to overwrite
+			// the columnNumber directly because when it eventually makes
+			// it down to the PRN over the FromBaseTable, it will be remapped
+			// for the FromBaseTable and the columnNumber will then be set
+			// correctly.  That remapping is done in the pushOptPredicate()
+			// method of ProjectRestrictNode.
+			ColumnReference cRef = (ColumnReference)rc.getExpression();
+			cRef.setColumnNumber(cr.getColumnNumber());
+			return cRef;
+		}
+
+		/* We can get here if the ResultColumn's expression isn't a
+		 * ColumnReference.  For example, the expression would be a
+		 * constant expression if childRSN represented something like:
+		 *
+		 *   select 1, 1 from t1
+		 *
+		 * In this case we just return the column reference as it is
+		 * because it's scoped as far as we can take it.
+		 */
+		return cr;
+	}
+
 }	
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ColumnReference.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ColumnReference.java?rev=382200&r1=382199&r2=382200&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ColumnReference.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ColumnReference.java Wed Mar  1 15:36:11 2006
@@ -499,6 +499,18 @@
 	}
 
 	/**
+	 * Set the column number for this ColumnReference.  This is
+	 * used when scoping predicates for pushdown.
+	 *
+	 * @param columnName	The new column number.
+	 */
+
+	public void setColumnNumber(int colNum)
+	{
+		this.columnNumber = colNum;
+	}
+
+	/**
 	 * Get the source this columnReference
 	 *
 	 * @return	The source of this columnReference
@@ -674,6 +686,17 @@
 		columnNumber = origColumnNumber;
 	}
 
+	/**
+	 * Returns true if this ColumnReference has been remapped; false
+	 * otherwise.
+	 *
+	 * @return Whether or not this ColumnReference has been remapped.
+	 */
+	protected boolean hasBeenRemapped()
+	{
+		return (origSource != null);
+	}
+
 	/*
 	 * Get the ResultColumn that the source points to.  This is useful for
 	 * getting what the source will be after this ColumnReference is remapped.
@@ -873,7 +896,7 @@
 		{
 			if (sourceResultSetNumber < 0)
 			{
-				SanityManager.THROWASSERT("sourceResultSetNumber expected to be >= 0");
+				SanityManager.THROWASSERT("sourceResultSetNumber expected to be >= 0 for " + getTableName() + "." + getColumnName());
 			}
 		}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java?rev=382200&r1=382199&r2=382200&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java Wed Mar  1 15:36:11 2006
@@ -74,6 +74,10 @@
 	 */
 	private Hashtable searchClauseHT;
 
+	// Whether or not this predicate has been scoped; see the
+	// getPredScopedForResultSet() method of this class for more.
+	private boolean scoped;
+
 	/**
 	 * Initializer.
 	 *
@@ -86,6 +90,7 @@
 		this.andNode = (AndNode) andNode;
 		pushable = false;
 		this.referencedSet = (JBitSet) referencedSet;
+		scoped = false;
 	}
 
 	/*
@@ -360,6 +365,20 @@
 	}
 
 	/**
+	 * Set whether or not this predicate is pushable.  This method
+	 * is intended for use when creating a copy of the predicate, ex
+	 * for predicate pushdown.  We choose not to add this assignment
+	 * to copyFields() because the comments for that method say that
+	 * it should copy all fields _except_ the two specified at init
+	 * time; "pushable" is one of the two specified at init time.
+	 *
+	 * @param pushable Whether or not the predicate is pushable.
+	 */
+	public void setPushable(boolean pushable) {
+		this.pushable = pushable;
+	}
+
+	/**
 	 * Return the referencedSet.
 	 *
 	 * @return JBitSet	The referencedSet.
@@ -699,8 +718,8 @@
 	{
 		if (SanityManager.DEBUG)
 		{
-			return "referencedSet: " + referencedSet  + "\n" +
-			   "pushable: " + pushable + "\n" +
+			return binaryRelOpColRefsToString() + "\nreferencedSet: " +
+				referencedSet  + "\n" + "pushable: " + pushable + "\n" +
 				super.toString();
 		}
 		else
@@ -710,6 +729,58 @@
 	}
 
 	/**
+	 * Get a string version of the column references for this predicate
+	 * IF it's a binary relational operator.  We only print out the
+	 * names of the operands if they are column references; otherwise
+	 * we just print a dummy value.  This is for debugging purposes
+	 * only--it's a convenient way to see what columns the predicate
+	 * is referencing, especially when tracing through code and printing
+	 * assert failure.
+	 */
+	public String binaryRelOpColRefsToString()
+	{
+		// We only consider binary relational operators here.
+		if (!(getAndNode().getLeftOperand()
+			instanceof BinaryRelationalOperatorNode))
+		{
+			return "";
+		}
+
+		final String DUMMY_VAL = "<expr>";
+		java.lang.StringBuffer sBuf = new java.lang.StringBuffer();
+		BinaryRelationalOperatorNode opNode =
+			(BinaryRelationalOperatorNode)getAndNode().getLeftOperand();
+
+		// Get left operand's name.
+		if (opNode.getLeftOperand() instanceof ColumnReference)
+		{
+			sBuf.append(
+				((ColumnReference)opNode.getLeftOperand()).getTableName() +
+				"." +
+				((ColumnReference)opNode.getLeftOperand()).getColumnName()
+			);
+		}
+		else
+			sBuf.append(DUMMY_VAL);
+
+		// Get the operator type.
+		sBuf.append(" " + opNode.operator + " ");
+
+		// Get right operand's name.
+		if (opNode.getRightOperand() instanceof ColumnReference) {
+			sBuf.append(
+				((ColumnReference)opNode.getRightOperand()).getTableName() +
+				"." +
+				((ColumnReference)opNode.getRightOperand()).getColumnName()
+			);
+		}
+		else
+			sBuf.append(DUMMY_VAL);
+
+		return sBuf.toString();
+	}
+
+	/**
 	 * Prints the sub-nodes of this object.  See QueryTreeNode.java for
 	 * how tree printing is supposed to work.
 	 *
@@ -775,6 +846,177 @@
 
 	public Hashtable getSearchClauseHT() {
 		return searchClauseHT;
+	}
+
+	/**
+	 * Determine whether or not this predicate is eligible for
+	 * push-down into subqueries.  Right now the only predicates
+	 * we consider to be eligible are those which 1) are Binary
+	 * Relational operator nodes, and 2) have a column reference
+	 * on BOTH sides.
+	 *
+	 * @return Whether or not this predicate is eligible to be
+	 *  pushed into subqueries.
+	 */
+	protected boolean pushableToSubqueries()
+		throws StandardException
+	{
+		// If the predicate isn't a binary relational operator,
+		// then we don't push it.
+		if (!(getAndNode().getLeftOperand()
+			instanceof BinaryRelationalOperatorNode))
+		{
+			return false;
+		}
+
+		BinaryRelationalOperatorNode opNode =
+			(BinaryRelationalOperatorNode)getAndNode().getLeftOperand();
+
+		return ((opNode.getLeftOperand() instanceof ColumnReference) &&
+				(opNode.getRightOperand() instanceof ColumnReference));
+	}
+
+	/**
+	 * If this predicate's operator is a BinaryRelationalOperatorNode,
+	 * then look at the operands and return a new, equivalent predicate
+	 * that is "scoped" to the received ResultSetNode.  By "scoped" we
+	 * mean that the operands, which shold be column references, have been
+	 * mapped to the appropriate result columns in the received RSN.
+	 * This is useful for pushing predicates from outer queries down
+	 * into inner queries, in which case the column references need
+	 * to be remapped.
+	 *
+	 * For example, let V1 represent
+	 *
+	 *    select i,j from t1 UNION select i,j from t2
+	 * 
+	 * and V2 represent
+	 *
+	 *    select a,b from t3 UNION select a,b from t4
+	 * 
+	 * Then assume we have the following query:
+	 *
+	 *    select * from V1, V2 where V1.j = V2.b
+	 *
+	 * Let's further assume that this Predicate object represents the
+	 * "V1.j = V2.b" operator and that the childRSN we received
+	 * as a parameter represents one of the subqueries to which we
+	 * want to push the predicate; let's say it's:
+	 *
+	 *    select i,j from t1
+	 *
+	 * Then this method will return a new predicate whose binary
+	 * operator represents the expression "T1.j = V2.b" (that is, V1.j
+	 * will be mapped to the corresponding column in T1).  For more on
+	 * how that mapping is made, see the "getScopedOperand()" method
+	 * in BinaryRelationalOperatorNode.java.
+	 *
+	 * ASSUMPTION: We should only get to this method if we know that
+	 * at least one operand in this predicate can and should be mapped
+	 * to the received childRSN.  For an example of where that check is
+	 * made, see the pushOptPredicate() method in SetOperatorNode.java.
+	 *
+	 * @param parentRSNsTables Set of all table numbers referenced by
+	 *  the ResultSetNode that is _parent_ to the received childRSN.
+	 *  We need this to make sure we don't scope the operands to a
+	 *  ResultSetNode to which they don't apply.
+	 * @param childRSN The result set node for which we want to create
+	 *  a scoped predicate.
+	 * @return A new predicate whose operands have been scoped to the
+	 *  received childRSN.
+	 */
+	protected Predicate getPredScopedForResultSet(
+		JBitSet parentRSNsTables, ResultSetNode childRSN)
+		throws StandardException
+	{
+		// We only deal with binary relational operators here.
+		if (!(getAndNode().getLeftOperand()
+			instanceof BinaryRelationalOperatorNode))
+		{
+			return this;
+		}
+
+		// The predicate must have an AndNode in CNF, so we
+		// need to create an AndNode representing:
+		//    <scoped_bin_rel_op> AND TRUE
+		// First create the boolean constant for TRUE.
+		ValueNode trueNode = (ValueNode) getNodeFactory().getNode(
+			C_NodeTypes.BOOLEAN_CONSTANT_NODE,
+			Boolean.TRUE,
+			getContextManager());
+
+		BinaryRelationalOperatorNode opNode =
+			(BinaryRelationalOperatorNode)getAndNode().getLeftOperand();
+
+		// Create a new op node with left and right operands that point
+		// to the received result set's columns as appropriate.
+		BinaryRelationalOperatorNode newOpNode = 
+			(BinaryRelationalOperatorNode) getNodeFactory().getNode(
+				opNode.getNodeType(),
+				opNode.getScopedOperand(
+					BinaryRelationalOperatorNode.LEFT,
+					parentRSNsTables,
+					childRSN),
+				opNode.getScopedOperand(
+					BinaryRelationalOperatorNode.RIGHT,
+					parentRSNsTables,
+					childRSN),
+				getContextManager());
+
+		// Bind the new op node.
+		newOpNode.bindComparisonOperator();
+
+		// Create and bind a new AND node in CNF form,
+		// i.e. "<newOpNode> AND TRUE".
+		AndNode newAnd = (AndNode) getNodeFactory().getNode(
+			C_NodeTypes.AND_NODE,
+			newOpNode,
+			trueNode,
+			getContextManager());
+		newAnd.postBindFixup();
+
+		// Categorize the new AND node; among other things, this
+		// call sets up the new operators's referenced table map,
+		// which is important for correct pushing of the new
+		// predicate.
+		JBitSet tableMap = new JBitSet(
+			childRSN.getReferencedTableMap().size());
+		newAnd.categorize(tableMap, false);
+
+		// Now put the pieces together to get a new predicate.
+		Predicate newPred = (Predicate) getNodeFactory().getNode(
+			C_NodeTypes.PREDICATE,
+			newAnd,
+			tableMap,
+			getContextManager());
+
+		// Copy all of this predicates other fields into the new predicate.
+		newPred.clearScanFlags();
+		newPred.copyFields(this);
+		newPred.setPushable(getPushable());
+
+		// Take note of the fact that the new predicate is scoped for
+		// the sake of pushing; we need this information during optimization
+		// to figure out what we should and should not "pull" back up.
+		newPred.markAsScopedForPush();
+		return newPred;
+	}
+
+	/**
+	 * Indicate that this predicate is a scoped copy of some other
+	 * predicate (i.e. it was created as the result of a call to
+	 * getPredScopedForResultSet() on some other predicate).
+	 */
+	protected void markAsScopedForPush() {
+		this.scoped = true;
+	}
+
+	/**
+	 * Return whether or not this predicate is a scoped copy of
+	 * another predicate.
+	 */
+	protected boolean isScopedForPush() {
+		return scoped;
 	}
 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java?rev=382200&r1=382199&r2=382200&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumn.java Wed Mar  1 15:36:11 2006
@@ -1740,4 +1740,50 @@
 			}
 		}
 	}
+
+	/**
+	 * Search the tree beneath this ResultColumn until we find
+	 * the number of the table to which this RC points, and
+	 * return that table number.  If we can't determine which
+	 * table this RC is for, then return -1.
+	 *
+	 * There are two places we can find the table number: 1) if
+	 * our expression is a ColumnReference, then we can get the
+	 * target table number from the ColumnReference and that's
+	 * it; 2) if expression is a VirtualColumnNode, then if
+	 * the VirtualColumnNode points to a FromBaseTable, we can
+	 * get that FBT's table number; otherwise, we walk the
+	 * VirtualColumnNode-ResultColumn chain and do a recursive
+	 * search.
+	 *
+	 * @return The number of the table to which this ResultColumn	
+	 *  points, or -1 if we can't determine that from where we are.
+	 */
+	public int getTableNumber()
+		throws StandardException
+	{
+		if (expression instanceof ColumnReference)
+			return ((ColumnReference)expression).getTableNumber();
+		else if (expression instanceof VirtualColumnNode)
+		{
+			VirtualColumnNode vcn = (VirtualColumnNode)expression;
+
+			// If the VCN points to a FromBaseTable, just get that
+			// table's number.
+			if (vcn.getSourceResultSet() instanceof FromBaseTable)
+			{
+				return ((FromBaseTable)vcn.getSourceResultSet()).
+					getTableNumber();
+			}
+
+			// Else recurse down the VCN.
+			return vcn.getSourceColumn().getTableNumber();
+		}
+
+		// We can get here if expression has neither a column
+		// reference nor a FromBaseTable beneath it--for example,
+		// if it is of type BaseColumnNode. 
+		return -1;
+	}
+
 }