[teiid-commits] teiid SVN: r2083 - in trunk: engine/src/main/java/com/metamatrix/query/processor/xml and 2 other directories.

teiid-commits at lists.jboss.org teiid-commits at lists.jboss.org
Tue Apr 27 13:56:41 EDT 2010


Author: shawkins
Date: 2010-04-27 13:56:40 -0400 (Tue, 27 Apr 2010)
New Revision: 2083

Removed:
   trunk/connectors/connector-jdbc/src/main/java/org/teiid/connector/jdbc/XAJDBCPropertyNames.java
Modified:
   trunk/engine/src/main/java/com/metamatrix/query/processor/xml/DocumentInProgress.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/xml/InitializeDocumentInstruction.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/xml/SAXDocumentInProgress.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLPlan.java
   trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLProcessorEnvironment.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/FakeXMLProcessorEnvironment.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestAddNodeInstruction.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java
   trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestSAXDocumentInProgress.java
Log:
TEIID-913 cutting out intermediate buffering of results with direct writes to a filestore

Deleted: trunk/connectors/connector-jdbc/src/main/java/org/teiid/connector/jdbc/XAJDBCPropertyNames.java
===================================================================
--- trunk/connectors/connector-jdbc/src/main/java/org/teiid/connector/jdbc/XAJDBCPropertyNames.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/connectors/connector-jdbc/src/main/java/org/teiid/connector/jdbc/XAJDBCPropertyNames.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -1,43 +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.
- */
-
-/*
- * Date: Jan 23, 2003
- * Time: 5:38:40 PM
- */
-package org.teiid.connector.jdbc;
-
-/**
- * XAJDBCPropertyNames.
- */
-public class XAJDBCPropertyNames {
-
-    /** An XA Transaction-related property. The <code>XADataSource</code>'s unique
-     * resource name for the underlying <code>ConnectionPoolDataSource</code> object.
-     */
-    public static final String DATASOURCE_NAME = "dataSourceName"; //$NON-NLS-1$
-
-    public static final String PORT_NUMBER = "portNumber";//$NON-NLS-1$
-
-    public static final String SERVER_NAME = "serverName";//$NON-NLS-1$
-
-}

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/xml/DocumentInProgress.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/xml/DocumentInProgress.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/xml/DocumentInProgress.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -25,6 +25,7 @@
 import org.xml.sax.SAXException;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
+import com.metamatrix.common.buffer.FileStore;
 
 /**
  * <p>This represents a document in construction.  It maintains a reference 
@@ -110,13 +111,6 @@
      */
     public void markAsFinished() throws MetaMatrixComponentException;
     
