You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by jo...@apache.org on 2010/12/08 05:51:36 UTC

svn commit: r1043300 - in /camel/trunk/components/camel-restlet/src: main/java/org/apache/camel/component/restlet/ test/java/org/apache/camel/component/restlet/

Author: joed
Date: Wed Dec  8 04:51:36 2010
New Revision: 1043300

URL: http://svn.apache.org/viewvc?rev=1043300&view=rev
Log:
Camel-Restlet, throwExceptionOnFailure for the producer.


Added:
    camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletOperationException.java
    camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletThrowExceptionOnFailureTest.java
Modified:
    camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletEndpoint.java
    camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletProducer.java
    camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletResponseTest.java
    camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletRouteBuilderAuthTest.java

Modified: camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletEndpoint.java?rev=1043300&r1=1043299&r2=1043300&view=diff
==============================================================================
--- camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletEndpoint.java (original)
+++ camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletEndpoint.java Wed Dec  8 04:51:36 2010
@@ -39,25 +39,26 @@ public class RestletEndpoint extends Def
     private static final int DEFAULT_PORT = 80;
     private static final String DEFAULT_PROTOCOL = "http";
     private static final String DEFAULT_HOST = "localhost";
-    
+
     private Method restletMethod = Method.GET;
 
     // Optional and for consumer only.  This allows a single route to service multiple 
     // methods.  If it is non-null, restletMethod is ignored.
     private Method[] restletMethods;
-    
+
     private String protocol = DEFAULT_PROTOCOL;
     private String host = DEFAULT_HOST;
     private int port = DEFAULT_PORT;
     private String uriPattern;
-    
+
     // Optional and for consumer only.  This allows a single route to service multiple 
     // URI patterns.  The URI pattern defined in the endpoint will still be honored.
     private List<String> restletUriPatterns;
-    
+
     private Map<String, String> restletRealm;
     private HeaderFilterStrategy headerFilterStrategy;
     private RestletBinding restletBinding;
+    private boolean throwExceptionOnFailure = true;
 
     public RestletEndpoint(RestletComponent component, String remaining) throws Exception {
         super(remaining, component);
@@ -67,7 +68,7 @@ public class RestletEndpoint extends Def
         return true;
     }
 
-    @Override 
+    @Override
     public boolean isLenientProperties() {
         // true to allow dynamic URI options to be configured and passed to external system.
         return true;
@@ -82,11 +83,11 @@ public class RestletEndpoint extends Def
     }
 
     public void connect(RestletConsumer restletConsumer) throws Exception {
-        ((RestletComponent)getComponent()).connect(restletConsumer);
+        ((RestletComponent) getComponent()).connect(restletConsumer);
     }
 
     public void disconnect(RestletConsumer restletConsumer) throws Exception {
-        ((RestletComponent)getComponent()).disconnect(restletConsumer);        
+        ((RestletComponent) getComponent()).disconnect(restletConsumer);
     }
 
     public Method getRestletMethod() {
@@ -140,7 +141,7 @@ public class RestletEndpoint extends Def
     public void setHeaderFilterStrategy(HeaderFilterStrategy headerFilterStrategy) {
         this.headerFilterStrategy = headerFilterStrategy;
         if (restletBinding instanceof HeaderFilterStrategyAware) {
-            ((HeaderFilterStrategyAware)restletBinding).setHeaderFilterStrategy(headerFilterStrategy);
+            ((HeaderFilterStrategyAware) restletBinding).setHeaderFilterStrategy(headerFilterStrategy);
         }
     }
 
@@ -186,11 +187,19 @@ public class RestletEndpoint extends Def
             restletBinding = new DefaultRestletBinding();
         }
         if (restletBinding instanceof HeaderFilterStrategyAware) {
-            ((HeaderFilterStrategyAware)restletBinding).setHeaderFilterStrategy(getHeaderFilterStrategy());
+            ((HeaderFilterStrategyAware) restletBinding).setHeaderFilterStrategy(getHeaderFilterStrategy());
         }
     }
 
     public void stop() throws Exception {
         // noop
     }
+
+    public boolean isThrowExceptionOnFailure() {
+        return throwExceptionOnFailure;
+    }
+
+    public void setThrowExceptionOnFailure(boolean throwExceptionOnFailure) {
+        this.throwExceptionOnFailure = throwExceptionOnFailure;
+    }
 }

