[teiid-commits] teiid SVN: r2396 - in trunk: console/src/main/resources/META-INF and 19 other directories.

teiid-commits at lists.jboss.org teiid-commits at lists.jboss.org
Mon Aug 2 11:33:36 EDT 2010


Author: shawkins
Date: 2010-08-02 11:33:33 -0400 (Mon, 02 Aug 2010)
New Revision: 2396

Added:
   trunk/api/src/main/java/org/teiid/CommandContext.java
Modified:
   trunk/console/src/main/resources/META-INF/rhq-plugin.xml
   trunk/documentation/developer-guide/src/main/docbook/en-US/content/udf.xml
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java
   trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java
   trunk/engine/src/main/java/org/teiid/query/optimizer/relational/RelationalPlanner.java
   trunk/engine/src/main/java/org/teiid/query/optimizer/relational/rules/NewCalculateCostUtil.java
   trunk/engine/src/main/java/org/teiid/query/processor/TempTableDataManager.java
   trunk/engine/src/main/java/org/teiid/query/processor/proc/ContinueInstruction.java
   trunk/engine/src/main/java/org/teiid/query/processor/proc/ProcedurePlan.java
   trunk/engine/src/main/java/org/teiid/query/processor/relational/RelationalNode.java
   trunk/engine/src/main/java/org/teiid/query/processor/xml/XMLPlan.java
   trunk/engine/src/main/java/org/teiid/query/util/CommandContext.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestRequest.java
   trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java
   trunk/engine/src/test/java/org/teiid/query/optimizer/relational/TestMaterialization.java
   trunk/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
   trunk/engine/src/test/java/org/teiid/query/processor/TestTempTables.java
   trunk/engine/src/test/java/org/teiid/query/processor/eval/TestExpressionEvaluator.java
   trunk/engine/src/test/java/org/teiid/query/processor/proc/TestProcedureProcessor.java
   trunk/engine/src/test/java/org/teiid/query/processor/relational/TestRelationalNodeStatistics.java
   trunk/engine/src/test/java/org/teiid/query/processor/xml/TestXMLProcessor.java
   trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFacade.java
   trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFactory.java
   trunk/test-integration/common/src/test/java/org/teiid/dqp/internal/process/BaseQueryTest.java
Log:
TEIID-1179 TEIID-168 adding a command context interface, removing the process debug flag, and cleaning up how the temp table datamanager works.

Added: trunk/api/src/main/java/org/teiid/CommandContext.java
===================================================================
--- trunk/api/src/main/java/org/teiid/CommandContext.java	                        (rev 0)
+++ trunk/api/src/main/java/org/teiid/CommandContext.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -0,0 +1,52 @@
+/*
+ * 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 org.teiid;
+
+import java.util.Properties;
+import java.util.TimeZone;
+
+/**
+ * Context information for the currently executing command.
+ * Can be used as an argument to UDFs.
+ */
+public interface CommandContext {
+	
+	String getUserName();
+	
+	String getVdbName();
+	
+	int getVdbVersion();
+	
+	String getConnectionID();
+	
+	Properties getEnvironmentProperties();
+	
+	double getNextRand();
+	
+	double getNextRand(long seed);
+	
+	int getProcessorBatchSize();
+	
+	TimeZone getServerTimeZone();
+
+}


Property changes on: trunk/api/src/main/java/org/teiid/CommandContext.java
___________________________________________________________________
Name: svn:mime-type
   + text/plain

Modified: trunk/console/src/main/resources/META-INF/rhq-plugin.xml
===================================================================
--- trunk/console/src/main/resources/META-INF/rhq-plugin.xml	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/console/src/main/resources/META-INF/rhq-plugin.xml	2010-08-02 15:33:33 UTC (rev 2396)
@@ -285,9 +285,6 @@
 					displayName="Time Slice In Milliseconds"
 					description="Query processor time slice, in milliseconds. (default 2000)"
 					required="false" readOnly="false" />
-				<c:simple-property name="processDebugAllowed"
-					displayName="Process Debug Allowed" description="True if process debug allowed"
-					required="false" readOnly="false" />
 				<c:simple-property name="lobChunkSizeInKB"
 					displayName="Lob Chunk Size In KB"
 					description="The max lob chunk size in KB transferred each time when processing blobs, clobs(100KB default)"

Modified: trunk/documentation/developer-guide/src/main/docbook/en-US/content/udf.xml
===================================================================
--- trunk/documentation/developer-guide/src/main/docbook/en-US/content/udf.xml	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/documentation/developer-guide/src/main/docbook/en-US/content/udf.xml	2010-08-02 15:33:33 UTC (rev 2396)
@@ -46,8 +46,7 @@
 				<para>
 					<emphasis>Input Parameters</emphasis>
 					- defines a type specific signature list. All arguments are
-					considered
-					required.
+					considered required.
 				</para>
 			</listitem>
 			<listitem>
@@ -163,7 +162,7 @@
 				</listitem>
 				<listitem>
 					<para>
-						Number of input arguments must match the function metadata defined
+						Number of input arguments and types must match the function metadata defined
 						in section
 						<link linkend="define_udf">Install user-defined functions</link>
 					</para>
@@ -177,6 +176,9 @@
 					</para>
 				</listitem>
 			</itemizedlist>
+			<para>You may optionally add an additional <code>org.teiid.CommandContext</code> argument as the first parameter.  
+			The <code>CommandContext</code> interface provides access to information about the current command, such as the executing user, the vdb, the session id, etc.  
+			This <code>CommandContext</code> parameter does not need to be delared in the function metadata.</para>
 			<example>
 				<title>Sample code</title>
 				<programlisting><![CDATA[package org.something;

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -43,7 +43,6 @@
     
 	private int maxThreads = DEFAULT_MAX_PROCESS_WORKERS;
 	private int timeSliceInMilli = DEFAULT_PROCESSOR_TIMESLICE;
-	private boolean processDebugAllowed;
 	private int maxRowsFetchSize = DEFAULT_FETCH_SIZE;
 	private int lobChunkSizeInKB = 100;
 	private int preparedPlanCacheMaxCount = SessionAwareCache.DEFAULT_MAX_SIZE_TOTAL;
@@ -85,15 +84,6 @@
 		this.timeSliceInMilli = timeSliceInMilli;
 	}
 	
-	@ManagementProperty(description="Process debug allowed")
-	public boolean isProcessDebugAllowed() {
-		return processDebugAllowed;
-	}
-
-	public void setProcessDebugAllowed(boolean processDebugAllowed) {
-		this.processDebugAllowed = processDebugAllowed;
-	}
-
 	@ManagementProperty(description="Maximum allowed fetch size, set via JDBC. User requested value ignored above this value. (default 20480)")
 	public int getMaxRowsFetchSize() {
 		return maxRowsFetchSize;

Modified: trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -194,7 +194,6 @@
     
     // Query worker pool for processing plans
     private int processorTimeslice = DQPConfiguration.DEFAULT_PROCESSOR_TIMESLICE;
-    private boolean processorDebugAllowed;
     
     private int maxSourceRows = DQPConfiguration.DEFAULT_MAX_SOURCE_ROWS;
     private boolean exceptionOnMaxSourceRows = true;
@@ -333,9 +332,9 @@
 	    }
 	    ClientState state = this.getClientState(workContext.getSessionId(), true);
 	    request.initialize(requestMsg, bufferManager,
-				dataTierMgr, transactionService, processorDebugAllowed,
-				state.sessionTables, workContext,
-				connectorManagerRepository, this.useEntitlements);
+				dataTierMgr, transactionService, state.sessionTables,
+				workContext, connectorManagerRepository,
+				this.useEntitlements);
 		
         ResultsFuture<ResultsMessage> resultsFuture = new ResultsFuture<ResultsMessage>();
         RequestWorkItem workItem = new RequestWorkItem(this, requestMsg, request, resultsFuture.getResultsReceiver(), requestID, workContext);
