You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by ab...@apache.org on 2007/03/07 23:02:54 UTC

svn commit: r515795 - in /db/derby/code/trunk/java/engine/org/apache/derby: iapi/sql/compile/ iapi/sql/execute/ impl/sql/compile/ impl/sql/execute/

Author: abrown
Date: Wed Mar  7 14:02:53 2007
New Revision: 515795

URL: http://svn.apache.org/viewvc?view=rev&rev=515795
Log:
DERBY-47 (partial): Code generation patch that does the following:

  1 - Moves the code for generating a list of IN values into a new method,
    InListOperatorNode.generateListAsArray()". The new method is then called
    from two places:

     A. InListOperatorNode.generateExpression(): the "normal" code-path for
        generating IN-list bytecode (prior to DERBY-47 changes).

     B. PredicateList.generateInListValues(): new method for generating the IN-list
        values that will serve as the execution-time index "probe" values. This
        method also generates a boolean to indicate whether or not the values
        are already sorted (i.e. if we sorted them at compile time, which means
        they all must have been constants).

  2 - Adds code to ParameterNode that allows generation of a "place-holder"
    value (instead of the ParameterNode itself) for probe-predicates. This is
    required because a probe predicate has the form "column = ?" where the right
    operand is an internally generated parameter node that does not actually
    correspond to a user parameter. Since that parameter node is "fake" we can't
    really generate it; instead we need to be able to generate a legitimate
    ValueNode--either a constant node or a "real" parameter node--to serve as
    the place-holder.

  3 - Updates the generateExpression() method of BinaryOperatorNode to account
    for situations where the optimizer chooses a plan for which a probe pred is
    *not* a useful start/stop key and thus is not being used for execution-time
    index probing. In this case we simply "revert" the probe predicate back to
    the InListOperatorNode from which it was created. Or put another way, we
    "give up" on index multi-probing and simply generate the original IN-list
    as a regular restriction.  This patch also removes the now unnecessary
    "revertToSourceInList()" calls from PredicateList.java.

  4 - Adds logic to NestedLoopJoinStrategy to generate a new type of result set,
    MultiProbeTableScanResultSet, for probing an index at execution time. The
    new result set does not yet exist (incremental development) but the code to
    generate such a result set is added as part of this patch.

  5 - Adds a new method, "getMultiProbeTableScanResultSet()", to the
    ResultSetFactory interface. Also adds a corresponding stub method to
    GenericResultSetFactory. The latter is just a dummy method and will be
    filled in with the appropriate code as part of a subsequent patch.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java Wed Mar  7 14:02:53 2007
@@ -179,10 +179,11 @@
 	/**
 	 * Get the name of the result set method for base table scans
 	 *
-	 * @param bulkFetch		True means bulk fetch is being done on the inner
-	 *						table
+	 * @param bulkFetch True means bulk fetch is being done on the inner table
+	 * @param multiprobe True means we are probing the inner table for rows
+	 *  matching a specified list of values.
 	 */
