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 2014/08/13 13:53:44 UTC

git commit: CAMEL-7354: Rest DSL. Integrate with camel-netty-http.

Repository: camel
Updated Branches:
  refs/heads/master 6dc9d8a48 -> 6fae19134


CAMEL-7354: Rest DSL. Integrate with camel-netty-http.


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

Branch: refs/heads/master
Commit: 6fae1913455279019c2dbcdc1199ab8685feaeba
Parents: 6dc9d8a
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Aug 13 13:53:33 2014 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Aug 13 13:53:33 2014 +0200

----------------------------------------------------------------------
 .../netty/http/ContextPathMatcher.java          | 26 +++++-
 .../netty/http/DefaultContextPathMatcher.java   | 14 +++-
 .../netty/http/RestContextPathMatcher.java      | 31 ++++---
 .../http/handlers/HttpServerChannelHandler.java |  4 +
 .../HttpServerMultiplexChannelHandler.java      | 60 ++++++++++++--
 .../RestNettyHttpContextPathMatchGetTest.java   | 68 ++++++++++++++++
 .../netty/http/rest/RestPathMatchingTest.java   | 86 ++++++++++++++++++++
 7 files changed, 265 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/ContextPathMatcher.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/ContextPathMatcher.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/ContextPathMatcher.java
index a3c02d8..df8498a 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/ContextPathMatcher.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/ContextPathMatcher.java
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.component.netty.http;
 
+import java.util.Locale;
+
 /**
  * A matcher used for selecting the correct {@link org.apache.camel.component.netty.http.handlers.HttpServerChannelHandler}
  * to handle an incoming {@link org.jboss.netty.handler.codec.http.HttpRequest} when you use multiple routes on the same
@@ -27,11 +29,29 @@ package org.apache.camel.component.netty.http;
 public interface ContextPathMatcher {
 
     /**
-     * Whether the target context-path matches.
+     * Whether the target context-path matches a regular url.
+     *
+     * @param path  the context-path from the incoming HTTP request
+     * @return <tt>true</tt> to match, <tt>false</tt> if not.
+     */
+    boolean matches(String path);
+
+    /**
+     * Whether the target context-path matches a REST url.
      *
-     * @param method the HTTP method such as GET, POST
      * @param path  the context-path from the incoming HTTP request
+     * @param wildcard whether to match strict or by wildcards
      * @return <tt>true</tt> to match, <tt>false</tt> if not.
      */
-    boolean matches(String method, String path);
+    boolean matchesRest(String path, boolean wildcard);
+
+    /**
+     * Matches the given request HTTP method with the configured HTTP method of the consumer
+     *
+     * @param method    the request HTTP method
+     * @param restrict  the consumer configured HTTP restrict method
+     * @return <tt>true</tt> if matched, <tt>false</tt> otherwise
+     */
+    boolean matchMethod(String method, String restrict);
+
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/DefaultContextPathMatcher.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/DefaultContextPathMatcher.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/DefaultContextPathMatcher.java
index e1a4d52..9f7d98f 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/DefaultContextPathMatcher.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/DefaultContextPathMatcher.java
@@ -31,7 +31,8 @@ public class DefaultContextPathMatcher implements ContextPathMatcher {
         this.matchOnUriPrefix = matchOnUriPrefix;
     }
 
-    public boolean matches(String method, String path) {
+    @Override
+    public boolean matches(String path) {
         path = path.toLowerCase(Locale.US);
         if (!matchOnUriPrefix) {
             // exact match
@@ -42,6 +43,17 @@ public class DefaultContextPathMatcher implements ContextPathMatcher {
         }
     }
 
+    @Override
+    public boolean matchesRest(String path, boolean wildcard) {
+        return false;
+    }
+
+    @Override
+    public boolean matchMethod(String method, String restrict) {
+        // always match as HttpServerChannelHandler will deal with HTTP method restrictions
+        return true;
+    }
+
     public String getPath() {
         return path;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/RestContextPathMatcher.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/RestContextPathMatcher.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/RestContextPathMatcher.java
index f0838e3..8e35f50 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/RestContextPathMatcher.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/RestContextPathMatcher.java
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.component.netty.http;
 
+import java.util.Locale;
+
 /**
  * A {@link org.apache.camel.component.netty.http.ContextPathMatcher} that supports the Rest DSL.
  */
@@ -23,25 +25,28 @@ public class RestContextPathMatcher extends DefaultContextPathMatcher {
 
     private final String rawPath;
 
-    // TODO: improve matching like we have done in camel-servlet
-
     public RestContextPathMatcher(String rawPath, String path, boolean matchOnUriPrefix) {
         super(path, matchOnUriPrefix);
         this.rawPath = rawPath;
     }
 
     @Override
-    public boolean matches(String method, String path) {
-        if (useRestMatching(rawPath)) {
-            return matchRestPath(path, rawPath);
-        } else {
-            return super.matches(method, path);
-        }
+    public boolean matchesRest(String path, boolean wildcard) {
+        return matchRestPath(path, rawPath, wildcard);
     }
 
-    private boolean useRestMatching(String path) {
-        // only need to do rest matching if using { } placeholders
-        return path.indexOf('{') > -1;
+    @Override
+    public boolean matchMethod(String method, String restrict) {
+        if (restrict == null) {
+            return true;
+        }
+
+        // always match OPTIONS as some REST clients uses that prior to calling the service
+        if ("OPTIONS".equals(method)) {
+            return true;
+        }
+
+        return restrict.toLowerCase(Locale.US).contains(method.toLowerCase(Locale.US));
     }
 
     /**
@@ -51,7 +56,7 @@ public class RestContextPathMatcher extends DefaultContextPathMatcher {
      * @param consumerPath  the consumer path which may use { } tokens
      * @return <tt>true</tt> if matched, <tt>false</tt> otherwise
      */
-    public boolean matchRestPath(String requestPath, String consumerPath) {
+    public boolean matchRestPath(String requestPath, String consumerPath, boolean wildcard) {
         // remove starting/ending slashes
         if (requestPath.startsWith("/")) {
             requestPath = requestPath.substring(1);
@@ -80,7 +85,7 @@ public class RestContextPathMatcher extends DefaultContextPathMatcher {
             String p1 = requestPaths[i];
             String p2 = consumerPaths[i];
 
-            if (p2.startsWith("{") && p2.endsWith("}")) {
+            if (wildcard && p2.startsWith("{") && p2.endsWith("}")) {
                 // always matches
                 continue;
             }

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerChannelHandler.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerChannelHandler.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerChannelHandler.java
index 0e20a4d..da17d25 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerChannelHandler.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerChannelHandler.java
@@ -74,6 +74,10 @@ public class HttpServerChannelHandler extends ServerChannelHandler {
         this.consumer = consumer;
     }
 
+    public NettyHttpConsumer getConsumer() {
+        return consumer;
+    }
+
     @Override
     public void messageReceived(ChannelHandlerContext ctx, MessageEvent messageEvent) throws Exception {
         // store request, as this channel handler is created per pipeline

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerMultiplexChannelHandler.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerMultiplexChannelHandler.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerMultiplexChannelHandler.java
index 7fccde5..b452bb8 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerMultiplexChannelHandler.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/handlers/HttpServerMultiplexChannelHandler.java
@@ -16,13 +16,15 @@
  */
 package org.apache.camel.component.netty.http.handlers;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.camel.Exchange;
 import org.apache.camel.component.netty.http.ContextPathMatcher;
-import org.apache.camel.component.netty.http.DefaultContextPathMatcher;
 import org.apache.camel.component.netty.http.HttpServerConsumerChannelFactory;
 import org.apache.camel.component.netty.http.NettyHttpConsumer;
 import org.apache.camel.component.netty.http.RestContextPathMatcher;
@@ -135,27 +137,71 @@ public class HttpServerMultiplexChannelHandler extends SimpleChannelUpstreamHand
     }
 
     private HttpServerChannelHandler getHandler(HttpRequest request) {
+        HttpServerChannelHandler answer = null;
+
         // need to strip out host and port etc, as we only need the context-path for matching
         String method = request.getMethod().getName();
+        if (method == null) {
+            return null;
+        }
 
         String path = request.getUri();
         int idx = path.indexOf(token);
         if (idx > -1) {
             path = path.substring(idx + len);
         }
-
         // use the path as key to find the consumer handler to use
         path = pathAsKey(path);
 
-        // TODO: improve matching like we have done in camel-servlet, eg using candidates
 
-        // find the one that matches
+        List<Map.Entry<ContextPathMatcher, HttpServerChannelHandler>> candidates = new ArrayList<Map.Entry<ContextPathMatcher, HttpServerChannelHandler>>();
+
+        // first match by http method
         for (Map.Entry<ContextPathMatcher, HttpServerChannelHandler> entry : consumers.entrySet()) {
-            if (entry.getKey().matches(method, path)) {
-                return entry.getValue();
+            NettyHttpConsumer consumer = entry.getValue().getConsumer();
+            String restrict = consumer.getEndpoint().getHttpMethodRestrict();
+            if (entry.getKey().matchMethod(method, restrict)) {
+                candidates.add(entry);
+            }
+        }
+
+        // then see if we got a direct match
+        Iterator<Map.Entry<ContextPathMatcher, HttpServerChannelHandler>> it = candidates.iterator();
+        while (it.hasNext()) {
+            Map.Entry<ContextPathMatcher, HttpServerChannelHandler> entry = it.next();
+            if (entry.getKey().matchesRest(path, false)) {
+                answer = entry.getValue();
+                break;
+            }
+        }
+
+        // then match by non wildcard path
+        if (answer == null) {
+            it = candidates.iterator();
+            while (it.hasNext()) {
+                Map.Entry<ContextPathMatcher, HttpServerChannelHandler> entry = it.next();
+                if (!entry.getKey().matchesRest(path, true)) {
+                    it.remove();
+                }
+            }
+
+            // there should only be one
+            if (candidates.size() == 1) {
+                answer = candidates.get(0).getValue();
             }
         }
-        return null;
+
+        // fallback to regular matching
+        if (answer == null) {
+            for (Map.Entry<ContextPathMatcher, HttpServerChannelHandler> entry : consumers.entrySet()) {
+                if (entry.getKey().matches(path)) {
+                    answer = entry.getValue();
+                    break;
+                }
+            }
+        }
+
+        return answer;
     }
 
     private static String pathAsKey(String path) {

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestNettyHttpContextPathMatchGetTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestNettyHttpContextPathMatchGetTest.java b/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestNettyHttpContextPathMatchGetTest.java
new file mode 100644
index 0000000..6e4ccf7
--- /dev/null
+++ b/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestNettyHttpContextPathMatchGetTest.java
@@ -0,0 +1,68 @@
+/**
+ * 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.netty.http.rest;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.netty.http.BaseNettyTest;
+import org.junit.Test;
+
+public class RestNettyHttpContextPathMatchGetTest extends BaseNettyTest {
+
+    @Test
+    public void testProducerGet() throws Exception {
+        String out = template.requestBody("netty-http:http://localhost:{{port}}/users/123", null, String.class);
+        assertEquals("123;Donald Duck", out);
+
+        out = template.requestBody("netty-http:http://localhost:{{port}}/users/list", null, String.class);
+        assertEquals("123;Donald Duck\n456;John Doe", out);
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // configure to use netty-http on localhost with the given port
+                restConfiguration().component("netty-http").host("localhost").port(getPort());
+
+                // use the rest DSL to define the rest services
+                rest("/users/")
+                    .get("{id}")
+                        .route()
+                        .to("mock:input")
+                        .process(new Processor() {
+                            public void process(Exchange exchange) throws Exception {
+                                String id = exchange.getIn().getHeader("id", String.class);
+                                exchange.getOut().setBody(id + ";Donald Duck");
+                            }
+                        })
+                    .endRest()
+                    .get("list")
+                        .route()
+                        .to("mock:input")
+                        .process(new Processor() {
+                            public void process(Exchange exchange) throws Exception {
+                                exchange.getOut().setBody("123;Donald Duck\n456;John Doe");
+                            }
+                        });
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/6fae1913/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestPathMatchingTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestPathMatchingTest.java b/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestPathMatchingTest.java
new file mode 100644
index 0000000..af364ae
--- /dev/null
+++ b/components/camel-netty-http/src/test/java/org/apache/camel/component/netty/http/rest/RestPathMatchingTest.java
@@ -0,0 +1,86 @@
+/**
+ * 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.netty.http.rest;
+
+import junit.framework.TestCase;
+import org.apache.camel.component.netty.http.RestContextPathMatcher;
+
+public class RestPathMatchingTest extends TestCase {
+
+    private RestContextPathMatcher matcher = new RestContextPathMatcher("", "", true);
+
+    public void testRestPathMatcher() throws Exception {
+        assertTrue(matcher.matchRestPath("/foo/", "/foo/", true));
+        assertTrue(matcher.matchRestPath("/foo/", "foo/", true));
+        assertTrue(matcher.matchRestPath("/foo/", "foo", true));
+        assertTrue(matcher.matchRestPath("foo/", "foo", true));
+        assertTrue(matcher.matchRestPath("foo", "foo", true));
+        assertTrue(matcher.matchRestPath("foo/", "foo", true));
+        assertTrue(matcher.matchRestPath("/foo/", "foo", true));
+
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2014", true));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2014", true));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2014/", true));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2014/", true));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/{user}/list/{year}", true));
+
+        assertFalse(matcher.matchRestPath("/foo/", "/bar/", true));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2015", true));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2015", true));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2015/", true));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2015/", true));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/{user}/list/", true));
+
+        assertTrue(matcher.matchRestPath("/foo/1/list/2", "/foo/{user}/list/{year}", true));
+        assertTrue(matcher.matchRestPath("/foo/1234567890/list/2", "/foo/{user}/list/{year}", true));
+        assertTrue(matcher.matchRestPath("/foo/1234567890/list/1234567890", "/foo/{user}/list/{year}", true));
+
+        assertTrue(matcher.matchRestPath("/123/list/2014", "/{user}/list/{year}", true));
+        assertTrue(matcher.matchRestPath("/1234567890/list/2014", "/{user}/list/{year}", true));
+    }
+
+    public void testRestPathMatcherNoWildcard() throws Exception {
+        assertTrue(matcher.matchRestPath("/foo/", "/foo/", false));
+        assertTrue(matcher.matchRestPath("/foo/", "foo/", false));
+        assertTrue(matcher.matchRestPath("/foo/", "foo", false));
+        assertTrue(matcher.matchRestPath("foo/", "foo", false));
+        assertTrue(matcher.matchRestPath("foo", "foo", false));
+        assertTrue(matcher.matchRestPath("foo/", "foo", false));
+        assertTrue(matcher.matchRestPath("/foo/", "foo", false));
+
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2014", false));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2014", false));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2014/", false));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2014/", false));
+        assertTrue(matcher.matchRestPath("/foo/1234/list/2014", "/foo/{user}/list/{year}", true));
+
+        assertFalse(matcher.matchRestPath("/foo/", "/bar/", false));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2015", false));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2015", false));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/1234/list/2015/", false));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014/", "/foo/1234/list/2015/", false));
+        assertFalse(matcher.matchRestPath("/foo/1234/list/2014", "/foo/{user}/list/", false));
+
+        assertFalse(matcher.matchRestPath("/foo/1/list/2", "/foo/{user}/list/{year}", false));
+        assertFalse(matcher.matchRestPath("/foo/1234567890/list/2", "/foo/{user}/list/{year}", false));
+        assertFalse(matcher.matchRestPath("/foo/1234567890/list/1234567890", "/foo/{user}/list/{year}", false));
+
+        assertFalse(matcher.matchRestPath("/123/list/2014", "/{user}/list/{year}", false));
+        assertFalse(matcher.matchRestPath("/1234567890/list/2014", "/{user}/list/{year}", false));
+    }
+
+}