You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@uima.apache.org by al...@apache.org on 2007/03/02 21:36:16 UTC

svn commit: r513937 - in /incubator/uima/uimaj/trunk/uimaj-core/src: main/java/org/apache/uima/analysis_engine/asb/impl/ main/java/org/apache/uima/flow/ test/java/org/apache/uima/analysis_engine/impl/ test/resources/TextAnalysisEngineImplTest/

Author: alally
Date: Fri Mar  2 12:36:14 2007
New Revision: 513937

URL: http://svn.apache.org/viewvc?view=rev&rev=513937
Log:
Added Flow.continueOnFailure method, which is called from the UIMA runtime.
Built-in Flows always return false so default behavior is unchanged.
UIMA-327: https://issues.apache.org/jira/browse/UIMA-327

Added:
    incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml
Modified:
    incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/ASB_impl.java
    incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/FlowContainer.java
    incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/CasFlow_ImplBase.java
    incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/Flow.java
    incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/JCasFlow_ImplBase.java
    incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/AnalysisEngine_implTest.java
    incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/FlowControllerForErrorTest.java
    incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateForErrorTest.xml
    incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/FlowControllerForErrorTest.xml

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/ASB_impl.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/ASB_impl.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/ASB_impl.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/ASB_impl.java Fri Mar  2 12:36:14 2007
@@ -19,7 +19,6 @@
 
 package org.apache.uima.analysis_engine.asb.impl;
 
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -492,19 +491,31 @@
               return null; // there are no more CAS Iterators to obtain CASes from
             }
             StackFrame frame = (StackFrame) casIteratorStack.peek();
-            if (frame.casIterator.hasNext()) {
-              cas = frame.casIterator.next();
-              // this is a new output CAS so we need to compute a flow for it
-              flow = frame.originalCasFlow.newCasProduced(cas, frame.casMultiplierAeKey);
-            } else {
+            try {
+              if (frame.casIterator.hasNext()) {
+                cas = frame.casIterator.next();
+                // this is a new output CAS so we need to compute a flow for it
+                flow = frame.originalCasFlow.newCasProduced(cas, frame.casMultiplierAeKey);
+              }
+            } 
+            catch(Exception e) {
+              //A CAS Multiplier (or possibly an aggregate) threw an exception trying to output the next CAS.
+              //We abandon trying to get further output CASes from that CAS Multiplier,
+              //and ask the Flow Controller if we should continue routing the CAS that was input to the CasMultiplier.
+              if (!frame.originalCasFlow.continueOnFailure(frame.casMultiplierAeKey, e)) {
+                throw e;              
+              }
+              //if the Flow says to continue, we fall through to the if (cas == null) block below, get
+              //the originalCas from the stack and continue with its flow.
+            }
+            if (cas == null) {
               // we've finished routing all the Output CASes from a StackFrame. Now
               // get the originalCas (the one that was input to the CasMultiplier) from
               // that stack frame and continue with its flow
               cas = frame.originalCas;
               flow = frame.originalCasFlow;
               nextStep = frame.incompleteParallelStep; //in case we need to resume a parallel step
-              cas.setCurrentComponentInfo(null); // this CAS is done being processed by the
-              // previous AnalysisComponent
+              cas.setCurrentComponentInfo(null); // this CAS is done being processed by the previous AnalysisComponent
               casIteratorStack.pop(); // remove this state from the stack now
             }
           }
@@ -526,12 +537,22 @@
               AnalysisEngine nextAe = (AnalysisEngine) mComponentAnalysisEngineMap.get(nextAeKey);
               if (nextAe != null) {
                 // invoke next AE in flow
-                CasIterator casIter;
-                casIter = nextAe.processAndOutputNewCASes(cas);
-                if (casIter.hasNext()) // new CASes are output
+                CasIterator casIter = null;
+                CAS outputCas = null; //used if the AE we call outputs a new CAS
+                try {
+                  casIter = nextAe.processAndOutputNewCASes(cas);
+                  if (casIter.hasNext()) {
+                    outputCas = casIter.next();
+                  }
+                }
+                catch(Exception e) {
+                  //ask the FlowController if we should continue
+                  //TODO: should this be configurable?
+                  if (!flow.continueOnFailure(nextAeKey, e))
+                    throw e;
+                }
+                if (outputCas != null) // new CASes are output
                 {
-                  // get the first output CAS
-                  CAS outputCas = casIter.next();
                   // push the CasIterator, original CAS, and Flow onto a stack so we
                   // can get the other output CASes and the original CAS later
                   casIteratorStack.push(new StackFrame(casIter, cas, flow, nextAeKey));
@@ -551,7 +572,7 @@
                         new Object[] { nextAeKey });
               }
             } 