-	String resultSetMethodName(boolean bulkFetch);
+	String resultSetMethodName(boolean bulkFetch, boolean multiprobe);
 
 	/**
 	 * Get the name of the join result set method for the join
@@ -215,6 +216,8 @@
 	 *						as always using table locking
 	 * @param isolationLevel		Isolation level specified (or not) for scans
 	 * @param maxMemoryPerTable	Max memory per table
+	 * @param genInListVals Whether or not we are going to generate IN-list
+	 *  values with which to probe the inner table.
 	 *
 	 * @return	Count of the expressions pushed to use as the parameters to the
 	 *			result set for the inner table
@@ -233,8 +236,9 @@
 							int indexColItem,
 							int lockMode,
 							boolean tableLocked,
-                            int isolationLevel,
-                            int maxMemoryPerTable
+							int isolationLevel,
+							int maxMemoryPerTable,
+							boolean genInListVals
 							)
 					throws StandardException;
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- 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 Wed Mar  7 14:02:53 2007
@@ -30,6 +30,7 @@
 import org.apache.derby.iapi.services.loader.GeneratedMethod;
 
 import org.apache.derby.iapi.store.access.Qualifier;
+import org.apache.derby.iapi.types.DataValueDescriptor;
 
 /**
  * ResultSetFactory provides a wrapper around all of
@@ -985,6 +986,51 @@
 								double optimizerEstimatedRowCount,
 								double optimizerEstimatedCost)
 			throws StandardException;
+
+	/**
+		A multi-probe result set, used for probing an index with one or more
+		target values (probeValues) and returning the matching rows.  This
+		type of result set is useful for IN lists as it allows us to avoid
+		scannning an entire, potentially very large, index for a mere handful
+		of rows (DERBY-47).
+
+		All arguments are the same as for TableScanResultSet, plus the
+		following:
+
+		@param probeVals List of values with which to probe the underlying
+			table. Should not be null.
+		@param probeValsAreSorted Whether or not the values in probeVals are
+			sorted.
+	 */
+	NoPutResultSet getMultiProbeTableScanResultSet(
+			                    Activation activation,
+								long conglomId,
+								int scociItem,
+								GeneratedMethod resultRowAllocator,
+								int resultSetNumber,
+								GeneratedMethod startKeyGetter,
+								int startSearchOperator,
+								GeneratedMethod stopKeyGetter,
+								int stopSearchOperator,
+								boolean sameStartStopPosition,
+								Qualifier[][] qualifiers,
+								DataValueDescriptor [] probeVals,
+								boolean probeValsAreSorted,
+								String tableName,
+								String userSuppliedOptimizerOverrides,
+								String indexName,
+								boolean isConstraint,
+								boolean forUpdate,
+								int colRefItem,
+								int indexColItem,
+								int lockMode,
+								boolean tableLocked,
+								int isolationLevel,
+								boolean oneRowScan,
+								double optimizerEstimatedRowCount,
+								double optimizerEstimatedCost)
+			throws StandardException;
+
 	/**
 		An index row to base row result set gets an index row from its source
 		and uses the RowLocation in its last column to get the row from the

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryOperatorNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryOperatorNode.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BinaryOperatorNode.java Wed Mar  7 14:02:53 2007
@@ -474,6 +474,31 @@
 											MethodBuilder mb)
 		throws StandardException
 	{
+		/* If this BinaryOperatorNode was created as a part of an IN-list
+		 * "probe predicate" then we do not want to generate the relational
+		 * operator itself; instead we want to generate the underlying
+		 * IN-list for which this operator node was created.
+		 *
+		 * We'll get here in situations where the optimizer chooses a plan
+		 * for which the probe predicate is *not* a useful start/stop key
+		 * and thus is not being used for execution-time index probing.
+		 * In this case we are effectively "reverting" the probe predicate
+		 * back to the InListOperatorNode from which it was created.  Or put
+		 * another way, we are "giving up" on index multi-probing and simply
+		 * generating the original IN-list as a regular restriction.
+		 */
+		if (this instanceof BinaryRelationalOperatorNode)
+		{
+			InListOperatorNode ilon =
+				((BinaryRelationalOperatorNode)this).getInListOp();
+
+			if (ilon != null)
+			{
+				ilon.generateExpression(acb, mb);
+				return;
+			}
+		}
+
 		String		resultTypeName;
 		String		receiverType;
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Wed Mar  7 14:02:53 2007
@@ -143,6 +143,11 @@
 	 */
 	boolean			bulkFetchTurnedOff;
 	
+	/* Whether or not we are going to do execution time "multi-probing"
+	 * on the table scan for this FromBaseTable.
+	 */
+	boolean			multiProbing = false;
+
 	private double	singleScanRowCount;
 
 	private FormatableBitSet referencedCols;
@@ -2732,6 +2737,7 @@
 			if ((pred.getSourceInList() != null) && pred.isStartKey())
 			{
 				disableBulkFetch();
+				multiProbing = true;
 				break;
 			}
 		}
@@ -3121,7 +3127,8 @@
 		int nargs = getScanArguments(acb, mb);
 
 		mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null,