-    /**
-     * Retrieve the next chunk of document.
-     * @param sizeInBytes size of the chunk in bytes. No limit if it is 0.
-     * @return character array containing the specfied number of characters, or less if 
-     * it has reached the end of the document. Return null if there is not enough characters
-     * and it has not reached the end of the document. 
-     */	
-    public char[] getNextChunk(int sizeInBytes);
-		
+    public FileStore getFileStore();
+    
 }

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/xml/InitializeDocumentInstruction.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/xml/InitializeDocumentInstruction.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/xml/InitializeDocumentInstruction.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -84,7 +84,7 @@
         // program stack (don't want to start a new doc in the middle of 
         // recursive processing)
         if (!env.isRecursiveProgramInStack()) {
-            DocumentInProgress doc = new SAXDocumentInProgress();
+            DocumentInProgress doc = new SAXDocumentInProgress(env.getBufferManager().createFileStore("xml")); //$NON-NLS-1$
             //DocumentInProgress doc = new JDOMDocumentInProgress();
             env.setDocumentInProgress(doc);
                 

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/xml/SAXDocumentInProgress.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/xml/SAXDocumentInProgress.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/xml/SAXDocumentInProgress.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -22,7 +22,7 @@
 
 package com.metamatrix.query.processor.xml;
 
-import java.io.CharArrayWriter;
+import java.io.BufferedOutputStream;
 import java.util.Iterator;
 
 import javax.xml.transform.OutputKeys;
@@ -36,6 +36,7 @@
 import org.xml.sax.SAXException;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
+import com.metamatrix.common.buffer.FileStore;
 import com.metamatrix.common.log.LogManager;
 import com.metamatrix.common.log.MessageLevel;
 import com.metamatrix.query.mapping.xml.MappingNodeConstants;
@@ -52,17 +53,17 @@
     private boolean finished;
     private String documentEncoding = MappingNodeConstants.Defaults.DEFAULT_DOCUMENT_ENCODING;
     private boolean isFormatted = MappingNodeConstants.Defaults.DEFAULT_FORMATTED_DOCUMENT.booleanValue();
-    private CharArrayWriter streamResultHolder = new CharArrayWriter();
+    private FileStore store;
     
-    
-    public SAXDocumentInProgress() throws MetaMatrixComponentException{
+    public SAXDocumentInProgress(FileStore store) throws MetaMatrixComponentException{
+        this.store = store;
         SAXTransformerFactory factory = new TransformerFactoryImpl();
     	//TODO use standard jaxp to create factory 
         //SAXTransformerFactory factory = (SAXTransformerFactory)TransformerFactory.newInstance();
         try {
 			//SAX2.0 ContentHandler
 			handler = factory.newTransformerHandler();
-			handler.setResult(new StreamResult(streamResultHolder));
+			handler.setResult(new StreamResult(new BufferedOutputStream(store.createOutputStream())));
 		} catch (Exception e) {
 			throw new MetaMatrixComponentException(e);
 		}
@@ -269,27 +270,6 @@
 		finished = true;
 	}
 
-	/**
-	 * @see com.metamatrix.query.processor.xml.DocumentInProgress#getNextChunk(int)
-	 */
-	public char[] getNextChunk(int sizeInBytes) {
-        if(sizeInBytes == 0 && !finished) {
-            return null;
-        }
-        
-		//the unit of the size is Kb
-		if(finished || streamResultHolder.size() >= sizeInBytes){
-			char[] chunk = streamResultHolder.toCharArray();
-			if(finished){
-				streamResultHolder.close();
-			}else{
-				streamResultHolder.reset();
-			}
-			return chunk;
-		}
-		return null;
-	}
-	
 	private Element makeElement(NodeDescriptor descripter) {
         showState( "makeElement - TOP" );   //$NON-NLS-1$
         Element element = new Element(descripter, handler);
@@ -359,5 +339,10 @@
             }
         }
     }
+    
+    @Override
+    public FileStore getFileStore() {
+    	return this.store;
+    }
 
 }

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLPlan.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLPlan.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLPlan.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -28,7 +28,6 @@
 import java.io.InputStream;
 import java.io.Reader;
 import java.io.StringReader;
-import java.nio.charset.Charset;
 import java.sql.SQLException;
 import java.sql.SQLXML;
 import java.util.ArrayList;
@@ -43,7 +42,6 @@
 import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 
-import org.teiid.client.lob.LobChunk;
 import org.teiid.client.plan.PlanNode;
 import org.xml.sax.Attributes;
 import org.xml.sax.EntityResolver;
@@ -59,15 +57,11 @@
 import com.metamatrix.api.exception.MetaMatrixProcessingException;
 import com.metamatrix.common.buffer.BlockedException;
 import com.metamatrix.common.buffer.BufferManager;
-import com.metamatrix.common.buffer.FileStore;
 import com.metamatrix.common.buffer.TupleBatch;
 import com.metamatrix.common.log.LogConstants;
 import com.metamatrix.common.log.LogManager;
 import com.metamatrix.common.types.DataTypeManager;
-import com.metamatrix.common.types.SQLXMLImpl;
-import com.metamatrix.common.types.Streamable;
 import com.metamatrix.common.types.XMLType;
-import com.metamatrix.core.util.Assertion;
 import com.metamatrix.query.analysis.AnalysisRecord;
 import com.metamatrix.query.execution.QueryExecPlugin;
 import com.metamatrix.query.processor.ProcessorDataManager;
@@ -95,14 +89,9 @@
 	// State initialized by processor
 	private ProcessorDataManager dataMgr;
     private BufferManager bufferMgr;
-    private int chunkSize = Streamable.STREAMING_BATCH_SIZE_IN_BYTES;
 
     private int nextBatchCount = 1;
         
-    // is document in progress currently?
-    boolean docInProgress = false;
-    private FileStore docInProgressStore;
-    
     // Post-processing
 	private Collection<SQLXML> xmlSchemas;
 
@@ -119,9 +108,6 @@
      */
     public void initialize(CommandContext context, ProcessorDataManager dataMgr, BufferManager bufferMgr) {
         setContext(context);
-        if(context.getStreamingBatchSize() != 0){
-        	this.chunkSize = context.getStreamingBatchSize();
-        }
         TempTableStore tempTableStore = new TempTableStoreImpl(bufferMgr, context.getConnectionID(), (TempTableStore)context.getTempTableStore());
         //this.dataMgr = new StagingTableDataManager(new TempTableDataManager(dataMgr, tempTableStore), env);
         this.dataMgr = new TempTableDataManager(dataMgr, tempTableStore);
@@ -165,137 +151,48 @@
         throws MetaMatrixComponentException, MetaMatrixProcessingException, BlockedException {
         
         while(true){
-            LobChunk chunk = getNextXMLChunk(this.chunkSize);  
-            
-            if (chunk == null) {
-            	Assertion.assertTrue(!this.docInProgress);
-            	TupleBatch batch = new TupleBatch(nextBatchCount++, Collections.EMPTY_LIST); 
-	        	batch.setTerminationFlag(true);
-	        	return batch;
-            }
-	                    
-            XMLType doc = processXML(chunk);
-            
-            if (doc != null) {
-    	        TupleBatch batch = new TupleBatch(nextBatchCount++, Arrays.asList(Arrays.asList(doc)));
-    	        return batch;
-            }
-        }
-    }
-    
-    /**
-     * <p>Process the XML, using the stack of Programs supplied by the
-     * ProcessorEnvironment.  With each pass through the loop, the
-     * current Program is gotten off the top of the stack, and the
-     * current instruction is gotten from that program; each call
-     * to an instruction's process method may alter the Program
-     * Stack and/or the current instruction pointer of a Program,
-     * so it's important that this method's loop refer to the
-     * call stack of the ProcessorEnvironment each time, and not
-     * cache things in local variables.  If the current Program's
-     * current instruction is null, then it's time to pop that
-     * Program off the stack.</p>
-     *
-     * <p>This method will return a single tuple (List) for
-     * each XML document chunk in the result sets. There may be
-     * many XML documents, if the root of the document model has
-     * a mapping class (i.e. result set) associated with it.</p>
-     *
-     */
-    private XMLType processXML(LobChunk chunk)
-        throws MetaMatrixComponentException, BlockedException {
+        	// do the xml processing.
+            ProcessorInstruction inst = env.getCurrentInstruction();
+            while (inst != null){
+            	LogManager.logTrace(LogConstants.CTX_XML_PLAN, "Executing instruction", inst); //$NON-NLS-1$
+                this.context = inst.process(this.env, this.context);
 
-        // Note that we need to stream the document right away, as multiple documents
-        // results are generated from the same "env" object. So the trick here is either
-        // post process will stream it, or save to buffer manager to stream the document
-        // right away so that we give way to next batch call. (this due to bad design of xml model)
-        // also note that there could be "inst" object alive but no more chunks; that is reason
-        // for "if" null block
-        XMLType xml = null;            
-        
-        // if the chunk size is less than one chunk unit then send this as string based xml.
-        if (!this.docInProgress && chunk.isLast()) {
-            xml = new XMLType(new SQLXMLImpl(chunk.getBytes()));
-        }
-        else {
-            
-            // if this is the first chunk, then create a tuple source id for this sequence of chunks
-            if (!this.docInProgress) {
-                this.docInProgress = true;
-                this.docInProgressStore = this.bufferMgr.createFileStore("xml"); //$NON-NLS-1$
+                //code to check for end of document, set current doc
+                //to null, and return the finished doc as a single tuple
+                DocumentInProgress doc = env.getDocumentInProgress();
+                if (doc != null && doc.isFinished()) {
+                    this.env.setDocumentInProgress(null);
+                    XMLType xml = new XMLType(XMLUtil.createSQLXML(doc.getFileStore()));
+                    // check to see if we need to do any post validation on the document.
+                    if (getContext().validateXML()){
+                    	Reader reader;
+            			try {
+            				reader = xml.getCharacterStream();
+            			} catch (SQLException e) {
+            				throw new MetaMatrixComponentException(e);
+            			}
+                    	try {
+                    		validateDoc(reader);
+                    	} finally {
+                    		try {
+            					reader.close();
+            				} catch (IOException e) {
+            				}
+                    	}
+                    }
+        	        TupleBatch batch = new TupleBatch(nextBatchCount++, Arrays.asList(Arrays.asList(xml)));
+        	        return batch;
+                }
+                inst = env.getCurrentInstruction();
             }
             
-            // now save the chunk of data to the buffer manager and move on.
-            this.docInProgressStore.write(chunk.getBytes());
-            // now document is finished, so create a xml object and return to the client.
-            if (chunk.isLast()) {
-                // we want this to be naturally feed by chunks whether inside
-                // or out side the processor
-                xml = new XMLType(XMLUtil.createSQLXML(this.docInProgressStore));
-                //reset current document state.
-                this.docInProgress = false;
-                this.docInProgressStore = null;
-            } else {
-            	return null; //need to continue processing
-            }
+        	TupleBatch batch = new TupleBatch(nextBatchCount++, Collections.EMPTY_LIST); 
+        	batch.setTerminationFlag(true);
+        	return batch;
         }
-
-        // check to see if we need to do any post validation on the document.
-        if (getContext().validateXML()){
-        	Reader reader;
-			try {
-				reader = xml.getCharacterStream();
-			} catch (SQLException e) {
-				throw new MetaMatrixComponentException(e);
-			}
-        	try {
-        		validateDoc(reader);
-        	} finally {
-        		try {
-					reader.close();
-				} catch (IOException e) {
-				}
-        	}
-        }
-                                
-        return xml;
     }
-        
     
     /**
-     * This methods gets the next XML chunk object from the document in progree object. Thi used by the 
-     * DocInProgressXMLTranslator to tuen this into a reader. 
-     * @return char[] of data of given size; if less than size is returned, it will be treated as
-     * document is finished or null if no chunk is available, just like blocked exception.
-     */
-    LobChunk getNextXMLChunk(int size) throws MetaMatrixComponentException, MetaMatrixProcessingException, BlockedException {
-
-        // do the xml processing.
-        ProcessorInstruction inst = env.getCurrentInstruction();
-        while (inst != null){
-        	LogManager.logTrace(LogConstants.CTX_XML_PLAN, "Executing instruction", inst); //$NON-NLS-1$
-            this.context = inst.process(this.env, this.context);
-
-            //code to check for end of document, set current doc
-            //to null, and return the finished doc as a single tuple
-            DocumentInProgress doc = env.getDocumentInProgress();
-            if (doc != null) {            
-                //chunk size 0 mean no limit; get the whole document
-                char[] chunk = doc.getNextChunk(size);
-                if (chunk != null) {
-                    if (doc.isFinished()) {
-                        this.env.setDocumentInProgress(null);
-                    }
-                    byte[] bytes = new String(chunk).getBytes(Charset.forName(Streamable.ENCODING));
-                    return new LobChunk(bytes, doc.isFinished()); 
-                }                
-            }
-            inst = env.getCurrentInstruction();
-        }
-        return null;
-    }
-        
-    /**
      * Sets the XML schema
      * @param xmlSchema
      */
@@ -518,19 +415,19 @@
 	 * any errors that occur during XML processing
 	 */
 	private static class MMErrorHandler implements ErrorHandler{
-		ArrayList exceptionList = null;
+		ArrayList<MetaMatrixException> exceptionList = null;
 		
 		/**
 		 * Keep track of all the exceptions
 		 */
 		private void addException(MetaMatrixException me) {
 		    if (exceptionList == null) {
-		        exceptionList = new ArrayList();
+		        exceptionList = new ArrayList<MetaMatrixException>();
 		    }
 		    exceptionList.add(me);
 		}
 		
-		public List getExceptionList() {
+		public List<MetaMatrixException> getExceptionList() {
 		    return exceptionList;
 		}
 		

Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLProcessorEnvironment.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLProcessorEnvironment.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/main/java/com/metamatrix/query/processor/xml/XMLProcessorEnvironment.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -23,10 +23,10 @@
 package com.metamatrix.query.processor.xml;
 
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
-import java.util.Map;
+import java.util.Set;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
 import com.metamatrix.common.buffer.BufferManager;
@@ -48,9 +48,9 @@
     public static final String JDOM_DOCUMENT_RESULT = "JDOM Document"; //$NON-NLS-1$
     
     /* Stack <ProgramState> */
-    private LinkedList programStack = new LinkedList();
+    private LinkedList<ProgramState> programStack = new LinkedList<ProgramState>();
     
-    private Map loadedStagingTables = Collections.synchronizedMap(new HashMap());
+    private Set<String> loadedStagingTables = Collections.synchronizedSet(new HashSet<String>());
 
     private DocumentInProgress documentInProgress;
     
@@ -101,7 +101,7 @@
     public Program getCurrentProgram() {
         // jh case 5266
         if ( this.programStack.size() > 0 ) {
-            ProgramState programState = (ProgramState)this.programStack.getFirst();
+            ProgramState programState = this.programStack.getFirst();
             return programState.program;
         }
         return null;
@@ -127,7 +127,7 @@
      * @see com.metamatrix.query.processor.xml.ProcessorEnvironment#incrementCurrentProgramCounter()
      */
     public void incrementCurrentProgramCounter() {
-        ProgramState programState = (ProgramState)this.programStack.getFirst();
+        ProgramState programState = this.programStack.getFirst();
         programState.programCounter++;
         
         // Always leave one Program in the Program stack, even if it is finished
@@ -137,7 +137,7 @@
             if(LogManager.isMessageToBeRecorded(com.metamatrix.common.log.LogConstants.CTX_XML_PLAN, MessageLevel.TRACE)) {
                 LogManager.logTrace(com.metamatrix.common.log.LogConstants.CTX_XML_PLAN, new Object[]{"Processor Environment popped program w/ recursion count " + programState.recursionCount, "; " + this.programStack.size(), " programs left."}); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
             }
-            programState = (ProgramState)this.programStack.getFirst();
+            programState = this.programStack.getFirst();
         }        
     }
     
@@ -171,14 +171,14 @@
      * @see com.metamatrix.query.processor.xml.ProcessorEnvironment#getCurrentInstruction()
      */
     public ProcessorInstruction getCurrentInstruction() {
-        ProgramState programState = (ProgramState)this.programStack.getFirst();
+        ProgramState programState = this.programStack.getFirst();
         
         //Case 5266: account for an empty program on to the stack; 
         //this is needed to handle an empty sequence or an excluded Choice properly.
         if (programState != null && programState.program.getProcessorInstructions().isEmpty()) {
             incrementCurrentProgramCounter();
             
-            programState = (ProgramState)this.programStack.getFirst();
+            programState = this.programStack.getFirst();
         }
         
         if (programState == null) {
@@ -202,9 +202,9 @@
     
     private ProgramState getProgramState(Program program) {
         ProgramState result = null;
-        Iterator stackedPrograms = this.programStack.iterator();
+        Iterator<ProgramState> stackedPrograms = this.programStack.iterator();
         while (stackedPrograms.hasNext()) {
-            ProgramState programState = (ProgramState) stackedPrograms.next();
+            ProgramState programState = stackedPrograms.next();
             Program stackedProgram = programState.program;
             if (stackedProgram == program) {
                 result = programState;
@@ -287,7 +287,7 @@
      */
     protected void copyIntoClone(XMLProcessorEnvironment clone) {
         // Programs - just get the one at the bottom of the stack
-        ProgramState initialProgramState = (ProgramState)this.programStack.getLast();
+        ProgramState initialProgramState = this.programStack.getLast();
         ProgramState newState = new ProgramState();
         newState.program = initialProgramState.program;
         newState.programCounter = 0;
@@ -308,11 +308,15 @@
     
     
     boolean isStagingTableLoaded(String tableName) {
-        return Boolean.TRUE.equals(this.loadedStagingTables.get(tableName));
+        return this.loadedStagingTables.contains(tableName);
     }
     
     void markStagingTableAsLoaded(String tableName) {
-        this.loadedStagingTables.put(tableName, Boolean.TRUE);
+        this.loadedStagingTables.add(tableName);
     }
     
+    public BufferManager getBufferManager() {
+		return bufferMgr;
+	}
+    
 }

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/TestProcessor.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -284,6 +284,10 @@
                     record = new ArrayList(record);
                     record.set(0, actualDoc);
             	}
+            	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$

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/xml/FakeXMLProcessorEnvironment.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/xml/FakeXMLProcessorEnvironment.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/FakeXMLProcessorEnvironment.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -27,6 +27,8 @@
 import java.util.Map;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
+import com.metamatrix.common.buffer.BufferManager;
+import com.metamatrix.common.buffer.BufferManagerFactory;
 import com.metamatrix.query.mapping.xml.ResultSetInfo;
 
 /**
@@ -57,4 +59,9 @@
         clone.schemaMap = this.schemaMap;
         return clone;
     }
+    
+    @Override
+    public BufferManager getBufferManager() {
+    	return BufferManagerFactory.getStandaloneBufferManager();
+    }
 }

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestAddNodeInstruction.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestAddNodeInstruction.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestAddNodeInstruction.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -27,6 +27,9 @@
 import junit.framework.TestCase;
 
 import com.metamatrix.api.exception.MetaMatrixComponentException;
+import com.metamatrix.common.buffer.BufferManagerFactory;
+import com.metamatrix.common.buffer.FileStore;
+import com.metamatrix.core.util.ObjectConverterUtil;
 import com.metamatrix.query.mapping.xml.MappingNodeConstants;
 import com.metamatrix.query.sql.symbol.ElementSymbol;
 import com.metamatrix.query.sql.util.VariableContext;
@@ -405,7 +408,8 @@
         
 		env.pushProgram(program);
         
-        DocumentInProgress doc = new SAXDocumentInProgress();
+		FileStore fs = BufferManagerFactory.getStandaloneBufferManager().createFileStore("test"); //$NON-NLS-1$
+        DocumentInProgress doc = new SAXDocumentInProgress(fs);
         env.setDocumentInProgress(doc);
         
         //add fake root, move to child
@@ -419,7 +423,7 @@
         doc.moveToParent();
         doc.markAsFinished();
         
-        String actualDoc = new String(env.getDocumentInProgress().getNextChunk(1));
+        String actualDoc = new String(ObjectConverterUtil.convertToByteArray(fs.createInputStream(0)));
         return actualDoc;    
     }
 

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	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestInstructions.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -24,9 +24,7 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.StringTokenizer;
 
 import junit.framework.TestCase;
 
@@ -34,7 +32,6 @@
 import com.metamatrix.api.exception.query.QueryParserException;
 import com.metamatrix.api.exception.query.QueryResolverException;
 import com.metamatrix.api.exception.query.QueryValidatorException;
-import com.metamatrix.common.buffer.BlockedException;
 import com.metamatrix.common.buffer.BufferManager;
 import com.metamatrix.common.buffer.BufferManagerFactory;
 import com.metamatrix.common.types.DataTypeManager;
@@ -42,6 +39,8 @@
 import com.metamatrix.query.mapping.xml.MappingNodeConstants;
 import com.metamatrix.query.mapping.xml.ResultSetInfo;
 import com.metamatrix.query.parser.QueryParser;
+import com.metamatrix.query.processor.FakeDataManager;
+import com.metamatrix.query.processor.TestProcessor;
 import com.metamatrix.query.resolver.QueryResolver;
 import com.metamatrix.query.resolver.util.ResolverVisitor;
 import com.metamatrix.query.rewriter.QueryRewriter;
@@ -133,61 +132,10 @@
         return crit;        
     }
 
-	private 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$
-		}
-	}
-	
-    public List helpProcessInstructions(Program prog, XMLProcessorEnvironment env) throws Exception {
-        int counter = 0;
-        XMLContext context = new XMLContext();
+    public void helpProcessInstructions(Program prog, XMLProcessorEnvironment env, String expected) throws Exception {
         env.pushProgram(prog);
-
-        DocumentInProgress doc = null;
-        
-        LinkedList resultDocs = new LinkedList();
-        
-        ProcessorInstruction inst = env.getCurrentInstruction();
-        while (inst != null){
-
-            try {
-                
-                context = inst.process(env, context);
-
-                //code to check for end of document, start new one,
-                doc = env.getDocumentInProgress();
-                if (doc != null){
-                    if (doc.isFinished()){
-                        env.setDocumentInProgress(null);
-                        String docString = new String(doc.getNextChunk(10));
-                        resultDocs.addLast(docString);
-                    }
-                }                    
-                
-            } catch(BlockedException e) {
-                
-            }
-            
-            // Catch run away processes
-            if(counter++ > 100) {
-                break;
-            }
-
-            inst = env.getCurrentInstruction();
-        }
-        
-        return resultDocs;
+    	XMLPlan plan = new XMLPlan(env);
+    	TestProcessor.doProcess(plan, new FakeDataManager(), new List[] {Arrays.asList(expected)}, new CommandContext());
     }
     
     public Program exampleProgram(FakeMetadataFacade metadata, XMLProcessorEnvironment env) throws Exception{
@@ -361,33 +309,26 @@
                     Arrays.asList( new Object[] { "002", "Screwdriver", new Integer(100) } ),         //$NON-NLS-1$ //$NON-NLS-2$
                     Arrays.asList( new Object[] { "003", "Goat", new Integer(4) } )         //$NON-NLS-1$ //$NON-NLS-2$
                     } );            
-        List resultDocs = helpProcessInstructions(program, env);
-        
-        String actualDoc = (String)resultDocs.iterator().next();
-        
-        String expectedDoc = 
-            "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\r\n" +  //$NON-NLS-1$
-            "<Catalogs>\r\n" + //$NON-NLS-1$
-            "    <Catalog>\r\n" +  //$NON-NLS-1$
-            "        <Items>\r\n" +  //$NON-NLS-1$
-            "            <Item ItemID=\"001\">\r\n" +  //$NON-NLS-1$
-            "                <Name>Lamp</Name>\r\n" +  //$NON-NLS-1$
-            "                <Quantity>5</Quantity>\r\n" +  //$NON-NLS-1$
-            "            </Item>\r\n" +  //$NON-NLS-1$
-            "            <Item ItemID=\"002\">\r\n" +  //$NON-NLS-1$
-            "                <Name>Screwdriver</Name>\r\n" +  //$NON-NLS-1$
-            "                <Quantity>100</Quantity>\r\n" +  //$NON-NLS-1$
-            "            </Item>\r\n" +  //$NON-NLS-1$
-            "            <Item ItemID=\"003\">\r\n" +  //$NON-NLS-1$
-            "                <Name>Goat</Name>\r\n" +  //$NON-NLS-1$
-            "                <Quantity>4</Quantity>\r\n" +  //$NON-NLS-1$
-            "            </Item>\r\n" +  //$NON-NLS-1$
-            "        </Items>\r\n" +  //$NON-NLS-1$
-            "    </Catalog>\r\n" +  //$NON-NLS-1$
-            "</Catalogs>\r\n\r\n"; //$NON-NLS-1$
-        
-        //assertEquals("XML doc mismatch: ", expectedDoc, actualDoc); //$NON-NLS-1$
-        compareDocuments(expectedDoc, actualDoc);
+        helpProcessInstructions(program, env,  
+            "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n" +  //$NON-NLS-1$
+            "<Catalogs>\n" + //$NON-NLS-1$
+            "   <Catalog>\n" +  //$NON-NLS-1$
+            "      <Items>\n" +  //$NON-NLS-1$
+            "         <Item ItemID=\"001\">\n" +  //$NON-NLS-1$
+            "            <Name>Lamp</Name>\n" +  //$NON-NLS-1$
+            "            <Quantity>5</Quantity>\n" +  //$NON-NLS-1$
+            "         </Item>\n" +  //$NON-NLS-1$
+            "         <Item ItemID=\"002\">\n" +  //$NON-NLS-1$
+            "            <Name>Screwdriver</Name>\n" +  //$NON-NLS-1$
+            "            <Quantity>100</Quantity>\n" +  //$NON-NLS-1$
+            "         </Item>\n" +  //$NON-NLS-1$
+            "         <Item ItemID=\"003\">\n" +  //$NON-NLS-1$
+            "            <Name>Goat</Name>\n" +  //$NON-NLS-1$
+            "            <Quantity>4</Quantity>\n" +  //$NON-NLS-1$
+            "         </Item>\n" +  //$NON-NLS-1$
+            "      </Items>\n" +  //$NON-NLS-1$
+            "   </Catalog>\n" +  //$NON-NLS-1$
+            "</Catalogs>"); //$NON-NLS-1$
     }
     
     public void testProcess2() throws Exception {
@@ -412,24 +353,17 @@
                     Arrays.asList( new Object[] { "003", "Goat", new Integer(4) } )         //$NON-NLS-1$ //$NON-NLS-2$
                     } );            
 
-        List resultDocs = helpProcessInstructions(program, env);
-        
-        String actualDoc = (String)resultDocs.iterator().next();
-
-        String expectedDoc = 
-            "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\r\n" +  //$NON-NLS-1$
-            "<Catalogs>\r\n" + //$NON-NLS-1$
-            "    <Catalog>\r\n" +  //$NON-NLS-1$
-            "        <Items>\r\n" +  //$NON-NLS-1$
-            "            <Item ItemID=\"002\">\r\n" +  //$NON-NLS-1$
-            "                <Name>Screwdriver</Name>\r\n" +  //$NON-NLS-1$
-            "                <Quantity>100</Quantity>\r\n" +  //$NON-NLS-1$
-            "            </Item>\r\n" +  //$NON-NLS-1$
-            "        </Items>\r\n" +  //$NON-NLS-1$
-            "    </Catalog>\r\n" +  //$NON-NLS-1$
-            "</Catalogs>\r\n\r\n"; //$NON-NLS-1$
-        
-        //assertEquals("XML doc mismatch: ", expectedDoc, actualDoc); //$NON-NLS-1$
-        compareDocuments(expectedDoc, actualDoc);
+        helpProcessInstructions(program, env, 
+        	"<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n" +  //$NON-NLS-1$
+            "<Catalogs>\n" + //$NON-NLS-1$
+            "   <Catalog>\n" +  //$NON-NLS-1$
+            "      <Items>\n" +  //$NON-NLS-1$
+            "         <Item ItemID=\"002\">\n" +  //$NON-NLS-1$
+            "            <Name>Screwdriver</Name>\n" +  //$NON-NLS-1$
+            "            <Quantity>100</Quantity>\n" +  //$NON-NLS-1$
+            "         </Item>\n" +  //$NON-NLS-1$
+            "      </Items>\n" +  //$NON-NLS-1$
+            "   </Catalog>\n" +  //$NON-NLS-1$
+            "</Catalogs>"); //$NON-NLS-1$
     }
 }

Modified: trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestSAXDocumentInProgress.java
===================================================================
--- trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestSAXDocumentInProgress.java	2010-04-26 20:34:52 UTC (rev 2082)
+++ trunk/engine/src/test/java/com/metamatrix/query/processor/xml/TestSAXDocumentInProgress.java	2010-04-27 17:56:40 UTC (rev 2083)
@@ -24,6 +24,8 @@
 
 import junit.framework.TestCase;
 
+import com.metamatrix.common.buffer.BufferManagerFactory;
+import com.metamatrix.common.buffer.FileStore;
 import com.metamatrix.query.mapping.xml.MappingNodeConstants;
 
 public class TestSAXDocumentInProgress extends TestCase {
@@ -34,7 +36,8 @@
 	}
     
     public void testLargeDocument()throws Exception{
-    	SAXDocumentInProgress doc = new SAXDocumentInProgress();
+		FileStore fs = BufferManagerFactory.getStandaloneBufferManager().createFileStore("test"); //$NON-NLS-1$
+    	SAXDocumentInProgress doc = new SAXDocumentInProgress(fs);
     	//long startTime = System.currentTimeMillis();
     	doc.setDocumentFormat(true);
         NodeDescriptor descriptor = NodeDescriptor.createNodeDescriptor("Root", null, true, null, null, null,false, null, MappingNodeConstants.NORMALIZE_TEXT_PRESERVE);//$NON-NLS-1$
@@ -44,20 +47,9 @@
     	doc.addAttribute(descriptor, "test1");//$NON-NLS-1$ 
         descriptor = NodeDescriptor.createNodeDescriptor("a1", null, false, null, null, null,false, null, MappingNodeConstants.NORMALIZE_TEXT_PRESERVE);//$NON-NLS-1$
     	doc.addAttribute(descriptor, "test2");//$NON-NLS-1$ 
-    	int j=1;
-    	int i=0;
         descriptor = NodeDescriptor.createNodeDescriptor("Child", null, true, null, null, null,false, null, MappingNodeConstants.NORMALIZE_TEXT_PRESERVE);//$NON-NLS-1$ 
-    	while(true){
+    	for (int i = 0; i < 50; i++) {
     		doc.addElement(descriptor, "test content");//$NON-NLS-1$ 
-    		char[] chunk = doc.getNextChunk(100);
-    		if(chunk != null){
-                j++;
-    			//System.out.println("Got chunk " + (j++) + " length="+chunk.length);//$NON-NLS-1$ //$NON-NLS-2$
-    		}
-    		if(j==51){
-    			break;
-    		}
-    		i++;
     	}
     	doc.moveToParent();
     	doc.markAsFinished();



More information about the teiid-commits mailing list