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 2012/03/15 09:12:29 UTC

svn commit: r1300839 - in /camel/branches/camel-2.8.x: ./ camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalancerSetFaultTest.java

Author: davsclaus
Date: Thu Mar 15 08:12:29 2012
New Revision: 1300839

URL: http://svn.apache.org/viewvc?rev=1300839&view=rev
Log:
Failover EIP - Should use defensive copy of exchange before failover to avoid side effects

Added:
    camel/branches/camel-2.8.x/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalancerSetFaultTest.java
      - copied unchanged from r1300835, camel/branches/camel-2.9.x/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalancerSetFaultTest.java
Modified:
    camel/branches/camel-2.8.x/   (props changed)
    camel/branches/camel-2.8.x/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java

Propchange: camel/branches/camel-2.8.x/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Mar 15 08:12:29 2012
@@ -1,2 +1,2 @@
-/camel/branches/camel-2.9.x:1227549,1228229,1229567,1234054,1236672,1238942,1240157,1241006,1241489,1243052,1243058,1244875,1244877,1291871,1292116,1292389,1292726,1292769,1293082,1293935,1294044,1294589,1294914,1294978,1299165,1300806
-/camel/trunk:1226860,1227540,1228223,1229565,1234043,1236667,1238937,1240025,1240950,1240967,1241482,1243046,1243057,1244870,1244872,1291848,1292114,1292384,1292725,1292767,1293079,1293828,1293855,1294588,1294909,1294976,1298993,1300805
+/camel/branches/camel-2.9.x:1227549,1228229,1229567,1234054,1236672,1238942,1240157,1241006,1241489,1243052,1243058,1244875,1244877,1291871,1292116,1292389,1292726,1292769,1293082,1293935,1294044,1294589,1294914,1294978,1299165,1300806,1300835
+/camel/trunk:1226860,1227540,1228223,1229565,1234043,1236667,1238937,1240025,1240950,1240967,1241482,1243046,1243057,1244870,1244872,1291848,1292114,1292384,1292725,1292767,1293079,1293828,1293855,1294588,1294909,1294976,1298993,1300805,1300831

Propchange: camel/branches/camel-2.8.x/
------------------------------------------------------------------------------
Binary property 'svnmerge-integrated' - no diff available.

Modified: camel/branches/camel-2.8.x/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java
URL: http://svn.apache.org/viewvc/camel/branches/camel-2.8.x/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java?rev=1300839&r1=1300838&r2=1300839&view=diff
==============================================================================
--- camel/branches/camel-2.8.x/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java (original)
+++ camel/branches/camel-2.8.x/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java Thu Mar 15 08:12:29 2012
@@ -26,6 +26,7 @@ import org.apache.camel.Processor;
 import org.apache.camel.impl.converter.AsyncProcessorTypeConverter;
 import org.apache.camel.processor.Traceable;
 import org.apache.camel.util.AsyncProcessorHelper;