-            //ParallelStep
+            //ParallelStep (TODO: refactor out common parts with SimpleStep?)
             else if (nextStep instanceof ParallelStep) {
               //create modifiable list of destinations 
               List destinations = new LinkedList(((ParallelStep)nextStep).getAnalysisEngineKeys());
@@ -563,11 +584,22 @@
                 AnalysisEngine nextAe = (AnalysisEngine) mComponentAnalysisEngineMap.get(nextAeKey);
                 if (nextAe != null) {
                   // invoke next AE in flow
-                  CasIterator casIter;
-                  casIter = nextAe.processAndOutputNewCASes(cas);
-                  if (casIter.hasNext()) // new CASes are output
+                  CasIterator casIter = null;
+                  CAS outputCas = null; //used if the AE we call outputs a new CAS
+                  try {
+                    casIter = nextAe.processAndOutputNewCASes(cas);
+                    if (casIter.hasNext()) {
+                      outputCas = casIter.next();
+                    }
+                  }
+                  catch(Exception e) {
+                    //ask the FlowController if we should continue
+                    //TODO: should this be configurable?
+                    if (!flow.continueOnFailure(nextAeKey, e))
+                      throw e;
+                  }
+                  if (outputCas != null) // new CASes are output
                   {
-                    CAS outputCas = casIter.next();
                     // when pushing the stack frame so we know where to pick up later,
                     // be sure to include the incomplete ParallelStep
                     if (!destinations.isEmpty()) {

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/FlowContainer.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/FlowContainer.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/FlowContainer.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/analysis_engine/asb/impl/FlowContainer.java Fri Mar  2 12:36:14 2007
@@ -19,6 +19,8 @@
 
 package org.apache.uima.analysis_engine.asb.impl;
 
+import java.util.Collection;
+
 import org.apache.uima.UIMAFramework;
 import org.apache.uima.analysis_engine.AnalysisEngineProcessException;
 import org.apache.uima.analysis_engine.impl.AnalysisEngineManagementImpl;
@@ -90,6 +92,11 @@
   public void aborted() {
     mFlow.aborted();
   }
+  
+  public boolean continueOnFailure(String failedAeKey, Exception failure) {
+    return mFlow.continueOnFailure(failedAeKey, failure);
+  }
+
 
   private CasManager getCasManager() {
     return mFlowControllerContainer.getResourceManager().getCasManager();

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/CasFlow_ImplBase.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/CasFlow_ImplBase.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/CasFlow_ImplBase.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/CasFlow_ImplBase.java Fri Mar  2 12:36:14 2007
@@ -19,6 +19,7 @@
 
 package org.apache.uima.flow;
 
+
 import org.apache.uima.UIMA_UnsupportedOperationException;
 import org.apache.uima.analysis_engine.AnalysisEngineProcessException;
 import org.apache.uima.cas.AbstractCas;
@@ -59,6 +60,17 @@
     }
   }
   
+  
+  
+  /**
+   * By default, returns false, indicating that processing cannot continue after a failure.
+   * May be overridden by subclasses to allow processing to continue.
+   * @see org.apache.uima.flow.Flow#continueOnFailure(String, java.lang.Exception)
+   */
+  public boolean continueOnFailure(String failedAeKey, Exception failure) {
+    return false;
+  }
+
   /** 
    * By default, does nothing.  May be overriden by subclasses to release resources 
    * when a flow is aborted.

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/Flow.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/Flow.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/Flow.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/Flow.java Fri Mar  2 12:36:14 2007
@@ -19,6 +19,7 @@
 
 package org.apache.uima.flow;
 
+
 import org.apache.uima.analysis_engine.AnalysisEngineProcessException;
 import org.apache.uima.cas.AbstractCas;
 
@@ -60,7 +61,34 @@
    * @throws AnalysisEngineProcessException
    */
   Flow newCasProduced(AbstractCas newCas, String producedBy) throws AnalysisEngineProcessException;
-  
+
+  /**
+   * May be called by the framework to ask the FlowController if processing of the CAS can
+   * continue after a failure occurred while executing the last Step that the Flow Controller
+   * returned.  
+   * <p>
+   * If this method returns true, then the framework may continue to call the {@link #next()} method
+   * to continue routing the CAS.  If this method returns false, the framework will not make any
+   * more calls to the {@link #next()} method, and will call the {@link #aborted()} method.
+   * <p>
+   * In the case where the last Step was a {@link ParallelStep}, if at least one of the destinations 
+   * resulted in a failure, this method will be called to report one of the failures.  If this method
+   * returns true, but one of the other destinations in the ParallelStep resulted in a failure, this
+   * method will be called again to report the next failure.  This continues until either this method
+   * returns false or there are no more failures.
+   * <p>
+   * Note that it is possible for processing of a CAS to be aborted without this method being called.
+   * This method is only called when an attempt is being made to continue processing of the CAS
+   * following an error, which may be an application configuration decision. 
+   *  
+   * @param failedAeKey The key of the analysis engine that failed.
+   * @param failure the Exception that occurred
+   * 
+   * @return true if the FlowController decides that processing of the CAS can continue; false if
+   *   processing of the CAS should be aborted.
+   */
+  boolean continueOnFailure(String failedAeKey, Exception failure);
+
   /**
    * Called by the framework if processing has been aborted for the CAS that was being
    * routed by this Flow object.  No further processing will take place on the CAS after
@@ -70,4 +98,5 @@
    * Also, it could be used to allow the FlowController to reuse a Flow object if desired.
    */  
   void aborted();
+  
 }

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/JCasFlow_ImplBase.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/JCasFlow_ImplBase.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/JCasFlow_ImplBase.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/main/java/org/apache/uima/flow/JCasFlow_ImplBase.java Fri Mar  2 12:36:14 2007
@@ -19,6 +19,7 @@
 
 package org.apache.uima.flow;
 
+
 import org.apache.uima.UIMA_UnsupportedOperationException;
 import org.apache.uima.analysis_engine.AnalysisEngineProcessException;
 import org.apache.uima.cas.AbstractCas;
@@ -57,6 +58,15 @@
               AnalysisEngineProcessException.INCORRECT_CAS_INTERFACE, new Object[] { JCas.class,
                   newCas.getClass() });
     }
