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 2016/04/18 18:16:11 UTC

[1/4] camel git commit: CAMEL-9879: Circuit Breaker EIP - That is using hystrix

Repository: camel
Updated Branches:
  refs/heads/hys [created] c534ac287


CAMEL-9879: Circuit Breaker EIP - That is using hystrix


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/252010ba
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/252010ba
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/252010ba

Branch: refs/heads/hys
Commit: 252010bac847caa4db9529ead25e248728fb0bc0
Parents: d334cbe
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Apr 18 16:37:51 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Apr 18 16:50:29 2016 +0200

----------------------------------------------------------------------
 .../apache/camel/model/FallbackDefinition.java  | 60 ++++++++++++
 .../model/HystrixCircuitBreakerDefinition.java  | 96 ++++++++++++++++++++
 .../apache/camel/model/ProcessorDefinition.java | 11 +++
 .../CircuitBreakerLoadBalancerDefinition.java   |  2 +-
 .../HystrixCircuitBreakerProcessor.java         | 96 ++++++++++++++++++++
 .../resources/org/apache/camel/model/jaxb.index |  2 +
 .../processor/HystrixCircuitBreakerTest.java    | 50 ++++++++++
 7 files changed, 316 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/java/org/apache/camel/model/FallbackDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/FallbackDefinition.java b/camel-core/src/main/java/org/apache/camel/model/FallbackDefinition.java
