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 rh...@apache.org on 2009/10/29 20:55:57 UTC

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

Author: rhillegas
Date: Thu Oct 29 19:55:56 2009
New Revision: 831072

URL: http://svn.apache.org/viewvc?rev=831072&view=rev
Log:
DERBY-4357: Pass projections and restrictions through to RestrictedVTIs.

Added:
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java   (with props)
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromVTI.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MethodCallNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/VTIResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/vti/Restriction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java
    db/derby/code/trunk/tools/javadoc/publishedapi.ant

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java Thu Oct 29 19:55:56 2009
@@ -644,16 +644,15 @@
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param constructor		The GeneratedMethod for the user's constructor
 		@param javaClassName	The java class name for the VTI
-		@param erdNumber		int for referenced column BitSet 
-								(so it can be turned back into an object)
+		@param erdNumber		int for referenced column BitSet (so it can be turned back into an object)
 		@param version2			Whether or not VTI is a version 2 VTI.
 		@param isTarget			Whether or not VTI is a target VTI.
-		@param optimizerEstimatedRowCount	Estimated total # of rows by
-											optimizer
+		@param optimizerEstimatedRowCount	Estimated total # of rows by optimizer
 		@param optimizerEstimatedCost		Estimated total cost by optimizer
 		@param isDerbyStyleTableFunction    True if this is a Derby-style table function
-		@param returnTypeNumber	Which saved object contains the return type
-								(a multi-set) serialized as a byte array
+		@param returnTypeNumber	Which saved object contains the return type (a multi-set) serialized as a byte array
+		@param vtiProjectionNumber	Which saved object contains the projection for a RestrictedVTI
+		@param vtiRestrictionNumber	Which saved object contains the restriction for a RestrictedVTI
 		@return the row as a result set.
 		@exception StandardException thrown when unable to create the
 			result set
@@ -671,7 +670,9 @@
 									 double optimizerEstimatedRowCount,
 									 double optimizerEstimatedCost,
                                      boolean isDerbyStyleTableFunction,
-                                     int returnTypeNumber
+                                     int returnTypeNumber,
+                                     int vtiProjectionNumber,
+                                     int vtiRestrictionNumber
                                           )
 		 throws StandardException;
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromVTI.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromVTI.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromVTI.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromVTI.java Thu Oct 29 19:55:56 2009
@@ -63,6 +63,8 @@
 import org.apache.derby.catalog.types.RoutineAliasInfo;
 
 import org.apache.derby.vti.DeferModification;
+import org.apache.derby.vti.RestrictedVTI;
+import org.apache.derby.vti.Restriction;
 import org.apache.derby.vti.VTICosting;
 import org.apache.derby.vti.VTIEnvironment;
 
@@ -76,6 +78,7 @@
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 
 import java.sql.PreparedStatement;
@@ -85,6 +88,8 @@
 import java.sql.Types;
 
 import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Properties; 
 import java.util.Vector;
 import org.apache.derby.iapi.services.io.FormatableHashtable;
@@ -108,6 +113,7 @@
 	boolean				materializable;
 	boolean				isTarget;
 	boolean				isDerbyStyleTableFunction;
+	boolean				isRestrictedTableFunction;
 	ResultSet			rs;
 
 	private	FormatableHashtable	compileTimeConstants;
@@ -150,6 +156,9 @@
     private boolean isInsensitive;
     private int resultSetType = ResultSet.TYPE_FORWARD_ONLY;
 
