[teiid-commits] teiid SVN: r788 - in trunk: engine/src/main/java/com/metamatrix/query/optimizer/batch and 15 other directories.

teiid-commits at lists.jboss.org teiid-commits at lists.jboss.org
Thu Apr 16 14:13:00 EDT 2009


Author: shawkins
Date: 2009-04-16 14:13:00 -0400 (Thu, 16 Apr 2009)
New Revision: 788

Removed:
   trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/PreparedBatchUpdatePlanner.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/batch/PreparedBatchUpdatePlan.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedCommandsEvaluator.java
   trunk/engine/src/main/java/com/metamatrix/query/sql/lang/PreparedBatchUpdate.java
Modified:
   trunk/engine/src/main/java/com/metamatrix/query/optimizer/QueryOptimizer.java
   trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/BatchedUpdatePlanner.java
   trunk/engine/src/main/java/com/metamatrix/query/optimizer/relational/RelationalPlanner.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/relational/AccessNode.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedUpdateNode.java
   trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Delete.java
   trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Insert.java
   trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Update.java
   trunk/engine/src/main/java/com/metamatrix/query/sql/symbol/Reference.java
   trunk/engine/src/main/java/com/metamatrix/query/util/CommandContext.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/MetaDataProcessor.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedPlanCache.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedStatementRequest.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/TestQueryProcessor.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/eval/TestExpressionEvaluator.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/proc/TestProcedureProcessor.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestBatchedUpdateNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestGroupingNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestJoinNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestProjectNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestRelationalNodeStatistics.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSelectNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSortNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestUnionAllNode.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestXMLProcessor.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestCodeTableCache.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedPlanCache.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java
   trunk/test-integration/src/test/java/com/metamatrix/server/integration/BaseQueryTest.java
Log:
TEIID-494 ensured that batched update commands get evaluated if needed.  also refactored the planning logic of batched updates.

Modified: trunk/engine/src/main/java/com/metamatrix/query/optimizer/QueryOptimizer.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/optimizer/QueryOptimizer.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/optimizer/QueryOptimizer.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -22,7 +22,9 @@
 
 package com.metamatrix.query.optimizer;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
 import com.metamatrix.api.exception.query.QueryMetadataException;
@@ -30,17 +32,19 @@
 import com.metamatrix.core.id.IDGenerator;
 import com.metamatrix.core.id.IntegerIDFactory;
 import com.metamatrix.query.analysis.AnalysisRecord;
-import com.metamatrix.query.metadata.*;
+import com.metamatrix.query.metadata.QueryMetadataInterface;
+import com.metamatrix.query.metadata.TempMetadataAdapter;
+import com.metamatrix.query.metadata.TempMetadataStore;
 import com.metamatrix.query.optimizer.batch.BatchedUpdatePlanner;
-import com.metamatrix.query.optimizer.batch.PreparedBatchUpdatePlanner;
 import com.metamatrix.query.optimizer.capabilities.CapabilitiesFinder;
 import com.metamatrix.query.optimizer.proc.ProcedurePlanner;
-import com.metamatrix.query.optimizer.relational.*;
+import com.metamatrix.query.optimizer.relational.MergeTreeNodeProcessor;
+import com.metamatrix.query.optimizer.relational.PlanHints;
+import com.metamatrix.query.optimizer.relational.RelationalPlanner;
 import com.metamatrix.query.optimizer.xml.XMLPlanner;
 import com.metamatrix.query.optimizer.xquery.XQueryPlanner;
 import com.metamatrix.query.processor.ProcessorPlan;
 import com.metamatrix.query.sql.lang.Command;
-import com.metamatrix.query.sql.lang.PreparedBatchUpdate;
 import com.metamatrix.query.sql.symbol.GroupSymbol;
 import com.metamatrix.query.sql.visitor.GroupCollectorVisitor;
 import com.metamatrix.query.util.CommandContext;
@@ -62,7 +66,6 @@
     private static final CommandPlanner XQUERY_PLANNER = new XQueryPlanner();
     private static final CommandPlanner BATCHED_UPDATE_PLANNER = new BatchedUpdatePlanner();
 	private static final CommandTreeProcessor MERGE_TREE_NODE_PROCESSOR = new MergeTreeNodeProcessor();
-	private static final CommandPlanner PREPARED_BATCH_UPDATE_PLANNER = new PreparedBatchUpdatePlanner();
 
 	// Can't construct	
 	private QueryOptimizer() {}
@@ -247,8 +250,6 @@
             result = XQUERY_PLANNER.optimize(node, idGenerator, metadata, capFinder, analysisRecord, context);
         } else if (node.getCommandType() == CommandTreeNode.TYPE_BATCHED_UPDATE_COMMAND){
             result = BATCHED_UPDATE_PLANNER.optimize(node, idGenerator, metadata, capFinder, analysisRecord, context);
-        }else if (node.getCommandType() == CommandTreeNode.TYPE_PREPARED_BATCH_UPDATE_COMMAND){
-        	result = PREPARED_BATCH_UPDATE_PLANNER.optimize(node, idGenerator, metadata, capFinder, analysisRecord, context);
         }
 		return result;
 	}
@@ -265,16 +266,6 @@
             optMetadata = new TempMetadataAdapter(metadata, new TempMetadataStore(tempMetadata));
         }
         
