You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@uima.apache.org by cw...@apache.org on 2020/06/10 13:14:48 UTC

[uima-async-scaleout] 04/34: UIMA-5280 modified to not depend on dd2spring, refactored code to support calling a service without a broker, many internal changes to support local transport better

This is an automated email from the ASF dual-hosted git repository.

cwiklik pushed a commit to branch uima-as-3
in repository https://gitbox.apache.org/repos/asf/uima-async-scaleout.git

commit 078502cefe4adb968552b62ab70d36c35ad91f39
Author: cwiklik <cwiklik>
AuthorDate: Mon Feb 26 18:54:11 2018 +0000

    UIMA-5280 modified to not depend on dd2spring, refactored code to support calling a service without a broker, many internal changes to support local transport better
---
 .../jms/activemq/ConcurrentMessageListener.java    |   60 +-
 .../jms/activemq/JmsEndpointConnection_impl.java   |   30 +-
 .../uima/adapter/jms/activemq/JmsInputChannel.java |  363 ++--
 .../adapter/jms/activemq/JmsOutputChannel.java     |  116 +-
 .../jms/activemq/TempDestinationResolver.java      |    9 +-
 .../UimaDefaultMessageListenerContainer.java       |  217 +-
 .../adapter/jms/client/ActiveMQMessageSender.java  |   13 +-
 .../client/BaseUIMAAsynchronousEngine_impl.java    |  593 +++--
 .../uima/adapter/jms/service/UIMA_Service.java     |   78 +-
 .../uima/adapter/jms/service/UimaASJmsService.java |  366 ++++
 .../jms/service/builder/ActiveMQFactory.java       |   48 +
 .../service/builder/JmsMessageListenerBuilder.java |  333 +++
 .../service/builder/UimaAsJmsServiceBuilder.java   |  316 +++
 .../uima/as/deployer/AbstractUimaASDeployer.java   |   85 +
 .../apache/uima/as/deployer/ServiceDeployers.java  |   74 +
 .../uima/as/deployer/UimaAsServiceDeployer.java    |   25 +-
 .../direct/UimaAsDirectServiceDeployer.java        |   77 +
 .../as/deployer/jms/UimaAsJmsServiceDeployer.java  |   58 +
 .../apache/uima/as/dispatcher/LocalDispatcher.java |  138 ++
 .../apache/uima/ee/test/TestUimaASExtended.java    |    3 +-
 .../apache/uima/ee/test/TestUimaASNoErrors.java    | 2285 ++++++++++++++++++++
 .../apache/uima/ee/test/utils/BaseTestSupport.java |   92 +-
 .../java/org/apache/uima/aae/InProcessCache.java   |  102 +-
 .../java/org/apache/uima/aae/InputChannel.java     |   30 +-
 .../java/org/apache/uima/aae/OutputChannel.java    |    7 +-
 .../main/java/org/apache/uima/aae/UimaASUtils.java |   48 +
 .../org/apache/uima/aae/UimaAsThreadFactory.java   |   31 +-
 .../uima/aae/client/UimaAsynchronousEngine.java    |   17 +-
 .../AggregateAnalysisEngineController.java         |    9 +-
 .../AggregateAnalysisEngineController_impl.java    |  240 +-
 .../aae/controller/AnalysisEngineController.java   |   45 +-
 .../controller/BaseAnalysisEngineController.java   |  349 ++-
 .../aae/controller/ControllerStatusListener.java   |   35 +
 .../uima/aae/controller/DelegateEndpoint.java      |  127 ++
 .../org/apache/uima/aae/controller/Endpoint.java   |   16 +
 .../apache/uima/aae/controller/Endpoint_impl.java  |   46 +-
 .../org/apache/uima/aae/controller/LocalCache.java |   52 +-
 .../PrimitiveAnalysisEngineController_impl.java    |  292 ++-
 .../org/apache/uima/aae/delegate/Delegate.java     |    1 +
 .../org/apache/uima/aae/error/ErrorHandler.java    |    2 +-
 .../apache/uima/aae/error/ErrorHandlerBase.java    |    2 +-
 .../apache/uima/aae/error/ErrorHandlerChain.java   |   16 +-
 .../java/org/apache/uima/aae/error/Thresholds.java |   89 +
 .../uima/aae/error/handler/CpcErrorHandler.java    |    2 +-
 .../aae/error/handler/ProcessCasErrorHandler.java  |   22 +-
 .../handler/input/MetadataRequestHandler_impl.java |    2 +-
 .../input/MetadataResponseHandler_impl.java        |   20 +-
 .../handler/input/ProcessRequestHandler_impl.java  |   41 +-
 .../aae/handler/input/ProcessResponseHandler.java  |   15 +-
 .../org/apache/uima/aae/jmx/JmxManagement.java     |    2 +
 .../java/org/apache/uima/aae/jmx/JmxManager.java   |   17 +
 .../apache/uima/aae/message/AsynchAEMessage.java   |   12 +
 .../aae/message/UimaASClientDirectMessage.java     |  100 +
 .../uima/aae/message/UimaASClientMessage.java      |   31 +-
 .../uima/aae/service/AbstractUimaASService.java    |  136 ++
 .../aae/service/AsynchronousUimaASService.java     |  255 +++
 .../uima/aae/service/ScaleoutSpecification.java    |   42 +-
 .../uima/aae/service/ServiceNotFoundException.java |   23 +-
 .../apache/uima/aae/service/ServiceRegistry.java   |   24 +-
 .../org/apache/uima/aae/service/UimaASService.java |   46 +
 .../uima/aae/service/UimaAsServiceRegistry.java    |  109 +
 .../builder/AbstractUimaAsServiceBuilder.java      | 1074 +++++++++
 .../uima/aae/service/builder/ServiceBuilder.java   |   16 +-
 .../builder/UimaAsDirectServiceBuilder.java        |  348 +++
 .../aae/service/command/AbstractUimaAsCommand.java |  454 ++++
 .../CollectionProcessCompleteRequestCommand.java   |   29 +-
 .../CollectionProcessCompleteResponseCommand.java  |   47 +
 .../uima/aae/service/command/CommandFactory.java   |  154 ++
 .../aae/service/command/GetMetaRequestCommand.java |   56 +
 .../service/command/GetMetaResponseCommand.java    |   86 +
 .../command/NoOpCommand.java}                      |   44 +-
 .../aae/service/command/PingRequestCommand.java    |   60 +
 .../command/ProcessChildCasRequestCommand.java     |  548 +++++
 .../command/ProcessChildCasResponseCommand.java    |  359 +++
 .../command/ProcessInputCasRequestCommand.java     |  341 +++
 .../command/ProcessInputCasResponseCommand.java    |  319 +++
 .../service/command/ReleaseCASRequestCommand.java  |   48 +
 .../aae/service/command/StopRequestCommand.java    |   78 +
 .../uima/aae/service/command/UimaAsCommand.java    |   17 +-
 .../delegate/AggregateAnalysisEngineDelegate.java  |   47 +
 .../service/delegate/AnalysisEngineDelegate.java   |  111 +
 .../aae/service/delegate/CasMultiplierNature.java  |   51 +
 .../delegate/RemoteAnalysisEngineDelegate.java     |   59 +
 .../apache/uima/as/client/DirectInputChannel.java  |  579 +++++
 .../org/apache/uima/as/client/DirectListener.java  |  217 ++
 .../org/apache/uima/as/client/DirectMessage.java   |  125 ++
 .../uima/as/client/DirectMessageContext.java       |  126 ++
 .../apache/uima/as/client/DirectOutputChannel.java |  322 +++
 .../apache/uima/as/client/JavaQueueListener.java   |   17 +-
 .../apache/uima/as/client/LifecycleListener.java   |   20 +-
 .../java/org/apache/uima/as/client/Listener.java   |   31 +-
 .../uima/resourceSpecifier/factory/CasPool.java    |   14 +
 .../factory/DelegateConfiguration.java             |   13 +
 .../resourceSpecifier/factory/ServiceContext.java  |   11 +-
 .../factory/impl/CasMultiplierImpl.java            |   14 +-
 .../factory/impl/CasPoolImpl.java                  |   16 +-
 .../factory/impl/ColocatedDelegateEngineImpl.java  |    8 +-
 .../factory/impl/RemoteDelegateEngineImpl.java     |    2 +-
 .../factory/impl/ServiceContextImpl.java           |   17 +-
 .../uima/adapter/jms/client/BaseMessageSender.java |   21 +-
 .../BaseUIMAAsynchronousEngineCommon_impl.java     | 1031 ++++++++-
 .../adapter/jms/message/JmsMessageContext.java     |    2 +-
 .../uima/adapter/jms/message/PendingMessage.java   |   22 +-
 103 files changed, 13578 insertions(+), 1210 deletions(-)

diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/ConcurrentMessageListener.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/ConcurrentMessageListener.java
index a5089f6..f74b35f 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/ConcurrentMessageListener.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/ConcurrentMessageListener.java
@@ -67,9 +67,9 @@ import org.springframework.jms.listener.SessionAwareMessageListener;
  * 
  */
 public class ConcurrentMessageListener implements SessionAwareMessageListener {
-  private static final Class CLASS_NAME = ConcurrentMessageListener.class;
+  private static final Class<?> CLASS_NAME = ConcurrentMessageListener.class;
 
-  private SessionAwareMessageListener delegateListener;
+  private SessionAwareMessageListener<Message> delegateListener;
 
   private int concurrentThreadCount = 0;
 
@@ -98,14 +98,14 @@ public class ConcurrentMessageListener implements SessionAwareMessageListener {
    *          - JmsInputChannel instance to delegate CAS to
    * @throws InvalidClassException
    */
-  public ConcurrentMessageListener(int concurrentThreads, Object delegateListener, String destination, ThreadGroup threadGroup, String threadPrefix)
+  public ConcurrentMessageListener(int concurrentThreads,  SessionAwareMessageListener<Message> inputChannel, String destination, ThreadGroup threadGroup, String threadPrefix)
           throws InvalidClassException {
-    if (!(delegateListener instanceof SessionAwareMessageListener)) {
-      throw new InvalidClassException("Invalid Delegate Listener. Expected Object of Type:"
-              + SessionAwareMessageListener.class + " Received:" + delegateListener.getClass());
-    }
+//    if (!(delegateListener instanceof SessionAwareMessageListener)) {
+//      throw new InvalidClassException("Invalid Delegate Listener. Expected Object of Type:"
+//              + SessionAwareMessageListener.class + " Received:" + delegateListener.getClass());
+//    }
     concurrentThreadCount = concurrentThreads;
-    this.delegateListener = (SessionAwareMessageListener) delegateListener;
+    this.delegateListener = inputChannel;
     if (concurrentThreads > 1) {
       //  created an unbounded queue. The throttling is controlled by the
       //  semaphore in the UimaBlockingExecutor initialized below
@@ -140,7 +140,34 @@ public class ConcurrentMessageListener implements SessionAwareMessageListener {
   private boolean isMessageFromCasMultiplier(final Message message) throws JMSException {
     return message.propertyExists(AsynchAEMessage.CasSequence);
   }
+  private void incrementChildCASes(String parentCASid) {
+      try {
+//          String parentCasReferenceId = message
+//                  .getStringProperty(AsynchAEMessage.InputCasReference);
+          // Fetch parent CAS entry from the local cache
+          CasStateEntry parentEntry = controller.getLocalCache().lookupEntry(parentCASid);
+          // increment number of child CASes this parent has in play
+          parentEntry.incrementSubordinateCasInPlayCount();
+          //  increment a counter that counts number of child CASes that have no
+          //  flow object yet. The flow object is created for each child CAS from
+          //  the parent flow object. The method below will actually acquire a 
+          //  permit from a binary semaphore to force the parent to block until
+          //  the last of its children acquires its Flow object.
+          parentEntry.incrementOutstandingFlowCounter();
 
+        } catch (Exception e) {
+          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
+            
+            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
+                    "onMessage", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
+                    "UIMAEE_service_exception_WARNING", controller.getComponentName());
+            
+            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
+                    "onMessage", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+                    "UIMAJMS_exception__WARNING", e);
+          }
+        }
+  }
   /**
    * Intercept a message to increment a child count of the input CAS. This method is always called
    * in a single thread, guaranteeing order of processing. The child CAS will always come here
@@ -150,12 +177,16 @@ public class ConcurrentMessageListener implements SessionAwareMessageListener {
    * 
    */
   public void onMessage(final Message message, final Session session) throws JMSException {
-    try {
-      // Wait until the controller is plugged in
-      controllerLatch.await();
-    } catch (InterruptedException e) {
-    }
+	  System.out.println("..........ConcurrentMessageListener.onMessage() got message");
+
+//	  try {
+//      // Wait until the controller is plugged in
+//      controllerLatch.await();
+//    } catch (InterruptedException e) {
+//    }
     if (isMessageFromCasMultiplier(message)) {
+    	  System.out.println("..........ConcurrentMessageListener.onMessage() message from remote CM");
+
       // Check if the message came from a Cas Multiplier and it contains a new Process Request
       int command = message.getIntProperty(AsynchAEMessage.Command);
       int messageType = message.getIntProperty(AsynchAEMessage.MessageType);
@@ -171,6 +202,8 @@ public class ConcurrentMessageListener implements SessionAwareMessageListener {
             delegate.setConcurrentConsumersOnReplyQueue();
           }
         }
+        incrementChildCASes(message.getStringProperty(AsynchAEMessage.InputCasReference));
+/*
         try {
           String parentCasReferenceId = message
                   .getStringProperty(AsynchAEMessage.InputCasReference);
@@ -197,6 +230,7 @@ public class ConcurrentMessageListener implements SessionAwareMessageListener {
                     "UIMAJMS_exception__WARNING", e);
           }
         }
+        */
       }
 
     }
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsEndpointConnection_impl.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsEndpointConnection_impl.java
index b029f2a..dc0bf7f 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsEndpointConnection_impl.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsEndpointConnection_impl.java
@@ -55,6 +55,7 @@ import org.apache.uima.aae.UIMAEE_Constants;
 import org.apache.uima.aae.controller.AggregateAnalysisEngineController;
 import org.apache.uima.aae.controller.AnalysisEngineController;
 import org.apache.uima.aae.controller.BaseAnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.Endpoint;
 import org.apache.uima.aae.controller.LocalCache.CasStateEntry;
 import org.apache.uima.aae.delegate.Delegate;
@@ -124,8 +125,9 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
     } else {
       //  If this is a reply to a client, use the same broker URL that manages this service input queue.
       //  Otherwise this is a request so use a broker specified in the endpoint object.
-      serverUri = (anEndpoint.isReplyEndpoint()) ? 
-              ((JmsOutputChannel) aController.getOutputChannel()).getServerURI() : anEndpoint.getServerURI();
+//      serverUri = (anEndpoint.isReplyEndpoint()) ? 
+//              ((JmsOutputChannel) aController.getOutputChannel()).getServerURI() : anEndpoint.getServerURI();
+      serverUri = anEndpoint.getServerURI();
     }
     isReplyEndpoint = anEndpoint.isReplyEndpoint();
     controller = aController;
@@ -183,7 +185,7 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
 		        // If replying to http request, reply to a queue managed by this service broker using tcp
 		        // protocol
 		        if (isReplyEndpoint && brokerUri.startsWith("http")) {
-		          brokerUri = ((JmsOutputChannel) aController.getOutputChannel()).getServerURI();
+		          brokerUri = ((JmsOutputChannel) aController.getOutputChannel(ENDPOINT_TYPE.JMS)).getServerURI();
 
 		          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
 		            UIMAFramework.getLogger(CLASS_NAME).logrb(
@@ -193,7 +195,7 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
 		                    JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
 		                    "UIMAJMS_override_connection_to_endpoint__FINE",
 		                    new Object[] {  aComponentName, getEndpoint(),
-		                      ((JmsOutputChannel) aController.getOutputChannel()).getServerURI() });
+		                      ((JmsOutputChannel) aController.getOutputChannel(ENDPOINT_TYPE.JMS)).getServerURI() });
 		          }
 		        } else if ( !brokerUri.startsWith("http") && !brokerUri.startsWith("failover") && !brokerUri.startsWith("vm://localhost?broker.persistent=false")){
 				  String prefix = "";
@@ -772,7 +774,7 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
     		        }
     	          UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
     	                  "send", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
-    	                  "UIMAEE_service_delivery_exception__WARNING",new Object[] { controller.getComponentName(), key, endpointName});
+    	                  "UIMAEE_service_delivery_exception__WARNING",new Object[] { controller.getComponentName(), endpointName,serverUri});
     		  }
    		      CasStateEntry casStateEntry = controller.getLocalCache().lookupEntry(casReferenceId);
    		      // Mark the CAS as failed so that the CAS is released and cache cleaned up
@@ -889,14 +891,14 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
    * @param delegateKey
    * @throws Exception
    */
-  private void createListener(String delegateKey) throws Exception {
-    if (controller instanceof AggregateAnalysisEngineController) {
-      // Fetch an InputChannel that handles messages for a given delegate
-      InputChannel iC = controller.getReplyInputChannel(delegateKey);
-      // Create a new Listener, new Temp Queue and associate the listener with the Input Channel
-      iC.createListener(delegateKey, null);
-    }
-  }
+//  private void createListener(String delegateKey) throws Exception {
+//    if (controller instanceof AggregateAnalysisEngineController) {
+//      // Fetch an InputChannel that handles messages for a given delegate
+//      InputChannel iC = controller.getReplyInputChannel(delegateKey);
+//      // Create a new Listener, new Temp Queue and associate the listener with the Input Channel
+//      iC.createListener(delegateKey, null);
+//    }
+//  }
 
   private synchronized boolean handleJmsException(JMSException ex) {
     if (!failed) {
@@ -990,7 +992,7 @@ public class JmsEndpointConnection_impl implements ConsumerListener {
 			if ( casState.isSubordinate()) {
 				try {
 					
-					String inputCasId = casState.getInputCasReferenceId();
+					String inputCasId = casState.getParentCasReferenceId();
 					if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
 						UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
 								"UimaAsAsyncCallbackListener.onException()", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsInputChannel.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsInputChannel.java
index 109e7c1..b9710da 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsInputChannel.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsInputChannel.java
@@ -41,8 +41,10 @@ import org.apache.uima.UIMAFramework;
 import org.apache.uima.aae.InputChannel;
 import org.apache.uima.aae.UIMAEE_Constants;
 import org.apache.uima.aae.client.UimaAsynchronousEngine;
+import org.apache.uima.aae.client.UimaAsynchronousEngine.Transport;
 import org.apache.uima.aae.controller.AggregateAnalysisEngineController;
 import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.BaseAnalysisEngineController.ServiceState;
 import org.apache.uima.aae.controller.Endpoint;
 import org.apache.uima.aae.controller.Endpoint_impl;
@@ -59,8 +61,11 @@ import org.apache.uima.aae.message.MessageWrapper;
 import org.apache.uima.aae.message.UIMAMessage;
 import org.apache.uima.adapter.jms.JmsConstants;
 import org.apache.uima.adapter.jms.message.JmsMessageContext;
+import org.apache.uima.as.client.Listener;
+import org.apache.uima.as.client.Listener.Type;
 import org.apache.uima.util.Level;
 import org.springframework.jms.listener.SessionAwareMessageListener;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
 
 /**
  * Thin adapter for receiving JMS messages from Spring. It delegates processing of all messages to
@@ -69,17 +74,17 @@ import org.springframework.jms.listener.SessionAwareMessageListener;
  * 
  */
 public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
-        SessionAwareMessageListener {
+        SessionAwareMessageListener<Message> {
   /**
 	 * 
 	 */
   private static final long serialVersionUID = -3318400773113552290L;
 
-  private static final Class CLASS_NAME = JmsInputChannel.class;
+  private static final Class<?> CLASS_NAME = JmsInputChannel.class;
 
-  private transient final CountDownLatch msgHandlerLatch = new CountDownLatch(1);
+  private final transient CountDownLatch msgHandlerLatch = new CountDownLatch(1);
 
-  private transient final CountDownLatch controllerLatch = new CountDownLatch(1);
+  private final transient  CountDownLatch controllerLatch = new CountDownLatch(1);
 
   // Reference to the first Message Handler in the Chain.
   private transient Handler handler;
@@ -92,9 +97,9 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 
   private int sessionAckMode;
 
-  private transient UimaDefaultMessageListenerContainer messageListener;
+  //private transient UimaDefaultMessageListenerContainer messageListener;
 
-  private transient Session jmsSession;
+  //private transient Session jmsSession;
 
   private String brokerURL = "";
 
@@ -110,21 +115,37 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 
   private transient RemoteJMXServer remoteJMXServer = null;
   //  synchronizes initialization of RemotBroker
-  private Object brokerMux = new Object();
+ // private Object brokerMux = new Object();
   
-  private ConcurrentHashMap<String, UimaDefaultMessageListenerContainer> failedListenerMap = new ConcurrentHashMap<String, UimaDefaultMessageListenerContainer>();
+  private ConcurrentHashMap<String, Listener> failedListenerMap = 
+		  new ConcurrentHashMap<>();
 
   //  A global flag that determines if we should create a connection to broker's MBeanServer to be
   //  able to determine if client's reply queue exists before processing a CAS. 
   public static transient boolean attachToBrokerMBeanServer=true;
   
-  private static Map<String, List<UimaDefaultMessageListenerContainer>> listenerMap =
-          new ConcurrentHashMap<String, List<UimaDefaultMessageListenerContainer>>();
+  private ENDPOINT_TYPE type = ENDPOINT_TYPE.JMS;
 
+  private static Map<String, List<Listener>> listenerMap =
+          new ConcurrentHashMap<>();
+
+  private ChannelType channelType;
+  
+  public JmsInputChannel(ChannelType type) {
+	  this.channelType = type;
+  }
+  public ChannelType getChannelType() {
+	  return channelType;
+  }
+  
   public AnalysisEngineController getController() {
     return controller;
   }
-
+  public ENDPOINT_TYPE getType() {
+	  return type;
+  }
+  
+  
   public String getName() {
     return endpointName;
   }
@@ -652,18 +673,18 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
                 .getIntProperty(AsynchAEMessage.MessageType));
       
         String msgSentFromIP = null;
-
+     //   System.out.println("<<<<<<<<<<<<< Service:"+ getController().getComponentName()+ " Received JMS Message - command:"+command+" type:"+messageType);
         if (aMessage.getIntProperty(AsynchAEMessage.MessageType) == AsynchAEMessage.Response
                 && aMessage.propertyExists(AsynchAEMessage.ServerIP)) {
           msgSentFromIP = aMessage.getStringProperty(AsynchAEMessage.ServerIP);
         }
-        // System.out.println("***********************************************************************************"
-        // +
-        // "           \n**CONTROLLER::"+controller.getName()+"**** Received New Message From [ "+aMessage.getStringProperty(AsynchAEMessage.MessageFrom)+" ]**************"
-        // +
-        // "           \n**MSGTYPE::"+messageType+" COMMAND:"+command +
-        // " Cas Reference Id::"+casRefId+
-        // "           \n******************************************************************************");
+         System.out.println("***********************************************************************************"
+         +
+         "           \n**CONTROLLER::"+controller.getName()+"**** Received New Message From [ "+aMessage.getStringProperty(AsynchAEMessage.MessageFrom)+" ]**************"
+         +
+         "           \n**MSGTYPE::"+messageType+" COMMAND:"+command +
+        " Cas Reference Id::"+casRefId+
+        "           \n******************************************************************************");
 
         String msgFrom = (String) aMessage.getStringProperty(AsynchAEMessage.MessageFrom);
         if (controller != null && msgFrom != null) {
@@ -674,7 +695,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
         	if ( ackClient(messageContext)  ) {
         		try {
         			//	Any exception while sending an ACK results in a dropped request
-        			getController().getOutputChannel().sendReply(AsynchAEMessage.ServiceInfo,
+        			getController().getOutputChannel(ENDPOINT_TYPE.JMS).sendReply(AsynchAEMessage.ServiceInfo,
                             messageContext.getEndpoint(), aMessage.getStringProperty(AsynchAEMessage.CasReference), true);
         		} catch( Exception ex) {
         			UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
@@ -800,30 +821,31 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
   public String getServerUri() {
     return brokerURL;
   }
- public List<UimaDefaultMessageListenerContainer> registerListener(UimaDefaultMessageListenerContainer messageListener) {
-	    List<UimaDefaultMessageListenerContainer> ll = null;
+ public List<Listener> registerListener(Listener messageListener) {
+	    List<Listener> ll = null;
 	    if ( listenerMap.containsKey(getController().getComponentName()+getController().hashCode()) ) {
 	      ll = listenerMap.get(getController().getComponentName()+getController().hashCode());
 	    } else {
-	      ll = new CopyOnWriteArrayList<UimaDefaultMessageListenerContainer>();
+	      ll = new CopyOnWriteArrayList<>();
 	      listenerMap.put(getController().getComponentName()+getController().hashCode(), ll);
 	    }
 	    if (!ll.contains(messageListener)) {
 	      ll.add(messageListener);
 	    } else {
 	    }
-	    for(Object container: ll ) {
-	      UimaDefaultMessageListenerContainer c = (UimaDefaultMessageListenerContainer) container;
-	    }
+//	    for(Object container: ll ) {
+//	      UimaDefaultMessageListenerContainer c = (UimaDefaultMessageListenerContainer) container;
+//	    }
 	    return ll;
   }
-  public List<UimaDefaultMessageListenerContainer> getListeners() {
-	    List<UimaDefaultMessageListenerContainer> ll = null;
+  public List<Listener> getListeners() {
+	    List<Listener> ll = null;
 	    if ( listenerMap.containsKey(getController().getComponentName()+getController().hashCode()) ) {
 	      ll = listenerMap.get(getController().getComponentName()+getController().hashCode());
 	    }
 	    return ll;
   }
+  /*
   public synchronized void setListenerContainer(UimaDefaultMessageListenerContainer jmsL) {
     this.messageListener = jmsL;
     System.setProperty("BrokerURI", messageListener.getBrokerUrl());
@@ -844,7 +866,13 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
       }
     } 
   }
-
+*/
+  public synchronized void addListenerContainer(UimaDefaultMessageListenerContainer messageListener) {
+	// Add new message listener to the list
+	  registerListener(messageListener);
+	 
+  }
+  /*
   public ActiveMQConnectionFactory getConnectionFactory() {
     if (messageListener == null) {
       return null;
@@ -852,7 +880,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
       return (ActiveMQConnectionFactory) messageListener.getConnectionFactory();
     }
   }
-
+*/
   public void ackMessage(MessageContext aMessageContext) {
     if (aMessageContext != null && sessionAckMode == Session.CLIENT_ACKNOWLEDGE) {
       try {
@@ -877,15 +905,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
   }
 
   public String getInputQueueName() {
-    if (messageListener != null)
-      if (messageListener.getDestination() != null) {
-        return messageListener.getDestination().toString();
-      } else {
-        return messageListener.getDestinationName();// getEndpointName();
-      }
-    else {
-      return "";
-    }
+     return endpointName;
   }
 
   public ServiceInfo getServiceInfo() {
@@ -914,7 +934,47 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
               .setBrokerURL(brokerURL);
     }
   }
+  private void stopListener(final UimaDefaultMessageListenerContainer mL) throws Exception {
+	  System.out.println(".... "+getController().getComponentName()+" Stopping Listener Type:"+mL.getType());
+	  mL.setTerminating();
+	  mL.setAcceptMessagesWhileStopping(false);
+
+	  mL.stop();
+	  System.out.println(".... "+getController().getComponentName()+" Stopping Listener - Calling destroy()");
+	  System.out.println(".... "+getController().getComponentName()+" Stopping Listener - Done Calling destroy()");
 
+	  if ( mL.getTaskExecutor() instanceof ThreadPoolTaskExecutor ) {
+		  System.out.println(".... "+getController().getComponentName()+" Stopping ThreadPoolTaskExecutor");
+		  ThreadPoolTaskExecutor tpe = ((ThreadPoolTaskExecutor)mL.getTaskExecutor());
+		  tpe.destroy();
+
+	  } else {
+		  System.out.println(".... "+getController().getComponentName()+" ActiveConsumerCount:"+mL.getActiveConsumerCount());
+		  if ( mL.getTaskExecutor() != null ) {
+			  System.out.println(".... "+getController().getComponentName()+" TaskExecutor Class:"+ mL.getTaskExecutor().getClass().getName());
+		  } else {
+			  //((ActiveMQDestination)mL.getDestination()).;
+			  //mL.getTransport().;
+			 // mL.notifyAll();
+			  if ( mL.getActiveConsumerCount() > 0 ) {
+				  Thread.dumpStack();
+			  }
+			  System.out.println(".... "+getController().getComponentName()+" Destroying Listener");
+			  mL.doDestroy();
+			  System.out.println(".... "+getController().getComponentName()+" ActiveConsumerCount:"+mL.getActiveConsumerCount());
+
+		  }
+
+		  
+		 
+		  
+	  }
+	  System.out.println(".... "+getController().getComponentName()+" Stopping Listener - Stopped Task Executor");
+	  mL.shutdown();
+
+		System.out.println("Stopped Listener ");
+
+  }
   private void stopChannel(UimaDefaultMessageListenerContainer mL, boolean shutdownNow) throws Exception {
     String eName = mL.getEndpointName();
     if (eName != null) {
@@ -924,9 +984,9 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
                 new Object[] { eName, shutdownNow });
       }
     }
-    mL.delegateStop();
+   // mL.delegateStop();
     if (shutdownNow) {
-        mL.destroy(shutdownNow);
+    	stopListener(mL);
     }
    
     String selector = "";
@@ -958,7 +1018,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
   public void stop(boolean shutdownNow) throws Exception {
     stop(InputChannel.CloseAllChannels, shutdownNow);
     listenerContainerList.clear();
-    List<UimaDefaultMessageListenerContainer> ll = getListeners();
+    List<Listener> ll = getListeners();
     if ( ll != null ) {
        ll.clear();
     }
@@ -969,26 +1029,31 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
     }
   }
   public void disconnectListenersFromQueue() throws Exception {
-	List<UimaDefaultMessageListenerContainer> ll = getListeners();
-	for (UimaDefaultMessageListenerContainer listenerObject : ll) {
-		  	 stopChannel(listenerObject, false);
+	List<Listener> ll = getListeners();
+	for (Listener listenerObject : ll) {
+		disconnectListenerFromQueue(listenerObject);
 	}	
 
   }
+  public void disconnectListenerFromQueue(Listener listener) throws Exception {
+	stopChannel((UimaDefaultMessageListenerContainer)listener, false);
+  }
   public void setTerminating() {
-    List<UimaDefaultMessageListenerContainer> ll = getListeners();
-    for (UimaDefaultMessageListenerContainer listenerObject : ll) {
-        listenerObject.setTerminating();
+    List<Listener> ll = getListeners();
+    for (Listener listenerObject : ll) {
+    	((UimaDefaultMessageListenerContainer)listenerObject).setTerminating();
       } 
 
   }
   public void terminate() {
 	 try {
-	    List<UimaDefaultMessageListenerContainer> ll = getListeners();
-	    for (UimaDefaultMessageListenerContainer listenerObject : ll) {
-		      listenerObject.closeConnection();
+	    List<Listener> ll = getListeners();
+	    for (Listener listenerObject : ll) {
+	    	UimaDefaultMessageListenerContainer lc =
+	    			(UimaDefaultMessageListenerContainer)listenerObject;
+		      lc.closeConnection();
 		      if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINEST)) {
-		        String msg = "................. JmsInputChannel.terminate() - Listener -"+listenerObject.getDestinationName();
+		        String msg = "................. JmsInputChannel.terminate() - Listener -"+lc.getDestinationName();
                 UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST, CLASS_NAME.getName(), "terminate",
                  JmsConstants.JMS_LOG_RESOURCE_BUNDLE, "UIMAJMS_debug_msg__FINEST",
                   new Object[] { msg });
@@ -1004,8 +1069,8 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 	  
   }
   public synchronized void stop(int channelsToClose, boolean shutdownNow) throws Exception {
-	  List<UimaDefaultMessageListenerContainer> listenersToRemove = new ArrayList<UimaDefaultMessageListenerContainer>();
-      List<UimaDefaultMessageListenerContainer> ll = getListeners();
+	//  List<UimaDefaultMessageListenerContainer> listenersToRemove = new ArrayList<UimaDefaultMessageListenerContainer>();
+      List<Listener> ll = getListeners();
       String msg = ">>>>>>>>>>>> JmsInputChannel. stop() - Controller:"+controller.getComponentName()+ " Listener Count:"+ll.size();
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINEST)) {
           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST, CLASS_NAME.getName(), "stop",
@@ -1013,13 +1078,14 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
                   new Object[] { msg });
       }
 
-      for (UimaDefaultMessageListenerContainer listenerObject : ll) {
-
-       if (listenerObject != null && doCloseChannel(listenerObject, channelsToClose)) {
-           listenerObject.setRecoveryInterval(0);  // https://issues.apache.org/jira/browse/UIMA-3437
-          listenerObject.setTerminating();
-          listenerObject.setAutoStartup(false);
-          stopChannel(listenerObject, shutdownNow);
+      for (Listener listenerObject : ll) {
+    	  UimaDefaultMessageListenerContainer lc =
+    			  (UimaDefaultMessageListenerContainer)listenerObject;
+       if (listenerObject != null && doCloseChannel(lc, channelsToClose)) {
+           lc.setRecoveryInterval(0);  // https://issues.apache.org/jira/browse/UIMA-3437
+          lc.setTerminating();
+          lc.setAutoStartup(false);
+          stopChannel(lc, shutdownNow);
 
       } else {
         if (getController() != null) {
@@ -1041,9 +1107,9 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
     return stopped;
   }
 
-  public int getConcurrentConsumerCount() {
-    return messageListener.getConcurrentConsumers();
-  }
+//  public int getConcurrentConsumerCount() {
+//    return messageListener.getConcurrentConsumers();
+//  }
 
   private void testIfBrokerRunning(String aBrokerUrl) throws Exception {
     ActiveMQConnectionFactory f = new ActiveMQConnectionFactory(aBrokerUrl);
@@ -1061,7 +1127,8 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
     }
   }
   public void createListenerOnTempQueue(ConnectionFactory cf, boolean isFreeCasDestination ) throws Exception {
-	  TempDestinationResolver resolver = new TempDestinationResolver(controller.getComponentName());
+	  TempDestinationResolver resolver = 
+			  new TempDestinationResolver(controller.getComponentName(),"");
 	  UimaDefaultMessageListenerContainer connector = new UimaDefaultMessageListenerContainer(true);
 	  connector.setConnectionFactory(cf);
 	  resolver.setListener(connector);
@@ -1069,9 +1136,9 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 	  connector.setDestinationResolver(resolver);
 	  connector.setController(getController());
 	  connector.setMessageListener(this);
-	  connector.initializeContainer();
+	  //connector.initializeContainer();
 	  connector.getDestination();
-	  connector.afterPropertiesSet(false);
+	  connector.afterPropertiesSet();
 	  UIMAFramework.getLogger(CLASS_NAME).log(Level.INFO,getController().getComponentName()+"-JmsInputChannel.createListenerOnTempQueue()-starting new Listener" );
 	  connector.start();
 	  boolean log = true;
@@ -1103,10 +1170,10 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 	  }
 
 	  if ( isFreeCasDestination ) {
-		  ((JmsOutputChannel) getController().getOutputChannel())
+		  ((JmsOutputChannel) getController().getOutputChannel(ENDPOINT_TYPE.JMS))
           .setFreeCasQueue(connector.getListenerEndpoint());
 	  }
-      setListenerContainer(connector);
+      addListenerContainer(connector);
 
 	  if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.CONFIG)) {
 		  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.CONFIG, CLASS_NAME.getName(),
@@ -1116,7 +1183,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 	  }
   }
   public void createListenerForTargetedMessages() throws Exception {
-	  List<UimaDefaultMessageListenerContainer> listeners =
+	  List<Listener> listeners =
 			  getListeners();
 	  // the TargetServiceId property value will become part of a jms selector. 
 	  String targetStringSelector = "";
@@ -1130,49 +1197,63 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
 	  // find a listener instance which handles Process requests. The targeted 
 	  // listener created here will share a Connection Factory and ThreadFactory.
 	  // 
-	  for(UimaDefaultMessageListenerContainer listener : listeners ) {
-	      // Is this a Process listener instance? Check the selector
-		  if ( listener.getMessageSelector().endsWith(UimaDefaultMessageListenerContainer.PROCESS_SELECTOR_SUFFIX) ) {
-	          // this will be a dedicated listener which handles targeted messages
-			  UimaDefaultMessageListenerContainer targetedListener = new UimaDefaultMessageListenerContainer();
-			  // setup jms selector
-			  if ( getController().isCasMultiplier()) {
-				  targetedListener.setMessageSelector(UimaAsynchronousEngine.TargetSelectorProperty+" = '"+targetStringSelector+"' AND"+UimaDefaultMessageListenerContainer.CM_PROCESS_SELECTOR_SUFFIX);//(Command=2000 OR Command=2002)");
- 	          } else {
- 				  targetedListener.setMessageSelector(UimaAsynchronousEngine.TargetSelectorProperty+" = '"+targetStringSelector+"' AND"+UimaDefaultMessageListenerContainer.PROCESS_SELECTOR_SUFFIX);//(Command=2000 OR Command=2002)");
- 	          }
-			  
-			  // use shared ConnectionFactory
-	          targetedListener.setConnectionFactory(listener.getConnectionFactory());
-	          // mark the listener as a 'Targeted' listener
-	          targetedListener.setTargetedListener();
-	          targetedListener.setController(getController());
-	          // there will only be one AMQ delivery thread. Its job will be to
-	          // add a targeted message to a BlockingQueue. Such thread will block
-	          // in an enqueue if a dequeue is not available. This will be prevent
-	          // the overwhelming the service with messages.
-	          targetedListener.setConcurrentConsumers(1);
-			  if ( listener.getMessageListener() instanceof PriorityMessageHandler ) {
-				  // the targeted listener will use the same message handler as the
-				  // Process listener. This handler will add a message wrapper 
-				  // to enable prioritizing messages. 
-				  targetedListener.setMessageListener(listener.getMessageListener());
+	  for(Listener l : listeners ) {
+		  UimaDefaultMessageListenerContainer listener = 
+		     (UimaDefaultMessageListenerContainer) l;
+		  if ( listener.getTransport().equals(Transport.JMS)) {
+			  // Ignore listeners for replies (message selector = null)
+			  if ( listener.getMessageSelector() == null ) {
+				  continue;
 			  }
-			  // Same queue as the Process queue
-			  targetedListener.setDestination(listener.getDestination());
-	          registerListener(targetedListener);
-	          targetedListener.afterPropertiesSet();
-	          targetedListener.initialize();
-	          targetedListener.start();
-	          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
-	            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
-	                    "createListenerForTargetedMessages", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-	                    "UIMAJMS_TARGET_LISTENER__INFO",
-	                    new Object[] {targetedListener.getMessageSelector(), controller.getComponentName() });
-	          }
-	          break;
+		      // Is this a Process listener instance? Check the selector
+			  if ( listener.getMessageSelector().endsWith(UimaDefaultMessageListenerContainer.PROCESS_SELECTOR_SUFFIX) ) {
+		          // this will be a dedicated listener which handles targeted messages
+				  UimaDefaultMessageListenerContainer targetedListener = new UimaDefaultMessageListenerContainer();
+				  // setup jms selector
+				  if ( getController().isCasMultiplier()) {
+					  targetedListener.setMessageSelector(UimaAsynchronousEngine.TargetSelectorProperty+" = '"+targetStringSelector+"' AND"+UimaDefaultMessageListenerContainer.CM_PROCESS_SELECTOR_SUFFIX);//(Command=2000 OR Command=2002)");
+	 	          } else {
+	 				  targetedListener.setMessageSelector(UimaAsynchronousEngine.TargetSelectorProperty+" = '"+targetStringSelector+"' AND"+UimaDefaultMessageListenerContainer.PROCESS_SELECTOR_SUFFIX);//(Command=2000 OR Command=2002)");
+	 	          }
+				  
+				  // use shared ConnectionFactory
+		          targetedListener.setConnectionFactory(listener.getConnectionFactory());
+		          // mark the listener as a 'Targeted' listener
+		          targetedListener.setTargetedListener();
+		          targetedListener.setController(getController());
+		          // there will only be one delivery thread. Its job will be to
+		          // add a targeted message to a BlockingQueue. Such thread will block
+		          // in an enqueue if a dequeue is not available. This will be prevent
+		          // the overwhelming the service with messages.
+		  		  ThreadPoolTaskExecutor threadExecutor = new ThreadPoolTaskExecutor();
+				  threadExecutor.setCorePoolSize(1);
+				  threadExecutor.setMaxPoolSize(1);
+				  targetedListener.setTaskExecutor(threadExecutor);
+		          targetedListener.setConcurrentConsumers(1);
+				  if ( listener.getMessageListener() instanceof PriorityMessageHandler ) {
+					  // the targeted listener will use the same message handler as the
+					  // Process listener. This handler will add a message wrapper 
+					  // to enable prioritizing messages. 
+					  targetedListener.setMessageListener(listener.getMessageListener());
+				  }
+				  // Same queue as the Process queue
+				  targetedListener.setDestination(listener.getDestination());
+		          registerListener(targetedListener);
+		          targetedListener.afterPropertiesSet();
+		          targetedListener.setType(Type.Target);
+		          targetedListener.initialize();
+		          targetedListener.start();
+		          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
+		            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
+		                    "createListenerForTargetedMessages", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+		                    "UIMAJMS_TARGET_LISTENER__INFO",
+		                    new Object[] {targetedListener.getMessageSelector(), controller.getComponentName() });
+		          }
+		          break;
+
+		      }
+		  }
 
-	      }
 	  }
   }
   public void createListener(String aDelegateKey, Endpoint endpointToUpdate) throws Exception {
@@ -1190,7 +1271,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
         newListener.setMessageListener(this);
         newListener.setController(getController());
 
-        TempDestinationResolver resolver = new TempDestinationResolver(controller.getComponentName());
+        TempDestinationResolver resolver = new TempDestinationResolver(controller.getComponentName(), aDelegateKey);
         resolver.setConnectionFactory(f);
         resolver.setListener(newListener);
         newListener.setDestinationResolver(resolver);
@@ -1212,7 +1293,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
         }
         newListener.afterPropertiesSet();
         if ( controller != null && controller.isStopped() ) {
-          newListener.destroy(true);  // shutdownNow
+          stopListener(newListener);  // shutdownNow
           //  we are aborting, the controller has been stopped
           return;
         }
@@ -1231,11 +1312,13 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
     }
   }
   public boolean isListenerActiveOnDestination(Destination destination ) {
-   List<UimaDefaultMessageListenerContainer> ll = getListeners();
-   for (UimaDefaultMessageListenerContainer mListener : ll ) {
-     if ( mListener.getDestination() != null && 
-          mListener.getDestination() == destination &&
-          mListener.isRunning()) {
+   List<Listener> ll = getListeners();
+   for (Listener mListener : ll ) {
+	   UimaDefaultMessageListenerContainer lc =
+			   (UimaDefaultMessageListenerContainer)mListener;
+     if ( lc.getDestination() != null && 
+          lc.getDestination() == destination &&
+          lc.isRunning()) {
        return true;
      }
    }
@@ -1252,23 +1335,25 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
    * 
    * @return - list of listeners
    */
-  private UimaDefaultMessageListenerContainer[] getListenersForEndpoint(String anEndpointName) {
-	    List<UimaDefaultMessageListenerContainer> ll = getListeners();
+  private Listener[] getListenersForEndpoint(String anEndpointName) {
+	    List<Listener> ll = getListeners();
 
 	    
-	    List<UimaDefaultMessageListenerContainer> listeners = new ArrayList<UimaDefaultMessageListenerContainer>();
-	    for (UimaDefaultMessageListenerContainer mListener : ll) {
-	      if (mListener.getDestinationName() != null
-	              && mListener.getDestinationName().equals(anEndpointName)) {
+	    List<Listener> listeners = new ArrayList<>();
+	    for (Listener mListener : ll) {
+	    	UimaDefaultMessageListenerContainer lc = 
+	    			(UimaDefaultMessageListenerContainer)mListener;
+	      if (lc.getDestinationName() != null
+	              && lc.getDestinationName().equals(anEndpointName)) {
 	        listeners.add(mListener);
-	      } else if (mListener.getDestination() != null
-	              && mListener.getDestination().toString().equals(anEndpointName)) {
+	      } else if (lc.getDestination() != null
+	              && lc.getDestination().toString().equals(anEndpointName)) {
 	        listeners.add(mListener);
 	      }
 	    }
-	    if (listeners.size() > 0) {
-	      UimaDefaultMessageListenerContainer[] listenerArray = new UimaDefaultMessageListenerContainer[listeners
-	              .size()];
+	    if (!listeners.isEmpty()) {
+	      Listener[] listenerArray = new Listener[listeners.size()];
+	              
 	      listeners.toArray(listenerArray);
 	      return listenerArray;
 	    }
@@ -1285,34 +1370,36 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
       return;
     }
     // Fetch all associated listeners.
-    final UimaDefaultMessageListenerContainer[] mListeners = getListenersForEndpoint(anEndpointName);
+    final Listener[] mListeners = getListenersForEndpoint(anEndpointName);
     if (mListeners == null) {
       return;
     }
     // Stop each listener
-    for (final UimaDefaultMessageListenerContainer mListener : mListeners) {
-      if (!mListener.isRunning()) {
+    for (final Listener mListener : mListeners) {
+    	UimaDefaultMessageListenerContainer lc = 
+    			(UimaDefaultMessageListenerContainer)mListener;
+      if (!lc.isRunning()) {
         continue; // Already Stopped
       }
 
       try {
         if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)
-                && mListener.getDestination() != null) {
+                && lc.getDestination() != null) {
           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
                   "destroyListener", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
                   "UIMAJMS_stop_listener__INFO",
-                  new Object[] { mListener.getDestination().toString() });
+                  new Object[] { lc.getDestination().toString() });
         }
-        mListener.stop();
+        lc.stop();
         if (getController() != null) {
           Endpoint endpoint = ((AggregateAnalysisEngineController) getController()).lookUpEndpoint(
                   aDelegateKey, false);
           endpoint.setStatus(Endpoint.FAILED);
-          List<UimaDefaultMessageListenerContainer> ll = null;
+          List<Listener> ll = null;
           if ( listenerMap.containsKey(getController().getComponentName()+getController().hashCode()) ) {
             ll = listenerMap.get(getController().getComponentName()+getController().hashCode());
           }
-          if (mListener.getConnectionFactory() != null) {
+          if (lc.getConnectionFactory() != null) {
             if (getController() instanceof AggregateAnalysisEngineController) {
               if (!failedListenerMap.containsKey(aDelegateKey)) {
                 failedListenerMap.put(aDelegateKey, mListener);
@@ -1345,7 +1432,7 @@ public class JmsInputChannel implements InputChannel, JmsInputChannelMBean,
     }
   }
   public boolean isListenerForDestination(String anEndpointName) {
-    UimaDefaultMessageListenerContainer[] mListeners = getListenersForEndpoint(anEndpointName);
+    Listener[] mListeners = getListenersForEndpoint(anEndpointName);
     if (mListeners == null) {
       return false;
     }
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsOutputChannel.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsOutputChannel.java
index 7be0624..a7e0742 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsOutputChannel.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/JmsOutputChannel.java
@@ -60,6 +60,7 @@ import org.apache.uima.aae.UIMAEE_Constants;
 import org.apache.uima.aae.UimaSerializer;
 import org.apache.uima.aae.controller.AggregateAnalysisEngineController;
 import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.Endpoint;
 import org.apache.uima.aae.controller.LocalCache.CasStateEntry;
 import org.apache.uima.aae.controller.PrimitiveAnalysisEngineController;
@@ -87,7 +88,7 @@ import com.thoughtworks.xstream.XStream;
 
 public class JmsOutputChannel implements OutputChannel {
 
-  private static final Class CLASS_NAME = JmsOutputChannel.class;
+  private static final Class<?> CLASS_NAME = JmsOutputChannel.class;
 
   private static final long INACTIVITY_TIMEOUT = 5; // MINUTES
 
@@ -159,7 +160,7 @@ public class JmsOutputChannel implements OutputChannel {
     serverURI = aServerURI;
   }
 
-  protected void setFreeCasQueue(Destination destination) {
+  public void setFreeCasQueue(Destination destination) {
     freeCASTempQueue = destination;
   }
 
@@ -175,11 +176,12 @@ public class JmsOutputChannel implements OutputChannel {
    * 
    * @param connectionFactory
    */
+  /*
   public void setConnectionFactory(ActiveMQConnectionFactory connectionFactory) {
     this.connectionFactory = connectionFactory;
     this.connectionFactory.setTrustAllPackages(true);
   }
-
+*/
   public void setServiceInputEndpoint(String anEnpoint) {
     serviceInputEndpoint = anEnpoint;
   }
@@ -359,11 +361,12 @@ public class JmsOutputChannel implements OutputChannel {
     Connection conn = brokerConnectionEntry.getConnection();
     try {
     	if ( conn != null)
-       if ( conn != null && !((ActiveMQConnection)conn).isClosed()) {
+       if ( conn != null && ((ActiveMQConnection)conn).isClosed()) {
            for (Entry<Object, JmsEndpointConnection_impl> endpoints : brokerConnectionEntry.endpointMap
                    .entrySet()) {
               endpoints.getValue().close(); // close session and producer
            }
+           brokerConnectionEntry.getConnection().stop();
            brokerConnectionEntry.getConnection().close();
            brokerConnectionEntry.setConnection(null);
        }
@@ -446,6 +449,7 @@ public class JmsOutputChannel implements OutputChannel {
     
     try {
         connectionSemaphore.acquire();
+        /*
         //  If sending a Free Cas Request to a remote Cas Multiplier always use the CM's
         //  broker
         if ( anEndpoint.isFreeCasEndpoint() && anEndpoint.isCasMultiplier() && anEndpoint.isReplyEndpoint()) {
@@ -456,6 +460,9 @@ public class JmsOutputChannel implements OutputChannel {
           brokerConnectionURL = (anEndpoint.isReplyEndpoint()) ? serverURI : anEndpoint.getServerURI();
           
         }
+        */
+        brokerConnectionURL = (anEndpoint.isReplyEndpoint()) ? serverURI : anEndpoint.getServerURI();
+
         String key = getLookupKey(anEndpoint);
         String destination = getDestinationName(anEndpoint);
 
@@ -540,7 +547,7 @@ public class JmsOutputChannel implements OutputChannel {
                     //  Returns InputChannel if the Reply Listener for the delegate has previously failed.
                     //  If the listener hasnt failed the getReplyInputChannel returns null
 //                    InputChannel iC = getAnalysisEngineController().getReplyInputChannel(anEndpoint.getDelegateKey());
-                    InputChannel iC = getAnalysisEngineController().getReplyInputChannel(anEndpoint.getDestination().toString());
+                    InputChannel iC = getAnalysisEngineController().getInputChannel();
                     if ( iC != null ) { 
                       try {
                         // Create a new Listener, new Temp Queue and associate the listener with the Input Channel
@@ -713,9 +720,12 @@ public class JmsOutputChannel implements OutputChannel {
         
         case AsynchAEMessage.Process:
           logRequest("UIMAEE_service_sending_process_request__FINE", anEndpoint);
-          serializeCasAndSend(getAnalysisEngineController().
-                  getInProcessCache().
-                    getCacheEntryForCAS(aCasReferenceId), anEndpoint);
+          CasStateEntry stateEntry =
+        		  getAnalysisEngineController().getLocalCache().lookupEntry(aCasReferenceId);
+          serializeCasAndSend(stateEntry, anEndpoint);
+//          serializeCasAndSend(getAnalysisEngineController().
+//                  getInProcessCache().
+//                    getCacheEntryForCAS(aCasReferenceId), anEndpoint);
           return;  /// <<<<< RETURN - Done here >>>>
           
         
@@ -747,7 +757,7 @@ public class JmsOutputChannel implements OutputChannel {
     }
   }
   
-  private void serializeCasAndSend(CacheEntry entry, Endpoint anEndpoint) throws Exception {
+  private void serializeCasAndSend(CasStateEntry entry, Endpoint anEndpoint) throws Exception {
     if (anEndpoint.getSerialFormat() == SerialFormat.XMI) {
       String serializedCAS = getSerializedCasAndReleaseIt(false, entry.getCasReferenceId(), anEndpoint,
               anEndpoint.isRetryEnabled());
@@ -785,7 +795,8 @@ public class JmsOutputChannel implements OutputChannel {
   }
 
 
-  public void sendReply(CacheEntry entry, Endpoint anEndpoint) throws AsynchAEException {
+//  public void sendReply(CacheEntry entry, Endpoint anEndpoint) throws AsynchAEException {
+	  public void sendReply(CasStateEntry entry, Endpoint anEndpoint) throws AsynchAEException {
 	  try {
       anEndpoint.setReplyEndpoint(true);
       if (anEndpoint.isRemote()) {
@@ -844,14 +855,15 @@ public class JmsOutputChannel implements OutputChannel {
 	        // If this service is a Cas Multiplier add to the message a FreeCasQueue.
 	        // The client may need send Stop request to that queue.
 	        if (aCommand == AsynchAEMessage.ServiceInfo
-	                && getAnalysisEngineController().isCasMultiplier() ) {
-	          if ( freeCASTempQueue != null ) {
+	                && getAnalysisEngineController().isCasMultiplier()  ) {
+	        	if ( freeCASTempQueue != null ) {
 		        	// Attach a temp queue to the outgoing message. This a queue where
 		          // Free CAS notifications need to be sent from the client
 		          tm.setJMSReplyTo(freeCASTempQueue);
-	          }
+	          
+	        	}
 	          // new services will receive FreeCas request via a targeted queue
-	          StringBuffer selector = new StringBuffer().
+	          StringBuilder selector = new StringBuilder().
 	        		  append("TargetServiceId = ").
 	        		  append("'").append(hostIP).append(":").
 	        		  append(getAnalysisEngineController().getPID()).
@@ -1044,9 +1056,15 @@ public class JmsOutputChannel implements OutputChannel {
     }
     long msgSize = 0;
 
+    System.out.println("------------ Service:"+getAnalysisEngineController().getComponentName()+" Sending GetMeta Reply");
     ByteArrayOutputStream bos = new ByteArrayOutputStream();
     try {
-    	
+    	Object destination = anEndpoint.getReplyDestination();
+    	if ( destination == null ) {
+    		destination = anEndpoint.getDestination();
+    	}
+    	System.out.println(".......... Service:"+getAnalysisEngineController().getComponentName()+" replying with GetMeta to reply queue:"+destination);
+
     	
       anEndpoint.setReplyEndpoint(true);
       // Initialize JMS connection to given endpoint
@@ -1237,7 +1255,7 @@ public class JmsOutputChannel implements OutputChannel {
     CAS cas = null;
     try {
       String serializedCAS = null;
-      // Using Cas reference Id retrieve CAS from the shared Cash
+      // Using Cas reference Id retrieve CAS from the shared Cache
       cas = getAnalysisEngineController().getInProcessCache().getCasByReference(aCasReferenceId);
       ServicePerformance casStats = getAnalysisEngineController().getCasStatistics(aCasReferenceId);
       if (cas == null) {
@@ -1509,7 +1527,7 @@ public class JmsOutputChannel implements OutputChannel {
     this.controllerInputEndpoint = controllerInputEndpoint;
   }
 
-  private void dispatch(Message aMessage, Endpoint anEndpoint, CacheEntry entry, boolean isRequest,
+  private void dispatch(Message aMessage, Endpoint anEndpoint, CasStateEntry entry, boolean isRequest,
           JmsEndpointConnection_impl endpointConnection, long msgSize) throws Exception {
 	  
 	  if ( anEndpoint == null ) { 
@@ -1627,10 +1645,9 @@ public class JmsOutputChannel implements OutputChannel {
     }
   }
 
-  private void sendCasToRemoteEndpoint(boolean isRequest, Object aSerializedCAS, CacheEntry entry,
+  private void sendCasToRemoteEndpoint(boolean isRequest, Object aSerializedCAS, CasStateEntry casStateEntry,
           Endpoint anEndpoint, boolean startTimer) throws AsynchAEException,
           ServiceShutdownException {
-    CasStateEntry casStateEntry = null;
     long msgSize = 0;
     try {
       if (aborting) {
@@ -1639,9 +1656,7 @@ public class JmsOutputChannel implements OutputChannel {
       //  If this is a reply to a client, use the same broker URL that manages this service input queue.
       //  Otherwise this is a request so use a broker specified in the endpoint object.
       String brokerConnectionURL = (anEndpoint.isReplyEndpoint()) ? serverURI : anEndpoint.getServerURI();
-
-      casStateEntry = getAnalysisEngineController().getLocalCache().lookupEntry(
-              entry.getCasReferenceId());
+/*
       if (casStateEntry == null) {
         UIMAFramework.getLogger(CLASS_NAME).logrb(
                 Level.WARNING,
@@ -1651,12 +1666,12 @@ public class JmsOutputChannel implements OutputChannel {
                 "UIMAJMS_unable_to_send_reply__WARNING",
                 new Object[] { getAnalysisEngineController().getComponentName(),
                   anEndpoint.getDestination(), brokerConnectionURL, 
-                  entry.getInputCasReferenceId() == null ? "" : entry.getInputCasReferenceId(), 
-                          entry.getCasReferenceId(), 0, 
+                  casStateEntry.getParentCasReferenceId() == null ? "" : casStateEntry.getParentCasReferenceId(), 
+                		  casStateEntry.getCasReferenceId(), 0, 
                           new Exception("Unable to lookup entry in Local Cache for a given Cas Id")  });
         return;
       }
-
+*/
       // Get the connection object for a given endpoint
       JmsEndpointConnection_impl endpointConnection = getEndpointConnection(anEndpoint);
 
@@ -1689,19 +1704,22 @@ public class JmsOutputChannel implements OutputChannel {
                   JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
                   "UIMAJMS_unable_to_send_reply__WARNING",
                   new Object[] { getAnalysisEngineController().getComponentName(),
-                  	anEndpoint.getDestination(), brokerConnectionURL, entry.getInputCasReferenceId() == null ? "" : entry.getInputCasReferenceId(), entry.getCasReferenceId(), 0, ex  });
+                  	anEndpoint.getDestination(), brokerConnectionURL, casStateEntry.getParentCasReferenceId() == null ? "" : casStateEntry.getParentCasReferenceId(), casStateEntry.getCasReferenceId(), 0, ex  });
         return;
       }
       // Add Cas Reference Id to the outgoing JMS Header
-      tm.setStringProperty(AsynchAEMessage.CasReference, entry.getCasReferenceId());
+      tm.setStringProperty(AsynchAEMessage.CasReference, casStateEntry.getCasReferenceId());
       // Add common properties to the JMS Header
       if (isRequest == true) {
         populateHeaderWithRequestContext(tm, anEndpoint, AsynchAEMessage.Process);
       } else {
-        populateHeaderWithResponseContext(tm, anEndpoint, AsynchAEMessage.Process);   
-        tm.setBooleanProperty(AsynchAEMessage.SentDeltaCas, entry.sentDeltaCas());
+        populateHeaderWithResponseContext(tm, anEndpoint, AsynchAEMessage.Process);  
+        CacheEntry cacheEntry = 
+        		getAnalysisEngineController().
+        		     getInProcessCache().getCacheEntryForCAS(casStateEntry.getCasReferenceId());
+        tm.setBooleanProperty(AsynchAEMessage.SentDeltaCas, cacheEntry.sentDeltaCas());
       }
-      // The following is true when the analytic is a CAS Multiplier
+      // The following is true when the service is a CAS Multiplier
       if (casStateEntry.isSubordinate() && !isRequest) {
         // Override MessageType set in the populateHeaderWithContext above.
         // Make the reply message look like a request. This message will contain a new CAS
@@ -1710,10 +1728,10 @@ public class JmsOutputChannel implements OutputChannel {
         tm.setIntProperty(AsynchAEMessage.MessageType, AsynchAEMessage.Request);
         isRequest = true;
         // Save the id of the parent CAS
-        tm.setStringProperty(AsynchAEMessage.InputCasReference, getTopParentCasReferenceId(entry
+        tm.setStringProperty(AsynchAEMessage.InputCasReference, getTopParentCasReferenceId(casStateEntry
                 .getCasReferenceId()));
         // Add a sequence number assigned to this CAS by the controller
-        tm.setLongProperty(AsynchAEMessage.CasSequence, entry.getCasSequence());
+        tm.setLongProperty(AsynchAEMessage.CasSequence, casStateEntry.getSequenceNumber());
         // If this is a Cas Multiplier, add a reference to a special queue where
         // the client sends Free Cas Notifications
         if (freeCASTempQueue != null) {
@@ -1729,11 +1747,11 @@ public class JmsOutputChannel implements OutputChannel {
                   JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
                   "UIMAJMS_send_cas_to_collocated_service_detail__FINE",
                   new Object[] { getAnalysisEngineController().getComponentName(), "Remote",
-                      anEndpoint.getEndpoint(), entry.getCasReferenceId(),
-                      entry.getInputCasReferenceId(), entry.getInputCasReferenceId() });
+                      anEndpoint.getEndpoint(), casStateEntry.getCasReferenceId(),
+                      casStateEntry.getParentCasReferenceId(), casStateEntry.getParentCasReferenceId() });
         }
       }
-      dispatch(tm, anEndpoint, entry, isRequest, endpointConnection, msgSize);
+      dispatch(tm, anEndpoint, casStateEntry, isRequest, endpointConnection, msgSize);
 
     } catch (JMSException e) {
       // Unable to establish connection to the endpoint. Logit and continue
@@ -1757,21 +1775,25 @@ public class JmsOutputChannel implements OutputChannel {
 
   private Delegate lookupDelegate(String aDelegateKey) {
     if (getAnalysisEngineController() instanceof AggregateAnalysisEngineController) {
-      Delegate delegate = ((AggregateAnalysisEngineController) getAnalysisEngineController())
+      return ((AggregateAnalysisEngineController) getAnalysisEngineController())
               .lookupDelegate(aDelegateKey);
-      return delegate;
     }
     return null;
   }
 
-  private void addCasToOutstandingList(CacheEntry entry, boolean isRequest, String aDelegateKey) {
+  private void addCasToOutstandingList(CasStateEntry casStateEntry, boolean isRequest, String aDelegateKey) 
+  throws AsynchAEException {
     Delegate delegate = null;
     if (isRequest && (delegate = lookupDelegate(aDelegateKey)) != null) {
-      delegate.addCasToOutstandingList(entry.getCasReferenceId(), entry.getCas().hashCode(), false);  // false=dont start timer thread per CAS
+        CacheEntry cacheEntry = 
+        		getAnalysisEngineController().
+        		     getInProcessCache().getCacheEntryForCAS(casStateEntry.getCasReferenceId());
+
+      delegate.addCasToOutstandingList(casStateEntry.getCasReferenceId(), cacheEntry.getCas().hashCode(), false);  // false=dont start timer thread per CAS
     }
   }
 
-  private void removeCasFromOutstandingList(CacheEntry entry, boolean isRequest, String aDelegateKey) {
+  private void removeCasFromOutstandingList(CasStateEntry entry, boolean isRequest, String aDelegateKey) {
     Delegate delegate = null;
     if (isRequest && (delegate = lookupDelegate(aDelegateKey)) != null) {
       delegate.removeCasFromOutstandingList(entry.getCasReferenceId());
@@ -1787,7 +1809,7 @@ public class JmsOutputChannel implements OutputChannel {
 
     if (casStateEntry.isSubordinate()) {
       // Recurse until the top CAS reference Id is found
-      return getTopParentCasReferenceId(casStateEntry.getInputCasReferenceId());
+      return getTopParentCasReferenceId(casStateEntry.getParentCasReferenceId());
     }
     // Return the top ancestor CAS id
     return casStateEntry.getCasReferenceId();
@@ -2097,7 +2119,7 @@ public class JmsOutputChannel implements OutputChannel {
   private static class RecoveryThread implements Runnable {
     Endpoint endpoint;
 
-    CacheEntry entry;
+    CasStateEntry entry;
 
     boolean isRequest;
 
@@ -2105,7 +2127,7 @@ public class JmsOutputChannel implements OutputChannel {
 
     JmsOutputChannel outputChannel;
 
-    public RecoveryThread(JmsOutputChannel channel, Endpoint anEndpoint, CacheEntry anEntry,
+    public RecoveryThread(JmsOutputChannel channel, Endpoint anEndpoint, CasStateEntry anEntry,
             boolean isRequest, AnalysisEngineController aController) {
       endpoint = anEndpoint;
       entry = anEntry;
@@ -2124,9 +2146,8 @@ public class JmsOutputChannel implements OutputChannel {
         // Mark this delegate as Failed
         delegate.getEndpoint().setStatus(Endpoint.FAILED);
         // Destroy listener associated with a reply queue for this delegate
-        InputChannel ic = controller.getInputChannel(delegate.getEndpoint().getDestination()
-                .toString());
-        if (ic != null && delegate != null && delegate.getEndpoint() != null) {
+        InputChannel ic = controller.getInputChannel();
+        if (delegate.getEndpoint() != null) {
           ic.destroyListener(delegate.getEndpoint().getDestination().toString(), endpoint
                   .getDelegateKey());
         }
@@ -2142,4 +2163,7 @@ public class JmsOutputChannel implements OutputChannel {
 
     }
   }
+  public ENDPOINT_TYPE getType() {
+		return ENDPOINT_TYPE.JMS;
+  }
 }
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/TempDestinationResolver.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/TempDestinationResolver.java
index 10d888a..53ea069 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/TempDestinationResolver.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/TempDestinationResolver.java
@@ -35,10 +35,13 @@ public class TempDestinationResolver implements DestinationResolver {
 
   private String serviceName = "";
   
+  private String endpoint = "";
+  
   public TempDestinationResolver() {
   }
-  public TempDestinationResolver(String name) {
+  public TempDestinationResolver(String name, String endpoint) {
 	  serviceName = name;
+	  this.endpoint = endpoint;
   }
   /**
    * This method is called by the Spring listener code. It creates a single temp queue for all
@@ -48,7 +51,9 @@ public class TempDestinationResolver implements DestinationResolver {
    */
   public Destination resolveDestinationName(Session session, String destinationName,
           boolean pubSubDomain) throws JMSException {
-    synchronized (mutex) {
+	  System.out.println("************ resolveDestinationName() Controller:"+serviceName+" Endpoint:"+endpoint+"************************");
+
+	  synchronized (mutex) {
       if (destination == null) {
         destination = session.createTemporaryQueue();
         if (listener != null) {
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/UimaDefaultMessageListenerContainer.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/UimaDefaultMessageListenerContainer.java
index ddd98f8..4cff60c 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/UimaDefaultMessageListenerContainer.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/activemq/UimaDefaultMessageListenerContainer.java
@@ -54,9 +54,11 @@ import org.apache.uima.aae.UimaAsPriorityBasedThreadFactory;
 //import org.apache.uima.aae.UimaASCredentials;
 import org.apache.uima.aae.UimaAsThreadFactory;
 import org.apache.uima.aae.client.UimaAsynchronousEngine;
+import org.apache.uima.aae.client.UimaAsynchronousEngine.Transport;
 //import org.apache.uima.aae.UimaAsThreadFactory.UsedFor;
 import org.apache.uima.aae.controller.AggregateAnalysisEngineController;
 import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.BaseAnalysisEngineController.ServiceState;
 import org.apache.uima.aae.controller.Endpoint;
 import org.apache.uima.aae.controller.PrimitiveAnalysisEngineController;
@@ -68,6 +70,7 @@ import org.apache.uima.aae.message.AsynchAEMessage;
 import org.apache.uima.aae.message.MessageWrapper;
 import org.apache.uima.adapter.jms.JmsConstants;
 import org.apache.uima.adapter.jms.activemq.JmsOutputChannel.BrokerConnectionEntry;
+import org.apache.uima.as.client.Listener;
 import org.apache.uima.resource.ResourceInitializationException;
 import org.apache.uima.util.Level;
 import org.springframework.core.task.TaskExecutor;
@@ -80,12 +83,15 @@ import org.springframework.jms.support.destination.DestinationResolver;
 import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
 
 public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerContainer implements
-        ExceptionListener {
+        Listener, ExceptionListener {
   private static final Class<?> CLASS_NAME = UimaDefaultMessageListenerContainer.class;
   public static final String PROCESS_SELECTOR_SUFFIX = "(Command=2000 OR Command=2002)";
   public static final String CM_PROCESS_SELECTOR_SUFFIX = "(Command=2000 OR Command=2002 OR Command=2005)";
   public static final String GETMETA_SELECTOR_SUFFIX = "(Command=2001)";
   
+  private Transport transport = Transport.JMS;
+  private Type type = Type.Unknown;
+  
   public static final int HIGH_PRIORITY = 9;
   
   private String destinationName = "";
@@ -153,12 +159,23 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     UIMAFramework.getLogger(CLASS_NAME).setLevel(Level.WARNING);
     __listenerRef = this;
     setRecoveryInterval(400);  // increase connection recovery to 30 sec
-    setAcceptMessagesWhileStopping(true);
+    //setAcceptMessagesWhileStopping(true);
     setExceptionListener(this);
     threadGroup = new ThreadGroup("ListenerThreadGroup_"
             + Thread.currentThread().getThreadGroup().getName());
   }
-
+  public Transport getTransport() {
+	  return transport;
+  }
+  public void setType(Type t) {
+	  this.type = t;
+  }
+  public Type getType() {
+	  return type;
+  }
+  protected void handleListenerException(Throwable t) {
+	  t.printStackTrace();
+  }
   public UimaDefaultMessageListenerContainer(boolean freeCasQueueListener) {
     this();
     this.freeCasQueueListener = freeCasQueueListener;
@@ -183,7 +200,38 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
    */
   protected void refreshConnectionUntilSuccessful() {
 	 // System.out.println("............refreshConnectionUntilSuccessful() called");
-
+	  while (isRunning()) {
+
+			try {
+				
+				if (sharedConnectionEnabled()) {
+					refreshSharedConnection();
+				} else {
+					Connection con = createConnection();
+					JmsUtils.closeConnection(con);
+				}
+				// Use UIMA-AS custom Destination Resolver to create a new temp queue for this reply listener
+				if( Type.Reply.equals(type) || Type.FreeCAS.equals(type)) {
+					getDestinationResolver().resolveDestinationName(getSharedConnection().createSession(false, Session.AUTO_ACKNOWLEDGE),"",false);
+				}
+				logger.info(getType().name()+" Listener Successfully refreshed JMS Connection to broker: "+getBrokerUrl()+" Endpoint:"+getDestination());
+				logListenerFailure = true;
+				break;
+			}
+			catch (Exception ex) {
+				
+				if (ex instanceof JMSException) {
+					if (logListenerFailure && ex.getCause() instanceof ConnectException) {
+						
+						logger.info(getType().name()+" Listener lost connection to broker: "+getBrokerUrl()+"- Retrying until successfull ...");
+						logListenerFailure = false;
+					} else {
+						invokeExceptionListener((JMSException) ex);
+					}
+				}
+			}
+	  }
+	  /*
 	  boolean doLogFailureMsg = true;
     try {
     	// Only one listener thread should enter to recover lost connection.
@@ -244,30 +292,11 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
        	            		}
     	                   }
     	                   
-    	                       	            	/*  
-    	               	if ( getMessageListener() instanceof JmsInputChannel ) {
-    	            		System.out.println("------------------- Creating new listener for the temp queue");
-    	            		try {
-    	                	((JmsInputChannel)getMessageListener()).createListenerOnTempQueue(getConnectionFactory());
-    	                		System.out.println("------------------- New listener on temp queue is ready");
-    	            		} catch( Exception e) {
-    	                		System.out.println("------------------- Error while creating new listener on temp queue");
-    	            	          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-    	            	              UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
-    	            	                      "handleTempQueueFailure", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-    	            	                      "UIMAJMS_exception__WARNING", e);
-    	            	            }
-    	            			
-    	            		}
-    	            	}
-*/
+
     	               
     	               }
     	               
-    	  /*
-    	               String delegateKey = ((AggregateAnalysisEngineController) controller)
-    	                .lookUpDelegateKey(endpoint.getEndpoint());
-    		*/
+ 
     		        //}
     		        break;
     		      }
@@ -293,6 +322,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     	}
     } catch( IllegalStateException e ) {
     }
+    */
   }
   protected void recoverAfterListenerSetupFailure() {
 	  if ( !terminating ) {
@@ -323,6 +353,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   /**
    * Stops this Listener
    */
+  /*
   private void handleListenerFailure() {
     // If shutdown already, nothing to do
     if (awaitingShutdown) {
@@ -363,12 +394,13 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
       }
     }
   }
-
+*/
   /**
    * Handles failure on a temp queue
    * 
    * @param t
    */
+  /*
   private void handleTempQueueFailure(Throwable t) {
     if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
       if ( controller != null ) {
@@ -463,7 +495,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     } else {
     }
   }
-
+*/
   private ErrorHandler fetchGetMetaErrorHandler() {
     ErrorHandler handler = null;
     Iterator it = controller.getErrorHandlerChain().iterator();
@@ -483,6 +515,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
    * 
    * @param t
    */
+  /*
   private void handleQueueFailure(Throwable t) {
 	//  System.out.println("............handleQueueFailure() called");
     final String endpointName = (getDestination() == null) ? ""
@@ -598,7 +631,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     }
 
   }
-
+*/
   /**
    * This method is called by Spring when a listener fails
    */
@@ -609,9 +642,10 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     }
     // If shutdown already, nothing to do
 	    // If controller is stopping no need to recover the connection
-    if (awaitingShutdown || terminating || (controller != null && controller.isStopped()) ) {
+    if (!super.isRunning() || awaitingShutdown || terminating || (controller != null && controller.isStopped()) ) {
       return;
     }
+    /*
     if ( controller != null ) {
       controller.changeState(ServiceState.FAILED);
     }
@@ -676,10 +710,12 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
       }
       failed = true;
     }
+    */
   }
   public Endpoint getEndpoint() {
 	  return endpoint;
   }
+  /*
   private void terminate(Throwable t) {
     // ****************************************
     // terminate the service
@@ -695,7 +731,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
       controller.stop();
     }
   }
-
+  
   protected void handleListenerException(Throwable t) {
 	 // System.out.println("............handleListenerException(Throwable t)");
 	  
@@ -703,17 +739,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     if (awaitingShutdown) {
       return;
     }
-    /*
-    String endpointName = (getDestination() == null) ? ""
-            : ((ActiveMQDestination) getDestination()).getPhysicalName();
-
-    if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-      UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, this.getClass().getName(),
-              "handleListenerException", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-              "UIMAJMS_jms_listener_failed_WARNING",
-              new Object[] { endpointName, getBrokerUrl(), t });
-    }
-    */
+   
     super.handleListenerException(t);
   }
 
@@ -731,14 +757,16 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     super.setConnectionFactory(connectionFactory);
   }
 
-
+*/
   public boolean isGetMetaListener() {
-	  
+	  return Type.GetMeta.equals(type);
+	  /*
     return getMessageSelector() != null
             && __listenerRef.getMessageSelector().endsWith(GETMETA_SELECTOR_SUFFIX);
 //    && __listenerRef.getMessageSelector().endsWith("(Command=2001)");
+ */
   }
-  
+  /*
   private boolean isActiveMQDestination() {
     return getDestination() != null && getDestination() instanceof ActiveMQDestination;
   }
@@ -762,24 +790,26 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
       }
     }
   }
-
+*/
   /**
    * Intercept Spring call to increment number of consumer threads. If the value > 1, don't
    * propagate to Spring. A new listener will be injected and it will use provided number of
    * consumer threads.
    **/
+  /*
   public void setConcurrentConsumers(int concurrentConsumers) {
     cc = concurrentConsumers;
     if (this.freeCasQueueListener) {
       super.setConcurrentConsumers(concurrentConsumers);
     }
   }
-
+*/
   /**
    * Intercept Spring call to inject application Pojo listener. Don't propagate the listener up to
    * Spring just yet. If more than one consumer thread is used, a different listener will be
    * injected.
    **/
+  /*
   public void setMessageListener(Object messageListener) {
     ml = messageListener;
     if (this.freeCasQueueListener || targetedListener ) {
@@ -791,10 +821,12 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   public void afterPropertiesSet() {
     afterPropertiesSet(true);
   }
+  */
   /**
    * Called by Spring and some Uima AS components when all properties have been set. This method
    * spins a thread in which the listener is initialized.
    */
+  /*
   public void afterPropertiesSet(final boolean propagate) {
     if (endpoint != null) {
 			// Override the prefetch size. The dd2spring always sets this to 1 which 
@@ -938,12 +970,13 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     });
     t.start();
   }
-
+*/
   /**
    * Inject instance of this listener into the InputChannel
    * 
    * @throws Exception
    */
+  /*
   private void connectWithInputChannel() throws Exception {
     Object pojoListener = getPojoListener();
 
@@ -961,7 +994,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
       ((ModifiableListener) pojoListener).setListener(__listenerRef);
     }
   }
-
+*/
   public String getDestinationName() {
 
     return destinationName;
@@ -975,7 +1008,9 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   }
 
   public String getBrokerUrl() {
-    return ((ActiveMQConnectionFactory) connectionFactory).getBrokerURL();
+		return ((ActiveMQConnectionFactory)super.getConnectionFactory()).getBrokerURL();
+
+//    return ((ActiveMQConnectionFactory) connectionFactory).getBrokerURL();
   }
 
   /*
@@ -984,28 +1019,17 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
    * deployment descriptor and create a new one with rewritten Broker URL. We will inject the
    * prefetch policy to the new CF based on what is found in the CF in the deployment descriptor.
    */
-
+/*
   public void setConnectionFactory(ConnectionFactory aConnectionFactory)  {
     connectionFactory = aConnectionFactory;
-    /*
-    if ( System.getProperty("uima.as.broker.credentials.file") != null ) {
-    	UimaASCredentials credentials = new UimaASCredentials();
-    	try {
-        	credentials.readCredentials(System.getProperty("uima.as.broker.credentials.file"));
-    	} catch( IOException e) {
-    		throw new RuntimeException(e);
-    	}
-    	((ActiveMQConnectionFactory)connectionFactory).setUserName(credentials.getUsername());
-        ((ActiveMQConnectionFactory)connectionFactory).setPassword(credentials.getPassword());
-    }
-    */
+   
     ConnectionFactoryIniter cfIniter =
             new ConnectionFactoryIniter((ActiveMQConnectionFactory)connectionFactory);
     cfIniter.whiteListPackages();
     ((ActiveMQConnectionFactory)connectionFactory).setTrustAllPackages(true);
     super.setConnectionFactory(connectionFactory);
   }
-
+*/
   public void setDestinationResolver(DestinationResolver resolver) {
     ((TempDestinationResolver) resolver).setListener(this);
     super.setDestinationResolver(resolver);
@@ -1057,8 +1081,17 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
 
   public void setDestination(Destination aDestination) {
     super.setDestination(aDestination);
+   
+    System.out.println("............................. "+endpoint+" Destination:"+aDestination.toString());
+    if ( Type.FreeCAS.equals(type)) {
+    	((JmsOutputChannel)controller.getOutputChannel(ENDPOINT_TYPE.JMS)).setFreeCasQueue(aDestination);
+    }
     if (endpoint != null) {
       endpoint.setDestination(aDestination);
+      if (aDestination instanceof TemporaryQueue ) {
+    	  endpoint.setTempReplyDestination(true);
+      }
+      /*
       //  Get the prefetch size. If > 1, it has been previously overriden. The override is done in
       // the code since dd2spring alwys sets the prefetch on a reply queue to 1. This may slow down
       // a throughput of a service.
@@ -1079,10 +1112,28 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
           ((JmsInputChannel) pojoListener).setListenerContainer(this);
         }
       }
+      */
       endpoint.setServerURI(getBrokerUrl());
     }
   }
+  public void start() {
+	  if ( isRunning()) {
+		  return;
+	  }
+	  int consumerThreadCount=-1;
+	  if ( getTaskExecutor() instanceof ThreadPoolTaskExecutor) {
+		  ((ThreadPoolTaskExecutor)getTaskExecutor()).initialize();
+		  // if this listener is a handling Process requests, the prestartAllCoreThreads() below
+		  // will force initialization of AEs if this is a primitive service.
+		  ((ThreadPoolTaskExecutor)getTaskExecutor()).getThreadPoolExecutor().prestartAllCoreThreads();
+	  }
+	  super.afterPropertiesSet();
+	  super.initialize();
+	  super.start();
+	  
+      System.out.println(">>>>>>> Listener Service:"+controller.getComponentName()+" Broker URL:"+getBrokerUrl()+" Endpoint:"+__listenerRef.getEndpoint()+" ConsumerThreadCount:"+consumerThreadCount);
 
+  }
   private Object getPojoListener() {
     Object pojoListener = null;
     if (ml != null) {
@@ -1098,6 +1149,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   }
 
   public void onException(JMSException arg0) {
+	  /*
     if (awaitingShutdown) {
       return;
     }
@@ -1121,7 +1173,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     if ( getDestination() != null && ((ActiveMQDestination)getDestination()).isTemporary() ) {
       handleTempQueueFailure(arg0);
     }
-  
+  */
   }
 
   public void setTargetEndpoint(Endpoint anEndpoint) {
@@ -1129,7 +1181,8 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   }
 
   public boolean isFreeCasQueueListener() {
-    return freeCasQueueListener;
+	  return Type.FreeCAS.equals(type);
+    //return freeCasQueueListener;
   }
 
   protected void setModifiedTaskExecutor(TaskExecutor taskExecutor) {
@@ -1185,6 +1238,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
                   "UIMAJMS_exception__WARNING", t);
 	  }
   }
+  /*
   public void shutdownTaskExecutor(ThreadPoolExecutor tpe, boolean stopImmediate) throws InterruptedException {
     tpe.awaitTermination(50, TimeUnit.MILLISECONDS);
     
@@ -1203,10 +1257,12 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
   public void destroy() {
 	  destroy(true); 
   }
+  */
   /**
    * Spins a shutdown thread and stops Sprint and ActiveMQ threads.
    * 
    */
+  /*
   public void destroy(final boolean stopImmediate) {
 	  
     if (awaitingShutdown) {
@@ -1432,26 +1488,9 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
           super.setTaskExecutor(es);
       }
     }
-    /*
-    else {
-        UimaAsThreadFactory tf = new UimaAsThreadFactory(threadGroup);
-        tf.setDaemon(true);
-        if ( isFreeCasQueueListener()) {
-          tf.setThreadNamePrefix(controller.getComponentName()+" - FreeCASRequest Thread");
-        } else if ( isGetMetaListener()  ) {
-          tf.setThreadNamePrefix(super.getBeanName()+" - Thread");
-        } else if ( getDestination() != null && getMessageSelector() != null ) {
-          tf.setThreadNamePrefix(controller.getComponentName() + " Process Thread");
-        } else if ( endpoint != null && endpoint.isTempReplyDestination() ) {
-          tf.setThreadNamePrefix(super.getBeanName()+" - Thread");
-        } else { 
-          throw new Exception("Unknown Context Detected in setUimaASThreadPoolExecutor()");
-        }
-        
-    }
-    */
+   
   }
-
+*/
   private boolean isPrimitiveService() {
 	  return controller != null && controller instanceof PrimitiveAnalysisEngineController &&
 			  controller.getInputChannel() != null;
@@ -1462,6 +1501,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
    */
   public void setTaskExecutor(TaskExecutor aTaskExecutor) {
     taskExecutor = aTaskExecutor;
+    super.setTaskExecutor(aTaskExecutor);
   }
 
   public TaskExecutor getTaskExecutor() {
@@ -1476,6 +1516,7 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
    * 
    * @throws Exception
    */
+  /*
   private void initializeTaskExecutor(int consumers) throws Exception {
     // TaskExecutor is only used with primitives
     if (controller instanceof PrimitiveAnalysisEngineController) {
@@ -1493,10 +1534,18 @@ public class UimaDefaultMessageListenerContainer extends DefaultMessageListenerC
     	threadPoolExecutor.prestartAllCoreThreads();
     }
   }
+  */
   public void delegateStop() {
     super.stop();
   }
+  /*
   public void stop() throws JmsException {
     destroy();
   }
+  */
+  @Override
+  public String getName() {
+  	// TODO Auto-generated method stub
+  	return null;
+  }
 }
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/ActiveMQMessageSender.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/ActiveMQMessageSender.java
index 3691a5d..0ec5a1f 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/ActiveMQMessageSender.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/ActiveMQMessageSender.java
@@ -44,6 +44,7 @@ import org.apache.uima.adapter.jms.JmsConstants;
 import org.apache.uima.adapter.jms.client.BaseUIMAAsynchronousEngineCommon_impl.ClientRequest;
 import org.apache.uima.adapter.jms.client.BaseUIMAAsynchronousEngineCommon_impl.SharedConnection;
 import org.apache.uima.adapter.jms.message.PendingMessage;
+import org.apache.uima.adapter.jms.message.PendingMessageImpl;
 import org.apache.uima.cas.CAS;
 import org.apache.uima.cas.SerialFormat;
 import org.apache.uima.util.Level;
@@ -284,11 +285,11 @@ public class ActiveMQMessageSender extends BaseMessageSender {
           // instead of a temp queue. Regular queues can be recovered in case of
           // a broker restart. The test below will be true for UIMA-AS v. 2.10.0 +.
           // Code in JmsOutputChannel will add the selector if the service is a CM.
-          if (pm.get(AsynchAEMessage.TargetingSelector) != null) {
-        	  selector = (String)pm.get(AsynchAEMessage.TargetingSelector);
+          if (pm.getPropertyAsString(AsynchAEMessage.TargetingSelector) != null) {
+        	  selector = (String)pm.getPropertyAsString(AsynchAEMessage.TargetingSelector);
           }
           if ( selector == null && (pm.getMessageType() == AsynchAEMessage.ReleaseCAS || pm.getMessageType() == AsynchAEMessage.Stop) ) {
-        	  d = (Destination)pm.get(AsynchAEMessage.Destination);
+        	  d = (Destination)pm.getProperty(AsynchAEMessage.Destination);
               
           } else {
               d = jmsSession.createQueue(destinationName);
@@ -313,7 +314,7 @@ public class ActiveMQMessageSender extends BaseMessageSender {
           }
           if (casProcessRequest) {
             cacheEntry = (ClientRequest) engine.getCache().get(
-                    pm.get(AsynchAEMessage.CasReference));
+                    pm.getPropertyAsString(AsynchAEMessage.CasReference));
             if (cacheEntry != null) {
             //    CAS cas = cacheEntry.getCAS();
                 // enable logging 
@@ -356,7 +357,7 @@ public class ActiveMQMessageSender extends BaseMessageSender {
                           JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
                           "UIMAJMS_failed_cache_lookup__WARNING",
                           new Object[] {
-                         	 pm.get(AsynchAEMessage.CasReference),
+                         	 pm.getPropertyAsString(AsynchAEMessage.CasReference),
                               UimaMessageValidator.decodeIntToString(AsynchAEMessage.Command, message
                                       .getIntProperty(AsynchAEMessage.Command)),
                               UimaMessageValidator.decodeIntToString(AsynchAEMessage.MessageType, message
@@ -403,7 +404,7 @@ public class ActiveMQMessageSender extends BaseMessageSender {
                         JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
                         "UIMAJMS_calling_onBeforeMessageSend__FINE",
                         new Object[] {
-                          pm.get(AsynchAEMessage.CasReference),
+                          pm.getPropertyAsString(AsynchAEMessage.CasReference),
                           String.valueOf(cacheEntry.getCAS().hashCode())
                         });
               }  
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/BaseUIMAAsynchronousEngine_impl.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/BaseUIMAAsynchronousEngine_impl.java
index c805979..7a39aab 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/BaseUIMAAsynchronousEngine_impl.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/client/BaseUIMAAsynchronousEngine_impl.java
@@ -20,10 +20,15 @@
 package org.apache.uima.adapter.jms.client;
 
 import java.io.File;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -74,13 +79,24 @@ import org.apache.uima.aae.error.UimaASMetaRequestTimeout;
 import org.apache.uima.aae.jmx.JmxManager;
 import org.apache.uima.aae.message.AsynchAEMessage;
 import org.apache.uima.aae.message.UIMAMessage;
+import org.apache.uima.aae.service.AsynchronousUimaASService;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.aae.service.UimaAsServiceRegistry;
+import org.apache.uima.aae.service.builder.UimaAsDirectServiceBuilder;
 import org.apache.uima.adapter.jms.JmsConstants;
 import org.apache.uima.adapter.jms.activemq.ConnectionFactoryIniter;
 import org.apache.uima.adapter.jms.activemq.SpringContainerDeployer;
 import org.apache.uima.adapter.jms.activemq.UimaEEAdminSpringContext;
 import org.apache.uima.adapter.jms.message.PendingMessage;
+import org.apache.uima.adapter.jms.message.PendingMessageImpl;
 import org.apache.uima.adapter.jms.service.Dd2spring;
 import org.apache.uima.analysis_engine.metadata.AnalysisEngineMetaData;
+import org.apache.uima.as.client.DirectMessage;
+import org.apache.uima.as.deployer.ServiceDeployers;
+import org.apache.uima.as.deployer.ServiceDeployers.Protocol;
+import org.apache.uima.as.deployer.ServiceDeployers.Provider;
+import org.apache.uima.as.deployer.UimaAsServiceDeployer;
+import org.apache.uima.as.dispatcher.LocalDispatcher;
 import org.apache.uima.cas.CAS;
 import org.apache.uima.cas.SerialFormat;
 import org.apache.uima.impl.UimaVersion;
@@ -90,11 +106,15 @@ import org.apache.uima.resource.ResourceConfigurationException;
 import org.apache.uima.resource.ResourceInitializationException;
 import org.apache.uima.resource.ResourceManager;
 import org.apache.uima.resource.ResourceProcessException;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
 import org.apache.uima.util.Level;
+import org.apache.xmlbeans.XmlDocumentProperties;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationEvent;
 import org.springframework.context.ApplicationListener;
 import org.springframework.context.support.FileSystemXmlApplicationContext;
+import org.xml.sax.XMLReader;
+import org.xml.sax.helpers.XMLReaderFactory;
 
 public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineCommon_impl
         implements UimaAsynchronousEngine, MessageListener, ControllerCallbackListener, ApplicationListener<ApplicationEvent>{
@@ -138,7 +158,12 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
   
   protected static Lock globalLock = new ReentrantLock();
   
-  //private String serviceTargetSelector = null;
+  protected UimaASService service = null;
+  
+  ExecutorService consumerService = null;
+  private int casPoolSize = 1;
+  
+  private Thread dispatchThread;
   
   protected volatile boolean stopped = false;
   public BaseUIMAAsynchronousEngine_impl() {
@@ -148,7 +173,11 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
     " UIMA-AS Version " + UimaAsVersion.getVersionString());
   }
 
-
+  protected void beforeProcessReply(String casReferenceId) {
+	  if ( service != null ) {
+		  service.removeFromCache(casReferenceId);
+	  }
+  }
   protected TextMessage createTextMessage() throws ResourceInitializationException {
     return new ActiveMQTextMessage();
   }
@@ -352,20 +381,30 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
   }
 	public void stop() {
 		try {
-			  if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(
-						Level.INFO)) {
-			     UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO,
-							CLASS_NAME.getName(), "stop",
-							JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-							"UIMAJMS_stopping_as_client_INFO");
-		      }
-			  stopConnection();
+			  if ( brokerURI != null && !brokerURI.equals("java")) {
+				  if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(
+							Level.INFO)) {
+				     UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO,
+								CLASS_NAME.getName(), "stop",
+								JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+								"UIMAJMS_stopping_as_client_INFO");
+			      }
+				  stopConnection();
+			  }
+
 
 		      super.doStop();
 		      if (!running) {
 		        return;
 		      }
 		      running = false;
+		      if ( consumerService != null ) {
+		    	  DirectMessage poisonPillMsg = new DirectMessage().withCommand(AsynchAEMessage.Stop);
+		    	  ((AsynchronousUimaASService)service).getReplyQueue().put(poisonPillMsg);
+		    	  // stop thread pool for local queue listener
+		    	 // consumerService.shutdown();
+		    	  consumerService.shutdown();
+		      }
 			  if (super.serviceDelegate != null) {
 				// Cancel all timers and purge lists
 				super.serviceDelegate.cleanup();
@@ -378,6 +417,9 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
 //				stopConnection();
 				// Undeploy all containers
 				undeploy();
+				if ( dispatchThread != null ) {
+					dispatchThread.interrupt();
+				}
 		 	    clientCache.clear();
 				if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(
 								Level.INFO)) {
@@ -407,7 +449,7 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
 		} 
 	}
 
-  protected void setCPCMessage(Message msg) throws Exception {
+  public void setCPCMessage(Message msg) throws Exception {
     msg.setStringProperty(AsynchAEMessage.MessageFrom, consumerDestination.getQueueName());
     msg.setStringProperty(UIMAMessage.ServerURI, brokerURI);
     msg.setIntProperty(AsynchAEMessage.MessageType, AsynchAEMessage.Request);
@@ -419,6 +461,183 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
       ((TextMessage) msg).setText("");
     }
   }
+  protected boolean isServiceRemote() {
+	  return transport.equals(Transport.JMS);
+//	  return (service instanceof UimaASJmsService);
+//	  return service == null;
+  }
+  private void startLocalConsumer(Map anApplicationContext) {
+
+	  consumerService = Executors.newFixedThreadPool(1);
+	  	  consumerService.execute( new Runnable() {
+		      
+		      public void run() {
+		        try {
+		        	while( running ) {
+			        	DirectMessage message =
+				    			  ((AsynchronousUimaASService)service).getReplyQueue().take();
+			        	if ( message.getAsInt(AsynchAEMessage.Command) == AsynchAEMessage.Stop) {
+				        	System.out.println("BaseUIMAAsynchronousEngine_impl.startLocalConsumer().run() - Direct Consumer Recv'd Stop Msg - Terminating");
+			        		return;
+			        	}
+			        	System.out.println("Client Direct Local Consumer() Recv'd Reply");
+			        	onMessage(message);
+		        	}
+
+		        } catch( InterruptedException e) {
+		        	System.out.println("BaseUIMAAsynchronousEngine_impl.startLocalConsumer().run() - Stopped Direct Consumer");
+		        	return;
+		        	
+		        } catch( Exception e) {
+		        	e.printStackTrace();
+		        	
+		        }
+		      }
+	   });
+	  
+	  
+  }
+  private void initializeLocal(Map anApplicationContext) throws ResourceInitializationException {
+    if ( dispatchThread == null ) {
+  	  // make sure we are in the running state. The local consumer depends on it
+      	running = true;
+      
+    	// start message consumer to handle replies  
+    	startLocalConsumer(anApplicationContext);
+    	// start dispatcher in its own thread. It will fetch messages from a shared 'pendingMessageQueue'
+      	LocalDispatcher dispatcher =
+      			new LocalDispatcher(this, service, pendingMessageQueue);
+      	dispatchThread = new Thread(dispatcher);
+      	dispatchThread.start();
+    }
+
+  }
+  private void initializeJMS(Map anApplicationContext) throws ResourceInitializationException {
+      if (!anApplicationContext.containsKey(UimaAsynchronousEngine.ServerUri)) {
+          throw new ResourceInitializationException();
+        }
+        if (!anApplicationContext.containsKey(UimaAsynchronousEngine.ENDPOINT)) {
+          throw new ResourceInitializationException();
+        }
+        if (anApplicationContext.containsKey(UimaAsynchronousEngine.SERIALIZATION_STRATEGY)) {
+            final String serializationStrategy = (String) anApplicationContext.get(UimaAsynchronousEngine.SERIALIZATION_STRATEGY);
+            // change this to support compressed filitered as the default
+            setSerialFormat((serializationStrategy.equalsIgnoreCase("xmi")) ? SerialFormat.XMI : SerialFormat.BINARY);
+            clientSideJmxStats.setSerialization(getSerialFormat());
+        }
+        if (anApplicationContext.containsKey(UimaAsynchronousEngine.userName)) {
+            amqUser = (String) anApplicationContext
+                    .get(UimaAsynchronousEngine.userName);
+        }
+        if (anApplicationContext.containsKey(UimaAsynchronousEngine.password)) {
+        	amqPassword = (String) anApplicationContext
+        	.get(UimaAsynchronousEngine.password);
+        }
+        
+        brokerURI = (String) anApplicationContext.get(UimaAsynchronousEngine.ServerUri);
+        endpoint = (String) anApplicationContext.get(UimaAsynchronousEngine.ENDPOINT);
+        
+        //  Check if a placeholder is passed in instead of actual broker URL or endpoint. 
+        //  The placeholder has the syntax ${placeholderName} and may be imbedded in text.
+        //  A system property with placeholderName must exist for successful placeholder resolution.
+        //  Throws ResourceInitializationException if placeholder is not in the System properties.
+        brokerURI = replacePlaceholder(brokerURI); 
+        endpoint = replacePlaceholder(endpoint); 
+        // Check if sharedConnection exists. If not create a new one. The sharedConnection
+        // is static and shared by all instances of UIMA AS client in a jvm. The check
+        // is made in a critical section by first acquiring a global static semaphore to
+        // prevent a race condition.
+        try {
+            createSharedConnection(brokerURI);
+            running = true;
+            //  This is done to give the broker enough time to 'finalize' creation of
+            //  temp reply queue. It's been observed (on MAC OS only) that AMQ
+            //  broker QueueSession.createTemporaryQueue() call is not synchronous. Meaning,
+            //  return from createTemporaryQueue() does not guarantee immediate availability
+            //  of the temp queue. It seems like this operation is asynchronous, causing: 
+            //  "InvalidDestinationException: Cannot publish to a deleted Destination..."
+            //  on the service side when it tries to reply to the client.
+            wait(100);
+        } catch( InterruptedException e) {
+      	  
+        } catch (Exception e) {
+          state = ClientState.FAILED;
+          notifyOnInitializationFailure(e);
+          throw new ResourceInitializationException(e);
+    }
+
+  }
+  private void fetchRequiredProperties(Map anApplicationContext, Properties performanceTuningSettings) throws ResourceInitializationException {
+	    ResourceManager rm = null;
+	    if (anApplicationContext.containsKey(Resource.PARAM_RESOURCE_MANAGER)) {
+	      rm = (ResourceManager) anApplicationContext.get(Resource.PARAM_RESOURCE_MANAGER);
+	    } else {
+	      rm = UIMAFramework.newDefaultResourceManager();
+	    }
+	   
+	    if (anApplicationContext.containsKey(UIMAFramework.CAS_INITIAL_HEAP_SIZE)) {
+	      String cas_initial_heap_size = (String) anApplicationContext
+	              .get(UIMAFramework.CAS_INITIAL_HEAP_SIZE);
+	      performanceTuningSettings.put(UIMAFramework.CAS_INITIAL_HEAP_SIZE, cas_initial_heap_size);
+	    }
+	    asynchManager = new AsynchAECasManager_impl(rm);
+
+
+	    clientSideJmxStats.setEndpointName(endpoint);
+	    
+
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.CasPoolSize)) {
+	      casPoolSize = ((Integer) anApplicationContext.get(UimaAsynchronousEngine.CasPoolSize))
+	              .intValue();
+	      clientSideJmxStats.setCasPoolSize(casPoolSize);
+	    }
+
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.Timeout)) {
+	      processTimeout = ((Integer) anApplicationContext.get(UimaAsynchronousEngine.Timeout))
+	              .intValue();
+	    }
+
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.GetMetaTimeout)) {
+	      metadataTimeout = ((Integer) anApplicationContext.get(UimaAsynchronousEngine.GetMetaTimeout))
+	              .intValue();
+	    }
+
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.CpcTimeout)) {
+	      cpcTimeout = ((Integer) anApplicationContext.get(UimaAsynchronousEngine.CpcTimeout))
+	              .intValue();
+	    }
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.ApplicationName)) {
+	      applicationName = (String) anApplicationContext.get(UimaAsynchronousEngine.ApplicationName);
+	    }
+
+	    if (anApplicationContext.containsKey(UimaAsynchronousEngine.TimerPerCAS)) {
+	        timerPerCAS = ((Boolean) anApplicationContext.get(UimaAsynchronousEngine.TimerPerCAS))
+	                .booleanValue();
+	    }
+     
+        brokerURI = (String) anApplicationContext.get(UimaAsynchronousEngine.ServerUri);
+        endpoint = (String) anApplicationContext.get(UimaAsynchronousEngine.ENDPOINT);
+        
+        //  Check if a placeholder is passed in instead of actual broker URL or endpoint. 
+        //  The placeholder has the syntax ${placeholderName} and may be imbedded in text.
+        //  A system property with placeholderName must exist for successful placeholder resolution.
+        //  Throws ResourceInitializationException if placeholder is not in the System properties.
+        brokerURI = replacePlaceholder(brokerURI); 
+        endpoint = replacePlaceholder(endpoint); 
+
+	    if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.CONFIG)) {
+	      UIMAFramework.getLogger(CLASS_NAME)
+	              .logrb(
+	                      Level.CONFIG,
+	                      CLASS_NAME.getName(),
+	                      "initialize",
+	                      JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+	                      "UIMAJMS_init_uimaee_client__CONFIG",
+	                      new Object[] { brokerURI, 0, casPoolSize, processTimeout, metadataTimeout,
+	                          cpcTimeout,timerPerCAS });
+	    }
+
+}
   protected void setFreeCasMessage(Message msg, String aCasReferenceId, String selector) throws Exception {
 	    msg.setIntProperty(AsynchAEMessage.Payload, AsynchAEMessage.None);
 	    msg.setStringProperty(AsynchAEMessage.CasReference, aCasReferenceId);
@@ -750,7 +969,27 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
       throw new ResourceInitializationException(new UIMA_IllegalStateException());
     }
     reset();
-    Properties performanceTuningSettings = null;
+    Properties performanceTuningSettings = new Properties();
+    fetchRequiredProperties(anApplicationContext, performanceTuningSettings);
+    transport = (Transport)anApplicationContext.get(UimaAsynchronousEngine.ClientTransport);
+    if ( Transport.JMS.equals(transport)) {
+    	initializeJMS(anApplicationContext);
+    } else if ( Transport.Java.equals(transport)) {
+    	if ( service == null ) {
+    		service = UimaAsServiceRegistry.getInstance().lookupByEndpoint(endpoint);
+    	}
+    	brokerURI = "java";
+    	initializeLocal(anApplicationContext);
+    	
+    } else if ( transport == null ){
+    	throw new IllegalArgumentException("Client Transport Not Specified - Add Transport.JMS or Transport.Java to ApplicationContext");
+    } else {
+    	throw new IllegalArgumentException("Unsupported Client Transport Specified - "+transport.toString()+" Expected Transport.JMS or Transport.Java");
+    }
+//    Properties performanceTuningSettings = new Properties();
+//    fetchRequiredProperties(anApplicationContext, performanceTuningSettings);
+ 
+    //Properties performanceTuningSettings = null;
 
     if (!anApplicationContext.containsKey(UimaAsynchronousEngine.ServerUri)) {
       throw new ResourceInitializationException();
@@ -847,33 +1086,7 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
     super.serviceDelegate.setCasProcessTimeout(processTimeout);
     super.serviceDelegate.setGetMetaTimeout(metadataTimeout);
     try {
-      // Generate unique identifier
-      String uuid = UUIDGenerator.generate();
-      // JMX does not allow ':' in the ObjectName so replace these with underscore
-      uuid = uuid.replaceAll(":", "_");
-      uuid = uuid.replaceAll("-", "_");
-      applicationName += "_" + uuid;
-      jmxManager = new JmxManager("org.apache.uima");
-      clientSideJmxStats.setApplicationName(applicationName);
-      clientJmxObjectName = new ObjectName("org.apache.uima:name=" + applicationName);
-      jmxManager.registerMBean(clientSideJmxStats, clientJmxObjectName);
-
-      // Check if sharedConnection exists. If not create a new one. The sharedConnection
-      // is static and shared by all instances of UIMA AS client in a jvm. The check
-      // is made in a critical section by first acquiring a global static semaphore to
-      // prevent a race condition.
-      createSharedConnection(brokerURI);
-      running = true;
-      //  This is done to give the broker enough time to 'finalize' creation of
-      //  temp reply queue. It's been observed (on MAC OS only) that AMQ
-      //  broker QueueSession.createTemporaryQueue() call is not synchronous. Meaning,
-      //  return from createTemporaryQueue() does not guarantee immediate availability
-      //  of the temp queue. It seems like this operation is asynchronous, causing: 
-      //  "InvalidDestinationException: Cannot publish to a deleted Destination..."
-      //  on the service side when it tries to reply to the client.
-      try {
-        wait(100);
-      } catch( InterruptedException e) {}
+    	initJMX();
       sendMetaRequest();
       waitForMetadataReply();
       if (abort || !running) {
@@ -902,7 +1115,8 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
           }
         }
         initialized = true;
-        remoteService = true;
+        //remoteService = true;
+        remoteService =  isServiceRemote();
         // running = true;
 
         for (int i = 0; listeners != null && i < listeners.size(); i++) {
@@ -929,7 +1143,71 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
     super.acquireCpcReadySemaphore();
     state = ClientState.RUNNING;
   }
-
+	private AnalysisEngineDeploymentDescriptionDocument parseDD(String descriptorPath) throws Exception {
+		org.apache.xmlbeans.XmlOptions options = new org.apache.xmlbeans.XmlOptions();
+		
+		
+		XMLReader xmlReader = XMLReaderFactory.createXMLReader();
+		xmlReader.setFeature("http://xml.org/sax/features/external-general-entities", false);
+		xmlReader.setFeature("http://xml.org/sax/features/external-parameter-entities", false);
+		xmlReader.setFeature("http://apache.org/xml/features/nonvalidating/load-external-dtd",false);
+		xmlReader.setFeature("http://apache.org/xml/features/disallow-doctype-decl",true);
+		options.setLoadUseXMLReader(xmlReader);
+		
+		return AnalysisEngineDeploymentDescriptionDocument.Factory.parse(new File(descriptorPath), options);
+//		return AnalysisEngineDeploymentDescriptionDocument.Factory.parse(new File(descriptorPath));	
+	}
+	
+	private Provider provider( AnalysisEngineDeploymentDescriptionDocument dd) {
+		 String provider =
+			  dd.getAnalysisEngineDeploymentDescription().getDeployment().getProvider();
+		 
+		 provider = UimaAsDirectServiceBuilder.resolvePlaceholder(provider);
+		 System.out.println("...... provider() - "+provider);
+
+		 if (Provider.JAVA.get().equals(provider)) {
+			 return Provider.JAVA;
+		 } else if (Provider.ACTIVEMQ.get().equals(provider)) {
+			 return Provider.ACTIVEMQ;
+		 } else {
+			 throw new RuntimeException("Invalid provider attribute value in Deployment Descriptor :{"+provider+"} please check <deployment> element. Expected \"java\" or \"activemq\"");
+		 }
+
+		 /*
+		 if (Provider.JAVA.get().equals(provider)) {
+			 return Provider.JAVA;
+		 } else if (Provider.ACTIVEMQ.get().equals(provider)) {
+			 return Provider.ACTIVEMQ;
+		 } else {
+			 throw new RuntimeException("Invalid provider attribute value in Deployment Descriptor :{"+provider+"} please check <deployment> element. Expected \"java\" or \"activemq\"");
+		 }
+		 */
+	}
+	private Protocol protocol( AnalysisEngineDeploymentDescriptionDocument dd) {
+	     String protocol =
+			  dd.getAnalysisEngineDeploymentDescription().getDeployment().getProtocol();
+	     
+		 protocol = UimaAsDirectServiceBuilder.resolvePlaceholder(protocol);
+		 
+		 System.out.println("...... protocol() - "+protocol);
+	     if (Protocol.JAVA.get().equalsIgnoreCase(protocol) ) {
+			 return Protocol.JAVA;
+		 } else if (Protocol.JMS.get().equalsIgnoreCase(protocol)  ) {
+			 return Protocol.JMS;
+		 } else {
+			 throw new RuntimeException("Invalid protocol attribute value in Deployment Descriptor :{"+protocol+"} please check <deployment> element. Expected \"java\" or \"jms\"");
+		 }
+
+		 /*
+	     if (Protocol.JAVA.get().equals(protocol)) {
+			 return Protocol.JAVA;
+		 } else if (Protocol.JMS.get().equals(protocol)) {
+			 return Protocol.JMS;
+		 } else {
+			 throw new RuntimeException("Invalid protocol attribute value in Deployment Descriptor :{"+protocol+"} please check <deployment> element. Expected \"java\" or \"jms\"");
+		 }
+		 */
+	}
   /**
    * First generates a Spring context from a given deploy descriptor and than deploys the context
    * into a Spring Container.
@@ -942,34 +1220,29 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
    * @return - a unique spring container id
    * 
    */
-  public String deploy(String aDeploymentDescriptor, Map anApplicationContext) throws Exception {
-	  String springContext = null;
-	  try {
-		springContext = generateSpringContext(aDeploymentDescriptor, anApplicationContext);
-
-        SpringContainerDeployer springDeployer = new SpringContainerDeployer(springContainerRegistry, this);
+  public String deploy(String deploymentDescriptorPath, Map anApplicationContext) throws Exception {
+	  AnalysisEngineDeploymentDescriptionDocument dd =
+	            parseDD(deploymentDescriptorPath);
+		  
+		  XmlDocumentProperties dp = dd.documentProperties();
+		  System.out.println(dp.getSourceName());
+
+		  // Use factory to create deployer instance for a given protocol and provider
+		  UimaAsServiceDeployer deployer = 
+				  ServiceDeployers.newDeployer(protocol(dd), provider(dd));
+		  
+		  service = deployer.deploy(dd, anApplicationContext);
+		  
+		  UimaAsServiceRegistry.getInstance().register(service);
+
+		  return service.getId();
 
-    	String id = springDeployer.deploy(springContext);
-      if ( springDeployer.isInitialized() ) {
-        springDeployer.startListeners();
-      }
-      return id;
-    } catch (Exception e) {
-      running = true;
-      if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-          UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
-                  "main", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                  "UIMAJMS_exception__WARNING", e);
-      }
-
-      throw e;
-    } finally {
-	  String uimaAsDebug = (String) anApplicationContext.get(UimaAsynchronousEngine.UimaEeDebug);
-	  if ( springContext != null && (null == uimaAsDebug || uimaAsDebug.equals("") ) ) {
-           disposeContextFiles(springContext);
-	  }
-    }
   }
+  
+  protected UimaASService getServiceReference()  {
+	  return service;
+  }
+  
   private void disposeContextFiles(String ...contextFiles) {
     for( String contextFile: contextFiles) {
       File file = new File(contextFile);
@@ -983,6 +1256,7 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
 	 */
   public String deploy(String[] aDeploymentDescriptorList, Map anApplicationContext)
           throws Exception {
+	  /*
     if (aDeploymentDescriptorList == null) {
       throw new ResourceConfigurationException(UIMA_IllegalArgumentException.ILLEGAL_ARGUMENT,
               new Object[] { "Null", "DeploymentDescriptorList", "deploy()" });
@@ -1018,11 +1292,29 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
 
 	//      disposeContextFiles(springContextFiles);
     }
-
+*/
+	  return "";
   }
 
   public void undeploy() throws Exception {
-    Iterator containerIterator = springContainerRegistry.keySet().iterator();
+		Iterator<Entry<String, List<UimaASService>>> iterator = 
+				UimaAsServiceRegistry.getInstance().getServiceList().entrySet().iterator();
+		// Need a separate list to hold service ids to prevent ConcurrentModificationException
+		List<String> serviceIdList = new ArrayList<String>();
+		while( iterator.hasNext() ) {
+			Iterator<UimaASService> listIterator = iterator.next().getValue().iterator();
+			while( listIterator.hasNext()) {
+				UimaASService service = listIterator.next();
+				serviceIdList.add(service.getId());
+			}	
+		}
+		// Now undeploy all services
+		for( String serviceId : serviceIdList ) {
+			undeploy(serviceId);
+		}
+		/*
+	  
+	  Iterator containerIterator = springContainerRegistry.keySet().iterator();
     if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINEST)) {
         String msg ="undeploying "+springContainerRegistry.size()+" Containers";
         UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST, CLASS_NAME.getName(), "undeploy",
@@ -1039,6 +1331,7 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
       }
       undeploy(containerId);
     }
+    */
   }
 
   public void undeploy(String aSpringContainerId) throws Exception {
@@ -1055,18 +1348,41 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
       return;
     }
     
-    UimaEEAdminSpringContext adminContext = null;
-    if (!springContainerRegistry.containsKey(aSpringContainerId)) {
-        return;
-        // throw new InvalidContainerException("Invalid Spring container Id:" + aSpringContainerId +
-        // ". Unable to undeploy the Spring container");
-      }
-      // Fetch an administrative context which contains a Spring Container
-      adminContext = (UimaEEAdminSpringContext) springContainerRegistry.get(aSpringContainerId);
-      if (adminContext == null) {
+//    UimaEEAdminSpringContext adminContext = null;
+    final UimaASService deployedService =
+    		UimaAsServiceRegistry.getInstance().lookupById(aSpringContainerId);
+    if ( deployedService == null ) {
         throw new InvalidContainerException(
                 "Spring Container Does Not Contain Valid UimaEEAdminSpringContext Object");
       }
+    switch (stop_level) {
+    case SpringContainerDeployer.QUIESCE_AND_STOP:
+      //((AnalysisEngineController) ctrer).quiesceAndStop();
+        //service.stop();
+    	deployedService.quiesce();
+
+      break;
+    case SpringContainerDeployer.STOP_NOW:
+     // ((AnalysisEngineController) ctrer).terminate();
+        //service.stop();
+        deployedService.stop();
+        
+ 
+      break;
+  }
+    UimaAsServiceRegistry.getInstance().unregister(deployedService);
+    /*
+//    if (!springContainerRegistry.containsKey(aSpringContainerId)) {
+//        return;
+//        // throw new InvalidContainerException("Invalid Spring container Id:" + aSpringContainerId +
+//        // ". Unable to undeploy the Spring container");
+//      }
+//      // Fetch an administrative context which contains a Spring Container
+//      adminContext = (UimaEEAdminSpringContext) springContainerRegistry.get(aSpringContainerId);
+//      if (adminContext == null) {
+//        throw new InvalidContainerException(
+//                "Spring Container Does Not Contain Valid UimaEEAdminSpringContext Object");
+//      }
       // Fetch instance of the Container from its context
       ApplicationContext ctx = adminContext.getSpringContainer();
       // Query the container for objects that implement
@@ -1144,8 +1460,21 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
       }
       // Remove the container from a local registry
       springContainerRegistry.remove(aSpringContainerId);
+      */
   }
+  private void initJMX() throws Exception {
+	  	// Generate unique identifier
+	      String uuid = UUIDGenerator.generate();
+	      // JMX does not allow ':' in the ObjectName so replace these with underscore
+	      uuid = uuid.replaceAll(":", "_");
+	      uuid = uuid.replaceAll("-", "_");
+	      applicationName += "_" + uuid;
+	      jmxManager = new JmxManager("org.apache.uima");
+	      clientSideJmxStats.setApplicationName(applicationName);
+	      clientJmxObjectName = new ObjectName("org.apache.uima:name=" + applicationName);
+	      jmxManager.registerMBean(clientSideJmxStats, clientJmxObjectName);
 
+	  }
   /**
    * Use dd2spring to generate Spring context file from a given deployment descriptor file.
    * 
@@ -1340,9 +1669,56 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
 
 //  private void stopProducingCases(ClientRequest clientCachedRequest) {
   private void stopProducingCases(String casReferenceId, Destination cmFreeCasQueue) {
-	     PendingMessage msg = new PendingMessage(AsynchAEMessage.Stop);
-	     msg.put(AsynchAEMessage.Destination, cmFreeCasQueue);
-		 msg.put(AsynchAEMessage.CasReference, casReferenceId);
+	    try {
+//	      if (clientCachedRequest.getFreeCasNotificationQueue() != null) {
+	      if (cmFreeCasQueue != null) {
+	        TextMessage msg = createTextMessage();
+	        msg.setText("");
+	        msg.setIntProperty(AsynchAEMessage.Payload, AsynchAEMessage.None);
+//	        msg.setStringProperty(AsynchAEMessage.CasReference, clientCachedRequest.getCasReferenceId());
+	        msg.setStringProperty(AsynchAEMessage.CasReference, casReferenceId);
+	        msg.setIntProperty(AsynchAEMessage.MessageType, AsynchAEMessage.Request);
+	        msg.setIntProperty(AsynchAEMessage.Command, AsynchAEMessage.Stop);
+	        msg.setStringProperty(UIMAMessage.ServerURI, brokerURI);
+	        try {
+	          MessageProducer msgProducer = getMessageProducer(cmFreeCasQueue);
+	          if (msgProducer != null) {
+	        	  
+	              if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
+	                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
+	                          "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+	                          "UIMAJMS_client_sending_stop_to_service__INFO", new Object[] {casReferenceId,cmFreeCasQueue});
+	              }
+	            // Send STOP message to Cas Multiplier Service
+	            msgProducer.send(msg);
+	          } else {
+	              if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
+	                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
+	                          "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+	                          "UIMAJMS_client_unable_to_send_stop_to_cm__WARNING");
+	              }
+	          }
+
+	        } catch (Exception ex) {
+	          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
+	            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
+	                    "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+	                    "UIMAJMS_exception__WARNING",
+	                    ex);
+	          }
+	        }
+	      }
+	    } catch (Exception e) {
+	      if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
+	        UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
+	                "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+	                "UIMAJMS_exception__WARNING", e);
+	      }
+	    }
+	    /*
+	  PendingMessage msg = new PendingMessageImpl(AsynchAEMessage.Stop);
+	     msg.addProperty(AsynchAEMessage.Destination, cmFreeCasQueue);
+		 msg.addProperty(AsynchAEMessage.CasReference, casReferenceId);
 	     try {
 	    	 if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
@@ -1365,64 +1741,17 @@ public class BaseUIMAAsynchronousEngine_impl extends BaseUIMAAsynchronousEngineC
               }
 	     }
 
-	  /*
-    try {
-//      if (clientCachedRequest.getFreeCasNotificationQueue() != null) {
-      if (cmFreeCasQueue != null) {
-        TextMessage msg = createTextMessage();
-        msg.setText("");
-        msg.setIntProperty(AsynchAEMessage.Payload, AsynchAEMessage.None);
-//        msg.setStringProperty(AsynchAEMessage.CasReference, clientCachedRequest.getCasReferenceId());
-        msg.setStringProperty(AsynchAEMessage.CasReference, casReferenceId);
-        msg.setIntProperty(AsynchAEMessage.MessageType, AsynchAEMessage.Request);
-        msg.setIntProperty(AsynchAEMessage.Command, AsynchAEMessage.Stop);
-        msg.setStringProperty(UIMAMessage.ServerURI, brokerURI);
-        try {
-          MessageProducer msgProducer = getMessageProducer(cmFreeCasQueue);
-          if (msgProducer != null) {
-        	  
-              if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
-                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
-                          "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                          "UIMAJMS_client_sending_stop_to_service__INFO", new Object[] {casReferenceId,cmFreeCasQueue});
-              }
-            // Send STOP message to Cas Multiplier Service
-            msgProducer.send(msg);
-          } else {
-              if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
-                          "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                          "UIMAJMS_client_unable_to_send_stop_to_cm__WARNING");
-              }
-          }
-
-        } catch (Exception ex) {
-          if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-            UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
-                    "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                    "UIMAJMS_exception__WARNING",
-                    ex);
-          }
-        }
-      }
-    } catch (Exception e) {
-      if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
-        UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
-                "stopProducingCases", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                "UIMAJMS_exception__WARNING", e);
-      }
-    }
-    */
+	 */
   }
   protected void dispatchFreeCasRequest(String casReferenceId, Message message) throws Exception {
-     PendingMessage msg = new PendingMessage(AsynchAEMessage.ReleaseCAS);
+     PendingMessage msg = new PendingMessageImpl(AsynchAEMessage.ReleaseCAS);
 //     if ( message.getStringProperty(AsynchAEMessage.TargetingSelector) != null ) {
 //    	 msg.put(AsynchAEMessage.TargetingSelector,message.getStringProperty(AsynchAEMessage.TargetingSelector) );
 //     } else {
 //         msg.put(AsynchAEMessage.Destination, message.getJMSReplyTo());
 //     }
-     msg.put(AsynchAEMessage.Destination, message.getJMSReplyTo());
-	 msg.put(AsynchAEMessage.CasReference, casReferenceId);
+     msg.addProperty(AsynchAEMessage.Destination, message.getJMSReplyTo());
+	 msg.addProperty(AsynchAEMessage.CasReference, casReferenceId);
      sender.dispatchMessage(msg, this, false);
   }
   protected MessageSender getDispatcher() {
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UIMA_Service.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UIMA_Service.java
index be22d4b..f546d18 100644
--- a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UIMA_Service.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UIMA_Service.java
@@ -23,6 +23,10 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InvalidClassException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.uima.UIMAFramework;
 import org.apache.uima.aae.UimaASApplicationExitEvent;
@@ -31,8 +35,11 @@ import org.apache.uima.aae.controller.AnalysisEngineController;
 import org.apache.uima.aae.jmx.monitor.BasicUimaJmxMonitorListener;
 import org.apache.uima.aae.jmx.monitor.JmxMonitor;
 import org.apache.uima.aae.jmx.monitor.JmxMonitorListener;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.aae.service.UimaAsServiceRegistry;
 import org.apache.uima.adapter.jms.JmsConstants;
 import org.apache.uima.adapter.jms.activemq.SpringContainerDeployer;
+import org.apache.uima.adapter.jms.client.BaseUIMAAsynchronousEngine_impl;
 import org.apache.uima.resource.ResourceInitializationException;
 import org.apache.uima.util.Level;
 import org.springframework.context.ApplicationEvent;
@@ -74,6 +81,7 @@ public class UIMA_Service implements ApplicationListener {
       // allow multiple args for one key
       deploymentDescriptors = getMultipleArg2("-dd", args);
     }
+    /*
     String saxonURL = getArg("-saxonURL", args);
     String xslTransform = getArg("-xslt", args);
     String uimaAsDebug = getArg("-uimaEeDebug", args);
@@ -84,6 +92,7 @@ public class UIMA_Service implements ApplicationListener {
       printUsageMessage();
       return null;
     }
+    */
     String brokerURL = getArg("-brokerURL", args);
     // Check if broker URL is specified on the command line. If it is not, use the default
     // localhost:61616. In either case, set the System property defaultBrokerURL. It will be used
@@ -99,6 +108,7 @@ public class UIMA_Service implements ApplicationListener {
       System.out.println(">>> Setting Inactivity Timeout To: "
               + System.getProperty(JmsConstants.SessionTimeoutOverride));
     }
+    /*
     if (deploymentDescriptors.length == 0) {
       // array of context files passed in
       springConfigFileArray = args;
@@ -149,8 +159,10 @@ public class UIMA_Service implements ApplicationListener {
         }
       }
     }
+    
     return springConfigFileArray;
-
+    */
+    return deploymentDescriptors;
   }
   public SpringContainerDeployer deploy(String[] springContextFiles, ApplicationListener<ApplicationEvent> listener) throws Exception {
 	    SpringContainerDeployer springDeployer;
@@ -415,6 +427,7 @@ public class UIMA_Service implements ApplicationListener {
   public static void main(String[] args) {
     try {
       UIMA_Service service = new UIMA_Service();
+      /*
       // parse command args and run dd2spring to generate spring context
       // files from deployment descriptors
       String contextFiles[] = service.initialize(args);
@@ -422,6 +435,9 @@ public class UIMA_Service implements ApplicationListener {
       if (contextFiles == null) {
         return;
       }
+      */
+      String dd[] = service.initialize(args);
+      /*
       // Deploy components defined in Spring context files. This method blocks until
       // the container is fully initialized and all UIMA-AS components are succefully
       // deployed.
@@ -430,6 +446,7 @@ public class UIMA_Service implements ApplicationListener {
         System.out.println(">>> Failed to Deploy UIMA Service. Check Logs for Details");
         System.exit(0);
       }
+      
       // remove temporary spring context files generated from DD
       for( String contextFile: contextFiles) {
         File file = new File(contextFile);
@@ -437,8 +454,16 @@ public class UIMA_Service implements ApplicationListener {
           file.delete();
         }
       }
+      */
+      BaseUIMAAsynchronousEngine_impl engine = 
+    		  new BaseUIMAAsynchronousEngine_impl();
+      
+      for( String deploymentDescriptorPath : dd ) {
+    	  engine.deploy(deploymentDescriptorPath, new HashMap<>());
+      }
       // Add a shutdown hook to catch kill signal and to force quiesce and stop
-      ServiceShutdownHook shutdownHook = new ServiceShutdownHook(serviceDeployer);
+      ServiceShutdownHook shutdownHook = new ServiceShutdownHook(engine);
+//      ServiceShutdownHook shutdownHook = new ServiceShutdownHook(serviceDeployer);
       Runtime.getRuntime().addShutdownHook(shutdownHook);
       // Check if we should start an optional JMX-based monitor that will provide service metrics
       // The monitor is enabled by existence of -Duima.jmx.monitor.interval=<number> parameter. By
@@ -451,21 +476,25 @@ public class UIMA_Service implements ApplicationListener {
         service.startMonitor(Long.parseLong(monitorCheckpointFrequency));
       }
       
-      AnalysisEngineController topLevelControllor = serviceDeployer.getTopLevelController();
+//      AnalysisEngineController topLevelControllor = serviceDeployer.getTopLevelController();
       String prompt = "Press 'q'+'Enter' to quiesce and stop the service or 's'+'Enter' to stop it now.\nNote: selected option is not echoed on the console.";
-      if (topLevelControllor != null) {
+  //    if (topLevelControllor != null) {
         System.out.println(prompt);
         // Loop forever or until the service is stopped
-        while (!topLevelControllor.isStopped()) {
+        while ( engine.isRunning() ) {
+//            while (!topLevelControllor.isStopped()) {
+
           if (System.in.available() > 0) {
             int c = System.in.read();
             if (c == 's') {
               service.stopMonitor();
-              serviceDeployer.undeploy(SpringContainerDeployer.STOP_NOW);
+              engine.undeploy();
+              //serviceDeployer.undeploy(SpringContainerDeployer.STOP_NOW);
               System.exit(0);
             } else if (c == 'q') {
-              service.stopMonitor();
-              serviceDeployer.undeploy(SpringContainerDeployer.QUIESCE_AND_STOP);
+            	engine.undeploy();
+              //service.stopMonitor();
+             // serviceDeployer.undeploy(SpringContainerDeployer.QUIESCE_AND_STOP);
               System.exit(0);
 
             } else if (Character.isLetter(c) || Character.isDigit(c)) {
@@ -473,13 +502,13 @@ public class UIMA_Service implements ApplicationListener {
             }
           }
           // This is a polling loop. Sleep for 1 sec
-          try {
-        	if (!topLevelControllor.isStopped()) 
-              Thread.sleep(1000);
-          } catch (InterruptedException ex) {
-          }
+//          try {
+//        	if (!topLevelControllor.isStopped()) 
+//              Thread.sleep(1000);
+//          } catch (InterruptedException ex) {
+//          }
         } // while
-      }
+      //}
     } catch (Exception e) {
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
         UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
@@ -491,21 +520,28 @@ public class UIMA_Service implements ApplicationListener {
   
   static class ServiceShutdownHook extends Thread {
     public SpringContainerDeployer serviceDeployer;
-
+    BaseUIMAAsynchronousEngine_impl engine;
+    
     public ServiceShutdownHook(SpringContainerDeployer serviceDeployer) {
       this.serviceDeployer = serviceDeployer;
     }
-
+    public ServiceShutdownHook(BaseUIMAAsynchronousEngine_impl engine) {
+        this.engine = engine;
+    }
     public void run() {
       try {
-      	AnalysisEngineController topLevelController = serviceDeployer.getTopLevelController();
-      	if (topLevelController != null && !topLevelController.isStopped() ) {
+    	  
+      	//AnalysisEngineController topLevelController = serviceDeployer.getTopLevelController();
+      	//if (topLevelController != null && !topLevelController.isStopped() ) {
           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
                 "run", JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
-                "UIMAJMS_caught_signal__INFO", new Object[] { topLevelController.getComponentName() });
-      	  serviceDeployer.undeploy(SpringContainerDeployer.QUIESCE_AND_STOP);
+                "UIMAJMS_caught_signal__INFO", new Object[] { "TopLevelService" });
+//          "UIMAJMS_caught_signal__INFO", new Object[] { topLevelController.getComponentName() });
+      	 // serviceDeployer.undeploy(SpringContainerDeployer.QUIESCE_AND_STOP);
+    	  engine.undeploy();
+
       	  Runtime.getRuntime().halt(0);
-    	  } 
+    	  //} 
       } catch( Exception e) {
         if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, CLASS_NAME.getName(),
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UimaASJmsService.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UimaASJmsService.java
new file mode 100644
index 0000000..b8f8d59
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/UimaASJmsService.java
@@ -0,0 +1,366 @@
+/*
+ * 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.
+ */
+package org.apache.uima.adapter.jms.service;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadFactory;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.command.ActiveMQDestination;
+import org.apache.activemq.command.ActiveMQQueue;
+import org.apache.uima.aae.AsynchAECasManager_impl;
+import org.apache.uima.aae.InProcessCache;
+import org.apache.uima.aae.InputChannel.ChannelType;
+import org.apache.uima.aae.UimaAsThreadFactory;
+import org.apache.uima.aae.UimaClassFactory;
+import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
+import org.apache.uima.aae.controller.PrimitiveAnalysisEngineController;
+import org.apache.uima.aae.controller.PrimitiveAnalysisEngineController_impl;
+import org.apache.uima.aae.error.ErrorHandlerChain;
+import org.apache.uima.aae.handler.HandlerBase;
+import org.apache.uima.aae.handler.input.MetadataRequestHandler_impl;
+import org.apache.uima.aae.handler.input.MetadataResponseHandler_impl;
+import org.apache.uima.aae.handler.input.ProcessRequestHandler_impl;
+import org.apache.uima.aae.handler.input.ProcessResponseHandler;
+import org.apache.uima.aae.service.AbstractUimaASService;
+import org.apache.uima.aae.service.ScaleoutSpecification;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.adapter.jms.activemq.JmsInputChannel;
+import org.apache.uima.adapter.jms.activemq.JmsOutputChannel;
+import org.apache.uima.adapter.jms.activemq.UimaDefaultMessageListenerContainer;
+import org.apache.uima.adapter.jms.service.builder.ActiveMQFactory;
+import org.apache.uima.adapter.jms.service.builder.JmsMessageListenerBuilder;
+import org.apache.uima.as.client.Listener.Type;
+import org.apache.uima.resource.ResourceManager;
+import org.apache.uima.resource.ResourceSpecifier;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
+
+public class UimaASJmsService extends AbstractUimaASService 
+implements UimaASService {
+	ActiveMQConnectionFactory factory = null;
+	private String brokerURL;
+	private String queueName;
+
+	private JmsOutputChannel outputChannel;
+	private JmsInputChannel inputChannel;
+	private CountDownLatch latchToCountNumberOfTerminatedThreads;
+	private CountDownLatch latchToCountNumberOfInitedThreads;
+	
+	ErrorHandlerChain errorHandlerChain;
+	private List<UimaDefaultMessageListenerContainer> listeners = 
+			new ArrayList<>();
+	
+	public static void main(String[] args) {
+		try {
+			
+			String queueName = "PersonTitleAnnotatorQueue";
+			String analysisEngineDescriptor = "C:/uima/releases/testing/uima/uima-as/2.9.0/target/uima-as-2.9.1-SNAPSHOT-bin/apache-uima-as-2.9.1-SNAPSHOT/examples/descriptors/analysis_engine/PersonTitleAnnotator.xml";
+			String brokerURL = "tcp://localhost:61616";
+			UimaASJmsService service = 
+					new UimaASJmsService();
+
+			
+			ScaleoutSpecification spec = 
+					new ScaleoutSpecification();
+			spec.withProcessScaleout(4).withGetMetaScaleout(1).withFreeCasScaleout(1);
+			
+			ErrorHandlerChain errorHandlerChain = null;
+			
+			InProcessCache inProcessCache = new InProcessCache();
+			
+			ResourceManager resourceManager =
+					UimaClassFactory.produceResourceManager();
+			
+			AsynchAECasManager_impl casManager = 
+					new AsynchAECasManager_impl(resourceManager);
+			casManager.setCasPoolSize(4);
+
+			PrimitiveAnalysisEngineController_impl controller =
+					new PrimitiveAnalysisEngineController_impl(null, queueName, analysisEngineDescriptor, casManager, inProcessCache, 1, 4);
+			
+			controller.setErrorHandlerChain(errorHandlerChain);
+						
+
+			service.withConttroller(controller)
+				.withErrorHandlerChain(errorHandlerChain)
+				.withBrokerURL(brokerURL)
+				.withInputChannel()
+				.withInputQueue(queueName)
+				.withOutputChannel()
+			    .build(4);
+			
+			service.start();
+			
+		} catch( Exception e) {
+			e.printStackTrace();
+		}
+
+	}
+	public UimaASJmsService withConttroller(AnalysisEngineController controller) {
+		this.controller = controller;
+		return this;
+	}
+	
+	public UimaASJmsService withInProcessCache(InProcessCache cache) {
+		this.inProcessCache = cache;
+		return this;
+	}
+	public UimaASJmsService withResourceSpecifier(ResourceSpecifier resourceSpecifier) {
+		this.resourceSpecifier = resourceSpecifier;
+		return this;
+	}
+	
+	public UimaASJmsService withBrokerURL(String brokerURL) {
+		this.brokerURL = brokerURL;
+		return this;
+	}
+	public UimaASJmsService withName(String name) {
+		this.name = name;
+		return this;
+	}
+	public UimaASJmsService withInputQueue(String queueName) {
+		this.queueName = queueName;
+		return this;
+	}
+	public UimaASJmsService withErrorHandlerChain(ErrorHandlerChain errorHandlerChain) {
+		this.errorHandlerChain = errorHandlerChain;
+		return this;
+	}
+	
+	private UimaASJmsService withInputChannel(){
+		inputChannel = new JmsInputChannel(ChannelType.REQUEST_REPLY);
+		return this;
+	}
+	private UimaASJmsService withOutputChannel() {
+		outputChannel = new JmsOutputChannel();
+		outputChannel.setController(controller);
+
+		return this;
+	}
+	private void createConnectionFactory() {
+		if ( factory == null ) {
+			factory = ActiveMQFactory.newConnectionFactory(brokerURL, 0);
+			factory.setTrustAllPackages(true);
+		}
+	}
+	public String getBrokerURL() {
+		return brokerURL;
+	}
+	private String getSelector(Type type) {
+		String selector = null;
+		switch(type) {
+		case ProcessCAS:
+			selector = "Command=2000 OR Command=2002";
+			break;
+			
+		case GetMeta:
+			selector = "Command=2001";
+			break;
+			
+		case FreeCAS:
+		case Unknown:
+		case Reply:
+			break;
+		}
+		return selector;  // OK to return NULL. This means no selector will be used
+	}
+	private boolean isTempQueueListener(Type type) {
+		if ( Type.ProcessCAS.equals(type) || Type.GetMeta.equals(type)) {
+			return false;
+		}
+		return true;
+	}
+	private UimaDefaultMessageListenerContainer createListener(Type type, int scaleout) throws Exception{
+		if ( inputChannel == null ) {
+			withInputChannel();
+		}
+		if ( outputChannel == null ) {
+			withOutputChannel();
+			outputChannel.setServerURI(getBrokerURL());
+		}
+		ThreadPoolTaskExecutor threadExecutor = new ThreadPoolTaskExecutor();
+		if (controller.isPrimitive() && Type.ProcessCAS.equals(type)) {
+			
+			 // Create a Custom Thread Factory. Provide it with an instance of
+		      // PrimitiveController so that every thread can call it to initialize
+		      // the next available instance of a AE.
+		      ThreadFactory tf = new UimaAsThreadFactory().
+		    		  withThreadGroup(Thread.currentThread().getThreadGroup()).
+		    		  withPrimitiveController((PrimitiveAnalysisEngineController) controller).
+		    		  withTerminatedThreadsLatch(latchToCountNumberOfTerminatedThreads).
+		    		  withInitedThreadsLatch(latchToCountNumberOfInitedThreads);
+		      ((UimaAsThreadFactory)tf).setDaemon(true);
+		      // This ThreadExecutor will use custom thread factory instead of defult one
+		      ((ThreadPoolTaskExecutor) threadExecutor).setThreadFactory(tf);
+		}
+		threadExecutor.setCorePoolSize(scaleout);
+		threadExecutor.setMaxPoolSize(scaleout);
+		
+		// destination can be NULL if this listener is meant for a 
+		// a temp queue. Such destinations are created on demand 
+		// using destination resolver which is plugged into the 
+		// listener. The resolver creates a temp queue lazily on
+		// listener startup.
+		ActiveMQDestination destination = null;
+		
+		if ( !isTempQueueListener(type) ) {
+			destination = new ActiveMQQueue(queueName);
+		}
+		JmsMessageListenerBuilder listenerBuilder = 
+				new JmsMessageListenerBuilder();
+
+		UimaDefaultMessageListenerContainer messageListener =
+				listenerBuilder.withController(controller)
+		       			.withType(type)
+						.withConectionFactory(factory)
+						.withThreadPoolExecutor(threadExecutor)
+						.withConsumerCount(scaleout)
+						.withInputChannel(inputChannel)
+						.withSelector(getSelector(type))
+						.withDestination(destination)
+						.build();
+		messageListener.setReceiveTimeout(500);
+		return messageListener;
+	}
+	public HandlerBase getMessageHandler(AnalysisEngineController controller) {
+		MetadataRequestHandler_impl metaHandler = new MetadataRequestHandler_impl("MetadataRequestHandler");
+		metaHandler.setController(controller);
+		ProcessRequestHandler_impl processHandler = new ProcessRequestHandler_impl("ProcessRequestHandler");
+		processHandler.setController(controller);
+		metaHandler.setDelegate(processHandler);
+		if ( !controller.isPrimitive() ) {
+			MetadataResponseHandler_impl metaResponseHandler = 
+					new MetadataResponseHandler_impl("MetadataResponseHandler");
+			metaResponseHandler.setController(controller);
+			processHandler.setDelegate(metaResponseHandler);
+			
+			ProcessResponseHandler processResponseHandler = 
+					new ProcessResponseHandler("ProcessResponseHandler");
+			processResponseHandler.setController(controller);
+			metaResponseHandler.setDelegate(processResponseHandler);
+			
+		}
+		return metaHandler;
+	}
+	public UimaASJmsService build(int scaleout) throws Exception {
+		// First create Connection Factory. This is needed by
+		// JMS listeners.
+		createConnectionFactory();
+		// counts number of initialized threads
+		latchToCountNumberOfInitedThreads = new CountDownLatch(scaleout);
+		// counts number of terminated threads
+		latchToCountNumberOfTerminatedThreads = new CountDownLatch(scaleout);
+		// Add one instance of JmsOutputChannel 
+		if ( controller.getOutputChannel(ENDPOINT_TYPE.JMS) == null ) {
+			withOutputChannel();
+			outputChannel.setServerURI(brokerURL);
+			outputChannel.setServiceInputEndpoint(queueName);
+			controller.addOutputChannel(outputChannel);
+		} else {
+			outputChannel = (JmsOutputChannel)controller.getOutputChannel(ENDPOINT_TYPE.JMS);
+			outputChannel.setServiceInputEndpoint(queueName);
+		}
+		// Add one instance of JmsInputChannel
+		if ( controller.getInputChannel(ENDPOINT_TYPE.JMS) == null ) {
+			withInputChannel();   // one input channel instance
+			controller.setInputChannel(inputChannel);
+		} else {
+			inputChannel = (JmsInputChannel)controller.getInputChannel(ENDPOINT_TYPE.JMS);
+		}
+		
+		inputChannel.setController(controller);
+		
+		inputChannel.setMessageHandler(getMessageHandler(controller));
+		
+		// Create service JMS listeners to handle Process, GetMeta and optional FreeCas
+		// requests.
+		
+		// listener to handle process CAS requests
+		UimaDefaultMessageListenerContainer processListener 
+		    = createListener(Type.ProcessCAS, scaleout);
+		inputChannel.addListenerContainer(processListener);
+		
+		listeners.add(processListener);
+		// listener to handle GetMeta requests
+		UimaDefaultMessageListenerContainer getMetaListener 
+	        = createListener(Type.GetMeta, 1);
+		inputChannel.addListenerContainer(getMetaListener);
+		listeners.add(getMetaListener);
+		
+		if ( controller.isCasMultiplier()) {
+			// listener to handle Free CAS requests
+			UimaDefaultMessageListenerContainer freeCasListener 
+		        = createListener(Type.FreeCAS, 1);
+			inputChannel.addListenerContainer(freeCasListener);
+			listeners.add(freeCasListener);
+		}
+		
+		return this;
+	}
+	public void quiesce() throws Exception {
+		controller.quiesceAndStop();
+	}
+
+	public void stop() throws Exception {
+	//	controller.stop();
+		
+		controller.terminate();
+/*
+		for( UimaDefaultMessageListenerContainer listener : listeners ) {
+			listener.setTerminating();
+			listener.stop();
+			// wait for all process threads to exit
+			if ( controller.isPrimitive() && Type.ProcessCAS.equals(listener.getType())) {
+				latchToCountNumberOfTerminatedThreads.await();
+			}
+			if ( listener.getTaskExecutor() != null ) {
+				if ( listener.getTaskExecutor() instanceof ThreadPoolTaskExecutor ) {
+					ThreadPoolTaskExecutor threadExecutor =
+							(ThreadPoolTaskExecutor)listener.getTaskExecutor();
+					threadExecutor.getThreadPoolExecutor().shutdownNow();
+					threadExecutor.shutdown();
+				}
+			}
+//			listener.closeConnection();
+//			listener.destroy();
+			System.out.println("Stopped Process Listener ....");
+		}
+*/
+	}
+	@Override
+	public String getEndpoint() {
+		return queueName;
+	}
+	public ResourceSpecifier getResourceSpecifier( ){
+		return resourceSpecifier;
+	}
+	@Override
+	public String getId() {
+		// TODO Auto-generated method stub
+		return id;
+	}
+	@Override
+	public String getName() {
+		return name;
+	}
+
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/ActiveMQFactory.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/ActiveMQFactory.java
new file mode 100644
index 0000000..846723e
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/ActiveMQFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+package org.apache.uima.adapter.jms.service.builder;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.ActiveMQPrefetchPolicy;
+import org.apache.activemq.command.ActiveMQQueue;
+
+public class ActiveMQFactory {
+	private ActiveMQFactory() {
+		
+	}
+	public static ActiveMQQueue newQueue(String withName) {
+		return new ActiveMQQueue(withName);
+	}
+	
+	public static ActiveMQPrefetchPolicy newPrefetchPolicy(int howMany) {
+		ActiveMQPrefetchPolicy prefetchPolicy =
+				new ActiveMQPrefetchPolicy();
+		prefetchPolicy.setQueuePrefetch(howMany);
+		return prefetchPolicy;
+	}
+	
+	public static ActiveMQConnectionFactory newConnectionFactory(String broker, int prefetch) {
+		ActiveMQConnectionFactory factory = 
+				new ActiveMQConnectionFactory();
+		factory.setBrokerURL(broker);
+		factory.setPrefetchPolicy(newPrefetchPolicy(prefetch));
+
+		return factory;
+	}
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/JmsMessageListenerBuilder.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/JmsMessageListenerBuilder.java
new file mode 100644
index 0000000..30d2cca
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/JmsMessageListenerBuilder.java
@@ -0,0 +1,333 @@
+/*
+ * 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.
+ */
+package org.apache.uima.adapter.jms.service.builder;
+
+import javax.jms.Destination;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.command.ActiveMQDestination;
+import org.apache.activemq.command.ActiveMQQueue;
+import org.apache.uima.aae.AsynchAECasManager_impl;
+import org.apache.uima.aae.InProcessCache;
+import org.apache.uima.aae.InputChannel;
+import org.apache.uima.aae.InputChannel.ChannelType;
+import org.apache.uima.aae.UimaClassFactory;
+import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.Endpoint;
+import org.apache.uima.aae.controller.PrimitiveAnalysisEngineController_impl;
+import org.apache.uima.aae.error.ErrorHandlerChain;
+import org.apache.uima.adapter.jms.activemq.ConcurrentMessageListener;
+import org.apache.uima.adapter.jms.activemq.JmsInputChannel;
+import org.apache.uima.adapter.jms.activemq.JmsOutputChannel;
+import org.apache.uima.adapter.jms.activemq.TempDestinationResolver;
+import org.apache.uima.adapter.jms.activemq.UimaDefaultMessageListenerContainer;
+import org.apache.uima.as.client.Listener.Type;
+import org.apache.uima.resource.ResourceManager;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
+
+public class JmsMessageListenerBuilder {
+	private AnalysisEngineController controller;
+	private ActiveMQConnectionFactory connectionFactory;
+	private int consumerCount=1;
+	private InputChannel inputChannel;
+	private Endpoint endpoint;
+	private boolean isReplyListener = false;
+	private String selector=null;
+	private Destination destination=null;  // queue 
+	private ThreadPoolTaskExecutor threadExecutor=null;
+	private Type type;
+	private TempDestinationResolver tempQueueDestinationResolver = null;
+	
+	public static void main(String[] args) {
+		try {
+			String endpointName = "PersonTitleAnnotatorQueue";
+			String analysisEngineDescriptor = "C:/uima/releases/testing/uima/uima-as/2.9.0/target/uima-as-2.9.1-SNAPSHOT-bin/apache-uima-as-2.9.1-SNAPSHOT/examples/descriptors/analysis_engine/PersonTitleAnnotator.xml";
+			String broker = "tcp://localhost:61616";
+			String processSelector = "Command=2000 OR Command=2002";
+			String getMetaSelector = "Command=2001";
+			int workQueueSize = 1;
+			int processScaleout = 4;
+			int scaleout = 1;
+			
+			System.setProperty("BrokerURI",broker);
+			ErrorHandlerChain errorHandlerChain = null;
+			
+			InProcessCache inProcessCache = new InProcessCache();
+			
+			ResourceManager resourceManager =
+					UimaClassFactory.produceResourceManager();
+			
+			AsynchAECasManager_impl casManager = 
+					new AsynchAECasManager_impl(resourceManager);
+			casManager.setCasPoolSize(processScaleout);
+			
+			
+			JmsInputChannel processInputChannel = new JmsInputChannel(ChannelType.REQUEST_REPLY);
+			JmsInputChannel getMetaInputChannel = new JmsInputChannel(ChannelType.REQUEST_REPLY);
+			
+			JmsOutputChannel outputChannel = new JmsOutputChannel();
+			outputChannel.setServerURI(broker);
+			PrimitiveAnalysisEngineController_impl controller =
+					new PrimitiveAnalysisEngineController_impl(null, endpointName, analysisEngineDescriptor, casManager, inProcessCache, workQueueSize, scaleout);
+			
+			controller.setOutputChannel(outputChannel);
+			controller.setErrorHandlerChain(errorHandlerChain);
+						
+			
+			ActiveMQConnectionFactory factory =
+					ActiveMQFactory.newConnectionFactory(broker, 0);
+			
+			factory.setTrustAllPackages(true);
+			ActiveMQDestination destination = 
+					new ActiveMQQueue(endpointName);
+			JmsMessageListenerBuilder processListenerBuilder = 
+					new JmsMessageListenerBuilder();
+			ThreadPoolTaskExecutor threadExecutor1 = new ThreadPoolTaskExecutor();
+			
+			threadExecutor1.setCorePoolSize(processScaleout);
+			threadExecutor1.setMaxPoolSize(processScaleout);
+
+			UimaDefaultMessageListenerContainer jmsProcessMessageListener =
+			       processListenerBuilder.withController(controller)
+			       			.withType(Type.ProcessCAS)
+							.withConectionFactory(factory)
+							.withThreadPoolExecutor(threadExecutor1)
+							.withConsumerCount(processScaleout)
+							.withInputChannel(processInputChannel)
+							.withSelector(processSelector)
+							.withDestination(destination)
+							.build();
+			
+			JmsMessageListenerBuilder getMetaListenerBuilder = 
+					new JmsMessageListenerBuilder();
+			ThreadPoolTaskExecutor threadExecutor2 = new ThreadPoolTaskExecutor();
+			threadExecutor2.setCorePoolSize(scaleout);
+			threadExecutor2.setMaxPoolSize(scaleout);
+			
+			UimaDefaultMessageListenerContainer jmsGetMetaMessageListener =
+					getMetaListenerBuilder.withController(controller)
+							.withType(Type.GetMeta)
+							.withConectionFactory(factory)
+							.withThreadPoolExecutor(threadExecutor2)
+							.withConsumerCount(scaleout)
+							.withInputChannel(getMetaInputChannel)
+							.withSelector(getMetaSelector)
+							.withDestination(destination)
+							.build();
+
+			ThreadPoolTaskExecutor threadExecutor3 = new ThreadPoolTaskExecutor();
+			threadExecutor3.setCorePoolSize(scaleout);
+			threadExecutor3.setMaxPoolSize(scaleout);
+			TempDestinationResolver resolver = new TempDestinationResolver(controller.getComponentName(),"");
+			resolver.setConnectionFactory(factory);
+			
+			UimaDefaultMessageListenerContainer replyListener =
+					getMetaListenerBuilder.withController(controller)
+							.withType(Type.Reply)
+							.withConectionFactory(factory)
+							.withThreadPoolExecutor(threadExecutor3)
+							.withConsumerCount(scaleout)
+							.withTempDestinationResolver(resolver)
+							.build();
+			
+			
+			processInputChannel.setController(controller);
+			processInputChannel.addListenerContainer(jmsProcessMessageListener);
+			
+			getMetaInputChannel.setController(controller);
+			getMetaInputChannel.addListenerContainer(jmsGetMetaMessageListener);
+			
+			threadExecutor1.initialize();
+			threadExecutor1.getThreadPoolExecutor().prestartAllCoreThreads();
+			threadExecutor2.initialize();
+			threadExecutor2.getThreadPoolExecutor().prestartAllCoreThreads();
+			threadExecutor3.initialize();
+			threadExecutor3.getThreadPoolExecutor().prestartAllCoreThreads();
+			
+			jmsProcessMessageListener.afterPropertiesSet();
+			jmsProcessMessageListener.initialize();
+			jmsProcessMessageListener.start();
+			
+			jmsGetMetaMessageListener.afterPropertiesSet();
+			jmsGetMetaMessageListener.initialize();
+			jmsGetMetaMessageListener.start();
+			
+
+			replyListener.afterPropertiesSet();
+			replyListener.initialize();
+			replyListener.start();
+			
+/*				
+			synchronized(inProcessCache ) {
+				inProcessCache.wait(5000);
+				System.out.println("Stopping Listeners ....");
+				jmsProcessMessageListener.setTerminating();
+				jmsProcessMessageListener.stop();
+				threadExecutor1.getThreadPoolExecutor().shutdownNow();
+				threadExecutor1.shutdown();
+				jmsProcessMessageListener.stop();
+				jmsProcessMessageListener.closeConnection();
+				jmsProcessMessageListener.destroy();
+				System.out.println("Stopped Process Listener ....");
+				
+				jmsGetMetaMessageListener.setTerminating();
+				jmsGetMetaMessageListener.stop();
+				
+				threadExecutor2.getThreadPoolExecutor().shutdownNow();
+				threadExecutor2.shutdown();
+				jmsGetMetaMessageListener.closeConnection();
+				jmsGetMetaMessageListener.destroy();
+				System.out.println("Stopped GetMeta Listener ....");
+			}
+			*/
+		} catch( Exception e) {
+			e.printStackTrace();
+		}
+
+	}
+
+	public JmsMessageListenerBuilder withController(AnalysisEngineController controller ) {
+		this.controller = controller;
+		return this;
+	}
+	
+	public JmsMessageListenerBuilder withTempDestinationResolver(TempDestinationResolver resolver ) {
+		this.tempQueueDestinationResolver = resolver;
+		return this;
+	}
+	public JmsMessageListenerBuilder withInputChannel(InputChannel inputChannel ) {
+		this.inputChannel = inputChannel;
+		return this;
+	}
+	public JmsMessageListenerBuilder withThreadPoolExecutor(ThreadPoolTaskExecutor threadExecutor) {
+		this.threadExecutor = threadExecutor;
+		return this;
+	}
+	public JmsMessageListenerBuilder withEndpoint(Endpoint endpoint ) {
+		this.endpoint = endpoint;
+		return this;
+	}
+	public JmsMessageListenerBuilder withSelector(String selector ) {
+		this.selector = selector;
+		return this;
+	}
+	public JmsMessageListenerBuilder withDestination(Destination destination ) {
+		this.destination = destination;
+		return this;
+	}
+	public JmsMessageListenerBuilder withConectionFactory(ActiveMQConnectionFactory connectionFactory ) {
+		this.connectionFactory = connectionFactory;
+		return this;
+	}
+
+	public JmsMessageListenerBuilder withConsumerCount(int howManyConsumers ) {
+		this.consumerCount = howManyConsumers;
+		return this;
+	}
+	public JmsMessageListenerBuilder asReplyListener() {
+		this.isReplyListener = true;
+		return this;
+	}
+	public JmsMessageListenerBuilder withType(Type t) {
+		this.type = t;
+		if ( Type.Reply.equals(t)) {
+			asReplyListener();
+		}
+		return this;
+	}
+	private void validate() {
+		
+	}
+	private boolean isRemoteCasMultiplier(Endpoint endpoint) {
+	       return (endpoint != null && endpoint.isRemote()  && endpoint.isCasMultiplier() );
+	}
+	public UimaDefaultMessageListenerContainer build() throws Exception{
+		UimaDefaultMessageListenerContainer listener = 
+				new UimaDefaultMessageListenerContainer();
+		/*
+		 * 
+		 * VALIDATE REQUIRED PROPERTIES
+		 * 
+		 */
+		// make sure all required properties are set
+		validate();
+		if ( threadExecutor != null ) {
+			threadExecutor.setThreadNamePrefix(controller.getComponentName()+"-"+type.name()+"Listener-Thread");
+			listener.setTaskExecutor(threadExecutor);
+			
+		}
+		
+		listener.setConcurrentConsumers(consumerCount);
+		listener.setController(controller);
+		
+		if ( selector != null ) {
+			listener.setMessageSelector(selector);
+		}
+		
+        if (isRemoteCasMultiplier(endpoint) ) {
+        	// for remote CM's we need special handling. See description of a 
+        	// possible race condition in ConcurrentMessageListener class.
+    		ThreadGroup tg = Thread.currentThread().getThreadGroup();
+            String prefix = endpoint.getDelegateKey()+" Reply Thread";
+    		ConcurrentMessageListener concurrentListener = 
+    				new ConcurrentMessageListener(consumerCount, (JmsInputChannel)inputChannel, "", tg,prefix);
+    		// register this listener with inputchannel so that we can stop it. The listener on a remote CM 
+    		// is ConcurrentMessageListener which imposes order of replies (parent last) before delegating 
+    		// msgs to the inputchannel. When stopping the service, all listeners must be registered with 
+    		// an inputchannel which is responsible for shutting down all listeners.
+    		((JmsInputChannel)inputChannel).registerListener(listener);
+            listener.setMessageListener(concurrentListener);
+            concurrentListener.setAnalysisEngineController(controller);
+        } else {
+    		((JmsInputChannel)inputChannel).registerListener(listener);
+    		listener.setMessageListener(inputChannel);
+        }
+
+		listener.setTargetEndpoint(endpoint);
+		listener.setConnectionFactory(connectionFactory);
+		// is this listener processing replies from a remote service. This can
+		// only be true if the controller is an aggregate. Primitive controller
+		// can only handle requests from remote services. An aggregate can send
+		// requests and expects replies.
+		if ( isReplyListener || Type.FreeCAS.equals(type)) {
+			String e = Type.FreeCAS.equals(type) ? "FreeCASEndpoint" :endpoint.getDelegateKey();
+			TempDestinationResolver resolver = new
+					TempDestinationResolver(controller.getComponentName(), e);
+			resolver.setListener(listener);
+			resolver.setConnectionFactory(connectionFactory);
+			listener.setDestinationResolver(resolver);
+			listener.setDestinationName("");
+			if ( Type.FreeCAS.equals(type)) {
+				listener.setBeanName(controller.getComponentName()+"-"+type.name()+"Listener For FreeCas Listener");
+			} else {
+				listener.setBeanName(controller.getComponentName()+"-"+type.name()+"Listener For Delegate:"+endpoint.getDelegateKey());
+			}
+		} else if ( destination != null ) {
+			listener.setDestinationName(((ActiveMQDestination)destination).getPhysicalName());
+			listener.setDestination(destination);
+			listener.setBeanName(controller.getComponentName()+"-"+type.name()+"Listener");
+
+		}
+
+		if ( type != null ) {
+			listener.setType(type);
+		}
+		return listener;
+	}
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/UimaAsJmsServiceBuilder.java b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/UimaAsJmsServiceBuilder.java
new file mode 100644
index 0000000..205e0c1
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/adapter/jms/service/builder/UimaAsJmsServiceBuilder.java
@@ -0,0 +1,316 @@
+/*
+ * 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.
+ */
+package org.apache.uima.adapter.jms.service.builder;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.ActiveMQPrefetchPolicy;
+import org.apache.uima.aae.InputChannel;
+import org.apache.uima.aae.InputChannel.ChannelType;
+import org.apache.uima.aae.client.UimaAsynchronousEngine.Transport;
+import org.apache.uima.aae.OutputChannel;
+import org.apache.uima.aae.UimaClassFactory;
+import org.apache.uima.aae.controller.AggregateAnalysisEngineController;
+import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
+import org.apache.uima.aae.controller.ControllerCallbackListener;
+import org.apache.uima.aae.controller.Endpoint_impl;
+import org.apache.uima.aae.error.ErrorHandler;
+import org.apache.uima.aae.error.ErrorHandlerChain;
+import org.apache.uima.aae.error.Threshold;
+import org.apache.uima.aae.error.Thresholds;
+import org.apache.uima.aae.error.handler.CpcErrorHandler;
+import org.apache.uima.aae.error.handler.GetMetaErrorHandler;
+import org.apache.uima.aae.error.handler.ProcessCasErrorHandler;
+import org.apache.uima.aae.handler.Handler;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.aae.service.builder.AbstractUimaAsServiceBuilder;
+import org.apache.uima.aae.service.delegate.AnalysisEngineDelegate;
+import org.apache.uima.aae.service.delegate.RemoteAnalysisEngineDelegate;
+import org.apache.uima.adapter.jms.activemq.JmsInputChannel;
+import org.apache.uima.adapter.jms.activemq.JmsOutputChannel;
+import org.apache.uima.adapter.jms.activemq.TempDestinationResolver;
+import org.apache.uima.adapter.jms.activemq.UimaDefaultMessageListenerContainer;
+import org.apache.uima.adapter.jms.service.UimaASJmsService;
+import org.apache.uima.analysis_engine.AnalysisEngineDescription;
+import org.apache.uima.as.client.DirectInputChannel;
+import org.apache.uima.as.client.Listener.Type;
+import org.apache.uima.resource.ResourceSpecifier;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
+import org.apache.uima.resourceSpecifier.AsyncPrimitiveErrorConfigurationType;
+import org.apache.uima.resourceSpecifier.CasPoolType;
+import org.apache.uima.resourceSpecifier.CollectionProcessCompleteErrorsType;
+import org.apache.uima.resourceSpecifier.ProcessCasErrorsType;
+import org.apache.uima.resourceSpecifier.ServiceType;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
+
+public class UimaAsJmsServiceBuilder extends AbstractUimaAsServiceBuilder{
+//	private static final String NoParent= "NoParent";
+//	private static enum FlowControllerType {
+//		FIXED
+//	}
+//		static Map<String, Object> ddAEMap = new HashMap<String, Object>();
+	
+	/*
+	private InProcessCache cache;
+	private AsynchAECasManager_impl casManager;
+    private ResourceManager resourceManager;
+    */
+    private int scaleout=1;
+//    private AnalysisEngineController controller;
+//    private List<ControllerStatusListener> listeners = new ArrayList<ControllerStatusListener>();
+//    private ServiceMode mode = ServiceMode.Asynchronous;   // default 
+//    private AnalysisEngineDescription topLevelAEDescriptor;
+    
+	public static void main(String[] args) {
+		try {
+			String tla = "C:/runtime/uima-as/2.8.1/apache-uima-as-2.8.1-SNAPSHOT/examples/descriptors/tutorial/ex4/MeetingDetectorTAEGovNameDetector.xml";
+			String ptDescriptor = "C:/runtime/uima-as/2.8.1/apache-uima-as-2.8.1-SNAPSHOT/examples/descriptors/analysis_engine/PersonTitleAnnotator.xml";
+			//			"C:/runtime/uima-as/2.8.1/apache-uima-as-2.8.1-SNAPSHOT/examples/descriptors/tutorial/ex4/MeetingDetectorTAE.xml";
+
+			//String dd1 = "C:/uima/releases/builds/uima-as/2.8.1/uimaj-as-activemq/src/test/resources/deployment/Deploy_TopLevelBlueJAggregateCM.xml";
+			String dd2 = "C:/uima/releases/builds/uima-as/2.8.1/uimaj-as-activemq/src/test/resources/deployment/Deploy_TopAggregateWithInnerAggregateCM.xml";
+			String dd = "C:/runtime/uima-as/2.8.1/apache-uima-as-2.8.1-SNAPSHOT/examples/deploy/as/Deploy_MeetingDetectorTAE.xml";
+			
+			String dd3 = "../uimaj-as-activemq/src/test/resources/deployment/Deploy_PersonTitleAnnotator.xml";
+			String dd4 = "../uimaj-as-activemq/src/test/resources/deployment/Deploy_AggregateAnnotator.xml";
+
+		} catch(Exception e) {
+			e.printStackTrace();
+		}
+	}
+	
+	
+	public static InputChannel createInputChannel(ChannelType type) {
+		return new JmsInputChannel(type);
+	}
+
+	
+	public OutputChannel createOutputChannel() {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	protected void addListenerForReplyHandling( AggregateAnalysisEngineController controller, Endpoint_impl endpoint, RemoteAnalysisEngineDelegate remoteDelegate) throws Exception {
+		String brokerURL =  resolvePlaceholder(remoteDelegate.getBrokerURI());
+		int prefetch = remoteDelegate.getPrefetch();
+		endpoint.setEndpoint(resolvePlaceholder(remoteDelegate.getQueueName())); 
+
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// WITH HTTP BROKER URL THE PRFETCH MUST BE > 0 
+		// OTHERWISE THE LISTENER DOES NOT GET MSGS
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		
+		if (prefetch == 0 ) {
+			prefetch = 1;
+		}
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		// !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
+		JmsInputChannel inputChannel;
+		if ((controller.getInputChannel(ENDPOINT_TYPE.JMS)) == null) {
+			inputChannel = new JmsInputChannel(ChannelType.REQUEST_REPLY);
+			Handler messageHandlerChain = getMessageHandler(controller);
+			inputChannel.setMessageHandler(messageHandlerChain);
+			controller.addInputChannel(inputChannel);
+			inputChannel.setController(controller);
+		} else {
+			inputChannel = (JmsInputChannel) controller.getInputChannel(ENDPOINT_TYPE.JMS);
+		}
+		// make the name unique
+		String qname = "rmtRtrnQ_"+controller.getComponentName().replaceAll("\\s","_")+"_"+endpoint.getDelegateKey()+"_"+UUID.randomUUID();
+		endpoint.setReplyToEndpoint(qname);
+		// remote always replies to a JMS temp queue
+		endpoint.setTempReplyDestination(true);
+		ThreadPoolTaskExecutor threadExecutor = new ThreadPoolTaskExecutor();
+		int consumerCount = 1; // default reply queue consumer count
+		// check if the DD includes reply queue scaleout for this remote delegate
+		if ( remoteDelegate.getReplyScaleout() > 1 ) { 
+			
+			// in this context the scaleout just means how many consumer threads
+			// this listener will start to handle messages arriving into the 
+			// temp reply queue.
+			consumerCount = remoteDelegate.getReplyScaleout();
+			endpoint.setConcurrentReplyConsumers(remoteDelegate.getReplyScaleout());
+			if ( endpoint.isCasMultiplier() ) {
+				// for remote CM, the listener will use a single thread to receive
+				// CASes. This is done to deal with a race condition described in
+				// class ConcurrentMessageListener. 
+				// if the remote is a cas multiplier, 
+				threadExecutor.setCorePoolSize(1);
+				threadExecutor.setMaxPoolSize(1);
+			} else {
+				threadExecutor.setCorePoolSize(consumerCount);
+				threadExecutor.setMaxPoolSize(consumerCount);
+			}
+		}  else {
+			threadExecutor.setCorePoolSize(consumerCount);
+			threadExecutor.setMaxPoolSize(consumerCount);
+		}
+		JmsMessageListenerBuilder replyListenerBuilder = 
+				new JmsMessageListenerBuilder();
+
+		ActiveMQConnectionFactory factory =
+				new ActiveMQConnectionFactory(brokerURL);
+		factory.setTrustAllPackages(true);
+		ActiveMQPrefetchPolicy pp = new ActiveMQPrefetchPolicy();
+		pp.setQueuePrefetch(prefetch);
+		
+		factory.setPrefetchPolicy(pp);
+		// Need a resolver to create temp reply queue. It will be created automatically
+		// by Spring.
+		TempDestinationResolver resolver = new TempDestinationResolver(controller.getComponentName(),remoteDelegate.getKey());
+		resolver.setConnectionFactory(factory);
+		
+		UimaDefaultMessageListenerContainer replyListener =
+				replyListenerBuilder.withController(controller)
+						.withType(Type.Reply)
+						.withInputChannel(inputChannel)
+						.withConectionFactory(factory)
+						.withThreadPoolExecutor(threadExecutor)
+						.withConsumerCount(consumerCount)
+						.withTempDestinationResolver(resolver)
+						.withEndpoint(endpoint)
+						.build();		
+		//replyListener.afterPropertiesSet();
+		replyListener.start();
+		
+//		replyListener.setTargetEndpoint(endpoint);
+
+		// there should be one instance of OutputChannel for JMS. Create it, if one does not exist 
+		if ( controller.getOutputChannel(ENDPOINT_TYPE.JMS) == null) {
+	  		JmsOutputChannel oc = new JmsOutputChannel();
+			oc.setController(controller);
+			oc.setServerURI(brokerURL);
+			oc.setControllerInputEndpoint("");
+			oc.setServiceInputEndpoint("");
+			oc.initialize();
+			controller.addOutputChannel(oc);
+		}
+		endpoint.setServerURI(brokerURL);
+		System.out.println("......... Service:"+controller.getComponentName()+" Reply Listener Started - Delegate:"+endpoint.getDelegateKey()+" Broker:"+endpoint.getServerURI()+" Endpoint:"+endpoint.getDestination());
+	}
+
+	public UimaASService buildAndDeploy(AnalysisEngineDeploymentDescriptionDocument doc, AnalysisEngineDelegate del,
+			UimaASJmsService service, ControllerCallbackListener callback) throws Exception {
+		// get top level CAS pool to
+		CasPoolType cp = getCasPoolConfiguration(doc);
+
+		super.addEnvironmentVariablesFromDD(doc);
+
+		System.setProperty("BrokerURI", service.getBrokerURL());
+
+		initialize(service, cp, Transport.JMS); 
+		service.withInProcessCache(super.cache);
+
+		int howMany = howManyInstances(doc);
+		AnalysisEngineController topLevelController = createController(del, service.getResourceSpecifier(),
+				service.getName(), null, howMany);
+		
+		// callback will be made when initialization succeeds or fails
+		topLevelController.addControllerCallbackListener(callback);
+
+		topLevelController.getServiceInfo().setBrokerURL(service.getBrokerURL());
+		topLevelController.setServiceId(service.getId());
+		// fetch service definition from DD
+		ServiceType s = getService(doc);
+
+		AsyncPrimitiveErrorConfigurationType pec;
+		if (s.getAnalysisEngine() != null && s.getAnalysisEngine().getAsyncPrimitiveErrorConfiguration() != null) {
+			pec = s.getAnalysisEngine().getAsyncPrimitiveErrorConfiguration();
+		} else {
+			pec = addDefaultErrorHandling(s);
+		}
+		service.withConttroller(topLevelController).withErrorHandlerChain(null);
+
+		configureTopLevelService(topLevelController, service, pec);
+
+		service.build(howMany);
+
+		return service;
+	}
+
+	private void configureTopLevelService(AnalysisEngineController topLevelController, UimaASJmsService service,
+			AsyncPrimitiveErrorConfigurationType pec) throws Exception {
+		// ResourceSpecifier resourceSpecifier = service.getResourceSpecifier();
+		if (!topLevelController.isPrimitive() && pec != null) {
+
+			ErrorHandlerChain chain = topLevelController.getErrorHandlerChain();
+			Iterator<ErrorHandler> handlers = chain.iterator();
+			while (handlers.hasNext()) {
+				ErrorHandler eh = handlers.next();
+				Map<String, Threshold> map = eh.getEndpointThresholdMap();
+				if (eh instanceof ProcessCasErrorHandler) {
+					if (pec.getProcessCasErrors() != null) {
+						map.put("", Thresholds.getThreshold(pec.getProcessCasErrors().getThresholdAction(),
+								pec.getProcessCasErrors().getMaxRetries()));
+					} else {
+						map.put("", Thresholds.newThreshold());
+					}
+				} else if (eh instanceof GetMetaErrorHandler) {
+					if (pec.getCollectionProcessCompleteErrors() != null) {
+						map.put("", Thresholds.getThreshold("terminate", 0));
+					}
+				} else if (eh instanceof CpcErrorHandler) {
+					map.put("", Thresholds.getThreshold("", 0));
+				}
+			}
+
+		}
+
+	}
+
+	public UimaASService build(AnalysisEngineDeploymentDescriptionDocument dd, ControllerCallbackListener callback)
+			throws Exception {
+		// get the top level AnalysisEngine descriptor
+		String aeDescriptorPath = getAEDescriptorPath(dd);
+		// parse AE descriptor
+		ResourceSpecifier resourceSpecifier = UimaClassFactory.produceResourceSpecifier(aeDescriptorPath);
+		validateDD(dd, resourceSpecifier);
+		ServiceType serviceDefinition = getService(dd);
+		AnalysisEngineDelegate topLevelService;
+		// in DD the analysisEngine specification is optional
+		if (serviceDefinition.getAnalysisEngine() == null) {
+			topLevelService = new AnalysisEngineDelegate("");
+			topLevelService.setResourceSpecifier((AnalysisEngineDescription) resourceSpecifier);
+		} else {
+			topLevelService = parse(getService(dd).getAnalysisEngine());
+		}
+		UimaASJmsService service = null;
+
+		String endpoint = resolvePlaceholder(serviceDefinition.getInputQueue().getEndpoint());
+		String brokerURL = resolvePlaceholder(serviceDefinition.getInputQueue().getBrokerURL());
+
+		if (resourceSpecifier instanceof AnalysisEngineDescription) {
+			AnalysisEngineDescription aeDescriptor = (AnalysisEngineDescription) resourceSpecifier;
+			// Create a Top Level Service (TLS) wrapper.
+			service = new UimaASJmsService().withName(aeDescriptor.getAnalysisEngineMetaData().getName())
+					.withResourceSpecifier(resourceSpecifier).withBrokerURL(brokerURL).withInputQueue(endpoint);
+
+			this.buildAndDeploy(dd, topLevelService, service, callback);
+		}
+		return service;
+	}
+
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/AbstractUimaASDeployer.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/AbstractUimaASDeployer.java
new file mode 100644
index 0000000..feafab5
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/AbstractUimaASDeployer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.
+ */
+package org.apache.uima.as.deployer;
+
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.uima.aae.UimaASApplicationEvent.EventTrigger;
+import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.ControllerCallbackListener;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
+
+public abstract class AbstractUimaASDeployer 
+implements UimaAsServiceDeployer, ControllerCallbackListener {
+	CountDownLatch latch;
+	
+	protected AbstractUimaASDeployer(CountDownLatch latch) {
+		this.latch = latch;
+	}
+	public abstract UimaASService deploy(AnalysisEngineDeploymentDescriptionDocument dd, Map<String, String> deploymentProperties) throws Exception;
+	
+	public void waitUntilInitialized() throws InterruptedException {
+		latch.await();
+	}
+	@Override
+	public void notifyOnTermination(String aMessage, EventTrigger cause) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public void notifyOnInitializationFailure(AnalysisEngineController aController, Exception e) {
+		// TODO Auto-generated method stub
+		System.out.println("------- Controller:"+aController.getName()+" Exception During Initialization - Error:\n");
+		e.printStackTrace();
+	}
+
+	@Override
+	public void notifyOnInitializationSuccess(AnalysisEngineController aController) {
+		System.out.println("------- Controller:"+aController.getName()+" Initialized");
+		latch.countDown();
+	}
+
+	@Override
+	public void notifyOnInitializationFailure(Exception e) {
+		// TODO Auto-generated method stub
+		latch.countDown();
+
+	}
+
+	@Override
+	public void notifyOnInitializationSuccess() {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public void notifyOnReconnecting(String aMessage) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public void notifyOnReconnectionSuccess() {
+		// TODO Auto-generated method stub
+
+	}
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/ServiceDeployers.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/ServiceDeployers.java
new file mode 100644
index 0000000..29e1521
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/ServiceDeployers.java
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+package org.apache.uima.as.deployer;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.uima.as.deployer.direct.UimaAsDirectServiceDeployer;
+import org.apache.uima.as.deployer.jms.UimaAsJmsServiceDeployer;
+
+/*
+ * Concrete Factory class which creates instances of UimaAsServiceDeployer based
+ * type of protocol and provider. To make a new Deployer, add new protocol and
+ * provider to the enums below, and instantiate your deployer in newDeployer()
+ */
+public class ServiceDeployers {
+	public enum Protocol {
+		JAVA("java"), JMS("jms");
+		String protocol;
+
+		Protocol(String dt) {
+			protocol = dt;
+		}
+
+		public String get() {
+			return protocol;
+		}
+	}
+
+	public enum Provider {
+		JAVA("java"), ACTIVEMQ("activemq");
+		String provider;
+
+		Provider(String provider) {
+			this.provider = provider;
+		}
+
+		public String get() {
+			return provider;
+		}
+	}
+	/**
+	 * Creates instance of a deployer for a given protocol and provider.
+	 * 
+	 * @param protocol
+	 * @param provider
+	 * @return - 
+	 */
+	public static UimaAsServiceDeployer newDeployer(Protocol protocol, Provider provider) {
+
+		UimaAsServiceDeployer deployer = null;
+		if (Protocol.JAVA.equals(protocol) && Provider.JAVA.equals(provider)) {
+			deployer = new UimaAsDirectServiceDeployer(new CountDownLatch(1));
+		} else if (Protocol.JMS.equals(protocol) && Provider.ACTIVEMQ.equals(provider)) {
+			deployer = new UimaAsJmsServiceDeployer(new CountDownLatch(1));
+		}
+		return deployer;
+	}
+}
diff --git a/uimaj-as-jms/src/main/java/org/apache/uima/adapter/jms/message/PendingMessage.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/UimaAsServiceDeployer.java
similarity index 60%
copy from uimaj-as-jms/src/main/java/org/apache/uima/adapter/jms/message/PendingMessage.java
copy to uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/UimaAsServiceDeployer.java
index fa9571c..006eb80 100644
--- a/uimaj-as-jms/src/main/java/org/apache/uima/adapter/jms/message/PendingMessage.java
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/UimaAsServiceDeployer.java
@@ -16,20 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.uima.adapter.jms.message;
+package org.apache.uima.as.deployer;
 
-import java.util.HashMap;
+import java.util.Map;
 
-public class PendingMessage extends HashMap<Object, Object> {
- 
-private static final long serialVersionUID = 3512718154731557413L;
-private int messageType;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
 
-  public PendingMessage(int aMessageType) {
-    messageType = aMessageType;
-  }
+public interface UimaAsServiceDeployer {
+	public final String Deployment = "DEPLOYMENT";
+	
+	public enum DeploymentStrategy {
+		JMS, 
+		LOCAL;
+	};
+	public UimaASService deploy(AnalysisEngineDeploymentDescriptionDocument dd, Map<String, String> deploymentProperties) throws Exception;
+	public void waitUntilInitialized() throws InterruptedException;
 
-  public int getMessageType() {
-    return messageType;
-  }
 }
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/direct/UimaAsDirectServiceDeployer.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/direct/UimaAsDirectServiceDeployer.java
new file mode 100644
index 0000000..5e6c29d
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/direct/UimaAsDirectServiceDeployer.java
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+package org.apache.uima.as.deployer.direct;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.aae.service.builder.UimaAsDirectServiceBuilder;
+import org.apache.uima.as.deployer.AbstractUimaASDeployer;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
+
+public class UimaAsDirectServiceDeployer  extends AbstractUimaASDeployer {
+	public static void main(String[] args) {
+		String dd4 = "../uimaj-as-activemq/src/test/resources/deployment/Deploy_AggregateAnnotator.xml";
+		try {
+			CountDownLatch latch = new CountDownLatch(1);
+
+			UimaAsDirectServiceDeployer deployer = new UimaAsDirectServiceDeployer(latch);
+
+			Map<String, String> deploymentProperties = new HashMap<String, String>();
+
+			deploymentProperties.put(Deployment, DeploymentStrategy.LOCAL.name());
+
+			AnalysisEngineDeploymentDescriptionDocument dd = AnalysisEngineDeploymentDescriptionDocument.Factory
+					.parse(new File(dd4));
+
+			deployer.deploy(dd, deploymentProperties);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+		}
+
+	}
+
+	public UimaAsDirectServiceDeployer(CountDownLatch latch) {
+		// pass in a latch object which will block until service
+		// is initialized. The blocking will take place in super.waitUntilInitialized()
+		super(latch);
+		System.out.println("........ UimaAsDirectServiceDeployer() - Direct Deployment");
+	}
+
+	public UimaASService deploy(AnalysisEngineDeploymentDescriptionDocument dd,
+			Map<String, String> deploymentProperties) throws Exception {
+		UimaASService uimaAsService = null;
+		try {
+			uimaAsService = new UimaAsDirectServiceBuilder().build(dd, this);
+			// start listeners
+			uimaAsService.start();
+			// 
+			waitUntilInitialized();
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw e;
+		}
+		return uimaAsService;
+	}
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/jms/UimaAsJmsServiceDeployer.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/jms/UimaAsJmsServiceDeployer.java
new file mode 100644
index 0000000..43b7a52
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/deployer/jms/UimaAsJmsServiceDeployer.java
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+package org.apache.uima.as.deployer.jms;
+
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.adapter.jms.service.builder.UimaAsJmsServiceBuilder;
+import org.apache.uima.as.deployer.AbstractUimaASDeployer;
+import org.apache.uima.resourceSpecifier.AnalysisEngineDeploymentDescriptionDocument;
+
+public class UimaAsJmsServiceDeployer extends AbstractUimaASDeployer {
+	public UimaAsJmsServiceDeployer(CountDownLatch latch) {
+		super(latch);
+		System.out.println("........ UimaAsJmsServiceDeployer() - JMS Deployment");
+
+	}
+
+	public UimaASService deploy(AnalysisEngineDeploymentDescriptionDocument dd,
+			Map<String, String> deploymentProperties) throws Exception {
+		
+   	   UimaASService uimaAsService = null;
+		try {
+			uimaAsService = new UimaAsJmsServiceBuilder().build(dd, this);
+			// start listeners. Nothing happens unless JMS listeners start
+			uimaAsService.start();
+			// block till service is ready
+			waitUntilInitialized();
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw e;
+		}
+		return uimaAsService;
+	}
+
+	public static void main(String[] args) {
+
+	}
+
+}
diff --git a/uimaj-as-activemq/src/main/java/org/apache/uima/as/dispatcher/LocalDispatcher.java b/uimaj-as-activemq/src/main/java/org/apache/uima/as/dispatcher/LocalDispatcher.java
new file mode 100644
index 0000000..646cd6e
--- /dev/null
+++ b/uimaj-as-activemq/src/main/java/org/apache/uima/as/dispatcher/LocalDispatcher.java
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+package org.apache.uima.as.dispatcher;
+
+import java.util.concurrent.BlockingQueue;
+
+import org.apache.uima.UIMAFramework;
+import org.apache.uima.aae.UIMAEE_Constants;
+import org.apache.uima.aae.client.UimaASProcessStatus;
+import org.apache.uima.aae.client.UimaASProcessStatusImpl;
+import org.apache.uima.aae.message.AsynchAEMessage;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.adapter.jms.JmsConstants;
+import org.apache.uima.adapter.jms.client.BaseUIMAAsynchronousEngineCommon_impl;
+import org.apache.uima.adapter.jms.message.PendingMessage;
+import org.apache.uima.cas.CAS;
+import org.apache.uima.util.Level;
+import org.apache.uima.util.impl.ProcessTrace_impl;
+
+public class LocalDispatcher implements Runnable  {
+	private static final Class<LocalDispatcher> CLASS_NAME = LocalDispatcher.class;
+
+	private BlockingQueue<PendingMessage> messageQueue = null;
+	private BaseUIMAAsynchronousEngineCommon_impl client;
+	private UimaASService service;
+
+	public LocalDispatcher(BaseUIMAAsynchronousEngineCommon_impl client, UimaASService service,
+			BlockingQueue<PendingMessage> pendingMessageQueue) {
+		this.service = service;
+		this.client = client;
+		this.messageQueue = pendingMessageQueue;
+	}
+
+	private boolean reject(PendingMessage pm) {
+		return false;
+	}
+
+	private void dispatch(PendingMessage pm) throws Exception {
+		boolean doCallback = false;
+
+		switch (pm.getMessageType()) {
+		case AsynchAEMessage.GetMeta:
+			service.sendGetMetaRequest();
+			System.out.println("LocalDispatcher.dispatch()-dispatched getMeta Request");
+			break;
+
+		case AsynchAEMessage.Process:
+			doCallback = true;
+			service.process((CAS) pm.getProperty(AsynchAEMessage.CAS), pm.getPropertyAsString(AsynchAEMessage.CasReference));
+			System.out.println("LocalDispatcher.dispatch()-dispatched Process Request");
+			break;
+
+		case AsynchAEMessage.CollectionProcessComplete:
+			service.collectionProcessComplete();
+			System.out.println("LocalDispatcher.dispatch()-dispatched CPC Request");
+			break;
+		}
+        if ( doCallback ) {
+            UimaASProcessStatus status = new UimaASProcessStatusImpl(new ProcessTrace_impl(),(CAS)pm.getProperty(AsynchAEMessage.CAS),
+                    pm.getPropertyAsString(AsynchAEMessage.CasReference));
+            // Notify engine before sending a message
+            if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
+                UIMAFramework.getLogger(CLASS_NAME).logrb(
+                        Level.FINE,
+                        CLASS_NAME.getName(),
+                        "run",
+                        JmsConstants.JMS_LOG_RESOURCE_BUNDLE,
+                        "UIMAJMS_calling_onBeforeMessageSend__FINE",
+                        new Object[] {
+                          pm.getPropertyAsString(AsynchAEMessage.CasReference),
+                          String.valueOf( ((CAS)(pm.getProperty(AsynchAEMessage.CAS))).hashCode())
+                        });
+              }  
+            // Note the callback is a misnomer. The callback is made *after* the send now
+            // Application receiving this callback can consider the CAS as delivere to a queue
+            client.onBeforeMessageSend(status);
+          
+          
+          }
+	}
+	public void run() {
+
+		while (client.isRunning()) {
+			PendingMessage pm = null;
+			try {
+				System.out.println("LocalDispatcher.run()- waiting for new message ...");
+				pm = messageQueue.take();
+				System.out.println("LocalDispatcher.run()-got new message to dispatch");
+			} catch (InterruptedException e) {
+				
+				return;
+			}
+			// we may have waited in the take() above, so check if the client is still running
+			if (!client.isRunning() ) {
+				break; 
+			}
+			
+			boolean rejectRequest = reject(pm);
+			if (!rejectRequest && client.isRunning()) {
+				if (client.getServiceDelegate().isAwaitingPingReply()
+						&& pm.getMessageType() == AsynchAEMessage.GetMeta) {
+					if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
+						UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, getClass().getName(), "run",
+								JmsConstants.JMS_LOG_RESOURCE_BUNDLE, "UIMAJMS_client_dispatching_getmeta_ping__INFO",
+								new Object[] {});
+					}
+				}
+				try {
+					client.beforeDispatch(pm);
+					
+					dispatch(pm);
+				} catch (Exception e) {
+					if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
+						UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, getClass().getName(), "run",
+								UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_exception__WARNING", e);
+					}
+				}
+			}
+		}
+	}
+
+}
diff --git a/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASExtended.java b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASExtended.java
index 42713ac..9b02cd0 100644
--- a/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASExtended.java
+++ b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASExtended.java
@@ -58,6 +58,7 @@ import org.apache.activemq.command.ActiveMQDestination;
 import org.apache.log4j.Logger;
 import org.apache.uima.UIMAFramework;
 import org.apache.uima.UIMA_IllegalStateException;
+import org.apache.uima.aae.InputChannel.ChannelType;
 import org.apache.uima.aae.UimaClassFactory;
 import org.apache.uima.aae.client.UimaASProcessStatus;
 import org.apache.uima.aae.client.UimaAsBaseCallbackListener;
@@ -659,7 +660,7 @@ public class TestUimaASExtended extends BaseTestSupport {
   	    c.setDestinationName("TestQ");
   	    c.setConcurrentConsumers(2);
   	    c.setBeanName("testServiceWithHttpListeners() - JUnit Test Listener");
-  	    c.setMessageListener(new JmsInputChannel());
+  	    c.setMessageListener(new JmsInputChannel(ChannelType.REQUEST_REPLY));
   	    //c.initialize();
   	    //c.afterPropertiesSet();
   	    c.start();
diff --git a/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASNoErrors.java b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASNoErrors.java
new file mode 100644
index 0000000..dd8bb5b
--- /dev/null
+++ b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/TestUimaASNoErrors.java
@@ -0,0 +1,2285 @@
+/*
+ * 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.
+ */
+package org.apache.uima.ee.test;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.broker.BrokerService;
+
+import org.apache.uima.UIMAFramework;
+import org.apache.uima.aae.InputChannel.ChannelType;
+import org.apache.uima.aae.client.UimaASProcessStatus;
+import org.apache.uima.aae.client.UimaAsBaseCallbackListener;
+import org.apache.uima.aae.client.UimaAsynchronousEngine;
+import org.apache.uima.aae.client.UimaAsynchronousEngine.Transport;
+import org.apache.uima.aae.monitor.statistics.AnalysisEnginePerformanceMetrics;
+import org.apache.uima.aae.service.UimaASService;
+import org.apache.uima.aae.service.UimaAsServiceRegistry;
+import org.apache.uima.adapter.jms.JmsConstants;
+import org.apache.uima.adapter.jms.activemq.JmsInputChannel;
+import org.apache.uima.adapter.jms.client.BaseUIMAAsynchronousEngine_impl;
+import org.apache.uima.cas.CAS;
+import org.apache.uima.cas.TypeSystem;
+import org.apache.uima.collection.CollectionReader;
+import org.apache.uima.collection.CollectionReaderDescription;
+import org.apache.uima.collection.EntityProcessStatus;
+import org.apache.uima.ee.test.utils.BaseTestSupport;
+import org.apache.uima.ee.test.utils.UimaASJunitTestFailFastListener;
+import org.apache.uima.ee.test.utils.UimaASTestRunner;
+import org.apache.uima.internal.util.XMLUtils;
+import org.apache.uima.resource.ResourceInitializationException;
+import org.apache.uima.resourceSpecifier.factory.DeploymentDescriptorFactory;
+import org.apache.uima.resourceSpecifier.factory.UimaASPrimitiveDeploymentDescriptor;
+import org.apache.uima.resourceSpecifier.factory.impl.ServiceContextImpl;
+import org.apache.uima.util.XMLInputSource;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runner.notification.RunNotifier;
+import org.junit.runners.BlockJUnit4ClassRunner;
+
+import junit.framework.Assert;
+import org.apache.log4j.Logger;
+@RunWith(UimaASTestRunner.class)
+public class TestUimaASNoErrors extends BaseTestSupport {
+	
+	private Map<String, Object> defaultContext(String aTopLevelServiceQueueName) {
+		Map<String, Object> appCtx = new HashMap<>();
+	    appCtx.put(UimaAsynchronousEngine.ENDPOINT, aTopLevelServiceQueueName);
+	    appCtx.put(UimaAsynchronousEngine.CasPoolSize, Integer.valueOf(1));
+	    appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+	    appCtx.put(UimaAsynchronousEngine.GetMetaTimeout,0);
+
+		return appCtx;
+	}
+	
+	private String deployTopLevelService(Map<String, Object> appCtx, Transport transport, BaseUIMAAsynchronousEngine_impl client, String topLevelDescriptor, String topLevelQueueName) 
+	throws Exception {
+		String serviceId = null;
+
+		if (Transport.Java.equals(transport)) {
+			serviceId = deployJavaService(client, topLevelDescriptor);
+		    appCtx.put(UimaAsynchronousEngine.ServerUri, "java");
+			appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.Java);
+
+		} else if (Transport.JMS.equals(transport)) {
+		    appCtx.put(UimaAsynchronousEngine.ServerUri, String.valueOf(getMasterConnectorURI(broker)));
+			appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.JMS);
+			serviceId = deployJmsService(client, topLevelDescriptor);
+		} else {
+			throw new IllegalArgumentException(
+					"Invalid Client Transport - Expected either Transport.JMS or Transport.Java");
+		}
+		return serviceId;
+	}
+    private String getMasterConnectorURI(BrokerService b) {
+
+//    {
+//    		System.setProperty("ProtocolOverride", Protocol.JAVA.name());
+//    		System.setProperty("ProviderOverride", Provider.JAVA.name());
+//
+//   	}
+	return b.getDefaultSocketURIString();
+    }  
+    
+    
+
+    /*
+     * 
+     * 	 
+
+
+     	  
+     	  @Test
+	  public void testMultipleSyncClientsWithMultipleBrokers() throws Exception  {
+		    System.out.println("-------------- testMultipleSyncClientsWithMultipleBrokers -------------");
+		    
+		    class RunnableClient implements Runnable {
+		    	String brokerURL;
+		    	BaseTestSupport testSupport;
+	            BaseUIMAAsynchronousEngine_impl uimaAsEngine;
+		    	
+		    	RunnableClient(String brokerURL,BaseTestSupport testSupport) {
+		    		this.brokerURL = brokerURL;
+		    		this.testSupport = testSupport;
+		    	}
+		    	public void initialize(String dd, String serviceEndpoint) throws Exception {
+		    		uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+		            // Deploy Uima AS Primitive Service
+		            deployService(uimaAsEngine, dd);
+
+		    		@SuppressWarnings("unchecked")
+				  Map<String, Object> appCtx = buildContext(brokerURL, serviceEndpoint);
+			  	  appCtx.put(UimaAsynchronousEngine.Timeout, 1100);
+			  	  appCtx.put(UimaAsynchronousEngine.CpcTimeout, 1100);
+			  	  testSupport.initialize(uimaAsEngine, appCtx);
+			  	  waitUntilInitialized();
+		    	}
+				public void run() {
+					try {
+			            for (int i = 0; i < 1000; i++) {
+				              CAS cas = uimaAsEngine.getCAS();
+				              cas.setDocumentText("Some Text");
+			//	              System.out.println("UIMA AS Client#"+ Thread.currentThread().getId()+" Sending CAS#"+(i + 1) + " Request to a Service Managed by Broker:"+brokerURL);
+				              try {
+					                uimaAsEngine.sendAndReceiveCAS(cas);
+				              } catch( Exception e) {
+				            	  e.printStackTrace();
+				              } finally {
+				                cas.release();
+				              }
+				            }
+				            System.out.println("Thread:"+Thread.currentThread().getId()+" Completed run()");
+				            uimaAsEngine.stop();
+					} catch( Exception e) {
+						e.printStackTrace();
+					}
+
+				}
+		    	
+		    }
+		    
+		    ExecutorService executor = Executors.newCachedThreadPool();
+
+		    //	change broker URl in system properties
+		    System.setProperty("BrokerURL", getMasterConnectorURI(broker).toString());
+		    
+		    RunnableClient client1 = 
+		    		new RunnableClient(getMasterConnectorURI(broker), this);
+		    client1.initialize(relativePath + "/Deploy_NoOpAnnotatorWithPlaceholder.xml", "NoOpAnnotatorQueue");
+
+		    final BrokerService broker2 = setupSecondaryBroker(true);
+
+		    //	change broker URl in system properties
+		    System.setProperty("BrokerURL", broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+
+		    RunnableClient client2 = 
+		    		new RunnableClient(broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString(), this);
+		    client2.initialize(relativePath + "/Deploy_NoOpAnnotatorWithPlaceholder.xml", "NoOpAnnotatorQueue");
+
+		    Future<?> f1 = executor.submit(client1);
+		    Future<?> f2 = executor.submit(client2);
+		    f1.get();
+		    f2.get();
+		    executor.shutdownNow();
+		    while( !executor.isShutdown() ) {
+		    	synchronized(broker) {
+		    		broker.wait(500);
+		    	}
+		    }
+		    try {
+			    broker2.stop();
+//			    broker.stop();
+//			    broker.waitUntilStopped();
+
+		    } catch( Exception e) {
+		    	// ignore this one. Always thrown by AMQ on stop().
+		    }
+		}
+	  
+
+
+
+
+     	  
+
+
+
+ 
+	  
+
+     */
+	  /*
+	   * Tests Uima AS client placeholder handling and substitution. The Uima Aggregate instantiates
+	   * UIMA AS client proxy using Jms Client Descriptor that contains a placeholder
+	   * ${defaultBrokerURL} instead of hard coded Broker URL. The client parses the 
+	   * placeholder string, retrieves the name (defaultBrokerURL) and uses it to look
+	   * up tha actual broker URL in System properties.
+	   */
+	  /**
+	   * Tests use of a JMS Service Adapter and an override of the MultipleDeploymentAllowed. 
+	   * In this test, the AE descriptor of the remote service is configured with MultipleDeploymentAllowed=false
+	   * Without the override this causes an exception when instantiating Uima aggregate with
+	   * MultipleDeploymentAllowed=true. 
+	   * 
+	   * @throws Exception
+	   */
+ 
+    /*
+     * 
+     * Uncomment test !!!!!!!!!!!!!!!11
+
+	  */
+	  /**
+	   * Tests Uima-AS client ability to test sendAndReceive in multiple/concurrent threads It spawns 4
+	   * thread each sending 100 CASes to a Primitive Uima-AS service
+	   * 
+	   * @throws Exception
+	   */
+    /*
+	  @Test
+	  public void testSynchCallProcessWithMultipleThreads() throws Exception {
+	    System.out.println("-------------- testSynchCallProcessWithMultipleThreads -------------");
+	    int howManyCASesPerRunningThread = 100;
+	    int howManyRunningThreads = 4;
+	    runTestWithMultipleThreads(relativePath + "/Deploy_PersonTitleAnnotator.xml",
+	            "PersonTitleAnnotatorQueue", howManyCASesPerRunningThread, howManyRunningThreads, 0, 0);
+	  }
+	*/
+
+	/*
+
+	*/
+	@Test
+	public void testCancelProcessAggregateWithRemoteMultiplierOverJava() throws Exception {
+		// DOES NOT WORK
+		testCancelProcessAggregateWithRemoteMultiplier(Transport.Java);
+	}
+
+	@Test
+	public void testCancelProcessAggregateWithRemoteMultiplierOverJms() throws Exception {
+		// WORKS
+		testCancelProcessAggregateWithRemoteMultiplier(Transport.JMS);
+	}
+	public void testCancelProcessAggregateWithRemoteMultiplier(Transport transport) throws Exception {
+		System.out.println("-------------- testStopAggregateWithRemoteMultiplier -------------");
+		System.setProperty("BrokerURL", getMasterConnectorURI(broker));
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplierWith1MillionDocs.xml");
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithRemoteMultiplier.xml", "TopLevelTaeQueue");
+//		Service(eeUimaEngine, relativePath + "/Deploy_AggregateWithRemoteMultiplier.xml");
+		// Spin a thread to cancel Process after 20 seconds
+		spinShutdownThread(eeUimaEngine, 20000);
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				EXCEPTION_LATCH);
+	}
+	@Test
+	public void testProcessWithAggregateUsingRemoteMergerOverJava() throws Exception {
+		testProcessWithAggregateUsingRemoteMerger(Transport.Java);
+	}
+
+	@Test
+	public void testProcessWithAggregateUsingRemoteMergerOverJms() throws Exception {
+		testProcessWithAggregateUsingRemoteMerger(Transport.JMS);
+	}
+
+	public void testProcessWithAggregateUsingRemoteMerger(Transport transport) throws Exception {
+		System.out.println("-------------- testProcessWithAggregateUsingRemoteMerger -------------");
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMerger.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithRemoteMerger.xml",
+				"TopLevelTaeQueue");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				PROCESS_LATCH);
+	}
+
+	@Test
+	public void testProcessWithAggregateUsingCollocatedMergerOverJava() throws Exception {
+		testProcessWithAggregateUsingCollocatedMerger(Transport.Java);
+	}
+
+	@Test
+	public void testProcessWithAggregateUsingCollocatedMergerOverJms() throws Exception {
+		testProcessWithAggregateUsingCollocatedMerger(Transport.JMS);
+	}
+
+	public void testProcessWithAggregateUsingCollocatedMerger(Transport transport) throws Exception {
+		System.out.println("-------------- testProcessWithAggregateUsingCollocatedMerger -------------");
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_AggregateWithCollocatedMerger.xml", "TopLevelTaeQueue");
+
+		// deployJavaService(eeUimaEngine, relativePath +
+		// "/Deploy_AggregateWithCollocatedMerger.xml");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				PROCESS_LATCH);
+	}
+	@Test
+	public void testDeployAggregateServiceWithBrokerPlaceholderOverJava() throws Exception {
+		testDeployAggregateServiceWithBrokerPlaceholder(Transport.Java);
+	}
+
+	@Test
+	public void testDeployAggregateServiceWithBrokerPlaceholderOverJms() throws Exception {
+		testDeployAggregateServiceWithBrokerPlaceholder(Transport.JMS);
+	}
+
+	public void testDeployAggregateServiceWithBrokerPlaceholder(final Transport transport) throws Exception {
+		System.out.println("-------------- testDeployAggregateServiceWithBrokerPlaceholder -------------");
+		final BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+
+		System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+
+		try {
+			Thread t = new Thread() {
+				public void run() {
+					BrokerService bs = null;
+					try {
+						// at this point the top level service should show a connection error
+						synchronized (this) {
+							this.wait(5000); // wait for 5 secs
+						}
+						// Create a new broker that runs a different port that the rest of testcases
+						bs = setupSecondaryBroker(false);
+						System.setProperty("AggregateBroker",
+								bs.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+						System.setProperty("NoOpBroker",
+								bs.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+						deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorUsingPlaceholder.xml");
+						Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+						deployTopLevelService(appCtx, transport, eeUimaEngine,
+								relativePath + "/Deploy_AggregateAnnotatorUsingPlaceholder.xml", "TopLevelTaeQueue");
+
+						// deployJavaService(eeUimaEngine, relativePath
+						// + "/Deploy_AggregateAnnotatorUsingPlaceholder.xml");
+						// Start the uima AS client. It connects to the top level service and sends
+						// 10 messages
+					    if ( transport.equals(Transport.JMS)) {
+							appCtx.put(UimaAsynchronousEngine.ServerUri, System.getProperty("AggregateBroker"));
+					    }
+						runTest(appCtx, eeUimaEngine, System.getProperty("AggregateBroker"), "TopLevelTaeQueue", 1,
+								PROCESS_LATCH);
+					} catch (InterruptedException e) {
+					} catch (Exception e) {
+						e.printStackTrace();
+						fail(e.getMessage());
+					} finally {
+						if (bs != null) {
+							try {
+								bs.stop();
+								bs.waitUntilStopped();
+
+							} catch (Exception e) {
+								e.printStackTrace();
+							}
+						}
+					}
+				}
+			};
+			t.start();
+			t.join();
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testStopAggregateWithRemoteMultiplierOverJava() throws Exception {
+		testStopAggregateWithRemoteMultiplier(Transport.Java);
+	}
+	@Test
+	public void testStopAggregateWithRemoteMultiplierOverJms() throws Exception {
+		testStopAggregateWithRemoteMultiplier(Transport.JMS);
+	}
+	public void testStopAggregateWithRemoteMultiplier(Transport transport) throws Exception {
+		System.out.println("-------------- testStopAggregateWithRemoteMultiplier -------------");
+
+		System.setProperty("BrokerURL", getMasterConnectorURI(broker));
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplier.xml");
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorWithExceptionOn5thCAS.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_AggregateWithRemoteMultiplier.xml", "TopLevelTaeQueue");
+		// deployService(eeUimaEngine, relativePath +
+		// "/Deploy_AggregateWithRemoteMultiplier.xml");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				EXCEPTION_LATCH);
+	}
+
+	@Test
+	public void testAggregateHttpTunnellingOverJava() throws Exception {
+		testAggregateHttpTunnelling(Transport.Java);
+	}
+	@Test
+	public void testAggregateHttpTunnellingOverJms() throws Exception {
+		testAggregateHttpTunnelling(Transport.JMS);
+	}
+	public void testAggregateHttpTunnelling(Transport transport) throws Exception {
+		System.out.println("-------------- testAggregateHttpTunnelling -------------");
+
+		// Create Uima-AS Client
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		// Deploy remote service
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		// Deploy top level aggregate that communicates with the remote via Http
+		// Tunnelling
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_AggregateAnnotatorWithHttpDelegate.xml","TopLevelTaeQueue");
+//		deployJmsService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithHttpDelegate.xml");
+	    //appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.JMS);
+
+		// Initialize and run the Test. Wait for a completion and cleanup resources.
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 10, CPC_LATCH);
+	}
+	
+
+	@Test
+	public void testJmsServiceAdapterWithOverrideOverJava() throws Exception {
+		testJmsServiceAdapterWithOverride(Transport.Java);
+	}
+
+	@Test
+	public void testJmsServiceAdapterWithOverrideOverJms() throws Exception {
+		testJmsServiceAdapterWithOverride(Transport.JMS);
+	}
+
+	public void testJmsServiceAdapterWithOverride(Transport transport) throws Exception {
+		System.out.println("-------------- testJmsServiceAdapterWithOverride -------------");
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_SingleInstancePersonTitleAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_SyncAggregateWithJmsServiceAndScaleoutOverride.xml", "TopLevelTaeQueue");
+
+		// deployJavaService(eeUimaEngine, relativePath +
+		// "/Deploy_SyncAggregateWithJmsServiceAndScaleoutOverride.xml");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 10,
+				PROCESS_LATCH);
+	}
+    
+	@Test
+	public void testJmsServiceAdapterWithPlaceholderOverJava() throws Exception {
+		testJmsServiceAdapterWithPlaceholder(Transport.Java);
+	}
+
+	@Test
+	public void testJmsServiceAdapterWithPlaceholderOverJms() throws Exception {
+		testJmsServiceAdapterWithPlaceholder(Transport.JMS);
+	}
+
+	public void testJmsServiceAdapterWithPlaceholder(Transport transport) throws Exception {
+		System.out.println("-------------- testJmsServiceAdapterWithPlaceholder -------------");
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_SyncAggregateWithJmsServiceUsingPlaceholder.xml", "TopLevelTaeQueue");
+
+		// deployJavaService(eeUimaEngine, relativePath +
+		// "/Deploy_SyncAggregateWithJmsServiceUsingPlaceholder.xml");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 10,
+				PROCESS_LATCH);
+	}
+	@Test
+	public void testScaledSyncAggregateProcessOverJava() throws Exception {
+		testScaledSyncAggregateProcess(Transport.Java);
+	}
+
+	@Test
+	public void testScaledSyncAggregateProcessOverJms() throws Exception {
+		testScaledSyncAggregateProcess(Transport.JMS);
+	}
+
+	public void testScaledSyncAggregateProcess(Transport transport) throws Exception {
+		System.out.println("-------------- testScaledSyncAggregateProcess -------------");
+		// Instantiate Uima-AS Client
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		// Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine,
+				relativePath + "/Deploy_ScaledPrimitiveAggregateAnnotator.xml", "TopLevelTaeQueue");
+
+		// deployJavaService(eeUimaEngine, relativePath +
+		// "/Deploy_ScaledPrimitiveAggregateAnnotator.xml");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 5,
+				PROCESS_LATCH);
+		System.out.println(eeUimaEngine.getPerformanceReport());
+	}
+    @Test
+	public void testComplexDeploymentOverJava() throws Exception {
+    	testComplexDeployment(Transport.Java);
+    }
+    @Test
+  	public void testComplexDeploymentOverJms() throws Exception {
+      	testComplexDeployment(Transport.JMS);
+      }
+     public void testComplexDeployment(Transport transport) throws Exception {
+	    System.out.println("-------------- testComplexDeployment -------------");
+	    Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+	    
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy replicated services for the inner remote aggregate CM
+	    StringBuilder sb = new StringBuilder(relativePath).append("/Deploy_NoOpAnnotator.xml");
+	    deployService(transport, eeUimaEngine, sb.toString()) ; //relativePath + "/Deploy_NoOpAnnotator.xml");
+//	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+//	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+//	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    // Deploy an instance of a remote aggregate CM containing a collocated Cas Multiplier
+	    // CM --> Replicated Remote Primitive --> NoOp CC
+//	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_CMAggregateWithCollocatedCM.xml");
+	    sb.setLength(0);   // clear
+	    sb.append(relativePath).append("/Deploy_CMAggregateWithCollocatedCM.xml");
+	    deployService(transport, eeUimaEngine, sb.toString()); //relativePath + "/Deploy_CMAggregateWithCollocatedCM.xml");
+	    // Deploy top level Aggregate Cas Multiplier with 2 collocated Cas Multipliers
+	    // CM1 --> CM2 --> Remote AggregateCM --> Candidate Answer --> CC
+	    deployTopLevelService(appCtx, transport,eeUimaEngine,relativePath + "/Deploy_TopLevelComplexAggregateCM.xml","TopLevelTaeQueue");
+	    runTest2(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "TopLevelTaeQueue", 1, PROCESS_LATCH);
+	  }
+	  @Test
+	  public void testDeployAggregateWithCollocatedAggregateServiceOverJava() throws Exception {
+		  testDeployAggregateWithCollocatedAggregateService(Transport.Java);
+	  }
+	  @Test
+	  public void testDeployAggregateWithCollocatedAggregateServiceOverJms() throws Exception {
+		  testDeployAggregateWithCollocatedAggregateService(Transport.JMS);
+	  }
+
+	public void testDeployAggregateWithCollocatedAggregateService(Transport transport) throws Exception {
+		System.out.println("-------------- testDeployAggregateWithCollocatedAggregateService -------------");
+	    Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployTopLevelService(appCtx,transport,eeUimaEngine,relativePath + "/Deploy_ComplexAggregate.xml","TopLevelTaeQueue");
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 10,
+				PROCESS_LATCH);
+
+	}
+	@Test
+	public void testProcessWithAggregateUsingRemoteMultiplierOnSeparateBrokerOverJava() throws Exception {
+		testProcessWithAggregateUsingRemoteMultiplierOnSeparateBroker(Transport.Java);
+	}
+	@Test
+	public void testProcessWithAggregateUsingRemoteMultiplierOnSeparateBrokerOverJms() throws Exception {
+		testProcessWithAggregateUsingRemoteMultiplierOnSeparateBroker(Transport.JMS);
+	}
+	
+	public void testProcessWithAggregateUsingRemoteMultiplierOnSeparateBroker(Transport transport) throws Exception {
+		System.out
+				.println("-------------- testProcessWithAggregateUsingRemoteMultiplierOnSeparateBroker -------------");
+		System.setProperty("activemq.broker.jmx.domain", "org.apache.activemq.test");
+		BrokerService broker2 = setupSecondaryBroker(true);
+		System.setProperty("BrokerURL", broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+	    Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplier.xml");
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployTopLevelService(appCtx,transport,eeUimaEngine,relativePath + "/Deploy_AggregateWithRemoteMultiplier.xml","TopLevelTaeQueue");
+
+//		Map<String, Object> appCtx = new HashMap<>();
+		appCtx.put(UimaAsynchronousEngine.ServerUri,
+				broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+//		appCtx.put(UimaAsynchronousEngine.ENDPOINT, "TopLevelTaeQueue");
+//		appCtx.put(UimaAsynchronousEngine.GetMetaTimeout, 0);
+		runTest(appCtx, eeUimaEngine, broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString(),
+				"TopLevelTaeQueue", 10, PROCESS_LATCH);
+		super.cleanBroker(broker2);
+
+		broker2.stop();
+		broker2.waitUntilStopped();
+
+	}
+
+	 @Test
+	 public void testProcessWithAggregateUsingRemoteMultiplierOverJava() throws Exception {
+		 testProcessWithAggregateUsingRemoteMultiplier(Transport.Java);
+	 }
+	 @Test
+	 public void testProcessWithAggregateUsingRemoteMultiplierOverJms() throws Exception {
+		 testProcessWithAggregateUsingRemoteMultiplier(Transport.JMS);
+	 }
+
+	public void testProcessWithAggregateUsingRemoteMultiplier(Transport transport) throws Exception {
+		System.out.println("-------------- testProcessWithAggregateUsingRemoteMultiplier -------------");
+		System.setProperty("BrokerURL", getMasterConnectorURI(broker));
+	    Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplier.xml");
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployTopLevelService(appCtx, transport,eeUimaEngine,relativePath + "/Deploy_AggregateWithRemoteMultiplier.xml","TopLevelTaeQueue");
+
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				PROCESS_LATCH);
+	}
+
+ @Test
+ public void testClientProcessWithRemoteMultiplierOverJava() throws Exception {
+	 testClientProcessWithRemoteMultiplier(Transport.Java);
+ }
+ @Test
+ public void testClientProcessWithRemoteMultiplierOverJms() throws Exception {
+	 testClientProcessWithRemoteMultiplier(Transport.JMS);
+ }
+ public void testClientProcessWithRemoteMultiplier(Transport transport) throws Exception {
+   System.out.println("-------------- testClientProcessWithRemoteMultiplier -------------");
+   Map<String, Object> appCtx = defaultContext("TestMultiplierQueue");
+
+   BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+   deployTopLevelService(appCtx,transport,eeUimaEngine,relativePath + "/Deploy_RemoteCasMultiplier.xml","TestMultiplierQueue");
+
+//   Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//           "TestMultiplierQueue");
+   appCtx.remove(UimaAsynchronousEngine.ShadowCasPoolSize);
+   appCtx.put(UimaAsynchronousEngine.ShadowCasPoolSize, Integer.valueOf(1));
+   runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+           "TestMultiplierQueue", 1, PROCESS_LATCH);
+ }
+
+ @Test
+ public void testClientProcessWithComplexAggregateRemoteMultiplierOverJava() throws Exception {
+	 testClientProcessWithComplexAggregateRemoteMultiplier(Transport.Java);
+ }
+ @Test
+ public void testClientProcessWithComplexAggregateRemoteMultiplierOverJms() throws Exception {
+	 testClientProcessWithComplexAggregateRemoteMultiplier(Transport .JMS);
+ }
+ public void testClientProcessWithComplexAggregateRemoteMultiplier(Transport transport) throws Exception {
+
+   System.out
+           .println("-------------- testClientProcessWithComplexAggregateRemoteMultiplier -------------");
+   Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+   BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+   deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+   deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplierWith10Docs_1.xml");
+   deployTopLevelService(appCtx, transport,eeUimaEngine,relativePath + "/Deploy_CasMultiplierAggregateWithRemoteCasMultiplier.xml","TopLevelTaeQueue");
+   runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+           1, PROCESS_LATCH);
+ }
+
+ @Test
+ public void testProcessWithAggregateUsing2RemoteMultipliersOverJava() throws Exception {
+	 testProcessWithAggregateUsing2RemoteMultipliers(Transport.Java);
+ }
+ @Test
+ public void testProcessWithAggregateUsing2RemoteMultipliersOverJms() throws Exception {
+	 testProcessWithAggregateUsing2RemoteMultipliers(Transport.JMS);
+ }
+ public void testProcessWithAggregateUsing2RemoteMultipliers(Transport transport) throws Exception {
+   System.out
+           .println("-------------- testProcessWithAggregateUsing2RemoteMultipliers -------------");
+   Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+   BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+   deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+   deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplierWith10Docs_1.xml");
+   deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplierWith10Docs_2.xml");
+   deployTopLevelService(appCtx, transport,eeUimaEngine,relativePath + "/Deploy_AggregateWith2RemoteMultipliers.xml","TopLevelTaeQueue");
+
+   runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+           1, PROCESS_LATCH);
+ }
+
+
+	  @Test
+	  public void testClientWithAggregateMultiplierOverJava() throws Exception {
+		  testClientWithAggregateMultiplier(Transport.Java);
+	  }
+	  @Test
+	  public void testClientWithAggregateMultiplierOverJms() throws Exception {
+		  testClientWithAggregateMultiplier(Transport.JMS);
+	  }
+
+	public void testClientWithAggregateMultiplier(Transport transport) throws Exception {
+		System.out.println("-------------- testClientWithAggregateMultiplier -------------");
+		System.setProperty("BrokerURL", broker.getConnectorByName(DEFAULT_BROKER_URL_KEY).getUri().toString());
+		//Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+
+		BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplier.xml");
+		deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateMultiplier.xml","TopLevelTaeQueue");
+//		Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue");
+		appCtx.remove(UimaAsynchronousEngine.ShadowCasPoolSize);
+		appCtx.put(UimaAsynchronousEngine.ShadowCasPoolSize, Integer.valueOf(1));
+		runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue", 1,
+				PROCESS_LATCH);
+		System.out.println("-------------- End testClientWithAggregateMultiplier -------------");
+
+	}
+	   @Test
+	    public void testClient() throws Exception {
+	      System.out.println("-------------- testClient -------------");
+	      System.setProperty("BrokerURL", broker.getConnectorByName(DEFAULT_BROKER_URL_KEY).getUri().toString());
+
+	      BaseUIMAAsynchronousEngine_impl uimaAsEngine1 = new BaseUIMAAsynchronousEngine_impl();
+	      BaseUIMAAsynchronousEngine_impl uimaAsEngine2 = new BaseUIMAAsynchronousEngine_impl();
+	      String sid1= deployJmsService(uimaAsEngine1, relativePath.concat( "/Deploy_AggregateMultiplierWith30SecDelay.xml") );
+	      String sid2 = deployJavaService(uimaAsEngine2, relativePath + "/Deploy_AggregateMultiplierWith30SecDelay.xml");
+	      
+	      uimaAsEngine1.undeploy(sid1);
+	      
+	      uimaAsEngine2.undeploy(sid2);
+	    }
+	    
+	    @Test
+	    public void testClientWithPrimitives() throws Exception {
+	      System.out.println("-------------- testClientRecoveryFromBrokerFailure -------------");
+	      System.setProperty("BrokerURL", broker.getConnectorByName(DEFAULT_BROKER_URL_KEY).getUri().toString());
+
+	      BaseUIMAAsynchronousEngine_impl uimaAsEngine1 = new BaseUIMAAsynchronousEngine_impl();
+	      BaseUIMAAsynchronousEngine_impl uimaAsEngine2 = new BaseUIMAAsynchronousEngine_impl();
+	      String sid1= deployJmsService(uimaAsEngine1, relativePath + "/Deploy_NoOpAnnotator.xml");
+	      String sid2 = deployJavaService(uimaAsEngine2, relativePath + "/Deploy_NoOpAnnotator.xml");
+	      
+	      uimaAsEngine1.undeploy(sid1);
+	      
+	      uimaAsEngine2.undeploy(sid2);
+	      
+	    }
+
+	        @Test
+	    public void testServiceWithHttpListeners() throws Exception {
+	  	    System.out.println("-------------- testServiceWithHttpListeners -------------");
+	  	    // Need java monitor object on which to sleep
+	  	    Object waitObject = new Object();
+	  	    // Custom spring listener with handleListenerSetupFailure() overriden to 
+	  	    // capture AMQ exception.
+	  	    TestDefaultMessageListenerContainer c = new TestDefaultMessageListenerContainer();
+	  	    c.setConnectionFactory(new ActiveMQConnectionFactory("http://localhost:18888"));
+	  	    c.setDestinationName("TestQ");
+	  	    c.setConcurrentConsumers(2);
+	  	    c.setBeanName("TestBean");
+	  	    c.setMessageListener(new JmsInputChannel(ChannelType.REQUEST_REPLY));
+	  	    c.initialize();
+	  	    c.start();
+	  	    
+	  	    if ( c.isRunning() ) {
+	  		    System.out.println("... Listener Ready");
+	  	    	
+	  	    }
+	  	    // Keep-alive has a default 30 secs timeout. Sleep for bit longer than that
+	  	    // If there is an exception due to keep-alive, an exception handler will be
+	  	    // called on the TestDefaultMessageListenerContainer instance where we 
+	  	    // capture the error.
+	  	    System.out.println("... Waiting for 40 secs");
+	  	    try {
+	  	    	synchronized(waitObject) {
+	  	    		waitObject.wait(40000);
+	  	    	}
+	  	    	// had there been broker issues relateds to keep-alive the listener's failed
+	  	    	// flag would have been set by now. Check it and fail the test 
+	  	    	if ( c.failed() ) {
+	  		    	fail("Broker Failed - Reason:"+c.getReasonForFailure());
+	  	    	} else {
+	  	    		System.out.println("Stopping Listener");
+	  	    		c.stop();
+
+	  	    	}
+	  	    } catch( Exception e) {
+	  	    	e.printStackTrace();
+	  	    	fail(e.getMessage());
+	  	    }
+	    }
+
+	        
+	  @Test
+	  public void testCompressedTypeFilteringOverJava() throws Exception {
+		  testCompressedTypeFiltering(Transport.Java);
+	  }
+	  
+	  @Test
+	  public void testCompressedTypeFilteringOverJms() throws Exception {
+		  testCompressedTypeFiltering(Transport.JMS);
+	  }
+	  public void testCompressedTypeFiltering(Transport transport) throws Exception {
+	    System.out.println("-------------- testCompressedTypeFiltering -------------");
+	    // Instantiate Uima-AS Client
+	    final BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+  	    deployJmsService(uimaAsEngine, relativePath + "/Deploy_RoomNumberAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("MeetingDetectorTaeQueue");
+		deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_MeetingDetectorTAE_RemoteRoomNumberBinary.xml","MeetingDetectorTaeQueue");
+
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)), "MeetingDetectorTaeQueue");
+	    // Set an explicit getMeta (Ping)timeout
+	    appCtx.put(UimaAsynchronousEngine.GetMetaTimeout, 2000);
+	    // Set an explicit process timeout so to test the ping on timeout
+	    appCtx.put(UimaAsynchronousEngine.Timeout, 1000);
+	    appCtx.put(UimaAsynchronousEngine.SERIALIZATION_STRATEGY, "binary");
+
+	    runTest(appCtx, uimaAsEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "MeetingDetectorTaeQueue", 1, PROCESS_LATCH);
+	  }
+
+	  
+	  /**
+	   * Tests Broker startup and shutdown
+	   */
+	  @Test
+	  public void testBrokerLifecycle() {
+	    System.out.println("-------------- testBrokerLifecycle -------------");
+	    System.out.println("UIMA_HOME=" + System.getenv("UIMA_HOME")
+	            + System.getProperty("file.separator") + "bin" + System.getProperty("file.separator")
+	            + "dd2spring.xsl");
+	  }
+	  @Test
+	  public void testGenerateAndDeployPrimitiveDDOverJava() throws Exception {
+		  testGenerateAndDeployPrimitiveDD(Transport.Java);
+	  }
+	  @Test
+	  public void testGenerateAndDeployPrimitiveDDOverJms() throws Exception {
+		  testGenerateAndDeployPrimitiveDD(Transport.JMS);
+	  }
+	  public void testGenerateAndDeployPrimitiveDD(Transport transport) throws Exception {
+		    System.out.println("-------------- testGenerateAndDeployPrimitiveDD -------------");
+		  File directory = new File (".");
+		  // Set up a context object containing basic service deployment
+		  // information
+		  org.apache.uima.resourceSpecifier.factory.ServiceContext context = new ServiceContextImpl("PersonTitle",
+				  "PersonTitle Annotator Description",
+				  directory.getCanonicalPath() + 
+				  System.getProperty("file.separator")+
+				  resourceDirPath+
+				  System.getProperty("file.separator")+
+				  "descriptors" +
+				  System.getProperty("file.separator")+
+				  "analysis_engine" +
+				  System.getProperty("file.separator")+
+				  "PersonTitleAnnotator.xml", 
+				  "PersonTitleAnnotatorQueue",
+				  getMasterConnectorURI(broker));
+		  context.setCasPoolSize(2);
+		  // create DD with default settings
+		  UimaASPrimitiveDeploymentDescriptor dd = DeploymentDescriptorFactory
+				  .createPrimitiveDeploymentDescriptor(context);
+
+		  // Get default Error Handler for process and change error threshold
+		  dd.getProcessErrorHandlingSettings().setThresholdCount(4);
+
+		  // Two instances of AE in a jvm
+		  dd.setScaleup(2);
+
+		  // Generate deployment descriptor in xml format
+		  String ddXML = dd.toXML();
+		  System.out.println(ddXML);
+		  
+		  File tempFile = File.createTempFile("Deploy_PersonTitle", ".xml");
+		  BufferedWriter out = new BufferedWriter(new FileWriter(tempFile));
+		  out.write(ddXML);
+		  out.close();
+		  char FS = System.getProperty("file.separator").charAt(0);
+		  
+		// create Map to hold required parameters
+		  Map<String,Object> appCtx = new HashMap<String,Object>();
+		  
+		  BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		  String aSpringContainerId =
+				  deployTopLevelService(appCtx, transport, eeUimaEngine, tempFile.getAbsolutePath(), "");
+
+//		      eeUimaEngine.deploy(tempFile.getAbsolutePath(), appCtx);
+		  
+		  eeUimaEngine.undeploy(aSpringContainerId);
+		  eeUimaEngine.stop();
+		  
+		  
+	  }
+	  @Test
+	  public void testSendAndReceiveOverJava() throws Exception {
+		  testSendAndReceive(Transport.Java);
+	  }
+	  
+	  @Test
+	  public void testSendAndReceiveOverJms() throws Exception {
+		  testSendAndReceive(Transport.JMS);
+	  }
+	  public void testSendAndReceive(Transport transport) throws Exception  {
+	      BaseUIMAAsynchronousEngine_impl uimaAsEngine 
+	      	= new BaseUIMAAsynchronousEngine_impl();
+	      deployJmsService(uimaAsEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	      deployJmsService(uimaAsEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		  Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		  deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_AggregateAnnotator.xml", "TopLevelTaeQueue");
+	      
+//	      appCtx.put(UimaAsynchronousEngine.Timeout, 1100);
+	      appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+	      appCtx.put(UimaAsynchronousEngine.CpcTimeout, 1100);
+	      initialize(uimaAsEngine, appCtx);
+	      waitUntilInitialized();
+	      int errorCount = 0;
+	      List<AnalysisEnginePerformanceMetrics> componentMetricsList = 
+	    		  new ArrayList<AnalysisEnginePerformanceMetrics>();
+	      for (int i = 0; i < 1; i++) {
+	        CAS cas = uimaAsEngine.getCAS();
+	        cas.reset();
+	        cas.setDocumentText("Some Text");
+	  //      System.out.println("UIMA AS Client Sending CAS#" + (i + 1) + " Request to a Service");
+	        try {
+	          uimaAsEngine.sendAndReceiveCAS(cas,componentMetricsList);
+	          System.out.println("-------> Client Received Performance Metrics of Size:"+componentMetricsList.size());
+	          for( AnalysisEnginePerformanceMetrics m :componentMetricsList ) {
+	        	  System.out.println(".............. Component:"+m.getName()+" AnalysisTime:"+m.getAnalysisTime());
+	          }
+	      //  	uimaAsEngine.sendCAS(cas);
+	          System.out.println("----------------------------------------------------");
+	          componentMetricsList.clear();
+	        } catch( Exception e) {
+	          errorCount++;
+	        } finally {
+	          cas.release();
+	          componentMetricsList.clear();
+	        }
+	      }
+	      
+	      
+	      Map<String, List<UimaASService>> services =
+	    		  UimaAsServiceRegistry.getInstance().getServiceList();
+		  for( Entry<String, List<UimaASService>> serviceListEntry : services.entrySet()) {
+				Iterator<UimaASService> listIterator = serviceListEntry.getValue().iterator();
+				while( listIterator.hasNext()) {
+					UimaASService service = listIterator.next();
+			    	  System.out.println("Registered Service:"+service.getName()+" Queue:"+service.getEndpoint());
+					
+				}
+		  }	
+	      uimaAsEngine.stop();
+	  }
+
+	  
+	  
+	  @Test
+	  public void testClientHttpTunnellingToAggregate() throws Exception {
+		  System.out.println("-------------- testClientHttpTunnellingToAggregate -------------");
+	    // Add HTTP Connector to the broker. 
+	    String httpURI = getHttpURI();
+	    // Create Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy remote service
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml");
+	    // Initialize and run the Test. Wait for a completion and cleanup resources.
+	    System.out.println("-------- Connecting Client To Service: "+httpURI);
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+//		deployTopLevelService(appCtx, Transport.JMS, eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml", "NoOpAnnotatorQueue");
+	    appCtx.put(UimaAsynchronousEngine.ServerUri, httpURI);
+	    appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.JMS);
+
+	    
+	    runTest(appCtx, eeUimaEngine, httpURI, "TopLevelTaeQueue", 1, CPC_LATCH);
+	  }
+	  @Test
+	  public void testClientHttpTunnelling() throws Exception {
+	    System.out.println("-------------- testClientHttpTunnelling -------------");
+	    String httpURI = getHttpURI();
+	    // Create Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("NoOpAnnotatorQueue");
+		deployTopLevelService(appCtx, Transport.JMS, eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml", "NoOpAnnotatorQueue");
+
+	    // Deploy remote service
+//	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    // Initialize and run the Test. Wait for a completion and cleanup resources.
+	    System.out.println("-------- Connecting Client To Service: "+httpURI);
+	    appCtx.put(UimaAsynchronousEngine.ServerUri, httpURI);
+	    appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.JMS);
+
+	    runTest(appCtx, eeUimaEngine, httpURI, "NoOpAnnotatorQueue", 1, PROCESS_LATCH);
+	  }
+
+
+	  @Test
+	  public void testClientHttpTunnellingWithDoubleByteText() throws Exception {
+	    System.out.println("-------------- testClientHttpTunnellingWithDoubleByteText -------------");
+
+	    BufferedReader in = null;
+	    try {
+	      File file = new File(relativeDataPath + "/DoubleByteText.txt");
+	      System.out.println("Checking for existence of File:" + file.getAbsolutePath());
+	      // Process only if the file exists
+	      if (file.exists()) {
+	        System.out
+	                .println(" *** DoubleByteText.txt exists and will be sent through http connector.");
+	        System.out.println(" ***   If the vanilla activemq release is being used,");
+	        System.out
+	                .println(" ***   and DoubleByteText.txt is bigger than 64KB or so, this test case will hang.");
+	        System.out
+	                .println(" *** To fix, override the classpath with the jar files in and under the");
+	        System.out
+	                .println(" ***   apache-uima-as/uima-as/src/main/apache-activemq-X.y.z directory");
+	        System.out.println(" ***   in the apache-uima-as source distribution.");
+
+	        String httpURI = getHttpURI();
+	      // Create Uima-AS Client
+	        BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	        // Deploy remote service
+			Map<String, Object> appCtx = defaultContext("NoOpAnnotatorQueue");
+			deployTopLevelService(appCtx, Transport.JMS, eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml", "NoOpAnnotatorQueue");
+
+//	        deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	        
+	        InputStream fis = new FileInputStream(file);
+	        Reader rd = new InputStreamReader(fis, "UTF-8");
+	        in = new BufferedReader(rd);
+	        // Set the double-byte text. This is what will be sent to the service
+	        String line = in.readLine();
+	        super.setDoubleByteText(line);
+	        int err = XMLUtils.checkForNonXmlCharacters(line);
+	        if (err >= 0) {
+	          fail("Illegal XML char at offset " + err);
+	        }
+	        System.out.println("-------- Connecting Client To Service: "+httpURI);
+	        // Initialize and run the Test. Wait for a completion and cleanup resources.
+		    appCtx.put(UimaAsynchronousEngine.ServerUri, httpURI);
+		    appCtx.put(UimaAsynchronousEngine.ClientTransport, Transport.JMS);
+
+	        runTest(appCtx, eeUimaEngine, httpURI, "NoOpAnnotatorQueue", 1, CPC_LATCH);
+	      }
+	    } catch (Exception e) {
+	      // Double-Byte Text file not present. Continue on with the next test
+	      e.printStackTrace();
+	      fail("Could not complete test");
+	    } finally {
+	      if (in != null) {
+	        in.close();
+	      }
+	    }
+	  }
+
+	  @Test
+	  public void testClientCRProcessOverJava() throws Exception {
+		  testClientCRProcess(Transport.Java);
+	  }
+	  @Test
+	  public void testClientCRProcessOverJms() throws Exception {
+		  testClientCRProcess(Transport.JMS);
+	  }
+	  public void testClientCRProcess(Transport transport) throws Exception {
+		    System.out.println("-------------- testClientCRProcess -------------");
+		    super.resetCASesProcessed();
+		    
+		    // Instantiate Uima AS Client
+		    final BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+//		    UIMAFramework.getLogger(BaseUIMAAsynchronousEngineCommon_impl.class).setLevel(Level.FINEST);
+//		    UIMAFramework.getLogger(BaseUIMAAsynchronousEngine_impl.class).setLevel(Level.FINEST);
+//		    UIMAFramework.getLogger().setLevel(Level.FINEST);
+//		    UIMAFramework.getLogger().setOutputStream(System.out);
+		    // Deploy Uima AS Primitive Service
+			Map<String, Object> appCtx = defaultContext("NoOpAnnotatorQueueLongDelay");
+			deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_NoOpAnnotatorWithLongDelay.xml", "NoOpAnnotatorQueueLongDelay");
+
+//		    deployJavaService(uimaAsEngine, relativePath + "/Deploy_NoOpAnnotatorWithLongDelay.xml");
+//		    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//		            "NoOpAnnotatorQueueLongDelay");
+//		    appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+		    appCtx.put(UimaAsynchronousEngine.CpcTimeout, 1100);
+		    appCtx.put(UimaAsynchronousEngine.CasPoolSize,10);
+		    
+		    String collectionReaderDescriptor =
+		    		resourceDirPath + System.getProperty("file.separator") +
+		    		"descriptors"+ System.getProperty("file.separator") +
+		    		"collection_reader"+ System.getProperty("file.separator") +
+		    		"FileSystemCollectionReader.xml";
+		   		  
+	         // add Collection Reader if specified
+	         try {
+	             CollectionReaderDescription collectionReaderDescription = 
+	                     UIMAFramework.getXMLParser()
+	                             .parseCollectionReaderDescription(new XMLInputSource(collectionReaderDescriptor));
+	             collectionReaderDescription.getCollectionReaderMetaData().
+	             	getConfigurationParameterSettings().
+	             		setParameterValue("InputDirectory", relativeDataPath);
+	             CollectionReader collectionReader = UIMAFramework
+	                     .produceCollectionReader(collectionReaderDescription);
+	             uimaAsEngine.setCollectionReader(collectionReader);	    
+	         } catch( Throwable e) {
+	        	 e.printStackTrace();
+	         }
+		    
+		    initialize(uimaAsEngine, appCtx);
+		    waitUntilInitialized();
+		    
+		    uimaAsEngine.process();
+
+		    Assert.assertEquals(8, getNumberOfCASesProcessed());
+		    System.clearProperty("DefaultBrokerURL");
+		    uimaAsEngine.stop();
+		  }
+
+	  @Test
+	  public void testClientProcessOverJava() throws Exception {
+		  testClientProcess(Transport.Java);
+	  }
+	  @Test
+	  public void testClientProcessOverJms() throws Exception {
+		  testClientProcess(Transport.JMS);
+	  }
+	  public void testClientProcess(Transport transport) throws Exception {
+	    System.out.println("-------------- testClientProcess -------------");
+	    
+	    // Instantiate Uima AS Client
+	    final BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml", "PersonTitleAnnotatorQueue");
+
+//	    deployJavaService(uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+	    
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//				"PersonTitleAnnotatorQueue");
+	  
+	    
+//	    Map<String, Object> appCtx = buildContext("tcp://localhost:61616",
+//				"TopLevelTaeQueue");
+//	    appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+	    appCtx.put(UimaAsynchronousEngine.CpcTimeout, 1100);
+	    appCtx.put(UimaAsynchronousEngine.CasPoolSize,2);
+	    initialize(uimaAsEngine, appCtx);
+	    waitUntilInitialized();
+
+	    for (int i = 0; i < 500; i++) {
+	      CAS cas = uimaAsEngine.getCAS();
+	      cas.setDocumentText("Some Text");
+	      System.out.println("UIMA AS Client Sending CAS#" + (i + 1) + " Request to a Service");
+	      uimaAsEngine.sendCAS(cas);
+	    }
+	    
+	    uimaAsEngine.collectionProcessingComplete();
+	    System.clearProperty("DefaultBrokerURL");
+	    uimaAsEngine.stop();
+	  }
+	  
+	 @Test
+	  public void testClientBrokerPlaceholderSubstitutionOverJava() throws Exception {
+		 testClientBrokerPlaceholderSubstitution(Transport.Java);
+	 }
+	 @Test
+	  public void testClientBrokerPlaceholderSubstitutionOverJms() throws Exception {
+		 testClientBrokerPlaceholderSubstitution(Transport.JMS);
+	 }
+	  public void testClientBrokerPlaceholderSubstitution(Transport transport) throws Exception {
+	    System.out.println("-------------- testClientBrokerPlaceholderSubstitution -------------");
+	    System.setProperty( "defaultBrokerURL", getMasterConnectorURI(broker));
+	    // Instantiate Uima AS Client
+	    BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml", "PersonTitleAnnotatorQueue");
+
+	  //  deployJavaService(uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+
+	   // Map<String, Object> appCtx = buildContext("${defaultBrokerURL}","PersonTitleAnnotatorQueue");
+
+	    initialize(uimaAsEngine, appCtx);
+	    waitUntilInitialized();
+	    for (int i = 0; i < 10; i++) {
+	      CAS cas = uimaAsEngine.getCAS();
+	      cas.setDocumentText("Some Text");
+	 //     System.out.println("UIMA AS Client Sending CAS#" + (i + 1) + " Request to a Service");
+	      uimaAsEngine.sendCAS(cas);
+	    }
+	    uimaAsEngine.collectionProcessingComplete();
+	    uimaAsEngine.stop();
+	    
+	  }
+
+	  @Test
+	  public void testClientEndpointPlaceholderSubstitutionOverJava() throws Exception {
+		  testClientEndpointPlaceholderSubstitution(Transport.Java);
+	  }
+	  @Test
+	  public void testClientEndpointPlaceholderSubstitutionOverJms() throws Exception {
+		  testClientEndpointPlaceholderSubstitution(Transport.JMS);
+	  }
+	  public void testClientEndpointPlaceholderSubstitution(Transport transport) throws Exception {
+		    System.out.println("-------------- testClientEndpointPlaceholderSubstitution -------------");
+		    // Instantiate Uima AS Client
+		    BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+		    // Nest the placeholders in the broker & endpoint strings
+		    String url = getMasterConnectorURI(broker);
+		    System.setProperty( "defaultBrokerURL", url.substring(2,url.length()-2));
+		    String brokerUrl = url.substring(0,2) + "${defaultBrokerURL}" + url.substring(url.length()-2);	    
+		    System.setProperty( "PersonTitleEndpoint", "TitleAnnotator");
+		    String endpoint = "Person${PersonTitleEndpoint}Queue";  // "PersonTitleAnnotatorQueue"
+		    // Deploy Uima AS Primitive Service
+			Map<String, Object> appCtx = defaultContext(endpoint);
+			deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml",endpoint);
+
+//		    deployJavaService(uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+
+
+		    //Map<String, Object> appCtx = buildContext(brokerUrl, endpoint);
+
+		    initialize(uimaAsEngine, appCtx);
+		    waitUntilInitialized();
+		    for (int i = 0; i < 10; i++) {
+		      CAS cas = uimaAsEngine.getCAS();
+		      cas.setDocumentText("Some Text");
+//		      System.out.println("UIMA AS Client Sending CAS#" + (i + 1) + " Request to a Service");
+		      uimaAsEngine.sendCAS(cas);
+		    }
+		    uimaAsEngine.collectionProcessingComplete();
+		    uimaAsEngine.stop();
+		    
+		  }
+	   /**
+	   * Tests deployment of a primitive Uima-AS Service (PersontTitleAnnotator). Deploys the primitive
+	   * in the same jvm using Uima-AS Client API and blocks on a monitor until the Uima Client calls
+	   * initializationComplete() method. Once the primitive service starts it is expected to send its
+	   * metadata to the Uima client which in turn notifies this object with a call to
+	   * initializationComplete() where the monitor is signaled to unblock the thread. This code will
+	   * block if the Uima Client does not call initializationComplete()
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testDeployPrimitiveServiceOverJava() throws Exception {
+		  testDeployPrimitiveService(Transport.Java);
+	  }
+	  @Test
+	  public void testDeployPrimitiveServiceOverJms() throws Exception {
+		  testDeployPrimitiveService(Transport.JMS);
+	  }
+	  public void testDeployPrimitiveService(Transport transport) throws Exception {
+	    System.out.println("-------------- testDeployPrimitiveService -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml","PersonTitleAnnotatorQueue");
+
+	//    deployJavaService(eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "PersonTitleAnnotatorQueue", 1000, PROCESS_LATCH);
+	    System.out.println("-------------- Terminating testDeployPrimitiveService -------------");
+	  }
+	 @Test
+	 public void testTypeSystemMergeOverJava() throws Exception {
+		 testTypeSystemMerge(Transport.Java);
+	 }
+	 @Test
+	 public void testTypeSystemMergeOverJms() throws Exception {
+		 testTypeSystemMerge(Transport.JMS);
+	 }
+	  public void testTypeSystemMerge(Transport transport) throws Exception {
+	    System.out.println("-------------- testTypeSystemMerge -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+	    deployJmsService(eeUimaEngine, relativePath+ "/Deploy_GovernmentOfficialRecognizer.xml");
+	    deployJmsService(eeUimaEngine, relativePath+ "/Deploy_NamesAndPersonTitlesRecognizer.xml");
+	    deployJmsService(eeUimaEngine, relativePath+ "/Deploy_TokenSentenceRecognizer.xml");
+	    
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateToTestTSMerge.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath+ "/Deploy_AggregateToTestTSMerge.xml");
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	    "TopLevelTaeQueue");
+	    
+	    try {
+	      initialize(eeUimaEngine, appCtx);
+	      waitUntilInitialized();
+	      //  Check if the type system returned from the service contains
+	      //  expected types
+	      CAS cas = eeUimaEngine.getCAS();
+	      TypeSystem ts = cas.getTypeSystem();
+	      //  "example.EmailsAddress" type was 'contributed' by the Flow Controller
+	      if ( ts.getType("example.EmailAddress") == null ) {
+	        fail("Incomplete Type system. Expected Type 'example.EmailAddress' missing from the CAS type system");
+	      } else if ( ts.getType("example.GovernmentOfficial") == null) {
+	        fail("Incomplete Type system. Expected Type 'example.GovernmentOfficial' missing from the CAS type system");
+	      } else if ( ts.getType("example.Name") == null) {
+	        fail("Incomplete Type system. Expected Type 'example.Name' missing from the CAS type system");
+	      } else if ( ts.getType("example.PersonTitle") == null) {
+	        fail("Incomplete Type system. Expected Type 'example.PersonTitle' missing from the CAS type system");
+	      } else if ( ts.getType("example.PersonTitleKind") == null) {
+	        fail("Incomplete Type system. Expected Type 'example.PersonTitleKind' missing from the CAS type system");
+	      } else if ( ts.getType("org.apache.uima.examples.tokenizer.Sentence") == null) {
+	        fail("Incomplete Type system. Expected Type 'org.apache.uima.examples.tokenizer.Sentence' missing from the CAS type system");
+	      } else if ( ts.getType("org.apache.uima.examples.tokenizer.Token") == null) {
+	        fail("Incomplete Type system. Expected Type 'org.apache.uima.examples.tokenizer.Token' missing from the CAS type system");
+	      } 
+	      
+	    } catch (ResourceInitializationException e) {
+	        fail("Initialization Exception");
+	    } catch (Exception e) {
+	    } finally {
+	      eeUimaEngine.stop();
+	    }
+	  }
+	 /**
+	   * Tests sending CPC request from a client that does not send CASes to a service
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testCpCWithNoCASesSentOverJava() throws Exception {
+		  testCpCWithNoCASesSent(Transport.Java);
+	  }
+	  @Test
+	  public void testCpCWithNoCASesSentOverJms() throws Exception {
+		  testCpCWithNoCASesSent(Transport.JMS);
+	  }
+	  public void testCpCWithNoCASesSent(Transport transport) throws Exception {
+	    System.out.println("-------------- testCpCWithNoCASesSent -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml","PersonTitleAnnotatorQueue");
+
+	//    deployJavaService(uimaAsEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	            "PersonTitleAnnotatorQueue");
+	    initialize(uimaAsEngine, appCtx);
+	    waitUntilInitialized();
+
+	    for (int i = 0; i < 10; i++) {
+	      System.out.println("UIMA AS Client Sending CPC Request to a Service");
+	      uimaAsEngine.collectionProcessingComplete();
+	    }
+	    uimaAsEngine.stop();
+	  }
+
+	 @Test
+	 public void testDeployAggregateServiceOverJava() throws Exception {
+		 testDeployAggregateService(Transport.Java);
+	 }
+	 @Test
+	 public void testDeployAggregateServiceOverJms() throws Exception {
+		 testDeployAggregateService(Transport.JMS);
+	 }
+	  public void testDeployAggregateService(Transport transport) throws Exception {
+	    System.out.println("-------------- testDeployAggregateService -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    
+	    
+	    //   System.setProperty("BrokerURL", "tcp::/localhost:61616");
+	       System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+//	       deployService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+			Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+			deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml","TopLevelTaeQueue");
+
+	//       deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml");
+
+	  //     Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	      		Map<String, Object> appCtx = buildContext("tcp://localhost:61616",
+	    //           "TopLevelTaeQueue");
+//	       appCtx.put(UimaAsynchronousEngine.Timeout, 1000);
+	       appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+	       appCtx.put(UimaAsynchronousEngine.GetMetaTimeout, 0);
+	       
+	       addExceptionToignore(org.apache.uima.aae.error.UimaEEServiceException.class); 
+	       
+//	       runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	       runTest(appCtx, eeUimaEngine, "tcp://localhost:61616", "TopLevelTaeQueue",
+	               1, PROCESS_LATCH);
+	  }
+	 
+	 
+/*
+	 @Test
+	  public void testDeployAggregateService() throws Exception {
+	    System.out.println("-------------- testDeployAggregateService -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    
+	    
+	 //   System.setProperty("BrokerURL", "tcp::/localhost:61616");
+
+	    
+	    System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+	  //  String serviceId = deployService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+//	    System.setProperty("NoOpService", serviceId);
+	    
+	 //   deployService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml");
+	    
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	   		Map<String, Object> appCtx = buildContext("tcp://localhost:61616",
+//	    		 "NoOpAnnotatorQueue");
+	 //           "TopLevelTaeQueue");
+	    Map<String, Object> appCtx = buildContext("tcp://localhost:61616",
+	         "MeetingDetectorTaeQueue");
+	    appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+	    appCtx.put(UimaAsynchronousEngine.GetMetaTimeout, 0);
+	    
+//	    addExceptionToignore(org.apache.uima.aae.error.UimaEEServiceException.class); 
+	    
+//	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	    
+	    
+	  //  runTest(appCtx, eeUimaEngine, "tcp://localhost:61616", "TopLevelTaeQueue",
+	  //          1, PROCESS_LATCH);
+	    runTest(appCtx, eeUimaEngine, "tcp://localhost:61616", "MeetingDetectorTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+	 */
+	  /**
+	   * Sends total of 10 CASes to async aggregate configured to process 2 CASes at a time.
+	   * The inner NoOp annotator is configured to sleep for 5 seconds. The client should
+	   * be receiving 2 ACKs simultaneously confirming that the aggregate is processing 2 
+	   * input CASes at the same time.
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testDeployAggregateServiceWithScaledInnerNoOpOverJava() throws Exception {
+		  testDeployAggregateServiceWithScaledInnerNoOp(Transport.Java);
+	  }
+	  @Test
+	  public void testDeployAggregateServiceWithScaledInnerNoOpOverJms() throws Exception {
+		  testDeployAggregateServiceWithScaledInnerNoOp(Transport.JMS);
+	  }
+	  public void testDeployAggregateServiceWithScaledInnerNoOp(Transport transport) throws Exception {
+		    System.out.println("-------------- testDeployAggregateServiceWithScaledInnerNoOp -------------");
+		    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		    System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+			Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+			deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithScaledInnerNoOp.xml","TopLevelTaeQueue");
+
+//		    deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithScaledInnerNoOp.xml");
+//		    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//		            "TopLevelTaeQueue");
+//		    appCtx.put(UimaAsynchronousEngine.Timeout, 0);
+		    appCtx.put(UimaAsynchronousEngine.CasPoolSize, 5);
+		    appCtx.put(UimaAsynchronousEngine.GetMetaTimeout, 0);
+		    
+		    addExceptionToignore(org.apache.uima.aae.error.UimaEEServiceException.class); 
+		    
+		    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+		            10, PROCESS_LATCH);
+		    System.out.println("-------------- End testDeployAggregateServiceWithScaledInnerNoOp -------------");
+
+		  }
+
+
+
+	  /**
+	   * Tests a simple Aggregate with one remote Delegate and collocated Cas Multiplier
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testDeployAggregateServiceWithTempReplyQueueOverJava() throws Exception {
+		  testDeployAggregateServiceWithTempReplyQueue(Transport.Java);
+	  }
+	  @Test
+	  public void testDeployAggregateServiceWithTempReplyQueueOverJms() throws Exception {
+		  testDeployAggregateServiceWithTempReplyQueue(Transport.JMS);
+	  }
+	  public void testDeployAggregateServiceWithTempReplyQueue(Transport transport) throws Exception {
+	    System.out.println("-------------- testDeployAggregateServiceWithTempReplyQueue -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateUsingRemoteTempQueue.xml","TopLevelTaeQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateUsingRemoteTempQueue.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+
+	  }
+
+	  /**
+	   * Tests a simple Aggregate with one remote Delegate and collocated Cas Multiplier
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testProcessAggregateServiceWith1000DocsOverJava() throws Exception {
+		  testProcessAggregateServiceWith1000Docs(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessAggregateServiceWith1000DocsOverJms() throws Exception {
+		  testProcessAggregateServiceWith1000Docs(Transport.JMS);
+	  }
+	  public void testProcessAggregateServiceWith1000Docs(Transport transport) throws Exception {
+	    System.out.println("-------------- testProcessAggregateServiceWith1000Docs -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithInternalCM1000Docs.xml","TopLevelTaeQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath
+//	            + "/Deploy_AggregateAnnotatorWithInternalCM1000Docs.xml");
+	    
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+
+	  }
+
+	  @Test
+	  public void testProcessAggregateWithInnerAggregateCMOverJava() throws Exception {
+		  testProcessAggregateWithInnerAggregateCM(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessAggregateWithInnerAggregateCMOverJms() throws Exception {
+		  testProcessAggregateWithInnerAggregateCM(Transport.JMS);
+	  }
+	  public void testProcessAggregateWithInnerAggregateCM(Transport transport) throws Exception {
+	    System.out.println("-------------- testProcessAggregateWithInnerAggregateCM() -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWithInnerAggregateCM.xml","TopLevelTaeQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWithInnerAggregateCM.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	    System.out.println("-------------- End testProcessAggregateWithInnerAggregateCM() -------------");
+
+	  }
+	  @Test
+	  public void testAggregateWithInnerSynchAggregateCMOverJava() throws Exception {
+		  testAggregateWithInnerSynchAggregateCM(Transport.Java);
+	  }
+	  @Test
+	  public void testAggregateWithInnerSynchAggregateCMOverJms() throws Exception {
+		  testAggregateWithInnerSynchAggregateCM(Transport.JMS);
+	  }
+	  
+	  public void testAggregateWithInnerSynchAggregateCM(Transport transport) throws Exception {
+		    System.out.println("-------------- testAggregateWithInnerSynchAggregateCM() -------------");
+		    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		    System.setProperty(JmsConstants.SessionTimeoutOverride, "2500000");
+			Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+			deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWithInnerUimaAggregateCM.xml","TopLevelTaeQueue");
+
+		   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWithInnerUimaAggregateCM.xml");
+		    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+		            1, PROCESS_LATCH);
+		    System.out.println("-------------- testAggregateWithInnerSynchAggregateCM() -------------");
+
+		  }
+
+	 /**
+	   * Deploys a Primitive Uima-AS service and sends 5 CASes to it.
+	   * 
+	   * @throws Exception
+	   */
+
+	  @Test
+	  public void testPrimitiveServiceProcessOverJava() throws Exception {
+		  testPrimitiveServiceProcess(Transport.Java);
+	  }
+	  @Test
+	  public void testPrimitiveServiceProcessOverJms() throws Exception {
+		  testPrimitiveServiceProcess(Transport.JMS);
+	  }
+	  public void testPrimitiveServiceProcess(Transport transport) throws Exception {
+	    System.out.println("-------------- testPrimitiveServiceProcess -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml","PersonTitleAnnotatorQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "PersonTitleAnnotatorQueue", 5, PROCESS_LATCH);
+	  }
+
+	  /**
+	   * Deploys a Primitive Uima-AS service and sends 5 CASes to it.
+	   * 
+	   * @throws Exception
+	   */
+
+	  @Test
+	  public void testSyncAggregateProcessOverJava() throws Exception {
+		  testSyncAggregateProcess(Transport.Java); 
+	  }
+	  @Test
+	  public void testSyncAggregateProcessOverJms() throws Exception {
+		  testSyncAggregateProcess(Transport.JMS);
+	  }
+	  public void testSyncAggregateProcess(Transport transport) throws Exception {
+	    System.out.println("-------------- testSyncAggregateProcess -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("MeetingDetectorQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_MeetingDetectorAggregate.xml","MeetingDetectorQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_MeetingDetectorAggregate.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "MeetingDetectorQueue", 5, PROCESS_LATCH);
+	  }
+
+
+	  /**
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testPrimitiveProcessCallWithLongDelayOverJava() throws Exception {
+		  testPrimitiveProcessCallWithLongDelay(Transport.Java);
+	  }
+	  @Test
+	  public void testPrimitiveProcessCallWithLongDelayOverJms() throws Exception {
+		  testPrimitiveProcessCallWithLongDelay(Transport.JMS);
+	  }
+	  public void testPrimitiveProcessCallWithLongDelay(Transport transport) throws Exception {
+	    System.out.println("-------------- testPrimitiveProcessCallWithLongDelay -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Service
+		Map<String, Object> appCtx = defaultContext("NoOpAnnotatorQueueLongDelay");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorWithLongDelay.xml","NoOpAnnotatorQueueLongDelay");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorWithLongDelay.xml");
+	    // We expect 18000ms to be spent in process method
+	    super.setExpectedProcessTime(6000);
+
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	            "NoOpAnnotatorQueueLongDelay");
+	    appCtx.remove(UimaAsynchronousEngine.ReplyWindow);
+	    appCtx.put(UimaAsynchronousEngine.ReplyWindow, 1);
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "NoOpAnnotatorQueueLongDelay", 4, PROCESS_LATCH, true);
+	  }
+
+	  /**
+	   * Tests time spent in process CAS. The CAS is sent to three remote delegates each with a delay of
+	   * 6000ms in the process method. The aggregate is expected to sum up the time spent in each
+	   * annotator process method. The final sum is returned to the client (the test) and compared
+	   * against expected 18000ms. The test actually allows for 20ms margin to account for any overhead
+	   * (garbage collecting, slow cpu, etc)
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testAggregateProcessCallWithLongDelayOverJava() throws Exception {
+		  testAggregateProcessCallWithLongDelay(Transport.Java);
+	  }
+	  @Test
+	  public void testAggregateProcessCallWithLongDelayOverJms() throws Exception {
+		  testAggregateProcessCallWithLongDelay(Transport.JMS);
+	  }
+	  public void testAggregateProcessCallWithLongDelay(Transport transport) throws Exception {
+
+	    System.out.println("-------------- testAggregateProcessCallWithLongDelay -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Services each with 6000ms delay in process()
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorAWithLongDelay.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorBWithLongDelay.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorCWithLongDelay.xml");
+	    
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithLongDelay.xml","TopLevelTaeQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithLongDelay.xml");
+	    // We expect 18000ms to be spent in process method
+	    super.setExpectedProcessTime(18000);
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	            "TopLevelTaeQueue");
+	    appCtx.remove(UimaAsynchronousEngine.ReplyWindow);
+	    // make sure we only send 1 CAS at a time
+	    appCtx.put(UimaAsynchronousEngine.ReplyWindow, 1);
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	            "TopLevelTaeQueue", 1, PROCESS_LATCH, true);
+	    System.out.println("///////////////");
+	  }
+
+	  /**
+	   * Tests Aggregate configuration where the Cas Multiplier delegate is the last delegate in the
+	   * Aggregate's pipeline
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testAggregateProcessCallWithLastCMOverJava() throws Exception {
+		  testAggregateProcessCallWithLastCM(Transport.Java);
+	  }
+	  @Test
+	  public void testAggregateProcessCallWithLastCMOverJms() throws Exception {
+		  testAggregateProcessCallWithLastCM(Transport.JMS);
+	  }
+	  public void testAggregateProcessCallWithLastCM(Transport transport) throws Exception {
+	    System.out.println("-------------- testAggregateProcessCallWithLastCM -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    // Deploy Uima-AS Primitive Services each with 6000ms delay in process()
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithLastCM.xml","TopLevelTaeQueue");
+
+	    //deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWithLastCM.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH, true);
+	  }
+
+
+	/**
+	   * Tests a parallel flow in the Uima-AS aggregate.
+	   * 
+	   * @throws Exception
+	   */
+	  @Test
+	  public void testProcessWithParallelFlowOverJava() throws Exception {
+		  testProcessWithParallelFlow(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessWithParallelFlowOverJms() throws Exception {
+		  testProcessWithParallelFlow(Transport.JMS);
+	  }
+	  public void testProcessWithParallelFlow(Transport transport) throws Exception {
+	    System.out.println("-------------- testProcessWithParallelFlow -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator2.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml","TopLevelTaeQueue");
+
+	    //deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+
+	  @Test
+	  public void testProcessWithAggregateUsingCollocatedMultiplierOverJava() throws Exception {
+		  testProcessWithAggregateUsingCollocatedMultiplier(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessWithAggregateUsingCollocatedMultiplierOverJms() throws Exception {
+		  testProcessWithAggregateUsingCollocatedMultiplier(Transport.JMS);
+	  }
+	  public void testProcessWithAggregateUsingCollocatedMultiplier(Transport transport) throws Exception {
+	    System.out
+	            .println("-------------- testProcessWithAggregateUsingCollocatedMultiplier -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotator.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+	  @Test
+	  public void testParentProcessLastOverJava() throws Exception {
+		  testParentProcessLast(Transport.Java);
+	  }
+	  @Test
+	  public void testParentProcessLastOverJms() throws Exception {
+		  testParentProcessLast(Transport.JMS);
+	  }
+	  public void testParentProcessLast(Transport transport) throws Exception {
+	    System.out
+	            .println("-------------- testParentProcessLast -------------");
+	    System.setProperty("BrokerURL", getMasterConnectorURI(broker));
+
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_RemoteCasMultiplierWith10Docs_1.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithProcessParentLastCMs.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateAnnotatorWithProcessParentLastCMs.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+	  /**
+	   * Starts two remote delegates on one broker and a top level client aggregate on 
+	   * another. Tests sending Free Cas requests to the appropriate broker. 
+	   * 
+	   * @throws Exception
+	   */
+
+	  
+	  /**
+	   * First CM feeds 100 CASes to a "merger" CM that generates one output CAS for every 5 input.
+	   * Second CM creates unique document text that is checked by the last component. The default FC
+	   * should let 4 childless CASes through, replacing every 5th by its child.
+	   * 
+	   * @throws Exception
+	   */
+
+	  
+
+	 
+	  @Test
+	  public void testProcessWithAggregateUsing2CollocatedMultipliersOverJava() throws Exception {
+		  testProcessWithAggregateUsing2CollocatedMultipliers(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessWithAggregateUsing2CollocatedMultipliersOverJms() throws Exception {
+		  testProcessWithAggregateUsing2CollocatedMultipliers(Transport.JMS);
+	  }
+	  
+	  public void testProcessWithAggregateUsing2CollocatedMultipliers(Transport transport) throws Exception {
+	    System.out
+	            .println("-------------- testProcessWithAggregateUsing2CollocatedMultipliers -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWith2Multipliers.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWith2Multipliers.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+	  @Test
+	  public void testProcessAggregateWithInnerCMAggregateOverJava() throws Exception {
+		  testProcessAggregateWithInnerCMAggregate(Transport.Java);
+	  }
+	  @Test
+	  public void testProcessAggregateWithInnerCMAggregateOverJms() throws Exception {
+		  testProcessAggregateWithInnerCMAggregate(Transport.JMS);
+	  }
+	  public void testProcessAggregateWithInnerCMAggregate(Transport transport) throws Exception {
+	    System.out.println("-------------- testProcessAggregateWithInnerCMAggregate -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_TopAggregateWithInnerAggregateCM.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_TopAggregateWithInnerAggregateCM.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+
+
+	  @Test
+	  public void testTypesystemMergeWithMultiplierOverJava() throws Exception {
+		  testTypesystemMergeWithMultiplier(Transport.Java);
+	  }
+	  @Test
+	  public void testTypesystemMergeWithMultiplierOverJms() throws Exception {
+		  testTypesystemMergeWithMultiplier(Transport.JMS);
+	  }
+	  public void testTypesystemMergeWithMultiplier(Transport transport) throws Exception {
+	    System.out.println("-------------- testTypesystemMergeWithMultiplier -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithMergedTypes.xml","TopLevelTaeQueue");
+
+	    //deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWithMergedTypes.xml");
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  } 
+
+	  @Test
+	  public void testCancelProcessAggregateWithCollocatedMultiplierOverJava() throws Exception {
+		  testCancelProcessAggregateWithCollocatedMultiplier(Transport.Java);
+	  }
+	  @Test
+	  public void testCancelProcessAggregateWithCollocatedMultiplierOverJms() throws Exception {
+		  testCancelProcessAggregateWithCollocatedMultiplier(Transport.JMS); 
+	  }
+	  public void testCancelProcessAggregateWithCollocatedMultiplier(Transport transport) throws Exception {
+	    System.out
+	            .println("-------------- testCancelProcessAggregateWithCollocatedMultiplier -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+		Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWith1MillionDocs.xml","TopLevelTaeQueue");
+
+	   // deployJavaService(eeUimaEngine, relativePath + "/Deploy_ComplexAggregateWith1MillionDocs.xml");
+	    // Spin a thread to cancel Process after 20 seconds
+	    spinShutdownThread(eeUimaEngine, 20000);
+	    runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	            1, PROCESS_LATCH);
+	  }
+
+
+
+	 /**
+	   * Tests exception thrown in the Uima-AS Client when the Collection Reader is added after the uima
+	   * ee client is initialized
+	   * 
+	   * @throws Exception
+	   */
+
+	  @Test
+	  public void testCollectionReaderOverJava() throws Exception {
+		  testCollectionReader(Transport.Java);
+	  }
+	  @Test
+	  public void testCollectionReaderOverJms() throws Exception {
+		  testCollectionReader(Transport.JMS);
+	  }
+	  public void testCollectionReader(Transport transport) throws Exception {
+	    System.out.println("-------------- testCollectionReader -------------");
+	    // Instantiate Uima-AS Client
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+		Map<String, Object> appCtx = defaultContext("PersonTitleAnnotatorQueue");
+		deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml","PersonTitleAnnotatorQueue");
+
+//	    deployJavaService(eeUimaEngine, relativePath + "/Deploy_PersonTitleAnnotator.xml");
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	            "PersonTitleAnnotatorQueue");
+	    // reduce the cas pool size and reply window
+	    appCtx.remove(UimaAsynchronousEngine.CasPoolSize);
+	    appCtx.put(UimaAsynchronousEngine.CasPoolSize, Integer.valueOf(2));
+	    appCtx.remove(UimaAsynchronousEngine.ReplyWindow);
+	    appCtx.put(UimaAsynchronousEngine.ReplyWindow, 1);
+
+	    // set the collection reader
+	    String filename = super
+	            .getFilepathFromClassloader("descriptors/collection_reader/ExtendedTestFileSystemCollectionReader.xml");
+	    if (filename == null) {
+	      fail("Unable to find file:" + "descriptors/collection_reader/ExtendedTestFileSystemCollectionReader.xml"
+	              + "in classloader");
+	    }
+	    File collectionReaderDescriptor = new File(filename);
+	    CollectionReaderDescription collectionReaderDescription = UIMAFramework.getXMLParser()
+	            .parseCollectionReaderDescription(new XMLInputSource(collectionReaderDescriptor));
+	    CollectionReader collectionReader = UIMAFramework
+	            .produceCollectionReader(collectionReaderDescription);
+	    eeUimaEngine.setCollectionReader(collectionReader);
+	    initialize(eeUimaEngine, appCtx);
+	    waitUntilInitialized();
+	    runCrTest(eeUimaEngine, 7);
+	    synchronized (this) {
+	      wait(50);
+	    }
+	    eeUimaEngine.stop();
+	  }
+
+
+
+	 @Test
+	 public void testDeployAgainAndAgainOverJava() throws Exception {
+		 testDeployAgainAndAgain(Transport.Java);
+	 }
+	 @Test
+	 public void testDeployAgainAndAgainOverJms() throws Exception {
+		 testDeployAgainAndAgain(Transport.JMS);
+	 }
+	  public void testDeployAgainAndAgain(Transport transport) throws Exception {
+	    System.out.println("-------------- testDeployAgainAndAgain -------------");
+	    for (int num = 1; num <= 50; num++) {
+	      BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl(); 
+	      System.out.println("\nRunning iteration " + num);
+	      deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator.xml");
+	      deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator2.xml");
+		  Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		  deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml","TopLevelTaeQueue");
+
+	      //deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml");
+	      runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)),
+	              "TopLevelTaeQueue", 1, PROCESS_LATCH);
+	    }
+	  }
+	  @Test
+	  public void testMultipleASClients() throws Exception  {
+		    System.out.println("-------------- testMultipleSyncClientsWithMultipleBrokers -------------");
+		    
+		    class RunnableClient implements Runnable {
+		    	String brokerURL;
+		    	BaseTestSupport testSupport;
+	          BaseUIMAAsynchronousEngine_impl uimaAsEngine;
+	          String serviceEndpoint;
+	          
+	          
+	          RunnableClient(BaseTestSupport testSupport, String brokerURL,String serviceEndpoint) {
+		    		this.brokerURL = brokerURL;
+		    		this.testSupport = testSupport;
+		    		this.serviceEndpoint = serviceEndpoint;
+		    		uimaAsEngine = new BaseUIMAAsynchronousEngine_impl();
+		    	}
+		    	public BaseUIMAAsynchronousEngine_impl getUimaAsClient() {
+		    		return uimaAsEngine;
+		    	}
+		    	public void initialize() throws Exception {
+		    		@SuppressWarnings("unchecked")
+				  Map<String, Object> appCtx = buildContext(brokerURL, serviceEndpoint);
+		    		appCtx.put(UimaAsynchronousEngine.Timeout, 1100);
+			  	  appCtx.put(UimaAsynchronousEngine.CpcTimeout, 1100);
+			  	  testSupport.initialize(getUimaAsClient(), appCtx);
+			  	  waitUntilInitialized();
+			  	  
+		    	}
+				public void run() {
+					try {
+						initialize();
+						System.out.println("Thread:"+Thread.currentThread().getId()+" Completed GetMeta() broker:"+brokerURL);
+					} catch( Exception e) {
+						e.printStackTrace();
+					} finally {
+						try {
+					        uimaAsEngine.stop();
+						} catch( Exception e) {
+							e.printStackTrace();
+						}
+					}
+
+				}
+		    	
+		    }
+		    
+		    ExecutorService executor = Executors.newCachedThreadPool();
+	      String serviceId1;
+	      String serviceId2;
+
+		    //	change broker URl in system properties
+		    System.setProperty("BrokerURL", getMasterConnectorURI(broker).toString());
+		    
+		    RunnableClient client1 = 
+		    		new RunnableClient(this, getMasterConnectorURI(broker), "NoOpAnnotatorQueue");
+		    BaseUIMAAsynchronousEngine_impl engine = client1.getUimaAsClient();
+		    serviceId1 = deployJmsService(engine, relativePath + "/Deploy_NoOpAnnotatorWithPlaceholder.xml");
+
+		    final BrokerService broker2 = setupSecondaryBroker(true);
+		    //	change broker URl in system properties
+		    System.setProperty("BrokerURL", broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString());
+		    
+		    RunnableClient client2 = 
+		    		new RunnableClient(this, "failover:tcp://f5n633:51514,tcp://f12n1133:51514","NoOpAnnotatorQueue");//broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString(), "NoOpAnnotatorQueue");
+		    		//new RunnableClient(this, "failover:ssl://f5n6:51514,ssl://f12n11:51514","NoOpAnnotatorQueue");//broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString(), "NoOpAnnotatorQueue");
+		    BaseUIMAAsynchronousEngine_impl engine2 = client2.getUimaAsClient();
+//		    serviceId2 = deployService(engine2, relativePath + "/Deploy_NoOpAnnotatorWithPlaceholder.xml");
+
+		    
+		    for( int x = 0; x < 100; x++) {
+			    List<Future<?>> list1 = new ArrayList<Future<?>>();
+			    List<Future<?>> list2 = new ArrayList<Future<?>>();
+			    String b;
+			  /*
+			    if ( x % 2 == 0 ) {
+			    	b = getMasterConnectorURI(broker);
+			    } else {
+			    	b = "failover:ssl://f5n6:51514,ssl://f12n11:51514";//broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString();
+			    }
+			    */
+			    List<Future<?>> list = new ArrayList<Future<?>>();
+			    for (int i = 0; i < 20; i++) {
+			    	  if ( i % 2 == 0 ) {
+					    	b = getMasterConnectorURI(broker);
+					    	list = list1;
+					    } else {
+					    	b = "failover:tcp://f5n633:51514,tcp://f12n1133:51514?maxReconnectAttempts=2&timeout=300&transport.maxReconnectAttempts=2&transport.timeout=300&startupMaxReconnectAttempts=1";//broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString();
+//					    	b = "failover:ssl://f5n6:51514,ssl://f12n11:51514?maxReconnectAttempts=2&timeout=300&transport.maxReconnectAttempts=2&transport.timeout=300&startupMaxReconnectAttempts=1";//broker2.getConnectorByName(DEFAULT_BROKER_URL_KEY_2).getUri().toString();
+					    	list = list2;
+					    }
+				    RunnableClient client = 
+				    		new RunnableClient(this, b, "NoOpAnnotatorQueue");
+				    list.add(executor.submit(client));;
+			    }
+			    /*
+			    
+				for (int i = 0; i < 10; i++) {
+			    	try {
+				    	list.get(i).get();//1, TimeUnit.SECONDS);
+			    	} catch( Exception e) {
+			    		e.printStackTrace();
+				    	list.get(i).cancel(true);
+			    	}
+			    }
+			    */
+
+		    Worker worker1 = new Worker(list1);
+		    Worker worker2 = new Worker(list1);
+		    Thread t1 = new Thread(worker1);
+		    Thread t2 = new Thread(worker2);
+		    t1.start();
+		    t2.start();
+		    
+		    t1.join();
+		    t2.join();
+		    
+		    list.clear();
+		    
+		    }
+		//    engine2.undeploy(serviceId2);
+		    engine.undeploy(serviceId1);
+		    
+		    //engine2.stop();
+		    executor.shutdownNow();
+		    while( !executor.isShutdown() ) {
+		    	synchronized(broker) {
+		    		broker.wait(100);
+		    	}
+		    }
+		    broker2.stop();
+		    broker2.waitUntilStopped();
+		    //broker.stop();
+		    //broker.waitUntilStopped();
+		    //System.out.println("Done");
+		}
+	  @Test
+	  public void testAsynchronousTerminateOverJava() throws Exception {
+		  testAsynchronousTerminate(Transport.Java);
+	  }
+	  @Test
+	  public void testAsynchronousTerminateOverJms() throws Exception {
+		  testAsynchronousTerminate(Transport.JMS);
+	  }
+	  public void testAsynchronousTerminate(Transport transport) throws Exception {
+	    System.out.println("-------------- testAsynchronousTerminate -------------");
+	    BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+//	    Map<String, Object> appCtx = buildContext(String.valueOf(getMasterConnectorURI(broker)),
+//	    "TopLevelTaeQueue");
+	    try {
+	      deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotatorWithDelay.xml");
+	      deployJmsService(eeUimaEngine, relativePath + "/Deploy_NoOpAnnotator2.xml");
+		  Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+		  deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml","TopLevelTaeQueue");
+
+	     // deployJavaService(eeUimaEngine, relativePath + "/Deploy_AggregateWithParallelFlow.xml");
+	      initialize(eeUimaEngine, appCtx);
+	      // Wait until the top level service returns its metadata
+	      waitUntilInitialized();
+	    } catch( Exception e) {
+				throw e;
+	    }
+
+
+	    CAS cas = eeUimaEngine.getCAS();
+	    System.out.println(" Sending CAS to kick off aggregate w/colocated CasMultiplier");
+	    eeUimaEngine.sendCAS(cas);
+
+	    System.out.println(" Waiting 1 seconds");
+	    Thread.sleep(1000);
+
+	    System.out.println(" Trying to stop service");
+	    eeUimaEngine.stop();
+	    System.out.println(" stop() returned!");
+	    Object o = new Object();
+	    
+	  }
+	  
+	 
+
+	  /**
+	   * Test use of a JMS Service Adapter. Invoke from a synchronous aggregate to emulate usage from
+	   * RunAE or RunCPE.
+	   * 
+	   * @throws Exception
+	   */
+	    @Test
+	    public void testJmsServiceAdapterOverJava() throws Exception {
+	    	testJmsServiceAdapter(Transport.Java);
+	    }
+	    @Test
+	    public void testJmsServiceAdapterOverJms() throws Exception {
+	    	testJmsServiceAdapter(Transport.JMS);
+	    }
+	    public void testJmsServiceAdapter(Transport transport) throws Exception {
+		  Logger.getLogger(this.getClass()).info("-------------- testJmsServiceAdapter -------------");
+		  //setUp();
+		  BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
+	    try {
+
+	        deployJmsService(eeUimaEngine, relativePath.concat("/Deploy_NoOpAnnotator.xml"));
+			Map<String, Object> appCtx = defaultContext("TopLevelTaeQueue");
+			deployTopLevelService(appCtx, transport, eeUimaEngine, relativePath + "/Deploy_SyncAggregateWithJmsService.xml","TopLevelTaeQueue");
+
+	        //deployJavaService(eeUimaEngine, relativePath + "/Deploy_SyncAggregateWithJmsService.xml");
+	        runTest(appCtx, eeUimaEngine, String.valueOf(getMasterConnectorURI(broker)), "TopLevelTaeQueue",
+	                1, PROCESS_LATCH);
+	       
+	    } catch( Exception e ) {
+	    	throw e;
+	    }
+	  }
+
+
+
+	  private static class TestListener extends UimaAsBaseCallbackListener implements Runnable {
+	    private String casReferenceId = null;
+
+	    private Object monitor = new Object();
+
+	    public TestListener(TestUimaASExtended aTester) {
+	    }
+
+	    public void collectionProcessComplete(EntityProcessStatus arg0) {
+	      // TODO Auto-generated method stub
+
+	    }
+
+	    public void onBeforeMessageSend(UimaASProcessStatus status) {
+	      System.out.println("TestListener Received onBeforeMessageSend Notification with Cas:"
+	              + status.getCasReferenceId());
+	    }
+
+	    public void entityProcessComplete(CAS aCAS, EntityProcessStatus aProcessStatus) {
+	      if (aProcessStatus.isException()) {
+	        if (aProcessStatus instanceof UimaASProcessStatus) {
+	          casReferenceId = ((UimaASProcessStatus) aProcessStatus).getCasReferenceId();
+	          if (casReferenceId != null) {
+	            synchronized (monitor) {
+	              monitor.notifyAll();
+	            }
+	          }
+	        }
+	      }
+	    }
+
+	    public void initializationComplete(EntityProcessStatus arg0) {
+	      // TODO Auto-generated method stub
+
+	    }
+
+	    public String getCasReferenceId() {
+	      synchronized (monitor) {
+	        while (casReferenceId == null) {
+	          try {
+	            monitor.wait();
+	          } catch (InterruptedException e) {
+	          }
+	        }
+	      }
+	      return casReferenceId;
+	    }
+
+	    public void doStop() {
+	    }
+
+	    public void run() {
+	      System.out.println("Stopping TestListener Callback Listener Thread");
+	    }
+	  }
+	  
+	  private class Worker implements Runnable {
+
+		  
+		List<Future<?>> list = new ArrayList<Future<?>>();
+		
+		public Worker(List<Future<?>> list ) {
+			this.list = list;
+		}
+		@Override
+		public void run() {
+			for (int i = 0; i < list.size(); i++) {
+		    	try {
+			    	list.get(i).get();//1, TimeUnit.SECONDS);
+		    	} catch( Exception e) {
+		    		e.printStackTrace();
+			    	list.get(i).cancel(true);
+		    	}
+		    }
+
+		}
+		  
+	  }
+	
+}
diff --git a/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/utils/BaseTestSupport.java b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/utils/BaseTestSupport.java
index b5b2726..a2ce0a7 100644
--- a/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/utils/BaseTestSupport.java
+++ b/uimaj-as-activemq/src/test/java/org/apache/uima/ee/test/utils/BaseTestSupport.java
@@ -38,6 +38,7 @@ import org.apache.uima.aae.client.UimaASProcessStatus;
 import org.apache.uima.aae.client.UimaASProcessStatusImpl;
 import org.apache.uima.aae.client.UimaAsBaseCallbackListener;
 import org.apache.uima.aae.client.UimaAsynchronousEngine;
+import org.apache.uima.aae.client.UimaAsynchronousEngine.Transport;
 import org.apache.uima.aae.error.ServiceShutdownException;
 import org.apache.uima.aae.error.UimaASPingTimeout;
 import org.apache.uima.aae.error.UimaASProcessCasTimeout;
@@ -114,9 +115,62 @@ public abstract class BaseTestSupport extends ActiveMQSupport
 
   protected long failedCasCountDueToBrokerFailure = 0;
   
+  protected String deployService(Transport transport, BaseUIMAAsynchronousEngine_impl eeUimaEngine,
+          String aDeploymentDescriptorPath) throws Exception  {
+	  String serviceId = null;
+	  if ( transport.equals(Transport.Java)) {
+		  serviceId = deployJavaService(eeUimaEngine, aDeploymentDescriptorPath);
+	  } else if ( transport.equals(Transport.JMS)) {
+		  serviceId = deployJmsService(eeUimaEngine, aDeploymentDescriptorPath);
+	  }
+	  return serviceId;
+  }
+  protected String deployJavaService(BaseUIMAAsynchronousEngine_impl eeUimaEngine,
+          String aDeploymentDescriptorPath) throws Exception  {
+    System.setProperty("Provider", "java");
+    System.setProperty("Protocol", "java");
+    System.setProperty("defaultBrokerURL", "java");
+    System.setProperty("DefaultBrokerURL", "java");
+
+    Map<String, Object> appCtx = new HashMap<>();
+    appCtx.put( AsynchAEMessage.Transport, Transport.Java);
+ 
+    return deployService(eeUimaEngine, aDeploymentDescriptorPath, appCtx);
+  }
+  protected String deployJmsService(BaseUIMAAsynchronousEngine_impl eeUimaEngine,
+          String aDeploymentDescriptorPath)  throws Exception  {
+     System.setProperty("Provider", "activemq");
+     System.setProperty("Protocol", "jms");
+     System.setProperty("defaultBrokerURL", "tcp://localhost:61617");
+     String defaultBrokerURL = System.getProperty("defaultBrokerURL");
+     if (defaultBrokerURL != null) {
+       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINEST)) {
+           String msg = ">>> runTest: Setting defaultBrokerURL to:" + defaultBrokerURL;
+           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST, CLASS_NAME.getName(), "deployService",
+                  JmsConstants.JMS_LOG_RESOURCE_BUNDLE, "UIMAJMS_debug_msg__FINEST",
+                   new Object[] { msg });
+       }
+       System.setProperty("defaultBrokerURL", defaultBrokerURL);
+     } else {
+       System.setProperty("defaultBrokerURL", "tcp://localhost:8118");
+     }
+
+     Map<String, Object> appCtx = new HashMap<>();
+	 appCtx.put( AsynchAEMessage.Transport, Transport.JMS);
+	 
+	 return deployService(eeUimaEngine, aDeploymentDescriptorPath, appCtx);
+  }
   protected String deployService(BaseUIMAAsynchronousEngine_impl eeUimaEngine,
           String aDeploymentDescriptorPath) throws Exception {
-    String defaultBrokerURL = System.getProperty("BrokerURL");
+	  return deployService(eeUimaEngine, aDeploymentDescriptorPath, new HashMap<String, Object>());
+  }
+  protected String deployService(BaseUIMAAsynchronousEngine_impl eeUimaEngine,
+          String aDeploymentDescriptorPath,  Map<String, Object> appCtx) throws Exception {
+
+ // protected String deployService(BaseUIMAAsynchronousEngine_impl eeUimaEngine,
+ //         String aDeploymentDescriptorPath) throws Exception {
+//    String defaultBrokerURL = System.getProperty("BrokerURL");
+    String defaultBrokerURL = System.getProperty("defaultBrokerURL");
     if (defaultBrokerURL != null) {
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINEST)) {
           String msg = ">>> runTest: Setting defaultBrokerURL to:" + defaultBrokerURL;
@@ -129,15 +183,15 @@ public abstract class BaseTestSupport extends ActiveMQSupport
       System.setProperty("defaultBrokerURL", "tcp://localhost:8118");
     }
 
-    Map<String, Object> appCtx = new HashMap();
-    appCtx.put(UimaAsynchronousEngine.DD2SpringXsltFilePath,
-            "../src/main/scripts/dd2spring.xsl".replace('/', FS));
-    appCtx.put(UimaAsynchronousEngine.SaxonClasspath,
-            "file:../src/main/saxon/saxon8.jar".replace('/', FS));
+    //Map<String, Object> appCtx = new HashMap();
+//    appCtx.put(UimaAsynchronousEngine.DD2SpringXsltFilePath,
+//            "../src/main/scripts/dd2spring.xsl".replace('/', FS));
+//    appCtx.put(UimaAsynchronousEngine.SaxonClasspath,
+//            "file:../src/main/saxon/saxon8.jar".replace('/', FS));
     // appCtx.put(UimaAsynchronousEngine.UimaEeDebug, UimaAsynchronousEngine.UimaEeDebug);
-    String containerId = null;
+    String serviceId = null;
     try {
-      containerId = eeUimaEngine.deploy(aDeploymentDescriptorPath, appCtx);
+    	serviceId = eeUimaEngine.deploy(aDeploymentDescriptorPath, appCtx);
     } catch (ResourceInitializationException e) {
       if (!ignoreException(ResourceInitializationException.class)) {
         System.out
@@ -151,7 +205,7 @@ public abstract class BaseTestSupport extends ActiveMQSupport
       System.out.println(">>>>>>>>>>> runTest: Exception:" + e.getClass().getName());
       throw e;
     }
-    return containerId;
+    return serviceId;
   }
 
   protected void addExceptionToignore(Class anExceptionToIgnore) {
@@ -196,19 +250,21 @@ public abstract class BaseTestSupport extends ActiveMQSupport
     return (url == null ? null : url.getPath());
   }
 
-  protected Map<String, Object> buildContext(String aTopLevelServiceBrokerURI, String aTopLevelServiceQueueName)
+  protected Map<String, Object> buildContext(String aTopLevelServiceServiceURI, String aTopLevelServiceQueueName)
           throws Exception {
-    return buildContext(aTopLevelServiceBrokerURI, aTopLevelServiceQueueName, 0);
+    return buildContext(aTopLevelServiceServiceURI, aTopLevelServiceQueueName, 0);
   }
 
-  protected Map<String, Object> buildContext(String aTopLevelServiceBrokerURI, String aTopLevelServiceQueueName,
+  protected Map<String, Object> buildContext(String aTopLevelServiceServiceURI, String aTopLevelServiceQueueName,
           int timeout) throws Exception {
     Map<String, Object> appCtx = new HashMap<String, Object>();
-    appCtx.put(UimaAsynchronousEngine.ServerUri, aTopLevelServiceBrokerURI);
+    appCtx.put(UimaAsynchronousEngine.ServerUri, aTopLevelServiceServiceURI);
     appCtx.put(UimaAsynchronousEngine.ENDPOINT, aTopLevelServiceQueueName);
     appCtx.put(UimaAsynchronousEngine.CasPoolSize, Integer.valueOf(4));
     appCtx.put(UimaAsynchronousEngine.ReplyWindow, 15);
     appCtx.put(UimaAsynchronousEngine.Timeout, timeout);
+    appCtx.put(UimaAsynchronousEngine.GetMetaTimeout,0);
+
     return appCtx;
   }
 
@@ -316,7 +372,7 @@ public abstract class BaseTestSupport extends ActiveMQSupport
     isStopping = false;
     final BaseUIMAAsynchronousEngine_impl eeUimaEngine = new BaseUIMAAsynchronousEngine_impl();
     // Deploy Uima EE Primitive Service
-    final String containerId = deployService(eeUimaEngine, serviceDeplyDescriptor);
+    final String containerId = deployJavaService(eeUimaEngine, serviceDeplyDescriptor);
 
     engine = eeUimaEngine;
 
@@ -577,6 +633,9 @@ public abstract class BaseTestSupport extends ActiveMQSupport
         // Wait until the CPC Thread is ready.
         waitOnMonitor(ctrlSemaphore);
         if (!isStopped) {
+           	StringBuilder sb = new StringBuilder().append("=================> Client sending ").
+        			append(i+1).append(" CAS of ").append(howMany);
+        	System.out.println(sb.toString());
           // Send an in CAS to the top level service
           sendCAS(aUimaEeEngine, 1, sendCasAsynchronously);
         }
@@ -633,6 +692,7 @@ public abstract class BaseTestSupport extends ActiveMQSupport
           boolean sendCasAsynchronously) throws Exception {
     engine = eeUimaEngine;
     for (int i = 0; i < howMany; i++) {
+ 
     	if (isStopping) {
     		break;
     	}
@@ -685,8 +745,8 @@ public abstract class BaseTestSupport extends ActiveMQSupport
     }
     public synchronized void onBeforeMessageSend(UimaASProcessStatus status) {
       casSent = status.getCasReferenceId();
-//      System.out.println("runTest: Received onBeforeMessageSend() Notification With CAS:"
-//              + status.getCasReferenceId());
+      System.out.println("runTest: Received onBeforeMessageSend() Notification With CAS:"
+              + status.getCasReferenceId());
     }
     public void onUimaAsServiceExit(EventTrigger cause) {
         System.out.println("runTest: Received onUimaAsServiceExit() Notification With Cause:"
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/InProcessCache.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/InProcessCache.java
index ca70a14..948fe9c 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/InProcessCache.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/InProcessCache.java
@@ -70,8 +70,15 @@ public class InProcessCache implements InProcessCacheMBean {
 
   private BaseAnalysisEngineController controller;
   
+  private boolean registeredWithJMX = false;
   
+  public boolean isRegisteredWithJMX() {
+	  return registeredWithJMX;
+  }
   
+  public void setRegisteredWithJMX() {
+	  registeredWithJMX = true;
+  }
   /**
 	  Register controller to call when the cache becomes empty.
     This call is made when the controller enters quiesce
@@ -225,41 +232,51 @@ public class InProcessCache implements InProcessCacheMBean {
   }
 
   public synchronized void dumpContents(String aControllerName) {
-//    int count = 0;
-    /*
-     * if ( UIMAFramework.getLogger().isLoggable(Level.FINEST) ) { Iterator it =
-     * cache.keySet().iterator(); StringBuffer sb = new StringBuffer("\n");
-     * 
-     * while( it.hasNext() ) { String key = (String) it.next(); CacheEntry entry =
-     * (CacheEntry)cache.get(key); count++; if ( entry.isSubordinate()) { sb.append(key+
-     * " Number Of Child CASes In Play:"
-     * +entry.getSubordinateCasInPlayCount()+" Parent CAS id:"+entry.getInputCasReferenceId()); }
-     * else { sb.append(key+
-     * " *** Input CAS. Number Of Child CASes In Play:"+entry.getSubordinateCasInPlayCount()); }
-     * 
-     * // if ( entry.isWaitingForRelease() ) // { //
-     * sb.append(" <<< Reached Final State in Controller:"+aControllerName); // }
-     * 
-     * sb.append("\n"); } UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST,
-     * CLASS_NAME.getName(), "dumpContents", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
-     * "UIMAEE_show_cache_entry_key__FINEST", new Object[] { aControllerName, count, sb.toString()
-     * });
-     * 
-     * sb.setLength(0); } else if ( UIMAFramework.getLogger().isLoggable(Level.FINE) ) { Iterator it
-     * = cache.keySet().iterator(); StringBuffer sb = new StringBuffer("\n"); int inFinalState=0;
-     * 
-     * while( it.hasNext() ) { String key = (String) it.next(); CacheEntry entry =
-     * (CacheEntry)cache.get(key); count++;
-     * 
-     * //if ( entry.isWaitingForRelease() ) //{ //inFinalState++; //}
-     * 
-     * } UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINE, CLASS_NAME.getName(), "dumpContents",
-     * UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_show_abbrev_cache_stats___FINE", new
-     * Object[] { aControllerName, count, inFinalState });
-     * 
-     * 
-     * }
-     */
+    int count = 0;
+    
+    //  if ( UIMAFramework.getLogger().isLoggable(Level.FINEST) ) { 
+    	  Iterator it =
+             cache.keySet().iterator(); 
+    	  StringBuilder sb = new StringBuilder("\n");
+      
+          while( it.hasNext() ) { 
+        	  String key = (String) it.next(); 
+        	  CacheEntry entry =
+                  (CacheEntry)cache.get(key); 
+        	  count++; 
+        	  sb.append("CAS ").append(entry.getCasReferenceId()).append(" Parent:").append(entry.getInputCasReferenceId());
+        	  /*
+        	  if ( entry.isSubordinate()) { 
+        		  sb.append(key+ " Number Of Child CASes In Play:"
+                     +entry.getSubordinateCasInPlayCount()+" Parent CAS id:"+entry.getInputCasReferenceId()); 
+        	  } else { sb.append(key+
+      " *** Input CAS. Number Of Child CASes In Play:"+entry.getSubordinateCasInPlayCount()); 
+        	  }
+      
+      // if ( entry.isWaitingForRelease() ) // { //
+      sb.append(" <<< Reached Final State in Controller:"+aControllerName); // }
+      
+      sb.append("\n"); } UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINEST,
+      CLASS_NAME.getName(), "dumpContents", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
+      "UIMAEE_show_cache_entry_key__FINEST", new Object[] { aControllerName, count, sb.toString()
+      });
+      
+      sb.setLength(0); } else if ( UIMAFramework.getLogger().isLoggable(Level.FINE) ) { Iterator it
+      = cache.keySet().iterator(); StringBuffer sb = new StringBuffer("\n"); int inFinalState=0;
+      
+      while( it.hasNext() ) { String key = (String) it.next(); CacheEntry entry =
+      (CacheEntry)cache.get(key); count++;
+      
+      //if ( entry.isWaitingForRelease() ) //{ //inFinalState++; //}
+      
+      } UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINE, CLASS_NAME.getName(), "dumpContents",
+      UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_show_abbrev_cache_stats___FINE", new
+      Object[] { aControllerName, count, inFinalState });
+      
+      */
+     // }
+      }
+      System.out.println(sb.toString());
   }
 
   public synchronized void remove(String aCasReferenceId) {
@@ -366,10 +383,15 @@ public class InProcessCache implements InProcessCacheMBean {
   }
 
   public MessageContext getMessageAccessorByReference(String aCasReferenceId) {
-    if (!cache.containsKey(aCasReferenceId)) {
+//    if (!cache.containsKey(aCasReferenceId)) {
+//      return null;
+//    }
+    CacheEntry casRefEntry = getEntry(aCasReferenceId);
+    if (casRefEntry == null) {
+    	System.out.println("... CAS "+aCasReferenceId+" Not Found In InprocessCache");
       return null;
     }
-    CacheEntry casRefEntry = getEntry(aCasReferenceId);
+
     return casRefEntry.getMessageAccessor();
   }
 
@@ -528,7 +550,13 @@ public class InProcessCache implements InProcessCacheMBean {
     }
     return casRefEntry;
   }
+  public static class UndefinedCacheEntry extends CacheEntry {
 
+	  public UndefinedCacheEntry() {
+		  super(null, null, null);
+	  }
+	  
+  }
   public static class CacheEntry {
     public static final int FINAL_STATE = 1;
 
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/InputChannel.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/InputChannel.java
index 83fe71b..e54edc6 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/InputChannel.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/InputChannel.java
@@ -19,12 +19,26 @@
 
 package org.apache.uima.aae;
 
+import java.util.List;
+
+import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.Endpoint;
+import org.apache.uima.aae.handler.Handler;
 import org.apache.uima.aae.jmx.ServiceInfo;
 import org.apache.uima.aae.message.MessageContext;
 import org.apache.uima.aae.message.MessageWrapper;
+import org.apache.uima.as.client.Listener;
 
 public interface InputChannel extends Channel {
+  // The REPLY type is only for handling replies from remote services.
+  public enum ChannelType {REPLY, REQUEST_REPLY };
+	
+  public void setController(AnalysisEngineController controller) throws Exception;
+  public void setEndpointName(String name);
+  public void setMessageHandler(Handler handler);
+  
+  public ChannelType getChannelType();
   public int getSessionAckMode();
 
   public void ackMessage(MessageContext aMessageContext);
@@ -39,7 +53,7 @@ public interface InputChannel extends Channel {
 
   public boolean isStopped();
 
-  public int getConcurrentConsumerCount();
+ // public int getConcurrentConsumerCount();
 
   public void destroyListener(String anEndpointName, String aDelegateKey);
 
@@ -47,6 +61,15 @@ public interface InputChannel extends Channel {
 
   public void createListenerForTargetedMessages() throws Exception;
   
+  public List<Listener> getListeners();
+  
+//  public void addListener(Listener listener);
+  public List<Listener> registerListener(Listener messageListener);
+  
+  public void disconnectListenersFromQueue() throws Exception;
+
+  public void disconnectListenerFromQueue(Listener listener) throws Exception;
+
   public boolean isFailed(String aDelegateKey);
 
   public boolean isListenerForDestination(String anEndpointName);
@@ -57,7 +80,8 @@ public interface InputChannel extends Channel {
   
   public void terminate();
   
-  public void disconnectListenersFromQueue() throws Exception;
+ 
+ // public void onMessage(MessageWrapper message);
   
-  public void onMessage(MessageWrapper message);
+  public ENDPOINT_TYPE getType();
 }
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/OutputChannel.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/OutputChannel.java
index e0c074a..dab922e 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/OutputChannel.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/OutputChannel.java
@@ -21,11 +21,16 @@ package org.apache.uima.aae;
 
 import org.apache.uima.aae.InProcessCache.CacheEntry;
 import org.apache.uima.aae.controller.AnalysisEngineController;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
+import org.apache.uima.aae.controller.LocalCache.CasStateEntry;
 import org.apache.uima.aae.controller.Endpoint;
 import org.apache.uima.aae.error.AsynchAEException;
 import org.apache.uima.resource.metadata.ProcessingResourceMetaData;
 
 public interface OutputChannel extends Channel {
+	
+  public ENDPOINT_TYPE getType();
+	
   public void setController(AnalysisEngineController aContainer);
 
   public void initialize() throws AsynchAEException;
@@ -36,7 +41,7 @@ public interface OutputChannel extends Channel {
   public void sendReply(int aCommand, Endpoint anEndpoint, String aCasReferenceId, boolean notifyOnJmsException)
   throws AsynchAEException;
 
-  public void sendReply(CacheEntry entry, Endpoint anEndpoint) throws AsynchAEException;
+  public void sendReply(CasStateEntry casStateEntry, Endpoint anEndpoint) throws AsynchAEException;
 
   public void sendReply(ProcessingResourceMetaData aProcessingResourceMetadata,
           Endpoint anEndpoint, boolean serialize) throws AsynchAEException;
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaASUtils.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaASUtils.java
new file mode 100644
index 0000000..f42ed90
--- /dev/null
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaASUtils.java
@@ -0,0 +1,48 @@
+package org.apache.uima.aae;
+
+import java.io.File;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class UimaASUtils {
+	public static boolean isAbsolutePath(String filePath) {
+		return new File(filePath).isAbsolute();
+	}
+	public static String getBaseDir(String filePath) {
+		// match the end of the path. An example:
+		// /user/path/foo.xml
+		// the pattern matches /foo.xml
+		String pattern = "/[^/]*?\\.[xX][mM][lL]";
+	    Pattern r = Pattern.compile(pattern);
+
+	    // Now create matcher object.
+	    Matcher m = r.matcher(filePath);
+	    if (m.find( )) {
+	    	// strip from a given path the filename and
+	    	// return parent path.
+	    	return filePath.replace(m.group(0),"");
+	    }
+	    return null;
+	}
+	public static String replaceBackslashesWithForwardSlashes(String filePath) {
+		return filePath.replace("\\", "/");
+	}
+	public static String fixPath( String parentPath, String childPath ) {
+		// operate on a copy of childPath. In case the childPath is absolute
+		// path we will return the original passed in as an arg
+		String adjustedPath = childPath;
+		
+    	if ( childPath.startsWith("file:")) {
+    		adjustedPath = childPath.substring(6);
+    	}
+    	adjustedPath = UimaASUtils.replaceBackslashesWithForwardSlashes(adjustedPath);
+    	if ( !UimaASUtils.isAbsolutePath(adjustedPath)) {
+    		// relative path to the enclosing descriptor
+    		String baseDir = UimaASUtils.getBaseDir(parentPath);
+    		adjustedPath = baseDir+'/'+adjustedPath;
+    	} else {
+    		adjustedPath = childPath;
+    	}
+    	return adjustedPath;
+	}
+}
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaAsThreadFactory.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaAsThreadFactory.java
index eecfec7..0b375a7 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaAsThreadFactory.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/UimaAsThreadFactory.java
@@ -57,19 +57,40 @@ public class UimaAsThreadFactory implements ThreadFactory {
   
   private volatile boolean initFailed=false;
   
+  private CountDownLatch latchToCountNumberOfInitedThreads;
+
+  public UimaAsThreadFactory() {
+	  
+  }
   public UimaAsThreadFactory(ThreadGroup tGroup) {
     this(tGroup,null);
   }
  
   public UimaAsThreadFactory(ThreadGroup tGroup, PrimitiveAnalysisEngineController aController) {
-    this( tGroup, aController, null);
+    this( tGroup, aController, null, null);
   }
-  public UimaAsThreadFactory(ThreadGroup tGroup, PrimitiveAnalysisEngineController aController, CountDownLatch latchToCountNumberOfTerminatedThreads) {
+  public UimaAsThreadFactory(ThreadGroup tGroup, PrimitiveAnalysisEngineController aController, CountDownLatch latchToCountNumberOfTerminatedThreads, CountDownLatch latchToCountNumberOfInitedThreads) {
     controller = aController;
     theThreadGroup = tGroup;
     this.latchToCountNumberOfTerminatedThreads = latchToCountNumberOfTerminatedThreads;
+    this.latchToCountNumberOfInitedThreads = latchToCountNumberOfInitedThreads;
   }
-  
+  public UimaAsThreadFactory withThreadGroup(ThreadGroup tGroup) {
+	  theThreadGroup = tGroup;
+	  return this;
+  }
+  public UimaAsThreadFactory withPrimitiveController(PrimitiveAnalysisEngineController aController) {
+	  controller = aController;
+	  return this;
+  }
+  public UimaAsThreadFactory withTerminatedThreadsLatch(CountDownLatch latchToCountNumberOfTerminatedThreads) {
+	    this.latchToCountNumberOfTerminatedThreads = latchToCountNumberOfTerminatedThreads;
+	  return this;
+  }
+  public UimaAsThreadFactory withInitedThreadsLatch(CountDownLatch latchToCountNumberOfInitedThreads) {
+	    this.latchToCountNumberOfInitedThreads = latchToCountNumberOfInitedThreads;
+	  return this;
+}
   public void setThreadNamePrefix(String prefix) {
     threadNamePrefix = prefix;
   }
@@ -117,6 +138,10 @@ public class UimaAsThreadFactory implements ThreadFactory {
             		  initFailed = true;
             		  e.printStackTrace();
             		  throw e;
+            	  }  finally {
+            		  if ( latchToCountNumberOfInitedThreads != null ) {
+                		  latchToCountNumberOfInitedThreads.countDown();
+            		  }
             	  }
               } else {
             	  return; // there was failure previously so just return
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/client/UimaAsynchronousEngine.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/client/UimaAsynchronousEngine.java
index 0515b04..600bf31 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/client/UimaAsynchronousEngine.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/client/UimaAsynchronousEngine.java
@@ -44,7 +44,7 @@ import org.apache.uima.resource.metadata.ProcessingResourceMetaData;
  * registered, and {@link #initialize(Map)} method is called, the application may call
  * {@link #process()} method.
  * 
- * 
+ * <p>
  * This API enables the application to dynamically deploy UIMA AS services that it intends to use
  * for processing. These services are deployed in a container and are collocated in the same JVM as
  * the application. The services are considered private and used exclusively by the application. To
@@ -53,24 +53,31 @@ import org.apache.uima.resource.metadata.ProcessingResourceMetaData;
  * descriptor or an array thereof. The application must deploy its "private" services *before*
  * calling {@link #initialize(Map)} method.
  * 
- * 
+ * <p>
  * The application may stop the UIMA AS client in the middle of processing by calling
  * {@link #stop()} method.
  * 
- * 
+ * <p>
  * Listeners can register with the <code>UimaAsynchronousEngine</code> by calling the
  * {@link #addStatusCallbackListener(UimaAsBaseCallbackListener)} method. These listeners receive
  * status callbacks during the processing. An exception to that is the synchronous processing via
  * {@link #sendAndReceiveCAS(CAS)} method. This method returns either a CAS containing results of
  * analysis or an exception. No callbacks are made while processing CASes synchronously.
- * 
+ * <p>
  * An application may choose to implement parallelization of the processing, calling either
  * {@link #sendAndReceiveCAS(CAS)} or {@link #sendCAS(CAS)} methods from multiple threads.
- * 
+ * <p>
  * 
  * 
  */
 public interface UimaAsynchronousEngine {
+	
+	public enum Transport {JMS, Java};
+	
+	public Transport transportType = Transport.Java;
+	
+	public final String ClientTransport = "ClientTransport";
+
   /**
    * @deprecated
    */
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController.java
index a70374f..b1385b5 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController.java
@@ -35,11 +35,12 @@ import org.apache.uima.aae.jmx.ServiceErrors;
 import org.apache.uima.aae.jmx.ServiceInfo;
 import org.apache.uima.aae.jmx.ServicePerformance;
 import org.apache.uima.flow.FinalStep;
+import org.apache.uima.resource.metadata.ResourceMetaData;
 
 public interface AggregateAnalysisEngineController extends AnalysisEngineController {
-  public void mergeTypeSystem(String aTypeSystem, String fromDestination) throws AsynchAEException;
+//  public void mergeTypeSystem(String aTypeSystem, String fromDestination) throws AsynchAEException;
 
-  public void mergeTypeSystem(String aTypeSystem, String fromDestination, String fromServer)
+  public void mergeTypeSystem(ResourceMetaData aProcessingResourceMetadata, String fromDestination, String fromServer)
           throws AsynchAEException;
 
   public void setRemoteSerializationSupported(int code, String fromDestination, String fromServer);
@@ -113,7 +114,7 @@ public interface AggregateAnalysisEngineController extends AnalysisEngineControl
 
   public void setRequestForMetaSentToRemotes();
 
-  public Map getDestinations();
+  public Map<String, Endpoint>  getDestinations();
 
   public ServicePerformance getServicePerformance(String aDelegateKey);
 
@@ -124,7 +125,7 @@ public interface AggregateAnalysisEngineController extends AnalysisEngineControl
   public boolean delayCasIfDelegateInTimedOutState(String aCasReferenceId, String aDelegateKey, long casHashcode)
           throws AsynchAEException;
 
-  public List getChildControllerList();
+  public List<AnalysisEngineController> getChildControllerList();
 
   public void stopCasMultiplier(Delegate casMultiplier, String aCasReferenceId);
 
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController_impl.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController_impl.java
index 8d62047..1e25163 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController_impl.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AggregateAnalysisEngineController_impl.java
@@ -100,7 +100,8 @@ import org.apache.uima.util.Logger;
 import org.apache.uima.util.TypeSystemUtil;
 import org.apache.uima.util.XMLInputSource;
 
-public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineController implements
+public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineController 
+implements
         AggregateAnalysisEngineController, AggregateAnalysisEngineController_implMBean {
 
   /**
@@ -217,6 +218,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
               new Object[] { getComponentName(), aChildController.getComponentName() });
     }
     synchronized(childControllerList) {
+    	System.out.println(":::::::::::::::::: Aggregate:"+getComponentName()+" Registering Child Controller:"+aChildController.getComponentName());;
+
       childControllerList.add(aChildController);
     }
   }
@@ -228,6 +231,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
  
   public void addMessageOrigin(String aCasReferenceId, Endpoint anEndpoint) {
+//	  Thread.dumpStack();
     if (anEndpoint == null) {
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
           UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, getClass().getName(), "addMessageOrigin",
@@ -236,23 +240,26 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       }
       return;
     }
+    System.out.println(".... Aggregate "+getComponentName()+" addMessageOrigin() - Adding endoint with hashCode:"+anEndpoint.hashCode()+" For CAS:"+aCasReferenceId);
+
     originMap.put(aCasReferenceId, anEndpoint);
-    if (UIMAFramework.getLogger().isLoggable(Level.FINE)) {
+//    if (UIMAFramework.getLogger().isLoggable(Level.FINE)) {
       Iterator it = originMap.keySet().iterator();
-      StringBuffer sb = new StringBuffer();
+      StringBuilder sb = new StringBuilder();
       while (it.hasNext()) {
         String key = (String) it.next();
         Endpoint e = (Endpoint) originMap.get(key);
         if (e != null) {
           sb.append("\t\nCAS:" + key + " Origin:" + e.getEndpoint());
         }
-      }
+ //     }
       /*
        * UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINE, CLASS_NAME.getName(),
        * "addMessageOrigin", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
        * "UIMAEE_dump_msg_origin__FINE", new Object[] {getComponentName(), sb.toString()});
        */
     }
+      System.out.println(".... Aggregate "+getComponentName()+" addMessageOrigin() - Origin Map Contents:\n"+sb.toString());
   }
 
   public boolean isDelegateDisabled(String aDelegateKey) {
@@ -270,6 +277,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
 
   public void setServiceEndpointName(String anEndpointName) {
+	  //Thread.currentThread().dumpStack();
     serviceEndpointName = anEndpointName;
     if (this.isTopLevelComponent()) {
       // This is done so that the collocated client application can determine where to send messages
@@ -293,6 +301,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
  
   public Endpoint getMessageOrigin(String aCasReferenceId) {
       if (originMap.containsKey(aCasReferenceId)) {
+  	    System.out.println(".... Aggregate "+getComponentName()+" getMessageOrigin() - endoint with hashCode:"+((Endpoint) originMap.get(aCasReferenceId)).hashCode()+" For CAS:"+aCasReferenceId);
+
         return (Endpoint) originMap.get(aCasReferenceId);
     }
     return null;
@@ -335,9 +345,9 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   public void mapEndpointsToKeys(ConcurrentHashMap aDestinationMap) {
     destinationMap = aDestinationMap;
-    Set set = destinationMap.entrySet();
-    for (Iterator it = set.iterator(); it.hasNext();) {
-      Map.Entry entry = (Map.Entry) it.next();
+    Set<Map.Entry<String, Endpoint>> set = destinationMap.entrySet();
+    for (Iterator<Map.Entry<String, Endpoint>> it = set.iterator(); it.hasNext();) {
+      Map.Entry<String, Endpoint> entry =  it.next();
       Endpoint endpoint = (Endpoint) entry.getValue();
       if (endpoint != null) {
         if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
@@ -346,6 +356,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                   "UIMAEE_endpoint_to_key_map__FINE",
                   new Object[] { getName(), (String) entry.getKey(), endpoint.getEndpoint() });
         }
+        System.out.println("............ Aggregate:"+getName()+" Delegate:"+entry.getKey()+" Endpoint:"+endpoint.getEndpoint() );
         if (destinationToKeyMap == null) {
           destinationToKeyMap = new HashMap();
         }
@@ -462,6 +473,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     if (aClientEndpoint == null) {
       aClientEndpoint = getClientEndpoint();
     }
+    /*
     if (!aClientEndpoint.isRemote()) {
       UimaTransport transport = getTransport(aClientEndpoint.getEndpoint());
       UimaMessage message = transport.produceMessage(AsynchAEMessage.CollectionProcessComplete,
@@ -471,7 +483,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     } else {
       getOutputChannel().sendReply(AsynchAEMessage.CollectionProcessComplete, aClientEndpoint, null, false);
     }
-
+*/
+    getOutputChannel(aClientEndpoint).sendReply(AsynchAEMessage.CollectionProcessComplete, aClientEndpoint, null, false);
     clearStats();
   }
 
@@ -486,10 +499,12 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       boolean cacheNotEmpty = true;
       boolean shownOnce = false;
       final Object localMux = new Object();
+      
+      /*
       while (cacheNotEmpty) {
         InProcessCache cache = getInProcessCache();
         if (!shownOnce) {
-          shownOnce = true;
+          //shownOnce = true;
           cache.dumpContents(getComponentName());
         }
 
@@ -501,6 +516,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
           }
         }
       }
+      */
     } catch (Exception e) {
       throw new AsynchAEException(e);
     }
@@ -563,10 +579,11 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         }
       }
     } else {
-      Set set = destinationMap.entrySet();
-      for (Iterator it = set.iterator(); it.hasNext();) {
-        Map.Entry entry = (Map.Entry) it.next();
+      Set<?> set = destinationMap.entrySet();
+      for (Iterator<?> it = set.iterator(); it.hasNext();) {
+        Map.Entry<String, Endpoint> entry = (Map.Entry) it.next();
         Endpoint endpoint = (Endpoint) entry.getValue();
+        /*
         if (endpoint != null && endpoint.getStatus() == Endpoint.OK) {
 
           if (!endpoint.isRemote()) {
@@ -593,6 +610,10 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
             endpoint.startCollectionProcessCompleteTimer();
           }
         }
+        */
+        getOutputChannel(endpoint).sendRequest(AsynchAEMessage.CollectionProcessComplete, null, endpoint);
+        endpoint.startCollectionProcessCompleteTimer();
+
       }
     }
   }
@@ -609,7 +630,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     return false;
   }
 
-  public Map getDestinations() {
+  public Map<String, Endpoint> getDestinations() {
     return destinationMap;
   }
 
@@ -639,23 +660,23 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   private void stopListener(String key, Endpoint endpoint) throws Exception {
     // Stop the Listener on endpoint that has been disabled
-    InputChannel iC = null;
+//    InputChannel iC = null;
     String destName = null;
     if (endpoint.getDestination() != null) {
       destName = endpoint.getDestination().toString();
-      iC = getInputChannel(destName);
+ //     iC = getInputChannel(destName);
     } else {
       destName = endpoint.getReplyToEndpoint();
-      iC = getInputChannel(destName);
+//      iC = getInputChannel(destName);
     }
-    if (iC != null) {
+//    if (iC != null) {
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
         UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, getClass().getName(), "stopListener",
                 UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_stopping_listener__INFO",
                 new Object[] { getComponentName(), destName, key });
       }
-      iC.destroyListener(destName, key);
-    }
+      inputChannel.destroyListener(destName, key);
+//    }
   }
 
   public void disableDelegates(List aDelegateList) throws AsynchAEException {
@@ -882,7 +903,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
           if (localCache.lookupEntry(aNewCasReferenceId) == null) {
             // Add this Cas Id to the local cache. Every input CAS goes through here
             CasStateEntry casStateEntry = localCache.createCasStateEntry(aNewCasReferenceId);
-            casStateEntry.setInputCasReferenceId(anInputCasReferenceId);
+            casStateEntry.setParentCasReferenceId(anInputCasReferenceId);
           }
 
           // Save the subordinate Flow Object in a cache. Flow exists in the
@@ -947,7 +968,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       CacheEntry cacheEntry = getInProcessCache().getCacheEntryForCAS(aCasReferenceId);
       Endpoint replyEndpoint = getReplyEndpoint(cacheEntry, casStateEntry);
       if (replyEndpoint != null) {
-        getOutputChannel().sendReply(new ServiceShutdownException(), aCasReferenceId, null,
+        getOutputChannel(replyEndpoint).sendReply(new ServiceShutdownException(), aCasReferenceId, null,
                 replyEndpoint, AsynchAEMessage.Process);
       }
     } catch (Exception ex) {
@@ -970,7 +991,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       // Check if this CAS has a parent
       if (casStateEntry.isSubordinate()) {
         // Fetch parent's cache entry
-        parentCasStateEntry = getLocalCache().lookupEntry(casStateEntry.getInputCasReferenceId());
+        parentCasStateEntry = getLocalCache().lookupEntry(casStateEntry.getParentCasReferenceId());
         // Check the state of the parent CAS. If it is marked as failed, it means that
         // one of its child CASes failed and error handling was configured to fail the
         // CAS. Such failure of a child CAS causes a failure of the parent CAS. All child
@@ -990,7 +1011,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
               fcEndpoint.setReplyEndpoint(true);
               fcEndpoint.setIsCasMultiplier(true);
               fcEndpoint.setFreeCasEndpoint(true);
-              getOutputChannel().sendRequest(AsynchAEMessage.ReleaseCAS, entry.getCasReferenceId(),
+              getOutputChannel(fcEndpoint).sendRequest(AsynchAEMessage.ReleaseCAS, entry.getCasReferenceId(),
                       fcEndpoint);
             }
             // Check if a request to stop generation of new CASes from the parent of
@@ -1490,7 +1511,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
   }
 
   public void sendRequestForMetadataToRemoteDelegates() throws AsynchAEException {
-    synchronized(childControllerList) {
+    /*
+	  synchronized(childControllerList) {
       //  Add a delay of 100ms before sending requests for metadata to remote delegates.
       //  This is done to give the broker enough time to 'finalize' creation of
       //  temp reply queues for each remote delegate. It's been observed (on MAC OS only) that AMQ
@@ -1513,6 +1535,9 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         }
       }
     }
+    */
+	    System.out.println("........... Aggregate.sendRequestForMetadataToRemoteDelegates():"+getName());
+
     Endpoint[] delegateEndpoints = new Endpoint[destinationMap.size()];
 
     // First copy endpoints to an array so that we dont get Concurrent access problems
@@ -1574,6 +1599,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
             return;
           }
           if (delegateEndpoints[i].getStatus() == Endpoint.OK ) {
+              System.out.println("----- Aggregate Service:"+getName()+ " dispatching GetMeta request to remote "+delegateEndpoints[i].getEndpoint());
+
             dispatchMetadataRequest(delegateEndpoints[i]);
           }
         }
@@ -1583,8 +1610,9 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         // collocated delegate
         delegateEndpoints[i].initialize();
         delegateEndpoints[i].setController(this);
-
         delegateEndpoints[i].setWaitingForResponse(true);
+        getOutputChannel(ENDPOINT_TYPE.DIRECT).sendRequest(AsynchAEMessage.GetMeta, null, delegateEndpoints[i]);
+/*
         try {
           UimaMessage message = getTransport(delegateEndpoints[i].getEndpoint()).produceMessage(
                   AsynchAEMessage.GetMeta, AsynchAEMessage.Request, getName());
@@ -1593,6 +1621,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         } catch (Exception e) {
           throw new AsynchAEException(e);
         }
+        */
       }
     }
   }
@@ -1600,7 +1629,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
   private CasStateEntry fetchParentCasFromLocalCache(CasStateEntry casStateEntry) throws Exception {
     // Lookup parent CAS in the local cache
     CasStateEntry parentCasStateEntry = localCache.lookupEntry(casStateEntry
-            .getInputCasReferenceId());
+            .getParentCasReferenceId());
     if (parentCasStateEntry == null) {
 
       if (UIMAFramework.getLogger().isLoggable(Level.INFO)) {
@@ -1623,7 +1652,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     try {
       // Fetch the parent Cas cache entry
       parentCASCacheEntry = getInProcessCache().getCacheEntryForCAS(
-              casStateEntry.getInputCasReferenceId());
+              casStateEntry.getParentCasReferenceId());
     } catch (Exception ex) {
 
       if (UIMAFramework.getLogger().isLoggable(Level.INFO)) {
@@ -1633,7 +1662,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                 "fetchParentCasFromGlobalCache",
                 UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
                 "UIMAEE_cas_not_found__INFO",
-                new Object[] { getComponentName(), casStateEntry.getInputCasReferenceId(),
+                new Object[] { getComponentName(), casStateEntry.getParentCasReferenceId(),
                     "InProcessCache" });
       }
     }
@@ -1641,6 +1670,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
   }
 
   private boolean casHasChildrenInPlay(CasStateEntry casStateEntry) throws Exception {
+	  System.out.println(".. FinalState.casHasChildrenInPlay()-CAS:"+casStateEntry.getCasReferenceId()+" Number of Child CASes in Play:"+casStateEntry.getSubordinateCasInPlayCount());
     if (casStateEntry.getSubordinateCasInPlayCount() > 0) {
       // This CAS has child CASes still in play. This CAS will remain in the cache
       // until all its children are fully processed.
@@ -1716,8 +1746,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     }
     // Found entries in caches for a given CAS id
     try {
-      endpoint = getInProcessCache().getEndpoint(null, aCasReferenceId);
-
+     // endpoint = getInProcessCache().getEndpoint(null, aCasReferenceId);
+    	endpoint = getMessageOrigin(aCasReferenceId);
       synchronized (super.finalStepMux) {
         // Check if the global cache still contains the CAS. It may have been deleted by another
         // thread already
@@ -1766,12 +1796,15 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                   new Object[] { getComponentName(), aCasReferenceId });
         }
         // Determine if this CAS is a child of some CAS
-        isSubordinate = casStateEntry.getInputCasReferenceId() != null;
+        isSubordinate = casStateEntry.getParentCasReferenceId() != null;
 
         if (isSubordinate) {
           // fetch the destination of a CM that produced this CAS, so that we know where to send
           // Free Cas Notification
-          freeCasEndpoint = cacheEntry.getFreeCasEndpoint();
+         // freeCasEndpoint = cacheEntry.getFreeCasEndpoint();
+          freeCasEndpoint = casStateEntry.getFreeCasNotificationEndpoint();
+     	  System.out.println(".........Service:"+getComponentName()+" Cas:"+aCasReferenceId+" is subordinate - freeCasEndpoint="+freeCasEndpoint);
+
           parentCasStateEntry = fetchParentCasFromLocalCache(casStateEntry);
           parentCASCacheEntry = fetchParentCasFromGlobalCache(casStateEntry);
           doDecrementChildCount = true;
@@ -1796,6 +1829,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
             if (isSubordinate) {
               // drop the flow since we no longer need it
               dropFlow(aCasReferenceId, true);
+              System.out.println(">>>>>>>>>>>> Controller:"+getComponentName()+" Dropping CAS:"+aCasReferenceId);
               // Drop the CAS and remove cache entry for it
               dropCAS(aCasReferenceId, true);
               casDropped = true;
@@ -1817,14 +1851,18 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
           cEndpoint = replyToClient(cacheEntry, casStateEntry);
           replySentToClient = true;
           
-          if (cEndpoint != null && cEndpoint.isRemote()) {
+          if (cEndpoint != null && cEndpoint.isRemote() && !cEndpoint.getServerURI().equals("java")) {
             // if this service is a Cas Multiplier don't remove the CAS. It will be removed
             // when a remote client sends explicit Release CAS Request
             if (!isCasMultiplier()) {
+            	System.out.println(".... Aggregate:"+getComponentName()+" Releasing CAS:"+aCasReferenceId +" Client is Remote");
+
               // Drop the CAS and remove cache entry for it
               dropCAS(aCasReferenceId, true);
             }
             casDropped = true;
+          } else if (cEndpoint != null && cEndpoint.getServerURI().equals("java")) {
+        	  casDropped = true;
           } else {
             // Remove entry from the local cache for this CAS. If the client
             // is remote the entry was removed in replyToClient()
@@ -1839,12 +1877,13 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         }
 
         if (parentCasStateEntry == null && isSubordinate) {
-          parentCasStateEntry = localCache.lookupEntry(casStateEntry.getInputCasReferenceId());
+          parentCasStateEntry = localCache.lookupEntry(casStateEntry.getParentCasReferenceId());
         }
         if (doDecrementChildCount) {
           // Child CAS has been fully processed, decrement its parent count of active child CASes
           if (parentCasStateEntry != null) {
             parentCasStateEntry.decrementSubordinateCasInPlayCount();
+        	  System.out.println("... Controller:"+getComponentName()+" CAS:"+parentCasStateEntry.getCasReferenceId()+" Decremented Child Count - Courrent Count:"+parentCasStateEntry.getSubordinateCasInPlayCount());
             // If debug level=FINEST dump the entire cache
             localCache.dumpContents();
             if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
@@ -1932,8 +1971,10 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
           freeCasEndpoint.setReplyEndpoint(true);
           freeCasEndpoint.setIsCasMultiplier(true);
           freeCasEndpoint.setFreeCasEndpoint(true);
+          System.out.println("....Aggregate "+getComponentName()+" Sending ReleaseCAS to remote CM - CASID:"+aCasReferenceId);
+
           // send Free CAS Notification to a Cas Multiplier
-          getOutputChannel().sendRequest(AsynchAEMessage.ReleaseCAS, aCasReferenceId,
+          getOutputChannel(freeCasEndpoint).sendRequest(AsynchAEMessage.ReleaseCAS, aCasReferenceId,
                   freeCasEndpoint);
         } catch (Exception e) {
           if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
@@ -1984,7 +2025,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
   }
 
   private boolean forceToDropTheCas(CasStateEntry entry, CacheEntry cacheEntry, FinalStep aStep) {
-    // Get the key of the Cas Producer
+   
+	  // Get the key of the Cas Producer
     String casProducer = cacheEntry.getCasProducerAggregateName();
     // CAS is considered new from the point of view of this service IF it was produced by it
     boolean isNewCas = (cacheEntry.isNewCas() && casProducer != null && getComponentName().equals(
@@ -2025,10 +2067,13 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
               new Object[] { getComponentName(), casStateEntry.getCasReferenceId(),
                   replyEndpoint.getEndpoint() });
     }
+    getOutputChannel(replyEndpoint).sendReply(casStateEntry.getErrors().get(0),
+            casStateEntry.getCasReferenceId(), null, replyEndpoint, AsynchAEMessage.Process);
+/*
     if (replyEndpoint.isRemote()) {
       // this is an input CAS that has been marked as failed. Return the input CAS
       // and an exception to the client.
-      getOutputChannel().sendReply(casStateEntry.getErrors().get(0),
+      getOutputChannel(replyEndpoint).sendReply(casStateEntry.getErrors().get(0),
               casStateEntry.getCasReferenceId(), null, replyEndpoint, AsynchAEMessage.Process);
     } else {
       replyEndpoint.setReplyEndpoint(true);
@@ -2056,6 +2101,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       vmTransport.getUimaMessageDispatcher(replyEndpoint.getEndpoint()).dispatch(message);
       dropStats(casStateEntry.getCasReferenceId(),getName());
     }
+    */
   }
 
   private boolean sendExceptionToClient(CacheEntry cacheEntry, CasStateEntry casStateEntry,
@@ -2070,7 +2116,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
       // Fetch the top ancestor CAS of this CAS.
       CasStateEntry topAncestorCasStateEntry = getLocalCache().getTopCasAncestor(
-              casStateEntry.getInputCasReferenceId());
+              casStateEntry.getParentCasReferenceId());
       if ( topAncestorCasStateEntry != null ) {
     	// check the state
           if (topAncestorCasStateEntry.isFailed() && casHasExceptions(casStateEntry)
@@ -2125,7 +2171,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       sendReplyWithException(cacheEntry, casStateEntry, replyEndpoint);
     } else {
       // Send response to a given endpoint
-      getOutputChannel().sendReply(cacheEntry, replyEndpoint);
+      getOutputChannel(replyEndpoint).sendReply(casStateEntry, replyEndpoint);
     }
     // Drop the CAS only if the client is remote and the CAS is an input CAS OR
     // the CAS is a child but there was a failure delivering it to a client. The client
@@ -2139,7 +2185,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 		if ( casStateEntry.isSubordinate()) {
 			try {
 				
-				String inputCasId = casStateEntry.getInputCasReferenceId();
+				String inputCasId = casStateEntry.getParentCasReferenceId();
 				if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
 					UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(),
 							"sendReplyToRemoteClient", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
@@ -2175,7 +2221,9 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                 "UIMAEE_client_dead__FINE",
                 new Object[] { getComponentName(), replyEndpoint.getDestination().toString(), casStateEntry.getCasReferenceId()});
         }
-    	dropCAS(casStateEntry.getCasReferenceId(), true);
+    	if ( !replyEndpoint.getServerURI().equals("java")) {
+        	dropCAS(casStateEntry.getCasReferenceId(), true);
+    	}
     	// If the cache is empty change the state of the Aggregate to idle
     	if (getInProcessCache().isEmpty()) {
     	  endProcess(AsynchAEMessage.Process);
@@ -2223,23 +2271,29 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         // client perspective, this Cas Multiplier Aggregate is a black box,
         // all CASes produced here must be linked with the input CAS.
         // Find the top ancestor of this CAS. It is the input CAS sent by the client
-        String inputCasId = getLocalCache().lookupInputCasReferenceId(casStateEntry);
+//        String inputCasId = getLocalCache().lookupInputCasReferenceId(casStateEntry);
+        String inputCasId = casStateEntry.getInputCasReferenceId(); //getLocalCache().lookupInputCasReferenceId(casStateEntry);
         // Modify the parent of this CAS.
         if (inputCasId != null ) {
-          if ( !inputCasId.equals(casStateEntry.getInputCasReferenceId())) {
-            cacheEntry.setInputCasReferenceId(inputCasId);
-          }
-          // Update counters in the parents controller local cache. 
-          CasStateEntry parentCasStateEntry = 
-            parentController.getLocalCache().lookupEntry(inputCasId);
-          if ( parentCasStateEntry != null ) {
-            parentCasStateEntry.incrementSubordinateCasInPlayCount();
-            parentCasStateEntry.incrementOutstandingFlowCounter();
+//          if ( !inputCasId.equals(casStateEntry.getParentCasReferenceId())) {
+//            cacheEntry.setInputCasReferenceId(inputCasId);
+//          }
+          if ( parentController != null ) {
+        	  // Update counters in the parents controller local cache. 
+              CasStateEntry parentCasStateEntry = 
+                parentController.getLocalCache().lookupEntry(inputCasId);
+              if ( parentCasStateEntry != null ) {
+                parentCasStateEntry.incrementSubordinateCasInPlayCount();
+                parentCasStateEntry.incrementOutstandingFlowCounter();
+              } 
           }
+         
         }
       }
       // Send CAS to a given reply endpoint
-      sendVMMessage(mType, replyEndpoint, cacheEntry);
+//      sendVMMessage(mType, replyEndpoint, cacheEntry);
+      getOutputChannel(replyEndpoint).sendReply(casStateEntry, replyEndpoint);
+
     }
   }
 
@@ -2263,7 +2317,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       HashMap map = new HashMap();
       map.put(AsynchAEMessage.Command, AsynchAEMessage.Process);
       map.put(AsynchAEMessage.CasReference, casStateEntry.getCasReferenceId());
-      handleError(map, new UnknownDestinationException());
+      handleError(map, new UnknownDestinationException("Controller:"+getComponentName()+" CasReferenceId:"+casStateEntry.getCasReferenceId()+" Destination Not Found"));
       return false;
     }
     // Dont send a reply to the client if the client is a CAS multiplier
@@ -2276,7 +2330,14 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   private Endpoint replyToClient(CacheEntry cacheEntry, CasStateEntry casStateEntry)
           throws Exception {
-    Endpoint endpoint = getReplyEndpoint(cacheEntry, casStateEntry);
+	 // Thread.dumpStack();
+   // Endpoint endpoint = getReplyEndpoint(cacheEntry, casStateEntry);
+    
+    CasStateEntry inputCasCasStateEntry = getLocalCache().get(casStateEntry.getInputCasReferenceId());
+    
+    Endpoint endpoint = inputCasCasStateEntry.getClientEndpoint();
+   // Endpoint endpoint = getMessageOrigin(cacheEntry.getCasReferenceId());
+    
     if (!validEndpoint(endpoint, casStateEntry)) {
       return null; // the reason has already been logged
     }
@@ -2294,8 +2355,12 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     if (!isStopped()) {
       try {
         if (endpoint.isRemote()) {
+       	  System.out.println("......replyToClient()[Remote] - Controller:"+getComponentName()+" CasID:"+cacheEntry.getCasReferenceId());
+
           sendReplyToRemoteClient(cacheEntry, casStateEntry, endpoint);
         } else {
+        	  System.out.println("......replyToClient()[Direct] - Controller:"+getComponentName()+" CasID:"+cacheEntry.getCasReferenceId());
+
           sendReplyToCollocatedClient(cacheEntry, casStateEntry, endpoint);
         }
       } catch ( Exception e) {
@@ -2323,6 +2388,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 	  }
 	  return false;
   }
+  /*
   private void sendVMMessage(int messageType, Endpoint endpoint, CacheEntry cacheEntry)
           throws Exception {
     // If the CAS was produced by this aggregate send the request message to the client
@@ -2352,7 +2418,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     dropStats(cacheEntry.getCasReferenceId(), getName());
 
   }
-
+*/
   private Endpoint getReplyEndpoint(CacheEntry cacheEntry, CasStateEntry casStateEntry)
           throws Exception {
     Endpoint endpoint = null;
@@ -2481,6 +2547,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   private void dispatch(CacheEntry entry, Endpoint anEndpoint) throws AsynchAEException {
     if (!anEndpoint.isRemote()) {
+        getOutputChannel(ENDPOINT_TYPE.DIRECT).sendRequest(AsynchAEMessage.Process, entry.getCasReferenceId(), anEndpoint);
+/*
       try {
         UimaTransport transport = getTransport(anEndpoint.getEndpoint());
         UimaMessage message = transport.produceMessage(AsynchAEMessage.Process,
@@ -2499,6 +2567,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                   "UIMAEE_exception__WARNING", e);
         }
       }
+      */
     } else {
       // Check delegate's state before sending it a CAS. The delegate
       // may have previously timed out and is in a process of pinging
@@ -2508,7 +2577,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       // delayed CASes will be dispatched to the delegate.
       if (!delayCasIfDelegateInTimedOutState(entry.getCasReferenceId(), anEndpoint.getDelegateKey(), entry.getCas().hashCode())) {
         // The delegate is in the normal state so send it this CAS
-        getOutputChannel().sendRequest(AsynchAEMessage.Process, entry.getCasReferenceId(), anEndpoint);
+        getOutputChannel(anEndpoint).sendRequest(AsynchAEMessage.Process, entry.getCasReferenceId(), anEndpoint);
       }
     }
   }
@@ -2612,7 +2681,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         if (endpoint.getSerialFormat() == SerialFormat.BINARY) {
           endpoint.setSerialFormat(SerialFormat.XMI);  
         }
-        getOutputChannel().sendRequest(AsynchAEMessage.Process, entry.getCasReferenceId(), endpoint);
+        getOutputChannel(endpoint).sendRequest(AsynchAEMessage.Process, entry.getCasReferenceId(), endpoint);
       }    
     }
 
@@ -2729,15 +2798,20 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                                                                                                      SerialFormat.COMPRESSED_FILTERED);
     }
   }
-  
-  public void mergeTypeSystem(String aTypeSystem, String fromDestination) throws AsynchAEException {
-    mergeTypeSystem(aTypeSystem, fromDestination, null);
-  }
+//  
+//  public void mergeTypeSystem(String aTypeSystem, String fromDestination) throws AsynchAEException {
+//    mergeTypeSystem(aTypeSystem, fromDestination, null);
+//  }
 
 //  public synchronized void mergeTypeSystem(String aTypeSystem, String fromDestination,
-  public void mergeTypeSystem(String aTypeSystem, String fromDestination,
+//  public void mergeTypeSystem(String aTypeSystem, String fromDestination,
+ //         String fromServer) throws AsynchAEException {
+  public void mergeTypeSystem(ResourceMetaData resource, String fromDestination,
           String fromServer) throws AsynchAEException {
+
     mergeLock.lock();
+    System.out.println("AggregateAnalysisEngineController.mergeTypeSystem() - from "+fromDestination);
+
     try {
       // Find the endpoint for this service, given its input queue name and broker URI.
       // We now allow endpoints managed by different servers to have the same queue name.
@@ -2763,9 +2837,10 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         if ( endpoint.getServiceInfo() != null ) {
           endpoint.getServiceInfo().setState(ServiceState.RUNNING.name());
         }
-        ResourceMetaData resource = null;
+//        ResourceMetaData resource = null;
         ServiceInfo remoteDelegateServiceInfo = null;
-        if (aTypeSystem.trim().length() > 0) {
+//        if (aTypeSystem.trim().length() > 0) {
+        if ( resource != null ) {  
           if (endpoint.isRemote()) {
             if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.CONFIG)) {
               UIMAFramework.getLogger(CLASS_NAME).logrb(Level.CONFIG, CLASS_NAME.getName(),
@@ -2779,10 +2854,12 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
                     "mergeTypeSystem", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
                     "UIMAEE_merge_ts_from_delegate__CONFIG", new Object[] { fromDestination });
           }
+          /*
           ByteArrayInputStream bis = new ByteArrayInputStream(aTypeSystem.getBytes());
           XMLInputSource in1 = new XMLInputSource(bis, null);
 
           resource = UIMAFramework.getXMLParser().parseResourceMetaData(in1);
+          */
           if (isStopped()) {
             return;
           }
@@ -2832,7 +2909,11 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
         //  
         if (collocatedAggregate || resource instanceof ProcessingResourceMetaData) {
-          if (allTypeSystemsMerged()) {
+        	System.out.println("#############  Aggregate:"+getName()+" Merged Typesystem from "+fromDestination);//+" allTypeSystemsMerged():"+allTypeSystemsMerged());
+
+        	if (allTypeSystemsMerged()) {
+                System.out.println("!!!!! AGGREGATE:"+getName()+" Got Metadata from ALL delegates");
+
             if (!isStopped()) {
               try {
                 completeInitialization();
@@ -2862,6 +2943,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         }
       }
     } catch (Exception e) {
+    	e.printStackTrace();
       throw new AsynchAEException(e);
     } finally {
     	mergeLock.unlock();
@@ -3007,11 +3089,14 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 	  return aggTypePriorities;
   }
   protected void startProcessing() throws Exception {
-	  
+	  System.out.println(",,,,,,,,,,, Controller "+getName()+" Opening Latch ,,,,,,,,,,,,");
+
 	    // Open latch to allow messages to be processed. The
 	    // latch was closed to prevent messages from entering
 	    // the controller before it is initialized.
 	    latch.openLatch(getName(), isTopLevelComponent(), true);
+		  System.out.println(",,,,,,,,,,, Controller "+getName()+" Latch is Opened,,,,,,,,,,,, Latch hashcode:"+latch.hashCode());
+
 	    initialized = true;
 	    // Notify client listener that the initialization of the controller was successfull
 	    notifyListenersWithInitializationStatus(null);
@@ -3055,6 +3140,8 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       }
       delegateCount++;
     }
+    System.out.println("+++++++++++++ delegateCount="+delegateCount+" destinationMap.size()="+destinationMap.size());
+
     if (delegateCount == destinationMap.size()) {
       return true; // All delegates responded to GetMeta request
     }
@@ -3110,7 +3197,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
       }
     }
-    getOutputChannel().sendRequest(AsynchAEMessage.GetMeta, null, anEndpoint);
+    getOutputChannel(anEndpoint).sendRequest(AsynchAEMessage.GetMeta, null, anEndpoint);
   }
 
   public void retryMetadataRequest(Endpoint anEndpoint) throws AsynchAEException {
@@ -3163,7 +3250,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
         getInProcessCache().getEndpoint(anEndpoint, casReferenceId).cancelTimer();
         Endpoint requestOrigin = cachedEntries[i].getMessageOrigin();
         try {
-          getOutputChannel().sendReply(
+          getOutputChannel(requestOrigin).sendReply(
                   new UimaEEServiceException("Delegates Not Found To Process CAS on Endpoint:"
                           + anEndpoint), casReferenceId, parentCasReferenceId, requestOrigin,
                   AsynchAEMessage.Process);
@@ -3186,7 +3273,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   public void retryLastCommand(int aCommand, Endpoint anEndpoint, String aCasReferenceId) {
     try {
-      getOutputChannel().sendRequest(aCommand, aCasReferenceId, anEndpoint);
+      getOutputChannel(anEndpoint).sendRequest(aCommand, aCasReferenceId, anEndpoint);
     } catch (AsynchAEException e) {
 
     }
@@ -3206,7 +3293,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       // block in getInputChannel() on the latch
       if (isTopLevelComponent() && getInputChannel() != null) {
         serviceInfo.setInputQueueName(getInputChannel().getName());
-        serviceInfo.setBrokerURL(super.getBrokerURL());
+     //   serviceInfo.setBrokerURL(super.getBrokerURL());
       } else {
         serviceInfo.setInputQueueName(getName());
         serviceInfo.setBrokerURL("vm://localhost");
@@ -3299,8 +3386,12 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
       cache.destroy();
     }
   }
-
+ 
   public void stop() {
+	  for( AnalysisEngineController delegate : getChildControllerList() ) {
+		  delegate.stop();
+	  }
+
 	  super.stop(true);  // shutdown now
 	  
 	  // enable blocked threads to finish // https://issues.apache.org/jira/browse/UIMA-3433
@@ -3327,7 +3418,7 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
     
   }
 
-  public List getChildControllerList() {
+  public List<AnalysisEngineController> getChildControllerList() {
     return childControllerList;
   }
 
@@ -3417,9 +3508,10 @@ public class AggregateAnalysisEngineController_impl extends BaseAnalysisEngineCo
   }
   
   public void changeCollocatedDelegateState( String delegateKey, ServiceState state ) throws Exception {
-    if ( delegateKey != null && state != null ) {
+	  System.out.println("............. changeCollocatedDelegateState - delegateKey:"+delegateKey+" state is Null:"+(state==null));
+	  if ( delegateKey != null && state != null ) {
       synchronized(childControllerList) {
-        if ( childControllerList.size() > 0 ) {
+        if ( !childControllerList.isEmpty() ) {
           for( AnalysisEngineController childController : childControllerList ) {
             if ( delegateKey.equals(childController.getKey())) {
               childController.changeState(state);
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AnalysisEngineController.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AnalysisEngineController.java
index 5291427..ddf7f4e 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AnalysisEngineController.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/AnalysisEngineController.java
@@ -19,6 +19,7 @@
 
 package org.apache.uima.aae.controller;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
@@ -29,6 +30,7 @@ import org.apache.uima.aae.InputChannel;
 import org.apache.uima.aae.OutputChannel;
 import org.apache.uima.aae.UimaAsContext;
 import org.apache.uima.aae.UimaEEAdminContext;
+import org.apache.uima.aae.controller.BaseAnalysisEngineController.ENDPOINT_TYPE;
 import org.apache.uima.aae.controller.BaseAnalysisEngineController.ServiceState;
 import org.apache.uima.aae.error.AsynchAEException;
 import org.apache.uima.aae.error.ErrorContext;
@@ -41,7 +43,10 @@ import org.apache.uima.aae.monitor.Monitor;
 import org.apache.uima.aae.spi.transport.UimaMessageListener;
 import org.apache.uima.aae.spi.transport.UimaTransport;
 import org.apache.uima.analysis_engine.AnalysisEngineProcessException;
+import org.apache.uima.as.client.DirectInputChannel;
+import org.apache.uima.as.client.Listener;
 import org.apache.uima.cas.CAS;
+import org.apache.uima.resource.ResourceSpecifier;
 
 public interface AnalysisEngineController extends ControllerLifecycle {
   public static final String CasPoolSize = "CasPoolSize";
@@ -54,17 +59,29 @@ public interface AnalysisEngineController extends ControllerLifecycle {
 
   public void setInputChannel(InputChannel anInputChannel) throws Exception;
 
+  public void setDirectInputChannel(DirectInputChannel anInputChannel) throws Exception;
+  
+  public void setJmsInputChannel(InputChannel anInputChannel) throws Exception;
+
+  public InputChannel getInputChannel(ENDPOINT_TYPE et);
+
   public void addInputChannel(InputChannel anInputChannel) throws Exception;
 
   public String getServiceEndpointName();
 
+  public void setServiceId(String name);
+
+  public String getServiceId();
+
   public void handleDelegateLifeCycleEvent(String anEndpoint, int aDelegateCount);
 
   public void takeAction(String anAction, String anEndpointName, ErrorContext anErrorContext);
 
   public InputChannel getInputChannel();
 
-  public InputChannel getInputChannel(String aQueueName);
+  public List<Listener> getAllListeners();
+
+  //public InputChannel getInputChannel(String aQueueName);
 
   public void saveReplyTime(long snapshot, String aKey);
 
@@ -109,10 +126,16 @@ public interface AnalysisEngineController extends ControllerLifecycle {
   public long getTime(String aCasReferenceId, String anEndpointName);
 
   public ErrorHandlerChain getErrorHandlerChain();
-
+/*
   public void setOutputChannel(OutputChannel anOutputChannel) throws Exception;
 
   public OutputChannel getOutputChannel();
+*/
+  public void addOutputChannel(OutputChannel anOutputChannel) throws Exception;
+
+  public OutputChannel getOutputChannel(Endpoint anEndpoint);
+  
+  public OutputChannel getOutputChannel(ENDPOINT_TYPE et);
 
   public void setCasManager(AsynchAECasManager aCasManager);
 
@@ -195,13 +218,13 @@ public interface AnalysisEngineController extends ControllerLifecycle {
 
   public UimaMessageListener getUimaMessageListener(String aDelegateKey);
 
-  public UimaTransport getTransport(UimaAsContext aContext, String aKey) throws Exception;
-
-  public UimaTransport getTransport(String aKey) throws Exception;
-
-  public void initializeVMTransport(int parentControllerReplyConsumerCount) throws Exception;
-
-  public InputChannel getReplyInputChannel(String aDelegateKey);
+//  public UimaTransport getTransport(UimaAsContext aContext, String aKey) throws Exception;
+//
+//  public UimaTransport getTransport(String aKey) throws Exception;
+//
+//  public void initializeVMTransport(int parentControllerReplyConsumerCount) throws Exception;
+//
+//  public InputChannel getReplyInputChannel(String aDelegateKey);
 
   public LocalCache getLocalCache();
 
@@ -235,6 +258,10 @@ public interface AnalysisEngineController extends ControllerLifecycle {
   
   public void addUimaObject(String objectName ) throws Exception;
   
+  public void setErrorHandlerChain(ErrorHandlerChain ehc);
+
+  public ResourceSpecifier getResourceSpecifier();
+
 }
   
   
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/BaseAnalysisEngineController.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/BaseAnalysisEngineController.java
index 1bf2e56..0f3e618 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/BaseAnalysisEngineController.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/BaseAnalysisEngineController.java
@@ -19,7 +19,6 @@
 
 package org.apache.uima.aae.controller;
 
-import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.lang.management.ManagementFactory;
 import java.lang.management.OperatingSystemMXBean;
@@ -52,11 +51,11 @@ import org.apache.uima.aae.EECasManager_impl;
 import org.apache.uima.aae.InProcessCache;
 import org.apache.uima.aae.InProcessCache.CacheEntry;
 import org.apache.uima.aae.InputChannel;
+import org.apache.uima.aae.InputChannel.ChannelType;
 import org.apache.uima.aae.OutputChannel;
 import org.apache.uima.aae.UIDGenerator;
 import org.apache.uima.aae.UIMAEE_Constants;
 import org.apache.uima.aae.UimaASApplicationEvent.EventTrigger;
-import org.apache.uima.aae.UimaAsContext;
 import org.apache.uima.aae.UimaAsVersion;
 import org.apache.uima.aae.UimaClassFactory;
 import org.apache.uima.aae.UimaEEAdminContext;
@@ -69,7 +68,6 @@ import org.apache.uima.aae.error.ErrorContext;
 import org.apache.uima.aae.error.ErrorHandler;
 import org.apache.uima.aae.error.ErrorHandlerChain;
 import org.apache.uima.aae.error.ForcedMessageTimeoutException;
-import org.apache.uima.aae.error.ServiceShutdownException;
 import org.apache.uima.aae.error.UimaAsUncaughtExceptionHandler;
 import org.apache.uima.aae.error.handler.ProcessCasErrorHandler;
 import org.apache.uima.aae.jmx.JmxManagement;
@@ -83,15 +81,16 @@ import org.apache.uima.aae.monitor.MonitorBaseImpl;
 import org.apache.uima.aae.monitor.statistics.LongNumericStatistic;
 import org.apache.uima.aae.monitor.statistics.Statistic;
 import org.apache.uima.aae.monitor.statistics.Statistics;
-import org.apache.uima.aae.spi.transport.UimaMessage;
 import org.apache.uima.aae.spi.transport.UimaMessageListener;
 import org.apache.uima.aae.spi.transport.UimaTransport;
-import org.apache.uima.aae.spi.transport.vm.VmTransport;
 import org.apache.uima.analysis_engine.AnalysisEngine;
 import org.apache.uima.analysis_engine.AnalysisEngineDescription;
 import org.apache.uima.analysis_engine.impl.AnalysisEngineManagementImpl;
 import org.apache.uima.analysis_engine.metadata.AnalysisEngineMetaData;
 import org.apache.uima.analysis_engine.metadata.SofaMapping;
+import org.apache.uima.as.client.DirectInputChannel;
+import org.apache.uima.as.client.Listener;
+import org.apache.uima.as.client.Listener.Type;
 import org.apache.uima.cas.CAS;
 import org.apache.uima.collection.CollectionReaderDescription;
 import org.apache.uima.resource.PearSpecifier;
@@ -104,10 +103,16 @@ import org.apache.uima.util.Level;
 
 public abstract class BaseAnalysisEngineController extends Resource_ImplBase implements
         AnalysisEngineController, EventSubscriber {
-  private static final Class CLASS_NAME = BaseAnalysisEngineController.class;
+  public static enum ENDPOINT_TYPE {
+	JMS,
+	DIRECT
+  };
+  private static final Class<?> CLASS_NAME = BaseAnalysisEngineController.class;
   private static final String JMS_PROVIDER_HOME = "ACTIVEMQ_HOME";
-  public static enum ServiceState { INITIALIZING, RUNNING, DISABLED, STOPPING, FAILED };
+  public enum ServiceState { INITIALIZING, RUNNING, DISABLED, STOPPING, FAILED };
   public static final boolean NO_RECOVERY = true;
+  public static final ENDPOINT_TYPE DEFAULT_OUTPUTCHANNEL_TYPE = ENDPOINT_TYPE.JMS;
+
   // Semaphore use only when quiesceAndStop is called
   // When the cache becomes empty the semaphore is released.
   private Semaphore quiesceSemaphore = new Semaphore(0);
@@ -148,9 +153,10 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
   protected long errorCount = 0;
 
-  protected List<InputChannel> inputChannelList = new ArrayList<InputChannel>();
+//  protected List<InputChannel> inputChannelList = new ArrayList<InputChannel>();
 
-  protected ConcurrentHashMap<String, InputChannel> inputChannelMap = new ConcurrentHashMap<String, InputChannel>();
+  protected ConcurrentHashMap<ENDPOINT_TYPE, InputChannel> inputChannelMap = 
+		  new ConcurrentHashMap<>();
 
   private UimaEEAdminContext adminContext;
 
@@ -219,6 +225,9 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
   
   private Object lock = new Object();
 
+  protected Map<String, OutputChannel> outputChannelMap = 
+		  new HashMap<String, OutputChannel>();
+ 
   // Local cache for this controller only. This cache stores state of
   // each CAS. The actual CAS is still stored in the global cache. The
   // local cache is used to determine when each CAS can be removed as
@@ -260,6 +269,8 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
   
   private String serviceName=null;
   
+  private String serviceId="";
+  
   protected UimaContext uimaContext=null;
   
   public abstract void dumpState(StringBuffer buffer, String lbl1);
@@ -297,7 +308,10 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
           int aComponentCasPoolSize, long anInitialCasHeapSize, String anEndpointName,
           String aDescriptor, AsynchAECasManager aCasManager, InProcessCache anInProcessCache,
           Map aDestinationMap, JmxManagement aJmxManagement,boolean disableJCasCache) throws Exception {
-    casManager = aCasManager;
+    
+	System.out.println("C'tor Called Descriptor:"+aDescriptor);
+
+	casManager = aCasManager;
     inProcessCache = anInProcessCache;
     localCache = new LocalCache(this);
     aeDescriptor = aDescriptor;
@@ -456,11 +470,25 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     }
 
     // Register InProcessCache with JMX under the top level component
-    if (inProcessCache != null && isTopLevelComponent()) {
-      inProcessCache.setName(jmxManagement.getJmxDomain() + jmxContext + ",name="
-              + inProcessCache.getName());
-      ObjectName on = new ObjectName(inProcessCache.getName());
-      jmxManagement.registerMBean(inProcessCache, on);
+//    if (inProcessCache != null && isTopLevelComponent()) {
+//      inProcessCache.setName(jmxManagement.getJmxDomain() + jmxContext + ",name="
+//              + inProcessCache.getName());
+//      ObjectName on = new ObjectName(inProcessCache.getName());
+//      jmxManagement.registerMBean(inProcessCache, on);
+//    }
+    if (inProcessCache != null && !inProcessCache.isRegisteredWithJMX() && isTopLevelComponent()) {
+  	  
+    	if ( jmxManagement.isRegistered(new ObjectName("org.apache.uima:type=ee.jms.services,s=Test Aggregate TAE Uima EE Service,p0=Test Aggregate TAE Components,name=InProcessCache"))) {
+    		System.out.println("!!!!!!!!!!!!!!!!!! InProcessCache Already Registered"); 
+    	} else {
+    		System.out.println(">>>>>>>>> Registering InProcessCache with JMX");
+        	inProcessCache.setName(jmxManagement.getJmxDomain() + jmxContext + ",name="
+                    + inProcessCache.getName());
+            System.out.println("-------->>>>>>>>> InProcessCache Object Name:"+inProcessCache.getName());
+            ObjectName on = new ObjectName(inProcessCache.getName());
+            jmxManagement.registerMBean(inProcessCache, on);
+            inProcessCache.setRegisteredWithJMX();
+    	}
     }
     initializeServiceStats();
 
@@ -582,7 +610,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
   public AnalysisEngineController getParentController() {
     return parentController;
   }
-
+/*
   public UimaTransport getTransport(String aKey) throws Exception {
     return getTransport(null, aKey);
   }
@@ -609,10 +637,11 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
     return transport;
   }
-
+*/
   /**
    * Initializes transport used for internal messaging between collocated Uima AS services.
    */
+  /*
   public void initializeVMTransport(int parentControllerReplyConsumerCount) throws Exception {
     // If this controller is an Aggregate Controller, force delegates to initialize
     // their internal transports.
@@ -686,7 +715,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     }
 
   }
-
+*/
   public synchronized UimaMessageListener getUimaMessageListener(String aDelegateKey) {
     return messageListeners.get(aDelegateKey);
   }
@@ -818,6 +847,10 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
   public void addUimaObject(String objectName ) throws Exception {
 	  jmxManagement.addObject(objectName);
   }
+  public void addOutputChannel(OutputChannel outputChannel) throws Exception {
+	  outputChannelMap.put( outputChannel.getType().name(), outputChannel);  
+//	  this.outputChannel = outputChannel;
+  }
   /**
    * Register a component with a given name with JMX MBeanServer
    * 
@@ -867,6 +900,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
     registerWithAgent(servicePerformance, name);
     servicePerformance.setIdleTime(System.nanoTime());
+    /*
     ServiceInfo serviceInfo = null;
     if (remote) {
       serviceInfo = getInputChannel().getServiceInfo();
@@ -886,6 +920,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
         }
       }
     }
+    */
     ServiceInfo pServiceInfo = null;
 
     if (this instanceof PrimitiveAnalysisEngineController) {
@@ -906,8 +941,12 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
       }
 
       
+//      name = jmxManagement.getJmxDomain() + key_value_list + ",name=" + thisComponentName + "_"
+//              + serviceInfo.getLabel();
+      
       name = jmxManagement.getJmxDomain() + key_value_list + ",name=" + thisComponentName + "_"
-              + serviceInfo.getLabel();
+              + pServiceInfo.getLabel();
+
       if (!isTopLevelComponent()) {
         pServiceInfo.setBrokerURL("Embedded Broker");
       } else {
@@ -1092,7 +1131,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
   public void setInputChannel(InputChannel anInputChannel) throws Exception {
     inputChannel = anInputChannel;
-    inputChannelList.add(anInputChannel);
+//    inputChannelList.add(anInputChannel);
 
     inputChannelLatch.countDown();
     if (!registeredWithJMXServer) {
@@ -1102,6 +1141,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
   }
 
   public void addInputChannel(InputChannel anInputChannel) {
+	  /*
 	  inputChannelLatch.countDown();
     if (!inputChannelMap.containsKey(anInputChannel.getInputQueueName())) {
       inputChannelMap.put(anInputChannel.getInputQueueName(), anInputChannel);
@@ -1109,18 +1149,65 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
         inputChannelList.add(anInputChannel);
       }
     }
-  }
+    */
+    if (!inputChannelMap.containsKey(anInputChannel.getType())) {
+  	  inputChannelMap.put(anInputChannel.getType(),anInputChannel);
+        inputChannelLatch.countDown();
+        if (!registeredWithJMXServer) {
+          registerServiceWithJMX(jmxContext, false);
+          registeredWithJMXServer = true;
+
+        }
+
+    }
+    if ( anInputChannel.getChannelType().equals(ChannelType.REQUEST_REPLY)) {
+  	  inputChannel = anInputChannel;
+    }
 
+  }
   public InputChannel getInputChannel() {
-    try {
+	  return inputChannel;
+  }
+  public InputChannel getInputChannel(ENDPOINT_TYPE type) {
+    /*
+	  try {
       inputChannelLatch.await();
 
     } catch (Exception e) {
     }
 
     return inputChannel;
+    */
+    return inputChannelMap.get(type);
+  }
+  public InputChannel getInputChannel(Class<?> clz) {
+	  return inputChannel;
+  }
+  public List<Listener> getAllListeners() {
+	  List<Listener> listeners = new ArrayList<Listener>();
+		if ( getInputChannel(ENDPOINT_TYPE.JMS) != null ) {
+			listeners.addAll(getInputChannel(ENDPOINT_TYPE.JMS).getListeners());
+		}   
+		if ( getInputChannel(ENDPOINT_TYPE.DIRECT) != null ) {
+			listeners.addAll(getInputChannel(ENDPOINT_TYPE.DIRECT).getListeners());
+		}
+		return listeners;
+  }
+  public void setDirectInputChannel(DirectInputChannel anInputChannel) throws Exception {
+ //   inputChannel = anInputChannel;
+    
+  //  inputChannelList.add(anInputChannel);
+	addInputChannel(anInputChannel);
+	   
+    inputChannelLatch.countDown();
+    if (!registeredWithJMXServer) {
+      registeredWithJMXServer = true;
+      registerServiceWithJMX(jmxContext, false);
+    }
+  }
+  public void setJmsInputChannel(InputChannel anInputChannel) throws Exception {
+	   addInputChannel(anInputChannel);
   }
-
   public void dropCAS(CAS aCAS) {
     if (aCAS != null) {
       // Check if this method was called while another thread is stopping the service.
@@ -1180,7 +1267,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
           if (!isStopped()) {
             Endpoint endpoint = (Endpoint) anErrorContext.get(AsynchAEMessage.Endpoint);
             if ( endpoint != null && !"WarmupDelegate".equals(endpoint.getDelegateKey() ) ) {
-              getOutputChannel().sendReply((Throwable) anErrorContext.get(ErrorContext.THROWABLE_ERROR), 
+              getOutputChannel(endpoint).sendReply((Throwable) anErrorContext.get(ErrorContext.THROWABLE_ERROR), 
                       casReferenceId, parentCasReferenceId,
                       endpoint, AsynchAEMessage.Process);
             }
@@ -1346,7 +1433,39 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     }
 
   }
-
+  public void dropCASFromLocal(String aCasReferenceId) {
+	  try {
+		    CacheEntry entry = null ;
+		    if ( inProcessCache.entryExists(aCasReferenceId)) {
+		        entry = inProcessCache.getCacheEntryForCAS(aCasReferenceId);
+		    }
+		    if ( entry != null ) {
+			      // Release semaphore which throttles ingestion of CASes from service
+			      // input queue.
+		        Semaphore semaphore=null;
+		        if ( !isPrimitive() && (semaphore = entry.getThreadCompletionSemaphore()) != null ) {
+		          semaphore.release();
+		        }
+		        if (localCache.containsKey(aCasReferenceId)) {
+		            try {
+		              localCache.lookupEntry(aCasReferenceId).setDropped(true);
+		            } catch (Exception e) {
+		          	    if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
+		          	        UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(), "dropCASFromLocal",
+		          	                UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_parent_cas_notin_cache__INFO",
+		          	                new Object[] {getComponentName(), aCasReferenceId  });
+		          	      }
+		            }
+		            localCache.remove(aCasReferenceId);
+		          }
+		        // Remove stats from the map maintaining CAS specific stats
+		        if (perCasStatistics.containsKey(aCasReferenceId)) {
+		          perCasStatistics.remove(aCasReferenceId);
+		        }
+		    }
+	  } catch( Exception e) {
+      }
+  }
   public void forceTimeoutOnPendingCases(String key) {
     Delegate delegate = ((AggregateAnalysisEngineController) this).lookupDelegate(key);
     // Cancel the delegate timer. No more responses are expected
@@ -1546,15 +1665,33 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
     return false;
   }
-
+/*
   public OutputChannel getOutputChannel() {
     return outputChannel;
   }
-
+*/
   public void setOutputChannel(OutputChannel outputChannel) throws Exception {
-    this.outputChannel = outputChannel;
+//    this.outputChannel = outputChannel;
+    outputChannelMap.put(outputChannel.getType().name(), outputChannel);
+
+  }
+  public OutputChannel getOutputChannel(ENDPOINT_TYPE et) {
+	return outputChannelMap.get(et.name());
+//	  return outputChannel;
+  }
+  public OutputChannel getOutputChannel(Endpoint anEndpoint) {
+      ENDPOINT_TYPE et  = ENDPOINT_TYPE.DIRECT;
+      
+      if ( anEndpoint.getServerURI().indexOf("http") > -1 ||
+    		  anEndpoint.getServerURI().indexOf("tcp") > -1 ) {
+    	 et = ENDPOINT_TYPE.JMS;
+      }
+//System.out.println("............... OutputChannel type:"+et.name());
+      OutputChannel oc = outputChannelMap.get(et.name());
+	return oc;
   }
 
+
   public AsynchAECasManager getCasManagerWrapper() {
     return casManager;
   }
@@ -1567,10 +1704,15 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     return inProcessCache;
   }
 
-  protected ResourceSpecifier getResourceSpecifier() {
+  public ResourceSpecifier getResourceSpecifier() {
     return resourceSpecifier;
   }
-
+  public void setServiceId(String sid) {
+	  serviceId = sid;
+  }
+  public String getServiceId() {
+	  return serviceId;
+  }
   public String getName() {
     return endpointName;
   }
@@ -1794,12 +1936,12 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 
   public String getBrokerURL() {
     // Wait until the connection factory is injected by Spring
-    while (System.getProperty("BrokerURI") == null) {
-      try {
-        Thread.sleep(50);
-      } catch (InterruptedException ex) {
-      }
-    }
+//    while (System.getProperty("BrokerURI") == null) {
+//      try {
+//        Thread.sleep(50);
+//      } catch (InterruptedException ex) {
+//      }
+//    }
     return System.getProperty("BrokerURI");
   }
 
@@ -1909,12 +2051,11 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     /*
      * Send an exception to the client if this is a top level service
      */
-    if (cause != null && aCasReferenceId != null && getOutputChannel() != null
-            && isTopLevelComponent()) {
+    if (cause != null && aCasReferenceId != null && isTopLevelComponent()) {
       Endpoint clientEndpoint = null;
       if ((clientEndpoint = getClientEndpoint()) != null) {
         try {
-          getOutputChannel().sendReply(cause, aCasReferenceId, null, clientEndpoint,
+          getOutputChannel(clientEndpoint).sendReply(cause, aCasReferenceId, null, clientEndpoint,
                   clientEndpoint.getCommand());
         } catch (Exception e) {
           if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.WARNING)) {
@@ -1937,11 +2078,16 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
               UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_stop__INFO",
               new Object[] { getComponentName() });
     }
-    if (getOutputChannel() != null) {
-      getOutputChannel().cancelTimers();
+    for( Entry<String, OutputChannel> oce : outputChannelMap.entrySet()) {
+    	if ( oce.getValue() != null ) {
+    		oce.getValue().cancelTimers();
+    	}
     }
+//    if (getOutputChannel() != null) {
+//      getOutputChannel().cancelTimers();
+//    }
     
-    if (this instanceof PrimitiveAnalysisEngineController) {
+    if (isPrimitive()) {
       getControllerLatch().release();
       // Stops the input channel of this service
       stopInputChannels(InputChannel.CloseAllChannels, shutdownNow);
@@ -1986,8 +2132,17 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
       adminContext = null;
     } else {
       // Stop output channel
-      getOutputChannel().stop();
-      
+        for( Entry<String, OutputChannel> oce : outputChannelMap.entrySet()) {
+        	if ( oce.getValue() != null ) {
+        	    oce.getValue().stop();
+        	}
+          }
+      //getOutputChannel().stop();
+        try {
+            // Remove all MBeans registered by this service
+            jmxManagement.destroy();
+          } catch (Exception e) {
+          }
       try {
         getInProcessCache().destroy();
       } catch (Exception e) {
@@ -2002,9 +2157,9 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     if (statsMap != null) {
       statsMap.clear();
     }
-    if (inputChannelList != null) {
-      inputChannelList.clear();
-    }
+//    if (inputChannelList != null) {
+//      inputChannelList.clear();
+//    }
     //inputChannel = null;
 
     if (serviceErrorMap != null) {
@@ -2031,9 +2186,9 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     if (threadStateMap != null) {
       threadStateMap.clear();
     }
-    if (inputChannelMap != null) {
-      inputChannelMap.clear();
-    }
+//    if (inputChannelMap != null) {
+//      inputChannelMap.clear();
+//    }
     if (controllerListeners != null) {
       controllerListeners.clear();
     }
@@ -2082,10 +2237,18 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
       // we proceed with the shutdown of delegates and finally of the top level service.
       if (isTopLevelComponent()) {
           getInputChannel().setTerminating();
-
+          try {
+              // Stops all input channels of this service, but keep temp reply queue input channels open
+              // to process replies.
+              stopReceivingCASes(false);  // dont kill listeners on temp queues. The remotes may send replies
+ 	  
+          } catch( Exception e) {
+        	  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, getClass().getName(),
+                      "quiesceAndStop", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
+                      "UIMAEE_exception__WARNING", e);
+          }
         // Stops all input channels of this service, but keep temp reply queue input channels open
         // to process replies.
-        stopReceivingCASes(false);  // dont kill listeners on temp queues. The remotes may send replies
         if ( this instanceof PrimitiveAnalysisEngineController_impl &&
         		((PrimitiveAnalysisEngineController_impl)this).aeInstancePool != null ) {
         	//	Since we are quiescing, destroy all AEs that are in AE pool. Those that
@@ -2120,8 +2283,11 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
           stopReceivingCASes(true);
           stopInputChannels(InputChannel.InputChannels, true);  
           System.out.println("UIMA-AS Service is Stopping, All CASes Have Been Processed");
-        } catch( InterruptedException e) {
-          
+        } catch( Exception e) {
+           	UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, getClass().getName(),
+                    "quiesceAndStop", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
+                    "UIMAEE_exception__WARNING", e);
+
         }
         stop(true); 
       }
@@ -2163,16 +2329,26 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
       ((BaseAnalysisEngineController) parentController).stop();
     } else if (!isStopped()) {
       stopDelegateTimers();
-      getOutputChannel().cancelTimers();
-      InputChannel iC = getInputChannel(endpointName);
+      for( Entry<String, OutputChannel> oce : outputChannelMap.entrySet()) {
+      	if ( oce.getValue() != null ) {
+      		oce.getValue().cancelTimers();
+      	}
+      }
+     // getOutputChannel().cancelTimers();
+      InputChannel iC = getInputChannel();
       if ( iC != null) {
           iC.setTerminating();
       }
-      // Stop the inflow of new input CASes
-      stopInputChannel(true);  // shutdownNow
-       if ( iC != null ) {
-        iC.terminate();
+      try {
+    	  // Stop the inflow of new input CASes
+          stopInputChannel(true);  // shutdownNow
+           if ( iC != null ) {
+            iC.terminate();
+          }
+      } catch (Exception e) {
+    	  
       }
+
       stopCasMultipliers();
       stopTransportLayer();
       if (cause != null && aCasReferenceId != null) {
@@ -2247,7 +2423,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
             Endpoint freeCasNotificationEndpoint = casEntry.getFreeCasNotificationEndpoint();
             if (freeCasNotificationEndpoint != null) {
               freeCasNotificationEndpoint.setCommand(AsynchAEMessage.Stop);
-              getOutputChannel().sendRequest(AsynchAEMessage.Stop, aCasReferenceId,
+              getOutputChannel(freeCasNotificationEndpoint).sendRequest(AsynchAEMessage.Stop, aCasReferenceId,
                       freeCasNotificationEndpoint);
             }
             if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
@@ -2298,7 +2474,14 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
    * Stops a listener on the main input channel
    * @param shutdownNow stop
    */
-  protected void stopInputChannel(boolean shutdownNow) {
+  protected void stopInputChannel(boolean shutdownNow) throws Exception {
+	  for( Listener listener : inputChannel.getListeners()) {
+		  if ( listener.getType().equals(Type.GetMeta) || 
+			   listener.getType().equals(Type.ProcessCAS)) {
+			  inputChannel.disconnectListenerFromQueue(listener);
+		  }
+	  }
+	  /*
     InputChannel iC = getInputChannel(endpointName);
     if (iC != null && !iC.isStopped()) {
       try {
@@ -2312,15 +2495,27 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
         }
       }
     }
+    */
   }
   private void setInputChannelForNoRecovery() {
+	  /*
 	  if ( inputChannelMap.size() > 0 ) {
 		  InputChannel iC = getInputChannel();
 		  iC.setTerminating();
 	  }
+	  */
+	  inputChannel.setTerminating();
   }
-  protected void stopReceivingCASes(boolean stopAllListeners)  {
-	  
+  private boolean isTempListener( Listener listener ) {
+	  return listener.getType().equals(Type.Reply) || listener.getType().equals(Type.FreeCAS);
+  }
+  protected void stopReceivingCASes(boolean stopAllListeners) throws Exception {
+	  for( Listener listener : inputChannel.getListeners()) {
+		  if ( stopAllListeners || !isTempListener(listener) ) {
+			  inputChannel.disconnectListenerFromQueue(listener);
+		  }
+	  }
+/*
 	    InputChannel iC = null;
 	    setInputChannelForNoRecovery();
 	    Iterator<String> it = inputChannelMap.keySet().iterator();
@@ -2357,10 +2552,21 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 	        }
 	      }
 	    }
-	  
+	  */
   }
   protected void stopInputChannels( int channelsToStop, boolean shutdownNow) {   //, boolean norecovery) {
-	    InputChannel iC = null;
+	  try {
+		  for( Entry<BaseAnalysisEngineController.ENDPOINT_TYPE, InputChannel> ic : inputChannelMap.entrySet()) {
+			  ic.getValue().stop(shutdownNow);
+		  }
+		 // inputChannel.stop(shutdownNow);
+	  } catch( Exception e ) {
+		  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.WARNING, getClass().getName(),
+                  "stopInputChannels", UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE,
+                  "UIMAEE_exception__WARNING", e); 
+	  }
+	  /*
+	  InputChannel iC = null;
 	    setInputChannelForNoRecovery();
 	    Iterator it = inputChannelMap.keySet().iterator();
 	    int i = 1;
@@ -2401,7 +2607,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
 	        }
 	      }
 	    }
-	  
+	  */
   }
   /**
    * Aggregates have more than one Listener channel. This method stops all configured input channels
@@ -2426,7 +2632,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     }
     return null;
   }
-
+/*
   public InputChannel getInputChannel(String anEndpointName) {
 
     for (int i = 0; inputChannelList != null && i < inputChannelList.size(); i++) {
@@ -2437,7 +2643,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     }
     return null;
   }
-
+*/
 //  public InputChannel getReplyInputChannel(String aDelegateKey) {
   public InputChannel getReplyInputChannel(String aDestination) {
 	  InputChannel IC = null;
@@ -2916,6 +3122,7 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
     	  if ( anEndpoint.getServerURI().equals("vm://localhost?broker.persistent=false")) {
     		  anEndpoint.setRemote(true);
     	  }
+    	  /*
         if (!anEndpoint.isRemote()) {
           ByteArrayOutputStream bos = new ByteArrayOutputStream();
           try {
@@ -2946,8 +3153,12 @@ public abstract class BaseAnalysisEngineController extends Resource_ImplBase imp
         } else {
           getOutputChannel().sendReply(metadata, anEndpoint, true);
         }
+        */
+          getOutputChannel(anEndpoint).sendReply(metadata, anEndpoint, true);
+
       }
     } catch (Exception e) {
+    	e.printStackTrace();
       HashMap map = new HashMap();
       map.put(AsynchAEMessage.Endpoint, anEndpoint);
       map.put(AsynchAEMessage.MessageType, Integer.valueOf(AsynchAEMessage.Request));
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/ControllerStatusListener.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/ControllerStatusListener.java
new file mode 100644
index 0000000..d099200
--- /dev/null
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/ControllerStatusListener.java
@@ -0,0 +1,35 @@
+package org.apache.uima.aae.controller;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.uima.aae.UimaASApplicationEvent.EventTrigger;
+
+public class ControllerStatusListener implements ControllerCallbackListener  {
+	CountDownLatch latch;
+	public ControllerStatusListener(CountDownLatch latch) {
+		this.latch = latch;
+	}
+	public void notifyOnTermination(String aMessage, EventTrigger cause) {
+	}
+
+	public void notifyOnInitializationFailure(AnalysisEngineController aController, Exception e) {
+	}
+
+	public void notifyOnInitializationSuccess(AnalysisEngineController aController) {
+		System.out.println("------- Controller:"+aController.getName()+" Initialized");
+		latch.countDown();
+	}
+
+	public void notifyOnInitializationFailure(Exception e) {
+	}
+
+	public void notifyOnInitializationSuccess() {
+	}
+
+	public void notifyOnReconnecting(String aMessage) {
+	}
+
+	public void notifyOnReconnectionSuccess() {
+	}
+
+}
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/DelegateEndpoint.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/DelegateEndpoint.java
new file mode 100644
index 0000000..22b9203
--- /dev/null
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/DelegateEndpoint.java
@@ -0,0 +1,127 @@
+package org.apache.uima.aae.controller;
+
+import org.apache.uima.cas.SerialFormat;
+import org.apache.uima.resource.ResourceSpecifier;
+
+public class DelegateEndpoint {
+	public class Builder {
+		private Endpoint_impl e1 = new Endpoint_impl();
+		private String key;
+		private String uri;
+		private String endpointName;
+		private Object replyTo;
+		private String replyDestinatioName;
+		private int getMetaTimeout=0;
+		private int processTimeout=0;
+		private String getMetaActionOnError;
+		private String processActionOnError;
+		private int getMetaMaxRetries=0;
+		private int processsMaxRetries=0;
+		private boolean isTempReplyDestination;
+		private int collectionProcessCompleteTimeout=0;
+		private String serializer=SerialFormat.XMI.getDefaultFileExtension();
+		private int scaleout=1;
+		private String descriptor;
+		private ResourceSpecifier rs;
+		private boolean isRemote;
+		
+		public Builder setRemote(boolean isRemote) {
+			this.isRemote = isRemote;
+			if ( !isRemote ) {
+				setServerURI("java");
+			}
+			return this;
+		}
+		public Builder withResourceSpecifier(ResourceSpecifier rs) {
+			this.rs = rs;
+			setDescriptor(rs.getSourceUrlString());
+			return this;
+		}
+		public Builder setTempDestination(boolean isTempReplyDestination) {
+			this.isTempReplyDestination = isTempReplyDestination;
+			return this;
+		}
+		public Builder setDescriptor(String descriptor) {
+			this.descriptor = descriptor;
+			return this;
+		}
+
+		public Builder setScaleout(int scaleout) {
+			this.scaleout = scaleout;
+			return this;
+		}
+
+		public Builder withDelegateKey(String key) {
+			this.key = key;
+			return this;
+		}
+
+		public Builder setServerURI(String uri) {
+			this.uri = uri;
+			return this;
+		}
+
+		public Builder withEndpointName(String endpointName) {
+			this.endpointName = endpointName;
+			return this;
+		}
+
+		public Builder setReplyToDestination(Object replyTo) {
+			this.replyTo = replyTo;
+			return this;
+		}
+
+		public Builder setReplyDestinationName(String replyDestinatioName) {
+			this.replyDestinatioName = replyDestinatioName;
+			return this;
+		}
+
+		public Builder setGetMetaErrorHandlingParams(int timeout, int maxRetries, String action) {
+			this.getMetaTimeout = timeout;
+			this.getMetaActionOnError = action;
+			this.getMetaMaxRetries = maxRetries;
+			return this;
+		}
+
+		public Builder setProcessErrorHandlingParams(int timeout, int maxRetries, String action) {
+			this.processTimeout = timeout;
+			this.processActionOnError = action;
+			this.processsMaxRetries = maxRetries;
+			return this;
+		}
+
+		public Builder setCollectionProcessCompleteTimeout(int timeout) {
+			this.collectionProcessCompleteTimeout = timeout;
+			return this;
+		}
+
+		public Builder setSerializer(String serializer) {
+			this.serializer = serializer;
+			return this;
+		}
+
+		public Endpoint_impl build() {
+			e1.setDelegateKey(key);
+			e1.setServerURI(uri);
+			e1.setEndpoint(endpointName);
+			e1.setDescriptor(descriptor);
+			e1.setConcurrentRequestConsumers(scaleout);
+			e1.setDestination(null); 
+			e1.setReplyToEndpoint(replyDestinatioName);
+			e1.setMetadataRequestTimeout(getMetaTimeout);
+			e1.setProcessRequestTimeout(processTimeout);
+			e1.setTempReplyDestination(isTempReplyDestination);
+			e1.setCollectionProcessCompleteTimeout(0);
+			e1.setSerializer(SerialFormat.XMI.getDefaultFileExtension());
+			e1.setRemote(isRemote);
+//			if ( uri != null && uri.length() > 0 && (uri.contains("tcp:") || uri.contains("http:") )) {
+//				e1.setRemote(true);
+//			} else {
+//				e1.setRemote(false);
+//			}
+			e1.setResourceSpecifier(rs);
+			return e1;
+		}
+
+	}
+}
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint.java
index 21811b6..b32d407 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint.java
@@ -31,8 +31,16 @@ public interface Endpoint {
 
   public static final int DISABLED = 3;
 
+  public boolean isJavaRemote();
+  
+  public void setJavaRemote();
+  
   public int getMetadataRequestTimeout();
 
+  public void setDisableJCasCache(boolean disableOrEnable);
+  
+  public void setCollectionProcessCompleteTimeout(int cpcTimeout);
+  
   public void setController(AnalysisEngineController aController);
 
   public void startCheckpointTimer();
@@ -47,6 +55,10 @@ public interface Endpoint {
 
   public void setReplyEndpoint(boolean tORf);
 
+  public void setReplyDestination(Object replyDestination);
+
+  public Object getReplyDestination();
+
   public boolean isReplyEndpoint();
 
   public void setProcessRequestTimeout(int processRequestTimeout);
@@ -129,6 +141,10 @@ public interface Endpoint {
 
   public void setDestination(Object aDestination);
 
+  public void setGetMetaDestination(Object aDestination);
+
+  public Object getMetaDestination();
+
   public void setCommand(int aCommand);
 
   public int getCommand();
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint_impl.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint_impl.java
index 25875cd..853a20b 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint_impl.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/Endpoint_impl.java
@@ -27,14 +27,21 @@ import org.apache.uima.aae.jmx.ServiceInfo;
 import org.apache.uima.aae.message.AsynchAEMessage;
 import org.apache.uima.cas.SerialFormat;
 import org.apache.uima.cas.impl.TypeSystemImpl;
+import org.apache.uima.resource.ResourceSpecifier;
 
 public class Endpoint_impl implements Endpoint, Cloneable {
-  private static final Class CLASS_NAME = Endpoint_impl.class;
+  private static final Class<?> CLASS_NAME = Endpoint_impl.class;
 
+  private volatile boolean javaRemote=false;
+  
   private volatile Object destination = null;
 
   private String endpoint;  // is the queue name (only)
 
+  private volatile Object getMetaDestination = null;
+  
+  private volatile Object replyDestination = null;
+
   private String serverURI;
 
   private volatile boolean initialized;
@@ -45,11 +52,11 @@ public class Endpoint_impl implements Endpoint, Cloneable {
 
   private volatile boolean waitingForResponse;
 
-  private int metadataRequestTimeout;
+  private int metadataRequestTimeout=0;
 
-  private int processRequestTimeout;
+  private int processRequestTimeout=0;
 
-  private int collectionProcessCompleteTimeout;
+  private int collectionProcessCompleteTimeout=0;
 
   private volatile boolean isRemote;
 
@@ -57,7 +64,7 @@ public class Endpoint_impl implements Endpoint, Cloneable {
 
   private SerialFormat serialFormat = null;
   
-  private String serializer = "xmi";  // spring bean interface
+  private String serializer = SerialFormat.XMI.getDefaultFileExtension(); 
 
   private volatile boolean finalEndpoint;
 
@@ -121,6 +128,23 @@ public class Endpoint_impl implements Endpoint, Cloneable {
   
   private volatile boolean disableJCasCache;
   
+ private ResourceSpecifier resourceSpecifier;
+  
+  public void setJavaRemote() {
+	 javaRemote = true;
+  }
+  public boolean isJavaRemote() {
+	  return javaRemote;
+  }
+  public void setReplyDestination(Object replyDestination) {
+	  this.replyDestination = replyDestination;
+  }
+  public Object getReplyDestination() {
+	  return replyDestination;
+  }
+  public void setResourceSpecifier(ResourceSpecifier rs ) {
+	  this.resourceSpecifier = rs;
+  }
   public boolean isDisableJCasCache() {
     return disableJCasCache;
   }
@@ -319,11 +343,6 @@ public class Endpoint_impl implements Endpoint, Cloneable {
 
   public void setServerURI(String aServerURI) {
     this.serverURI = aServerURI;
-    if ( aServerURI != null && ( aServerURI.startsWith("vm:") == true && !aServerURI.equals("vm://localhost?broker.persistent=false")  ) ){
-      setRemote(false);
-    } else {
-      setRemote(true);
-    }
   }
 
   public void setWaitingForResponse(boolean isWaiting) {
@@ -436,7 +455,12 @@ public class Endpoint_impl implements Endpoint, Cloneable {
     isRemote = aRemote;
 
   }
-
+  public void setGetMetaDestination(Object aDestination) {
+	  getMetaDestination = aDestination;
+  }
+  public Object getMetaDestination() {
+	  return getMetaDestination;
+  }
   public String getDescriptor() {
     return descriptor;
   }
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/LocalCache.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/LocalCache.java
index 08cf898..2d71314 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/LocalCache.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/LocalCache.java
@@ -71,7 +71,7 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
       if (entry != null && entry.isSubordinate()) {
         // recursively call each parent until we get to the top of the
         // Cas hierarchy
-        parentCasReferenceId = lookupInputCasReferenceId(entry.getInputCasReferenceId());
+        parentCasReferenceId = lookupInputCasReferenceId(entry.getParentCasReferenceId());
       } else {
         return aCasReferenceId;
       }
@@ -85,14 +85,14 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
       // recursively call each parent until we get to the top of the
       // Cas hierarchy
       parentCasReferenceId = lookupInputCasReferenceId((CasStateEntry) get(entry
-              .getInputCasReferenceId()));
+              .getParentCasReferenceId()));
     } else {
       return entry.getCasReferenceId();
     }
     return parentCasReferenceId;
   }
   public void dumpContents() {
-    dumpContents(false);
+    dumpContents(true);
   }
   public synchronized void dumpContents(boolean dump2Stdout) {
     int count = 0;
@@ -109,7 +109,7 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
         if (casStateEntry.isSubordinate()) {
           sb.append(entry.getKey() + " Number Of Child CASes In Play:"
                   + casStateEntry.getSubordinateCasInPlayCount() + " Parent CAS id:"
-                  + casStateEntry.getInputCasReferenceId());
+                  + casStateEntry.getParentCasReferenceId());
         } else {
           sb.append(entry.getKey() + " *** Input CAS. Number Of Child CASes In Play:"
                   + casStateEntry.getSubordinateCasInPlayCount());
@@ -179,7 +179,7 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
     CasStateEntry casStateEntry = lookupEntry(casReferenceId);
     if (casStateEntry.isSubordinate()) {
       // Recurse until the top CAS reference Id is found
-      return getTopCasAncestor(casStateEntry.getInputCasReferenceId());
+      return getTopCasAncestor(casStateEntry.getParentCasReferenceId());
     }
     // Return the top ancestor CAS id
     return casStateEntry;
@@ -187,7 +187,12 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
 
   public static class CasStateEntry {
 	private String casReferenceId;
-
+	// id of a parent CAS 
+    private String parentCasReferenceId;
+    // stores the id of an input CAS sent by the client. This is used
+    // to identify client endpoint if the service is a CM.
+    private String inputCasReferenceId;
+    
     private volatile boolean waitingForChildren; // true if in FinalState and still has children in play
     
     private volatile boolean waitingForRealease;
@@ -212,8 +217,8 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
 
     private Object childCountMux = new Object();
 
-    private String inputCasReferenceId;
-
+    private long seqNo;
+    
     private int numberOfParallelDelegates = 1;
 
     private Delegate lastDelegate = null;
@@ -222,6 +227,9 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
 
     private Endpoint freeCasNotificationEndpoint;
 
+    // client endpoint where the input CAS will be returned
+    private Endpoint clientEndpoint;
+
     private volatile boolean deliveryToClientFailed;
     
     private String hostIpProcessingCAS;
@@ -249,6 +257,12 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
     
     protected Map<String, AEMetrics > casMetrics = new TreeMap<String, AEMetrics>();
     
+    public Endpoint getClientEndpoint() {
+    	return clientEndpoint;
+    }
+    public void setClientEndpoint(Endpoint ce) {
+    	clientEndpoint = ce;
+    }
     public boolean waitingForChildrenToFinish() {
     	return waitingForChildren;
     }
@@ -297,7 +311,12 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
 	public void setDeliveryToClientFailed() {
 		this.deliveryToClientFailed = true;
 	}
-
+	public void setSequenceNumber(long seq) {
+		this.seqNo = seq;
+	}
+	public long getSequenceNumber() {
+		return this.seqNo;
+	}
 	public boolean isDropped() {
       return dropped;
     }
@@ -330,15 +349,20 @@ public class LocalCache extends ConcurrentHashMap<String, LocalCache.CasStateEnt
       return casReferenceId;
     }
 
-    public String getInputCasReferenceId() {
-      return inputCasReferenceId;
+    public String getParentCasReferenceId() {
+      return parentCasReferenceId;
     }
 
-    public void setInputCasReferenceId(String anInputCasReferenceId) {
-      inputCasReferenceId = anInputCasReferenceId;
+    public void setParentCasReferenceId(String parentCasReferenceId) {
+      this.parentCasReferenceId = parentCasReferenceId;
       subordinateCAS = true;
     }
-
+    public String getInputCasReferenceId() {
+    	return inputCasReferenceId;
+    }
+    public void setInputCasReferenceId(String inputCasId) {
+    	inputCasReferenceId = inputCasId;
+    }
     public void setWaitingForRelease(boolean flag) {
       waitingForRealease = flag;
     }
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/PrimitiveAnalysisEngineController_impl.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/PrimitiveAnalysisEngineController_impl.java
index a83340e..9d285a4 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/PrimitiveAnalysisEngineController_impl.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/controller/PrimitiveAnalysisEngineController_impl.java
@@ -63,6 +63,7 @@ import org.apache.uima.analysis_engine.AnalysisEngineDescription;
 import org.apache.uima.analysis_engine.AnalysisEngineManagement;
 import org.apache.uima.analysis_engine.CasIterator;
 import org.apache.uima.analysis_engine.metadata.AnalysisEngineMetaData;
+import org.apache.uima.as.client.DirectMessageContext;
 import org.apache.uima.cas.CAS;
 import org.apache.uima.cas.impl.CASImpl;
 import org.apache.uima.collection.CollectionReaderDescription;
@@ -263,7 +264,7 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
     	    }
       }
       AnalysisEngine ae = UIMAFramework.produceAnalysisEngine(rSpecifier, paramsMap);
-   
+      System.out.println("------------ initializeAnalysisEngine()-Created instance of AE:"+getComponentName()+" Thread iD:"+Thread.currentThread().getId());
       super.addUimaObject(ae.getManagementInterface().getUniqueMBeanName());
       //  Call to produceAnalysisEngine() may take a long time to complete. While this
         //  method was executing, the service may have been stopped. Before continuing 
@@ -476,7 +477,7 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
           if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
             UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(), "postInitialize",
                   UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_initialized_controller__INFO",
-                  new Object[] { getComponentName() });
+                  new Object[] { getComponentName(), super.getBrokerURL() });
           }
           super.serviceInitialized = true;
         }
@@ -549,6 +550,7 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
                 "UIMAEE_cpc_all_cases_processed__FINEST", new Object[] { getComponentName() });
       }
       getServicePerformance().incrementAnalysisTime(super.getCpuTime() - start);
+      /*
       if (!anEndpoint.isRemote()) {
         UimaTransport transport = getTransport(anEndpoint.getEndpoint());
         UimaMessage message = transport.produceMessage(AsynchAEMessage.CollectionProcessComplete,
@@ -558,6 +560,8 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
       } else {
         getOutputChannel().sendReply(AsynchAEMessage.CollectionProcessComplete, anEndpoint, null, false);
       }
+*/
+      getOutputChannel(anEndpoint).sendReply(AsynchAEMessage.CollectionProcessComplete, anEndpoint, null, false);
 
       if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
         UIMAFramework.getLogger(CLASS_NAME).logrb(Level.FINE, getClass().getName(),
@@ -741,7 +745,14 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
             aem.getNumberOfCASesProcessed());
     
   }
-  
+  private void cancelStackDumpTimer(StackDumpTimer stackDumpTimer) {
+      if ( stackDumpTimer != null ) {
+    	  stackDumpTimer.cancel();
+    	  stackDumpTimer = null;   // nullify timer instance so that we dont have to worry about
+          // it in case an exception happens below
+      }
+
+  }
   /**
    * This is called when a Stop request is received from a client. Add the provided Cas id to the
    * list of aborted CASes. The process() method checks this list to determine if it should continue
@@ -758,9 +769,9 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
     if (stopped) {
       return;
     }
-    
-    List<AnalysisEnginePerformanceMetrics> beforeAnalysisManagementObjects = new ArrayList<AnalysisEnginePerformanceMetrics>();
-    List<AnalysisEnginePerformanceMetrics> afterAnalysisManagementObjects = new ArrayList<AnalysisEnginePerformanceMetrics>();
+    System.out.println("Service:"+getComponentName()+" CAS:"+aCasReferenceId+" CAS Hashcode:"+aCAS.hashCode());
+    List<AnalysisEnginePerformanceMetrics> beforeAnalysisManagementObjects = new ArrayList<>();
+    List<AnalysisEnginePerformanceMetrics> afterAnalysisManagementObjects = new ArrayList<>();
     CasStateEntry parentCasStateEntry = null;
     //	If enabled, keep a reference to a timer which
     //  when it expires, will cause a JVM to dump a stack
@@ -810,19 +821,15 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
       }
       
       CasIterator casIterator = ae.processAndOutputNewCASes(aCAS);
-      if ( stackDumpTimer != null ) {
-    	  stackDumpTimer.cancel();
-    	  stackDumpTimer = null;   // nullify timer instance so that we dont have to worry about
-          // it in case an exception happens below
-      }
-      
+      cancelStackDumpTimer(stackDumpTimer);
       // Store how long it took to call processAndOutputNewCASes()
       totalProcessTime = (super.getCpuTime() - time);
       long sequence = 1;
       long hasNextTime = 0; // stores time in hasNext()
-      long getNextTime = 0; // stores time in next();
+      long getNextTime = 0; // stores time in next()
       boolean moreCASesToProcess = true;
       boolean casAbortedDueToExternalRequest = false;
+      
       while (moreCASesToProcess) {
         long timeToProcessCAS = 0; // stores time in hasNext() and next() for each CAS
         hasNextTime = super.getCpuTime();
@@ -831,23 +838,25 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
         //	method is allowed to complete. If the method is not complete in allowed window
         //	the heap and stack trace dump of all threads will be produced.
         stackDumpTimer = ifEnabledStartHeapDumpTimer();
+        
+        /* ********************************************** */
+        /* CHECK IF THERE ARE MORE CHILD CASes TO PROCESS */
+        /* ********************************************** */
         if (!casIterator.hasNext()) {
           moreCASesToProcess = false;
           // Measure how long it took to call hasNext()
           timeToProcessCAS = (super.getCpuTime() - hasNextTime);
           totalProcessTime += timeToProcessCAS;
-          if ( stackDumpTimer != null ) {
-        	  stackDumpTimer.cancel();
-        	  stackDumpTimer = null;   // nullify timer instance so that we dont have to worry about
-              // it in case an exception happens below
-          }
+          cancelStackDumpTimer(stackDumpTimer);
+          /* ************************************* */
+          /* WE ARE DONE PROCESSING INPUT CAS HERE */
+          /* ************************************* */
           break; // from while
         }
-        if ( stackDumpTimer != null ) {
-        	stackDumpTimer.cancel();
-        	stackDumpTimer = null;   // nullify timer instance so that we dont have to worry about
-                                    // it in case an exception happens below
-        }
+
+        
+        cancelStackDumpTimer(stackDumpTimer);
+        
         // Measure how long it took to call hasNext()
         timeToProcessCAS = (super.getCpuTime() - hasNextTime);
         getNextTime = super.getCpuTime();
@@ -856,17 +865,20 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
         //	method is allowed to complete. If the method is not complete in allowed window
         //	the heap and stack trace dump of all threads will be produced.
         stackDumpTimer = ifEnabledStartHeapDumpTimer();
-        CAS casProduced = casIterator.next();
-        if ( stackDumpTimer != null ) {
-        	stackDumpTimer.cancel();
-        	stackDumpTimer = null;   // nullify timer instance so that we dont have to worry about
-            // it in case an exception happens below
-        }
+
+        /* ****************************** */
+        /* GET THE NEXT CHILD CAS         */
+        /* ****************************** */
+        CAS childCAS = casIterator.next();
+        
+        cancelStackDumpTimer(stackDumpTimer);
+
         // Add how long it took to call next()
         timeToProcessCAS += (super.getCpuTime() - getNextTime);
         // Add time to call hasNext() and next() to the running total
         totalProcessTime += timeToProcessCAS;
         casAbortedDueToExternalRequest = abortGeneratingCASes(aCasReferenceId);
+
         // If the service is stopped or aborted, stop generating new CASes and just return the input
         // CAS
         if (stopped || casAbortedDueToExternalRequest) {
@@ -891,8 +903,8 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
                 // We are either stopping the service or aborting input CAS due to explicit STOP
                 // request
                 // from a client. If a new CAS was produced, release it back to the pool.
-                if (casProduced != null) {
-                  casProduced.release();
+                if (childCAS != null) {
+                	childCAS.release();
                 }
               } catch (Exception e) {
             	  if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
@@ -935,24 +947,22 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
 //        OutOfTypeSystemData otsd = getInProcessCache().getOutOfTypeSystemData(aCasReferenceId);
         MessageContext mContext = getInProcessCache()
                 .getMessageAccessorByReference(aCasReferenceId);
-        CacheEntry newEntry = getInProcessCache().register(casProduced, mContext /*, otsd*/);
+        CacheEntry newEntry = getInProcessCache().register(childCAS, mContext /*, otsd*/);
         // if this Cas Multiplier is not Top Level service, add new Cas Id to the private
         // cache of the parent aggregate controller. The Aggregate needs to know about
         // all CASes it has in play that were generated from the input CAS.
         CasStateEntry childCasStateEntry = null;
         if (!isTopLevelComponent()) {
-          newEntry.setNewCas(true, parentController.getComponentName());
-          // Create CAS state entry in the aggregate's local cache
-          childCasStateEntry = parentController.getLocalCache().createCasStateEntry(
-                  newEntry.getCasReferenceId());
-          // Fetch the parent CAS state entry from the aggregate's local cache. We need to increment
-          // number of child CASes associated with it.
-          parentCasStateEntry = parentController.getLocalCache().lookupEntry(aCasReferenceId);
-        } else {
-          childCasStateEntry = getLocalCache().createCasStateEntry(newEntry.getCasReferenceId());
-        }
+            newEntry.setNewCas(true, parentController.getComponentName());
+            // Fetch the parent CAS state entry from the aggregate's local cache. We need to increment
+            // number of child CASes associated with it.
+            parentCasStateEntry = parentController.getLocalCache().lookupEntry(aCasReferenceId);
+          } 
+        childCasStateEntry = getLocalCache().createCasStateEntry(newEntry.getCasReferenceId());
+
         // Associate parent CAS (input CAS) with the new CAS.
-        childCasStateEntry.setInputCasReferenceId(aCasReferenceId);
+        childCasStateEntry.setParentCasReferenceId(aCasReferenceId);
+        childCasStateEntry.setInputCasReferenceId(parentCasStateEntry.getInputCasReferenceId());
         // Increment number of child CASes generated from the input CAS
         parentCasStateEntry.incrementSubordinateCasInPlayCount();
         parentCasStateEntry.incrementOutstandingFlowCounter();
@@ -960,6 +970,8 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
         // Associate input CAS with the new CAS
         newEntry.setInputCasReferenceId(aCasReferenceId);
         newEntry.setCasSequence(sequence);
+        childCasStateEntry.setSequenceNumber(sequence);
+        
         // Add to the cache how long it took to process the generated (subordinate) CAS
         getCasStatistics(newEntry.getCasReferenceId()).incrementAnalysisTime(timeToProcessCAS);
         if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.FINE)) {
@@ -1007,48 +1019,26 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
                     "UIMAEE_exception__WARNING", exx);
         }
 
-        if (!anEndpoint.isRemote()) {
-          UimaTransport transport = getTransport(anEndpoint.getEndpoint());
-          UimaMessage message = transport.produceMessage(AsynchAEMessage.Process,
-                  AsynchAEMessage.Request, getName());
-          message.addStringProperty(AsynchAEMessage.CasReference, newEntry.getCasReferenceId());
-          message.addStringProperty(AsynchAEMessage.InputCasReference, aCasReferenceId);
-          message.addLongProperty(AsynchAEMessage.CasSequence, sequence);
-          ServicePerformance casStats = getCasStatistics(aCasReferenceId);
-
-          message.addLongProperty(AsynchAEMessage.TimeToSerializeCAS, casStats
-                  .getRawCasSerializationTime());
-          message.addLongProperty(AsynchAEMessage.TimeToDeserializeCAS, casStats
-                  .getRawCasDeserializationTime());
-          message.addLongProperty(AsynchAEMessage.TimeInProcessCAS, casStats.getRawAnalysisTime());
-          long iT = getIdleTimeBetweenProcessCalls(AsynchAEMessage.Process);
-          message.addLongProperty(AsynchAEMessage.IdleTime, iT);
-          if (!stopped) {
-            transport.getUimaMessageDispatcher(anEndpoint.getEndpoint()).dispatch(message);
-            dropStats(newEntry.getCasReferenceId(), getName());
-          }
-        } else {
-          // Send generated CAS to the remote client
-          if (!stopped) {
-              getOutputChannel().sendReply(newEntry, anEndpoint);
-            
-              //	Check for delivery failure. The client may have terminated while an input CAS was being processed
-            if ( childCasStateEntry.deliveryToClientFailed() ) {
-              if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
-                    UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(), "process",
-                            UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_delivery_to_client_failed_INFO",
-                            new Object[] { getComponentName(), aCasReferenceId });
-              }
-              clientUnreachable = true;
-              if ( cmOutstandingCASes.containsKey(childCasStateEntry.getCasReferenceId())) {
-              	  cmOutstandingCASes.remove(childCasStateEntry.getCasReferenceId());
-          	  }
-              //	Stop generating new CASes. We failed to send a CAS to a client. Most likely
-              //	the client has terminated. 
-          	  moreCASesToProcess = false; // exit the while loop
-          	  
-          	  dropCAS(childCasStateEntry.getCasReferenceId(), true);
+        // Send generated CAS to the client
+        if (!stopped) {
+            getOutputChannel(anEndpoint).sendReply(childCasStateEntry, anEndpoint);
+          
+            //	Check for delivery failure. The client may have terminated while an input CAS was being processed
+          if ( childCasStateEntry.deliveryToClientFailed() ) {
+            if (UIMAFramework.getLogger(CLASS_NAME).isLoggable(Level.INFO)) {
+                  UIMAFramework.getLogger(CLASS_NAME).logrb(Level.INFO, CLASS_NAME.getName(), "process",
+                          UIMAEE_Constants.JMS_LOG_RESOURCE_BUNDLE, "UIMAEE_delivery_to_client_failed_INFO",
+                          new Object[] { getComponentName(), aCasReferenceId });
             }
+            clientUnreachable = true;
+            if ( cmOutstandingCASes.containsKey(childCasStateEntry.getCasReferenceId())) {
+            	  cmOutstandingCASes.remove(childCasStateEntry.getCasReferenceId());
+        	  }
+            //	Stop generating new CASes. We failed to send a CAS to a client. Most likely
+            //	the client has terminated. 
+        	  moreCASesToProcess = false; // exit the while loop
+        	  
+        	  dropCAS(childCasStateEntry.getCasReferenceId(), true);
           }
         }
         // Remove new CAS state entry from the local cache if this is a top level primitive.
@@ -1117,6 +1107,8 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
       //  Create a List to hold per CAS analysisTime and total number of CASes processed
       //  by each AE. This list will be serialized and sent to the client
       List<AnalysisEnginePerformanceMetrics> performanceList = 
+    		  getCasMetricList(parentCasStateEntry, afterAnalysisManagementObjects, beforeAnalysisManagementObjects);
+      /*
         new ArrayList<AnalysisEnginePerformanceMetrics>();
       //  Diff the before process() performance metrics with post process performance
       //  metrics
@@ -1158,70 +1150,14 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
           }
         }
       }
+      */
       parentCasStateEntry.getAEPerformanceList().addAll(performanceList);
-      if (!anEndpoint.isRemote()) {
-        inputCASReturned = true;
-        UimaTransport transport = getTransport(anEndpoint.getEndpoint());
 
-        if (getInProcessCache() != null && getInProcessCache().getSize() > 0
-                && getInProcessCache().entryExists(aCasReferenceId)) {
-          try {
-            CacheEntry ancestor = 
-                      getInProcessCache().
-                        getTopAncestorCasEntry(getInProcessCache().getCacheEntryForCAS(aCasReferenceId));
-            if ( ancestor != null ) {
-               ancestor.addDelegateMetrics(getKey(), performanceList);
-            }
-          } catch (Exception e) {
-            // An exception be be thrown here if the service is being stopped.
-            // The top level controller may have already cleaned up the cache
-            // and the getCacheEntryForCAS() will throw an exception. Ignore it
-            // here, we are shutting down.
-          }
-        }          
-        
-        UimaMessage message = transport.produceMessage(AsynchAEMessage.Process,
-                AsynchAEMessage.Response, getName());
-        message.addStringProperty(AsynchAEMessage.CasReference, aCasReferenceId);
-        ServicePerformance casStats = getCasStatistics(aCasReferenceId);
-
-        message.addLongProperty(AsynchAEMessage.TimeToSerializeCAS, casStats
-                .getRawCasSerializationTime());
-        message.addLongProperty(AsynchAEMessage.TimeToDeserializeCAS, casStats
-                .getRawCasDeserializationTime());
-        message.addLongProperty(AsynchAEMessage.TimeInProcessCAS, casStats.getRawAnalysisTime());
-        long iT = getIdleTimeBetweenProcessCalls(AsynchAEMessage.Process);
-        message.addLongProperty(AsynchAEMessage.IdleTime, iT);
-        // Send reply back to the client. Use internal (non-jms) transport
-        if (!stopped) {
-          transport.getUimaMessageDispatcher(anEndpoint.getEndpoint()).dispatch(message);
-          dropStats(aCasReferenceId, getName());
-        }
-      } else {
         try {
-        	List<AnalysisEnginePerformanceMetrics> perfMetrics =
-					new ArrayList<AnalysisEnginePerformanceMetrics>();
-          String aeName = getMetaData().getName();
-         
+          
           CacheEntry entry =
                   getInProcessCache().getCacheEntryForCAS(aCasReferenceId);
-          for( AnalysisEnginePerformanceMetrics m : performanceList ) {
- //       	  System.out.println("...............BEFORE:  Name:"+m.getName()+" UniqueName:"+m.getUniqueName()+" How Many="+m.getNumProcessed());
-				boolean aggregate = m.getUniqueName().startsWith("/"+aeName);
-				int pos = m.getUniqueName().indexOf("/",1);
-				String uName = m.getUniqueName();
-				if ( pos > -1 && aeInstancePool.size() > 1 && aeName != null && aggregate) {
-					String st = m.getUniqueName().substring(pos);
-					uName = "/"+aeName+st;
-				} 
-				AnalysisEnginePerformanceMetrics newMetrics = 
-						new AnalysisEnginePerformanceMetrics(m.getName(),uName,m.getAnalysisTime(), m.getNumProcessed());
-			//	System.out.println("... Metrics - AE:"+metrics.getUniqueName()+" AE Analysis Time:"+metrics.getAnalysisTime());
-				perfMetrics.add(newMetrics);
-//	        	  System.out.println("...............AFTER:  Name:"+newMetrics.getName()+" UniqueName:"+newMetrics.getUniqueName()+" How Many="+newMetrics.getNumProcessed());
-
-          }
-          entry.addDelegateMetrics(getKey(), perfMetrics); //performanceList);
+          entry.addDelegateMetrics(getKey(), performanceList);
         } catch (Exception e) {
           // An exception be be thrown here if the service is being stopped.
           // The top level controller may have already cleaned up the cache
@@ -1230,12 +1166,11 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
         }
 
         if (!stopped && !clientUnreachable ) {
-            getOutputChannel().sendReply(getInProcessCache().getCacheEntryForCAS(aCasReferenceId), anEndpoint);
+//            getOutputChannel(anEndpoint).sendReply(getInProcessCache().getCacheEntryForCAS(aCasReferenceId), anEndpoint);
+            getOutputChannel(anEndpoint).sendReply(getLocalCache().lookupEntry(aCasReferenceId), anEndpoint);
         }
 
         inputCASReturned = true;
-      }
-      
       // Remove input CAS state entry from the local cache
       if (!isTopLevelComponent()) {
         localCache.lookupEntry(aCasReferenceId).setDropped(true);
@@ -1297,13 +1232,67 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
         		((CASImpl) aCAS).enableReset(true);
         	}
           // Remove input CAS cache entry if the CAS has been sent to the client
-          dropCAS(aCasReferenceId, true);
+            MessageContext mContext = getInProcessCache()
+                    .getMessageAccessorByReference(aCasReferenceId);
+            if ( mContext != null && mContext instanceof DirectMessageContext) {
+            	dropCASFromLocal(aCasReferenceId);
+            } else {
+                // Remove input CAS cache entry if the CAS has been sent to the client
+                dropCAS(aCasReferenceId, true);
+            }
+        	//dropCAS(aCasReferenceId, true);
+ 
           localCache.dumpContents();
         }
       }
     }
   }
 
+  private List<AnalysisEnginePerformanceMetrics>  getCasMetricList(CasStateEntry parentCasStateEntry, List<AnalysisEnginePerformanceMetrics> afterAnalysisList, List<AnalysisEnginePerformanceMetrics> beforeAnalysisList) {
+      List<AnalysisEnginePerformanceMetrics> performanceList = 
+    	        new ArrayList<AnalysisEnginePerformanceMetrics>();
+	  //  Diff the before process() performance metrics with post process performance
+      //  metrics
+      for (AnalysisEnginePerformanceMetrics after : afterAnalysisList) {
+        for( AnalysisEnginePerformanceMetrics before: beforeAnalysisList) {
+        	if ( before.getUniqueName().equals(after.getUniqueName())) {
+        	  boolean found = false;
+        	  AnalysisEnginePerformanceMetrics metrics = null;
+        	  for( AnalysisEnginePerformanceMetrics met : parentCasStateEntry.getAEPerformanceList() ) {
+                  String un = after.getUniqueName();
+        		  if ( un.indexOf("Components") >= -1 ) {
+        			  un = un.substring(un.indexOf("/"));
+        		  }
+           		  if ( met.getUniqueName().equals(un)) {
+                      long at = after.getAnalysisTime()- before.getAnalysisTime();
+                      metrics = new AnalysisEnginePerformanceMetrics(after.getName(),
+                              un,//after.getUniqueName(),
+                              met.getAnalysisTime()+at,
+                              after.getNumProcessed());
+                      found = true;
+                      parentCasStateEntry.getAEPerformanceList().remove(met);
+                      break;
+        		  } 
+        	  }
+        	  if ( !found ) {
+        		  String un = after.getUniqueName();
+        		  
+        		  if ( un.indexOf("Components") >= -1 ) {
+        			  un = un.substring(un.indexOf("/"));
+        		  }
+                  metrics = new AnalysisEnginePerformanceMetrics(after.getName(),
+                          un,//after.getUniqueName(),
+                          after.getAnalysisTime()- before.getAnalysisTime(),
+                          after.getNumProcessed());
+        		  
+        	  }
+            performanceList.add(metrics);
+            break;
+          }
+        }
+      }
+      return performanceList;
+  }
   private void addConfigIntParameter(String aParamName, int aParamValue) {
     ConfigurationParameter cp = new ConfigurationParameter_impl();
     cp.setMandatory(false);
@@ -1414,6 +1403,13 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
 
   public void stop() {
     super.stop(true);  // shutdown now
+    if ( getLocalCache().size() > 0 ) {
+    	for( Entry<String, LocalCache.CasStateEntry> entry : getLocalCache().entrySet()) {
+    		System.out.println("........... Controller:"+getComponentName()+" - stop() - Releasing CAS:"+entry.getKey());
+    		releaseNextCas(entry.getKey());
+    	}
+    }
+
     if (aeInstancePool != null) {
       try {
         aeInstancePool.destroy();
@@ -1517,7 +1513,7 @@ public class PrimitiveAnalysisEngineController_impl extends BaseAnalysisEngineCo
   
   /**
    * The HeapDumpTimer is optionally used to dump the heap if a task takes too much time to finish.
-   * It is enabled from the System property -DheapDumpThreshold=x where x is a number of seconds 
+   * It is enabled from the System property -DheapDumpThreshold=<x> where x is a number of seconds 
    * the task is allowed to complete. If the task is not completed, the heap dump will be created. 
    * 
    *
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/delegate/Delegate.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/delegate/Delegate.java
index 3616638..7ba1473 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/delegate/Delegate.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/delegate/Delegate.java
@@ -822,6 +822,7 @@ public abstract class Delegate {
    *          - command for which the timer is started
    */
   private void startDelegateGetMetaTimer(final String aCasReferenceId, final int aCommand) {
+	  Thread.dumpStack();
     synchronized( getMetaTimerLock ) {
       final long timeToWait = getTimeoutValueForCommand(aCommand);
       Date timeToRun = new Date(System.currentTimeMillis() + timeToWait);
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandler.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandler.java
index 263e239..58fe210 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandler.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandler.java
@@ -43,6 +43,6 @@ public interface ErrorHandler {
   public boolean handleError(Throwable t, ErrorContext anErrorContext,
           AnalysisEngineController aController);
 
-  public Map getEndpointThresholdMap();
+  public Map<String,Threshold> getEndpointThresholdMap();
 
 }
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerBase.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerBase.java
index b60e6c7..4daecd9 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerBase.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerBase.java
@@ -225,7 +225,7 @@ public abstract class ErrorHandlerBase {
 
                 if (aController instanceof AggregateAnalysisEngineController && (masterEndpoint != null && masterEndpoint.getStatus() == Endpoint.FAILED)) {
                   // Fetch an InputChannel that handles messages for a given delegate
-                  InputChannel iC = aController.getReplyInputChannel(masterEndpoint.getDelegateKey());
+                  InputChannel iC = aController.getInputChannel();
                   // Create a new Listener, new Temp Queue and associate the listener with the Input Channel
                   iC.createListener(masterEndpoint.getDelegateKey(), null);
                   iC.removeDelegateFromFailedList(masterEndpoint.getDelegateKey());
diff --git a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerChain.java b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerChain.java
index f60ee52..c64c632 100644
--- a/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerChain.java
+++ b/uimaj-as-core/src/main/java/org/apache/uima/aae/error/ErrorHandlerChain.java
@@ -27,11 +27,21 @@ import java.util.Map;
 
 import org.apache.uima.aae.controller.AnalysisEngineController;
 
-public class ErrorHandlerChain extends LinkedList {
-  public ErrorHandlerChain(List aChainofHandlers) {
+public class ErrorHandlerChain extends LinkedList<ErrorHandler>  {
+  /**
+	 * 
+	 */
+	private static final long serialVersionUID = 1L;
+
+  public ErrorHandlerChain(List<ErrorHandler> aChainofHandlers) {
     this.addAll(aChainofHandlers);
   }
+  public ErrorHandlerChain() {
+  }
 
... 9055 lines suppressed ...