-			trulyTheBestJoinStrategy.resultSetMethodName(bulkFetch != UNSET),
+			trulyTheBestJoinStrategy.resultSetMethodName(
+				(bulkFetch != UNSET), multiProbing),
 			ClassName.NoPutResultSet, nargs);
 
 		/* If this table is the target of an update or a delete, then we must 
@@ -3442,7 +3449,8 @@
 																getLockMode(),
 											(tableDescriptor.getLockGranularity() == TableDescriptor.TABLE_LOCK_GRANULARITY),
 											getCompilerContext().getScanIsolationLevel(),
-											ap.getOptimizer().getMaxMemoryPerTable()
+											ap.getOptimizer().getMaxMemoryPerTable(),
+											multiProbing
 											);
 
 		return nargs;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java Wed Mar  7 14:02:53 2007
@@ -300,7 +300,7 @@
 	}
 
 	/** @see JoinStrategy#resultSetMethodName */
-	public String resultSetMethodName(boolean bulkFetch) {
+	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
 		return "getHashScanResultSet";
 	}
 
@@ -333,9 +333,40 @@
 							int lockMode,
 							boolean tableLocked,
 							int isolationLevel,
-                            int maxMemoryPerTable
+							int maxMemoryPerTable,
+							boolean genInListVals
 							)
-						throws StandardException {
+						throws StandardException
+	{
+		/* If we're doing a Hash join then we shouldn't have any IN-list
+		 * probe predicates in the store restriction list.  The reason
+		 * is that those predicates are one-sided and thus if they
+		 * make it this far they will be pushed down to the base table
+		 * as restrictions on the rows read from disk.  That would be
+		 * wrong because a probe predicate is of the form "col = <val>"
+		 * where <val> is the first value in the IN-list.  But that's
+		 * not correct--we need to return all rows having any value that
+		 * appears in the IN-list (not just those rows matching the
+		 * first value).  Checks elsewhere in the code should ensure
+		 * that no probe predicates have made it this far, but if we're
+		 * running in SANE mode it doesn't hurt to verify.
+		 */
+		if (SanityManager.DEBUG)
+		{
+			Predicate pred = null;
+			for (int i = storeRestrictionList.size() - 1; i >= 0; i--)
+			{
+				pred = (Predicate)storeRestrictionList.getOptPredicate(i);
+				if (pred.getSourceInList() != null)
+				{
+					SanityManager.THROWASSERT("Found IN-list probing " +
+						"(" + pred.binaryRelOpColRefsToString() +
+						") while generating HASH join, which should " +
+						"not happen.");
+				}
+			}
+		}
+
 		ExpressionClassBuilder acb = (ExpressionClassBuilder) acbi;
 
 		fillInScanArgs1(tc,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InListOperatorNode.java Wed Mar  7 14:02:53 2007
@@ -397,14 +397,48 @@
 		resultTypeName = getTypeCompiler().interfaceName();
 
 		// Generate the code to build the array
-		LocalField arrayField =
-			acb.newFieldDeclaration(Modifier.PRIVATE, rightInterfaceType);
+		LocalField arrayField = generateListAsArray(acb, mb);
+
+		/*
+		** Call the method for this operator.
+		*/
+		/*
+		** Generate (field = <left expression>).  This assignment is
+		** used as the receiver of the method call for this operator,
+		** and the field is used as the left operand:
+		**
+		**	(field = <left expression>).method(field, <right expression>...)
+		*/
+
+		//LocalField receiverField =
+		//	acb.newFieldDeclaration(Modifier.PRIVATE, receiverType);
+
+		leftOperand.generateExpression(acb, mb);
+		mb.dup();
+		//mb.putField(receiverField); // instance for method call
+		/*mb.getField(receiverField);*/ mb.upCast(leftInterfaceType); // first arg
+		mb.getField(arrayField); // second arg
+		mb.push(isOrdered); // third arg
+		mb.callMethod(VMOpcode.INVOKEINTERFACE, receiverType, methodName, resultTypeName, 3);
+	}
+
+	/**
+	 * Generate the code to create an array of DataValueDescriptors that
+	 * will hold the IN-list values at execution time.  The array gets
+	 * created in the constructor.  All constant elements in the array
+	 * are initialized in the constructor.  All non-constant elements,
+	 * if any, are initialized each time the IN list is evaluated.
+	 *
+	 * @param acb The ExpressionClassBuilder for the class we're generating
+	 * @param mb The MethodBuilder the expression will go into
+	 */
+	protected LocalField generateListAsArray(ExpressionClassBuilder acb,
+		MethodBuilder mb) throws StandardException
+	{
+		int listSize = rightOperandList.size();
+		LocalField arrayField = acb.newFieldDeclaration(
+			Modifier.PRIVATE, ClassName.DataValueDescriptor + "[]");
 
-		/* The array gets created in the constructor.
-		 * All constant elements in the array are initialized
-		 * in the constructor.  All non-constant elements, if any,
-		 * are initialized each time the IN list is evaluated.
-		 */
 		/* Assign the initializer to the DataValueDescriptor[] field */
 		MethodBuilder cb = acb.getConstructor();
 		cb.pushNewArray(ClassName.DataValueDescriptor, listSize);
@@ -455,7 +489,7 @@
 
 			setArrayMethod.getField(arrayField); // first arg
 			((ValueNode) rightOperandList.elementAt(index)).generateExpression(acb, setArrayMethod);
-			setArrayMethod.upCast(receiverType); // second arg
+			setArrayMethod.upCast(ClassName.DataValueDescriptor); // second arg
 			setArrayMethod.setArrayElement(index);
 		}
 
@@ -472,29 +506,7 @@
 			mb.callMethod(VMOpcode.INVOKEVIRTUAL, (String) null, nonConstantMethod.getName(), "void", 0);
 		}
 
-		/*
-		** Call the method for this operator.
-		*/
-		/*
-		** Generate (field = <left expression>).  This assignment is
-		** used as the receiver of the method call for this operator,
-		** and the field is used as the left operand:
-		**
-		**	(field = <left expression>).method(field, <right expression>...)
-		*/
-
-		//LocalField receiverField =
-		//	acb.newFieldDeclaration(Modifier.PRIVATE, receiverType);
-
-		leftOperand.generateExpression(acb, mb);
-		mb.dup();
-		//mb.putField(receiverField); // instance for method call
-		/*mb.getField(receiverField);*/ mb.upCast(leftInterfaceType); // first arg
-		mb.getField(arrayField); // second arg
-		mb.push(isOrdered); // third arg
-		mb.callMethod(VMOpcode.INVOKEINTERFACE, receiverType, methodName, resultTypeName, 3);
-
-
+		return arrayField;
 	}
 
 