-        //attach a new node for prepared statement batch update
-        if(node.getParent() == null &&command instanceof PreparedBatchUpdate && ((PreparedBatchUpdate)command).isBatchedUpdate()){
-        	node.setCommandType(CommandTreeNode.TYPE_PREPARED_BATCH_UPDATE_COMMAND);
-        	CommandTreeNode child = new CommandTreeNode();
-        	node.addLastChild(child);
-        	child.setParent(node);
-        	recursiveGenerateCanonical(child, command, metadata, analysisRecord, context);
-        	return;        	
-        }
-        
         // Easy to detect batched update planner, procedural planner, or XQueryPlanner
         int commandType = command.getType();
         if (commandType == Command.TYPE_BATCHED_UPDATE) {

Modified: trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/BatchedUpdatePlanner.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/BatchedUpdatePlanner.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/BatchedUpdatePlanner.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -39,6 +39,7 @@
 import com.metamatrix.query.optimizer.capabilities.CapabilitiesFinder;
 import com.metamatrix.query.optimizer.capabilities.SourceCapabilities;
 import com.metamatrix.query.optimizer.capabilities.SourceCapabilities.Capability;
+import com.metamatrix.query.optimizer.relational.RelationalPlanner;
 import com.metamatrix.query.processor.ProcessorPlan;
 import com.metamatrix.query.processor.batch.BatchedUpdatePlan;
 import com.metamatrix.query.processor.relational.BatchedUpdateNode;
@@ -51,6 +52,8 @@
 import com.metamatrix.query.sql.lang.Query;
 import com.metamatrix.query.sql.lang.Update;
 import com.metamatrix.query.sql.symbol.GroupSymbol;
+import com.metamatrix.query.sql.util.VariableContext;
+import com.metamatrix.query.sql.visitor.NeedsEvaluationVisitor;
 import com.metamatrix.query.util.CommandContext;
 
 
@@ -105,7 +108,7 @@
         BatchedUpdateCommand batchedUpdateCommand = (BatchedUpdateCommand)node.getCommand();
         List updateCommands = batchedUpdateCommand.getUpdateCommands();
         int numCommands = updateCommands.size();
-        
+        List<VariableContext> allContexts = (List<VariableContext>)node.getProperty(RelationalPlanner.VARIABLE_CONTEXTS);
         for (int commandIndex = 0; commandIndex < numCommands; commandIndex++) {
             // Potentially the first command of a batch
             Command updateCommand = (Command)updateCommands.get(commandIndex);
@@ -119,9 +122,17 @@
                 // Only attempt batching if the source supports batching
                 if (caps.supportsCapability(Capability.BATCHED_UPDATES)) {
                     // Start a new batch
-                    List batch = new ArrayList();
+                    List<Command> batch = new ArrayList<Command>();
+                    List<VariableContext> contexts = new ArrayList<VariableContext>();
+                    List<Boolean> shouldEvaluate = new ArrayList<Boolean>();
                     // This is the first command in a potential batch, so add it to the batch
                     batch.add(updateCommand);
+                    if (allContexts != null) {
+                    	contexts.add(allContexts.get(commandIndex));
+                    	shouldEvaluate.add(Boolean.TRUE);
+                    } else {
+                    	shouldEvaluate.add(NeedsEvaluationVisitor.needsEvaluation(updateCommand));
+                    }
                     // Find out if there are other commands called on the same physical model
                     // immediately and contiguously after this one
                     batchLoop: for (int batchIndex = commandIndex+1; batchIndex < numCommands; batchIndex++) {
@@ -129,6 +140,12 @@
                         // If this command updates the same model, and is eligible for batching, add it to the batch
                         if (canBeAddedToBatch(batchingCandidate, batchModelID, metadata)) {
                             batch.add(batchingCandidate);
+                            if (allContexts != null) {
+                            	contexts.add(allContexts.get(batchIndex));
+                            	shouldEvaluate.add(Boolean.TRUE);
+                            } else {
+                            	shouldEvaluate.add(NeedsEvaluationVisitor.needsEvaluation(batchingCandidate));
+                            }
                         } else { // Otherwise, stop batching at this point. The next command may well be the start of a new batch
                             break batchLoop;
                         }
@@ -138,7 +155,7 @@
                         ProjectNode projectNode = new ProjectNode(((IntegerID)idGenerator.create()).getValue());
                         // Create a BatchedUpdateNode that creates a batched request for the connector
                         BatchedUpdateNode batchNode = new BatchedUpdateNode(((IntegerID)idGenerator.create()).getValue(),
-                                                                            batch,
+                                                                            batch, contexts, shouldEvaluate,
                                                                             modelName);
                         List symbols = batchedUpdateCommand.getProjectedSymbols();
                         projectNode.setSelectSymbols(symbols);
@@ -168,7 +185,7 @@
      * @return the group being updated
      * @since 4.2
      */
-    private static GroupSymbol getUpdatedGroup(Command command) {
+    public static GroupSymbol getUpdatedGroup(Command command) {
         int type = command.getType();
         if (type == Command.TYPE_INSERT) {
             return ((Insert)command).getGroup();
@@ -176,9 +193,7 @@
             return ((Update)command).getGroup();
         } else if (type == Command.TYPE_DELETE) {
             return ((Delete)command).getGroup();
-        } else if (type == Command.TYPE_QUERY) { // SELECT INTO
-            // Assumes that the group is not a temp group, because the
-            // resolver would have thrown an exception if it was
+        } else if (type == Command.TYPE_QUERY) { 
             return ((Query)command).getInto().getGroup();
         }
         throw new MetaMatrixRuntimeException(QueryExecPlugin.Util.getString("BatchedUpdatePlanner.unrecognized_command", command)); //$NON-NLS-1$
@@ -193,7 +208,7 @@
      * @throws MetaMatrixComponentException
      * @since 4.2
      */
-    private static boolean isEligibleForBatching(Command command, QueryMetadataInterface metadata) throws QueryMetadataException, MetaMatrixComponentException {
+    public static boolean isEligibleForBatching(Command command, QueryMetadataInterface metadata) throws QueryMetadataException, MetaMatrixComponentException {
         // If it's a SELECT INTO, it shouldn't be part of a connector batch.
         if (command.getType() == Command.TYPE_QUERY) {
             return false;

Deleted: trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/PreparedBatchUpdatePlanner.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/PreparedBatchUpdatePlanner.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/optimizer/batch/PreparedBatchUpdatePlanner.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -1,178 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership.  Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- * 
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- * 
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * Lesser General Public License for more details.
- * 
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-
-package com.metamatrix.query.optimizer.batch;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.teiid.dqp.internal.process.PreparedStatementRequest;
-
-import com.metamatrix.api.exception.MetaMatrixComponentException;
-import com.metamatrix.api.exception.query.ExpressionEvaluationException;
-import com.metamatrix.api.exception.query.QueryMetadataException;
-import com.metamatrix.api.exception.query.QueryPlannerException;
-import com.metamatrix.api.exception.query.QueryResolverException;
-import com.metamatrix.core.id.IDGenerator;
-import com.metamatrix.core.id.IntegerID;
-import com.metamatrix.query.analysis.AnalysisRecord;
-import com.metamatrix.query.metadata.QueryMetadataInterface;
-import com.metamatrix.query.optimizer.CommandPlanner;
-import com.metamatrix.query.optimizer.CommandTreeNode;
-import com.metamatrix.query.optimizer.capabilities.CapabilitiesFinder;
-import com.metamatrix.query.optimizer.capabilities.SourceCapabilities;
-import com.metamatrix.query.processor.ProcessorPlan;
-import com.metamatrix.query.processor.batch.BatchedUpdatePlan;
-import com.metamatrix.query.processor.batch.PreparedBatchUpdatePlan;
-import com.metamatrix.query.processor.relational.BatchedCommandsEvaluator;
-import com.metamatrix.query.processor.relational.BatchedUpdateNode;
-import com.metamatrix.query.processor.relational.ProjectNode;
-import com.metamatrix.query.processor.relational.RelationalPlan;
-import com.metamatrix.query.sql.lang.Command;
-import com.metamatrix.query.sql.lang.Delete;
-import com.metamatrix.query.sql.lang.Insert;
-import com.metamatrix.query.sql.lang.PreparedBatchUpdate;
-import com.metamatrix.query.sql.lang.StoredProcedure;
-import com.metamatrix.query.sql.lang.Update;
-import com.metamatrix.query.sql.symbol.GroupSymbol;
-import com.metamatrix.query.sql.visitor.EvaluateExpressionVisitor;
-import com.metamatrix.query.util.CommandContext;
-
-
-/** 
- * Planner for PreparedBatchUpdate
- * @since 5.5.2
- */
-public class PreparedBatchUpdatePlanner implements CommandPlanner {
-
-    /** 
-     * @see com.metamatrix.query.optimizer.CommandPlanner#generateCanonical(com.metamatrix.query.optimizer.CommandTreeNode, com.metamatrix.query.metadata.QueryMetadataInterface, com.metamatrix.query.analysis.AnalysisRecord, CommandContext)
-     * @since 5.5.2
-     */
-    public void generateCanonical(CommandTreeNode rootNode,
-                                  QueryMetadataInterface metadata,
-                                  AnalysisRecord analysisRecord, CommandContext context)
-    throws QueryPlannerException, QueryMetadataException, MetaMatrixComponentException {
-        // do nothing. the planner framework takes care of generating the canonical plan for each of the child commands
-    }
-
-    /** 
-     * If the updates are on a physical source and the source support prepared statement batch update,
-     * just use the plan for the update comand. Otherwise, use PreparedBatchUpdatePlan.
-     * @see com.metamatrix.query.optimizer.CommandPlanner#optimize(com.metamatrix.query.optimizer.CommandTreeNode, com.metamatrix.core.id.IDGenerator, com.metamatrix.query.metadata.QueryMetadataInterface, com.metamatrix.query.optimizer.capabilities.CapabilitiesFinder, com.metamatrix.query.analysis.AnalysisRecord, CommandContext)
-     * @since 5.5.2
-     */
-    public ProcessorPlan optimize(CommandTreeNode node,
-                                  IDGenerator idGenerator,
-                                  QueryMetadataInterface metadata,
-                                  CapabilitiesFinder capFinder,
-                                  AnalysisRecord analysisRecord, CommandContext context)
-    throws QueryPlannerException, QueryMetadataException, MetaMatrixComponentException {
-    	//should have only one child
-    	CommandTreeNode childNode = node.getFirstChild();
-    	Command command = childNode.getCommand();
-    	boolean supportPreparedBatchUpdate = false;
-    	boolean supportBatchedUpdate = false;
-    	if (isEligibleForBatching(command, metadata)) {
-    		GroupSymbol group = getUpdatedGroup(command);
-    		if(group != null){
-                Object batchModelID = metadata.getModelID(group.getMetadataID());
-                String modelName = metadata.getFullName(batchModelID);
-                SourceCapabilities caps = capFinder.findCapabilities(modelName);
-                supportPreparedBatchUpdate = caps.supportsCapability(SourceCapabilities.Capability.PREPARED_BATCH_UPDATE);
-                supportBatchedUpdate = caps.supportsCapability(SourceCapabilities.Capability.BATCHED_UPDATES);
-    		}
-    	}
-    	
-    	ProcessorPlan plan = childNode.getProcessorPlan();
-    	if(supportPreparedBatchUpdate){
-    		return plan;
-    	}
-    	
-    	List paramValues = context.getPreparedBatchUpdateValues();
-    	PreparedBatchUpdate batchUpdate = (PreparedBatchUpdate)childNode.getCommand();
-    	if(supportBatchedUpdate && paramValues.size() > 1){
-    		List batch = new ArrayList();
-    		for(int i=0; i<paramValues.size(); i++ ){
-    			batch.add(batchUpdate.clone());
-    		}
-    		GroupSymbol group = getUpdatedGroup(command);
-            Object batchModelID = metadata.getModelID(group.getMetadataID());
-            String modelName = metadata.getFullName(batchModelID);
-            BatchedUpdateNode batchNode = new BatchedUpdateNode(((IntegerID)idGenerator.create()).getValue(),
-                    batch,
-                    modelName);
-            batchNode.setCommandsEvaluator(
-            		new BatchedCommandsEvaluator(){
-						public void evaluateExpressions(List commands, CommandContext context) throws ExpressionEvaluationException, QueryResolverException, MetaMatrixComponentException {
-		            		PreparedBatchUpdate batchUpdate = (PreparedBatchUpdate)commands.get(0);
-		            		List parameters = batchUpdate.getParameterReferences();
-		            		Iterator valuesIter = context.getPreparedBatchUpdateValues().iterator();
-		            		Iterator commandIter = commands.iterator();
-		            		while(valuesIter.hasNext()){
-		            			List values = (List)valuesIter.next();
-		            			batchUpdate = (PreparedBatchUpdate)commandIter.next();
-		            			PreparedStatementRequest.resolveParameterValues(parameters, values, context);
-		            			EvaluateExpressionVisitor.replaceExpressions(batchUpdate, true, null, context);
-		            		}
-						}         			
-            		});
-            List symbols = Command.getUpdateCommandSymbol();
-            batchNode.setElements(symbols);
-            ProjectNode projectNode = new ProjectNode(((IntegerID)idGenerator.create()).getValue());
-            projectNode.setSelectSymbols(symbols);
-            projectNode.setElements(symbols);
-            projectNode.addChild(batchNode);
-            List childPlans = new ArrayList();
-            RelationalPlan rPlan = new RelationalPlan(projectNode);
-            rPlan.setOutputElements(symbols);
-            childPlans.add(rPlan);
-            return new BatchedUpdatePlan(childPlans, paramValues.size());
-    	}
-    	
-    	return new PreparedBatchUpdatePlan(plan, paramValues, batchUpdate.getParameterReferences());
-    }
-    
-    private static GroupSymbol getUpdatedGroup(Command command) {
-        int type = command.getType();
-        if (type == Command.TYPE_INSERT) {
-            return ((Insert)command).getGroup();
-        } else if (type == Command.TYPE_UPDATE) {
-            return ((Update)command).getGroup();
-        } else if (type == Command.TYPE_DELETE) {
-            return ((Delete)command).getGroup();
-        } else if (type == Command.TYPE_STORED_PROCEDURE) {
-            return ((StoredProcedure)command).getGroup();
-        } else {
-        	return null;
-        }
-    }
-    
-    private static boolean isEligibleForBatching(Command command, QueryMetadataInterface metadata) throws QueryMetadataException, MetaMatrixComponentException {
-        if (command.getType() == Command.TYPE_QUERY) {
-            return false;
-        }
-        // If the command updates a physical group, it's eligible
-        return !metadata.isVirtualGroup(getUpdatedGroup(command).getMetadataID());
-    }
-}

Modified: trunk/engine/src/main/java/com/metamatrix/query/optimizer/relational/RelationalPlanner.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/optimizer/relational/RelationalPlanner.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/optimizer/relational/RelationalPlanner.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -72,7 +72,8 @@
     /**
 	 * Key for a {@link PlanHints PlanHints} object 
 	 */
-	public static final Integer HINTS = new Integer(0);
+	public static final Integer HINTS = Integer.valueOf(0);
+	public static final Integer VARIABLE_CONTEXTS = Integer.valueOf(1);
 
 	/**
      * @see com.metamatrix.query.optimizer.CommandPlanner#generateCanonical(com.metamatrix.query.optimizer.CommandTreeNode, boolean)

Deleted: trunk/engine/src/main/java/com/metamatrix/query/processor/batch/PreparedBatchUpdatePlan.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/batch/PreparedBatchUpdatePlan.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/batch/PreparedBatchUpdatePlan.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -1,185 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership.  Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- * 
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- * 
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * Lesser General Public License for more details.
- * 
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-
-package com.metamatrix.query.processor.batch;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.teiid.dqp.internal.process.PreparedStatementRequest;
-
-import com.metamatrix.api.exception.MetaMatrixComponentException;
-import com.metamatrix.api.exception.MetaMatrixProcessingException;
-import com.metamatrix.common.buffer.BlockedException;
-import com.metamatrix.common.buffer.BufferManager;
-import com.metamatrix.common.buffer.TupleBatch;
-import com.metamatrix.query.processor.BaseProcessorPlan;
-import com.metamatrix.query.processor.DescribableUtil;
-import com.metamatrix.query.processor.ProcessorDataManager;
-import com.metamatrix.query.processor.ProcessorPlan;
-import com.metamatrix.query.sql.lang.Command;
-import com.metamatrix.query.util.CommandContext;
-
-
-/** 
- * Plan for execution for PreparedBatchUpdate.
- * @since 5.5.2
- */
-public class PreparedBatchUpdatePlan extends BaseProcessorPlan {
-    private ProcessorPlan[] updatePlans;
-    private int[] updateCounts;
-    private boolean isPlanOpened;
-    private int planIndex = 0;
-    private List parameterValuesList;
-    private List parameterReferences;
-    
-    public PreparedBatchUpdatePlan(ProcessorPlan plan, List parameterValuesList, List parameterReferences) {
-    	this.parameterValuesList = parameterValuesList;
-    	this.parameterReferences = parameterReferences;
-    	updatePlans = new ProcessorPlan[parameterValuesList.size()];
-    	updatePlans[0] = plan;
-    	for(int i=1; i<updatePlans.length; i++){
-    		updatePlans[i] = (ProcessorPlan)plan.clone();
-    	}
-    	updateCounts = new int[updatePlans.length];
-    }
-
-    public Object clone() {
-        return new PreparedBatchUpdatePlan((ProcessorPlan)updatePlans[0].clone(), parameterValuesList, parameterReferences);
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#initialize(com.metamatrix.query.util.CommandContext, com.metamatrix.query.processor.ProcessorDataManager, com.metamatrix.common.buffer.BufferManager)
-     * @since 5.5.2
-     */
-    public void initialize(CommandContext context, ProcessorDataManager dataMgr, BufferManager bufferMgr) {
-    	for (int i = 0; i < updatePlans.length; i++) {
-    		updatePlans[i].initialize(context, dataMgr, bufferMgr);
-    	}
-    }
-    
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#getOutputElements()
-     * @since 5.5.2
-     */
-    public List getOutputElements() {
-        return Command.getUpdatesCommandSymbol();
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#open()
-     * @since 5.5.2
-     */
-    public void open() throws MetaMatrixComponentException, MetaMatrixProcessingException{
-    	if (!isPlanOpened) { // Open the plan only once
-        	PreparedStatementRequest.resolveParameterValues(parameterReferences, (List)parameterValuesList.get(planIndex), this.getContext());          	
-            updatePlans[planIndex].open();
-            isPlanOpened = true;
-        }
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#nextBatch()
-     * @since 5.5.2
-     */
-    public TupleBatch nextBatch() throws BlockedException, MetaMatrixComponentException, MetaMatrixProcessingException {
-        for (;planIndex < updatePlans.length; planIndex++) {
-        	open();
-            // Execute nextBatch() on each plan in sequence
-            List[] currentBatch = updatePlans[planIndex].nextBatch().getAllTuples(); // Can throw BlockedException
-            //updateCounts[planIndex] = currentBatch[0].get(0);
-            updateCounts[planIndex] = ((Integer)currentBatch[0].get(0)).intValue();
-            
-            // since we are done with the plan explicitly close it.
-            updatePlans[planIndex].close();
-            isPlanOpened = false;
-        }
-        // Add tuples to current batch
-        List rows = new ArrayList();
-        List row = new ArrayList();
-        row.add(updateCounts);
-        rows.add(row);
-        TupleBatch batch = new TupleBatch(1, rows);
-        batch.setTerminationFlag(true);
-        return batch;
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#close()
-     * @since 5.5.2
-     */
-    public void close() throws MetaMatrixComponentException {
-    }
-
-    
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#reset()
-     * @since 5.5.2
-     */
-    public void reset() {
-        super.reset();
-        for (int i = 0; i < updatePlans.length; i++) {
-            updatePlans[i].reset();
-        }
-    	Arrays.fill(updateCounts, -1);
-        planIndex = 0;
-        isPlanOpened = false;
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.Describable#getDescriptionProperties()
-     * @since 5.5.2
-     */
-    public Map getDescriptionProperties() {
-        Map props = new HashMap();
-        props.put(PROP_TYPE, "Batched Update Plan"); //$NON-NLS-1$
-        List children = new ArrayList();
-        for (int i = 0; i < updatePlans.length; i++) {
-            children.add(updatePlans[i].getDescriptionProperties());
-        }
-        props.put(PROP_CHILDREN, children);
-        props.put(PROP_OUTPUT_COLS, DescribableUtil.getOutputColumnProperties(getOutputElements()));
-        return props;
-    }
-    
-    public String toString() {
-        StringBuffer val = new StringBuffer("PreparedBatchUpdatePlan\n"); //$NON-NLS-1$
-        val.append(updatePlans[0]);
-        val.append("\nValues:");//$NON-NLS-1$
-        val.append(parameterValuesList);
-        val.append("\n"); //$NON-NLS-1$
-        return val.toString();
-    }
-
-    /** 
-     * @see com.metamatrix.query.processor.ProcessorPlan#getChildPlans()
-     * @since 5.5.2
-     */
-    public Collection getChildPlans() {
-        return Arrays.asList(updatePlans);
-    }
-
-}

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/relational/AccessNode.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/relational/AccessNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/relational/AccessNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -28,6 +28,8 @@
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
 import com.metamatrix.api.exception.MetaMatrixProcessingException;
+import com.metamatrix.api.exception.query.CriteriaEvaluationException;
+import com.metamatrix.api.exception.query.ExpressionEvaluationException;
 import com.metamatrix.api.exception.query.QueryValidatorException;
 import com.metamatrix.common.buffer.BlockedException;
 import com.metamatrix.common.buffer.TupleBatch;
@@ -36,6 +38,7 @@
 import com.metamatrix.query.rewriter.QueryRewriter;
 import com.metamatrix.query.sql.lang.Command;
 import com.metamatrix.query.sql.visitor.EvaluateExpressionVisitor;
+import com.metamatrix.query.util.CommandContext;
 
 public class AccessNode extends RelationalNode {
 
@@ -105,18 +108,24 @@
 	}
 
     protected boolean prepareNextCommand(Command atomicCommand) throws MetaMatrixComponentException, MetaMatrixProcessingException {
-    	// evaluate all references and any functions on constant values
-        EvaluateExpressionVisitor.replaceExpressions(atomicCommand, true, getDataManager(), getContext());                            
+    	return prepareCommand(atomicCommand, this, this.getContext());
+    }
+
+	static boolean prepareCommand(Command atomicCommand, RelationalNode node, CommandContext context)
+			throws ExpressionEvaluationException, MetaMatrixComponentException,
+			MetaMatrixProcessingException, CriteriaEvaluationException {
+		// evaluate all references and any functions on constant values
+        EvaluateExpressionVisitor.replaceExpressions(atomicCommand, true, node.getDataManager(), context);                            
         
         try {
             // Defect 16059 - Rewrite the command once the references have been replaced with values.
-            QueryRewriter.rewrite(atomicCommand, null, null, getContext());
+            QueryRewriter.rewrite(atomicCommand, null, null, context);
         } catch (QueryValidatorException e) {
             throw new MetaMatrixProcessingException(e, QueryExecPlugin.Util.getString("AccessNode.rewrite_failed", atomicCommand)); //$NON-NLS-1$
         }
         
         return RelationalNodeUtil.shouldExecute(atomicCommand, true);
-    }
+	}
     
 	public TupleBatch nextBatchDirect()
 		throws BlockedException, MetaMatrixComponentException, MetaMatrixProcessingException {

Deleted: trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedCommandsEvaluator.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedCommandsEvaluator.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedCommandsEvaluator.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -1,34 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership.  Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- * 
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- * 
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * Lesser General Public License for more details.
- * 
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-
-package com.metamatrix.query.processor.relational;
-
-import java.util.List;
-
-import com.metamatrix.api.exception.MetaMatrixComponentException;
-import com.metamatrix.api.exception.query.ExpressionEvaluationException;
-import com.metamatrix.api.exception.query.QueryResolverException;
-import com.metamatrix.query.util.CommandContext;
-
-public interface BatchedCommandsEvaluator {
-	void evaluateExpressions(List commands, CommandContext context) throws ExpressionEvaluationException, QueryResolverException, MetaMatrixComponentException;
-}

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedUpdateNode.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedUpdateNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/relational/BatchedUpdateNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -36,6 +36,8 @@
 import com.metamatrix.query.execution.QueryExecPlugin;
 import com.metamatrix.query.sql.lang.BatchedUpdateCommand;
 import com.metamatrix.query.sql.lang.Command;
+import com.metamatrix.query.sql.util.VariableContext;
+import com.metamatrix.query.util.CommandContext;
 
 
 /** 
@@ -44,22 +46,23 @@
  */
 public class BatchedUpdateNode extends RelationalNode {
     
-    private static final List ZERO_COUNT_TUPLE = Arrays.asList(new Object[] {new Integer(0)});
+    private static final List<Integer> ZERO_COUNT_TUPLE = Arrays.asList(Integer.valueOf(0));
 
     /** The commands in this batch. */
-    private List updateCommands;
+    private List<Command> updateCommands;
+    private List<VariableContext> contexts;
+    private List<Boolean> shouldEvaluate;
+    
     /** The model name within the scope of which these commands are being executed. */
     private String modelName;
     /** The tuple source containing the update counts after the batch has been executed. */
     private TupleSource tupleSource;
     
     /** Set containing the indexes of commands that weren't executed. */
-    private Set unexecutedCommands;
+    private Set<Integer> unexecutedCommands;
     /** Flag indicating that at least one command was sent to the DataManager. */
     private boolean commandsWereExecuted = true;
     
-    private BatchedCommandsEvaluator commandsEvaluator;
-
     /**
      *  
      * @param nodeID
@@ -67,8 +70,10 @@
      * @param modelName The name of the model. All the commands in this batch must update groups only within this model.
      * @since 4.2
      */
-    public BatchedUpdateNode(int nodeID, List commands, String modelName) {
+    public BatchedUpdateNode(int nodeID, List<Command> commands, List<VariableContext> contexts, List<Boolean> shouldEvaluate, String modelName) {
         super(nodeID);
+        this.shouldEvaluate = shouldEvaluate;
+        this.contexts = contexts;
         this.updateCommands = commands;
         this.modelName = modelName;
     }
@@ -79,18 +84,27 @@
      */
     public void open() throws MetaMatrixComponentException, MetaMatrixProcessingException {
         super.open();
-        unexecutedCommands = new HashSet(updateCommands.size(), 1.0f);
-        List commandsToExecute = new ArrayList(updateCommands.size());
-        if(commandsEvaluator != null){
-        	commandsEvaluator.evaluateExpressions(updateCommands, this.getContext());
-        }
+        unexecutedCommands = new HashSet<Integer>();
+        List<Command> commandsToExecute = new ArrayList<Command>(updateCommands.size());
         // Find the commands to be executed
         for (int i = 0; i < updateCommands.size(); i++) {
-            Command updateCommand = (Command)updateCommands.get(i);
-            if (RelationalNodeUtil.shouldExecute(updateCommand, true)) {
+            Command updateCommand = (Command)updateCommands.get(i).clone();
+            CommandContext context = this.getContext();
+            if (this.contexts != null) {
+            	context = (CommandContext)context.clone();
+            	context.setVariableContext(this.contexts.get(i));
+            }
+            boolean needProcessing = false;
+            if(shouldEvaluate != null && shouldEvaluate.get(i)) {
+                updateCommand = (Command) updateCommand.clone();
+                needProcessing = AccessNode.prepareCommand(updateCommand, this, context);
+            } else {
+                needProcessing = RelationalNodeUtil.shouldExecute(updateCommand, true);
+            }
+            if (needProcessing) {
                 commandsToExecute.add(updateCommand);
             } else {
-                unexecutedCommands.add(new Integer(i));
+                unexecutedCommands.add(Integer.valueOf(i));
             }
         }
         if (commandsToExecute.isEmpty()) {
@@ -114,7 +128,7 @@
             List tuple;
             for ( int i = 0; i < numExpectedCounts; i++) {
                 // If the command at this index was not executed
-                if (unexecutedCommands.contains(new Integer(i))) {
+                if (unexecutedCommands.contains(Integer.valueOf(i))) {
                     addBatchRow(ZERO_COUNT_TUPLE);
                 } else { // Otherwise, get the next count in the batch
                     tuple = tupleSource.nextTuple();
@@ -172,14 +186,9 @@
         for (int i = 0; i < updateCommands.size(); i++) {
             clonedCommands.add(((Command)updateCommands.get(i)).clone());
         }
-        BatchedUpdateNode clonedNode = new BatchedUpdateNode(getID(), clonedCommands, modelName);
-        clonedNode.commandsEvaluator = this.commandsEvaluator;
+        BatchedUpdateNode clonedNode = new BatchedUpdateNode(getID(), clonedCommands, contexts, shouldEvaluate, modelName);
         super.copy(this, clonedNode);
         return clonedNode;
     }
 
-	public void setCommandsEvaluator(BatchedCommandsEvaluator commandsEvaluator) {
-		this.commandsEvaluator = commandsEvaluator;
-	}
-    
 }

Modified: trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Delete.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Delete.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Delete.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -37,7 +37,7 @@
  * "DELETE FROM <group> [WHERE <criteria>]".
  * Implements Command interface.
  */
-public class Delete extends PreparedBatchUpdate {
+public class Delete extends ProcedureContainer {
 
     /** Identifies the group to delete data from. */
     private GroupSymbol group;
@@ -186,7 +186,6 @@
 
         Delete copy = new Delete(copyGroup, copyCrit);
         copyMetadataState(copy);
-        copy.setParameterReferences(this.getParameterReferences());
 		return copy;
 	}
 

Modified: trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Insert.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Insert.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Insert.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -42,7 +42,7 @@
  * Represents a SQL Insert statement of the form:
  * "INSERT INTO <group> (<variables>) VALUES <values>".
  */
-public class Insert extends PreparedBatchUpdate {
+public class Insert extends ProcedureContainer {
 
     /** Identifies the group to be udpdated. */
     private GroupSymbol group;
@@ -261,7 +261,6 @@
         
 	    Insert copy = new Insert(copyGroup, copyVars, copyVals);
         this.copyMetadataState(copy);
-        copy.setParameterReferences(this.getParameterReferences());
 		return copy;
 	}
 	

Deleted: trunk/engine/src/main/java/com/metamatrix/query/sql/lang/PreparedBatchUpdate.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/sql/lang/PreparedBatchUpdate.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/sql/lang/PreparedBatchUpdate.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -1,56 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership.  Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- * 
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- * 
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * Lesser General Public License for more details.
- * 
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-
-package com.metamatrix.query.sql.lang;
-
-import java.util.List;
-
-import com.metamatrix.api.exception.MetaMatrixComponentException;
-import com.metamatrix.query.metadata.QueryMetadataInterface;
-
-public abstract class PreparedBatchUpdate extends ProcedureContainer{
-	private List parameterReferences; //List<Reference>
-	private int updatingModelCount = -1;
-	
-	public List getParameterReferences() {
-		return parameterReferences;
-	}
-	
-	public void setParameterReferences(List parameterReferences) {
-		this.parameterReferences = parameterReferences;
-	}
-	
-	public boolean isBatchedUpdate(){
-		return parameterReferences != null;
-	}	
-	
-	public int updatingModelCount(QueryMetadataInterface metadata) throws MetaMatrixComponentException{
-		if(updatingModelCount != -1){
-			return updatingModelCount;
-		}
-		return super.updatingModelCount(metadata);
-	}
-
-	public void setUpdatingModelCount(int updatingModelCount) {
-		this.updatingModelCount = updatingModelCount;
-	}
-}

Modified: trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Update.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Update.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/sql/lang/Update.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -41,7 +41,7 @@
  * Represents a SQL Update statement of the form:
  * "UPDATE <group> SET <element> = <expression>, ... [WHERE <criteria>]".
  */
-public class Update extends PreparedBatchUpdate {
+public class Update extends ProcedureContainer {
 
     /** Identifies the group to be udpdated. */
     private GroupSymbol group;
@@ -234,7 +234,6 @@
 		}
 
         this.copyMetadataState(copy);
-        copy.setParameterReferences(this.getParameterReferences());
 		return copy;
 	}
 	

Modified: trunk/engine/src/main/java/com/metamatrix/query/sql/symbol/Reference.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/sql/symbol/Reference.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/sql/symbol/Reference.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -76,7 +76,7 @@
 
     public ElementSymbol getExpression() {
     	if (this.isPositional() && this.expression == null) {
-    		return new ElementSymbol("$param.pos" + this.refIndex); //$NON-NLS-1$
+    		return new ElementSymbol("$param/pos" + this.refIndex); //$NON-NLS-1$
     	}
         return this.expression;    
     }

Modified: trunk/engine/src/main/java/com/metamatrix/query/util/CommandContext.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/util/CommandContext.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/com/metamatrix/query/util/CommandContext.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -23,7 +23,6 @@
 package com.metamatrix.query.util;
 
 import java.io.Serializable;
-import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 import java.util.Stack;
@@ -58,8 +57,6 @@
 
     private String userName;
     
-    private Serializable trustedPayload;
-    
     private Serializable commandPayload;
     
     private String vdbName;
@@ -84,8 +81,6 @@
     
     private SecurityFunctionEvaluator securityFunctionEvaluator;
     
-    private List preparedBatchUpdateValues;
-    
     private Object tempTableStore;
     
     private TimeZone timezone = TimeZone.getDefault();
@@ -101,12 +96,10 @@
      * @param collectNodeStatistics TODO
      */
     public CommandContext(Object processorID, String connectionID, String userName, 
-        Serializable trustedPayload, Serializable commandPayload, String vdbName, String vdbVersion, Properties envProperties, boolean processDebug, 
-        boolean collectNodeStatistics) {
+        Serializable commandPayload, String vdbName, String vdbVersion, Properties envProperties, boolean processDebug, boolean collectNodeStatistics) {
         setProcessorID(processorID);
         setConnectionID(connectionID);
         setUserName(userName);
-        setTrustedPayload(trustedPayload);
         setCommandPayload(commandPayload);
         setVdbName(vdbName);
         setVdbVersion(vdbVersion);  
@@ -118,11 +111,11 @@
     /**
      * Construct a new context.
      */
-    public CommandContext(Object processorID, String connectionID, int outputBatchSize, 
-        String userName, Serializable trustedPayLoad, String vdbName, String vdbVersion) {
+    public CommandContext(Object processorID, String connectionID, String userName, 
+        String vdbName, String vdbVersion) {
 
-        this(processorID, connectionID, userName, trustedPayLoad, null, 
-            vdbName, vdbVersion, null, false, false);            
+        this(processorID, connectionID, userName, null, vdbName, 
+            vdbVersion, null, false, false);            
              
     }
 
@@ -133,7 +126,6 @@
         setProcessorID(context.processorID);
             
         setUserName(context.userName);
-        setTrustedPayload(context.trustedPayload);
         setCommandPayload(context.commandPayload);
         setVdbName(context.vdbName);
         setVdbVersion(context.vdbVersion);   
@@ -147,7 +139,6 @@
         }
         setOptimisticTransaction(context.isOptimisticTransaction());
         this.setSecurityFunctionEvaluator(context.getSecurityFunctionEvaluator());
-        this.preparedBatchUpdateValues = context.preparedBatchUpdateValues;
         this.planToProcessConverter = context.planToProcessConverter;
         this.queryProcessorFactory = context.queryProcessorFactory;
         this.variableContext = context.variableContext;
@@ -179,21 +170,6 @@
         processorID = object;
     }
 
-    public boolean equals(Object obj) {
-        if(this == obj) {
-            return true;
-        }
-        
-        if(obj instanceof CommandContext)  {
-            return this.processorID.equals(((CommandContext)obj).getProcessorID());
-        }
-        return false;
-    }
-    
-    public int hashCode() {
-        return this.processorID.hashCode();
-    }
-    
     public Object clone() {
     	return new CommandContext(this);
     }
@@ -210,13 +186,6 @@
     }
 
     /**
-     * @return Serializable
-     */
-    public Serializable getTrustedPayload() {
-        return trustedPayload;
-    }
-
-    /**
      * @return String
      */
     public String getUserName() {
@@ -246,14 +215,6 @@
     }
 
     /**
-     * Sets the trustedPayLoad.
-     * @param trustedPayLoad The trustedPayLoad to set
-     */
-    public void setTrustedPayload(Serializable trustedPayLoad) {
-        this.trustedPayload = trustedPayLoad;
-    }
-
-    /**
      * Sets the userName.
      * @param userName The userName to set
      */
@@ -402,14 +363,6 @@
         this.securityFunctionEvaluator = securityFunctionEvaluator;
     }
 
-	public List getPreparedBatchUpdateValues() {
-		return preparedBatchUpdateValues;
-	}
-
-	public void setPreparedBatchUpdateValues(List preparedBatchUpdateValues) {
-		this.preparedBatchUpdateValues = preparedBatchUpdateValues;
-	}
-    
 	public Object getTempTableStore() {
 		return tempTableStore;
 	}

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/MetaDataProcessor.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/MetaDataProcessor.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/MetaDataProcessor.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -207,7 +207,7 @@
     private MetadataResult obtainMetadataForPreparedSql(String sql, String sessionId, boolean isDoubleQuotedVariablesAllowed) throws QueryParserException, QueryResolverException, MetaMatrixComponentException {
         Command command = null;
         
-        PreparedPlanCache.PreparedPlan plan = planCache.getPreparedPlan(sessionId, sql, false);
+        PreparedPlanCache.PreparedPlan plan = planCache.getPreparedPlan(sessionId, sql);
         if(plan != null) {
             command = plan.getCommand();
         } else {

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedPlanCache.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedPlanCache.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedPlanCache.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -30,6 +30,7 @@
 import com.metamatrix.query.analysis.AnalysisRecord;
 import com.metamatrix.query.processor.ProcessorPlan;
 import com.metamatrix.query.sql.lang.Command;
+import com.metamatrix.query.sql.symbol.Reference;
 
 /**
  * This class is used to cahce plan and related objects for prepared statement
@@ -37,7 +38,7 @@
 public class PreparedPlanCache {
 	public static final int DEFAULT_MAX_SIZE_TOTAL = 100;
 
-	private LRUCache cache;
+	private LRUCache<CacheID, PreparedPlan> cache;
 	
 	PreparedPlanCache(){
 		this(DEFAULT_MAX_SIZE_TOTAL);
@@ -47,33 +48,33 @@
 		if(maxSize < 0){
 			maxSize = DEFAULT_MAX_SIZE_TOTAL;
 		}
-		cache = new LRUCache(maxSize);
+		cache = new LRUCache<CacheID, PreparedPlan>(maxSize);
 	}	
 	
 	/**
 	 * Return the PreparedPlan for the given session and SQl query
+	 * @param sql SQL query string
 	 * @param session ClientConnection
-	 * @param sql SQL query string
 	 * @return PreparedPlan for the given clientConn and SQl query. Null if not exist.
 	 */
-	public synchronized PreparedPlan getPreparedPlan(String sessionId, String sql, boolean isPreparedBatchUpdate){
+	public synchronized PreparedPlan getPreparedPlan(String sessionId, String sql){
 		ArgCheck.isNotNull(sessionId);
 		ArgCheck.isNotNull(sql);
 		
-		CacheID cID = new CacheID(sessionId, sql, isPreparedBatchUpdate);
+		CacheID cID = new CacheID(sessionId, sql);
 		
-		return (PreparedPlan)cache.get(cID);
+		return cache.get(cID);
 	}
 	
 	/**
 	 * Create PreparedPlan for the given clientConn and SQl query
 	 */
-	public synchronized PreparedPlan createPreparedPlan(String sessionId, String sql, boolean isPreparedBatchUpdate){
+	public synchronized PreparedPlan createPreparedPlan(String sessionId, String sql){
 		ArgCheck.isNotNull(sessionId);
 		ArgCheck.isNotNull(sql);
 		
-		CacheID cID = new CacheID(sessionId, sql, isPreparedBatchUpdate);
-		PreparedPlan preparedPlan = (PreparedPlan)cache.get(cID);
+		CacheID cID = new CacheID(sessionId, sql);
+		PreparedPlan preparedPlan = cache.get(cID);
 		if(preparedPlan == null){
 			preparedPlan = new PreparedPlan();
 			cache.put(cID, preparedPlan);
@@ -104,10 +105,9 @@
 		int hashCode;
 		private boolean isPreparedBatchUpdate;
 		
-		CacheID(String sessionId, String sql, boolean isPreparedBatchUpdate){
+		CacheID(String sessionId, String sql){
 			this.sessionId = sessionId;
 			this.sql = sql;
-			this.isPreparedBatchUpdate = isPreparedBatchUpdate;
 			hashCode = HashCodeUtil.hashCode(HashCodeUtil.hashCode(0, sessionId), sql);
 		}
 		
@@ -132,7 +132,8 @@
 	static class PreparedPlan{
 		private ProcessorPlan plan;
 		private Command command;
-		private List refs;
+		private Command rewritenCommand;
+		private List<Reference> refs;
 		private AnalysisRecord analysisRecord;
 		
 		/**
@@ -159,7 +160,7 @@
 		/**
 		 * Return the list of Reference.
 		 */
-		public List getReferences(){
+		public List<Reference> getReferences(){
 			return refs;
 		}
 		
@@ -187,9 +188,17 @@
 		/**
 		 * Set the list of Reference.
 		 */
-		public void setReferences(List refsValue){
+		public void setReferences(List<Reference> refsValue){
 			refs = refsValue;
 		}
+		
+		public void setRewritenCommand(Command rewritenCommand) {
+			this.rewritenCommand = rewritenCommand;
+		}
+		
+		public Command getRewritenCommand() {
+			return rewritenCommand;
+		}
 					
 	}
 

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedStatementRequest.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedStatementRequest.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/PreparedStatementRequest.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -23,24 +23,31 @@
 package org.teiid.dqp.internal.process;
 
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
 import com.metamatrix.api.exception.query.ExpressionEvaluationException;
+import com.metamatrix.api.exception.query.QueryMetadataException;
 import com.metamatrix.api.exception.query.QueryParserException;
 import com.metamatrix.api.exception.query.QueryPlannerException;
 import com.metamatrix.api.exception.query.QueryResolverException;
 import com.metamatrix.api.exception.query.QueryValidatorException;
 import com.metamatrix.common.log.LogManager;
 import com.metamatrix.common.types.DataTypeManager;
-import com.metamatrix.dqp.DQPPlugin;
 import com.metamatrix.dqp.util.LogConstants;
 import com.metamatrix.query.QueryPlugin;
 import com.metamatrix.query.eval.Evaluator;
+import com.metamatrix.query.optimizer.CommandTreeNode;
+import com.metamatrix.query.optimizer.batch.BatchedUpdatePlanner;
+import com.metamatrix.query.optimizer.capabilities.SourceCapabilities;
+import com.metamatrix.query.optimizer.relational.RelationalPlanner;
 import com.metamatrix.query.processor.ProcessorPlan;
+import com.metamatrix.query.processor.relational.AccessNode;
+import com.metamatrix.query.processor.relational.RelationalPlan;
 import com.metamatrix.query.resolver.util.ResolverUtil;
+import com.metamatrix.query.sql.lang.BatchedUpdateCommand;
 import com.metamatrix.query.sql.lang.Command;
-import com.metamatrix.query.sql.lang.PreparedBatchUpdate;
 import com.metamatrix.query.sql.lang.SPParameter;
 import com.metamatrix.query.sql.lang.StoredProcedure;
 import com.metamatrix.query.sql.symbol.Constant;
@@ -61,27 +68,23 @@
     }
     
     @Override
-    protected void checkReferences(List references)
+    protected void checkReferences(List<Reference> references)
     		throws QueryValidatorException {
-    	//do nothing - references are allowed
+        prepPlan.setReferences(references);
     }
     
     /** 
      * @see org.teiid.dqp.internal.process.Request#resolveCommand(com.metamatrix.query.sql.lang.Command)
      */
-    protected void resolveCommand(Command command, List references) throws QueryResolverException,
+    @Override
+    protected void resolveCommand(Command command) throws QueryResolverException,
                                                   MetaMatrixComponentException {
+    	handleCallableStatement(command);
     	
-    	handleCallableStatement(command, references);
-    	
-    	super.resolveCommand(command, references);
+    	super.resolveCommand(command);
 
-    	if(requestMsg.isPreparedBatchUpdate()){
-        	((PreparedBatchUpdate)command).setParameterReferences(references);
-        }
     	//save the command in it's present form so that it can be validated later
         prepPlan.setCommand((Command) command.clone());
-        prepPlan.setReferences(references);
     }
 
     /**
@@ -89,7 +92,7 @@
      * @param command
      * @param references
      */
-	private void handleCallableStatement(Command command, List references) {
+	private void handleCallableStatement(Command command) {
 		if (!this.requestMsg.isCallableStatement() || !(command instanceof StoredProcedure)) {
     		return;
     	}
@@ -112,31 +115,13 @@
 			}
 			if (param.getExpression() instanceof Reference && index > inParameterCount) {
 				//assume it's an output parameter
-				references.remove(param.getExpression());
+				this.prepPlan.getReferences().remove(param.getExpression());
 				continue;
 			}
 			param.setIndex(index++);
 			proc.setParameter(param);					
 		}
 	}
-
-    protected void resolveParameterValues() throws QueryResolverException, MetaMatrixComponentException {
-        List params = prepPlan.getReferences();
-        List values = requestMsg.getParameterValues();
-        if(requestMsg.isPreparedBatchUpdate()){
-        	if(values.size() > 1){
-        		((PreparedBatchUpdate)userCommand).setUpdatingModelCount(2);
-        	}
-        	for(int i=0; i<values.size(); i++){
-        	   if (params.size() != ((List)values.get(i)).size()) {
-        		   String msg = DQPPlugin.Util.getString("DQPCore.wrong_number_of_values", new Object[] {new Integer(values.size()), new Integer(params.size())}); //$NON-NLS-1$
-        		   throw new QueryResolverException(msg);
-        	   }
-        	}
-        } else {
-        	PreparedStatementRequest.resolveParameterValues(params, values, this.context);
-        }
-    }
     
     @Override
     protected void validateQueryValues(Command command)
@@ -152,13 +137,13 @@
      * @throws QueryPlannerException 
      * @see org.teiid.dqp.internal.process.Request#generatePlan()
      */
-    protected void generatePlan() throws QueryPlannerException, QueryParserException, QueryResolverException, QueryValidatorException, MetaMatrixComponentException {
+    protected Command generatePlan() throws QueryPlannerException, QueryParserException, QueryResolverException, QueryValidatorException, MetaMatrixComponentException {
     	
     	String sqlQuery = requestMsg.getCommands()[0];
-        prepPlan = prepPlanCache.getPreparedPlan(this.workContext.getConnectionID(), sqlQuery, requestMsg.isPreparedBatchUpdate());
+        prepPlan = prepPlanCache.getPreparedPlan(this.workContext.getConnectionID(), sqlQuery);
         if (prepPlan == null) {
             //if prepared plan does not exist, create one
-            prepPlan = prepPlanCache.createPreparedPlan(this.workContext.getConnectionID(), sqlQuery, requestMsg.isPreparedBatchUpdate());
+            prepPlan = prepPlanCache.createPreparedPlan(this.workContext.getConnectionID(), sqlQuery);
             LogManager.logTrace(LogConstants.CTX_DQP, new Object[] { "Query does not exist in cache: ", sqlQuery}); //$NON-NLS-1$
         }
 
@@ -166,7 +151,7 @@
         Command command = prepPlan.getCommand();
         
         if (cachedPlan == null) {
-        	super.generatePlan();
+        	prepPlan.setRewritenCommand(super.generatePlan());
         	
         	if (!this.addedLimit) { //TODO: this is a little problematic 
 		        // Defect 13751: Clone the plan in its current state (i.e. before processing) so that it can be used for later queries
@@ -183,12 +168,88 @@
             this.userCommand = command;
             createCommandContext(command);
         }
-        
-        // validate parameters values - right number and right type
-        resolveParameterValues();
+                
+        if (requestMsg.isPreparedBatchUpdate()) {
+	        handlePreparedBatchUpdate();
+        } else {
+	        List<Reference> params = prepPlan.getReferences();
+	        List<?> values = requestMsg.getParameterValues();
+	
+			resolveAndValidateParameters(command, params, values);
+        }
+        return prepPlan.getRewritenCommand();
+    }
+
+    /**
+     * There are two cases
+     *   if 
+     *     The source supports preparedBatchUpdate -> just let the command and values pass to the source
+     *   else 
+     *     create a batchedupdatecommand that represents the batch operation 
+     * @param command
+     * @throws QueryMetadataException
+     * @throws MetaMatrixComponentException
+     * @throws QueryResolverException
+     * @throws QueryPlannerException 
+     * @throws QueryValidatorException 
+     */
+	private void handlePreparedBatchUpdate() throws QueryMetadataException,
+			MetaMatrixComponentException, QueryResolverException, QueryPlannerException, QueryValidatorException {
+		ProcessorPlan plan = this.processPlan;
+		List<List<?>> paramValues = requestMsg.getParameterValues();
+		if (paramValues.isEmpty()) {
+			throw new QueryValidatorException("No batch values sent for prepared batch update"); //$NON-NLS-1$
+		}
+		boolean supportPreparedBatchUpdate = false;
+		if (plan instanceof RelationalPlan) {
+			RelationalPlan rPlan = (RelationalPlan)plan;
+			if (rPlan.getRootNode() instanceof AccessNode) {
+				AccessNode aNode = (AccessNode)rPlan.getRootNode();
+			    String modelName = aNode.getModelName();
+		        SourceCapabilities caps = capabilitiesFinder.findCapabilities(modelName);
+		        supportPreparedBatchUpdate = caps.supportsCapability(SourceCapabilities.Capability.PREPARED_BATCH_UPDATE);
+			}
+		}
+		CommandTreeNode ctn = new CommandTreeNode();
+		List<Command> commands = new LinkedList<Command>();
+		List<VariableContext> contexts = new LinkedList<VariableContext>();
+		for (List<?> values : paramValues) {
+			resolveAndValidateParameters(this.userCommand, this.prepPlan.getReferences(), values);
+			contexts.add(this.context.getVariableContext());
+			if(supportPreparedBatchUpdate){
+				continue; // -- TODO make this work
+			}
+			Command c = (Command)this.prepPlan.getRewritenCommand().clone();
+			commands.add(c);
+			CommandTreeNode child = new CommandTreeNode();
+			child.setCommand(c);
+			child.setProcessorPlan((ProcessorPlan)this.processPlan.clone());
+			ctn.addLastChild(child);
+		}
+		
+		if (paramValues.size() > 1) {
+			this.context.setVariableContext(null);
+		} 
+		
+		if (supportPreparedBatchUpdate || paramValues.size() == 1) {
+			return; // just use the existing plan
+		}
+		
+		BatchedUpdateCommand buc = new BatchedUpdateCommand(commands);
+		ctn.setCommand(buc);
+		ctn.setProperty(RelationalPlanner.VARIABLE_CONTEXTS, contexts);
+		BatchedUpdatePlanner planner = new BatchedUpdatePlanner();
+		this.processPlan = planner.optimize(ctn, idGenerator, metadata, capabilitiesFinder, analysisRecord, context);
+	}
+
+	private void resolveAndValidateParameters(Command command, List<Reference> params,
+			List<?> values) throws QueryResolverException,
+			MetaMatrixComponentException, QueryValidatorException {
+		// validate parameters values - right number and right type
+    	PreparedStatementRequest.resolveParameterValues(params, values, this.context);
         // call back to Request.validateQueryValues to ensure that bound references are valid
         super.validateQueryValues(command);
-    }
+	}
 
 	/** 
 	 * @param params

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -95,6 +95,7 @@
 import com.metamatrix.query.sql.lang.StoredProcedure;
 import com.metamatrix.query.sql.lang.XQuery;
 import com.metamatrix.query.sql.symbol.Constant;
+import com.metamatrix.query.sql.symbol.Reference;
 import com.metamatrix.query.sql.visitor.ReferenceCollectorVisitor;
 import com.metamatrix.query.tempdata.TempTableStore;
 import com.metamatrix.query.util.CommandContext;
@@ -121,15 +122,15 @@
     private ProcessorDataManager processorDataManager;
     private TransactionService transactionService;
     private TempTableStore tempTableStore;
-    private IDGenerator idGenerator = new IDGenerator();
+    protected IDGenerator idGenerator = new IDGenerator();
     private boolean procDebugAllowed = false;
     private Map connectorCapabilitiesCache;
     DQPWorkContext workContext;
     RequestID requestId;
 
     // acquired state
-    private CapabilitiesFinder capabilitiesFinder;
-    private QueryMetadataInterface metadata;
+    protected CapabilitiesFinder capabilitiesFinder;
+    protected QueryMetadataInterface metadata;
     private Set multiSourceModels;
 
     // internal results
@@ -271,12 +272,11 @@
                 reqID,
                 groupName,
                 workContext.getUserName(),
-                workContext.getTrustedPayload(), 
                 requestMsg.getExecutionPayload(), 
-                workContext.getVdbName(),
+                workContext.getVdbName(), 
                 workContext.getVdbVersion(),
-                props, 
-                useProcDebug(command),
+                props,
+                useProcDebug(command), 
                 collectNodeStatistics(command));
         this.context.setProcessorBatchSize(bufferManager.getProcessorBatchSize());
         this.context.setConnectorBatchSize(bufferManager.getConnectorBatchSize());
@@ -290,79 +290,21 @@
         }
 
         context.setSecurityFunctionEvaluator((SecurityFunctionEvaluator)this.env.findService(DQPServiceNames.AUTHORIZATION_SERVICE));
-        if(requestMsg.isPreparedBatchUpdate()){
-        	context.setPreparedBatchUpdateValues(requestMsg.getParameterValues());
-        }
         context.setTempTableStore(tempTableStore);
         context.setQueryProcessorFactory(this);
     }
 
-    /**
-     * Side effects:
-     * 		creates the analysis record
-     * 		creates the command context
-     * 		sets the pre-rewrite command on the request
-     * 		adds a limit clause if the row limit is specified
-     * 
-     * @return the post rewrite query
-     * @throws QueryParserException
-     * @throws QueryResolverException
-     * @throws QueryValidatorException
-     * @throws MetaMatrixComponentException
-     */
-    protected Command prepareCommand() throws QueryParserException,
-                                   QueryResolverException,
-                                   QueryValidatorException,
-                                   MetaMatrixComponentException {
-        
-        Command command = getCommand();
-
-        List references = ReferenceCollectorVisitor.getReferences(command);
-        
-        //there should be no reference (?) for query/update executed as statement
-        checkReferences(references);
-        
-        createAnalysisRecord(command);
-                
-        resolveCommand(command, references);
-        
-        createCommandContext(command);
-        
-        validateQuery(command, true);
-        
-        validateQueryValues(command);
-        
-        Command preRewrite = command;
-        
-        command = QueryRewriter.rewrite(command, null, metadata, context);
-        
-        /*
-         * Adds a row limit to a query if Statement.setMaxRows has been called and the command
-         * doesn't already have a limit clause.
-         */
-        if (requestMsg.getRowLimit() > 0 && command instanceof QueryCommand) {
-            QueryCommand query = (QueryCommand)command;
-            if (query.getLimit() == null) {
-                query.setLimit(new Limit(null, new Constant(new Integer(requestMsg.getRowLimit()), DataTypeManager.DefaultDataClasses.INTEGER)));
-                this.addedLimit = true;
-            }
-        }
-        
-        this.userCommand = preRewrite;
-        return command;
-    }
-    
-    protected void checkReferences(List references) throws QueryValidatorException {
+    protected void checkReferences(List<Reference> references) throws QueryValidatorException {
     	referenceCheck(references);
     }
     
-    static void referenceCheck(List references) throws QueryValidatorException {
+    static void referenceCheck(List<Reference> references) throws QueryValidatorException {
     	if (references != null && !references.isEmpty()) {
     		throw new QueryValidatorException(DQPPlugin.Util.getString("Request.Invalid_character_in_query")); //$NON-NLS-1$
     	}
     }
 
-    protected void resolveCommand(Command command, List references) throws QueryResolverException, MetaMatrixComponentException {
+    protected void resolveCommand(Command command) throws QueryResolverException, MetaMatrixComponentException {
         if (this.tempTableStore != null) {
         	QueryResolver.setChildMetadata(command, tempTableStore.getMetadataStore().getData(), null);
         }
@@ -391,17 +333,15 @@
         validateWithVisitor(visitor, metadata, command, false);
     }
 
-    private Command getCommand() throws QueryParserException {
+    private Command parseCommand() throws QueryParserException {
         String[] commands = requestMsg.getCommands();
         ParseInfo parseInfo = new ParseInfo();
-        if (requestMsg.isDoubleQuotedVariableAllowed()) {
-        	parseInfo.allowDoubleQuotedVariable = true;
-        }
+    	parseInfo.allowDoubleQuotedVariable = requestMsg.isDoubleQuotedVariableAllowed();
         if (!requestMsg.isBatchedUpdate()) {
         	String commandStr = commands[0];
             return QueryParser.getQueryParser().parseCommand(commandStr, parseInfo);
         } 
-        List parsedCommands = new ArrayList(commands.length);
+        List<Command> parsedCommands = new ArrayList<Command>(commands.length);
         for (int i = 0; i < commands.length; i++) {
         	String updateCommand = commands[i];
             parsedCommands.add(QueryParser.getQueryParser().parseCommand(updateCommand, parseInfo));
@@ -497,7 +437,11 @@
     }
 
     /**
-     * side effects:
+     * state side effects:
+     *      creates the analysis record
+     * 		creates the command context
+     * 		sets the pre-rewrite command on the request
+     * 		adds a limit clause if the row limit is specified
      * 		sets the processor plan
      * 
      * @throws MetaMatrixComponentException
@@ -506,9 +450,40 @@
      * @throws QueryResolverException
      * @throws QueryValidatorException
      */
-    protected void generatePlan() throws MetaMatrixComponentException, QueryPlannerException, QueryParserException, QueryResolverException, QueryValidatorException {
-    	Command command = prepareCommand();
+    protected Command generatePlan() throws MetaMatrixComponentException, QueryPlannerException, QueryParserException, QueryResolverException, QueryValidatorException {
+        Command command = parseCommand();
+
+        List<Reference> references = ReferenceCollectorVisitor.getReferences(command);
         
+        //there should be no reference (?) for query/update executed as statement
+        checkReferences(references);
+        
+        createAnalysisRecord(command);
+                
+        resolveCommand(command);
+        
+        createCommandContext(command);
+        
+        validateQuery(command, true);
+        
+        validateQueryValues(command);
+        
+        this.userCommand = command;
+        
+        command = QueryRewriter.rewrite(command, null, metadata, context);
+        
+        /*
+         * Adds a row limit to a query if Statement.setMaxRows has been called and the command
+         * doesn't already have a limit clause.
+         */
+        if (requestMsg.getRowLimit() > 0 && command instanceof QueryCommand) {
+            QueryCommand query = (QueryCommand)command;
+            if (query.getLimit() == null) {
+                query.setLimit(new Limit(null, new Constant(new Integer(requestMsg.getRowLimit()), DataTypeManager.DefaultDataClasses.INTEGER)));
+                this.addedLimit = true;
+            }
+        }
+        
         try {
         	// If using multi-source models, insert a proxy to simplify the supported capabilities.  This is 
             // done OUTSIDE the cache (wrapped around the cache) intentionally to avoid caching the simplified
@@ -538,6 +513,7 @@
             String msg = DQPPlugin.Util.getString("DQPCore.Unknown_query_metadata_exception_while_registering_query__{0}.", params); //$NON-NLS-1$
             throw new QueryPlannerException(e, msg);
         }
+        return command;
     }
 
     private void setSchemasForXMLPlan(Command command, QueryMetadataInterface metadata)
@@ -628,7 +604,7 @@
 		Command newCommand = QueryParser.getQueryParser().parseCommand(query, parseInfo);
         QueryResolver.resolveCommand(newCommand, metadata);            
         
-        List references = ReferenceCollectorVisitor.getReferences(newCommand);
+        List<Reference> references = ReferenceCollectorVisitor.getReferences(newCommand);
         
         referenceCheck(references);
         

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -233,7 +233,7 @@
 
         try {   
             BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-            CommandContext context = new CommandContext("0", "test", 5, null, null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
+            CommandContext context = new CommandContext("0", "test", null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
             QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataManager);
             processor.process();
             fail("Expected error during processing, but got none."); //$NON-NLS-1$
@@ -312,7 +312,7 @@
 		Properties props = new Properties();
 		props.setProperty("soap_host", "my.host.com"); //$NON-NLS-1$ //$NON-NLS-2$
 		props.setProperty("soap_port", "12345"); //$NON-NLS-1$ //$NON-NLS-2$
-		CommandContext context = new CommandContext("0", "test", "user", null, null, "myvdb", "1", props, DEBUG, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
+		CommandContext context = new CommandContext("0", "test", "user", null, "myvdb", "1", props, DEBUG, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
         context.setProcessorBatchSize(2000);
         context.setConnectorBatchSize(2000);
 		return context;

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/TestQueryProcessor.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/TestQueryProcessor.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/TestQueryProcessor.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -54,7 +54,7 @@
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
         FakeDataManager dataManager = new FakeDataManager();
 
-        CommandContext context = new CommandContext("pid", "group", 100, null, null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "group", null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
         QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataManager);
                  
         while(true) {
@@ -171,7 +171,7 @@
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
         FakeDataManager dataManager = new FakeDataManager();
 
-        CommandContext context = new CommandContext("pid", "group", 100, null, null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "group", null, null, null); //$NON-NLS-1$ //$NON-NLS-2$
         final QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataManager);
         
         processor.setBatchHandler(new QueryProcessor.BatchHandler() {

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/eval/TestExpressionEvaluator.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/eval/TestExpressionEvaluator.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/eval/TestExpressionEvaluator.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -310,7 +310,7 @@
         func.setFunctionDescriptor(desc);
 
         FakeDataManager dataMgr = new FakeDataManager();
-        CommandContext context = new CommandContext(new Long(1), null, -1, null, null, null, null);
+        CommandContext context = new CommandContext(new Long(1), null, null, null, null);
         context.setUserName("logon");  //$NON-NLS-1$
         assertEquals(context.getUserName(), new Evaluator(Collections.emptyMap(), dataMgr, context).evaluate(func, Collections.emptyList()) );       
     } 
@@ -331,7 +331,7 @@
         Properties props = new Properties();
         props.setProperty("http_host", "testHostName"); //$NON-NLS-1$ //$NON-NLS-2$
         props.setProperty("http_port", "8000"); //$NON-NLS-1$ //$NON-NLS-2$
-        CommandContext context = new CommandContext(new Long(1), null, null, null, null, null, null, props, false, false);
+        CommandContext context = new CommandContext(new Long(1), null, null, null, null, null, props, false, false);
         
         func.setArgs(new Expression[] {new Constant("http_host")}); //$NON-NLS-1$
         assertEquals("testHostName", new Evaluator(Collections.emptyMap(), dataMgr, context).evaluate(func, Collections.emptyList())); //$NON-NLS-1$
@@ -353,7 +353,7 @@
         func.setFunctionDescriptor(desc);
         
         FakeDataManager dataMgr = new FakeDataManager();       
-        CommandContext context = new CommandContext(new Long(-1), null, "user", null, payload, "vdb", "1", null, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ 
+        CommandContext context = new CommandContext(new Long(-1), null, "user", payload, "vdb", "1", null, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ 
 
         if(property != null) {
             func.setArgs(new Expression[] {new Constant(property)}); 

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/proc/TestProcedureProcessor.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/proc/TestProcedureProcessor.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/proc/TestProcedureProcessor.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -115,7 +115,7 @@
         // Process twice, testing reset and clone method of Processor plan
         for (int i=1; i<=2; i++) {
 	        BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-            CommandContext context = new CommandContext("pID", null, 10, null, null, null, null); //$NON-NLS-1$
+            CommandContext context = new CommandContext("pID", null, null, null, null); //$NON-NLS-1$
             context.getNextRand(0);
             context.setOptimisticTransaction(optimistic);
             context.setProcessDebug(DEBUG);

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestBatchedUpdateNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestBatchedUpdateNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestBatchedUpdateNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -22,14 +22,15 @@
 
 package com.metamatrix.query.processor.relational;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.junit.Test;
 import org.mockito.Mockito;
 
-import junit.framework.TestCase;
-
 import com.metamatrix.api.exception.MetaMatrixComponentException;
 import com.metamatrix.common.buffer.BlockedException;
 import com.metamatrix.common.buffer.BufferManager;
@@ -38,7 +39,9 @@
 import com.metamatrix.query.metadata.QueryMetadataInterface;
 import com.metamatrix.query.optimizer.batch.TestBatchedUpdatePlanner;
 import com.metamatrix.query.processor.ProcessorDataManager;
+import com.metamatrix.query.sql.lang.BatchedUpdateCommand;
 import com.metamatrix.query.sql.lang.Command;
+import com.metamatrix.query.sql.visitor.NeedsEvaluationVisitor;
 import com.metamatrix.query.unittest.FakeMetadataFactory;
 import com.metamatrix.query.util.CommandContext;
 
@@ -46,14 +49,15 @@
 /** 
  * @since 4.2
  */
-public class TestBatchedUpdateNode extends TestCase {
-    
-    public TestBatchedUpdateNode(String name) {
-        super(name);
-    }
-        
+public class TestBatchedUpdateNode {
+            
     private BatchedUpdateNode helpGetNode(String[] sql, QueryMetadataInterface md, ProcessorDataManager pdm) throws Exception {
-        BatchedUpdateNode node = new BatchedUpdateNode(1, TestBatchedUpdatePlanner.helpGetCommands(sql, md), "myModelName"); //$NON-NLS-1$
+    	List<Command> commands = TestBatchedUpdatePlanner.helpGetCommands(sql, md);
+    	List<Boolean> shouldEvaluate = new ArrayList<Boolean>(commands.size());
+    	for (Command command : commands) {
+			shouldEvaluate.add(NeedsEvaluationVisitor.needsEvaluation(command));
+		}
+        BatchedUpdateNode node = new BatchedUpdateNode(1, commands, null, shouldEvaluate, "myModelName"); //$NON-NLS-1$
         CommandContext context = new CommandContext();
         context.setProcessorID("myProcessorID"); //$NON-NLS-1$
         node.initialize(context, Mockito.mock(BufferManager.class), pdm); 
@@ -72,31 +76,28 @@
         assertEquals(Arrays.asList(expectedCommands), pdm.commands);
     }
     
-    private void helpTestNextBatch(String[] commands, int[] expectedResults) throws Exception {
+    private FakePDM helpTestNextBatch(String[] commands, int[] expectedResults) throws Exception {
         int numExecutedCommands = 0;
         for (int i = 0; i < expectedResults.length; i++) {
             numExecutedCommands += expectedResults[i];
         }
-        BatchedUpdateNode node = helpOpen(commands, new FakePDM(numExecutedCommands));
-        try {
-            TupleBatch batch = node.nextBatch();
-            assertNotNull(batch);
-            assertTrue(batch.getTerminationFlag());
-            assertEquals(expectedResults.length, batch.getRowCount());
-            for (int i = 0; i < expectedResults.length; i++) {
-                List tuple = batch.getTuple(i+1);
-                assertNotNull(tuple);
-                Object result = tuple.get(0);
-                assertNotNull(result);
-                assertEquals(new Integer(expectedResults[i]), result);
-            }
-        } catch (BlockedException e) {
-            e.printStackTrace();
-            fail("Should not have blocked on call to nextBatch()");//$NON-NLS-1$
+        FakePDM fakePDM = new FakePDM(numExecutedCommands);
+        BatchedUpdateNode node = helpOpen(commands, fakePDM);
+        TupleBatch batch = node.nextBatch();
+        assertNotNull(batch);
+        assertTrue(batch.getTerminationFlag());
+        assertEquals(expectedResults.length, batch.getRowCount());
+        for (int i = 0; i < expectedResults.length; i++) {
+            List tuple = batch.getTuple(i+1);
+            assertNotNull(tuple);
+            Object result = tuple.get(0);
+            assertNotNull(result);
+            assertEquals(new Integer(expectedResults[i]), result);
         }
+        return fakePDM;
     }
     
-    public void testOpen1() throws Exception {
+    @Test public void testOpen1() throws Exception {
         String[] sql = {"INSERT INTO pm1.g1 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)", //$NON-NLS-1$
                         "INSERT INTO pm1.g2 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)" //$NON-NLS-1$
         };
@@ -104,7 +105,7 @@
         helpTestOpen(sql, expectedCommands);
     }
     
-    public void testOpen2() throws Exception {
+    @Test public void testOpen2() throws Exception {
         String[] sql = {"INSERT INTO pm1.g1 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)", //$NON-NLS-1$
                         "UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                         "DELETE FROM pm1.g2 WHERE e2 = 50", //$NON-NLS-1$
@@ -114,7 +115,7 @@
         helpTestOpen(sql, expectedCommands);
     }
     
-    public void testOpenAllCommandsExecuted() throws Exception {
+    @Test public void testOpenAllCommandsExecuted() throws Exception {
         String[] sql = {"UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                         "DELETE FROM pm1.g2 WHERE e2 = 50", //$NON-NLS-1$
                         "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE e1 = 'myrow'" //$NON-NLS-1$
@@ -123,7 +124,7 @@
         helpTestOpen(sql, expectedCommands);
     }
     
-    public void testOpenNoCommandsExecuted() throws Exception {
+    @Test public void testOpenNoCommandsExecuted() throws Exception {
         String[] sql = {"UPDATE pm1.g1 SET e2 = 50 WHERE 1 = 0", //$NON-NLS-1$
                         "DELETE FROM pm1.g2 WHERE 1 = 0", //$NON-NLS-1$
                         "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE 1 = 0" //$NON-NLS-1$
@@ -132,7 +133,7 @@
         helpTestOpen(sql, expectedCommands);
     }
     
-    public void testOpenSomeCommandsExecuted() throws Exception {
+    @Test public void testOpenSomeCommandsExecuted() throws Exception {
         String[] sql = {"UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                         "DELETE FROM pm1.g2 WHERE 1 = 0", //$NON-NLS-1$
                         "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE e1 = 'myrow'" //$NON-NLS-1$
@@ -141,7 +142,7 @@
         helpTestOpen(sql, expectedCommands);
     }
     
-    public void testNextBatch1() throws Exception {
+    @Test public void testNextBatch1() throws Exception {
         String[] commands = {"INSERT INTO pm1.g1 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)", //$NON-NLS-1$
                              "INSERT INTO pm1.g2 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)" //$NON-NLS-1$
         };
@@ -149,7 +150,7 @@
         helpTestNextBatch(commands, expectedResults);
     }
     
-    public void testNextBatch2() throws Exception {
+    @Test public void testNextBatch2() throws Exception {
         String[] commands = {"INSERT INTO pm1.g1 (e1, e2, e3, e4) values ('string1', 1, {b'true'}, 1.0)", //$NON-NLS-1$
                              "UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                              "DELETE FROM pm1.g2 WHERE e2 = 50", //$NON-NLS-1$
@@ -159,7 +160,7 @@
         helpTestNextBatch(commands, expectedResults);
     }
     
-    public void testNextBatchAllcommandsExecuted() throws Exception {
+    @Test public void testNextBatchAllcommandsExecuted() throws Exception {
         String[] commands = {"UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                              "DELETE FROM pm1.g2 WHERE e2 = 50", //$NON-NLS-1$
                              "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE e1 = 'myrow'" //$NON-NLS-1$
@@ -168,7 +169,7 @@
         helpTestNextBatch(commands, expectedResults);
     }
     
-    public void testNextBatchNoCommandsExecuted() throws Exception {
+    @Test public void testNextBatchNoCommandsExecuted() throws Exception {
         String[] commands = {"UPDATE pm1.g1 SET e2 = 50 WHERE 1 = 0", //$NON-NLS-1$
                              "DELETE FROM pm1.g2 WHERE 1 = 0", //$NON-NLS-1$
                              "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE 1 = 0" //$NON-NLS-1$
@@ -177,7 +178,7 @@
         helpTestNextBatch(commands, expectedResults);
     }
     
-    public void testNextBatchSomeCommandsExecuted() throws Exception {
+    @Test public void testNextBatchSomeCommandsExecuted() throws Exception {
         String[] commands = {"UPDATE pm1.g1 SET e2 = 50 WHERE e1 = 'criteria'", //$NON-NLS-1$
                              "DELETE FROM pm1.g2 WHERE 1 = 0", //$NON-NLS-1$
                              "UPDATE pm1.g2 set e2 = 5, e3 = {b'false'}, e4 = 3.33 WHERE e1 = 'myrow'", //$NON-NLS-1$
@@ -187,9 +188,18 @@
         helpTestNextBatch(commands, expectedResults);
     }
     
+    @Test public void testNextBatchCommandNeedsEvaluated() throws Exception {
+        String[] commands = {"INSERT INTO pm1.g1 (e1, e2, e3, e4) values (commandpayload(), 1, {b'true'}, 1.0)" //$NON-NLS-1$
+        };
+        int[] expectedResults = {1};
+        FakePDM fpdm = helpTestNextBatch(commands, expectedResults);
+        assertEquals("INSERT INTO pm1.g1 (e1, e2, e3, e4) VALUES (null, 1, TRUE, 1.0)", ((BatchedUpdateCommand)fpdm.actualCommands.get(0)).getContainedCommands().get(0).toString()); //$NON-NLS-1$
+    }
+    
     private static final class FakePDM implements ProcessorDataManager {
     	private int numExecutedCommands;
-        private List commands = new ArrayList();
+        private List<String> commands = new ArrayList<String>();
+        private List<Command> actualCommands = new ArrayList<Command>();
         private FakePDM(int numExecutedCommands) {
         	this.numExecutedCommands = numExecutedCommands;
         }
@@ -199,6 +209,7 @@
             assertEquals("myModelName", modelName); //$NON-NLS-1$
             assertEquals(1, nodeID);
             commands.add(command.toString());
+            actualCommands.add(command);
             return new FakeTupleSource(numExecutedCommands);
         }
     }

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestGroupingNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestGroupingNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestGroupingNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -158,7 +158,7 @@
 		List groupingElements = new ArrayList();
 		groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
 		node.setGroupingElements(groupingElements);	  
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] expected = new List[] {
@@ -192,7 +192,7 @@
         List groupingElements = new ArrayList();
         groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
         node.setGroupingElements(groupingElements);         
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] expected = new List[] {
@@ -228,7 +228,7 @@
         List groupingElements = new ArrayList();
         groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
         node.setGroupingElements(groupingElements);         
-        CommandContext context = new CommandContext("pid", "test", 100, null,  null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null,  null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] expected = new List[] {
@@ -260,7 +260,7 @@
         
         // Set grouping elements to null 
         node.setGroupingElements(null);         
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] data = new List[] {
@@ -301,7 +301,7 @@
         List groupingElements = new ArrayList();
         groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
         node.setGroupingElements(groupingElements);         
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] data = new List[] {
@@ -350,7 +350,7 @@
         List groupingElements = new ArrayList();
         groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
         node.setGroupingElements(groupingElements);   
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);    //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);    //$NON-NLS-1$ //$NON-NLS-2$
         
         FakeDataManager dataMgr = new FakeDataManager();
         dataMgr.setThrowBlocked(true);
@@ -399,7 +399,7 @@
             groupingElements.add(new ElementSymbol("col1")); //$NON-NLS-1$
             node.setGroupingElements(groupingElements);
         }
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         node.initialize(context, mgr, null);
         
         List[] data = new List[] {

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestJoinNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestJoinNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestJoinNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -227,7 +227,7 @@
     public void helpTestJoinDirect(List[] expectedResults, boolean swapExpected) throws MetaMatrixComponentException, MetaMatrixProcessingException {
         helpCreateJoin();                
         BufferManager mgr = NodeTestUtil.getTestBufferManager(1, getProcessorBatchSize());
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         
         join.addChild(leftNode);
         join.addChild(rightNode);

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestProjectNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestProjectNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestProjectNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -63,7 +63,7 @@
     
     public ProjectNode helpSetupProject(List elements, List[] data, List childElements, ProcessorDataManager dataMgr) throws MetaMatrixComponentException {
         BufferManager mgr = BufferManagerFactory.getStandaloneBufferManager();
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         
         FakeRelationalNode dataNode = new FakeRelationalNode(2, data);
         dataNode.setElements(childElements);

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestRelationalNodeStatistics.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestRelationalNodeStatistics.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestRelationalNodeStatistics.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -116,7 +116,7 @@
         
         FakeRelationalNode fakeNode = new FakeRelationalNode(1, data, 100);
         fakeNode.setElements(elements);
-        CommandContext context = new CommandContext("pid", "group", null, null, null, null, null, null, false, true); //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "group", null, null, null, null, null, false, true); //$NON-NLS-1$ //$NON-NLS-2$
         fakeNode.initialize(context, BufferManagerFactory.getStandaloneBufferManager(), null);
         return fakeNode;
     }

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSelectNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSelectNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSelectNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -67,7 +67,7 @@
     
     public void helpTestSelect(List elements, Criteria criteria, List childElements, ProcessorDataManager dataMgr, List[] expected, RelationalNode child) throws MetaMatrixComponentException, MetaMatrixProcessingException {
         BufferManager mgr = BufferManagerFactory.getStandaloneBufferManager();
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         
         child.setElements(childElements);
         child.initialize(context, mgr, dataMgr);    

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSortNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSortNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestSortNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -63,7 +63,7 @@
         impl.getConfig().setTotalAvailableMemory(bytesInMemory);
         impl.getConfig().setGroupUsePercentage(100);
         impl.getConfig().setManagementInterval(0);
-        CommandContext context = new CommandContext ("pid", "test", BATCH_SIZE, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext ("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         
         FakeRelationalNode dataNode = new FakeRelationalNode(2, data);
         dataNode.setElements(elements);

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestUnionAllNode.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestUnionAllNode.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/relational/TestUnionAllNode.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -51,7 +51,7 @@
     
     public void helpTestUnion(RelationalNode[] children, RelationalNode union, List[] expected) throws MetaMatrixComponentException, MetaMatrixProcessingException {
         BufferManager mgr = NodeTestUtil.getTestBufferManager(1, 2);
-        CommandContext context = new CommandContext("pid", "test", 100, null, null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "test", null, null, null);               //$NON-NLS-1$ //$NON-NLS-2$
         
         for(int i=0; i<children.length; i++) {
             union.addChild(children[i]);

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -348,7 +348,7 @@
         
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
         XMLPlan temp = new XMLPlan(env);
-        CommandContext context = new CommandContext("pid", null, 100, null, null, null, null); //$NON-NLS-1$
+        CommandContext context = new CommandContext("pid", null, null, null, null); //$NON-NLS-1$
         temp.initialize(context,null,bufferMgr);
 
         List schema = new ArrayList();
@@ -404,7 +404,7 @@
                 
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
         XMLPlan temp = new XMLPlan(env);
-        CommandContext context = new CommandContext("pid", null, 100, null, null, null, null); //$NON-NLS-1$
+        CommandContext context = new CommandContext("pid", null, null, null, null); //$NON-NLS-1$
         temp.initialize(context,null,bufferMgr);
         env.addData(resultSetName, command.getProjectedSymbols(), new List[] { 
                     Arrays.asList( new Object[] { "001", "Lamp", new Integer(5) } ),         //$NON-NLS-1$ //$NON-NLS-2$

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestXMLProcessor.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestXMLProcessor.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestXMLProcessor.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -2966,7 +2966,7 @@
             // Process twice, to test reset and clone methods
             for (int i=1; i<=2; i++) {
                 BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();                
-                CommandContext context = new CommandContext("pID", "TestConn", 10, "testUser", null, null, null); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+                CommandContext context = new CommandContext("pID", "TestConn", "testUser", null, null); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
                 context.setProcessDebug(DEBUG);
                 QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
     	
@@ -3012,7 +3012,7 @@
             XMLPlan plan = (XMLPlan)QueryOptimizer.optimizePlan(command, metadata, null, new DefaultCapabilitiesFinder(), analysisRecord, null);
     
             BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-            CommandContext context = new CommandContext("pID", null, 10, null, null, null, null);                                                                 //$NON-NLS-1$
+            CommandContext context = new CommandContext("pID", null, null, null, null);                                                                 //$NON-NLS-1$
             QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
             processor.process();
         } catch (Exception e){
@@ -3056,7 +3056,7 @@
         }
         
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-        CommandContext context = new CommandContext("pID", null, 10, null, null, null, null);                                 //$NON-NLS-1$
+        CommandContext context = new CommandContext("pID", null, null, null, null);                                 //$NON-NLS-1$
         QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
         processor.process();
         
@@ -6063,7 +6063,7 @@
         XMLPlan plan = TestXMLPlanner.preparePlan(command, metadata, new DefaultCapabilitiesFinder(), null);
 
         BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-        CommandContext context = new CommandContext("pID", null, 10, null, null, null, null);                                 //$NON-NLS-1$
+        CommandContext context = new CommandContext("pID", null, null, null, null);                                 //$NON-NLS-1$
         QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
 
         MetaMatrixComponentException failOnDefaultException = null;

Modified: trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestCodeTableCache.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestCodeTableCache.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestCodeTableCache.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -38,8 +38,8 @@
  */
 public class TestCodeTableCache extends TestCase {
 
-	private static CommandContext TEST_CONTEXT = new CommandContext("pid", "1", 5, null,  null, "test", "1"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$  
-	private static CommandContext TEST_CONTEXT_1 = new CommandContext("pid", "1", 5, null,  null, "test", "2"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
+	private static CommandContext TEST_CONTEXT = new CommandContext("pid", "1", null, "test",  "1"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$  
+	private static CommandContext TEST_CONTEXT_1 = new CommandContext("pid", "1", null, "test",  "2"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
 	
 	public TestCodeTableCache(String name) {
 		super(name);

Modified: trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedPlanCache.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedPlanCache.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedPlanCache.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -52,11 +52,11 @@
     	PreparedPlanCache cache = new PreparedPlanCache();
     	
     	//No PreparedPlan at the begining
-    	assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 1, false));
+    	assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 1));
     	//create one
-    	cache.createPreparedPlan(token, EXAMPLE_QUERY + 1, false);
+    	cache.createPreparedPlan(token, EXAMPLE_QUERY + 1);
     	//should have one now
-    	assertNotNull("Unable to get prepared plan from cache", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1, false)); //$NON-NLS-1$
+    	assertNotNull("Unable to get prepared plan from cache", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1)); //$NON-NLS-1$
     }
     
     public void testGetPreparedPlan(){
@@ -65,7 +65,7 @@
     	helpCreatePreparedPlans(cache, token2, 0, 15);
     	
     	//read an entry for session2 (token2)
-    	PreparedPlanCache.PreparedPlan pPlan = cache.getPreparedPlan(token2, EXAMPLE_QUERY + 12, false);
+    	PreparedPlanCache.PreparedPlan pPlan = cache.getPreparedPlan(token2, EXAMPLE_QUERY + 12);
     	assertNotNull("Unable to get prepared plan from cache", pPlan); //$NON-NLS-1$
     	assertEquals("Error getting plan from cache", new RelationalPlan(new ProjectNode(12)).toString(), pPlan.getPlan().toString()); //$NON-NLS-1$
     	assertEquals("Error getting command from cache", EXAMPLE_QUERY + 12, pPlan.getCommand().toString()); //$NON-NLS-1$
@@ -77,27 +77,27 @@
     	PreparedPlanCache cache = new PreparedPlanCache();
     	
     	//create one for each session token
-    	cache.createPreparedPlan(token, EXAMPLE_QUERY + 1, false);
-    	cache.createPreparedPlan(token2, EXAMPLE_QUERY + 1, false);
+    	cache.createPreparedPlan(token, EXAMPLE_QUERY + 1);
+    	cache.createPreparedPlan(token2, EXAMPLE_QUERY + 1);
     	//should have one
-    	assertNotNull("Unable to get prepared plan from cache for token", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1, false)); //$NON-NLS-1$
+    	assertNotNull("Unable to get prepared plan from cache for token", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1)); //$NON-NLS-1$
     	cache.clearAll();
     	//should not exist for token
-    	assertNull("Failed remove from cache", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1, false)); //$NON-NLS-1$ 
+    	assertNull("Failed remove from cache", cache.getPreparedPlan(token, EXAMPLE_QUERY + 1)); //$NON-NLS-1$ 
     	//should not exist for token2
-    	assertNull("Unable to get prepared plan from cache for token2", cache.getPreparedPlan(token2, EXAMPLE_QUERY + 1, false)); //$NON-NLS-1$ 
+    	assertNull("Unable to get prepared plan from cache for token2", cache.getPreparedPlan(token2, EXAMPLE_QUERY + 1)); //$NON-NLS-1$ 
     }
     
     public void testMaxSize(){
         PreparedPlanCache cache = new PreparedPlanCache(100);
         helpCreatePreparedPlans(cache, token, 0, 101);
         //the first one should be gone because the max size is 100
-        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 0, false)); 
+        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 0)); 
         
-        assertNotNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 12, false)); 
+        assertNotNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 12)); 
         helpCreatePreparedPlans(cache, token, 102, 50);
         //"sql12" should still be there based on lru  policy
-        assertNotNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 12, false)); 
+        assertNotNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 12)); 
         
         helpCreatePreparedPlans(cache, token2, 0, 121);
         helpCreatePreparedPlans(cache, token, 0, 50);
@@ -111,12 +111,12 @@
         
         // Add 1 plan and verify it is not in the cache
         helpCreatePreparedPlans(cache, token, 0, 1);
-        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 0, false)); 
+        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 0)); 
         assertEquals(0, cache.getSpaceUsed());
         
         // Add another plan and verify it is not in the cache
         helpCreatePreparedPlans(cache, token, 1, 1);
-        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 1, false)); 
+        assertNull(cache.getPreparedPlan(token, EXAMPLE_QUERY + 1)); 
         assertEquals(0, cache.getSpaceUsed());        
     }
     
@@ -138,7 +138,7 @@
 			} catch (QueryParserException e) {
 				throw new RuntimeException(e);
 			}
-    		PreparedPlanCache.PreparedPlan pPlan = cache.createPreparedPlan(token, dummy.toString(), false);
+    		PreparedPlanCache.PreparedPlan pPlan = cache.createPreparedPlan(token, dummy.toString());
     		pPlan.setCommand(dummy); 
     		pPlan.setPlan(new RelationalPlan(new ProjectNode(i)));
             Map props = new HashMap();

Modified: trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -69,9 +69,8 @@
 		
 		@Override
 		public synchronized PreparedPlan getPreparedPlan(
-				String clientConn, String sql,
-				boolean isPreparedBatchUpdate) {
-			PreparedPlan plan = super.getPreparedPlan(clientConn, sql, isPreparedBatchUpdate);
+				String clientConn, String sql) {
+			PreparedPlan plan = super.getPreparedPlan(clientConn, sql);
 			if (plan != null && plan.getPlan() != null) {
 				hitCount++;
 			}

Modified: trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -120,7 +120,7 @@
         idGenerator.setDefaultFactory(new IntegerIDFactory());            
         
         Properties props = new Properties();
-        CommandContext context = new CommandContext("0", "test", "user", null, null, vdbName, vdbVersion, props, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+        CommandContext context = new CommandContext("0", "test", "user", null, vdbName, vdbVersion, props, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
         context.setPlanToProcessConverter(new MultiSourcePlanToProcessConverter(metadata, idGenerator, analysis, finder, multiSourceModels, vdbName, vdbService, vdbVersion));
 
         ProcessorPlan plan = QueryOptimizer.optimizePlan(command, wrapper, idGenerator, finder, analysis, context);

Modified: trunk/test-integration/src/test/java/com/metamatrix/server/integration/BaseQueryTest.java
===================================================================
--- trunk/test-integration/src/test/java/com/metamatrix/server/integration/BaseQueryTest.java	2009-04-16 17:33:22 UTC (rev 787)
+++ trunk/test-integration/src/test/java/com/metamatrix/server/integration/BaseQueryTest.java	2009-04-16 18:13:00 UTC (rev 788)
@@ -152,7 +152,7 @@
     protected CommandContext createCommandContext() {
         Properties props = new Properties();
         //props.setProperty(ContextProperties.SOAP_HOST, "my.host.com"); //$NON-NLS-1$
-        CommandContext context = new CommandContext("0", "test", "user", null, null, "myvdb", "1", props, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
+        CommandContext context = new CommandContext("0", "test", "user", null, "myvdb", "1", props, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
         
         return context;
     }       




More information about the teiid-commits mailing list