@@ -640,7 +639,6 @@
 	public void start(DQPConfiguration config) {
 		this.processorTimeslice = config.getTimeSliceInMilli();
         this.maxFetchSize = config.getMaxRowsFetchSize();
-        this.processorDebugAllowed = config.isProcessDebugAllowed();
         this.maxCodeTableRecords = config.getCodeTablesMaxRowsPerTable();
         this.maxCodeTables = config.getCodeTablesMaxCount();
         this.maxCodeRecords = config.getCodeTablesMaxRows();
@@ -659,9 +657,6 @@
         //prepared plan cache
         prepPlanCache = new SessionAwareCache<PreparedPlan>(config.getPreparedPlanCacheMaxCount(), this.cacheFactory, Cache.Type.PREPAREDPLAN);
 		
-        // Processor debug flag
-        LogManager.logInfo(LogConstants.CTX_DQP, DQPPlugin.Util.getString("DQPCore.Processor_debug_allowed_{0}", this.processorDebugAllowed)); //$NON-NLS-1$
-                        
         //get buffer manager
         this.bufferManager = bufferService.getBufferManager();
 

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	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/dqp/internal/process/Request.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -108,7 +108,6 @@
     private TransactionService transactionService;
     private TempTableStore tempTableStore;
     protected IDGenerator idGenerator = new IDGenerator();
-    private boolean procDebugAllowed = false;
     DQPWorkContext workContext;
     RequestID requestId;
 
@@ -137,7 +136,6 @@
                               BufferManager bufferManager,
                               ProcessorDataManager processorDataManager,
                               TransactionService transactionService,
-                              boolean procDebugAllowed,
                               TempTableStore tempTableStore,
                               DQPWorkContext workContext,
                               ConnectorManagerRepository repo,
@@ -149,7 +147,6 @@
         this.bufferManager = bufferManager;
         this.processorDataManager = processorDataManager;
         this.transactionService = transactionService;
-        this.procDebugAllowed = procDebugAllowed;
         this.tempTableStore = tempTableStore;
         idGenerator.setDefaultFactory(new IntegerIDFactory());
         this.workContext = workContext;
@@ -243,7 +240,6 @@
                 workContext.getVdbName(), 
                 workContext.getVdbVersion(),
                 props,
-                useProcDebug(), 
                 this.requestMsg.getShowPlan() != ShowPlan.OFF);
         this.context.setProcessorBatchSize(bufferManager.getProcessorBatchSize());
         this.context.setConnectorBatchSize(bufferManager.getConnectorBatchSize());
@@ -366,16 +362,9 @@
         } 
         
         this.transactionContext = tc;
-        this.processor = new QueryProcessor(processPlan, context, bufferManager, new TempTableDataManager(processorDataManager, tempTableStore));
+        this.processor = new QueryProcessor(processPlan, context, bufferManager, new TempTableDataManager(processorDataManager));
     }
 
-    private boolean useProcDebug() {
-        if(this.procDebugAllowed) {
-            return requestMsg.getShowPlan() == ShowPlan.DEBUG;
-        }
-        return false;
-    }
-    
     /**
      * state side effects:
      *      creates the analysis record

Modified: trunk/engine/src/main/java/org/teiid/query/optimizer/relational/RelationalPlanner.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/optimizer/relational/RelationalPlanner.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/optimizer/relational/RelationalPlanner.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -231,7 +231,7 @@
      * @param groups List of groups (Strings) to be made dependent
      * @param plan The canonical plan
      */
-    private void distributeDependentHints(Collection<String> groups, PlanNode plan, QueryMetadataInterface metadata, NodeConstants.Info hintProperty)
+    private void distributeDependentHints(Collection<String> groups, PlanNode plan, NodeConstants.Info hintProperty)
         throws QueryMetadataException, TeiidComponentException {
     
         if(groups == null || groups.isEmpty()) {
@@ -406,10 +406,10 @@
         // Distribute make dependent hints as necessary
         if (cmd.getOption() != null) {
 	        if(cmd.getOption().getDependentGroups() != null) {
-	            distributeDependentHints(cmd.getOption().getDependentGroups(), result, metadata, NodeConstants.Info.MAKE_DEP);
+	            distributeDependentHints(cmd.getOption().getDependentGroups(), result, NodeConstants.Info.MAKE_DEP);
 	        }
 	        if (cmd.getOption().getNotDependentGroups() != null) {
-	            distributeDependentHints(cmd.getOption().getNotDependentGroups(), result, metadata, NodeConstants.Info.MAKE_NOT_DEP);
+	            distributeDependentHints(cmd.getOption().getNotDependentGroups(), result, NodeConstants.Info.MAKE_NOT_DEP);
 	        }
         }
         this.option = savedOption;
@@ -966,6 +966,7 @@
 								int pos = metadata.getPosition(coldId) - 1;
 								primaryKey.add(id.getElements().get(pos));
 							}
+							id.setPrimaryKey(primaryKey);
 							break;
 						}
 					}
@@ -976,15 +977,16 @@
 			}
 		}
 		if (noCache) {
+    		recordMaterializationTableAnnotation(virtualGroup, analysisRecord, matTableName, "SimpleQueryResolver.materialized_table_not_used"); //$NON-NLS-1$
 			//TODO: update the table
 			return result;
 		} 
-/*		QueryMetadataInterface qmi = new TempMetadataAdapter(metadata, context.getGlobalTableStore().getMetadataStore());
+		QueryMetadataInterface qmi = new TempMetadataAdapter(metadata, context.getGlobalTableStore().getMetadataStore());
 		QueryNode qn = new QueryNode(virtualGroup.getName(), "select * from " + matTableName); //$NON-NLS-1$
 		Query query = (Query)getCommand(virtualGroup, qn, "matview", qmi); //$NON-NLS-1$
 		query.getFrom().getGroups().get(0).setGlobalTable(true);
-		return query;*/
-		return result;
+        recordMaterializationTableAnnotation(virtualGroup, analysisRecord, matTableName, "SimpleQueryResolver.Query_was_redirected_to_Mat_table"); //$NON-NLS-1$                
+		return query;
 	}
 
     public static boolean isNoCacheGroup(QueryMetadataInterface metadata,

Modified: trunk/engine/src/main/java/org/teiid/query/optimizer/relational/rules/NewCalculateCostUtil.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/optimizer/relational/rules/NewCalculateCostUtil.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/optimizer/relational/rules/NewCalculateCostUtil.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -782,26 +782,25 @@
         }    
      
         // Sort elements into groups
