You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tuscany.apache.org by jm...@apache.org on 2006/09/29 07:31:28 UTC

svn commit: r451133 - in /incubator/tuscany/java/sca: kernel/core/src/main/java/org/apache/tuscany/core/builder/ kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/ kernel/core/src/main/java/org/apache/tuscany/core/wire/ kernel/core/...

Author: jmarino
Date: Thu Sep 28 22:31:27 2006
New Revision: 451133

URL: http://svn.apache.org/viewvc?view=rev&rev=451133
Log:
move thread switching for non-blocking from async target invokers to a non-blocking bridging interceptor

Added:
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java   (with props)
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java   (with props)
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java   (contents, props changed)
      - copied, changed from r450960, incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java   (with props)
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java   (with props)
Modified:
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/builder/ConnectorImpl.java
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvoker.java
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncMonitor.java
    incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/JavaAtomicComponent.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorImplTestCase.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorPostProcessTestCase.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/OutboundToInboundConnectTestCase.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvokerTestCase.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/CallbackInvocationTestCase.java
    incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/OneWayWireInvocationTestCase.java
    incubator/tuscany/java/sca/kernel/spi/src/main/java/org/apache/tuscany/spi/builder/Connector.java
    incubator/tuscany/java/sca/runtime/osgi/src/main/resources/META-INF/sca/osgibinding.scdl
    incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncGroovyInvoker.java
    incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncMonitor.java
    incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/GroovyAtomicComponent.java
    incubator/tuscany/java/sca/services/containers/container.groovy/src/test/java/org/apache/tuscany/container/groovy/AsyncInvokerTestCase.java
    incubator/tuscany/java/sca/test/src/main/java/org/apache/tuscany/test/ArtifactFactory.java

Modified: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/builder/ConnectorImpl.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/builder/ConnectorImpl.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/builder/ConnectorImpl.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/builder/ConnectorImpl.java Thu Sep 28 22:31:27 2006
@@ -35,9 +35,11 @@
 import org.apache.tuscany.spi.component.Reference;
 import org.apache.tuscany.spi.component.SCAObject;
 import org.apache.tuscany.spi.component.Service;
+import org.apache.tuscany.spi.component.WorkContext;
 import org.apache.tuscany.spi.model.Operation;
 import org.apache.tuscany.spi.model.Scope;
 import org.apache.tuscany.spi.model.ServiceContract;
+import org.apache.tuscany.spi.services.work.WorkScheduler;
 import org.apache.tuscany.spi.wire.InboundInvocationChain;
 import org.apache.tuscany.spi.wire.InboundWire;
 import org.apache.tuscany.spi.wire.Interceptor;
@@ -48,9 +50,9 @@
 
 import org.apache.tuscany.core.implementation.composite.CompositeReference;
 import org.apache.tuscany.core.implementation.composite.CompositeService;
-import org.apache.tuscany.core.wire.BridgingInterceptor;
-import org.apache.tuscany.core.wire.InvokerInterceptor;
+import org.apache.tuscany.core.wire.NonBlockingBridgingInterceptor;
 import org.apache.tuscany.core.wire.OutboundAutowire;
+import org.apache.tuscany.core.wire.SynchronousBridgingInterceptor;
 
 /**
  * The default connector implmentation
@@ -61,14 +63,21 @@
 
     private WirePostProcessorRegistry postProcessorRegistry;
     private WireService wireService;
+    private WorkContext workContext;
+    private WorkScheduler scheduler;
 
     public ConnectorImpl() {
     }
 
-    @Constructor({"wireService", "processorRegistry"})
-    public ConnectorImpl(@Autowire WireService wireService, @Autowire WirePostProcessorRegistry processorRegistry) {
+    @Constructor({"wireService", "processorRegistry", "scheduler", "workContext"})
+    public ConnectorImpl(@Autowire WireService wireService,
+                         @Autowire WirePostProcessorRegistry processorRegistry,
+                         @Autowire WorkScheduler scheduler,
+                         @Autowire WorkContext workContext) {
         this.postProcessorRegistry = processorRegistry;
         this.wireService = wireService;
+        this.scheduler = scheduler;
+        this.workContext = workContext;
     }
 
     public void connect(SCAObject source) {
@@ -219,12 +228,22 @@
 
             if (source instanceof Service && !(source instanceof CompositeService)) {
                 // services are a special case: invoker must go on the inbound chain
-                connect(outboundChain, inboundChain, null);
+                if (target instanceof Component && (isOneWayOperation || operationHasCallback)) {
+                    // if the target is a component and the operation is non-blocking
+                    connect(outboundChain, inboundChain, null, true);
+                } else {
+                    connect(outboundChain, inboundChain, null, false);
+                }
                 Service service = (Service) source;
                 InboundInvocationChain chain = service.getInboundWire().getInvocationChains().get(operation);
                 chain.setTargetInvoker(invoker);
             } else {
-                connect(outboundChain, inboundChain, invoker);
+                if (target instanceof Component && (isOneWayOperation || operationHasCallback)) {
+                    // if the target is a component and the operation is non-blocking
+                    connect(outboundChain, inboundChain, invoker, true);
+                } else {
+                    connect(outboundChain, inboundChain, invoker, false);
+                }
             }
         }
 
@@ -245,17 +264,17 @@
             if (source instanceof Component) {
                 Component component = (Component) source;
                 TargetInvoker invoker = component.createTargetInvoker(null, operation);
-                connect(outboundChain, inboundChain, invoker);
+                connect(outboundChain, inboundChain, invoker, false);
             } else if (source instanceof CompositeReference) {
                 CompositeReference compRef = (CompositeReference) source;
                 ServiceContract sourceContract = sourceWire.getServiceContract();
                 TargetInvoker invoker = compRef.createCallbackTargetInvoker(sourceContract, operation);
-                connect(outboundChain, inboundChain, invoker);
+                connect(outboundChain, inboundChain, invoker, false);
             } else if (source instanceof Service) {
                 Service service = (Service) source;
                 ServiceContract sourceContract = sourceWire.getServiceContract();
                 TargetInvoker invoker = service.createCallbackTargetInvoker(sourceContract, operation);
-                connect(outboundChain, inboundChain, invoker);
+                connect(outboundChain, inboundChain, invoker, false);
             }
         }
     }
@@ -266,18 +285,22 @@
      * @param sourceChain the source chain
      * @param targetChain the target chain
      * @param invoker     the invoker to place on the source chain for dispatching invocations
+     * @param nonBlocking true if the operation is non-blocking
      */