@@ -580,4 +592,14 @@
 
 		}
 	}
+
+	/**
+	 * Return whether or not the IN-list values for this node are ordered.
+	 * This is used for determining whether or not we need to do an execution-
+	 * time sort.
+	 */
+	protected boolean isOrdered()
+	{
+		return isOrdered;
+	} 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java Wed Mar  7 14:02:53 2007
@@ -171,9 +171,11 @@
 	}
 
 	/** @see JoinStrategy#resultSetMethodName */
-	public String resultSetMethodName(boolean bulkFetch) {
+	public String resultSetMethodName(boolean bulkFetch, boolean multiprobe) {
 		if (bulkFetch)
 			return "getBulkTableScanResultSet";
+		else if (multiprobe)
+			return "getMultiProbeTableScanResultSet";
 		else
 			return "getTableScanResultSet";
 	}
@@ -207,7 +209,8 @@
 							int lockMode,
 							boolean tableLocked,
 							int isolationLevel,
-                            int maxMemoryPerTable
+							int maxMemoryPerTable,
+							boolean genInListVals
 							)
 						throws StandardException {
 		ExpressionClassBuilder acb = (ExpressionClassBuilder) acbi;
@@ -223,7 +226,17 @@
 			}
 		}
 
-		if (bulkFetch > 1)
+		/* If we're going to generate a list of IN-values for index probing
+		 * at execution time then we push TableScanResultSet arguments plus
+		 * two additional arguments: 1) the list of IN-list values, and 2)
+		 * a boolean indicating whether or not the IN-list values are already
+		 * sorted.
+		 */
+		if (genInListVals)
+		{
+			numArgs = 26;
+		}
+		else if (bulkFetch > 1)
 		{
 			numArgs = 25;
 		}
@@ -237,6 +250,31 @@
 										storeRestrictionList,
 										acb,
 										resultRowAllocator);
+
+		if (genInListVals)
+			((PredicateList)storeRestrictionList).generateInListValues(acb, mb);
+
+		if (SanityManager.DEBUG)
+		{
+			/* If we're not generating IN-list values with which to probe
+			 * the table then storeRestrictionList should not have any
+			 * IN-list probing predicates.  Make sure that's the case.
+			 */
+			if (!genInListVals)
+			{
+				Predicate pred = null;
+				for (int i = storeRestrictionList.size() - 1; i >= 0; i--)
+				{
+					pred = (Predicate)storeRestrictionList.getOptPredicate(i);
+					if (pred.getSourceInList() != null)
+					{
+						SanityManager.THROWASSERT("Found IN-list probing " +
+							"predicate (" + pred.binaryRelOpColRefsToString() +
+							") when no such predicates were expected.");
+					}
+				}
+			}
+		}
 
 		fillInScanArgs2(mb,
 						innerTable,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ParameterNode.java Wed Mar  7 14:02:53 2007
@@ -86,6 +86,20 @@
 	private ValueNode returnOutputParameter;
 
 	/**
+	 * If this parameter node was created as part of a "probe predicate"
+	 * for an InListOperatorNode then it does not actually correspond to
+	 * a specific value--we just created it as a start-key place-holder
+	 * for IN-list values at execution time.  In order to serve that
+	 * purpose we need to generate some value that can be used as the
+	 * place-holder.  Since this parameter node is "fake" and does not
+	 * correspond to an actual parameter, we can't really generate it;
+	 * so the following field holds some legitimate ValueNode--either a
+	 * constant node or a "real" parameter node--that we can generate to
+	 * serve as the place-holder.
+	 */
+	private ValueNode valToGenerate;
+
+	/**
 	 * Constructor for use by the NodeFactory
 	 */
 	public ParameterNode()
@@ -322,6 +336,15 @@
 											MethodBuilder mb)
 									throws StandardException
 	{
+		/* If we were given a specific ValueNode to generate then
+		 * just use that.
+		 */
+		if (valToGenerate != null)
+		{
+			valToGenerate.generateExpression(acb, mb);
+			return;
+		}
+
 		DataTypeDescriptor dtd = getTypeServices();
 		if ((dtd != null) && dtd.getTypeId().isXMLTypeId()) {
 		// We're a parameter that corresponds to an XML column/target,
@@ -491,5 +514,17 @@
     protected boolean isEquivalent(ValueNode o)
     {
     	return false;
+    }
+
+    /**
+     * Save the received ValueNode locally so that we can generate it
+     * (in place of "this") at generation time.  See the preprocess()
+     * method of InListOperatorNode for more on how this is used.
+     *
+     * @param The ValueNode to generate in place of this ParameterNode.
+     */
+    protected void setValueToGenerate(ValueNode vn)
+    {
+        valToGenerate = vn;
     }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/Predicate.java Wed Mar  7 14:02:53 2007
@@ -1324,18 +1324,4 @@
 			return ((BinaryRelationalOperatorNode)relop).getInListOp();
 		return null;
 	}
-
-	/**
-	 * If this predicate is an IN-list "probe predicate" then "revert"
-	 * it back to its original IN-list form.  This turns out to be
-	 * very easy: we just set the left operand of andNode to be the
-	 * original InListOperatorNode (if non-null).
-	 */
-	protected void revertToSourceInList()
-	{
-		InListOperatorNode ilon = getSourceInList();
-		if (ilon != null)
-			andNode.setLeftOperand(ilon);
-		return;
-	}
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/PredicateList.java Wed Mar  7 14:02:53 2007
@@ -539,7 +539,15 @@
 
                     if (!pred.isPushableOrClause(optTable))
                     {
-                        // NOT an OR or AND, so go on to next predicate.
+                        /* NOT an OR or AND, so go on to next predicate.
+                         *
+                         * Note: if "pred" (or any predicates in the tree
+                         * beneath "pred") is an IN-list probe predicate
+                         * then we'll "revert" it to its original form
+                         * (i.e. to the source InListOperatorNode from
+                         * which it originated) as part of code generation.
+                         * See generateExpression() in BinaryOperatorNode.
+                         */
                         continue;
                     }
                 }
