You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2009/05/13 10:51:02 UTC

svn commit: r774252 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/ camel-core/src/main/java/org/apache/camel/impl/ camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/processor/ camel-core/src/main/...

Author: davsclaus
Date: Wed May 13 08:51:02 2009
New Revision: 774252

URL: http://svn.apache.org/viewvc?rev=774252&view=rev
Log:
CAMEL-1604: First cut for onCompletion DSL.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java   (with props)
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java   (with props)
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkSynchronizationAdapterTest.java
      - copied, changed from r773783, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java   (contents, props changed)
      - copied, changed from r774235, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml   (contents, props changed)
      - copied, changed from r774235, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.xml
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Synchronization.java
    camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java?rev=774252&r1=774251&r2=774252&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java Wed May 13 08:51:02 2009
@@ -55,15 +55,20 @@
     String FILE_PATH = "CamelFilePath";
     String FILE_PARENT = "CamelFileParent";
 
+    String INTERCEPTED_ENDPOINT = "CamelInterceptedEndpoint";
+
     String LOOP_INDEX = "CamelLoopIndex";
     String LOOP_SIZE = "CamelLoopSize";
 
+    String MULTICAST_INDEX = "CamelMulticastIndex";
+
+    String ON_COMPLETION = "CamelOnCompletion";
+
     String ROUTE_STOP = "CamelRouteStop";
 
     String REDELIVERED = "CamelRedelivered";
     String REDELIVERY_COUNTER = "CamelRedeliveryCounter";
 
-    String MULTICAST_INDEX = "CamelMulticastIndex";
     String SPLIT_INDEX = "CamelSplitIndex";
     String SPLIT_SIZE = "CamelSplitSize";
 
@@ -75,8 +80,6 @@
     String TRANSACTED = "CamelTransacted";
     String ROLLBACK_ONLY = "CamelRollbackOnly";
 
-    String INTERCEPTED_ENDPOINT = "CamelInterceptedEndpoint";
-
     /**
      * Returns the {@link ExchangePattern} (MEP) of this exchange.
      *

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java Wed May 13 08:51:02 2009
@@ -0,0 +1,37 @@
+/**
+ * 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.camel.impl;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.spi.Synchronization;
+
+/**
+ * Simple {@link Synchronization} adapter with empty methods for easier overriding
+ * of single methods.
+ *
+ * @version $Revision$
+ */
+public class SynchronizationAdapter implements Synchronization {
+
+    public void onComplete(Exchange exchange) {
+        // noop
+    }
+
+    public void onFailure(Exchange exchange) {
+        // noop
+    }
+}

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/SynchronizationAdapter.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java Wed May 13 08:51:02 2009
@@ -0,0 +1,114 @@
+/**
+ * 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.camel.model;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.Processor;
+import org.apache.camel.processor.OnCompletionProcessor;
+import org.apache.camel.spi.RouteContext;
+
+/**
+ * Represents an XML <onCompletion/> element
+ *
+ * @version $Revision$
+ */
+@XmlRootElement(name = "onCompletion")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class OnCompletionDefinition extends OutputDefinition<OnCompletionDefinition> {
+
+    @XmlAttribute(required = false)
+    private Boolean onCompleteOnly = Boolean.TRUE;
+    @XmlAttribute(required = false)
+    private Boolean onFailureOnly = Boolean.TRUE;
+
+    public OnCompletionDefinition() {
+    }
+
+    @Override
+    public String toString() {
+        return "Synchronize[" + getOutputs() + "]";
+    }
+
+    @Override
+    public String getShortName() {
+        return "onCompletion";
+    }
+
+    @Override
+    public String getLabel() {
+        return "onCompletion";
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor childProcessor = createOutputsProcessor(routeContext);
+        return new OnCompletionProcessor(childProcessor, onCompleteOnly, onFailureOnly);
+    }
+
+    @Override
+    public ProcessorDefinition<? extends ProcessorDefinition> end() {
+        // pop parent block, as we added outself as block to parent when synchronized was defined in the route
+        getParent().popBlock();
+        return super.end();
+    }
+
+    /**
+     * Will only synchronize when the {@link org.apache.camel.Exchange} completed succesfully (no errors).
+     *
+     * @return the builder
+     */
+    public OutputDefinition onCompleteOnly() {
+        // must define return type as OutputDefinition and not this type to avoid end user being able
+        // to invoke onFailureOnly/onCompleteOnly more than once
+        setOnCompleteOnly(Boolean.TRUE);
+        setOnFailureOnly(Boolean.FALSE);
+        return this;
+    }
+
+    /**
+     * Will only synchronize when the {@link org.apache.camel.Exchange} ended with failure (exception or FAULT message).
+     *
+     * @return the builder
+     */
+    public OutputDefinition onFailureOnly() {
+        // must define return type as OutputDefinition and not this type to avoid end user being able
+        // to invoke onFailureOnly/onCompleteOnly more than once
+        setOnCompleteOnly(Boolean.FALSE);
+        setOnFailureOnly(Boolean.TRUE);
+        return this;
+    }
+
+    public Boolean getOnCompleteOnly() {
+        return onCompleteOnly;
+    }
+
+    public void setOnCompleteOnly(Boolean onCompleteOnly) {
+        this.onCompleteOnly = onCompleteOnly;
+    }
+
+    public Boolean getOnFailureOnly() {
+        return onFailureOnly;
+    }
+
+    public void setOnFailureOnly(Boolean onFailureOnly) {
+        this.onFailureOnly = onFailureOnly;
+    }
+}

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java?rev=774252&r1=774251&r2=774252&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java Wed May 13 08:51:02 2009
@@ -1820,20 +1820,42 @@
      * Enriches an exchange with additional data obtained from a
      * <code>resourceUri</code>.
      * 