-    void connect(OutboundInvocationChain sourceChain, InboundInvocationChain targetChain, TargetInvoker invoker) {
-        Interceptor headInterceptor = targetChain.getHeadInterceptor();
-        if (headInterceptor == null) {
+    void connect(OutboundInvocationChain sourceChain,
+                 InboundInvocationChain targetChain,
+                 TargetInvoker invoker,
+                 boolean nonBlocking) {
+        Interceptor head = targetChain.getHeadInterceptor();
+        if (head == null) {
             BuilderConfigException e = new BuilderConfigException("No interceptor for operation");
             e.setIdentifier(targetChain.getOperation().getName());
             throw e;
         }
-        if (!(sourceChain.getTailInterceptor() instanceof InvokerInterceptor
-            && headInterceptor instanceof InvokerInterceptor)) {
-            // check that we do not have the case where the only interceptors are invokers since we just need one
-            sourceChain.setTargetInterceptor(headInterceptor);
+        if (nonBlocking) {
+            sourceChain.setTargetInterceptor(new NonBlockingBridgingInterceptor(scheduler, workContext, head));
+        } else {
+            sourceChain.setTargetInterceptor(new SynchronousBridgingInterceptor(head));
         }
         sourceChain.prepare(); //FIXME prepare should be moved out
         sourceChain.setTargetInvoker(invoker);
@@ -291,8 +314,8 @@
      * @param targetChain
      */
     void connect(InboundInvocationChain sourceChain, OutboundInvocationChain targetChain) {
-        // the are always interceptors so the connection algorithm is simple
-        sourceChain.addInterceptor(new BridgingInterceptor(targetChain.getHeadInterceptor()));
+        // invocations from inbound to outbound chains are always syncrhonius as they occur in services and references
+        sourceChain.addInterceptor(new SynchronousBridgingInterceptor(targetChain.getHeadInterceptor()));
     }
 
     /**

Modified: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvoker.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvoker.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvoker.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvoker.java Thu Sep 28 22:31:27 2006
@@ -18,21 +18,13 @@
  */
 package org.apache.tuscany.core.implementation.java;
 
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 
-import org.osoa.sca.CompositeContext;
-import org.osoa.sca.CurrentCompositeContext;
-import org.osoa.sca.SCA;
-import org.osoa.sca.ServiceRuntimeException;
-
 import org.apache.tuscany.spi.component.TargetException;
 import org.apache.tuscany.spi.component.WorkContext;
-import org.apache.tuscany.spi.services.work.WorkScheduler;
 import org.apache.tuscany.spi.wire.InboundWire;
 import org.apache.tuscany.spi.wire.InvocationRuntimeException;
 import org.apache.tuscany.spi.wire.Message;
-import org.apache.tuscany.spi.wire.TargetInvoker;
 
 import org.apache.tuscany.core.wire.PojoTargetInvoker;
 
@@ -43,98 +35,51 @@
  */
 public class AsyncJavaTargetInvoker extends PojoTargetInvoker {
 
-    private static final ContextBinder BINDER = new ContextBinder();
-    private static final Message RESPONSE = new ImmutableMessage();
-
     private JavaAtomicComponent component;
     private InboundWire wire;
-    private WorkScheduler workScheduler;
     private AsyncMonitor monitor;
     private WorkContext workContext;
     private Object target;
-    private Object messageId;
 
     /**
      * Creates a new invoker
      *
-     * @param operation     the operation the invoker is associated with
+     * @param operation   the operation the invoker is associated with
      * @param wire
-     * @param component     the target component
-     * @param workScheduler the work scheduler to run the invocation
-     * @param monitor       the monitor to pass events to
+     * @param component   the target component
+     * @param monitor     the monitor to pass events to
      * @param workContext
      */
     public AsyncJavaTargetInvoker(Method operation,
                                   InboundWire wire,
                                   JavaAtomicComponent component,
-                                  WorkScheduler workScheduler,
                                   AsyncMonitor monitor,
                                   WorkContext workContext) {
         super(operation);
         this.wire = wire;
         this.component = component;
-        this.workScheduler = workScheduler;
         this.monitor = monitor;
         this.workContext = workContext;
     }
 
-    // Override invocation methods to defer invocation to work item
-    // Both methods return null to indicate asynchrony; result will
-    // be conveyed by callback
-    @Override
-    public Object invokeTarget(final Object payload) throws InvocationTargetException {
-        final CompositeContext currentContext = CurrentCompositeContext.getContext();
-        // Schedule the invocation of the next interceptor in a new Work instance
-        try {
-            workScheduler.scheduleWork(new Runnable() {
-                private Object currentMessageId = messageId;
-
-                public void run() {
-                    workContext.setCurrentMessageId(null);
-                    workContext.setCurrentCorrelationId(currentMessageId);
-                    CompositeContext oldContext = CurrentCompositeContext.getContext();
-                    try {
-                        BINDER.setContext(currentContext);
-                        AsyncJavaTargetInvoker.super.invokeTarget(payload);
-                    } catch (Exception e) {
-                        // REVIEW uncomment when it is available
-                        // monitor.executionError(e);
-                        e.printStackTrace();
-                    } finally {
-                        BINDER.setContext(oldContext);
-                    }
-                }
-            });
-        } catch (Exception e) {
-            throw new ServiceRuntimeException(e);
-        }
-        return RESPONSE;
-    }
-
     public Message invoke(Message msg) throws InvocationRuntimeException {
-        // can't just call overriden invoke because it would bypass async
         try {
-            messageId = msg.getMessageId();
+            Object messageId = msg.getMessageId();
             wire.addMapping(messageId, msg.getFromAddress());
-            Object resp = invokeTarget(msg.getBody());
-            return (Message) resp;
-        } catch (InvocationTargetException e) {
-            // FIXME need to log exceptions
-            e.printStackTrace();
+            workContext.setCurrentMessageId(null);
+            workContext.setCurrentCorrelationId(messageId);
+            invokeTarget(msg.getBody());
+            // async so no return value
             return null;
         } catch (Throwable e) {
             // FIXME need to log exceptions
-            e.printStackTrace();
+            monitor.executionError(e);
             return null;
         }
     }
 
     public AsyncJavaTargetInvoker clone() throws CloneNotSupportedException {
-        AsyncJavaTargetInvoker invoker = (AsyncJavaTargetInvoker) super.clone();
-        invoker.workScheduler = this.workScheduler;
-        invoker.monitor = this.monitor;
-
-        return invoker;
+        return (AsyncJavaTargetInvoker) super.clone();
     }
 
     /**
@@ -149,78 +94,5 @@
             }
             return target;
         }
-    }
-
-    private static class ContextBinder extends SCA {
-        public void setContext(CompositeContext context) {
-            setCompositeContext(context);
-        }
-
-        public void start() {
-            throw new AssertionError();
-        }
-
-        public void stop() {
-            throw new AssertionError();
-        }
-    }
-
-    /**
-     * A dummy message passed back on an invocation
-     */
-    private static class ImmutableMessage implements Message {
-
-        public Object getBody() {
-            return null;
-        }
-
-        public void setBody(Object body) {
-            throw new UnsupportedOperationException();
-        }
-
-        public void setTargetInvoker(TargetInvoker invoker) {
-            throw new UnsupportedOperationException();
-        }
-
-        public TargetInvoker getTargetInvoker() {
-            return null;
-        }
-
-        public Message getRelatedCallbackMessage() {
-            return null;
-        }
-
-        public Object getFromAddress() {
-            return null;
-        }
-
-        public void setFromAddress(Object fromAddress) {
-            throw new UnsupportedOperationException();
-        }
-
-        public Object getMessageId() {
-            return null;
-        }
-
-        public void setMessageId(Object messageId) {
-            throw new UnsupportedOperationException();
-        }
-
-        public Object getCorrelationId() {
-            return null;
-        }
-
-        public void setCorrelationId(Object correlationId) {
-            throw new UnsupportedOperationException();
-        }
-        
-        public boolean isFault() {
-            return false;
-        }
-
-        public void setBodyWithFault(Object fault) {
-            throw new UnsupportedOperationException();
-        }
-        
     }
 }

Modified: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncMonitor.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncMonitor.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncMonitor.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/AsyncMonitor.java Thu Sep 28 22:31:27 2006
@@ -30,6 +30,6 @@
     /**
      * Logs an exception thrown during an invocation
      */
-    void executionError(Exception e);
+    void executionError(Throwable e);
 
 }

Modified: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/JavaAtomicComponent.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/JavaAtomicComponent.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/JavaAtomicComponent.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/implementation/java/JavaAtomicComponent.java Thu Sep 28 22:31:27 2006
@@ -98,7 +98,7 @@
         } else {
             method = findMethod(operation, operation.getServiceContract().getInterfaceClass().getMethods());
         }
-        return new AsyncJavaTargetInvoker(method, wire, this, workScheduler, monitor, workContext);
+        return new AsyncJavaTargetInvoker(method, wire, this, monitor, workContext);
     }
 
     protected void onServiceWire(InboundWire wire) {

Added: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java?view=auto&rev=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java (added)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java Thu Sep 28 22:31:27 2006
@@ -0,0 +1,29 @@
+/*
+ * 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.tuscany.core.wire;
+
+import org.apache.tuscany.spi.wire.Interceptor;
+
+/**
+ * Responsible for bridging an outbound to an inbound invocation chain associated with a source and target respectively
+ *
+ * @version $Rev$ $Date$
+ */
+public interface BridgingInterceptor extends Interceptor {
+}

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java?view=auto&rev=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java (added)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java Thu Sep 28 22:31:27 2006
@@ -0,0 +1,152 @@
+package org.apache.tuscany.core.wire;
+
+import org.osoa.sca.CompositeContext;
+import org.osoa.sca.CurrentCompositeContext;
+import org.osoa.sca.SCA;
+import org.osoa.sca.ServiceRuntimeException;
+
+import org.apache.tuscany.spi.component.WorkContext;
+import org.apache.tuscany.spi.services.work.WorkScheduler;
+import org.apache.tuscany.spi.wire.Interceptor;
+import org.apache.tuscany.spi.wire.Message;
+import org.apache.tuscany.spi.wire.TargetInvoker;
+
+/**
+ * Bridges interceptors in a non-blocking fashion between an {@link org.apache.tuscany.spi.wire.InboundInvocationChain}
+ * and an {@link org.apache.tuscany.spi.wire.OutboundInvocationChain} by using a {@link
+ * org.apache.tuscany.spi.component.WorkContext}.
+ *
+ * @version $$Rev$$ $$Date$$
+ */
+public class NonBlockingBridgingInterceptor implements BridgingInterceptor {
+
+    private static final ContextBinder BINDER = new ContextBinder();
+    private static final Message RESPONSE = new ImmutableMessage();
+
+    private WorkScheduler workScheduler;
+    private WorkContext workContext;
+    private Interceptor next;
+
+    public NonBlockingBridgingInterceptor(WorkScheduler workScheduler, WorkContext workContext) {
+        this.workScheduler = workScheduler;
+        this.workContext = workContext;
+    }
+
+    public NonBlockingBridgingInterceptor(WorkScheduler workScheduler, WorkContext workContext, Interceptor next) {
+        this.workScheduler = workScheduler;
+        this.workContext = workContext;
+        this.next = next;
+    }
+
+    public Message invoke(final Message msg) {
+        final CompositeContext currentContext = CurrentCompositeContext.getContext();
+        // Schedule the invocation of the next interceptor in a new Work instance
+        try {
+            workScheduler.scheduleWork(new Runnable() {
+                public void run() {
+                    workContext.setCurrentMessageId(null);
+                    CompositeContext oldContext = CurrentCompositeContext.getContext();
+                    try {
+                        BINDER.setContext(currentContext);
+                        next.invoke(msg);
+                    } catch (Exception e) {
+                        // REVIEW uncomment when it is available
+                        // monitor.executionError(e);
+                        e.printStackTrace();
+                    } finally {
+                        BINDER.setContext(oldContext);
+                    }
+                }
+            });
+        } catch (Exception e) {
+            throw new ServiceRuntimeException(e);
+        }
+        return RESPONSE;
+    }
+
+    public Interceptor getNext() {
+        return next;
+    }
+
+    public void setNext(Interceptor next) {
+        this.next = next;
+    }
+
+    public boolean isOptimizable() {
+        return true;
+    }
+
+    private static class ContextBinder extends SCA {
+        public void setContext(CompositeContext context) {
+            setCompositeContext(context);
+        }
+
+        public void start() {
+            throw new AssertionError();
+        }
+
+        public void stop() {
+            throw new AssertionError();
+        }
+    }
+
+    /**
+     * A dummy message passed back on an invocation
+     */
+    private static class ImmutableMessage implements Message {
+
+        public Object getBody() {
+            return null;
+        }
+
+        public void setBody(Object body) {
+            throw new UnsupportedOperationException();
+        }
+
+        public void setTargetInvoker(TargetInvoker invoker) {
+            throw new UnsupportedOperationException();
+        }
+
+        public TargetInvoker getTargetInvoker() {
+            return null;
+        }
+
+        public Message getRelatedCallbackMessage() {
+            return null;
+        }
+
+        public Object getFromAddress() {
+            return null;
+        }
+
+        public void setFromAddress(Object fromAddress) {
+            throw new UnsupportedOperationException();
+        }
+
+        public Object getMessageId() {
+            return null;
+        }
+
+        public void setMessageId(Object messageId) {
+            throw new UnsupportedOperationException();
+        }
+
+        public Object getCorrelationId() {
+            return null;
+        }
+
+        public void setCorrelationId(Object correlationId) {
+            throw new UnsupportedOperationException();
+        }
+
+        public boolean isFault() {
+            return false;
+        }
+
+        public void setBodyWithFault(Object fault) {
+            throw new UnsupportedOperationException();
+        }
+
+    }
+
+}

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptor.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java (from r450960, incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java)
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java?view=diff&rev=451133&p1=incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java&r1=450960&p2=incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/BridgingInterceptor.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java Thu Sep 28 22:31:27 2006
@@ -22,18 +22,18 @@
 import org.apache.tuscany.spi.wire.Message;
 
 /**
- * Bridges interceptors between an {@link org.apache.tuscany.spi.wire.InboundInvocationChain} and an {@link
- * org.apache.tuscany.spi.wire.OutboundInvocationChain}.
+ * Synchronously bridges interceptors between an {@link org.apache.tuscany.spi.wire.InboundInvocationChain} and an
+ * {@link org.apache.tuscany.spi.wire.OutboundInvocationChain}.
  *
  * @version $$Rev$$ $$Date$$
  */
-public class BridgingInterceptor implements Interceptor {
+public class SynchronousBridgingInterceptor implements BridgingInterceptor {
     private Interceptor next;
 
-    public BridgingInterceptor() {
+    public SynchronousBridgingInterceptor() {
     }
 
-    public BridgingInterceptor(Interceptor next) {
+    public SynchronousBridgingInterceptor(Interceptor next) {
         this.next = next;
     }
 

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/tuscany/java/sca/kernel/core/src/main/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptor.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorImplTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorImplTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorImplTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorImplTestCase.java Thu Sep 28 22:31:27 2006
@@ -24,7 +24,7 @@
 import org.apache.tuscany.spi.wire.TargetInvoker;
 
 import junit.framework.TestCase;
-import org.apache.tuscany.core.wire.BridgingInterceptor;
+import org.apache.tuscany.core.wire.SynchronousBridgingInterceptor;
 import org.easymock.EasyMock;
 
 /**
@@ -43,7 +43,7 @@
         // create the inbound wire and chain
         InboundInvocationChain inboundChain = EasyMock.createMock(InboundInvocationChain.class);
         EasyMock.expect(inboundChain.getOperation()).andReturn(operation).atLeastOnce();
-        inboundChain.addInterceptor(EasyMock.isA(BridgingInterceptor.class));
+        inboundChain.addInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         inboundChain.setTargetInvoker(null);
         inboundChain.prepare();
         EasyMock.replay(inboundChain);
@@ -117,7 +117,7 @@
         // create the inbound wire and chain for the source service
         InboundInvocationChain inboundChain = EasyMock.createMock(InboundInvocationChain.class);
         EasyMock.expect(inboundChain.getOperation()).andReturn(operation).atLeastOnce();
-        inboundChain.addInterceptor(EasyMock.isA(BridgingInterceptor.class));
+        inboundChain.addInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         inboundChain.setTargetInvoker(null);
         EasyMock.replay(inboundChain);
         Map<Operation<?>, InboundInvocationChain> inboundChains = new HashMap<Operation<?>, InboundInvocationChain>();
@@ -129,9 +129,8 @@
 
         // create the outbound wire and chain for the source service
         OutboundInvocationChain outboundChain = EasyMock.createMock(OutboundInvocationChain.class);
-        EasyMock.expect(outboundChain.getTailInterceptor()).andReturn(tailInterceptor);
         EasyMock.expect(outboundChain.getHeadInterceptor()).andReturn(headInterceptor);
-        outboundChain.setTargetInterceptor(headInterceptor);
+        outboundChain.setTargetInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         outboundChain.prepare();
         outboundChain.setTargetInvoker(null);
         EasyMock.expect(outboundChain.getOperation()).andReturn(operation);
@@ -204,9 +203,8 @@
 
         // create the outbound wire and chain from the source component
         OutboundInvocationChain outboundChain = EasyMock.createMock(OutboundInvocationChain.class);
-        EasyMock.expect(outboundChain.getTailInterceptor()).andReturn(tailInterceptor);
         EasyMock.expect(outboundChain.getOperation()).andReturn(operation).atLeastOnce();
-        outboundChain.setTargetInterceptor(EasyMock.eq(headInterceptor));
+        outboundChain.setTargetInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         outboundChain.setTargetInvoker(null);
         outboundChain.prepare();
         EasyMock.replay(outboundChain);
@@ -312,18 +310,17 @@
         EasyMock.replay(inboundChain);
 
         OutboundInvocationChain outboundChain = EasyMock.createMock(OutboundInvocationChain.class);
-        EasyMock.expect(outboundChain.getTailInterceptor()).andReturn(tailInterceptor);
         outboundChain.prepare();
-        outboundChain.setTargetInterceptor(headInterceptor);
+        outboundChain.setTargetInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         outboundChain.setTargetInvoker(invoker);
         EasyMock.replay(outboundChain);
-        connector.connect(outboundChain, inboundChain, invoker);
+        connector.connect(outboundChain, inboundChain, invoker, false);
         EasyMock.verify(outboundChain);
     }
 
     public void testInboundToOutboundChainConnect() {
         InboundInvocationChain inboundChain = EasyMock.createMock(InboundInvocationChain.class);
-        inboundChain.addInterceptor(EasyMock.isA(BridgingInterceptor.class));
+        inboundChain.addInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         EasyMock.replay(inboundChain);
 
         OutboundInvocationChain outboundChain = EasyMock.createMock(OutboundInvocationChain.class);
@@ -361,9 +358,8 @@
 
         // create the outbound wire and chain from the source component
         OutboundInvocationChain outboundChain = EasyMock.createMock(OutboundInvocationChain.class);
-        EasyMock.expect(outboundChain.getTailInterceptor()).andReturn(tailInterceptor);
         EasyMock.expect(outboundChain.getOperation()).andReturn(operation).atLeastOnce();
-        outboundChain.setTargetInterceptor(EasyMock.eq(headInterceptor));
+        outboundChain.setTargetInterceptor(EasyMock.isA(SynchronousBridgingInterceptor.class));
         outboundChain.setTargetInvoker(null);
         outboundChain.prepare();
         EasyMock.replay(outboundChain);

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorPostProcessTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorPostProcessTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorPostProcessTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/ConnectorPostProcessTestCase.java Thu Sep 28 22:31:27 2006
@@ -58,7 +58,7 @@
         expect(wireService.isWireable((ServiceContract<?>) EasyMock.anyObject(),
             (ServiceContract<?>) EasyMock.anyObject())).andReturn(true).anyTimes();
         replay(wireService);
-        ConnectorImpl connector = new ConnectorImpl(wireService, registry);
+        ConnectorImpl connector = new ConnectorImpl(wireService, registry, null, null);
         connector.connect(iwire, owire, false);
         verify(registry);
     }
@@ -86,7 +86,7 @@
         expect(wireService.isWireable((ServiceContract<?>) EasyMock.anyObject(),
             (ServiceContract<?>) EasyMock.anyObject())).andReturn(true).anyTimes();
         replay(wireService);
-        ConnectorImpl connector = new ConnectorImpl(wireService, registry);
+        ConnectorImpl connector = new ConnectorImpl(wireService, registry, null, null);
 
         connector.connect(owire, iwire, false);
         verify(registry);

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/OutboundToInboundConnectTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/OutboundToInboundConnectTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/OutboundToInboundConnectTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/builder/OutboundToInboundConnectTestCase.java Thu Sep 28 22:31:27 2006
@@ -61,7 +61,7 @@
         TargetInvoker invoker = createNiceMock(TargetInvoker.class);
         expect(invoker.invokeTarget(EasyMock.eq(val))).andReturn(val);
         replay(invoker);
-        connector.connect(outboundChain, inboundChain, invoker);
+        connector.connect(outboundChain, inboundChain, invoker, false);
         inboundChain.prepare();
         assertEquals(val, outboundChain.getTargetInvoker().invokeTarget(val));
         verify(invoker);
@@ -83,7 +83,7 @@
         expect(invoker.invoke(EasyMock.eq(msg))).andReturn(msg);
         replay(invoker);
         assertEquals(0, interceptor.getCount());
-        connector.connect(outboundChain, inboundChain, invoker);
+        connector.connect(outboundChain, inboundChain, invoker, false);
         inboundChain.prepare();
         msg.setTargetInvoker(outboundChain.getTargetInvoker());
         assertEquals(msg, outboundChain.getHeadInterceptor().invoke(msg));
@@ -106,7 +106,7 @@
         expect(invoker.invoke(EasyMock.eq(msg))).andReturn(msg);
         replay(invoker);
         assertEquals(0, interceptor.getCount());
-        connector.connect(outboundChain, inboundChain, invoker);
+        connector.connect(outboundChain, inboundChain, invoker, false);
         inboundChain.prepare();
         msg.setTargetInvoker(outboundChain.getTargetInvoker());
         assertEquals(msg, outboundChain.getHeadInterceptor().invoke(msg));
@@ -133,7 +133,7 @@
         replay(invoker);
         assertEquals(0, sourceInterceptor.getCount());
         assertEquals(0, targetInterceptor.getCount());
-        connector.connect(outboundChain, inboundChain, invoker);
+        connector.connect(outboundChain, inboundChain, invoker, false);
         inboundChain.prepare();
         msg.setTargetInvoker(outboundChain.getTargetInvoker());
         assertEquals(msg, outboundChain.getHeadInterceptor().invoke(msg));

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvokerTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvokerTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvokerTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/AsyncJavaTargetInvokerTestCase.java Thu Sep 28 22:31:27 2006
@@ -51,23 +51,19 @@
         AsyncMonitor monitor = createMock(AsyncMonitor.class);
         replay(monitor);
 
-        WorkScheduler scheduler = createMock(WorkScheduler.class);
-        scheduler.scheduleWork(isA(Runnable.class));
-        expectLastCall().andStubAnswer(new IAnswer<Object>() {
-            public Object answer() throws Throwable {
-                Runnable runnable = (Runnable) getCurrentArguments()[0];
-                runnable.run();
-                return null;
-            }
-        });
-        replay(scheduler);
+        Message msg = new MessageImpl();
+        Object id = new Object();
+        msg.setMessageId(id);
+
         WorkContext context = createMock(WorkContext.class);
+        context.setCurrentMessageId(null);
+        context.setCurrentCorrelationId(id);
+        replay(context);
         Method method = AsyncTarget.class.getMethod("invoke");
         method.setAccessible(true);
         InboundWire wire = createMock(InboundWire.class);
         AsyncJavaTargetInvoker invoker =
-            new AsyncJavaTargetInvoker(method, wire, component, scheduler, monitor, context);
-        Message msg = new MessageImpl();
+            new AsyncJavaTargetInvoker(method, wire, component, monitor, context);
         invoker.invoke(msg);
         verify(target);
     }
@@ -95,7 +91,7 @@
         Method method = AsyncTarget.class.getMethod("invoke");
         method.setAccessible(true);
         AsyncJavaTargetInvoker invoker =
-            new AsyncJavaTargetInvoker(method, wire, component, scheduler, monitor, context);
+            new AsyncJavaTargetInvoker(method, wire, component, monitor, context);
         AsyncJavaTargetInvoker clone = invoker.clone();
         Message msg = new MessageImpl();
         clone.invoke(msg);

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/CallbackInvocationTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/CallbackInvocationTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/CallbackInvocationTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/CallbackInvocationTestCase.java Thu Sep 28 22:31:27 2006
@@ -27,6 +27,7 @@
 import org.apache.tuscany.spi.builder.Connector;
 import org.apache.tuscany.spi.component.CompositeComponent;
 import org.apache.tuscany.spi.component.ScopeContainer;
+import org.apache.tuscany.spi.component.WorkContext;
 import org.apache.tuscany.spi.deployer.DeploymentContext;
 import org.apache.tuscany.spi.idl.InvalidServiceContractException;
 import org.apache.tuscany.spi.idl.java.JavaInterfaceProcessorRegistry;
@@ -69,7 +70,8 @@
     private DeploymentContext context;
     private JavaComponentBuilder builder;
     private WireService wireService;
-
+    private WorkScheduler scheduler;
+    private WorkContext workContext;
     /**
      * Verifies callback wires are built and callback invocations are handled properly
      */
@@ -91,7 +93,7 @@
         wireService.createWires(clientComponent, sourceDefinition);
         container.register(clientComponent);
 
-        Connector connector = new ConnectorImpl(new JDKWireService(), null);
+        Connector connector = new ConnectorImpl(new JDKWireService(), null, scheduler , workContext);
 
         connector.connect(clientComponent);
         FooClient client = (FooClient) clientComponent.getServiceInstance();
@@ -126,7 +128,7 @@
         wireService.createWires(clientComponent2, sourceDefinition2);
         container.register(clientComponent2);
 
-        Connector connector = new ConnectorImpl(new JDKWireService(), null);
+        Connector connector = new ConnectorImpl(new JDKWireService(), null, scheduler, workContext);
         connector.connect(clientComponent1);
         connector.connect(clientComponent2);
         FooClient client1 = (FooClient) clientComponent1.getServiceInstance();
@@ -244,7 +246,7 @@
         expectLastCall().andReturn(container).anyTimes();
         replay(context);
 
-        WorkScheduler scheduler = createMock(WorkScheduler.class);
+        scheduler = createMock(WorkScheduler.class);
         scheduler.scheduleWork(isA(Runnable.class));
         expectLastCall().andStubAnswer(new IAnswer<Object>() {
             public Object answer() throws Throwable {
@@ -256,7 +258,7 @@
         replay(scheduler);
 
         builder = new JavaComponentBuilder();
-        WorkContextImpl workContext = new WorkContextImpl();
+        workContext = new WorkContextImpl();
         builder.setWorkContext(workContext);
         builder.setWireService(new JDKWireService(workContext, null));
         builder.setWorkScheduler(scheduler);

Modified: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/OneWayWireInvocationTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/OneWayWireInvocationTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/OneWayWireInvocationTestCase.java (original)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/implementation/java/integration/component/OneWayWireInvocationTestCase.java Thu Sep 28 22:31:27 2006
@@ -74,7 +74,7 @@
         method.setAccessible(true);
         InboundWire inboundWire = createMock(InboundWire.class);
         AsyncJavaTargetInvoker invoker =
-            new AsyncJavaTargetInvoker(method, inboundWire, component, scheduler, null, context);
+            new AsyncJavaTargetInvoker(method, inboundWire, component, null, context);
         InboundWire wire = createServiceWire("foo", AsyncTarget.class, null);
         Map<Operation<?>, InboundInvocationChain> chains = wire.getInvocationChains();
         InboundInvocationChain chain = chains.get(wire.getServiceContract().getOperations().get("invoke"));

Added: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java?view=auto&rev=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java (added)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java Thu Sep 28 22:31:27 2006
@@ -0,0 +1,45 @@
+package org.apache.tuscany.core.wire;
+
+import org.apache.tuscany.spi.component.WorkContext;
+import org.apache.tuscany.spi.services.work.WorkScheduler;
+import org.apache.tuscany.spi.wire.Interceptor;
+import org.apache.tuscany.spi.wire.Message;
+import org.apache.tuscany.spi.wire.MessageImpl;
+
+import junit.framework.TestCase;
+import org.easymock.EasyMock;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.getCurrentArguments;
+import static org.easymock.EasyMock.isA;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import org.easymock.IAnswer;
+
+/**
+ * @version $Rev$ $Date$
+ */
+public class NonBlockingBridgingInterceptorTestCase extends TestCase {
+
+    public void testInvoke() throws Exception {
+        WorkScheduler scheduler = createMock(WorkScheduler.class);
+        scheduler.scheduleWork(isA(Runnable.class));
+        expectLastCall().andStubAnswer(new IAnswer<Object>() {
+            public Object answer() throws Throwable {
+                Runnable runnable = (Runnable) getCurrentArguments()[0];
+                runnable.run();
+                return null;
+            }
+        });
+        replay(scheduler);
+        WorkContext context = createMock(WorkContext.class);
+        Message msg = new MessageImpl();
+        Interceptor next = EasyMock.createMock(Interceptor.class);
+        EasyMock.expect(next.invoke(EasyMock.eq(msg))).andReturn(msg);
+        EasyMock.replay(next);
+        Interceptor interceptor = new NonBlockingBridgingInterceptor(scheduler, context, next);
+        interceptor.invoke(msg);
+        verify(next);
+    }
+
+}

Propchange: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/NonBlockingBridgingInterceptorTestCase.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java?view=auto&rev=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java (added)
+++ incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java Thu Sep 28 22:31:27 2006
@@ -0,0 +1,26 @@
+package org.apache.tuscany.core.wire;
+
+import org.apache.tuscany.spi.wire.Interceptor;
+import org.apache.tuscany.spi.wire.Message;
+import org.apache.tuscany.spi.wire.MessageImpl;
+
+import org.easymock.EasyMock;
+import static org.easymock.EasyMock.verify;
+import junit.framework.TestCase;
+
+/**
+ * @version $Rev$ $Date$
+ */
+public class SynchronousBridgingInterceptorTestCase extends TestCase {
+
+    public void testInvoke() throws Exception {
+        Message msg = new MessageImpl();
+        Interceptor next = EasyMock.createMock(Interceptor.class);
+        EasyMock.expect(next.invoke(EasyMock.eq(msg))).andReturn(msg);
+        EasyMock.replay(next);
+        Interceptor interceptor = new SynchronousBridgingInterceptor(next);
+        interceptor.invoke(msg);
+        verify(next);
+    }
+
+}

Propchange: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/tuscany/java/sca/kernel/core/src/test/java/org/apache/tuscany/core/wire/SynchronousBridgingInterceptorTestCase.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/tuscany/java/sca/kernel/spi/src/main/java/org/apache/tuscany/spi/builder/Connector.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/kernel/spi/src/main/java/org/apache/tuscany/spi/builder/Connector.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/kernel/spi/src/main/java/org/apache/tuscany/spi/builder/Connector.java (original)
+++ incubator/tuscany/java/sca/kernel/spi/src/main/java/org/apache/tuscany/spi/builder/Connector.java Thu Sep 28 22:31:27 2006
@@ -45,6 +45,7 @@
      * @param optimizable if the bridge may be optimized
      * @throws BuilderConfigException
      */
-    void connect(InboundWire inbound, OutboundWire outbound, boolean optimizable) throws BuilderConfigException;
+    void connect(InboundWire inbound, OutboundWire outbound, boolean optimizable)
+        throws BuilderConfigException;
 
 }

Modified: incubator/tuscany/java/sca/runtime/osgi/src/main/resources/META-INF/sca/osgibinding.scdl
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/runtime/osgi/src/main/resources/META-INF/sca/osgibinding.scdl?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/runtime/osgi/src/main/resources/META-INF/sca/osgibinding.scdl (original)
+++ incubator/tuscany/java/sca/runtime/osgi/src/main/resources/META-INF/sca/osgibinding.scdl Thu Sep 28 22:31:27 2006
@@ -20,7 +20,7 @@
            name="binding.osgi">
 
     <component name="osgi.implementationLoader">
-        <system:implementation.system class="org.apache.tuscany.osgi.binding.OSGiBindingLoader"/>
+        <system:implementation.system class="org.apache.tuscany.osgOSGiBindingBuilderi.binding.OSGiBindingLoader"/>
     </component>
     <component name="osgi.bindingBuilder">
         <system:implementation.system class="org.apache.tuscany.osgi.binding.OSGiBindingBuilder"/>

Modified: incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncGroovyInvoker.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncGroovyInvoker.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncGroovyInvoker.java (original)
+++ incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncGroovyInvoker.java Thu Sep 28 22:31:27 2006
@@ -18,20 +18,11 @@
  */
 package org.apache.tuscany.container.groovy;
 
-import java.lang.reflect.InvocationTargetException;
-
-import org.osoa.sca.CompositeContext;
-import org.osoa.sca.CurrentCompositeContext;
-import org.osoa.sca.SCA;
-import org.osoa.sca.ServiceRuntimeException;
-
 import org.apache.tuscany.spi.component.TargetException;
 import org.apache.tuscany.spi.component.WorkContext;
-import org.apache.tuscany.spi.services.work.WorkScheduler;
 import org.apache.tuscany.spi.wire.InboundWire;
 import org.apache.tuscany.spi.wire.InvocationRuntimeException;
 import org.apache.tuscany.spi.wire.Message;
-import org.apache.tuscany.spi.wire.TargetInvoker;
 
 /**
  * Responsible for performing a non-blocking dispatch on a Groovy component implementation instance
@@ -40,95 +31,49 @@
  */
 public class AsyncGroovyInvoker extends GroovyInvoker {
 
-    private static final ContextBinder BINDER = new ContextBinder();
-    private static final Message RESPONSE = new AsyncGroovyInvoker.ImmutableMessage();
-
     private InboundWire wire;
-    private WorkScheduler workScheduler;
     private AsyncMonitor monitor;
     private WorkContext workContext;
     private Object target;
-    private Object messageId;
 
     /**
      * Creates a new invoker
      *
-     * @param operation     the operation the invoker is associated with
+     * @param operation   the operation the invoker is associated with
      * @param wire
-     * @param component     the target component
-     * @param workScheduler the work scheduler to run the invocation
-     * @param monitor       the monitor to pass events to
+     * @param component   the target component
+     * @param monitor     the monitor to pass events to
      * @param workContext
      */
     public AsyncGroovyInvoker(String operation,
                               InboundWire wire,
                               GroovyAtomicComponent component,
-                              WorkScheduler workScheduler,
                               AsyncMonitor monitor,
                               WorkContext workContext) {
         super(operation, component);
         this.wire = wire;
-        this.workScheduler = workScheduler;
         this.monitor = monitor;
         this.workContext = workContext;
     }
 
-    // Override invocation methods to defer invocation to work item
-    // Both methods return null to indicate asynchrony; result will
-    // be conveyed by callback
-    @Override
-    public Object invokeTarget(final Object payload) throws InvocationTargetException {
-        final CompositeContext currentContext = CurrentCompositeContext.getContext();
-        // Schedule the invocation of the next interceptor in a new Work instance
-        try {
-            workScheduler.scheduleWork(new Runnable() {
-                private Object currentMessageId = messageId;
-
-                public void run() {
-                    workContext.setCurrentMessageId(null);
-                    workContext.setCurrentCorrelationId(currentMessageId);
-                    CompositeContext oldContext = CurrentCompositeContext.getContext();
-                    try {
-                        BINDER.setContext(currentContext);
-                        // REVIEW response must be null for one-way and non-null for callback
-                        AsyncGroovyInvoker.super.invokeTarget(payload);
-                    } catch (Exception e) {
-                        // REVIEW uncomment when it is available
-                        // monitor.executionError(e);
-                        e.printStackTrace();
-                    } finally {
-                        BINDER.setContext(oldContext);
-                    }
-                }
-            });
-        } catch (Exception e) {
-            throw new ServiceRuntimeException(e);
-        }
-        return RESPONSE;
-    }
-
     public Message invoke(Message msg) throws InvocationRuntimeException {
-        // can't just call overriden invoke because it would bypass async
         try {
-            messageId = msg.getMessageId();
+            Object messageId = msg.getMessageId();
             wire.addMapping(messageId, msg.getFromAddress());
-            return (Message) invokeTarget(msg.getBody());
-        } catch (InvocationTargetException e) {
-            // FIXME need to log exceptions
-            e.printStackTrace();
+            workContext.setCurrentMessageId(null);
+            workContext.setCurrentCorrelationId(messageId);
+            invokeTarget(msg.getBody());
+            // async so no return value
             return null;
         } catch (Throwable e) {
             // FIXME need to log exceptions
-            e.printStackTrace();
+            monitor.executionError(e);
             return null;
         }
     }
 
     public AsyncGroovyInvoker clone() throws CloneNotSupportedException {
-        AsyncGroovyInvoker invoker = (AsyncGroovyInvoker) super.clone();
-        invoker.workScheduler = this.workScheduler;
-        invoker.monitor = this.monitor;
-        return invoker;
+        return (AsyncGroovyInvoker) super.clone();
     }
 
     /**
@@ -142,74 +87,6 @@
                 target = component.getTargetInstance();
             }
             return target;
-        }
-    }
-
-    private static class ContextBinder extends SCA {
-        public void setContext(CompositeContext context) {
-            setCompositeContext(context);
-        }
-
-        public void start() {
-            throw new AssertionError();
-        }
-
-        public void stop() {
-            throw new AssertionError();
-        }
-    }
-
-    /**
-     * A dummy message passed back on an invocation
-     */
-    private static class ImmutableMessage implements Message {
-
-        public Object getBody() {
-            return null;
-        }
-
-        public void setBody(Object body) {
-            throw new UnsupportedOperationException();
-        }
-
-        public void setTargetInvoker(TargetInvoker invoker) {
-            throw new UnsupportedOperationException();
-        }
-
-        public TargetInvoker getTargetInvoker() {
-            return null;
-        }
-
-        public Object getFromAddress() {
-            return null;
-        }
-
-        public void setFromAddress(Object fromAddress) {
-            throw new UnsupportedOperationException();
-        }
-
-        public Object getMessageId() {
-            return null;
-        }
-
-        public void setMessageId(Object messageId) {
-            throw new UnsupportedOperationException();
-        }
-
-        public Object getCorrelationId() {
-            return null;
-        }
-
-        public void setCorrelationId(Object correlationId) {
-            throw new UnsupportedOperationException();
-        }
-
-        public boolean isFault() {
-            return false;
-        }
-
-        public void setBodyWithFault(Object fault) {
-            throw new UnsupportedOperationException();
         }
     }
 }

Modified: incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncMonitor.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncMonitor.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncMonitor.java (original)
+++ incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/AsyncMonitor.java Thu Sep 28 22:31:27 2006
@@ -29,6 +29,6 @@
     /**
      * Logs an exception thrown during an invocation
      */
-    void executionError(Exception e);
+    void executionError(Throwable e);
 
 }

Modified: incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/GroovyAtomicComponent.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/GroovyAtomicComponent.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/GroovyAtomicComponent.java (original)
+++ incubator/tuscany/java/sca/services/containers/container.groovy/src/main/java/org/apache/tuscany/container/groovy/GroovyAtomicComponent.java Thu Sep 28 22:31:27 2006
@@ -68,7 +68,7 @@
     }
 
     public TargetInvoker createAsyncTargetInvoker(InboundWire wire, Operation operation) {
-        return new AsyncGroovyInvoker(operation.getName(), wire, this, workScheduler, monitor, workContext);
+        return new AsyncGroovyInvoker(operation.getName(), wire, this, monitor, workContext);
     }
 
     public Object createInstance() throws ObjectCreationException {

Modified: incubator/tuscany/java/sca/services/containers/container.groovy/src/test/java/org/apache/tuscany/container/groovy/AsyncInvokerTestCase.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/services/containers/container.groovy/src/test/java/org/apache/tuscany/container/groovy/AsyncInvokerTestCase.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/services/containers/container.groovy/src/test/java/org/apache/tuscany/container/groovy/AsyncInvokerTestCase.java (original)
+++ incubator/tuscany/java/sca/services/containers/container.groovy/src/test/java/org/apache/tuscany/container/groovy/AsyncInvokerTestCase.java Thu Sep 28 22:31:27 2006
@@ -68,7 +68,7 @@
         Method method = AsyncTarget.class.getMethod("invoke");
         method.setAccessible(true);
         InboundWire wire = createMock(InboundWire.class);
-        AsyncGroovyInvoker invoker = new AsyncGroovyInvoker("invoke", wire, component, scheduler, monitor, context);
+        AsyncGroovyInvoker invoker = new AsyncGroovyInvoker("invoke", wire, component, monitor, context);
         Message msg = new MessageImpl();
         invoker.invoke(msg);
         verify(instance);

Modified: incubator/tuscany/java/sca/test/src/main/java/org/apache/tuscany/test/ArtifactFactory.java
URL: http://svn.apache.org/viewvc/incubator/tuscany/java/sca/test/src/main/java/org/apache/tuscany/test/ArtifactFactory.java?view=diff&rev=451133&r1=451132&r2=451133
==============================================================================
--- incubator/tuscany/java/sca/test/src/main/java/org/apache/tuscany/test/ArtifactFactory.java (original)
+++ incubator/tuscany/java/sca/test/src/main/java/org/apache/tuscany/test/ArtifactFactory.java Thu Sep 28 22:31:27 2006
@@ -54,7 +54,7 @@
     }
 
     public static Connector createConnector() {
-        return new ConnectorImpl(createWireService(), null);
+        return new ConnectorImpl(createWireService(), null, null, null);
     }
 
     public static WireService createWireService() {



---------------------------------------------------------------------
To unsubscribe, e-mail: tuscany-commits-unsubscribe@ws.apache.org
For additional commands, e-mail: tuscany-commits-help@ws.apache.org