new file mode 100644
index 0000000..301c6d1
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/FallbackDefinition.java
@@ -0,0 +1,60 @@
+/**
+ * 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 java.util.List;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.Processor;
+import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.CollectionStringBuffer;
+
+/**
+ * Route to be executed when circuit breaker executes fallback
+ */
+@Metadata(label = "eip,routing,circuitbreaker")
+@XmlRootElement(name = "fallback")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class FallbackDefinition extends OutputDefinition<FallbackDefinition> {
+
+    public FallbackDefinition() {
+    }
+
+    @Override
+    public String toString() {
+        return "Fallback[" + getOutputs() + "]";
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return this.createChildProcessor(routeContext, false);
+    }
+
+    @Override
+    public String getLabel() {
+        CollectionStringBuffer buffer = new CollectionStringBuffer("fallback[");
+        List<ProcessorDefinition<?>> list = getOutputs();
+        for (ProcessorDefinition<?> type : list) {
+            buffer.append(type.getLabel());
+        }
+        buffer.append("]");
+        return buffer.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
new file mode 100644
index 0000000..c7b4cd4
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
@@ -0,0 +1,96 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.Processor;
+import org.apache.camel.processor.HystrixCircuitBreakerProcessor;
+import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.RouteContext;
+
+@Metadata(label = "eip,routing,circuitbreaker")
+@XmlRootElement(name = "hystrixCircuitBreaker")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class HystrixCircuitBreakerDefinition extends OutputDefinition<HystrixCircuitBreakerDefinition> {
+
+    // TODO: we can rename to circuitBreaker and then deprecated the CB in the load balancer
+    // the trick is to avoid a clash in the generated xml schema
+    // so for know we call it hystrixCircuitBreaker
+
+    @XmlElement
+    private FallbackDefinition fallback;
+
+    public HystrixCircuitBreakerDefinition() {
+    }
+
+    @Override
+    public String toString() {
+        return "HystrixCircuitBreaker[" + getOutputs() + "]";
+    }
+
+    @Override
+    public String getLabel() {
+        return "hystrixCircuitBreaker";
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor children = this.createChildProcessor(routeContext, true);
+
+        Processor fallbackProcessor = null;
+        if (fallback != null) {
+            fallbackProcessor = createProcessor(routeContext, fallback);
+        }
+        return new HystrixCircuitBreakerProcessor(children, fallbackProcessor);
+    }
+
+    @Override
+    public void addOutput(ProcessorDefinition<?> output) {
+        if (fallback != null) {
+            fallback.addOutput(output);
+        } else {
+            super.addOutput(output);
+        }
+    }
+
+    @Override
+    public ProcessorDefinition<?> end() {
+        if (fallback != null) {
+            // end fallback as well
+            fallback.end();
+        }
+        return super.end();
+    }
+
+    // Fluent API
+    // -------------------------------------------------------------------------
+
+    /**
+     * Sets the otherwise node
+     *
+     * @return the builder
+     */
+    public HystrixCircuitBreakerDefinition fallback() {
+        fallback = new FallbackDefinition();
+        return this;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index 1ba4bb5..aa82e88 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -1470,6 +1470,17 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
     }
 
     /**
+     * Creates a Circuit Breaker EIP that is using Hystrix.
+     *
+     * @return  the builder
+     */
+    public HystrixCircuitBreakerDefinition hystrixCircuitBreaker() {
+        HystrixCircuitBreakerDefinition answer = new HystrixCircuitBreakerDefinition();
+        addOutput(answer);
+        return answer;
+    }
+
+    /**
      * <a href="http://camel.apache.org/load-balancer.html">Load Balancer EIP:</a>
      * Creates a loadbalance
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CircuitBreakerLoadBalancerDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CircuitBreakerLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CircuitBreakerLoadBalancerDefinition.java
index 963339e..75b0e26 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CircuitBreakerLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CircuitBreakerLoadBalancerDefinition.java
@@ -41,7 +41,7 @@ import org.apache.camel.util.ObjectHelper;
  * timeout is reached. After this timeout is reached, if there is a new call, it will pass and if the result is
  * success the Circuit Breaker will move to closed state, or to open state if there was an error.
  */
-@Metadata(label = "eip,routing,loadbalance")
+@Metadata(label = "eip,routing,loadbalance,circuitbreaker")
 @XmlRootElement(name = "circuitBreaker")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class CircuitBreakerLoadBalancerDefinition extends LoadBalancerDefinition {

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/java/org/apache/camel/processor/HystrixCircuitBreakerProcessor.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/processor/HystrixCircuitBreakerProcessor.java b/camel-core/src/main/java/org/apache/camel/processor/HystrixCircuitBreakerProcessor.java
new file mode 100644
index 0000000..79d5037
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/processor/HystrixCircuitBreakerProcessor.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.Exchange;
+import org.apache.camel.Navigate;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.IdAware;
+import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.util.AsyncProcessorHelper;
+
+public class HystrixCircuitBreakerProcessor extends ServiceSupport implements AsyncProcessor, Navigate<Processor>, org.apache.camel.Traceable, IdAware {
+
+    private String id;
+    private final Processor processor;
+    private final Processor fallback;
+
+    public HystrixCircuitBreakerProcessor(Processor processor, Processor fallback) {
+        this.processor = processor;
+        this.fallback = fallback;
+    }
+
+    @Override
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    @Override
+    public String getId() {
+        return id;
+    }
+
+    @Override
+    public String getTraceLabel() {
+        return "hystrixCircuitBreaker";
+    }
+
+    @Override
+    public List<Processor> next() {
+        if (!hasNext()) {
+            return null;
+        }
+        List<Processor> answer = new ArrayList<Processor>();
+        answer.add(processor);
+        if (fallback != null) {
+            answer.add(fallback);
+        }
+        return answer;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return true;
+    }
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+        AsyncProcessorHelper.process(this, exchange);
+    }
+
+    @Override
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        // TODO: use camel-hystrix to process this
+        callback.done(true);
+        return true;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        // noop
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/org/apache/camel/model/jaxb.index b/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
index 0fb9ac1..6232753 100644
--- a/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
+++ b/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
@@ -27,9 +27,11 @@ DescriptionDefinition
 DynamicRouterDefinition
 EnrichDefinition
 ExpressionSubElementDefinition
+FallbackDefinition
 FilterDefinition
 FinallyDefinition
 FromDefinition
+HystrixCircuitBreakerDefinition
 IdempotentConsumerDefinition
 InOnlyDefinition
 InOutDefinition

http://git-wip-us.apache.org/repos/asf/camel/blob/252010ba/camel-core/src/test/java/org/apache/camel/processor/HystrixCircuitBreakerTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/processor/HystrixCircuitBreakerTest.java b/camel-core/src/test/java/org/apache/camel/processor/HystrixCircuitBreakerTest.java
new file mode 100644
index 0000000..a44ddd5
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/processor/HystrixCircuitBreakerTest.java
@@ -0,0 +1,50 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.builder.RouteBuilder;
+
+public class HystrixCircuitBreakerTest extends ContextTestSupport {
+
+    public void testHystrixCircuitBreaker() throws Exception {
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hello 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")
+                    .hystrixCircuitBreaker()
+                        .to("direct:foo")
+                    .fallback()
+                        .transform().constant("Fallback message")
+                    .end()
+                    .to("mock:result");
+
+                from("direct:foo")
+                    .throwException(new IllegalArgumentException("Forced"));
+            }
+        };
+    }
+}


[3/4] camel git commit: CAMEL-9879: Circuit Breaker EIP - That is using hystrix

Posted by da...@apache.org.
CAMEL-9879: Circuit Breaker EIP - That is using hystrix


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/285cd7ef
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/285cd7ef
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/285cd7ef

Branch: refs/heads/hys
Commit: 285cd7ef96260150ff62c3425d34060f063889ef
Parents: 7e2b719
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Apr 18 18:11:28 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Apr 18 18:11:28 2016 +0200

----------------------------------------------------------------------
 .../model/HystrixCircuitBreakerDefinition.java  |   9 ++
 .../apache/camel/model/ProcessorDefinition.java |   3 +
 .../hystrix/HystrixDummyProcessor.java          |  28 -----
 .../component/hystrix/HystrixProcessor.java     | 107 +++++++++++++++++++
 .../hystrix/HystrixProcessorCommand.java        |  87 +++++++++++++++
 .../hystrix/HystrixProcessorFactory.java        |  18 +++-
 .../hystrix/HystrixCircuitBreakerTest.java      |   7 +-
 7 files changed, 225 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
index c7b4cd4..3228596 100644
--- a/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/HystrixCircuitBreakerDefinition.java
@@ -80,6 +80,14 @@ public class HystrixCircuitBreakerDefinition extends OutputDefinition<HystrixCir
         return super.end();
     }
 
+    public FallbackDefinition getFallback() {
+        return fallback;
+    }
+
+    public void setFallback(FallbackDefinition fallback) {
+        this.fallback = fallback;
+    }
+
     // Fluent API
     // -------------------------------------------------------------------------
 
@@ -90,6 +98,7 @@ public class HystrixCircuitBreakerDefinition extends OutputDefinition<HystrixCir
      */
     public HystrixCircuitBreakerDefinition fallback() {
         fallback = new FallbackDefinition();
+        fallback.setParent(this);
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index aa82e88..2cdfa02 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -287,6 +287,9 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
         } else if (defn instanceof OnExceptionDefinition || ProcessorDefinitionHelper.isParentOfType(OnExceptionDefinition.class, defn, true)) {
             log.trace("{} is part of OnException so no error handler is applied", defn);
             // do not use error handler for onExceptions blocks as it will handle errors itself
+        } else if (defn instanceof HystrixCircuitBreakerDefinition || ProcessorDefinitionHelper.isParentOfType(HystrixCircuitBreakerDefinition.class, defn, true)) {
+            log.trace("{} is part of HystrixCircuitBreaker so no error handler is applied", defn);
+            // do not use error handler for hystrixCircuitBreaker blocks as it will handle errors itself
         } else if (defn instanceof MulticastDefinition) {
             // do not use error handler for multicast as it offers fine grained error handlers for its outputs
             // however if share unit of work is enabled, we need to wrap an error handler on the multicast parent

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
deleted file mode 100644
index 48d52f5..0000000
--- a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * 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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.component.hystrix;
-
-import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
-
-public class HystrixDummyProcessor implements Processor {
-
-    @Override
-    public void process(Exchange exchange) throws Exception {
-        System.out.println("Dummy processor");
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
new file mode 100644
index 0000000..6690234
--- /dev/null
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.component.hystrix;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.netflix.hystrix.HystrixCommandGroupKey;
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.Exchange;
+import org.apache.camel.Navigate;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.IdAware;
+import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.util.AsyncProcessorConverterHelper;
+import org.apache.camel.util.AsyncProcessorHelper;
+
+public class HystrixProcessor extends ServiceSupport implements AsyncProcessor, Navigate<Processor>, org.apache.camel.Traceable, IdAware {
+
+    private String id;
+    private final AsyncProcessor processor;
+    private final AsyncProcessor fallback;
+
+    public HystrixProcessor(String id, Processor processor, Processor fallback) {
+        this.id = id;
+        this.processor = AsyncProcessorConverterHelper.convert(processor);
+        this.fallback = AsyncProcessorConverterHelper.convert(fallback);
+    }
+
+    @Override
+    public String getId() {
+        return id;
+    }
+
+    @Override
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    @Override
+    public String getTraceLabel() {
+        return "hystrixCircuitBreaker";
+    }
+
+    @Override
+    public List<Processor> next() {
+        if (!hasNext()) {
+            return null;
+        }
+        List<Processor> answer = new ArrayList<Processor>();
+        answer.add(processor);
+        if (fallback != null) {
+            answer.add(fallback);
+        }
+        return answer;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return true;
+    }
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+        AsyncProcessorHelper.process(this, exchange);
+    }
+
+    @Override
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        HystrixCommandGroupKey key = HystrixCommandGroupKey.Factory.asKey(id);
+
+        HystrixProcessorCommand command = new HystrixProcessorCommand(key, exchange, callback, processor, fallback);
+        try {
+            command.execute();
+        } catch (Throwable e) {
+            exchange.setException(e);
+        }
+
+        callback.done(true);
+        return true;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        // noop
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorCommand.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorCommand.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorCommand.java
new file mode 100644
index 0000000..39d64d1
--- /dev/null
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorCommand.java
@@ -0,0 +1,87 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.component.hystrix;
+
+import com.netflix.hystrix.HystrixCommand;
+import com.netflix.hystrix.HystrixCommandGroupKey;
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+
+public class HystrixProcessorCommand extends HystrixCommand<Exchange> {
+
+    private final Exchange exchange;
+    private final AsyncCallback callback;
+    private final AsyncProcessor processor;
+    private final AsyncProcessor fallback;
+
+    public HystrixProcessorCommand(HystrixCommandGroupKey group, Exchange exchange, AsyncCallback callback,
+                                   AsyncProcessor processor, AsyncProcessor fallback) {
+        super(group);
+        this.exchange = exchange;
+        this.callback = callback;
+        this.processor = processor;
+        this.fallback = fallback;
+    }
+
+    @Override
+    protected Exchange getFallback() {
+        if (fallback != null) {
+            try {
+                Exception e = exchange.getException();
+                // store the last to endpoint as the failure endpoint
+                if (exchange.getProperty(Exchange.FAILURE_ENDPOINT) == null) {
+                    exchange.setProperty(Exchange.FAILURE_ENDPOINT, exchange.getProperty(Exchange.TO_ENDPOINT));
+                }
+                // give the rest of the pipeline another chance
+                exchange.setProperty(Exchange.EXCEPTION_HANDLED, true);
+                exchange.setProperty(Exchange.EXCEPTION_CAUGHT, e);
+                exchange.setException(null);
+                // and we should not be regarded as exhausted as we are in a try .. catch block
+                exchange.removeProperty(Exchange.REDELIVERY_EXHAUSTED);
+
+                fallback.process(exchange, callback);
+            } catch (Exception e) {
+                exchange.setException(e);
+            } finally {
+                callback.done(true);
+            }
+            return exchange;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    protected Exchange run() throws Exception {
+        try {
+            processor.process(exchange, callback);
+        } catch (Exception e) {
+            exchange.setException(e);
+        } finally {
+            callback.done(true);
+        }
+
+        // if we failed then throw an exception
+        if (exchange.getException() != null) {
+            throw exchange.getException();
+        }
+
+        return exchange;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
index c0924b1..40b5eb0 100644
--- a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
@@ -17,6 +17,7 @@
 package org.apache.camel.component.hystrix;
 
 import org.apache.camel.Processor;
+import org.apache.camel.model.HystrixCircuitBreakerDefinition;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.ProcessorFactory;
 import org.apache.camel.spi.RouteContext;
@@ -25,11 +26,26 @@ public class HystrixProcessorFactory implements ProcessorFactory {
 
     @Override
     public Processor createChildProcessor(RouteContext routeContext, ProcessorDefinition<?> definition, boolean mandatory) throws Exception {
+        // not in use
         return null;
     }
 
     @Override
     public Processor createProcessor(RouteContext routeContext, ProcessorDefinition<?> definition) throws Exception {
-        return new HystrixDummyProcessor();
+        if (definition instanceof HystrixCircuitBreakerDefinition) {
+            HystrixCircuitBreakerDefinition cb = (HystrixCircuitBreakerDefinition) definition;
+            String id = cb.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
+
+            // create the regular processor
+            Processor processor = cb.createChildProcessor(routeContext, true);
+            Processor fallback = null;
+            if (cb.getFallback() != null) {
+                fallback = cb.getFallback().createProcessor(routeContext);
+            }
+
+            return new HystrixProcessor(id, processor, fallback);
+        } else {
+            return null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/285cd7ef/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
index d714435..f4c035d 100644
--- a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
+++ b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
@@ -24,7 +24,7 @@ public class HystrixCircuitBreakerTest extends CamelTestSupport {
 
     @Test
     public void testHystrixCircuitBreaker() throws Exception {
-        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Fallback message");
 
         template.sendBody("direct:start", "Hello World");
 
@@ -38,14 +38,11 @@ public class HystrixCircuitBreakerTest extends CamelTestSupport {
             public void configure() throws Exception {
                 from("direct:start")
                     .hystrixCircuitBreaker()
-                        .to("direct:foo")
+                        .throwException(new IllegalArgumentException("Forced"))
                     .fallback()
                         .transform().constant("Fallback message")
                     .end()
                     .to("mock:result");
-
-                from("direct:foo")
-                        .throwException(new IllegalArgumentException("Forced"));
             }
         };
     }


[2/4] camel git commit: CAMEL-9879: Circuit Breaker EIP - That is using hystrix

Posted by da...@apache.org.
CAMEL-9879: Circuit Breaker EIP - That is using hystrix


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/7e2b719d
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/7e2b719d
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/7e2b719d

Branch: refs/heads/hys
Commit: 7e2b719d712ea4db28d03b818b17fd8af8e09536
Parents: 252010b
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Apr 18 17:33:01 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Apr 18 17:33:01 2016 +0200

----------------------------------------------------------------------
 .../apache/camel/impl/DefaultCamelContext.java  |  2 +-
 .../camel/impl/DefaultProcessorFactory.java     | 67 ++++++++++++++++++++
 .../hystrix/HystrixDummyProcessor.java          | 28 ++++++++
 .../hystrix/HystrixProcessorFactory.java        | 35 ++++++++++
 .../camel/model/HystrixCircuitBreakerDefinition | 18 ++++++
 .../hystrix/HystrixCircuitBreakerTest.java      | 53 ++++++++++++++++
 6 files changed, 202 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
index 654c7b9..d96d646 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
@@ -245,7 +245,7 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
     private ServicePool<Endpoint, Producer> producerServicePool = new SharedProducerServicePool(100);
     private ServicePool<Endpoint, PollingConsumer> pollingConsumerServicePool = new SharedPollingConsumerServicePool(100);
     private NodeIdFactory nodeIdFactory = new DefaultNodeIdFactory();
-    private ProcessorFactory processorFactory;
+    private ProcessorFactory processorFactory = new DefaultProcessorFactory();
     private MessageHistoryFactory messageHistoryFactory = new DefaultMessageHistoryFactory();
     private InterceptStrategy defaultTracer;
     private InterceptStrategy defaultBacklogTracer;

http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/camel-core/src/main/java/org/apache/camel/impl/DefaultProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultProcessorFactory.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultProcessorFactory.java
new file mode 100644
index 0000000..078743d
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultProcessorFactory.java
@@ -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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.NoFactoryAvailableException;
+import org.apache.camel.Processor;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.spi.FactoryFinder;
+import org.apache.camel.spi.ProcessorFactory;
+import org.apache.camel.spi.RouteContext;
+
+public class DefaultProcessorFactory implements ProcessorFactory {
+
+    public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/model/";
+
+    @Override
+    public Processor createChildProcessor(RouteContext routeContext, ProcessorDefinition<?> definition, boolean mandatory) throws Exception {
+        String name = definition.getClass().getSimpleName();
+        FactoryFinder finder = routeContext.getCamelContext().getFactoryFinder(RESOURCE_PATH);
+        try {
+            if (finder != null) {
+                Object object = finder.newInstance(name);
+                if (object != null && object instanceof ProcessorFactory) {
+                    ProcessorFactory pc = (ProcessorFactory) object;
+                    return pc.createChildProcessor(routeContext, definition, mandatory);
+                }
+            }
+        } catch (NoFactoryAvailableException e) {
+            // ignore there is no custom factory
+        }
+
+        return null;
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext, ProcessorDefinition<?> definition) throws Exception {
+        String name = definition.getClass().getSimpleName();
+        FactoryFinder finder = routeContext.getCamelContext().getFactoryFinder(RESOURCE_PATH);
+        try {
+            if (finder != null) {
+                Object object = finder.newInstance(name);
+                if (object != null && object instanceof ProcessorFactory) {
+                    ProcessorFactory pc = (ProcessorFactory) object;
+                    return pc.createProcessor(routeContext, definition);
+                }
+            }
+        } catch (NoFactoryAvailableException e) {
+            // ignore there is no custom factory
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
new file mode 100644
index 0000000..48d52f5
--- /dev/null
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixDummyProcessor.java
@@ -0,0 +1,28 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.component.hystrix;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+
+public class HystrixDummyProcessor implements Processor {
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+        System.out.println("Dummy processor");
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
new file mode 100644
index 0000000..c0924b1
--- /dev/null
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessorFactory.java
@@ -0,0 +1,35 @@
+/**
+ * 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.component.hystrix;
+
+import org.apache.camel.Processor;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.spi.ProcessorFactory;
+import org.apache.camel.spi.RouteContext;
+
+public class HystrixProcessorFactory implements ProcessorFactory {
+
+    @Override
+    public Processor createChildProcessor(RouteContext routeContext, ProcessorDefinition<?> definition, boolean mandatory) throws Exception {
+        return null;
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext, ProcessorDefinition<?> definition) throws Exception {
+        return new HystrixDummyProcessor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/components/camel-hystrix/src/main/resources/META-INF/services/org/apache/camel/model/HystrixCircuitBreakerDefinition
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/resources/META-INF/services/org/apache/camel/model/HystrixCircuitBreakerDefinition b/components/camel-hystrix/src/main/resources/META-INF/services/org/apache/camel/model/HystrixCircuitBreakerDefinition
new file mode 100644
index 0000000..0e00349
--- /dev/null
+++ b/components/camel-hystrix/src/main/resources/META-INF/services/org/apache/camel/model/HystrixCircuitBreakerDefinition
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+class=org.apache.camel.component.hystrix.HystrixProcessorFactory

http://git-wip-us.apache.org/repos/asf/camel/blob/7e2b719d/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
new file mode 100644
index 0000000..d714435
--- /dev/null
+++ b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
@@ -0,0 +1,53 @@
+/**
+ * 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.component.hystrix;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class HystrixCircuitBreakerTest extends CamelTestSupport {
+
+    @Test
+    public void testHystrixCircuitBreaker() throws Exception {
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hello 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")
+                    .hystrixCircuitBreaker()
+                        .to("direct:foo")
+                    .fallback()
+                        .transform().constant("Fallback message")
+                    .end()
+                    .to("mock:result");
+
+                from("direct:foo")
+                        .throwException(new IllegalArgumentException("Forced"));
+            }
+        };
+    }
+
+}


[4/4] camel git commit: CAMEL-9879: Circuit Breaker EIP - That is using hystrix

Posted by da...@apache.org.
CAMEL-9879: Circuit Breaker EIP - That is using hystrix


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/c534ac28
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/c534ac28
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/c534ac28

Branch: refs/heads/hys
Commit: c534ac28741012f9475a031d52e848cf2e553e25
Parents: 285cd7e
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Apr 18 18:15:28 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Apr 18 18:15:28 2016 +0200

----------------------------------------------------------------------
 .../component/hystrix/HystrixProcessor.java     |  6 ++-
 .../HystrixCircuitBreakerFallbackTest.java      | 53 ++++++++++++++++++++
 .../hystrix/HystrixCircuitBreakerOkTest.java    | 53 ++++++++++++++++++++
 .../hystrix/HystrixCircuitBreakerTest.java      | 50 ------------------
 4 files changed, 111 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/c534ac28/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
index 6690234..0a9f708 100644
--- a/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
+++ b/components/camel-hystrix/src/main/java/org/apache/camel/component/hystrix/HystrixProcessor.java
@@ -82,8 +82,10 @@ public class HystrixProcessor extends ServiceSupport implements AsyncProcessor,
 
     @Override
     public boolean process(Exchange exchange, AsyncCallback callback) {
-        HystrixCommandGroupKey key = HystrixCommandGroupKey.Factory.asKey(id);
+        // run this as if we run inside try .. catch so there is no regular Camel error handler
+        exchange.setProperty(Exchange.TRY_ROUTE_BLOCK, true);
 
+        HystrixCommandGroupKey key = HystrixCommandGroupKey.Factory.asKey(id);
         HystrixProcessorCommand command = new HystrixProcessorCommand(key, exchange, callback, processor, fallback);
         try {
             command.execute();
@@ -91,6 +93,8 @@ public class HystrixProcessor extends ServiceSupport implements AsyncProcessor,
             exchange.setException(e);
         }
 
+        exchange.removeProperty(Exchange.TRY_ROUTE_BLOCK);
+
         callback.done(true);
         return true;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/c534ac28/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerFallbackTest.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerFallbackTest.java b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerFallbackTest.java
new file mode 100644
index 0000000..3c7e45a
--- /dev/null
+++ b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerFallbackTest.java
@@ -0,0 +1,53 @@
+/**
+ * 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.component.hystrix;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class HystrixCircuitBreakerFallbackTest extends CamelTestSupport {
+
+    @Test
+    public void testHystrixCircuitBreaker() throws Exception {
+        getMockEndpoint("mock:result").expectedBodiesReceived("Fallback message");
+
+        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")
+                    .hystrixCircuitBreaker()
+                        .to("direct:foo")
+                    .fallback()
+                        .transform().constant("Fallback message")
+                    .end()
+                    .to("mock:result");
+
+                from("direct:foo")
+                    .throwException(new IllegalArgumentException("Forced"));
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/c534ac28/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerOkTest.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerOkTest.java b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerOkTest.java
new file mode 100644
index 0000000..b34c6c2
--- /dev/null
+++ b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerOkTest.java
@@ -0,0 +1,53 @@
+/**
+ * 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.component.hystrix;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class HystrixCircuitBreakerOkTest extends CamelTestSupport {
+
+    @Test
+    public void testHystrixCircuitBreaker() throws Exception {
+        getMockEndpoint("mock:result").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")
+                    .hystrixCircuitBreaker()
+                        .to("direct:foo")
+                    .fallback()
+                        .transform().constant("Fallback message")
+                    .end()
+                    .to("mock:result");
+
+                from("direct:foo")
+                    .transform().constant("Bye World");
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/c534ac28/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
----------------------------------------------------------------------
diff --git a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java b/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
deleted file mode 100644
index f4c035d..0000000
--- a/components/camel-hystrix/src/test/java/org/apache/camel/component/hystrix/HystrixCircuitBreakerTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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.component.hystrix;
-
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.test.junit4.CamelTestSupport;
-import org.junit.Test;
-
-public class HystrixCircuitBreakerTest extends CamelTestSupport {
-
-    @Test
-    public void testHystrixCircuitBreaker() throws Exception {
-        getMockEndpoint("mock:result").expectedBodiesReceived("Fallback message");
-
-        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")
-                    .hystrixCircuitBreaker()
-                        .throwException(new IllegalArgumentException("Forced"))
-                    .fallback()
-                        .transform().constant("Fallback message")
-                    .end()
-                    .to("mock:result");
-            }
-        };
-    }
-
-}