-     * @param resourceUri
-     *            URI of resource endpoint for obtaining additional data.
-     * @param aggregationStrategy
-     *            aggregation strategy to aggregate input data and additional
-     *            data.
-     * @return this processor type
+     * @param resourceUri           URI of resource endpoint for obtaining additional data.
+     * @param aggregationStrategy   aggregation strategy to aggregate input data and additional data.
+     * @return the builder
      * @see org.apache.camel.processor.Enricher
      */
     @SuppressWarnings("unchecked")
     public Type enrich(String resourceUri, AggregationStrategy aggregationStrategy) {
         addOutput(new EnrichDefinition(aggregationStrategy, resourceUri));
-        return (Type)this;
+        return (Type) this;
+    }
+
+    /**
+     * Adds a onComplection {@link org.apache.camel.spi.Synchronization} hook that invoke this route as
+     * a callback when the {@link org.apache.camel.Exchange} has finished being processed.
+     * The hook invoke callbacks for either onComplete or onFailure.
+     * <p/>
+     * Will by default always trigger when the {@link org.apache.camel.Exchange} is complete
+     * (either with success or failed).
+     * <br/>
+     * You can limit the callback to either onComplete or onFailure but invoking the nested
+     * builder method.
+     * <p/>
+     * For onFailure the caused exception is stored as a property on the {@link org.apache.camel.Exchange}
+     * with the key {@link org.apache.camel.Exchange#EXCEPTION_CAUGHT}.
+     *
+     * @return the builder
+     */
+    @SuppressWarnings("unchecked")
+    public OnCompletionDefinition onCompletion() {
+        OnCompletionDefinition answer = new OnCompletionDefinition();
+        popBlock();
+        addOutput(answer);
+        pushBlock(answer);
+        return answer;
     }