+import org.apache.camel.util.ExchangeHelper;
 import org.apache.camel.util.ObjectHelper;
 
 /**
@@ -86,6 +87,10 @@ public class FailOverLoadBalancer extend
      * @return <tt>true</tt> to failover
      */
     protected boolean shouldFailOver(Exchange exchange) {
+        if (exchange == null) {
+            return false;
+        }
+
         boolean answer = false;
 
         if (exchange.getException() != null) {
@@ -108,12 +113,15 @@ public class FailOverLoadBalancer extend
         return answer;
     }
 
-    public boolean process(Exchange exchange, AsyncCallback callback) {
+    public boolean process(final Exchange exchange, final AsyncCallback callback) {
         final List<Processor> processors = getProcessors();
 
         final AtomicInteger index = new AtomicInteger();
         final AtomicInteger attempts = new AtomicInteger();
         boolean first = true;
+        // use a copy of the original exchange before failover to avoid populating side effects
+        // directly into the original exchange
+        Exchange copy = null;
 
         // get the next processor
         if (isRoundRobin()) {
@@ -124,7 +132,7 @@ public class FailOverLoadBalancer extend
         }
         log.trace("Failover starting with endpoint index {}", index);
 
-        while (first || shouldFailOver(exchange)) {
+        while (first || shouldFailOver(copy)) {
             if (!first) {
                 attempts.incrementAndGet();
                 // are we exhausted by attempts?
@@ -153,12 +161,12 @@ public class FailOverLoadBalancer extend
                 }
             }
 
-            // try again but prepare exchange before we failover
-            prepareExchangeForFailover(exchange);
+            // try again but copy original exchange before we failover
+            copy = prepareExchangeForFailover(exchange);
             Processor processor = processors.get(index.get());
 
             // process the exchange
-            boolean sync = processExchange(processor, exchange, attempts, index, callback, processors);
+            boolean sync = processExchange(processor, exchange, copy, attempts, index, callback, processors);
 
             // continue as long its being processed synchronously
             if (!sync) {
@@ -171,8 +179,11 @@ public class FailOverLoadBalancer extend
             log.trace("Processing exchangeId: {} is continued being processed synchronously", exchange.getExchangeId());
         }
 
+        // and copy the current result to original so it will contain this result of this eip
+        if (copy != null) {
+            ExchangeHelper.copyResults(exchange, copy);
+        }
         log.debug("Failover complete for exchangeId: {} >>> {}", exchange.getExchangeId(), exchange);
-
         callback.done(true);
         return true;
     }
@@ -181,35 +192,23 @@ public class FailOverLoadBalancer extend
      * Prepares the exchange for failover
      *
      * @param exchange the exchange
+     * @return a copy of the exchange to use for failover
      */
-    protected void prepareExchangeForFailover(Exchange exchange) {
-        if (exchange.getException() != null) {
-            if (log.isDebugEnabled()) {
-                log.debug("Failover due {} for exchangeId: {}", exchange.getException().getMessage(), exchange.getExchangeId());
-            }
-
-            // clear exception so we can try failover
-            exchange.setException(null);
-        }
-
-        exchange.setProperty(Exchange.ERRORHANDLER_HANDLED, null);
-        exchange.setProperty(Exchange.FAILURE_HANDLED, null);
-        exchange.setProperty(Exchange.EXCEPTION_CAUGHT, null);
-        exchange.getIn().removeHeader(Exchange.REDELIVERED);
-        exchange.getIn().removeHeader(Exchange.REDELIVERY_COUNTER);
-        exchange.getIn().removeHeader(Exchange.REDELIVERY_MAX_COUNTER);
+    protected Exchange prepareExchangeForFailover(Exchange exchange) {
+        // use a copy of the exchange to avoid side effects on the original exchange
+        return ExchangeHelper.createCopy(exchange, true);
     }
 
-    private boolean processExchange(Processor processor, Exchange exchange,
+    private boolean processExchange(Processor processor, Exchange exchange, Exchange copy,
                                     AtomicInteger attempts, AtomicInteger index,
                                     AsyncCallback callback, List<Processor> processors) {
         if (processor == null) {
-            throw new IllegalStateException("No processors could be chosen to process " + exchange);
+            throw new IllegalStateException("No processors could be chosen to process " + copy);
         }
-        log.debug("Processing failover at attempt {} for {}", attempts, exchange);
+        log.debug("Processing failover at attempt {} for {}", attempts, copy);
 
         AsyncProcessor albp = AsyncProcessorTypeConverter.convert(processor);
-        return AsyncProcessorHelper.process(albp, exchange, new FailOverAsyncCallback(exchange, attempts, index, callback, processors));
+        return AsyncProcessorHelper.process(albp, copy, new FailOverAsyncCallback(exchange, copy, attempts, index, callback, processors));
     }
 
     /**
@@ -219,13 +218,15 @@ public class FailOverLoadBalancer extend
     private final class FailOverAsyncCallback implements AsyncCallback {
 
         private final Exchange exchange;
+        private Exchange copy;
         private final AtomicInteger attempts;
         private final AtomicInteger index;
         private final AsyncCallback callback;
         private final List<Processor> processors;
 
-        private FailOverAsyncCallback(Exchange exchange, AtomicInteger attempts, AtomicInteger index, AsyncCallback callback, List<Processor> processors) {
+        private FailOverAsyncCallback(Exchange exchange, Exchange copy, AtomicInteger attempts, AtomicInteger index, AsyncCallback callback, List<Processor> processors) {
             this.exchange = exchange;
+            this.copy = copy;
             this.attempts = attempts;
             this.index = index;
             this.callback = callback;
@@ -238,7 +239,7 @@ public class FailOverLoadBalancer extend
                 return;
             }
 
-            while (shouldFailOver(exchange)) {
+            while (shouldFailOver(copy)) {
                 attempts.incrementAndGet();
                 // are we exhausted by attempts?
                 if (maximumFailoverAttempts > -1 && attempts.get() > maximumFailoverAttempts) {
@@ -263,11 +264,11 @@ public class FailOverLoadBalancer extend
                 }
 
                 // try again but prepare exchange before we failover
-                prepareExchangeForFailover(exchange);
+                copy = prepareExchangeForFailover(exchange);
                 Processor processor = processors.get(index.get());
 
                 // try to failover using the next processor
-                doneSync = processExchange(processor, exchange, attempts, index, callback, processors);
+                doneSync = processExchange(processor, exchange, copy, attempts, index, callback, processors);
                 if (!doneSync) {
                     log.trace("Processing exchangeId: {} is continued being processed asynchronously", exchange.getExchangeId());
                     // the remainder of the failover will be completed async
@@ -276,8 +277,11 @@ public class FailOverLoadBalancer extend
                 }
             }
 
+            // and copy the current result to original so it will contain this result of this eip
+            if (copy != null) {
+                ExchangeHelper.copyResults(exchange, copy);
+            }
             log.debug("Failover complete for exchangeId: {} >>> {}", exchange.getExchangeId(), exchange);
-
             // signal callback we are done
             callback.done(false);
         };