+  }
+  
+  /**
+   * By default, returns false, indicating that processing cannot continue after a failure.
+   * May be overridden by subclasses to allow processing to continue.
+   * @see org.apache.uima.flow.Flow#continueOnFailure(String, java.lang.Exception)
+   */
+  public boolean continueOnFailure(String failedAeKey, Exception failure) {
+    return false;
   }
   
   /** 

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/AnalysisEngine_implTest.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/AnalysisEngine_implTest.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/AnalysisEngine_implTest.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/AnalysisEngine_implTest.java Fri Mar  2 12:36:14 2007
@@ -1005,7 +1005,7 @@
       CAS cas = tae.newCAS();
       for (int i = 0; i < 2; i++) // verify we can do this more than once
       {
-        FlowControllerForErrorTest.abortedDocuments.clear();
+        FlowControllerForErrorTest.reset();
         cas.setDocumentText("Line one\nLine two\nERROR");
         CasIterator iter = tae.processAndOutputNewCASes(cas);
         assertTrue(iter.hasNext());
@@ -1042,7 +1042,7 @@
       cas = tae.newCAS();
       for (int i = 0; i < 2; i++) // verify we can do this more than once
       {
-        FlowControllerForErrorTest.abortedDocuments.clear();
+        FlowControllerForErrorTest.reset();
         cas.setDocumentText("Line one\nLine two\nERROR");
         CasIterator iter = tae.processAndOutputNewCASes(cas);
         assertTrue(iter.hasNext());
@@ -1132,6 +1132,59 @@
       } catch (AnalysisEngineProcessException e) {
         // should get here
       }
+      
+      // bad segmenter in an aggregate
+      AnalysisEngineDescription aggWithBadSegmenterDesc = UIMAFramework.getXMLParser()
+      .parseAnalysisEngineDescription(
+              new XMLInputSource(JUnitExtension
+                      .getFile("TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml")));
+      tae = UIMAFramework.produceAnalysisEngine(aggWithBadSegmenterDesc);
+      FlowControllerForErrorTest.reset();
+      cas = tae.newCAS();
+      cas.setDocumentText("Line one\nLine two\nLine three");
+      iter = tae.processAndOutputNewCASes(cas);
+      assertTrue(iter.hasNext());
+      outCas = iter.next(); // first call OK
+      outCas.release();
+      assertTrue(FlowControllerForErrorTest.abortedDocuments.isEmpty());
+      assertTrue(FlowControllerForErrorTest.failedAEs.isEmpty());
+      // next call should fail with AnalysisEngineProcessException
+      try {
+        if (iter.hasNext()) {
+          iter.next();
+        }
+        fail(); // should not get here
+      } catch (AnalysisEngineProcessException e) {
+        // should get here
+      }
+      assertEquals(1, FlowControllerForErrorTest.abortedDocuments.size());
+      assertTrue(FlowControllerForErrorTest.abortedDocuments.contains("Line one\nLine two\nLine three"));
+      assertEquals(1,FlowControllerForErrorTest.failedAEs.size());
+      assertTrue(FlowControllerForErrorTest.failedAEs.contains("Segmenter"));
+
+      //configure AE to continue after error
+      tae = UIMAFramework.produceAnalysisEngine(aggWithBadSegmenterDesc);
+      tae.setConfigParameterValue("ContinueOnFailure", Boolean.TRUE);
+      tae.reconfigure();
+      FlowControllerForErrorTest.reset();
+
+      cas.reset();
+      cas.setDocumentText("Line one\nLine two\nLine three");
+      iter = tae.processAndOutputNewCASes(cas);
+      assertTrue(iter.hasNext());
+      outCas = iter.next(); // first call OK
+      outCas.release();
+      assertTrue(FlowControllerForErrorTest.abortedDocuments.isEmpty());
+      assertTrue(FlowControllerForErrorTest.failedAEs.isEmpty());
+      
+      //next call should not have aborted, but FC should have been notified of the failiure,
+      // and no CAS should come back
+      assertFalse(iter.hasNext());
+      assertEquals(0, FlowControllerForErrorTest.abortedDocuments.size());
+      assertEquals(1, FlowControllerForErrorTest.failedAEs.size());
+      assertTrue(FlowControllerForErrorTest.failedAEs.contains("Segmenter"));
+      
+      
     } catch (Exception e) {
       JUnitExtension.handleException(e);
     }
@@ -1237,13 +1290,14 @@
               .parseAnalysisEngineDescription(
                       new XMLInputSource(JUnitExtension.getFile("TextAnalysisEngineImplTest/AggregateForErrorTest.xml")));
       AnalysisEngine ae = UIMAFramework.produceAnalysisEngine(aeDesc);
-      FlowControllerForErrorTest.abortedDocuments.clear();
+      FlowControllerForErrorTest.reset();
       CAS cas = ae.newCAS();
       //try document that should succeed
       cas.setDocumentText("This is OK");
       ae.process(cas);
       //flow controller should not be notified
       assertTrue(FlowControllerForErrorTest.abortedDocuments.isEmpty());
+      assertTrue(FlowControllerForErrorTest.failedAEs.isEmpty());
       
       //now one that fails
       cas.reset();
@@ -1257,13 +1311,29 @@
       }
       assertEquals(1, FlowControllerForErrorTest.abortedDocuments.size());
       assertTrue(FlowControllerForErrorTest.abortedDocuments.contains("ERROR"));
-      
+      assertEquals(1, FlowControllerForErrorTest.failedAEs.size());
+      assertTrue(FlowControllerForErrorTest.failedAEs.contains("ErrorAnnotator"));
+    
       //AE should still be able to process a new document now
-      FlowControllerForErrorTest.abortedDocuments.clear();
+      FlowControllerForErrorTest.reset();
       cas.reset();
       cas.setDocumentText("This is OK");
       ae.process(cas);
       assertTrue(FlowControllerForErrorTest.abortedDocuments.isEmpty());
+      assertTrue(FlowControllerForErrorTest.failedAEs.isEmpty());
+      
+      //configure AE to continue after error
+      ae.setConfigParameterValue("ContinueOnFailure", Boolean.TRUE);
+      ae.reconfigure();
+      cas.reset();
+      cas.setDocumentText("ERROR");
+      ae.process(cas); //should not throw exception now
+      
+      //document should not have aborted, but FC should have been notified of the failiure
+      assertEquals(0, FlowControllerForErrorTest.abortedDocuments.size());
+      assertEquals(1, FlowControllerForErrorTest.failedAEs.size());
+      assertTrue(FlowControllerForErrorTest.failedAEs.contains("ErrorAnnotator"));
+      
     } catch (Exception e) {
       JUnitExtension.handleException(e);
     }    

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/FlowControllerForErrorTest.java
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/FlowControllerForErrorTest.java?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/FlowControllerForErrorTest.java (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/test/java/org/apache/uima/analysis_engine/impl/FlowControllerForErrorTest.java Fri Mar  2 12:36:14 2007
@@ -40,14 +40,20 @@
  * when an error occurs.
  */
 public class FlowControllerForErrorTest extends CasFlowController_ImplBase {
-  String[] mSequence;
+  public static final String PARAM_CONTINUE_ON_FAILURE = "ContinueOnFailure";
+  
+  private String[] mSequence;
+  private boolean mContinueOnFailure;
   
   public static List abortedDocuments = new ArrayList();
+  public static List failedAEs = new ArrayList();
 
   public void initialize(FlowControllerContext aContext) throws ResourceInitializationException {
     super.initialize(aContext);
     FlowConstraints flowConstraints = aContext.getAggregateMetadata().getFlowConstraints();
     mSequence = ((FixedFlow) flowConstraints).getFixedFlow();
+    Boolean paramVal = (Boolean)aContext.getConfigParameterValue(PARAM_CONTINUE_ON_FAILURE);
+    mContinueOnFailure = paramVal != null && paramVal.booleanValue(); 
   }
 
   /*
@@ -113,6 +119,16 @@
         i++;
       return new FixedFlowObject(newCas, i + 1, true);
     }
+    
+    
+
+    /* (non-Javadoc)
+     * @see org.apache.uima.flow.CasFlow_ImplBase#continueOnFailure(java.lang.String, java.lang.Exception)
+     */
+    public boolean continueOnFailure(String failedAeKey, Exception failure) {
+      failedAEs.add(failedAeKey);
+      return mContinueOnFailure;
+    }
 
     /* (non-Javadoc)
      * @see org.apache.uima.flow.CasFlow_ImplBase#aborted()
@@ -122,5 +138,13 @@
     }
     
     
+  }
+
+  /**
+   * 
+   */
+  public static void reset() {
+    abortedDocuments.clear();
+    failedAEs.clear();    
   }
 }

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateForErrorTest.xml
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateForErrorTest.xml?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateForErrorTest.xml (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateForErrorTest.xml Fri Mar  2 12:36:14 2007
@@ -1,58 +1,69 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- -->
-
-<!-- Aggregate descriptor for testing merging of type system, type priorities, and indexes. -->
-
-<taeDescription xmlns="http://uima.apache.org/resourceSpecifier">
-<frameworkImplementation>org.apache.uima.java</frameworkImplementation>
-<primitive>false</primitive>
-
-<delegateAnalysisEngineSpecifiers>
-  <delegateAnalysisEngine key="ErrorAnnotator">
-    <import location="ErrorAnnotator.xml"/>
-  </delegateAnalysisEngine>
-</delegateAnalysisEngineSpecifiers>
-<flowController>
-  <import location="FlowControllerForErrorTest.xml"/>
-</flowController>
-
-<analysisEngineMetaData>
-<name>Aggregate for Error Test</name>
-<description>An aggregate for testing error beahvior.</description>
-<version>1.0</version>
-<vendor>The Apache Software Foundation</vendor>
-
-<flowConstraints>
-  <fixedFlow>
-    <node>ErrorAnnotator</node>
-  </fixedFlow>  
-</flowConstraints>
-
-<!-- Capabilities: Inputs and Outputs -->
-<capabilities>
-<capability>
-<inputs/>
-<outputs/>
-<languagesSupported/>
-</capability>
-</capabilities>
-
-</analysisEngineMetaData>
-</taeDescription>
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+* 
+*   http://www.apache.org/licenses/LICENSE-2.0
+* 
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+-->
+
+<!-- Aggregate descriptor for testing merging of type system, type priorities, and indexes. -->
+
+<taeDescription xmlns="http://uima.apache.org/resourceSpecifier">
+  <frameworkImplementation>org.apache.uima.java</frameworkImplementation>
+  <primitive>false</primitive>
+  
+  <delegateAnalysisEngineSpecifiers>
+    <delegateAnalysisEngine key="ErrorAnnotator">
+      <import location="ErrorAnnotator.xml"/>
+    </delegateAnalysisEngine>
+  </delegateAnalysisEngineSpecifiers>
+  <flowController key="FlowController">
+    <import location="FlowControllerForErrorTest.xml"/>
+  </flowController>
+  
+  <analysisEngineMetaData>
+    <name>Aggregate for Error Test</name>
+    <description>An aggregate for testing error beahvior.</description>
+    <version>1.0</version>
+    <vendor>The Apache Software Foundation</vendor>
+    
+    <configurationParameters>
+      <configurationParameter>
+        <name>ContinueOnFailure</name>
+        <description/>
+        <type>Boolean</type>
+        <overrides>
+          <parameter>FlowController/ContinueOnFailure</parameter>
+        </overrides>
+      </configurationParameter>
+    </configurationParameters>
+    
+    <flowConstraints>
+      <fixedFlow>
+        <node>ErrorAnnotator</node>
+      </fixedFlow>
+    </flowConstraints>
+    
+    <!-- Capabilities: Inputs and Outputs -->
+    <capabilities>
+      <capability>
+        <inputs/>
+        <outputs/>
+        <languagesSupported/>
+      </capability>
+    </capabilities>
+    
+  </analysisEngineMetaData>
+</taeDescription>
\ No newline at end of file

Added: incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml?view=auto&rev=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml (added)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/AggregateWithBadSegmenterForErrorTest.xml Fri Mar  2 12:36:14 2007
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ -->
+
+<taeDescription xmlns="http://uima.apache.org/resourceSpecifier">
+  <frameworkImplementation>org.apache.uima.java</frameworkImplementation>
+  <primitive>false</primitive>
+  <delegateAnalysisEngineSpecifiers>
+    <delegateAnalysisEngine key="Segmenter">
+      <import location="BadSegmenter.xml"/>
+    </delegateAnalysisEngine>
+    <delegateAnalysisEngine key="Annotator">
+      <import location="ErrorAnnotator.xml"/>
+    </delegateAnalysisEngine>
+  </delegateAnalysisEngineSpecifiers>
+	<flowController key="FlowController">
+    <import location="FlowControllerForErrorTest.xml"/>
+  </flowController>
+  
+  <analysisEngineMetaData>
+    <name>Example Aggregate TAE including a Segmenter</name>
+    <description>For testing only</description>
+    <configurationParameters>
+      <configurationParameter>
+        <name>ContinueOnFailure</name>
+        <description/>
+        <type>Boolean</type>
+        <overrides>
+          <parameter>FlowController/ContinueOnFailure</parameter>
+        </overrides>
+      </configurationParameter>
+    </configurationParameters>
+    <flowConstraints>
+      <fixedFlow>
+        <node>Segmenter</node>
+        <node>Annotator</node>
+      </fixedFlow>
+    </flowConstraints>    
+    <capabilities>
+      <capability>
+        <inputs/>
+        <outputs/>
+      </capability>
+    </capabilities>
+    <operationalProperties>
+      <modifiesCas>true</modifiesCas>
+      <multipleDeploymentAllowed>true</multipleDeploymentAllowed>
+      <outputsNewCASes>true</outputsNewCASes>
+    </operationalProperties>
+  </analysisEngineMetaData>
+</taeDescription>

Modified: incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/FlowControllerForErrorTest.xml
URL: http://svn.apache.org/viewvc/incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/FlowControllerForErrorTest.xml?view=diff&rev=513937&r1=513936&r2=513937
==============================================================================
--- incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/FlowControllerForErrorTest.xml (original)
+++ incubator/uima/uimaj/trunk/uimaj-core/src/test/resources/TextAnalysisEngineImplTest/FlowControllerForErrorTest.xml Fri Mar  2 12:36:14 2007
@@ -29,5 +29,13 @@
 <description>For testing only.</description>
 <version>1.0</version>
 <vendor>The Apache Software Foundation</vendor>
+<configurationParameters>
+  <configurationParameter>
+    <name>ContinueOnFailure</name>
+    <description/>
+    <type>Boolean</type>
+    <mandatory>false</mandatory>
+  </configurationParameter>
+</configurationParameters>
 </processingResourceMetaData>
 </flowControllerDescription>