-    
+
     // DataFormat support
     // -------------------------------------------------------------------------
 

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java Wed May 13 08:51:02 2009
@@ -0,0 +1,156 @@
+/**
+ * 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.camel.processor;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.ServiceSupport;
+import org.apache.camel.impl.SynchronizationAdapter;
+import org.apache.camel.util.ServiceHelper;
+import org.apache.camel.util.concurrent.ExecutorServiceHelper;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * @version $Revision$
+ */
+public class OnCompletionProcessor extends ServiceSupport implements Processor {
+
+    private static final transient Log LOG = LogFactory.getLog(OnCompletionProcessor.class);
+    private ExecutorService executorService;
+    private Processor processor;
+    private boolean onComplete;
+    private boolean onFailure;
+
+    public OnCompletionProcessor(Processor processor, boolean onComplete, boolean onFailure) {
+        this.processor = processor;
+        this.onComplete = onComplete;
+        this.onFailure = onFailure;
+    }
+
+    protected void doStart() throws Exception {
+        ServiceHelper.startService(processor);
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        if (executorService != null) {
+            executorService.shutdown();
+        }
+        ServiceHelper.stopService(processor);
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        if (processor == null) {
+            return;
+        }
+
+        if (!onComplete && !onFailure) {
+            // no need to register callbacks not to be used
+            return;
+        }
+
+        // register callback
+        exchange.getUnitOfWork().addSynchronization(new SynchronizationAdapter() {
+            @Override
+            public void onComplete(Exchange exchange) {
+                if (!onComplete) {
+                    return;
+                }
+
+                // must use a copy as we dont want it to cause side effects of the original exchange
+                final Exchange copy = exchange.newCopy();
+                // set MEP to InOnly as this wire tap is a fire and forget
+                copy.setPattern(ExchangePattern.InOnly);
+                // add a header flag to indicate its a on completion exchange
+                copy.setProperty(Exchange.ON_COMPLETION, Boolean.TRUE);
+
+                getExecutorService().submit(new Callable<Exchange>() {
+                    public Exchange call() throws Exception {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Processing onCompleteOnly: " + copy);
+                        }
+                        processor.process(copy);
+                        return copy;
+                    }
+                });
+            }
+
+            public void onFailure(Exchange exchange) {
+                if (!onFailure) {
+                    return;
+                }
+
+                // must use a copy as we dont want it to cause side effects of the original exchange
+                final Exchange copy = exchange.newCopy();
+                // set MEP to InOnly as this wire tap is a fire and forget
+                copy.setPattern(ExchangePattern.InOnly);
+                // add a header flag to indicate its a on completion exchange
+                copy.setProperty(Exchange.ON_COMPLETION, Boolean.TRUE);
+                // must remove exception otherwise onFaulure routing will fail as well
+                // the caused exception is stored as a property (Exchange.EXCEPTION_CAUGHT) on the exchange
+                copy.setException(null);
+
+                getExecutorService().submit(new Callable<Exchange>() {
+                    public Exchange call() throws Exception {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Processing onFailureOnly: " + copy);
+                        }
+
+                        processor.process(copy);
+                        return copy;
+                    }
+                });
+            }
+
+            @Override
+            public String toString() {
+                if (onComplete && onFailure) {
+                    return "onCompleteOrFailure";
+                } else if (onComplete) {
+                    return "onCompleteOnly";
+                } else {
+                    return "onFailureOnly";
+                }
+            }
+        });
+    }
+
+    public ExecutorService getExecutorService() {
+        if (executorService == null) {
+            executorService = createExecutorService();
+        }
+        return executorService;
+    }
+
+    private ExecutorService createExecutorService() {
+        return ExecutorServiceHelper.newScheduledThreadPool(5, this.toString(), true);
+    }
+
+    public void setExecutorService(ExecutorService executorService) {
+        this.executorService = executorService;
+    }
+
+    @Override
+    public String toString() {
+        return "OnCompletionProcessor";
+    }
+}

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/OnCompletionProcessor.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Synchronization.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Synchronization.java?rev=774252&r1=774251&r2=774252&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Synchronization.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Synchronization.java Wed May 13 08:51:02 2009
@@ -24,7 +24,6 @@
  * <a href="http://static.springframework.org/spring/docs/2.5.x/api/org/springframework/transaction/
  * support/TransactionSynchronization.html">TransactionSynchronization</a>
  *