@@ -554,6 +562,17 @@
 
 				pred.markQualifier();
 
+				if (SanityManager.DEBUG)
+				{
+					if (pred.getSourceInList() != null)
+					{
+						SanityManager.THROWASSERT("Found an IN-list probe " +
+							"predicate (" + pred.binaryRelOpColRefsToString() +
+							") that was marked as a qualifier, which should " +
+							"not happen.");
+					}
+				}
+
 				if (pushPreds)
 				{
 					/* Push the predicate down.
@@ -666,14 +685,12 @@
 			{
 				/* If we're pushing predicates then this is the last time
 				 * we'll get here before code generation.  So if we have
-				 * any IN-list probe predicates that are not useful, we
+				 * any IN-list probe predicates that are not useful, we'll
 				 * need to "revert" them back to their original IN-list
 				 * form so that they can be generated as regular IN-list
-				 * restrictions.
+				 * restrictions.  That "revert" operation happens in
+				 * the generateExpression() method of BinaryOperatorNode.
 				 */
-				if (pushPreds && isInListProbePred)
-					pred.revertToSourceInList();
-
 				continue;
 			}
 
@@ -867,11 +884,10 @@
 					 * which means we are preparing to generate code.  Those
 					 * two facts together mean we have to "revert" the
 					 * probe predicate back to its original state so that
-					 * it can be generated as normal IN-list.
+					 * it can be generated as normal IN-list.  That "revert"
+					 * operation happens from within the generateExpression()
+					 * method of BinaryOperatorNode.java.
 					 */
-					if (isInListProbePred)
-						thisPred.revertToSourceInList();
-
 					continue;
 				}
 
@@ -923,7 +939,7 @@
 					 * restriction--*unless* we're dealing with a probe
 					 * predicate, in which case the restriction is handled
 					 * via execution-time index probes (for more see
-					 * execute/TableScanResultSet.java).
+					 * execute/MultiProbeTableScanResultSet.java).
 					 */
 					if (!isIn || isInListProbePred)
 						removeOptPredicate(thisPred);
@@ -2808,6 +2824,75 @@
     }
 
 	/**
+	 * If there is an IN-list probe predicate in this list then generate
+	 * the corresponding IN-list values as a DataValueDescriptor array,
+	 * to be used for probing at execution time.  Also generate a boolean
+	 * value indicating whether or not the values are already in sorted
+	 * order.
+	 *
+	 * Assumption is that by the time we get here there is at most one
+	 * IN-list probe predicate in this list.
+	 *
+	 * @param acb The ActivationClassBuilder for the class we're building
+	 * @param exprFun The MethodBuilder for the method we're building
+	 */
+	protected void generateInListValues(ExpressionClassBuilder acb,
+		MethodBuilder mb) throws StandardException
+	{
+		int size = size();
+		InListOperatorNode ilon = null;
+		for (int index = size - 1; index >= 0; index--)
+		{
+			Predicate pred = (Predicate)elementAt(index);
+			ilon = pred.getSourceInList();
+
+			// Don't do anything if it's not an IN-list probe predicate.
+			if (ilon == null)
+				continue;
+
+			/* We're going to generate the relevant code for the probe
+			 * predicate below, so we no longer need it to be in the
+			 * list.  Remove it now.
+			 */
+			removeOptPredicate(pred);
+
+			/* This list is a store restriction list for a specific base
+			 * table, and we can only have one probe predicate per base
+			 * table (any others, if any, will be "reverted" back to
+			 * their original InListOperatorNodes and generated as
+			 * qualifiers). So make sure there are no other probe preds
+			 * in this list.
+			 */
+			if (SanityManager.DEBUG)
+			{
+				for (int i = 0; i < index; i++)
+				{
+					pred = (Predicate)elementAt(i);
+					if (pred.getSourceInList() != null)
+					{
+						SanityManager.THROWASSERT("Found multiple probe " +
+							"predicates for IN-list when only one was " +
+							"expected.");
+					}
+				}
+			}
+
+			break;
+		}
+
+		if (ilon != null)
+		{
+			mb.getField(ilon.generateListAsArray(acb, mb));
+			mb.push(ilon.isOrdered());
+		}
+		else
+		{
+			mb.pushNull(ClassName.DataValueDescriptor + "[]");
+			mb.push(false);
+		}
+	}
+
+	/**
 	 * @see OptimizablePredicateList#generateQualifiers
 	 *
 	 * @exception StandardException		Thrown on error
@@ -3333,6 +3418,25 @@
 		boolean[]	isAscending = optTable.getTrulyTheBestAccessPath().
 								getConglomerateDescriptor().
 									getIndexDescriptor().isAscending();
+
+		/* If the predicate is an IN-list probe predicate then we are
+		 * using it as a start/stop key "placeholder", to be over-ridden
+		 * at execution time.  Put differently, we want to generate
+		 * "column = ?" as a start/stop key and then use the "?" value
+		 * as a placeholder into which we'll plug the various IN values
+		 * at execution time.
+		 *
+		 * In that case "isIn" will be false here, which is fine: there's
+		 * no need to generate dynamic start/stop keys like we do for
+		 * "normal" IN lists because we're just using the key as a place-
+		 * holder.  So by generating the probe predicate ("column = ?")
+		 * as a normal one-sided start/stop key, we get our requisite
+		 * execution-time placeholder and that's that.  For more on how
+		 * we use this "placeholder", see MultiProbeTableScanResultSet.
+		 *
+		 * Note that we generate the corresponding IN-list values
+		 * separately (see generateInListValues() in this class).
+		 */
 		boolean isIn = pred.getAndNode().getLeftOperand() instanceof InListOperatorNode;
 
 		/*

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?view=diff&rev=515795&r1=515794&r2=515795
==============================================================================
--- 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 Wed Mar  7 14:02:53 2007
@@ -51,6 +51,8 @@
 import org.apache.derby.iapi.sql.conn.Authorizer;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 
+import org.apache.derby.iapi.types.DataValueDescriptor;
+
 import java.util.Properties;
 /**
  * ResultSetFactory provides a wrapper around all of
@@ -723,6 +725,62 @@
 								oneRowScan,
 								optimizerEstimatedRowCount,
 								optimizerEstimatedCost);
+	}
+
+	/**
+		Multi-probing scan that probes an index for specific values contained
+		in the received probe list.
+
+		All index rows for which the first column equals probeVals[0] will
+		be returned, followed by all rows for which the first column equals
+		probeVals[1], and so on.  Assumption is that we only get here if
+		probeVals has at least one value.
+
+		@see ResultSetFactory#getMultiProbeTableScanResultSet
+		@exception StandardException thrown on error
+	 */
+	public NoPutResultSet getMultiProbeTableScanResultSet(
+                        			Activation activation,
+									long conglomId,
+									int scociItem,
+									GeneratedMethod resultRowAllocator,
+									int resultSetNumber,
+									GeneratedMethod startKeyGetter,
+									int startSearchOperator,
+									GeneratedMethod stopKeyGetter,
+									int stopSearchOperator,
+									boolean sameStartStopPosition,
+									Qualifier[][] qualifiers,
+									DataValueDescriptor [] probeVals,
+									boolean probeValsAreSorted,
+									String tableName,
+									String userSuppliedOptimizerOverrides,
+									String indexName,
+									boolean isConstraint,
+									boolean forUpdate,
+									int colRefItem,
+									int indexColItem,
+									int lockMode,
+									boolean tableLocked,
+									int isolationLevel,
+									boolean oneRowScan,
+									double optimizerEstimatedRowCount,
+									double optimizerEstimatedCost)
+			throws StandardException
+	{
+		/* Incremental development: For now we should never actually get to
+		 * this method, so just return null.  When the appropriate execution
+		 * logic is in place (i.e. MultiProbeTableScanResultSet exists) then
+		 * we will add a call to create an instance of the result set here.
+		 */
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.THROWASSERT("Tried to instantiate " +
+				"MultiProbeTableScanResultSet, which does not " +
+				"yet exist (DERBY-47 incremental development).");
+		}
+
+		return (NoPutResultSetImpl)null;
 	}
 
 	/**