+    private String[] projectedColumnNames; // for RestrictedVTIs
+    private Restriction vtiRestriction; // for RestrictedVTIs
+
     /**
 	 * @param invocation		The constructor or static method for the VTI
 	 * @param correlationName	The correlation name
@@ -480,6 +489,7 @@
 		version2 = true;
 	}
 
+
 	/**
 	 * Bind the non VTI tables in this ResultSetNode.  This includes getting their
 	 * descriptors from the data dictionary and numbering them.
@@ -554,7 +564,14 @@
             )			{
             isDerbyStyleTableFunction = true;
         }
-        
+
+        if ( isDerbyStyleTableFunction )
+        {
+            Method boundMethod = (Method) methodCall.getResolvedMethod();
+
+            isRestrictedTableFunction = RestrictedVTI.class.isAssignableFrom( boundMethod.getReturnType() );
+        }
+
 		/* If we have a valid constructor, does class implement the correct interface? 
 		 * If version2 is true, then it must implement PreparedStatement, otherwise
 		 * it can implement either PreparedStatement or ResultSet.  (We check for
@@ -1199,6 +1216,279 @@
 				);
 	}
 
+    /**
+     * Compute the projection and restriction to be pushed to the external
+     * table function if it is a RestrictedVTI. This method is called by the
+     * parent ProjectRestrictNode at code generation time. See DERBY-4357.
+     *
+     * @param parentProjection The column list which the parent ProjectRestrictNode expects to return.
+     * @param parentPredicates The full list of predicates to be applied by the parent ProjectRestrictNode
+     */
+    void computeProjectionAndRestriction( ResultColumnList parentProjection, PredicateList parentPredicates )
+        throws StandardException
+    {
+        // nothing to do if this is a not a restricted table function
+        if ( !isRestrictedTableFunction ) { return; }
+
+        computeRestriction( parentPredicates, computeProjection( parentProjection ) );
+    }
+    /**
+     * Fills in the array of projected column names suitable for handing to
+     * RestrictedVTI.initScan(). Returns a mapping of the exposed column names
+     * to the actual names of columns in the table function. This is useful
+     * because the predicate refers to the exposed column names.
+     *
+     * @param parentProjection The column list which the parent ProjectRestrictNode expects to return.
+     */
+    private HashMap computeProjection( ResultColumnList parentProjection ) throws StandardException
+    {
+        HashSet  projectedColumns = new HashSet();
+        HashMap  nameMap = new HashMap();
+        String[] exposedNames = parentProjection.getColumnNames();
+        int      projectedCount = exposedNames.length;
+
+        for ( int i = 0; i < projectedCount; i++ ) { projectedColumns.add( exposedNames[ i ] ); }
+
+        ResultColumnList allVTIColumns = getResultColumns();
+        int              totalColumnCount = allVTIColumns.size();
+
+        projectedColumnNames = new String[ totalColumnCount ];
+
+        for ( int i = 0; i < totalColumnCount; i++ )
+        {
+            ResultColumn column = allVTIColumns.getResultColumn( i + 1 );
+            String       exposedName = column.getName();
+
+            if ( projectedColumns.contains( exposedName ) )
+            {
+                String       baseName = column.getBaseColumnNode().getColumnName();
+                
+                projectedColumnNames[ i ] = baseName;
+
+                nameMap.put( exposedName, baseName );
+            }
+        }
+
+        return nameMap;
+    }
+    /**
+     * Fills in the restriction to be handed to a RestrictedVTI at run-time.
+     *
+     * @param parentPredicates The full list of predicates to be applied by the parent ProjectRestrictNode
+     * @param columnNameMap Mapping between the exposed column names used in the predicates and the actual column names declared for the table function at CREATE FUNCTION time.
+     */
+    private void computeRestriction( PredicateList parentPredicates, HashMap columnNameMap )
+        throws StandardException
+    {
+        if ( parentPredicates == null )  { return; }
+
+        int predicateCount = parentPredicates.size();
+
+        // walk the list, looking for qualifiers, that is, WHERE clause
+        // fragments (conjuncts)  which can be pushed into the table function
+        for ( int i = 0; i < predicateCount; i++ )
+        {
+            Predicate predicate = (Predicate) parentPredicates.elementAt( i );
+
+            if ( predicate.isQualifier() )
+            {
+                // A Predicate has a top level AND node
+                Restriction newRestriction = makeRestriction( predicate.getAndNode(), columnNameMap );
+
+                // If newRestriction is null, then we are confused. Don't push
+                // the restriction into the table function
+                if ( newRestriction == null )
+                {
+                    vtiRestriction = null;
+                    return;
+                }
+
+                // If we get here, then we still understand the restriction
+                // we're compiling.
+
+                if ( vtiRestriction == null ) { vtiRestriction = newRestriction; }
+                else { vtiRestriction = new Restriction.AND( vtiRestriction, newRestriction ); }
+            }
+        }
+    }
+    /**
+     * Turn a compile-time WHERE clause fragment into a run-time
+     * Restriction. Returns null if the clause could not be understood.
+     *
+     * @param clause The clause which should be turned into a Restriction.
+     * @param columnNameMap Mapping between the exposed column names used in the predicates and the actual column names declared for the table function at CREATE FUNCTION time.
+     */
+    private Restriction makeRestriction( ValueNode clause, HashMap columnNameMap )
+        throws StandardException
+    {
+        if ( clause instanceof AndNode )
+        {
+            AndNode andOperator = (AndNode) clause;
+
+            // strip off trailing vacuous constant if present
+            if ( andOperator.getRightOperand() instanceof BooleanConstantNode )
+            { return makeRestriction( andOperator.getLeftOperand(), columnNameMap ); }
+            
+            Restriction leftRestriction = makeRestriction( andOperator.getLeftOperand(), columnNameMap );
+            Restriction rightRestriction = makeRestriction( andOperator.getRightOperand(), columnNameMap );
+
+            if ( (leftRestriction == null) || (rightRestriction == null) ) { return null; }
+
+            return new Restriction.AND( leftRestriction, rightRestriction );
+        }
+        else if ( clause instanceof OrNode )
+        {
+            OrNode orOperator = (OrNode) clause;
+            
+            // strip off trailing vacuous constant if present
+            if ( orOperator.getRightOperand() instanceof BooleanConstantNode )
+            { return makeRestriction( orOperator.getLeftOperand(), columnNameMap ); }
+            
+            Restriction leftRestriction = makeRestriction( orOperator.getLeftOperand(), columnNameMap );
+            Restriction rightRestriction = makeRestriction( orOperator.getRightOperand(), columnNameMap );
+
+            if ( (leftRestriction == null) || (rightRestriction == null) ) { return null; }
+
+            return new Restriction.OR( leftRestriction, rightRestriction );
+        }
+        else if ( clause instanceof BinaryRelationalOperatorNode )
+        { return makeLeafRestriction( (BinaryRelationalOperatorNode) clause, columnNameMap ); }
+        else if ( clause instanceof IsNullNode )
+        { return makeIsNullRestriction( (IsNullNode) clause, columnNameMap ); }
+        else { return iAmConfused( clause ); }
+    }
+    /**
+     * Makes a Restriction out of a comparison between a constant and a column
+     * in the VTI.
+     *
+     * @param clause The clause which should be turned into a Restriction.
+     * @param columnNameMap Mapping between the exposed column names used in the predicates and the actual column names declared for the table function at CREATE FUNCTION time.
+     */
+    private Restriction makeLeafRestriction( BinaryRelationalOperatorNode clause, HashMap columnNameMap )
+        throws StandardException
+    {
+        int rawOperator = clause.getOperator();
+        ColumnReference rawColumn;
+        ValueNode rawValue;
+
+        if ( clause.getLeftOperand() instanceof ColumnReference )
+        {
+            rawColumn = (ColumnReference) clause.getLeftOperand();
+            rawValue = clause.getRightOperand();
+        }
+        else if ( clause.getRightOperand() instanceof ColumnReference )
+        {
+            rawColumn = (ColumnReference) clause.getRightOperand();
+            rawValue = clause.getLeftOperand();
+            rawOperator = flipOperator( rawOperator );
+        }
+        else { return iAmConfused( clause ); }
+
+        int comparisonOperator = mapOperator( rawOperator );
+        if ( comparisonOperator < 0 ) { return iAmConfused( clause ); }
+
+        String columnName = (String) columnNameMap.get( rawColumn.getColumnName() );
+        Object constantOperand = squeezeConstantValue( rawValue );
+        if ( (columnName == null) || (constantOperand == null) ) { return iAmConfused( clause ); }
+
+        return new Restriction.ColumnQualifier( columnName, comparisonOperator, constantOperand );
+    }
+    /**
+     * Makes an IS NULL comparison of a column
+     * in the VTI.
+     *
+     * @param clause The IS NULL (or IS NOT NULL) node
+     * @param columnNameMap Mapping between the exposed column names used in the predicates and the actual column names declared for the table function at CREATE FUNCTION time.
+     */
+    private Restriction makeIsNullRestriction( IsNullNode clause, HashMap columnNameMap )
+        throws StandardException
+    {
+        ColumnReference rawColumn = (ColumnReference) clause.getOperand();
+
+        int comparisonOperator = mapOperator( clause.getOperator() );
+        if ( comparisonOperator < 0 ) { return iAmConfused( clause ); }
+        if (
+            (comparisonOperator != Restriction.ColumnQualifier.ORDER_OP_ISNULL) &&
+            (comparisonOperator != Restriction.ColumnQualifier.ORDER_OP_ISNOTNULL)
+            ) { return iAmConfused( clause ); }
+
+        String columnName = (String) columnNameMap.get( rawColumn.getColumnName() );
+        if ( columnName == null ) { return iAmConfused( clause ); }
+
+        return new Restriction.ColumnQualifier( columnName, comparisonOperator, null );
+    }
+    /** This is a handy place to put instrumentation for tracing trees which we don't understand */
+    private Restriction iAmConfused( ValueNode clause ) throws StandardException
+    {
+        return null;
+    }
+    /** Flip the sense of a comparison */
+    private int flipOperator( int rawOperator ) throws StandardException
+    {
+        switch( rawOperator )
+        {
+        case RelationalOperator.EQUALS_RELOP:         return RelationalOperator.EQUALS_RELOP;
+        case RelationalOperator.GREATER_EQUALS_RELOP: return RelationalOperator.LESS_EQUALS_RELOP;
+        case RelationalOperator.GREATER_THAN_RELOP:   return RelationalOperator.LESS_THAN_RELOP;
+        case RelationalOperator.LESS_EQUALS_RELOP:    return RelationalOperator.GREATER_EQUALS_RELOP;
+        case RelationalOperator.LESS_THAN_RELOP:      return RelationalOperator.GREATER_THAN_RELOP;
+        case RelationalOperator.NOT_EQUALS_RELOP:     return RelationalOperator.NOT_EQUALS_RELOP;
+
+        case RelationalOperator.IS_NOT_NULL_RELOP:
+        case RelationalOperator.IS_NULL_RELOP:
+        default:
+            if ( SanityManager.DEBUG )
+            {
+                SanityManager.THROWASSERT( "Unrecognized relational operator: " + rawOperator );
+            }
+        }
+
+        return -1;
+    }
+    /** Map internal operator constants to user-visible ones */
+    private int mapOperator( int rawOperator ) throws StandardException
+    {
+        switch( rawOperator )
+        {
+        case RelationalOperator.EQUALS_RELOP:         return Restriction.ColumnQualifier.ORDER_OP_EQUALS;
+        case RelationalOperator.GREATER_EQUALS_RELOP: return Restriction.ColumnQualifier.ORDER_OP_GREATEROREQUALS;
+        case RelationalOperator.GREATER_THAN_RELOP:   return Restriction.ColumnQualifier.ORDER_OP_GREATERTHAN;
+        case RelationalOperator.LESS_EQUALS_RELOP:    return Restriction.ColumnQualifier.ORDER_OP_LESSOREQUALS;
+        case RelationalOperator.LESS_THAN_RELOP:      return Restriction.ColumnQualifier.ORDER_OP_LESSTHAN;
+        case RelationalOperator.IS_NULL_RELOP:        return Restriction.ColumnQualifier.ORDER_OP_ISNULL;
+        case RelationalOperator.IS_NOT_NULL_RELOP:    return Restriction.ColumnQualifier.ORDER_OP_ISNOTNULL;
+
+        case RelationalOperator.NOT_EQUALS_RELOP:
+        default:
+            if ( SanityManager.DEBUG )
+            {
+                SanityManager.THROWASSERT( "Unrecognized relational operator: " + rawOperator );
+            }
+        }
+
+        return -1;
+    }
+    /**
+     * Get the constant or parameter reference out of a comparand. Return null
+     * if we are confused. A parameter reference is wrapped in an integer array
+     * to distinguish it from a constant integer.
+     */
+    private Object squeezeConstantValue( ValueNode valueNode ) throws StandardException
+    {
+        if ( valueNode instanceof ParameterNode )
+        {
+            return new int[] { ((ParameterNode) valueNode).getParameterNumber() };
+        }
+        else if ( valueNode instanceof ConstantNode )
+        {
+            return ((ConstantNode) valueNode).getValue().getObject();
+        }
+        else
+        {
+            return iAmConfused( valueNode );
+        }
+    }
+    
 	/**
 	 * Generation on a FromVTI creates a wrapper around
 	 * the user's java.sql.ResultSet
@@ -1325,14 +1615,26 @@
 
 		// Push the return type
         int rtNum = -1;
-        if ( isDerbyStyleTableFunction )
+        if ( isDerbyStyleTableFunction  )
         {
             rtNum = acb.addItem(methodCall.getRoutineInfo().getReturnType());
         }
         mb.push(rtNum);
 
-		return 16;
+        // push the projection and restriction for RestrictedVTIs
+        mb.push( storeObjectInPS( acb, projectedColumnNames ) );
+        mb.push( storeObjectInPS( acb, vtiRestriction ) );        
+
+		return 18;
 	}
+    /** Store an object in the prepared statement.  Returns -1 if the object is
+     * null. Otherwise returns the object's retrieval handle.
+     */
+    private int storeObjectInPS( ActivationClassBuilder acb, Object obj ) throws StandardException
+    {
+        if ( obj == null ) { return -1; }
+        else { return acb.addItem( obj ); }
+    }
 
 	private void generateConstructor(ActivationClassBuilder acb,
 										   MethodBuilder mb, boolean reuseablePs)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MethodCallNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MethodCallNode.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MethodCallNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/MethodCallNode.java Thu Oct 29 19:55:56 2009