- * @deprecated a new synchronization API is planned for Camel 2.0
  * @version $Revision$
  */
 public interface Synchronization {

Modified: camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index?rev=774252&r1=774251&r2=774252&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index (original)
+++ camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index Wed May 13 08:51:02 2009
@@ -37,6 +37,7 @@
 LoopDefinition
 MarshalDefinition
 MulticastDefinition
+OnCompletionDefinition
 OnExceptionDefinition
 OptionalIdentifiedType
 OtherwiseDefinition

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java Wed May 13 08:51:02 2009
@@ -0,0 +1,67 @@
+/**
+ * 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.camel.processor;
+
+import org.apache.camel.CamelExecutionException;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision$
+ */
+public class OnCompletionOnCompleteOnlyTest extends OnCompletionTest {
+
+    public void testSynchronizeFailure() throws Exception {
+        // do not expect a message since we only do onCompleteOnly
+        getMockEndpoint("mock:sync").expectedMessageCount(0);
+
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(0);
+
+        try {
+            template.sendBody("direct:start", "Kabom");
+            fail("Should throw exception");
+        } catch (CamelExecutionException e) {
+            assertEquals("Kabom", e.getCause().getMessage());
+        }
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // START SNIPPET: e1
+                from("direct:start")
+                    // here we qualify onCompletion to only invoke when the exchange completed with success
+                    // if the exchange failed this onCompletion route will NOT be routed then
+                    .onCompletion().onCompleteOnly()
+                        .to("log:sync")
+                        .to("mock:sync")
+                    // must use end to denote the end of the onCompletion route
+                    .end()
+                    // here the original route contiues
+                    .process(new MyProcessor())
+                    .to("mock:result");
+                // END SNIPPET: e1
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnCompleteOnlyTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java Wed May 13 08:51:02 2009
@@ -0,0 +1,55 @@
+/**
+ * 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.camel.processor;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision$
+ */
+public class OnCompletionOnFailureOnlyTest extends OnCompletionTest {
+
+    public void testSynchronizeComplete() throws Exception {
+        // do not expect a message since we only do onFailureOnly
+        getMockEndpoint("mock:sync").expectedMessageCount(0);
+
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .onCompletion().onFailureOnly()
+                        .to("log:sync")
+                        .to("mock:sync")
+                    .end()
+                    .process(new MyProcessor())
+                    .to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionOnFailureOnlyTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java Wed May 13 08:51:02 2009
@@ -0,0 +1,94 @@
+/**
+ * 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.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.CamelExecutionException;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision$
+ */
+public class OnCompletionTest extends ContextTestSupport {
+
+    public void testSynchronizeComplete() throws Exception {
+        getMockEndpoint("mock:sync").expectedBodiesReceived("Bye World");
+        getMockEndpoint("mock:sync").expectedPropertyReceived(Exchange.ON_COMPLETION, true);
+
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testSynchronizeFailure() throws Exception {
+        getMockEndpoint("mock:sync").expectedMessageCount(1);
+        getMockEndpoint("mock:sync").expectedPropertyReceived(Exchange.ON_COMPLETION, true);
+
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(0);
+
+        try {
+            template.sendBody("direct:start", "Kabom");
+            fail("Should throw exception");
+        } catch (CamelExecutionException e) {
+            assertEquals("Kabom", e.getCause().getMessage());
+        }
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // START SNIPPET: e1
+                from("direct:start")
+                    .onCompletion()
+                        // this route is only invoked when the original route is complete as a kind
+                        // of completion callback
+                        .to("log:sync")
+                        .to("mock:sync")
+                    // must use end to denote the end of the onCompletion route
+                    .end()
+                    // here the original route contiues
+                    .process(new MyProcessor())
+                    .to("mock:result");
+                // END SNIPPET: e1
+            }
+        };
+    }
+
+    public static class MyProcessor implements Processor {
+
+        public MyProcessor() {
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            if ("Kabom".equals(exchange.getIn().getBody())) {
+                throw new IllegalArgumentException("Kabom");
+            }
+            exchange.getIn().setBody("Bye World");
+        }
+    }
+}

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/OnCompletionTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkSynchronizationAdapterTest.java (from r773783, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkSynchronizationAdapterTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkSynchronizationAdapterTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkTest.java&r1=773783&r2=774252&rev=774252&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/UnitOfWorkSynchronizationAdapterTest.java Wed May 13 08:51:02 2009
@@ -16,80 +16,15 @@
  */
 package org.apache.camel.processor;
 
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.spi.Synchronization;
+import org.apache.camel.impl.SynchronizationAdapter;
 
 /**
  * @version $Revision$
  */
-public class UnitOfWorkTest extends ContextTestSupport {
-    protected Synchronization synchronization;
-    protected Exchange completed;
-    protected Exchange failed;
-    protected String uri = "direct:foo";
-    protected CountDownLatch doneLatch = new CountDownLatch(1);
-
-    public void testSuccess() throws Exception {
-        sendMessage();
-
-        assertTrue("Exchange did not complete.", doneLatch.await(5, TimeUnit.SECONDS));
-        assertNull("Should not have failed", failed);
-        assertNotNull("Should have received completed notification", completed);
-
-        log.info("Received completed: " + completed);
-    }
-
-    public void testFail() throws Exception {
-        sendMessage();
-
-        assertTrue("Exchange did not complete.", doneLatch.await(5, TimeUnit.SECONDS));
-        assertNull("Should not have completed", completed);
-        assertNotNull("Should have received failed notification", failed);
-
-        log.info("Received fail: " + failed);
-    }
-
-    public void testException() throws Exception {
-        sendMessage();
-
-        assertTrue("Exchange did not complete.", doneLatch.await(5, TimeUnit.SECONDS));
-        assertNull("Should not have completed", completed);
-        assertNotNull("Should have received failed notification", failed);
-
-        log.info("Received fail: " + failed);
-    }
-
-    @Override
-    protected void setUp() throws Exception {
-        synchronization = new Synchronization() {
-            public void onComplete(Exchange exchange) {
-                completed = exchange;
-                doneLatch.countDown();
-            }
-
-            public void onFailure(Exchange exchange) {
-                failed = exchange;
-                doneLatch.countDown();
-            }
-        };
-
-        super.setUp();
-    }
-
-    protected void sendMessage() throws InterruptedException {
-        
-        template.send(uri, new Processor() {
-            public void process(Exchange exchange) throws Exception {
-                exchange.getIn().setBody("<hello>world!</hello>");
-            }
-        });
-    }
+public class UnitOfWorkSynchronizationAdapterTest extends UnitOfWorkTest {
 
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
@@ -98,7 +33,22 @@
                 from("direct:foo").process(new Processor() {
                     public void process(Exchange exchange) throws Exception {
                         log.info("Received: " + exchange);
-                        exchange.getUnitOfWork().addSynchronization(synchronization);
+
+                        exchange.getUnitOfWork().addSynchronization(new SynchronizationAdapter() {
+                            @Override
+                            public void onComplete(Exchange exchange) {
+                                completed = exchange;
+                                doneLatch.countDown();
+                            }
+                        });
+
+                        exchange.getUnitOfWork().addSynchronization(new SynchronizationAdapter() {
+                            @Override
+                            public void onFailure(Exchange exchange) {
+                                failed = exchange;
+                                doneLatch.countDown();
+                            }
+                        });
 
                         String name = getName();
                         if (name.equals("testFail")) {

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java Wed May 13 08:51:02 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.camel.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.OnCompletionOnCompleteOnlyTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringOnCompletionOnCompleteOnlyTest extends OnCompletionOnCompleteOnlyTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml");
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java?rev=774252&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java Wed May 13 08:51:02 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.camel.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.OnCompletionOnFailureOnlyTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringOnCompletionOnFailureOnlyTest extends OnCompletionOnFailureOnlyTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml");
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java (from r774235, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.java&r1=774235&r2=774252&rev=774252&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java Wed May 13 08:51:02 2009
@@ -17,16 +17,16 @@
 package org.apache.camel.spring.processor;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.processor.DeadLetterChannelUseOriginalInBodyTest;
+import org.apache.camel.processor.OnCompletionTest;
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
 /**
  * @version $Revision$
  */
-public class SpringDeadLetterChannelUseOriginalBodyTest extends DeadLetterChannelUseOriginalInBodyTest {
+public class SpringOnCompletionTest extends OnCompletionTest {
 
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringOnCompletionTest.xml");
     }
 
-}
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringOnCompletionTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml?rev=774252&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml Wed May 13 08:51:02 2009
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <bean id="myProcessor" class="org.apache.camel.processor.OnCompletionTest$MyProcessor"/>
+
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <!-- START SNIPPET: e1 -->
+        <route>
+            <from uri="direct:start"/>
+            <!-- this onCompletion block will only be executed when the exchange is done being routed -->
+            <!-- this callback is only triggered when the exchange completed with no errors, as we have onCompleteOnly=true -->
+            <onCompletion onCompleteOnly="true">
+                <to uri="log:sync"/>
+                <to uri="mock:sync"/>
+            </onCompletion>
+            <process ref="myProcessor"/>
+            <to uri="mock:result"/>
+        </route>
+        <!-- END SNIPPET: e1 -->
+    </camelContext>
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnCompleteOnlyTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml?rev=774252&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml Wed May 13 08:51:02 2009
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <bean id="myProcessor" class="org.apache.camel.processor.OnCompletionTest$MyProcessor"/>
+
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <!-- START SNIPPET: e1 -->
+        <route>
+            <from uri="direct:start"/>
+            <!-- this onCompletion block will only be executed when the exchange is done being routed -->
+            <!-- this callback is only triggered when the exchange failed, as we have onFailure=true -->
+            <onCompletion onFailureOnly="true">
+                <to uri="log:sync"/>
+                <to uri="mock:sync"/>
+            </onCompletion>
+            <process ref="myProcessor"/>
+            <to uri="mock:result"/>
+        </route>
+        <!-- END SNIPPET: e1 -->
+    </camelContext>
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionOnFailureOnlyTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml (from r774235, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.xml&r1=774235&r2=774252&rev=774252&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringDeadLetterChannelUseOriginalBodyTest.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml Wed May 13 08:51:02 2009
@@ -22,49 +22,22 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
     ">
 
-    <!-- START SNIPPET: e1 -->
-    <bean id="a" class="org.apache.camel.builder.DeadLetterChannelBuilder">
-        <!-- move failed messages to the mock:a dead letter queue -->
-        <property name="deadLetterUri" value="mock:a"/>
-        <!-- we mark all exchanges as handled when they are moved to the dead letter queue, so the client
-             does not receive an exception -->
-        <property name="handled" value="true"/>
-        <!-- use the original input body when moving to dead letter queue -->
-        <property name="useOriginalBody" value="true"/>
-        <property name="redeliveryPolicy" ref="myRedelivery"/>
-    </bean>
-
-    <bean id="myRedelivery" class="org.apache.camel.processor.RedeliveryPolicy">
-        <property name="maximumRedeliveries" value="2"/>
-        <property name="delay" value="0"/>
-        <property name="logStackTrace" value="false"/>
-    </bean>
-    <!-- END SNIPPET: e1 -->
-
-    <bean id="b" class="org.apache.camel.builder.DeadLetterChannelBuilder">
-        <!-- move failed messages to the mock:b dead letter queue -->
-        <property name="deadLetterUri" value="mock:b"/>
-        <!-- we mark all exchanges as handled when they are moved to the dead letter queue, so the client
-             does not receive an exception -->
-        <property name="handled" value="true"/>
-        <!-- do NOT use the original input body when moving to dead letter queue -->
-        <property name="useOriginalBody" value="false"/>
-        <property name="redeliveryPolicy" ref="myRedelivery"/>
-    </bean>
-
-    <bean id="myThrowProcessor" class="org.apache.camel.processor.DeadLetterChannelUseOriginalInBodyTest$MyThrowProcessor"/>
+    <bean id="myProcessor" class="org.apache.camel.processor.OnCompletionTest$MyProcessor"/>
 
     <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
-        <route errorHandlerRef="a">
-            <from uri="direct:a"/>
-            <setBody><simple>${in.body} World</simple></setBody>
-            <process ref="myThrowProcessor"/>
-        </route>
-
-        <route errorHandlerRef="b">
-            <from uri="direct:b"/>
-            <setBody><simple>${in.body} World</simple></setBody>
-            <process ref="myThrowProcessor"/>
+        <!-- START SNIPPET: e1 -->
+        <route>
+            <from uri="direct:start"/>
+            <!-- this onCompletion block will only be executed when the exchange is done being routed -->
+            <!-- this callback is always triggered even if the exchange failed -->
+            <onCompletion>
+                <!-- so this is a kinda like an after completion callback -->
+                <to uri="log:sync"/>
+                <to uri="mock:sync"/>
+            </onCompletion>
+            <process ref="myProcessor"/>
+            <to uri="mock:result"/>
         </route>
+        <!-- END SNIPPET: e1 -->
     </camelContext>
 </beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringOnCompletionTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml