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/01/18 21:46:54 UTC

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

Author: bandaram
Date: Wed Jan 18 12:46:47 2006
New Revision: 370254

URL: http://svn.apache.org/viewcvs?rev=370254&view=rev
Log:
DERBY-772: Optimize IN predicates on derived tables with UNIONs by pushing
them into underlying select statements.

Add some test cases to show index being used. Will also add more tests in
another pending change to improve finding matching column.

Submitted by Satheesh Bandaram (satheesh@sourcery.org)

Modified:
    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/SelectNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/predicatesIntoViews.sql

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java?rev=370254&r1=370253&r2=370254&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java Wed Jan 18 12:46:47 2006
@@ -1394,42 +1394,44 @@
 				}
 			}
 
-			if (state && !copyPredicate)
-			{
-				// keep the counters up to date when removing a predicate
-				if (predicate.isStartKey())
-					numberOfStartPredicates--;
-				if (predicate.isStopKey())
-					numberOfStopPredicates--;
-				if (predicate.isQualifier())
-					numberOfQualifiers--;
-
-				/* Clear all of the scan flags since they may be different
-				 * due to the splitting of the list.
-				 */
-				predicate.clearScanFlags();
-				// Remove this predicate from the list
-				removeElementAt(index);
-			}
+			if (!state)
+				continue;
 
 			if (copyPredicate)
 			{
 				// Copy this predicate and push this instead
 				AndNode andNode = predicate.getAndNode();
-
-				// Make sure we are only pushing simple binary relations for now
-				// It should be benificial to push expressions that can be pushed, so they can be applied
-				// closer to the data.
-				if (! (andNode.getLeftOperand() instanceof BinaryRelationalOperatorNode))
-					continue;
-
-				BinaryRelationalOperatorNode opNode = (BinaryRelationalOperatorNode) andNode.getLeftOperand();
-				if (! (opNode.getLeftOperand() instanceof ColumnReference) ||
-				    ! (opNode.getRightOperand() instanceof ConstantNode ||
-					 opNode.getRightOperand() instanceof ParameterNode))
+				ValueNode leftOperand;
+				ColumnReference crNode;
+				BinaryRelationalOperatorNode opNode=null;
+				InListOperatorNode inNode=null;
+
+				// Make sure we are only pushing binary relations and InList for
+				// copyPredicate case. It should be benificial to push expressions that
+				// can be pushed, so they can be applied closer to the data.
+
+				if (andNode.getLeftOperand() instanceof BinaryRelationalOperatorNode)
+				{
+					opNode = (BinaryRelationalOperatorNode) andNode.getLeftOperand();
+					// Investigate using invariant interface to check rightOperand
+					if (! (opNode.getLeftOperand() instanceof ColumnReference) ||
+					    ! (opNode.getRightOperand() instanceof ConstantNode ||
+							 opNode.getRightOperand() instanceof ParameterNode))
+						continue;
+
+					crNode = (ColumnReference) opNode.getLeftOperand();
+				}
+				else if (andNode.getLeftOperand() instanceof InListOperatorNode)
+				{
+					inNode = (InListOperatorNode) andNode.getLeftOperand();
+					if (! (inNode.getRightOperandList().isConstantExpression()))
+						continue;
+
+					crNode = (ColumnReference) inNode.getLeftOperand();
+				}
+				else
 					continue;
 
-				ColumnReference crNode = (ColumnReference) opNode.getLeftOperand();
 				// Remap this crNode to underlying column reference in the select, if possible.
 				ColumnReference newCRNode = select.findColumnReferenceInResult(crNode.columnName);
 				if (newCRNode == null)
@@ -1437,37 +1439,69 @@
 
 				// Create a copy of the predicate to push down
 				// <column> <relop> <value> AND TRUE
-				BinaryRelationalOperatorNode newRelop = (BinaryRelationalOperatorNode)
+				if (andNode.getLeftOperand() instanceof BinaryRelationalOperatorNode)
+				{
+					BinaryRelationalOperatorNode newRelop = (BinaryRelationalOperatorNode)
 							getNodeFactory().getNode(
 										opNode.getNodeType(),
 										newCRNode,
 										opNode.getRightOperand(),
 										getContextManager());
+					newRelop.bindComparisonOperator();
+					leftOperand = newRelop;
+				}
+				else
+				{
+					InListOperatorNode newInNode = (InListOperatorNode)
+							getNodeFactory().getNode(
+								C_NodeTypes.IN_LIST_OPERATOR_NODE,
+								newCRNode,
+								inNode.getRightOperandList(),
+								getContextManager());
+					newInNode.setType(inNode.getTypeServices());
+					leftOperand = newInNode;
+				}
 
-				newRelop.bindComparisonOperator();
-
+				// Convert the predicate into CNF form
 				ValueNode trueNode = (ValueNode) getNodeFactory().getNode(
 										C_NodeTypes.BOOLEAN_CONSTANT_NODE,
 										Boolean.TRUE,
 										getContextManager());
 				AndNode newAnd = (AndNode) getNodeFactory().getNode(
 													C_NodeTypes.AND_NODE,
-													newRelop,
+													leftOperand,
 													trueNode,
 													getContextManager());
 				newAnd.postBindFixup();
 				JBitSet tableMap = new JBitSet(select.referencedTableMap.size());
-				Predicate newPred = (Predicate) getNodeFactory().getNode(
+
+				// Use newly constructed predicate
+				predicate = (Predicate) getNodeFactory().getNode(
 												C_NodeTypes.PREDICATE,
 												newAnd,
 												tableMap,
 												getContextManager());
-				predicate = newPred;
+			}
+			else
+			{
+				// keep the counters up to date when removing a predicate
+				if (predicate.isStartKey())
+					numberOfStartPredicates--;
+				if (predicate.isStopKey())
+					numberOfStopPredicates--;
+				if (predicate.isQualifier())
+					numberOfQualifiers--;
+
+				/* Clear all of the scan flags since they may be different
+				 * due to the splitting of the list.
+				 */
+				predicate.clearScanFlags();
+				// Remove this predicate from the list
+				removeElementAt(index);
 			}
 
 			// Push it into the select
-			if (state)
- 				select.pushExpressionsIntoSelect(predicate);
+ 			select.pushExpressionsIntoSelect(predicate);
 		}		
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java?rev=370254&r1=370253&r2=370254&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/SelectNode.java Wed Jan 18 12:46:47 2006
@@ -293,16 +293,16 @@
 		return groupByList;
 	}
 