@@ -58,6 +58,7 @@
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Member;
 
+import java.sql.ResultSet;
 import java.util.Enumeration;
 import java.util.StringTokenizer;
 import java.util.Vector;
@@ -141,6 +142,14 @@
     }
 
     /**
+     * @return get the Java method or constructor determined during the bind() phase.
+     */
+    public Member getResolvedMethod()
+    {
+        return method;
+    }
+
+    /**
      * Get the details on the invoked routines.
      */
     public RoutineAliasInfo getRoutineInfo()
@@ -767,7 +776,7 @@
 				    ( routineInfo.getParameterStyle() == RoutineAliasInfo.PS_DERBY_JDBC_RESULT_SET )
 				)
 				{
-				    requiredType = "java.sql.ResultSet";
+				    requiredType = ResultSet.class.getName();
 				}
 				else
 				{
@@ -792,7 +801,20 @@
 				}
 			}
 
-			if (!requiredType.equals(typeName))
+            boolean foundCorrectType;
+            if ( ResultSet.class.getName().equals( requiredType )  )
+            {
+                // allow subtypes of ResultSet too
+                try {
+                    Class actualType = classInspector.getClass( typeName );
+
+                    foundCorrectType = ResultSet.class.isAssignableFrom( actualType );
+                }
+                catch (ClassNotFoundException cnfe) { foundCorrectType = false; }
+            }
+            else{ foundCorrectType = requiredType.equals(typeName); }
+
+			if (!foundCorrectType)
 			{
 				throwNoMethodFound(requiredType + " " + javaClassName, parmTypeNames, primParmTypeNames);
 			}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ProjectRestrictNode.java Thu Oct 29 19:55:56 2009
@@ -1353,6 +1353,16 @@
 									 MethodBuilder mb, boolean genChildResultSet)
 									throws StandardException
 	{
+        //
+        // If we are projecting and restricting the stream from a table
+        // function, then give the table function all of the information that
+        // it needs in order to push the projection and qualifiers into
+        // the table function. See DERBY-4357.
+        //
+        if ( childResult instanceof FromVTI )
+        {
+            ((FromVTI) childResult).computeProjectionAndRestriction( resultColumns, restrictionList );
+        }
 
 		/* If this ProjectRestrict doesn't do anything, bypass its generation.
 		 * (Remove any true and true predicates first, as they could be left

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java Thu Oct 29 19:55:56 2009
@@ -449,7 +449,9 @@
 									 double optimizerEstimatedRowCount,
 									 double optimizerEstimatedCost,
                                      boolean isDerbyStyleTableFunction,
-                                     int returnTypeNumber
+                                     int returnTypeNumber,
+                                     int vtiProjectionNumber,
+                                     int vtiRestrictionNumber
                                           )
 		throws StandardException
 	{
@@ -465,7 +467,9 @@
 							    optimizerEstimatedRowCount,
 								optimizerEstimatedCost,
 								isDerbyStyleTableFunction,
-                                returnTypeNumber
+                                returnTypeNumber,
+                                vtiProjectionNumber,
+                                vtiRestrictionNumber
                                 );
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/VTIResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/VTIResultSet.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/VTIResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/VTIResultSet.java Thu Oct 29 19:55:56 2009
@@ -38,6 +38,7 @@
 
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.ResultDescription;
+import org.apache.derby.iapi.sql.ParameterValueSet; 
 import org.apache.derby.iapi.types.TypeId;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.DataValueDescriptor;
@@ -59,6 +60,8 @@
 import org.apache.derby.vti.DeferModification;
 import org.apache.derby.vti.IFastPath;
 import org.apache.derby.vti.VTIEnvironment;
+import org.apache.derby.vti.RestrictedVTI;
+import org.apache.derby.vti.Restriction;
 
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -102,6 +105,9 @@
 
     private DataTypeDescriptor[]    returnColumnTypes;
 
+    private String[] vtiProjection;
+    private Restriction vtiRestriction;
+
 	/**
 		Specified isolation level of SELECT (scan). If not set or
 		not application, it will be set to ExecutionContext.UNSPECIFIED_ISOLATION_LEVEL
@@ -123,7 +129,9 @@
 			     double optimizerEstimatedRowCount,
 				 double optimizerEstimatedCost,
 				 boolean isDerbyStyleTableFunction,
-                 int returnTypeNumber
+                 int returnTypeNumber,
+                 int vtiProjectionNumber,
+                 int vtiRestrictionNumber
                  ) 
 		throws StandardException
 	{
@@ -143,6 +151,14 @@
             (TypeDescriptor)
             activation.getPreparedStatement().getSavedObject(returnTypeNumber);
 
+        this.vtiProjection = vtiProjectionNumber == -1 ? null :
+            (String[])
+            activation.getPreparedStatement().getSavedObject(vtiProjectionNumber);
+
+        this.vtiRestriction = vtiRestrictionNumber == -1 ? null :
+            (Restriction)
+            activation.getPreparedStatement().getSavedObject(vtiRestrictionNumber);
+
 		if (erdNumber != -1)
 		{
 			this.referencedColumns = (FormatableBitSet)(activation.getPreparedStatement().
@@ -222,6 +238,13 @@
 			else
 			{
 				userVTI = (ResultSet) constructor.invoke(activation);
+
+                if ( userVTI instanceof RestrictedVTI )
+                {
+                    RestrictedVTI restrictedVTI = (RestrictedVTI) userVTI;
+
+                    restrictedVTI.initScan( vtiProjection, cloneRestriction( activation ) );
+                }
 			}
 
 			// Set up the nullablity of the runtime columns, may be delayed
@@ -236,6 +259,67 @@
 		openTime += getElapsedMillis(beginTime);
 	}
 
+    /**
+     * Clone the restriction for a Restricted VTI, filling in parameter values
+     * as necessary.
+     */
+    private Restriction cloneRestriction( Activation activation ) throws StandardException
+    {
+        if ( vtiRestriction == null ) { return null; }
+        else { return cloneRestriction( activation, vtiRestriction ); }
+    }
+    private Restriction cloneRestriction( Activation activation, Restriction original )
+        throws StandardException
+    {
+        if ( original instanceof Restriction.AND)
+        {
+            Restriction.AND and = (Restriction.AND) original;
+            
+            return new Restriction.AND
+                (
+                 cloneRestriction( activation, and.getLeftChild() ),
+                 cloneRestriction( activation, and.getRightChild() )
+                 );
+        }
+        else if ( original instanceof Restriction.OR)
+        {
+            Restriction.OR or = (Restriction.OR) original;
+            
+            return new Restriction.OR
+                (
+                 cloneRestriction( activation, or.getLeftChild() ),
+                 cloneRestriction( activation, or.getRightChild() )
+                 );
+        }
+        else if ( original instanceof Restriction.ColumnQualifier)
+        {
+            Restriction.ColumnQualifier cq = (Restriction.ColumnQualifier) original;
+            Object originalConstant = cq.getConstantOperand();
+            Object newConstant;
+
+            if ( originalConstant ==  null ) { newConstant = null; }
+            else if ( originalConstant instanceof int[] )
+            {
+                int parameterNumber = ((int[]) originalConstant)[ 0 ];
+                ParameterValueSet pvs = activation.getParameterValueSet();
+
+                newConstant = pvs.getParameter( parameterNumber ).getObject();
+            }
+            else { newConstant = originalConstant; }
+           
+            return new Restriction.ColumnQualifier
+                (
+                 cq.getColumnName(),
+                 cq.getComparisonOperator(),
+                 newConstant
+                 );
+        }
+        else
+        {
+            throw StandardException.newException( SQLState.NOT_IMPLEMENTED, original.getClass().getName() );
+        }
+    }
+
 	private boolean[] setNullableColumnList() throws SQLException, StandardException {
 
 		if (runtimeNullableColumn != null)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/vti/Restriction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/vti/Restriction.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/vti/Restriction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/vti/Restriction.java Thu Oct 29 19:55:56 2009
@@ -20,6 +20,7 @@
  */
 package org.apache.derby.vti;
 
+import java.io.Serializable;
 import java.sql.SQLException;
 
 /**
@@ -32,11 +33,25 @@
  * the Table Function.
  * </p>
  */
-public abstract class Restriction
+public abstract class Restriction implements Serializable
 {
+    /**
+     * Turn this Restriction into a string suitable for use in a WHERE clause.
+     */
+    public abstract String toSQL();
+
+    /** Utility method to double quote a string */
+    protected String doubleQuote( String raw ) { return "\"" + raw + "\""; }
+
+    /** Utility method to parenthesize an expression */
+    protected String parenthesize( String raw ) { return "( " + raw + " )"; }
+    
     /** An AND of two Restrictions */
     public static class AND extends Restriction
     {
+        /** Derby serializes these objects in PreparedStatements */
+        public static final long serialVersionUID = -8205388794606605844L;
+        
         private Restriction _leftChild;
         private Restriction _rightChild;
 
@@ -52,11 +67,19 @@
 
         /** Get the right Restriction */
         public Restriction getRightChild() { return _rightChild; }
+        
+        public String toSQL()
+        {
+            return parenthesize( _leftChild.toSQL() ) + " AND " + parenthesize( _rightChild.toSQL() );
+        }
     }
 
     /** An OR of two Restrictions */
     public static class OR extends Restriction
     {
+        /** Derby serializes these objects in PreparedStatements */
+        public static final long serialVersionUID = -8205388794606605844L;
+        
         private Restriction _leftChild;
         private Restriction _rightChild;
 
@@ -72,6 +95,11 @@
 
         /** Get the right Restriction */
         public Restriction getRightChild() { return _rightChild; }
+
+        public String toSQL()
+        {
+            return parenthesize( _leftChild.toSQL() ) + " OR " + parenthesize( _rightChild.toSQL() );
+        }
     }
 
     /**
@@ -89,7 +117,7 @@
        * </p>
        *
        * <blockquote><pre>
-       *  <     =     <=     >      >=
+       *  <     =     <=     >      >=    IS NULL    IS NOT NULL
        * </pre></blockquote>
        */
     public static class ColumnQualifier extends Restriction
@@ -99,21 +127,33 @@
         // CONSTANTS
         //
         ////////////////////////////////////////////////////////////////////////////////////////
+
+        /** Derby serializes these objects in PreparedStatements */
+        public static final long serialVersionUID = -8205388794606605844L;
         
         /**	 Ordering operation constant representing '<' **/
-        public static final int ORDER_OP_LESSTHAN = 1;
+        public static final int ORDER_OP_LESSTHAN = 0;
 
         /**	 Ordering operation constant representing '=' **/
-        public static final int ORDER_OP_EQUALS = 2;
+        public static final int ORDER_OP_EQUALS = 1;
 
         /**	 Ordering operation constant representing '<=' **/
-        public static final int ORDER_OP_LESSOREQUALS = 3;
+        public static final int ORDER_OP_LESSOREQUALS = 2;
 
         /**	 Ordering operation constant representing '>' **/
-        public static final int ORDER_OP_GREATERTHAN = 4;
+        public static final int ORDER_OP_GREATERTHAN = 3;
 
         /**	 Ordering operation constant representing '>=' **/
-        public static final int ORDER_OP_GREATEROREQUALS = 5;
+        public static final int ORDER_OP_GREATEROREQUALS = 4;
+
+        /**	 Ordering operation constant representing 'IS NULL' **/
+        public static final int ORDER_OP_ISNULL = 5;
+
+        /**	 Ordering operation constant representing 'IS NOT NULL' **/
+        public static final int ORDER_OP_ISNOTNULL = 6;
+
+        // Visible forms of the constants above
+        private String[] OPERATOR_SYMBOLS = new String[] {  "<", "=", "<=", ">", ">=", "IS NULL", "IS NOT NULL" };
 
         ////////////////////////////////////////////////////////////////////////////////////////
         //
@@ -127,9 +167,6 @@
         /** comparison operator, one of the ORDER_OP constants */
         private int     _comparisonOperator;
 
-        /** null handling */
-        private boolean _nullEqualsNull;
-
         /** value to compare the column to */
         private Object _constantOperand;
         
@@ -146,20 +183,17 @@
          *
          * @param columnName Name of column as declared in the CREATE FUNCTION statement.
          * @param comparisonOperator One of the ORDER_OP constants.
-         * @param nullEqualsNull True if NULLS should be treated like ordinary values which sort before all other values. Used to encode IS NULL comparisons.
          * @param constantOperand Constant value to which the column should be compared.
          */
         public ColumnQualifier
             (
              String columnName,
              int comparisonOperator,
-             boolean nullEqualsNull,
              Object constantOperand
              )
         {
             _columnName = columnName;
             _comparisonOperator = comparisonOperator;
-            _nullEqualsNull = nullEqualsNull;
             _constantOperand = constantOperand;
         }
         
@@ -186,27 +220,28 @@
 
         /**
          * <p>
-         * Specifies how nulls behave in comparisons. If true, then nulls are
-         * treated as values which sort before all other values; and the = comparison
-         * between two nulls evaluates to TRUE. If this method returns false, then
-         * any comparison involving a null evaluates to UNKNOWN. This is useful
-         * for encoding IS NULL comparisons.
-         * </p>
-         */
-        public boolean nullEqualsNull() { return _nullEqualsNull; }
-
-        /**
-         * <p>
          * Get the constant value to which the column should be compared. The
          * constant value must be an Object of the Java type which corresponds to
          * the SQL type of the column. The column's SQL type was declared in the CREATE FUNCTION statement.
          * The mapping of SQL types to Java types is defined in table 4 of chapter 14
          * of the original JDBC 1 specification (dated 1996). Bascially, these are the Java
          * wrapper values you would expect. For instance, SQL INT maps to java.lang.Integer, SQL CHAR
-         * maps to java.lang.String, etc..
+         * maps to java.lang.String, etc.. This object will be null if the
+         * comparison operator is ORDER_OP_ISNULL or ORDER_OP_ISNOTNULL.
          * </p>
          */
         public Object getConstantOperand() { return _constantOperand; }
+        
+        public String toSQL()
+        {
+            StringBuffer buffer = new StringBuffer();
+
+            buffer.append( doubleQuote( _columnName ) );
+            buffer.append( " " + OPERATOR_SYMBOLS[ _comparisonOperator ] + " " );
+            if ( _constantOperand != null ) { buffer.append( _constantOperand ); }
+
+            return buffer.toString();
+        }
     }
     
 }

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java?rev=831072&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java Thu Oct 29 19:55:56 2009
@@ -0,0 +1,126 @@
+/*
+
+Derby - Class org.apache.derbyTesting.functionTests.tests.lang.IntegerArrayVTI
+
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You 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.derbyTesting.functionTests.tests.lang;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+
+import org.apache.derby.vti.RestrictedVTI;
+import org.apache.derby.vti.Restriction;
+
+/**
+ * A VTI which returns a row of ints.
+ */
+public class IntegerArrayVTI extends StringArrayVTI implements RestrictedVTI
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    private static String[] _lastProjection;
+    private static Restriction _lastRestriction;
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTORS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    public IntegerArrayVTI( String[] columnNames, int[][] rows )
+    {
+        super( columnNames, stringify( rows ) );
+    }
+    public IntegerArrayVTI( String[] columnNames, Integer[][] rows )
+    {
+        super( columnNames, stringify( rows ) );
+    }
+    private static String[][] stringify( int[][] rows )
+    {
+        int outerCount = rows.length;
+
+        String[][] retval = new String[ outerCount ][];
+
+        for ( int i = 0; i < outerCount; i++ )
+        {
+            int[] rawRow = rows[ i ];
+            int innerCount = rawRow.length;
+            String[] row = new String[ innerCount ];
+            
+            retval[ i ] = row;
+
+            for ( int j = 0; j < innerCount; j++ )
+            {
+                row[ j ] = Integer.toString( rawRow[ j ] );
+            }
+        }
+
+        return retval;
+    }
+    private static String[][] stringify( Integer[][] rows )
+    {
+        int outerCount = rows.length;
+
+        String[][] retval = new String[ outerCount ][];
+
+        for ( int i = 0; i < outerCount; i++ )
+        {
+            Integer[] rawRow = rows[ i ];
+            int innerCount = rawRow.length;
+            String[] row = new String[ innerCount ];
+            
+            retval[ i ] = row;
+
+            for ( int j = 0; j < innerCount; j++ )
+            {
+                Integer raw = rawRow[ j ];
+                String value = raw == null ? null : raw.toString();
+                row[ j ] = value;
+            }
+        }
+
+        return retval;
+    }
+    
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // RestrictedVTI BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    public void initScan( String[] columnNames, Restriction restriction ) throws SQLException
+    {
+        _lastProjection = columnNames;
+        _lastRestriction = restriction;
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // OTHER PUBLIC BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    public static String getLastProjection() { return Arrays.asList( _lastProjection ).toString(); }
+    public static String getLastRestriction() { return ( ( _lastRestriction == null ) ? null : _lastRestriction.toSQL() ); }
+    
+}

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IntegerArrayVTI.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java?rev=831072&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java Thu Oct 29 19:55:56 2009
@@ -0,0 +1,500 @@
+/*
+
+   Derby - Class org.apache.derbyTesting.functionTests.tests.lang.RestrictedVTITest
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you 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.derbyTesting.functionTests.tests.lang;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.TestConfiguration;
+import org.apache.derbyTesting.junit.CleanDatabaseTestSetup;
+
+/**
+ * <p>
+ * Test RestrictedVTIs. See DERBY-4357.
+ * </p>
+ */
+public class RestrictedVTITest  extends GeneratedColumnsHelper
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTANTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+
+    /**
+     * Create a new instance.
+     */
+
+    public RestrictedVTITest(String name)
+    {
+        super(name);
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // JUnit BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+
+    /**
+     * Construct top level suite in this JUnit test
+     */
+    public static Test suite()
+    {
+        TestSuite suite = (TestSuite) TestConfiguration.embeddedSuite(RestrictedVTITest.class);
+        Test        result = new CleanDatabaseTestSetup( suite );
+
+        return result;
+    }
+
+    protected void    setUp()
+        throws Exception
+    {
+        super.setUp();
+
+        Connection conn = getConnection();
+
+        if ( !routineExists( conn, "GETLASTPROJECTION" ) )
+        {
+            goodStatement
+                (
+                 conn,
+                 "create function getLastProjection\n" +
+                 "()\n" +
+                 "returns varchar( 32672 )\n" +
+                 "language java parameter style java no sql\n" +
+                 "external name 'org.apache.derbyTesting.functionTests.tests.lang.IntegerArrayVTI.getLastProjection'\n"
+                 );
+        }
+        if ( !routineExists( conn, "GETLASTRESTRICTION" ) )
+        {
+            goodStatement
+                (
+                 conn,
+                 "create function getLastRestriction\n" +
+                 "()\n" +
+                 "returns varchar( 32672 )\n" +
+                 "language java parameter style java no sql\n" +
+                 "external name 'org.apache.derbyTesting.functionTests.tests.lang.IntegerArrayVTI.getLastRestriction'\n"
+                 );
+        }
+        if ( !routineExists( conn, "INTEGERLIST" ) )
+        {
+            goodStatement
+                (
+                 conn,
+                 "create function integerList()\n" +
+                 "returns table( s_r int, s_nr int, ns_r int, ns_nr int )\n" +
+                 "language java\n" +
+                 "parameter style derby_jdbc_result_set\n" +
+                 "no sql\n" +
+                 "external name 'org.apache.derbyTesting.functionTests.tests.lang.RestrictedVTITest.integerList'\n"
+                 );
+        }
+        if ( !routineExists( conn, "NULLABLEINTEGERLIST" ) )
+        {
+            goodStatement
+                (
+                 conn,
+                 "create function nullableIntegerList()\n" +
+                 "returns table( s_r int, s_nr int, ns_r int, ns_nr int )\n" +
+                 "language java\n" +
+                 "parameter style derby_jdbc_result_set\n" +
+                 "no sql\n" +
+                 "external name 'org.apache.derbyTesting.functionTests.tests.lang.RestrictedVTITest.nullableIntegerList'\n"
+                 );
+        }
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // TESTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Heartbeat test to verify that projections and restrictions are being
+     * pushed into RestrictedVTIs.
+     * </p>
+     */
+    public void test_01_heartbeat() throws Exception
+    {
+        Connection conn = getConnection();
+
+        // test basic pushing of projection and restriction
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( integerList() ) s where s_r > 1 and ns_r < 3000\n",
+             new String[][]
+             {
+                 { "100" ,         "200"  },
+             },
+             "[S_R, S_NR, NS_R, null]",
+             "( \"NS_R\" < 3000 ) AND ( \"S_R\" > 1 )"
+             );
+
+        // similar test except with a ? parameter
+        PreparedStatement ps = chattyPrepare
+            (
+             conn,
+             "select s_r from table( integerList() ) s where s_r > 1 and ns_r < ?"
+             );
+        ps.setInt( 1, 30000 );
+        ResultSet rs = ps.executeQuery();
+        assertResults
+            (
+             rs,
+             new String[][]
+             {
+                 { "100" },
+                 { "1000" },
+             },
+             false
+             );
+        assertResults
+            (
+             conn,
+             "values ( getLastProjection() )\n",
+             new String[][] { { "[S_R, null, NS_R, null]" } },
+             false
+             );
+        assertResults
+            (
+             conn,
+             "values ( getLastRestriction() )\n",
+             new String[][] { { "( \"NS_R\" < 30000 ) AND ( \"S_R\" > 1 )" } },
+             false
+             );
+        
+        // similar to the first test except NOT the qualification
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( integerList() ) s where not( s_r > 1 and ns_r < 3000 )\n",
+             new String[][]
+             {
+                 { "1" ,         "2"  },
+                 { "1000" ,         "2000"  },
+                 { "10000" ,         "20000"  },
+             },
+             "[S_R, S_NR, NS_R, null]",
+             "( \"S_R\" <= 1 ) OR ( \"NS_R\" >= 3000 )"
+             );
+    }
+
+    /**
+     * <p>
+     * Verify that aliases are correctly mapped to table column names. Also
+     * verify that SELECT list expressions cause columns to be included in the
+     * column list. Also verify that predicates which aren't qualifiers are not included in the restriction.
+     * </p>
+     */
+    public void test_02_aliasing() throws Exception
+    {
+        Connection conn = getConnection();
+
+        assertPR
+            (
+             conn,
+             "select 2*w, x from table( integerList() ) as s( w, x, y, z ) where w > 1 and mod( y, 3 ) = 0\n",
+             new String[][]
+             {
+                 { "200" ,         "200"  },
+                 { "2000" ,         "2000"  },
+                 { "20000" ,         "20000"  },
+             },
+             "[S_R, S_NR, NS_R, null]",
+             "\"S_R\" > 1"
+             );
+    }
+    
+    /**
+     * <p>
+     * Verify that all relational operators are handled.
+     * </p>
+     */
+    public void test_03_allRelationalOperators() throws Exception
+    {
+        Connection conn = getConnection();
+
+        // IS NULL
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r is null\n",
+             new String[][]
+             {
+                 { null ,         "2"  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" IS NULL "
+             );
+
+        // IS NOT NULL
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r is not null\n",
+             new String[][]
+             {
+                 { "100",         null  },
+                 { "1000",         "2000"  },
+                 { "10000",         "20000"  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" IS NOT NULL "
+             );
+
+        // <
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r < 1000\n",
+             new String[][]
+             {
+                 { "100",         null  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" < 1000"
+             );
+
+        // <=
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r <= 100\n",
+             new String[][]
+             {
+                 { "100",         null  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" <= 100"
+             );
+
+        // =
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r = 100\n",
+             new String[][]
+             {
+                 { "100",         null  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" = 100"
+             );
+
+        // >
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r > 100\n",
+             new String[][]
+             {
+                 { "1000",         "2000"  },
+                 { "10000",         "20000"  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" > 100"
+             );
+
+        // >=
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r >= 100\n",
+             new String[][]
+             {
+                 { "100",         null  },
+                 { "1000",         "2000"  },
+                 { "10000",         "20000"  },
+             },
+             "[S_R, S_NR, null, null]",
+             "\"S_R\" >= 100"
+             );
+    }
+    
+    /**
+     * <p>
+     * Miscellaneous conditions.
+     * </p>
+     */
+    public void test_04_misc() throws Exception
+    {
+        Connection conn = getConnection();
+
+        // Arithmetic expressions are not qualifiers.
+        assertPR
+            (
+             conn,
+             "select s_r, s_nr from table( nullableIntegerList() ) s where s_r < s_nr + ns_r\n",
+             new String[][]
+             {
+                 { "10000" ,         "20000"  },
+             },
+             "[S_R, S_NR, NS_R, null]",
+             null
+             );
+
+        // Casting a literal to an int is computed by the compiler and so is a qualifier
+        assertPR
+            (
+             conn,
+             "select s_r from table( nullableIntegerList() ) s where ns_r = cast( '300' as int)\n",
+             new String[][]
+             {
+                 { "100"  },
+             },
+             "[S_R, null, NS_R, null]",
+             "\"NS_R\" = 300"
+             );
+
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // SQL ROUTINES
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    public static IntegerArrayVTI integerList()
+    {
+        // S => in SELECT list
+        // NS => NOT in SELECT LIST
+        // R => in restriction
+        // NR => NOT in restriction
+        return new IntegerArrayVTI
+            (
+             new String[] { "S_R", "S_NR", "NS_R", "NS_NR" },
+             new int[][]
+             {
+                 new int[] { 1, 2, 3, 4 },
+                 new int[] { 100, 200, 300, 400 },
+                 new int[] { 1000, 2000, 3000, 4000 },
+                 new int[] { 10000, 20000, 30000, 40000 },
+             }
+             );
+    }
+    
+    public static IntegerArrayVTI nullableIntegerList()
+    {
+        // S => in SELECT list
+        // NS => NOT in SELECT LIST
+        // R => in restriction
+        // NR => NOT in restriction
+        return new IntegerArrayVTI
+            (
+             new String[] { "S_R", "S_NR", "NS_R", "NS_NR" },
+             new Integer[][]
+             {
+                 new Integer[] { null, i(2), i(3), i(4) },
+                 new Integer[] { i(100), null, i(300), i(400) },
+                 new Integer[] { i(1000), i(2000), null, i(4000) },
+                 new Integer[] { i(10000), i(20000), i(30000), null },
+             }
+             );
+    }
+    private static Integer i( int intValue ) { return new Integer( intValue ); }
+    
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // MINIONS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /** Return true if the SQL routine exists */
+    private boolean routineExists( Connection conn, String functionName ) throws Exception
+    {
+        PreparedStatement ps = chattyPrepare( conn, "select count (*) from sys.sysaliases where alias = ?" );
+        ps.setString( 1, functionName );
+
+        ResultSet rs = ps.executeQuery();
+        rs.next();
+
+        boolean retval = rs.getInt( 1 ) > 0 ? true : false;
+
+        rs.close();
+        ps.close();
+
+        return retval;
+    }
+
+    /**
+     * <p>
+     * Run a query against a RestrictedVTI and verify that the expected
+     * projection and restriction are pushed into the VTI.
+     * </p>
+     */
+    private void assertPR
+        (
+         Connection conn,
+         String query,
+         String[][] expectedResults,
+         String expectedProjection,
+         String expectedRestriction
+         ) throws Exception
+    {
+        assertResults
+            (
+             conn,
+             query,
+             expectedResults,
+             false
+             );
+        assertResults
+            (
+             conn,
+             "values ( getLastProjection() )\n",
+             new String[][] { { expectedProjection } },
+             false
+             );
+        assertResults
+            (
+             conn,
+             "values ( getLastRestriction() )\n",
+             new String[][] { { expectedRestriction } },
+             false
+             );
+    }
+
+}

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RestrictedVTITest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/_Suite.java Thu Oct 29 19:55:56 2009
@@ -198,6 +198,7 @@
         suite.addTest(PrecedenceTest.suite());
         suite.addTest(GeneratedColumnsTest.suite());
         suite.addTest(GeneratedColumnsPermsTest.suite());
+        suite.addTest(RestrictedVTITest.suite());
         suite.addTest(AlterColumnTest.suite());
         suite.addTest(UserLobTest.suite());
         suite.addTest(OffsetFetchNextTest.suite());

Modified: db/derby/code/trunk/tools/javadoc/publishedapi.ant
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/tools/javadoc/publishedapi.ant?rev=831072&r1=831071&r2=831072&view=diff
==============================================================================
--- db/derby/code/trunk/tools/javadoc/publishedapi.ant (original)
+++ db/derby/code/trunk/tools/javadoc/publishedapi.ant Thu Oct 29 19:55:56 2009
@@ -50,6 +50,8 @@
 
 # package: org.apache.derby.vti
 
+org/apache/derby/vti/RestrictedVTI.java
+org/apache/derby/vti/Restriction.java
 org/apache/derby/vti/VTICosting.java
 org/apache/derby/vti/VTIEnvironment.java