-        Map groupMap = new HashMap();
+        Map<GroupSymbol, List<Object>> groupMap = new HashMap<GroupSymbol, List<Object>>();
         for (SingleElementSymbol ses : allElements) {
         	if (!(ses instanceof ElementSymbol)) {
         		continue;
         	}
         	ElementSymbol element = (ElementSymbol)ses;
             GroupSymbol group = element.getGroupSymbol();
-            List elements = (List) groupMap.get(group);
+            List<Object> elements = groupMap.get(group);
             if(elements == null) { 
-                elements = new ArrayList();
+                elements = new ArrayList<Object>();
                 groupMap.put(group, elements);
             }
             elements.add(element.getMetadataID());
         }    
              
         // Walk through each group
-        Iterator groupIter = groupMap.keySet().iterator();
-        while(groupIter.hasNext()) { 
-            GroupSymbol group = (GroupSymbol) groupIter.next();
-            List elements = (List) groupMap.get(group);
+        for (Map.Entry<GroupSymbol, List<Object>> entry : groupMap.entrySet()) {
+            GroupSymbol group = entry.getKey();
+            List<Object> elements = entry.getValue();
             
             // Look up keys
             Collection keys = metadata.getUniqueKeysInGroup(group.getMetadataID());

Modified: trunk/engine/src/main/java/org/teiid/query/processor/TempTableDataManager.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/processor/TempTableDataManager.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/processor/TempTableDataManager.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -37,7 +37,6 @@
 public class TempTableDataManager implements ProcessorDataManager {
 
     private ProcessorDataManager processorDataManager;
-    private TempTableStore tempTableStore;
 
     /**
      * Constructor takes the "real" ProcessorDataManager that this object will be a proxy to,
@@ -45,9 +44,8 @@
      * a temp group will this proxy do it's thing.
      * @param processorDataManager the real ProcessorDataManager that this object is a proxy to
      */
-    public TempTableDataManager(ProcessorDataManager processorDataManager, TempTableStore tempTableStore){
+    public TempTableDataManager(ProcessorDataManager processorDataManager){
         this.processorDataManager = processorDataManager;
-        this.tempTableStore = tempTableStore;
     }
 
 	public TupleSource registerRequest(
@@ -57,6 +55,7 @@
 		String connectorBindingId, int nodeID)
 		throws TeiidComponentException, TeiidProcessingException {          
 
+		TempTableStore tempTableStore = context.getTempTableStore();
         if(tempTableStore != null) {
             TupleSource result = tempTableStore.registerRequest(context, command);
             if (result != null) {

Modified: trunk/engine/src/main/java/org/teiid/query/processor/proc/ContinueInstruction.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/processor/proc/ContinueInstruction.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/processor/proc/ContinueInstruction.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -24,13 +24,9 @@
  */
 package org.teiid.query.processor.proc;
 
-import java.util.*;
-
 import org.teiid.client.plan.PlanNode;
 import org.teiid.core.TeiidComponentException;
-import org.teiid.query.processor.program.*;
 
-
 /**
  * <p>This {@link ProgramInstruction} continue with the next loop when processed</p>.
  */

Modified: trunk/engine/src/main/java/org/teiid/query/processor/proc/ProcedurePlan.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/processor/proc/ProcedurePlan.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/processor/proc/ProcedurePlan.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -54,7 +54,6 @@
 import org.teiid.query.processor.ProcessorDataManager;
 import org.teiid.query.processor.ProcessorPlan;
 import org.teiid.query.processor.QueryProcessor;
-import org.teiid.query.processor.TempTableDataManager;
 import org.teiid.query.processor.relational.SubqueryAwareEvaluator;
 import org.teiid.query.sql.ProcedureReservedWords;
 import org.teiid.query.sql.lang.Criteria;
@@ -140,7 +139,7 @@
     public void initialize(CommandContext context, ProcessorDataManager dataMgr, BufferManager bufferMgr) {       
         this.bufferMgr = bufferMgr;
         this.batchSize = bufferMgr.getProcessorBatchSize();
-        setContext(context);
+        setContext(context.clone());
         this.dataMgr = dataMgr;
         this.parentDataMrg = dataMgr;
         if (evaluator == null) {
@@ -203,7 +202,7 @@
 				}
     		}
     		tempTableStore = new TempTableStore(bufferMgr, getContext().getConnectionID());
-            this.dataMgr = new TempTableDataManager(dataMgr, tempTableStore);
+    		getContext().setTempTableStore(tempTableStore);
     	}
     	this.evaluatedParams = true;
     }

Modified: trunk/engine/src/main/java/org/teiid/query/processor/relational/RelationalNode.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/processor/relational/RelationalNode.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/processor/relational/RelationalNode.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -251,7 +251,7 @@
      * @since 4.2
      */
     public final TupleBatch nextBatch() throws BlockedException,  TeiidComponentException, TeiidProcessingException {
-        boolean recordStats = this.context != null && (this.context.getCollectNodeStatistics() || this.context.getProcessDebug());
+        boolean recordStats = this.context != null && this.context.getCollectNodeStatistics();
         
         try {
             while (true) {
@@ -392,7 +392,7 @@
      * @param batch Batch being sent
      */
     private void recordBatch(TupleBatch batch) {
-        if (!this.context.getProcessDebug() || !LogManager.isMessageToBeRecorded(org.teiid.logging.LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+        if (!LogManager.isMessageToBeRecorded(org.teiid.logging.LogConstants.CTX_DQP, MessageLevel.TRACE)) {
         	return;
         }
     	// Print summary

Modified: trunk/engine/src/main/java/org/teiid/query/processor/xml/XMLPlan.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/processor/xml/XMLPlan.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/processor/xml/XMLPlan.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -110,10 +110,12 @@
      * @see ProcessorPlan#initialize(ProcessorDataManager, Object)
      */
     public void initialize(CommandContext context, ProcessorDataManager dataMgr, BufferManager bufferMgr) {
-        setContext(context);
+    	context = context.clone();
+    	setContext(context);
         TempTableStore tempTableStore = new TempTableStore(bufferMgr, context.getConnectionID());
         tempTableStore.setParentTempTableStore(context.getTempTableStore());
-        this.dataMgr = new TempTableDataManager(dataMgr, tempTableStore);
+        context.setTempTableStore(tempTableStore);
+        this.dataMgr = dataMgr;
         this.bufferMgr = bufferMgr;
         this.env.initialize(context, this.dataMgr, this.bufferMgr);
     }

Modified: trunk/engine/src/main/java/org/teiid/query/util/CommandContext.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/query/util/CommandContext.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/main/java/org/teiid/query/util/CommandContext.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -50,7 +50,7 @@
  * a means to pass context-specific information between users of the query processor
  * framework.
  */
-public class CommandContext implements Cloneable {
+public class CommandContext implements Cloneable, org.teiid.CommandContext {
 	
 	private static class GlobalState {
 	    /** Uniquely identify the command being processed */
@@ -73,9 +73,6 @@
 	    
 	    private Properties environmentProperties;
 	    
-	    /** Indicate whether data should be dumped for debugging purposes while processing the query */
-	    private boolean processDebug;  
-	        
 	    /** Indicate whether statistics should be collected for relational node processing*/
 	    private boolean collectNodeStatistics;
 	    
@@ -118,7 +115,7 @@
      * Construct a new context.
      */
     public CommandContext(Object processorID, String connectionID, String userName, 
-        Serializable commandPayload, String vdbName, int vdbVersion, Properties envProperties, boolean processDebug, boolean collectNodeStatistics) {
+        Serializable commandPayload, String vdbName, int vdbVersion, Properties envProperties, boolean collectNodeStatistics) {
         setProcessorID(processorID);
         setConnectionID(connectionID);
         setUserName(userName);
@@ -126,7 +123,6 @@
         setVdbName(vdbName);
         setVdbVersion(vdbVersion);  
         setEnvironmentProperties(envProperties);        
-        setProcessDebug(processDebug);
         setCollectNodeStatistics(collectNodeStatistics);
     }
 
@@ -137,7 +133,7 @@
         String vdbName, int vdbVersion) {
 
         this(processorID, connectionID, userName, null, vdbName, 
-            vdbVersion, null, false, false);            
+            vdbVersion, null, false);            
              
     }
 
@@ -167,14 +163,6 @@
         return globalState.processorID;
     }
 
-    public boolean getProcessDebug() {
-        return globalState.processDebug;
-    }
-    
-    public void setProcessDebug(boolean processDebug) {
-    	globalState.processDebug = processDebug;
-    }
-
     /**
      * @param object
      */

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	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestPreparedStatement.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -245,7 +245,7 @@
         ConnectorManagerRepository repo = Mockito.mock(ConnectorManagerRepository.class);
         Mockito.stub(repo.getConnectorManager(Mockito.anyString())).toReturn(new AutoGenDataService());
         
-        serverRequest.initialize(request, BufferManagerFactory.getStandaloneBufferManager(), null, new FakeTransactionService(), DEBUG, null, workContext, repo,false);
+        serverRequest.initialize(request, BufferManagerFactory.getStandaloneBufferManager(), null, new FakeTransactionService(), null, workContext, repo, false);
 
         serverRequest.setMetadata(capFinder, metadata, null);
         serverRequest.processRequest();

Modified: trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestRequest.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestRequest.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/TestRequest.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -84,7 +84,7 @@
         RequestMessage message = new RequestMessage();
         DQPWorkContext workContext = FakeMetadataFactory.buildWorkContext(metadata, FakeMetadataFactory.example1VDB());
         
-        request.initialize(message, null, null,new FakeTransactionService(),false, null, workContext, repo, false);
+        request.initialize(message, null, null,new FakeTransactionService(),null, workContext, repo, false);
         request.initMetadata();
         request.validateAccess(command);
     }
@@ -138,8 +138,8 @@
         Mockito.stub(repo.getConnectorManager(Mockito.anyString())).toReturn(new AutoGenDataService());
         
         request.initialize(message, Mockito.mock(BufferManager.class),
-				new FakeDataManager(), new FakeTransactionService(), false, null, workContext,
-				repo, false);
+				new FakeDataManager(), new FakeTransactionService(), null, workContext, repo,
+				false);
         
         request.processRequest();
         return request;

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	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/dqp/internal/process/multisource/TestMultiSourcePlanToProcessConverter.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -121,7 +121,7 @@
         idGenerator.setDefaultFactory(new IntegerIDFactory());            
         
         Properties props = new Properties();
-        CommandContext context = new CommandContext("0", "test", "user", null, vdb.getName(), vdb.getVersion(), props, false, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+        CommandContext context = new CommandContext("0", "test", "user", null, vdb.getName(), vdb.getVersion(), props, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
         context.setPlanToProcessConverter(new MultiSourcePlanToProcessConverter(metadata, idGenerator, analysis, finder, multiSourceModels, dqpContext, context));
 
         ProcessorPlan plan = QueryOptimizer.optimizePlan(command, wrapper, idGenerator, finder, analysis, context);

Modified: trunk/engine/src/test/java/org/teiid/query/optimizer/relational/TestMaterialization.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/optimizer/relational/TestMaterialization.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/optimizer/relational/TestMaterialization.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -30,13 +30,19 @@
 import org.junit.Ignore;
 import org.junit.Test;
 import org.teiid.client.plan.Annotation;
+import org.teiid.common.buffer.BufferManagerFactory;
 import org.teiid.query.analysis.AnalysisRecord;
 import org.teiid.query.metadata.QueryMetadataInterface;
 import org.teiid.query.optimizer.TestOptimizer;
+import org.teiid.query.optimizer.TestOptimizer.ComparisonMode;
+import org.teiid.query.processor.ProcessorPlan;
+import org.teiid.query.processor.relational.RelationalPlan;
 import org.teiid.query.sql.lang.Command;
+import org.teiid.query.tempdata.TempTableStore;
 import org.teiid.query.unittest.FakeMetadataFactory;
+import org.teiid.query.util.CommandContext;
 
-
+ at SuppressWarnings("nls")
 public class TestMaterialization {
 	
     @Test public void testMaterializedTransformation() throws Exception {
@@ -137,5 +143,40 @@
         
         TestOptimizer.helpPlanCommand(command, metadata, getGenericFinder(), analysis, new String[] {"SELECT g_0.x FROM MatSrc.MatSrc AS g_0 WHERE g_0.x = '1'"}, ComparisonMode.EXACT_COMMAND_STRING); //$NON-NLS-1$
     }
+    
+    @Test public void testCacheHint() throws Exception {
+        String userSql = "SELECT * from vgroup2"; //$NON-NLS-1$
+        
+        QueryMetadataInterface metadata = FakeMetadataFactory.exampleMaterializedView();
+        AnalysisRecord analysis = new AnalysisRecord(true, DEBUG);
+        
+        Command command = helpGetCommand(userSql, metadata, null);
+        CommandContext cc = new CommandContext();
+        cc.setGlobalTableStore(new TempTableStore(BufferManagerFactory.getStandaloneBufferManager(), "SYSTEM"));
+        ProcessorPlan plan = TestOptimizer.getPlan(command, metadata, getGenericFinder(), analysis, true, cc);
+        TestOptimizer.checkAtomicQueries(new String[] {"SELECT #MAT_MatView.VGroup2.X FROM #MAT_MatView.VGroup2"}, plan);
+        Collection<Annotation> annotations = analysis.getAnnotations();
+        assertNotNull("Expected annotations but got none", annotations); //$NON-NLS-1$
+        assertTrue("Expected one annotation", annotations.size() == 1); //$NON-NLS-1$
+        assertEquals("Expected catagory mat view", annotations.iterator().next().getCategory(), Annotation.MATERIALIZED_VIEW); //$NON-NLS-1$
+    }
+    
+    @Test public void testCacheHintWithPk() throws Exception {
+        String userSql = "SELECT * from vgroup3 where x = 'foo'"; //$NON-NLS-1$
+        
+        QueryMetadataInterface metadata = FakeMetadataFactory.exampleMaterializedView();
+        AnalysisRecord analysis = new AnalysisRecord(true, DEBUG);
+        
+        Command command = helpGetCommand(userSql, metadata, null);
+        CommandContext cc = new CommandContext();
+        cc.setGlobalTableStore(new TempTableStore(BufferManagerFactory.getStandaloneBufferManager(), "SYSTEM"));
+        RelationalPlan plan = (RelationalPlan)TestOptimizer.getPlan(command, metadata, getGenericFinder(), analysis, true, cc);
+        assertEquals(1f, plan.getRootNode().getEstimateNodeCardinality());
+        TestOptimizer.checkAtomicQueries(new String[] {"SELECT #MAT_MatView.VGroup3.X, #MAT_MatView.VGroup3.y FROM #MAT_MatView.VGroup3 WHERE #MAT_MatView.VGroup3.X = 'foo'"}, plan);
+        Collection<Annotation> annotations = analysis.getAnnotations();
+        assertNotNull("Expected annotations but got none", annotations); //$NON-NLS-1$
+        assertTrue("Expected one annotation", annotations.size() == 1); //$NON-NLS-1$
+        assertEquals("Expected catagory mat view", annotations.iterator().next().getCategory(), Annotation.MATERIALIZED_VIEW); //$NON-NLS-1$
+    }
 
 }

Modified: trunk/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/TestProcessor.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/TestProcessor.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -39,6 +39,7 @@
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.StringTokenizer;
 
 import org.junit.Test;
 import org.teiid.client.metadata.ParameterInfo;
@@ -84,6 +85,7 @@
 import org.teiid.query.sql.symbol.Reference;
 import org.teiid.query.sql.util.VariableContext;
 import org.teiid.query.sql.visitor.ReferenceCollectorVisitor;
+import org.teiid.query.tempdata.TempTableStore;
 import org.teiid.query.unittest.FakeMetadataFacade;
 import org.teiid.query.unittest.FakeMetadataFactory;
 import org.teiid.query.unittest.FakeMetadataObject;
@@ -225,6 +227,12 @@
         bufferMgr.setProcessorBatchSize(context.getProcessorBatchSize());
         bufferMgr.setConnectorBatchSize(context.getProcessorBatchSize());
         context.getNextRand(0);
+        if (context.getTempTableStore() == null) {
+        	context.setTempTableStore(new TempTableStore(bufferMgr, context.getConnectionID()));
+        }
+        if (!(dataManager instanceof TempTableDataManager)) {
+        	dataManager = new TempTableDataManager(dataManager);
+        }
         TupleBuffer id = null;
         try {
             QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataManager);
@@ -278,29 +286,42 @@
             	if(cellValue instanceof XMLType){
                     XMLType id =  (XMLType)cellValue; 
                     String actualDoc = id.getString(); 
-                    record = new ArrayList(record);
-                    record.set(0, actualDoc);
+                	if (expectedResults[i].size() == 1) {
+                		compareDocuments((String)expectedResults[i].get(0), actualDoc);
+                        continue;
+                	}
             	}
-            	if (expectedResults[i].size() == 1) {
-                    assertEquals("Row " + i + " does not match expected: ", expectedResults[i].get(0), record.get(0));                 //$NON-NLS-1$ //$NON-NLS-2$
-                    continue;
-            	}
             }
             
             assertEquals("Row " + i + " does not match expected: ", expectedResults[i], record);                 //$NON-NLS-1$ //$NON-NLS-2$
         }
         ts.closeSource();
     }
+    
+	public static void compareDocuments(String expectedDoc, String actualDoc) {
+		StringTokenizer tokens1 = new StringTokenizer(expectedDoc, "\r\n"); //$NON-NLS-1$
+		StringTokenizer tokens2 = new StringTokenizer(actualDoc, "\r\n");//$NON-NLS-1$
+		while(tokens1.hasMoreTokens()){
+			String token1 = tokens1.nextToken().trim();
+			if(!tokens2.hasMoreTokens()){
+				fail("XML doc mismatch: expected=" + token1 + "\nactual=none");//$NON-NLS-1$ //$NON-NLS-2$
+			}
+			String token2 = tokens2.nextToken().trim();
+			assertEquals("XML doc mismatch: ", token1, token2); //$NON-NLS-1$
+		}
+		if(tokens2.hasMoreTokens()){
+			fail("XML doc mismatch: expected=none\nactual=" + tokens2.nextToken().trim());//$NON-NLS-1$
+		}
+	}
 
 	public static CommandContext createCommandContext() {
 		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, "myvdb", 1, props, DEBUG, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
+		CommandContext context = new CommandContext("0", "test", "user", null, "myvdb", 1, props, false); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
         context.setProcessorBatchSize(BufferManager.DEFAULT_PROCESSOR_BATCH_SIZE);
         context.setConnectorBatchSize(BufferManager.DEFAULT_CONNECTOR_BATCH_SIZE);
         context.setBufferManager(BufferManagerFactory.getStandaloneBufferManager());
-        context.setProcessDebug(DEBUG);
 		return context;
 	}   
     	

Modified: trunk/engine/src/test/java/org/teiid/query/processor/TestTempTables.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/TestTempTables.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/TestTempTables.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -35,27 +35,31 @@
 import org.teiid.query.metadata.TempMetadataAdapter;
 import org.teiid.query.tempdata.TempTableStore;
 import org.teiid.query.unittest.FakeMetadataFactory;
+import org.teiid.query.util.CommandContext;
 
 @SuppressWarnings("nls")
 public class TestTempTables {
 	
 	private TempMetadataAdapter metadata;
 	private TempTableDataManager dataManager;
-
+	private TempTableStore tempStore;
+	
 	private void execute(String sql, List[] expectedResults) throws Exception {
 		execute(TestProcessor.helpGetPlan(sql, metadata), expectedResults);
 	}
 	
 	private void execute(ProcessorPlan processorPlan, List[] expectedResults) throws Exception {
-		TestProcessor.doProcess(processorPlan, dataManager, expectedResults, TestProcessor.createCommandContext());
+		CommandContext cc = TestProcessor.createCommandContext();
+		cc.setTempTableStore(tempStore);
+		TestProcessor.doProcess(processorPlan, dataManager, expectedResults, cc);
 	}
 
 	@Before public void setUp() {
-		TempTableStore tempStore = new TempTableStore(BufferManagerFactory.getStandaloneBufferManager(), "1"); //$NON-NLS-1$
+		tempStore = new TempTableStore(BufferManagerFactory.getStandaloneBufferManager(), "1"); //$NON-NLS-1$
 		metadata = new TempMetadataAdapter(FakeMetadataFactory.example1Cached(), tempStore.getMetadataStore());
 		FakeDataManager fdm = new FakeDataManager();
 	    TestProcessor.sampleData1(fdm);
-		dataManager = new TempTableDataManager(fdm, tempStore);
+		dataManager = new TempTableDataManager(fdm);
 	}
 
 	@Test public void testInsertWithQueryExpression() throws Exception {

Modified: trunk/engine/src/test/java/org/teiid/query/processor/eval/TestExpressionEvaluator.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/eval/TestExpressionEvaluator.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/eval/TestExpressionEvaluator.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -332,7 +332,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, 0, props, false, false);
+        CommandContext context = new CommandContext(new Long(1), null, null, null, null, 0, props, 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$
@@ -354,7 +354,7 @@
         func.setFunctionDescriptor(desc);
         
         FakeDataManager dataMgr = new FakeDataManager();       
-        CommandContext context = new CommandContext(new Long(-1), null, "user", 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); //$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/org/teiid/query/processor/proc/TestProcedureProcessor.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/proc/TestProcedureProcessor.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/proc/TestProcedureProcessor.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -98,7 +98,6 @@
     static void helpTestProcess(ProcessorPlan procPlan, int rowsUpdated, List[] expectedResults, boolean shouldFail, ProcessorDataManager dataMgr, QueryMetadataInterface metadata) throws Exception {
         CommandContext context = new CommandContext("pID", null, null, null, 1); //$NON-NLS-1$
         context.getNextRand(0);
-        context.setProcessDebug(DEBUG);
         context.setMetadata(metadata);        	
         
         if (expectedResults == null) {

Modified: trunk/engine/src/test/java/org/teiid/query/processor/relational/TestRelationalNodeStatistics.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/relational/TestRelationalNodeStatistics.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/relational/TestRelationalNodeStatistics.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -111,7 +111,7 @@
         
         FakeRelationalNode fakeNode = new FakeRelationalNode(1, data, 100);
         fakeNode.setElements(elements);
-        CommandContext context = new CommandContext("pid", "group", null, null, null, 1, null, false, true); //$NON-NLS-1$ //$NON-NLS-2$
+        CommandContext context = new CommandContext("pid", "group", null, null, null, 1, null, true); //$NON-NLS-1$ //$NON-NLS-2$
         fakeNode.initialize(context, BufferManagerFactory.getStandaloneBufferManager(), null);
         return fakeNode;
     }

Modified: trunk/engine/src/test/java/org/teiid/query/processor/xml/TestXMLProcessor.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/processor/xml/TestXMLProcessor.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/processor/xml/TestXMLProcessor.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -26,25 +26,20 @@
 
 import java.io.ByteArrayOutputStream;
 import java.io.FileInputStream;
-import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.StringTokenizer;
 
 import org.junit.Test;
 import org.teiid.api.exception.query.QueryPlannerException;
 import org.teiid.client.metadata.ParameterInfo;
 import org.teiid.common.buffer.BufferManager;
 import org.teiid.common.buffer.BufferManagerFactory;
-import org.teiid.common.buffer.TupleBuffer;
-import org.teiid.common.buffer.TupleSource;
 import org.teiid.core.TeiidComponentException;
 import org.teiid.core.TeiidProcessingException;
 import org.teiid.core.types.DataTypeManager;
-import org.teiid.core.types.XMLType;
 import org.teiid.core.util.UnitTestUtil;
 import org.teiid.query.analysis.AnalysisRecord;
 import org.teiid.query.mapping.relational.QueryNode;
@@ -71,8 +66,7 @@
 import org.teiid.query.processor.FakeDataManager;
 import org.teiid.query.processor.ProcessorPlan;
 import org.teiid.query.processor.QueryProcessor;
-import org.teiid.query.processor.xml.ExecStagingTableInstruction;
-import org.teiid.query.processor.xml.XMLPlan;
+import org.teiid.query.processor.TestProcessor;
 import org.teiid.query.resolver.QueryResolver;
 import org.teiid.query.rewriter.QueryRewriter;
 import org.teiid.query.sql.lang.Command;
@@ -90,6 +84,7 @@
  * metadata, and then that XMLPlan being processed with metadata, a 
  * ProcessorDataManager and a QueryProcessor.
  */
+ at SuppressWarnings("nls")
 public class TestXMLProcessor {
     private static final boolean DEBUG = false;
     
@@ -2934,20 +2929,19 @@
     }
 
     static ProcessorPlan helpTestProcess(String sql, String expectedDoc, FakeMetadataFacade metadata, FakeDataManager dataMgr) throws Exception{
-        return helpTestProcess(sql, expectedDoc, metadata, dataMgr, true, TeiidComponentException.class, null);
+        return helpTestProcess(sql, expectedDoc, metadata, dataMgr, null);
     }
 
-    static ProcessorPlan helpTestProcess(String sql, String expectedDoc, FakeMetadataFacade metadata, FakeDataManager dataMgr, final boolean shouldSucceed, Class expectedException, final String shouldFailMsg) throws Exception{
+    static ProcessorPlan helpTestProcess(String sql, String expectedDoc, FakeMetadataFacade metadata, FakeDataManager dataMgr, Class expectedException) throws Exception{
 
-        return helpTestProcess(sql, expectedDoc, metadata, dataMgr, shouldSucceed, expectedException, shouldFailMsg, new DefaultCapabilitiesFinder());
+        return helpTestProcess(sql, metadata, dataMgr, expectedException, new DefaultCapabilitiesFinder(), expectedDoc);
     }
 
-    static ProcessorPlan helpTestProcess(String sql, String expectedDoc, FakeMetadataFacade metadata, FakeDataManager dataMgr, final boolean shouldSucceed, Class expectedException, final String shouldFailMsg, CapabilitiesFinder capFinder) throws Exception{
+    static ProcessorPlan helpTestProcess(String sql, FakeMetadataFacade metadata, FakeDataManager dataMgr, Class expectedException, CapabilitiesFinder capFinder, String... expectedDoc) throws Exception{
         Command command = helpGetCommand(sql, metadata);
+        AnalysisRecord analysisRecord = new AnalysisRecord(false, DEBUG);
 
-        if (shouldSucceed){
-            
-            AnalysisRecord analysisRecord = new AnalysisRecord(false, DEBUG);
+        try {
             CommandContext planningContext = new CommandContext(); //this should be the same as the processing context, but that's not easy to do
             
             ProcessorPlan plan = QueryOptimizer.optimizePlan(command, metadata, null, capFinder, analysisRecord, planningContext);
@@ -2955,124 +2949,26 @@
             if(DEBUG) {
                 System.out.println(analysisRecord.getDebugLog());
             }
-            
-            // 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", "testUser", null, 1); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
-                context.setProcessDebug(DEBUG);
-                QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
-                processor.setNonBlocking(true);
-                BatchCollector collector = processor.createBatchCollector();
-                TupleBuffer id = collector.collectTuples();
-            
-                TupleSource ts = id.createIndexedTupleSource();
-                List row = ts.nextTuple();
-                assertEquals("Incorrect number of columns: ", 1, row.size()); //$NON-NLS-1$
-               
-
-                XMLType result = (XMLType)row.get(0);
-                String actualDoc = result.getString();
-                
-                id.remove();
-                
-                if(DEBUG) {
-                    System.out.println("expectedDoc = \n" + expectedDoc); //$NON-NLS-1$
-                    System.out.println("actualDoc = \n" + actualDoc); //$NON-NLS-1$
-                }
-                //assertEquals("XML doc mismatch: ", expectedDoc, actualDoc); //$NON-NLS-1$
-                compareDocuments(expectedDoc, actualDoc);
-                //Test reset, clone methods
-                if (i==1) {
-                    plan.reset();
-                    plan = plan.clone();
-                }
-            }
+            List[] expected = new List[expectedDoc.length];
+            for (int i = 0; i < expectedDoc.length; i++) {
+				expected[i] = Arrays.asList(expectedDoc[i]);
+			}
+            TestProcessor.helpProcess(plan, planningContext, dataMgr, expected);
+            assertNull("Expected failure", expectedException);
             return plan;
-        } 
-        Exception expected = null;
-        AnalysisRecord analysisRecord = new AnalysisRecord(false, DEBUG);                                              
-        try{
-            ProcessorPlan plan = QueryOptimizer.optimizePlan(command, metadata, null, new DefaultCapabilitiesFinder(), analysisRecord, null);
-    
-            BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-            CommandContext context = new CommandContext("pID", null, null, null, 1);                                                                 //$NON-NLS-1$
-            QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
-            processor.setNonBlocking(true);
-            BatchCollector collector = processor.createBatchCollector();
-            collector.collectTuples();
-        } catch (Exception e){
-            if (expectedException.isInstance(e)){
-                expected = e;
-            } else {
-                throw e;
-            }
+        } catch (Exception e) {
+        	if (expectedException == null) {
+        		throw e;
+        	}
+        	assertTrue(expectedException.isInstance(e));
         } finally {
             if(DEBUG) {
                 System.out.println(analysisRecord.getDebugLog());
             }                
         }
-        
-        assertNotNull(shouldFailMsg, expected);
         return null;
     }
 
-	public static void compareDocuments(String expectedDoc, String actualDoc) {
-		StringTokenizer tokens1 = new StringTokenizer(expectedDoc, "\r\n"); //$NON-NLS-1$
-		StringTokenizer tokens2 = new StringTokenizer(actualDoc, "\n");//$NON-NLS-1$
-		while(tokens1.hasMoreTokens()){
-			String token1 = tokens1.nextToken().trim();
-			if(!tokens2.hasMoreTokens()){
-				fail("XML doc mismatch: expected=" + token1 + "\nactual=none");//$NON-NLS-1$ //$NON-NLS-2$
-			}
-			String token2 = tokens2.nextToken().trim();
-			assertEquals("XML doc mismatch: ", token1, token2); //$NON-NLS-1$
-		}
-		if(tokens2.hasMoreTokens()){
-			fail("XML doc mismatch: expected=none\nactual=" + tokens2.nextToken().trim());//$NON-NLS-1$
-		}
-	}
-
-	private void helpTestProcess(String sql, String[] expectedDocs, FakeMetadataFacade metadata, FakeDataManager dataMgr) throws Exception{
-        Command command = helpGetCommand(sql, metadata);
-        AnalysisRecord analysisRecord = new AnalysisRecord(false, DEBUG);                                              
-        XMLPlan plan = (XMLPlan)QueryOptimizer.optimizePlan(command, metadata, null, new DefaultCapabilitiesFinder(), analysisRecord, null);
-        if(DEBUG) {
-            System.out.println(analysisRecord.getDebugLog());
-        }
-        
-        helpTestProcess(expectedDocs, dataMgr, plan);
-    }
-
-	private void helpTestProcess(String[] expectedDocs, FakeDataManager dataMgr,
-			ProcessorPlan plan) throws TeiidComponentException,
-			TeiidProcessingException, SQLException {
-		BufferManager bufferMgr = BufferManagerFactory.getStandaloneBufferManager();
-        CommandContext context = new CommandContext("pID", null, null, null, 1);                                 //$NON-NLS-1$
-        context.setProcessDebug(DEBUG);
-        QueryProcessor processor = new QueryProcessor(plan, context, bufferMgr, dataMgr);
-        processor.setNonBlocking(true);
-        BatchCollector collector = processor.createBatchCollector();
-        TupleBuffer tupleBuffer = collector.collectTuples();
-        int count = tupleBuffer.getRowCount();
-        assertEquals("Incorrect number of records: ", expectedDocs.length, count); //$NON-NLS-1$
-        
-        TupleSource ts = tupleBuffer.createIndexedTupleSource();
-        for (int i=0; i<expectedDocs.length; i++){        
-            List row = ts.nextTuple();
-            if(row.isEmpty()){
-            	continue;
-            }
-            assertEquals("Incorrect number of columns: ", 1, row.size()); //$NON-NLS-1$
-            XMLType result = (XMLType)row.get(0);
-            String actualDoc = result.getString();
-                 
-            //assertEquals("XML doc result # " + i +" mismatch: ", expectedDocs[i], actualDoc); //$NON-NLS-1$ //$NON-NLS-2$
-            compareDocuments(expectedDocs[i], actualDoc);
-        }
-        tupleBuffer.remove();
-	}    
-
     // =============================================================================================
     // T E S T S 
     // =============================================================================================
@@ -3946,7 +3842,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManager(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc1 WHERE Catalog = 'something'", null, metadata, dataMgr, false, QueryPlannerException.class, null);         //$NON-NLS-1$
+        helpTestProcess("SELECT * FROM xmltest.doc1 WHERE Catalog = 'something'", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$
     }    
 
     //defect 8130
@@ -3954,7 +3850,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManager(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc1 WHERE Item = 'something'", null, metadata, dataMgr, false, QueryPlannerException.class, null);         //$NON-NLS-1$
+        helpTestProcess("SELECT * FROM xmltest.doc1 WHERE Item = 'something'", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$
     }  
     
     @Test public void testNested() throws Exception {
@@ -5141,7 +5037,7 @@
         Class expectedException = QueryPlannerException.class;
         String shouldFailMsg = "expected failure since two different contexts were specified in conjunct"; //$NON-NLS-1$
 
-        helpTestProcess("SELECT * FROM xmltest.doc9 WHERE context(Item, OrderID)='5' OR context(SupplierID, OrderID)='2'", expectedDoc, metadata, dataMgr, shouldSucceed, expectedException, shouldFailMsg);         //$NON-NLS-1$
+        helpTestProcess("SELECT * FROM xmltest.doc9 WHERE context(Item, OrderID)='5' OR context(SupplierID, OrderID)='2'", expectedDoc, metadata, dataMgr, expectedException);         //$NON-NLS-1$
     }
 
     @Test public void testNested2WithContextCriteria6() throws Exception {
@@ -5843,7 +5739,7 @@
     @Test public void test2b() throws Exception {
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManager(metadata);
-        helpTestProcess("SELECT * FROM xmltest.doc2b", null, metadata, dataMgr, false, TeiidComponentException.class, "Should have failed on default");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc2b", null, metadata, dataMgr, TeiidComponentException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }
 
     @Test public void test2c() throws Exception {
@@ -6159,9 +6055,7 @@
             "</Item>\r\n\r\n"; //$NON-NLS-1$
 
 
-        String[] expectedDocs = new String[]{expectedDoc1, expectedDoc2, expectedDoc3};
-
-        helpTestProcess("SELECT * FROM xmltest.doc11", expectedDocs, metadata, dataMgr);         //$NON-NLS-1$
+        helpTestProcess("SELECT * FROM xmltest.doc11", metadata, dataMgr, null, new DefaultCapabilitiesFinder(), expectedDoc1, expectedDoc2, expectedDoc3);         //$NON-NLS-1$
     }
 
     @Test public void testRecursive() throws Exception {
@@ -6474,7 +6368,7 @@
     @Test public void testRecursive4Exception() throws Exception {
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
-        helpTestProcess("SELECT * FROM xmltest.doc15", null, metadata, dataMgr, false, TeiidComponentException.class, "Query processing should have failed on recursion limit."); //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc15", null, metadata, dataMgr, TeiidComponentException.class); //$NON-NLS-1$ //$NON-NLS-2$
     }
 
     /**
@@ -9896,7 +9790,7 @@
         
         final boolean SHOULD_SUCCEED = true;
         helpTestProcess("SELECT Item FROM xmltest.doc9c", //$NON-NLS-1$
-            expectedDoc, metadata, dataMgr, SHOULD_SUCCEED, null, null);       
+            expectedDoc, metadata, dataMgr, null);       
     }
 
     @Test public void testNestedWithStoredQueryInMappingClass() throws Exception {
@@ -10061,7 +9955,7 @@
         caps.setFunctionSupport("convert", true); //$NON-NLS-1$
         CapabilitiesFinder capFinder = new DefaultCapabilitiesFinder(caps); 
         
-        helpTestProcess("SELECT * FROM xmltest.doc12260", expectedDoc, metadata, dataMgr, true, TeiidComponentException.class, null, capFinder); //$NON-NLS-1$
+        helpTestProcess("SELECT * FROM xmltest.doc12260", metadata, dataMgr, null, capFinder, expectedDoc); //$NON-NLS-1$
     }
     
     @Test public void testDefect8373() throws Exception{
@@ -10617,7 +10511,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2", null, metadata, dataMgr, false, TeiidProcessingException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2", null, metadata, dataMgr, TeiidProcessingException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }      
     
     /** Two row limits on the same mapping class should be harmless as long as the row limits are identical. */
@@ -10681,7 +10575,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2 AND rowlimitexception(supplierid) = 2", null, metadata, dataMgr, false, TeiidProcessingException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2 AND rowlimitexception(supplierid) = 2", null, metadata, dataMgr, TeiidProcessingException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }      
     
     /** compound criteria */
@@ -10721,7 +10615,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE ItemID='002' AND rowlimitexception(supplier) = 2", null, metadata, dataMgr, false, TeiidProcessingException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE ItemID='002' AND rowlimitexception(supplier) = 2", null, metadata, dataMgr, TeiidProcessingException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }     
     
     @Test public void testCase2951MaxRows4() throws Exception {
@@ -10828,7 +10722,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(catalogs) = 2", null, metadata, dataMgr, false, QueryPlannerException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(catalogs) = 2", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }     
 
     /** two conflicting row limits on the same mapping class */
@@ -10837,7 +10731,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(supplier) = 2 AND rowlimit(supplierID) = 3", null, metadata, dataMgr, false, QueryPlannerException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(supplier) = 2 AND rowlimit(supplierID) = 3", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }
     
     /** arg to rowlimitexception function isn't in the scope of any mapping class */
@@ -10846,7 +10740,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(catalogs) = 2", null, metadata, dataMgr, false, QueryPlannerException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(catalogs) = 2", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }     
 
     /** two conflicting rowlimitexceptions on the same mapping class */
@@ -10855,7 +10749,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2 AND rowlimitexception(supplierID) = 3", null, metadata, dataMgr, false, QueryPlannerException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimitexception(supplier) = 2 AND rowlimitexception(supplierID) = 3", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }    
     
     /** two conflicting rowlimit and rowlimitexceptions on the same mapping class fails planning */
@@ -10864,7 +10758,7 @@
         FakeMetadataFacade metadata = exampleMetadataCached();
         FakeDataManager dataMgr = exampleDataManagerNested(metadata);
         
-        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(supplier) = 2 AND rowlimitexception(supplierID) = 3", null, metadata, dataMgr, false, QueryPlannerException.class, "");         //$NON-NLS-1$ //$NON-NLS-2$
+        helpTestProcess("SELECT * FROM xmltest.doc8 WHERE rowlimit(supplier) = 2 AND rowlimitexception(supplierID) = 3", null, metadata, dataMgr, QueryPlannerException.class);         //$NON-NLS-1$ //$NON-NLS-2$
     }    
 
     /** try rowlimit criteria written the reverse way */

Modified: trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFacade.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFacade.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFacade.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -774,4 +774,11 @@
     public FunctionLibrary getFunctionLibrary() {
     	return this.functionLibrary;
     }
+    
+    @Override
+    public boolean isPrimaryKey(Object metadataID) {
+    	ArgCheck.isInstanceOf(FakeMetadataObject.class, metadataID);
+    	FakeMetadataObject object =  (FakeMetadataObject) metadataID;
+    	return object.getProperty(FakeMetadataObject.Props.KEY_TYPE) == FakeMetadataObject.TYPE_PRIMARY_KEY;
+    }
 }

Modified: trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFactory.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFactory.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/engine/src/test/java/org/teiid/query/unittest/FakeMetadataFactory.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -2127,12 +2127,18 @@
         virtGroup1.putProperty(FakeMetadataObject.Props.MAT_GROUP, physGroup1);
         virtGroup1.putProperty(FakeMetadataObject.Props.MAT_STAGE, physGroupStage1);
 
-        //add one virtual group that uses the materialized group in transformation with NOCACHE option
         QueryNode vTrans2 = new QueryNode("MatView.VGroup2", "/* cache */ SELECT x FROM matsrc");         //$NON-NLS-1$ //$NON-NLS-2$
         FakeMetadataObject vGroup2 = createVirtualGroup("MatView.VGroup2", virtModel, vTrans2); //$NON-NLS-1$
         List vElements2 = createElements(vGroup2,
                                       new String[] { "x" }, //$NON-NLS-1$
                                       new String[] { DataTypeManager.DefaultDataTypes.STRING});
+        
+        QueryNode vTrans3 = new QueryNode("MatView.VGroup3", "/* cache */ SELECT x, 'z' || substring(x, 2) as y FROM matsrc");         //$NON-NLS-1$ //$NON-NLS-2$
+        FakeMetadataObject vGroup3 = createVirtualGroup("MatView.VGroup3", virtModel, vTrans3); //$NON-NLS-1$
+        List vElements3 = createElements(vGroup3,
+                                      new String[] { "x", "y" }, //$NON-NLS-1$
+                                      new String[] { DataTypeManager.DefaultDataTypes.STRING, DataTypeManager.DefaultDataTypes.STRING});
+        FakeMetadataObject vGroup3pk = createKey("pk", vGroup3, vElements3.subList(0, 1));
 
         
         FakeMetadataStore store = new FakeMetadataStore();
@@ -2157,6 +2163,9 @@
         store.addObjects(virtElements1);
         store.addObject(vGroup2);
         store.addObjects(vElements2);
+        store.addObject(vGroup3);
+        store.addObjects(vElements3);
+        store.addObject(vGroup3pk);
         return new FakeMetadataFacade(store);
     }
     

Modified: trunk/test-integration/common/src/test/java/org/teiid/dqp/internal/process/BaseQueryTest.java
===================================================================
--- trunk/test-integration/common/src/test/java/org/teiid/dqp/internal/process/BaseQueryTest.java	2010-07-31 22:31:31 UTC (rev 2395)
+++ trunk/test-integration/common/src/test/java/org/teiid/dqp/internal/process/BaseQueryTest.java	2010-08-02 15:33:33 UTC (rev 2396)
@@ -56,7 +56,6 @@
         
     protected void doProcess(QueryMetadataInterface metadata, String sql, CapabilitiesFinder capFinder, ProcessorDataManager dataManager, List[] expectedResults, boolean debug) throws Exception {
     	CommandContext context = createCommandContext();
-    	context.setProcessDebug(debug);
         Command command = TestOptimizer.helpGetCommand(sql, metadata, null);
 
         // plan



More information about the teiid-commits mailing list