Added: camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletOperationException.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletOperationException.java?rev=1043300&view=auto
==============================================================================
--- camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletOperationException.java (added)
+++ camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletOperationException.java Wed Dec  8 04:51:36 2010
@@ -0,0 +1,70 @@
+/**
+ * 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.restlet;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.camel.CamelException;
+
+public class RestletOperationException extends CamelException implements Serializable {
+
+    private static final long serialVersionUID = -1696222841120185992L;
+    private final String uri;
+    private final String redirectLocation;
+    private final int statusCode;
+    private final String statusText;
+    private final Map<String, String> responseHeaders;
+    private final String responseBody;
+
+    public RestletOperationException(String uri, int statusCode, String statusText, String location, Map<String, String> responseHeaders,
+                                     String responseBody) {
+        super("Restlet operation failed invoking " + uri + " with statusCode: " + statusCode + (location != null ? ", redirectLocation: " + location
+            : ""));
+        this.uri = uri;
+        this.statusCode = statusCode;
+        this.statusText = statusText;
+        this.redirectLocation = location;
+        this.responseHeaders = responseHeaders;
+        this.responseBody = responseBody;
+    }
+
+    public String getRedirectLocation() {
+        return redirectLocation;
+    }
+
+    public String getResponseBody() {
+        return responseBody;
+    }
+
+    public Map<String, String> getResponseHeaders() {
+        return responseHeaders;
+    }
+
+    public int getStatusCode() {
+        return statusCode;
+    }
+
+    public String getStatusText() {
+        return statusText;
+    }
+
+    public String getUri() {
+        return uri;
+    }
+}

Modified: camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletProducer.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletProducer.java?rev=1043300&r1=1043299&r2=1043300&view=diff
==============================================================================
--- camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletProducer.java (original)
+++ camel/trunk/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletProducer.java Wed Dec  8 04:51:36 2010
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.component.restlet;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -38,9 +40,11 @@ public class RestletProducer extends Def
     private static final Log LOG = LogFactory.getLog(RestletProducer.class);
     private static final Pattern PATTERN = Pattern.compile("\\{([\\w\\.]*)\\}");
     private Client client;
+    private boolean throwException;
 
     public RestletProducer(RestletEndpoint endpoint) throws Exception {
         super(endpoint);
+        this.throwException = endpoint.isThrowExceptionOnFailure();
         client = new Client(endpoint.getProtocol());
         client.setContext(new Context());
     }
@@ -50,34 +54,40 @@ public class RestletProducer extends Def
         super.doStart();
         client.start();
     }
-    
+
     @Override
     public void doStop() throws Exception {
         client.stop();
         super.doStop();
     }
-    
+
     public void process(Exchange exchange) throws Exception {
-        RestletEndpoint endpoint = (RestletEndpoint)getEndpoint();
-        
+        RestletEndpoint endpoint = (RestletEndpoint) getEndpoint();
+
         String resourceUri = buildUri(endpoint, exchange);
         Request request = new Request(endpoint.getRestletMethod(), resourceUri);
 
         RestletBinding binding = endpoint.getRestletBinding();
         binding.populateRestletRequestFromExchange(request, exchange);
-        
+
         if (LOG.isDebugEnabled()) {
-            LOG.debug("Client sends a request (method: " + request.getMethod() 
-                    + ", uri: " + resourceUri + ")");
+            LOG.debug("Client sends a request (method: " + request.getMethod() + ", uri: " + resourceUri + ")");
         }
-        
+
         Response response = client.handle(request);
+        if (throwException) {
+            if (response instanceof Response) {
+                Integer respCode = response.getStatus().getCode();
+                if (respCode > 207) {
+                    throw populateRestletProducerException(exchange, response, respCode);
+                }
+            }
+        }
         binding.populateExchangeFromRestletResponse(exchange, response);
     }
 
     private static String buildUri(RestletEndpoint endpoint, Exchange exchange) throws CamelExchangeException {
-        String uri = endpoint.getProtocol() + "://" + endpoint.getHost() + ":" 
-            + endpoint.getPort() + endpoint.getUriPattern();
+        String uri = endpoint.getProtocol() + "://" + endpoint.getHost() + ":" + endpoint.getPort() + endpoint.getUriPattern();
 
         // substitute { } placeholders in uri and use mandatory headers
         if (LOG.isTraceEnabled()) {
@@ -108,4 +118,44 @@ public class RestletProducer extends Def
         return uri;
     }
 
+    protected RestletOperationException populateRestletProducerException(Exchange exchange, Response response, int responseCode) {
+        RestletOperationException exception;
+        String uri = exchange.getFromEndpoint().getEndpointUri();
+        String statusText = response.getStatus().getDescription();
+        Map<String, String> headers = parseResponseHeaders(response, exchange);
+        String copy = response.toString();
+        LOG.warn(headers);
+        if (responseCode >= 300 && responseCode < 400) {
+            String redirectLocation;
+            if (response.getStatus().isRedirection()) {
+                redirectLocation = response.getLocationRef().getHostIdentifier();
+                exception = new RestletOperationException(uri, responseCode, statusText, redirectLocation, headers, copy);
+            } else {
+                //no redirect location
+                exception = new RestletOperationException(uri, responseCode, statusText, null, headers, copy);
+            }
+        } else {
+            //internal server error(error code 500)
+            exception = new RestletOperationException(uri, responseCode, statusText, null, headers, copy);
+        }
+
+        return exception;
+    }
+
+    protected Map<String, String> parseResponseHeaders(Object response, Exchange camelExchange) {
+
+        Map<String, String> answer = new HashMap<String, String>();
+        if (response instanceof Response) {
+
+            for (Map.Entry<String, Object> entry : ((Response) response).getAttributes().entrySet()) {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Parse external header " + entry.getKey() + "=" + entry.getValue());
+                }
+                LOG.info("Parse external header " + entry.getKey() + "=" + entry.getValue());
+                answer.put(entry.getKey(), entry.getValue().toString());
+            }
+        }
+
+        return answer;
+    }
 }

Modified: camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletResponseTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletResponseTest.java?rev=1043300&r1=1043299&r2=1043300&view=diff
==============================================================================
--- camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletResponseTest.java (original)
+++ camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletResponseTest.java Wed Dec  8 04:51:36 2010
@@ -19,6 +19,7 @@ package org.apache.camel.component.restl
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.camel.CamelExecutionException;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
@@ -58,7 +59,7 @@ public class RestletResponseTest extends
         assertHttpResponse(response, 417, "application/JSON");
     }
     
-    @Test
+    @Test(expected = CamelExecutionException.class)
     public void testRestletProducer() throws Exception {
         Map<String, Object> headers = new HashMap<String, Object>();        
         headers.put("username", "homer");

Modified: camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletRouteBuilderAuthTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletRouteBuilderAuthTest.java?rev=1043300&r1=1043299&r2=1043300&view=diff
==============================================================================
--- camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletRouteBuilderAuthTest.java (original)
+++ camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletRouteBuilderAuthTest.java Wed Dec  8 04:51:36 2010
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-
+import org.apache.camel.CamelExecutionException;
 import org.apache.camel.test.junit4.CamelSpringTestSupport;
 import org.junit.Test;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
@@ -46,7 +46,7 @@ public class RestletRouteBuilderAuthTest
                 response);
     }
 
-    @Test
+    @Test(expected = CamelExecutionException.class)
     public void testhBasicAuthError() throws IOException {
         Map<String, Object> headers = new HashMap<String, Object>();
         headers.put(RestletConstants.RESTLET_LOGIN, "admin");

Added: camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletThrowExceptionOnFailureTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletThrowExceptionOnFailureTest.java?rev=1043300&view=auto
==============================================================================
--- camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletThrowExceptionOnFailureTest.java (added)
+++ camel/trunk/components/camel-restlet/src/test/java/org/apache/camel/component/restlet/RestletThrowExceptionOnFailureTest.java Wed Dec  8 04:51:36 2010
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.component.restlet;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.camel.CamelExecutionException;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.junit.Test;
+
+public class RestletThrowExceptionOnFailureTest extends RestletTestSupport {
+
+    @Test(expected = CamelExecutionException.class)
+    public void testRestletProducerGet2() throws Exception {
+        Map<String, Object> headers = new HashMap<String, Object>();
+        headers.put("id", 123);
+        headers.put("beverage.beer", "Carlsberg");
+
+        String out = template.requestBodyAndHeaders("direct:start", null, headers, String.class);
+        assertEquals("123;Donald Duck;Carlsberg", out);
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start").to("restlet:http://localhost:9080/user/{id}/like/{beverage.beer}");
+
+                from("restlet:http://localhost:9080/users/{id}/like/{beer}").process(new Processor() {
+                    public void process(Exchange exchange) throws Exception {
+                        String id = exchange.getIn().getHeader("id", String.class);
+                        String beer = exchange.getIn().getHeader("beer", String.class);
+                        exchange.getOut().setBody(id + ";Donald Duck;" + beer);
+                    }
+                });
+            }
+        };
+    }
+}