-	/*
-	 * DERBY-649: Find colName in the result columns and return underlying
-	 * columnReference.  This is useful for pushing union predicates into underlying
-	 * select statements.
+	/**
+	 * Find colName in the result columns and return underlying columnReference.
+	 * Note that this function returns null if there are more than one FromTable
+	 * for this SelectNode and the columnReference needs to be directly under
+	 * the resultColumn. So having an expression under the resultSet would cause
+	 * returning null.
 	 *
-	 * Handle the case of single table selects for now. Also if there is an
-	 * expression under the result column, it is not possible yet to push the
-	 * predicates for now.
+	 * @param	colName		Name of the column
 	 *
-	 * @return ColumnReference	If colName could be remapped to a table reference
+	 * @return	ColumnReference	ColumnReference to the column, if found
 	 */
 	public ColumnReference findColumnReferenceInResult(String colName)
 					throws StandardException

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java?rev=370254&r1=370253&r2=370254&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UnionNode.java Wed Jan 18 12:46:47 2006
@@ -257,23 +257,35 @@
 		return costEstimate;
 	}
 
-	/*
-	 * DERBY-649: Handle pushing predicates into UnionNodes. For now, we only push simple
-	 * single column predicates that are binaryOperations. It should be possible to expand
-	 * this logic to cover more cases. Even pushing expressions (like a+b = 10) into SELECTs
-	 * would improve performance, even if they don't make Qualifiers. It would mean
-	 * evaluating expressions closer to data and hence could avoid sorting or other
-	 * overheads that UNION may require.
+	/**
+	 * DERBY-649: Handle pushing predicates into UnionNodes. It is possible to push
+	 * single table predicates that are binaryOperations or inListOperations. 
+	 *
+	 * Predicates of the form <columnReference> <RELOP> <constant> or <columnReference>
+	 * IN <constantList> are currently handled. Since these predicates would allow
+	 * optimizer to pick available indices, pushing them provides maximum benifit.
+	 *
+	 * It should be possible to expand this logic to cover more cases. Even pushing
+	 * expressions (like a+b = 10) into SELECTs would improve performance, even if
+	 * they don't allow use of index. It would mean evaluating expressions closer to
+	 * data and hence could avoid sorting or other overheads that UNION may require.
 	 *
 	 * Note that the predicates are not removed after pushing. This is to ensure if
 	 * pushing is not possible or only partially feasible.
+	 *
+	 * @param 	predicateList		List of single table predicates to push
+	 *
+	 * @return	Nothing
+	 *
+	 * @exception	StandardException		Thrown on error
 	 */
 	public void pushExpressions(PredicateList predicateList)
 					throws StandardException
 	{
 		// If left or right side is a UnionNode, further push the predicate list
 		// Note, it is OK not to push these predicates since they are also evaluated
-		// in the ProjectRestrictNode.
+		// in the ProjectRestrictNode. There are other types of operations possible
+		// here in addition to UnionNode or SelectNode, like RowResultSetNode.
 		if (leftResultSet instanceof UnionNode)
 			((UnionNode)leftResultSet).pushExpressions(predicateList);
 		else if (leftResultSet instanceof SelectNode)

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/predicatesIntoViews.out?rev=370254&r1=370253&r2=370254&view=diff
==============================================================================
--- 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 Jan 18 12:46:47 2006
@@ -1977,6 +1977,8 @@
 ij> create view test.view1(a,b) as select all a,b from test.table1 union all select a,b from test.table2
 			union all select 1,1 from test.table1;
 0 rows inserted/updated/deleted
+ij> create view test.view2(c,d) as select all a+1,b+1 from test.table1 union all select a,b from test.table2;
+0 rows inserted/updated/deleted
 ij> -- Following Selects using the tables directly would use index
 CALL SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(1);
 0 rows inserted/updated/deleted
@@ -2244,6 +2246,388 @@
 	Ordered null semantics on the following columns: 
 					qualifiers:
 None
+ij> -- Can't use index for the following
+select a from test.view0 where b=25+a;
+A          
+-----------
+ij> VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	null
+Statement Text: 
+	-- Can't use index for the following
+select a from test.view0 where b=25+a
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (5):
+Number of opens = 1
+Rows seen = 0
+Rows filtered = 0
+restriction = false
+projection = true
+	constructor time (milliseconds) = 0
+	open time (milliseconds) = 0
+	next time (milliseconds) = 0
+	close time (milliseconds) = 0
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Project-Restrict ResultSet (4):
+	Number of opens = 1
+	Rows seen = 0
+	Rows filtered = 0
+	restriction = true
+	projection = false
+		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:
+		Union ResultSet:
+		Number of opens = 1
+		Rows seen from the left = 0
+		Rows seen from the right = 0
+		Rows returned = 0
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+		Left result set:
+			Table Scan ResultSet for TABLE1 at serializable isolation level using share table 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={0, 1}
+				Number of columns fetched=2
+				Number of pages visited=1
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=heap
+				start position: 
+null				stop position: 
+null				qualifiers:
+None
+		Right result set:
+			Table Scan ResultSet for TABLE2 at serializable isolation level using share table 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={0, 1}
+				Number of columns fetched=2
+				Number of pages visited=1
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=heap
+				start position: 
+null				stop position: 
+null				qualifiers:
+None
+ij> -- Inlist tests
+select * from test.view0 where b in (1, 2, 3);
+A          |B          
+-----------------------
+ij> VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	null
+Statement Text: 
+	-- Inlist tests
+select * from test.view0 where b in (1, 2, 3)
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (8):
+Number of opens = 1
+Rows seen = 0
+Rows filtered = 0
+restriction = true
+projection = false
+	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:
+	Union ResultSet:
+	Number of opens = 1
+	Rows seen from the left = 0
+	Rows seen from the right = 0
+	Rows returned = 0
+		constructor time (milliseconds) = 0
+		open time (milliseconds) = 0
+		next time (milliseconds) = 0
+		close time (milliseconds) = 0
+	Left result set:
+		Project-Restrict ResultSet (4):
+		Number of opens = 1
+		Rows seen = 0
+		Rows filtered = 0
+		restriction = true
+		projection = true
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			restriction time (milliseconds) = 0
+			projection time (milliseconds) = 0
+		Source result set:
+			Index Row to Base Row ResultSet for TABLE1:
+			Number of opens = 1
+			Rows seen = 0
+			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 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: 
+	>= 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
+	Right result set:
+		Project-Restrict ResultSet (7):
+		Number of opens = 1
+		Rows seen = 0
+		Rows filtered = 0
+		restriction = true
+		projection = true
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			restriction time (milliseconds) = 0
+			projection time (milliseconds) = 0
+		Source result set:
+			Index Row to Base Row ResultSet for TABLE2:
+			Number of opens = 1
+			Rows seen = 0
+			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 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: 
+	>= 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> prepare s as 'select * from test.view0 where b in (?, ?, ?)';
+ij> execute s using 'values (1,2,3)';
+IJ WARNING: Autocommit may close using result set
+A          |B          
+-----------------------
+ij> VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	null
+Statement Text: 
+	select * from test.view0 where b in (?, ?, ?)
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (8):
+Number of opens = 1
+Rows seen = 0
+Rows filtered = 0
+restriction = true
+projection = false
+	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:
+	Union ResultSet:
+	Number of opens = 1
+	Rows seen from the left = 0
+	Rows seen from the right = 0
+	Rows returned = 0
+		constructor time (milliseconds) = 0
+		open time (milliseconds) = 0
+		next time (milliseconds) = 0
+		close time (milliseconds) = 0
+	Left result set:
+		Project-Restrict ResultSet (4):
+		Number of opens = 1
+		Rows seen = 0
+		Rows filtered = 0
+		restriction = true
+		projection = true
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			restriction time (milliseconds) = 0
+			projection time (milliseconds) = 0
+		Source result set:
+			Index Row to Base Row ResultSet for TABLE1:
+			Number of opens = 1
+			Rows seen = 0
+			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 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: 
+	>= 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
+	Right result set:
+		Project-Restrict ResultSet (7):
+		Number of opens = 1
+		Rows seen = 0
+		Rows filtered = 0
+		restriction = true
+		projection = true
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+			restriction time (milliseconds) = 0
+			projection time (milliseconds) = 0
+		Source result set:
+			Index Row to Base Row ResultSet for TABLE2:
+			Number of opens = 1
+			Rows seen = 0
+			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 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: 
+	>= 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> -- This select should use index for first two selects, table scan for the third
 select a from test.view1 where b=25;
 A          
@@ -2391,6 +2775,117 @@
 					stop position: 
 	None
 					qualifiers:
+None
+ij> select d from test.view2 where d=25;
+D          
+-----------
+ij> VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS();
+1                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                    
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -----------------------------------
+Statement Name: 
+	null
+Statement Text: 
+	select d from test.view2 where d=25
+Parse Time: 0
+Bind Time: 0
+Optimize Time: 0
+Generate Time: 0
+Compile Time: 0
+Execute Time: 0
+Begin Compilation Timestamp : null
+End Compilation Timestamp : null
+Begin Execution Timestamp : null
+End Execution Timestamp : null
+Statement Execution Plan Text: 
+Project-Restrict ResultSet (6):
+Number of opens = 1
+Rows seen = 0
+Rows filtered = 0
+restriction = false
+projection = true
+	constructor time (milliseconds) = 0
+	open time (milliseconds) = 0
+	next time (milliseconds) = 0
+	close time (milliseconds) = 0
+	restriction time (milliseconds) = 0
+	projection time (milliseconds) = 0
+Source result set:
+	Project-Restrict ResultSet (5):
+	Number of opens = 1
+	Rows seen = 0
+	Rows filtered = 0
+	restriction = true
+	projection = false
+		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:
+		Union ResultSet:
+		Number of opens = 1
+		Rows seen from the left = 0
+		Rows seen from the right = 0
+		Rows returned = 0
+			constructor time (milliseconds) = 0
+			open time (milliseconds) = 0
+			next time (milliseconds) = 0
+			close time (milliseconds) = 0
+		Left result set:
+			Project-Restrict ResultSet (3):
+			Number of opens = 1
+			Rows seen = 0
+			Rows filtered = 0
+			restriction = false
+			projection = true
+				constructor time (milliseconds) = 0
+				open time (milliseconds) = 0
+				next time (milliseconds) = 0
+				close time (milliseconds) = 0
+				restriction time (milliseconds) = 0
+				projection time (milliseconds) = 0
+			Source result set:
+				Table Scan ResultSet for TABLE1 at serializable isolation level using share table 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={0, 1}
+					Number of columns fetched=2
+					Number of pages visited=1
+					Number of rows qualified=0
+					Number of rows visited=0
+					Scan type=heap
+					start position: 
+null					stop position: 
+null					qualifiers:
+None
+		Right result set:
+			Table Scan ResultSet for TABLE2 at serializable isolation level using share table 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={0, 1}
+				Number of columns fetched=2
+				Number of pages visited=1
+				Number of rows qualified=0
+				Number of rows visited=0
+				Scan type=heap
+				start position: 
+null				stop position: 
+null				qualifiers:
 None
 ij> CALL SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(0);
 0 rows inserted/updated/deleted

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/predicatesIntoViews.sql
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/predicatesIntoViews.sql?rev=370254&r1=370253&r2=370254&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/predicatesIntoViews.sql (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/predicatesIntoViews.sql Wed Jan 18 12:46:47 2006
@@ -98,6 +98,8 @@
 create view test.view1(a,b) as select all a,b from test.table1 union all select a,b from test.table2
 			union all select 1,1 from test.table1;
  
+create view test.view2(c,d) as select all a+1,b+1 from test.table1 union all select a,b from test.table2; 
+
 -- Following Selects using the tables directly would use index
 CALL SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(1); 
 select a from test.table1 where b=25; 
@@ -109,10 +111,26 @@
 select a from test.view0 where b=25; 
 VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
 
+-- Can't use index for the following
+select a from test.view0 where b=25+a;
+VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
+
+-- Inlist tests
+select * from test.view0 where b in (1, 2, 3);
+VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
+
+prepare s as 'select * from test.view0 where b in (?, ?, ?)';
+
+execute s using 'values (1,2,3)';
+VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
+
 -- This select should use index for first two selects, table scan for the third
 select a from test.view1 where b=25; 
 VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
 
+select d from test.view2 where d=25; 
+VALUES SYSCS_UTIL.SYSCS_GET_RUNTIMESTATISTICS(); 
+
 CALL SYSCS_UTIL.SYSCS_SET_RUNTIMESTATISTICS(0); 
 
 -- Beetle 4983, customer case, complex query involving views, wrong column remapping
@@ -4252,4 +4270,4 @@
 select * from V1 where B  <> x'83452213' ;
 
 DROP VIEW V1;
-DROP TABLE D1;
\ No newline at end of file
+DROP TABLE D1;