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 2015/09/23 11:35:37 UTC

[01/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Repository: camel
Updated Branches:
  refs/heads/master 8ce5a2b2f -> e98a801b5


CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: e82caac4605d8a7dc55a5cac2629f5a9b21b9c08
Parents: 646c9bc
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 17:33:34 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:03 2015 +0200

----------------------------------------------------------------------
 .../camel/spi/RestApiResponseAdapter.java       | 32 ---------
 .../spi/RestApiResponseAdapterFactory.java      | 34 ---------
 .../netty4/http/NettyHttpComponent.java         | 70 +++++++++++++++---
 .../http/NettyRestApiResponseAdapter.java       | 48 -------------
 .../http/handlers/HttpClientChannelHandler.java |  3 +-
 components/camel-swagger-java/pom.xml           |  5 ++
 .../swagger/ExchangeRestApiResponseAdapter.java | 45 ++++++++++++
 .../camel/swagger/RestApiResponseAdapter.java   | 28 ++++++++
 .../camel/swagger/RestSwaggerProcessor.java     | 76 +-------------------
 .../camel/swagger/RestSwaggerSupport.java       | 23 ++++--
 .../swagger/servlet/RestSwaggerServlet.java     |  6 +-
 .../servlet/ServletRestApiResponseAdapter.java  |  2 +-
 .../apache/camel/swagger/RestApiNettyTest.java  | 65 +++++++++++++++++
 13 files changed, 230 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
deleted file mode 100644
index 01b1e4f..0000000
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
+++ /dev/null
@@ -1,32 +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.spi;
-
-import java.io.IOException;
-
-/**
- * An adapter to allow Camel rest-api to use Camel components to render the api response.
- */
-public interface RestApiResponseAdapter {
-
-    void addHeader(String name, String value);
-
-    void writeBytes(byte[] bytes) throws IOException;
-
-    void noContent();
-
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
deleted file mode 100644
index 71bffd8..0000000
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
+++ /dev/null
@@ -1,34 +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.spi;
-
-import org.apache.camel.Exchange;
-
-/**
- * Factory to create {@link RestApiResponseAdapter} which allows Camel components
- * to provide a response adapter to be used by the rest-dsl api support.
- */
-public interface RestApiResponseAdapterFactory {
-
-    /**
-     * Creates a new {@link RestApiResponseAdapter}
-     *
-     * @param exchange  the exchange
-     * @return the adapter
-     */
-    RestApiResponseAdapter newAdapter(Exchange exchange);
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
index d036cff..9012d93 100644
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Consumer;
 import org.apache.camel.Endpoint;
-import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.component.netty4.NettyComponent;
 import org.apache.camel.component.netty4.NettyConfiguration;
@@ -32,8 +31,7 @@ import org.apache.camel.component.netty4.NettyServerBootstrapConfiguration;
 import org.apache.camel.component.netty4.http.handlers.HttpServerMultiplexChannelHandler;
 import org.apache.camel.spi.HeaderFilterStrategy;
 import org.apache.camel.spi.HeaderFilterStrategyAware;
-import org.apache.camel.spi.RestApiResponseAdapter;
-import org.apache.camel.spi.RestApiResponseAdapterFactory;
+import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
@@ -49,7 +47,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Netty HTTP based component.
  */
-public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory, RestApiResponseAdapterFactory {
+public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory, RestApiConsumerFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(NettyHttpComponent.class);
 
@@ -300,12 +298,66 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
     }
 
     @Override
-    public RestApiResponseAdapter newAdapter(Exchange exchange) {
-        NettyHttpMessage http = exchange.getIn(NettyHttpMessage.class);
-        if (http != null) {
-            return new NettyRestApiResponseAdapter(http.getHttpResponse());
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
+        String path = FileUtil.stripLeadingSeparator(contextPath);
+
+        String scheme = "http";
+        String host = "";
+        int port = 0;
+
+        // if no explicit port/host configured, then use port from rest configuration
+        RestConfiguration config = getCamelContext().getRestConfiguration("netty4-http", true);
+        if (config.getScheme() != null) {
+            scheme = config.getScheme();
+        }
+        if (config.getHost() != null) {
+            host = config.getHost();
+        }
+        int num = config.getPort();
+        if (num > 0) {
+            port = num;
+        }
+
+        // if no explicit hostname set then resolve the hostname
+        if (ObjectHelper.isEmpty(host)) {
+            if (config.getRestHostNameResolver() == RestConfiguration.RestHostNameResolver.localHostName) {
+                host = HostUtils.getLocalHostName();
+            } else if (config.getRestHostNameResolver() == RestConfiguration.RestHostNameResolver.localIp) {
+                host = HostUtils.getLocalIp();
+            }
         }
-        return null;
+
+        Map<String, Object> map = new HashMap<String, Object>();
+        // build query string, and append any endpoint configuration properties
+        if (config != null && (config.getComponent() == null || config.getComponent().equals("netty4-http"))) {
+            // setup endpoint options
+            if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
+                map.putAll(config.getEndpointProperties());
+            }
+        }
+
+        String query = URISupport.createQueryString(map);
+
+        String url = "netty4-http:%s://%s:%s/%s?httpMethodRestrict=%s&matchOnUriPrefix=true";
+        // must use upper case for restrict
+        String restrict = "GET";
+        // get the endpoint
+        url = String.format(url, scheme, host, port, path, restrict);
+
+        if (!query.isEmpty()) {
+            url = url + "&" + query;
+        }
+
+        NettyHttpEndpoint endpoint = camelContext.getEndpoint(url, NettyHttpEndpoint.class);
+        setProperties(endpoint, parameters);
+
+        // configure consumer properties
+        Consumer consumer = endpoint.createConsumer(processor);
+        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+            setProperties(consumer, config.getConsumerProperties());
+        }
+
+        return consumer;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
deleted file mode 100644
index 263f1e3..0000000
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
+++ /dev/null
@@ -1,48 +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.netty4.http;
-
-import java.io.IOException;
-
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.camel.spi.RestApiResponseAdapter;
-
-public class NettyRestApiResponseAdapter implements RestApiResponseAdapter {
-
-    private final FullHttpResponse httpResponse;
-
-    public NettyRestApiResponseAdapter(FullHttpResponse httpResponse) {
-        this.httpResponse = httpResponse;
-    }
-
-    @Override
-    public void addHeader(String name, String value) {
-        httpResponse.headers().set(name, value);
-    }
-
-    @Override
-    public void writeBytes(byte[] bytes) throws IOException {
-        httpResponse.content().writeBytes(bytes);
-    }
-
-    @Override
-    public void noContent() {
-        httpResponse.setStatus(HttpResponseStatus.NO_CONTENT);
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/handlers/HttpClientChannelHandler.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/handlers/HttpClientChannelHandler.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/handlers/HttpClientChannelHandler.java
index 6657c24..1a9ab1e 100644
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/handlers/HttpClientChannelHandler.java
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/handlers/HttpClientChannelHandler.java
@@ -26,9 +26,8 @@ import org.apache.camel.component.netty4.handlers.ClientChannelHandler;
 import org.apache.camel.component.netty4.http.NettyHttpProducer;
 
 /**
- * Netty HTTP {@link org.apache.camel.component.netty.handlers.ClientChannelHandler} that handles the response combing
+ * Netty HTTP {@link org.apache.camel.component.netty4.handlers.ClientChannelHandler} that handles the response combing
  * back from the HTTP server, called by this client.
- *
  */
 public class HttpClientChannelHandler extends ClientChannelHandler {
     private final NettyHttpProducer producer;

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/pom.xml b/components/camel-swagger-java/pom.xml
index 742a9e5..33072cf 100644
--- a/components/camel-swagger-java/pom.xml
+++ b/components/camel-swagger-java/pom.xml
@@ -114,6 +114,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-netty4-http</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/ExchangeRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/ExchangeRestApiResponseAdapter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/ExchangeRestApiResponseAdapter.java
new file mode 100644
index 0000000..e6e911b
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/ExchangeRestApiResponseAdapter.java
@@ -0,0 +1,45 @@
+/**
+ * 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.swagger;
+
+import java.io.IOException;
+
+import org.apache.camel.Exchange;
+
+public class ExchangeRestApiResponseAdapter implements RestApiResponseAdapter {
+
+    private final Exchange exchange;
+
+    public ExchangeRestApiResponseAdapter(Exchange exchange) {
+        this.exchange = exchange;
+    }
+
+    @Override
+    public void addHeader(String name, String value) {
+        exchange.getIn().setHeader(name, value);
+    }
+
+    @Override
+    public void writeBytes(byte[] bytes) throws IOException {
+        exchange.getIn().setBody(bytes);
+    }
+
+    @Override
+    public void noContent() {
+        exchange.getIn().setHeader(Exchange.HTTP_RESPONSE_CODE, 204);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestApiResponseAdapter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestApiResponseAdapter.java
new file mode 100644
index 0000000..9a64140
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestApiResponseAdapter.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
+ *
+ *      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.swagger;
+
+import java.io.IOException;
+
+public interface RestApiResponseAdapter {
+
+    void addHeader(String name, String value);
+
+    void writeBytes(byte[] bytes) throws IOException;
+
+    void noContent();
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
index 4b3b9cd..9380d91 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -17,32 +17,23 @@
 package org.apache.camel.swagger;
 
 import java.util.Map;
-import java.util.Set;
 
 import io.swagger.jaxrs.config.BeanConfig;
-import org.apache.camel.CamelContext;
-import org.apache.camel.Component;
 import org.apache.camel.Exchange;
-import org.apache.camel.NoSuchBeanException;
 import org.apache.camel.Processor;
-import org.apache.camel.spi.RestApiResponseAdapter;
-import org.apache.camel.spi.RestApiResponseAdapterFactory;
-import org.apache.camel.support.ServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class RestSwaggerProcessor extends ServiceSupport implements Processor {
+public class RestSwaggerProcessor implements Processor {
 
     private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerProcessor.class);
     private final BeanConfig swaggerConfig;
     private final RestSwaggerSupport support;
-    private final String componentName;
 
     public RestSwaggerProcessor(Map<String, Object> parameters) {
         support = new RestSwaggerSupport();
         swaggerConfig = new BeanConfig();
         support.initSwagger(swaggerConfig, parameters);
-        componentName = (String) parameters.get("componentName");
     }
 
     @Override
@@ -51,9 +42,9 @@ public class RestSwaggerProcessor extends ServiceSupport implements Processor {
         String contextId;
         String route = exchange.getIn().getHeader(Exchange.HTTP_PATH, String.class);
 
-        try {
-            RestApiResponseAdapter adapter = lookupAdapter(exchange);
+        RestApiResponseAdapter adapter = new ExchangeRestApiResponseAdapter(exchange);
 
+        try {
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
                 support.renderCamelContexts(adapter);
@@ -75,65 +66,4 @@ public class RestSwaggerProcessor extends ServiceSupport implements Processor {
         }
     }
 
-    protected RestApiResponseAdapter lookupAdapter(Exchange exchange) {
-        CamelContext camelContext = exchange.getContext();
-
-        RestApiResponseAdapterFactory factory = null;
-
-        if (componentName != null) {
-            Object comp = camelContext.getRegistry().lookupByName(componentName);
-            if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
-                factory = (RestApiResponseAdapterFactory) comp;
-            } else {
-                comp = camelContext.getComponent(componentName);
-                if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
-                    factory = (RestApiResponseAdapterFactory) comp;
-                }
-            }
-
-            if (factory == null) {
-                if (comp != null) {
-                    throw new IllegalArgumentException("Component " + componentName + " is not a RestApiResponseAdapterFactory");
-                } else {
-                    throw new NoSuchBeanException(componentName, RestApiResponseAdapterFactory.class.getName());
-                }
-            }
-        }
-
-        // try all components
-        if (factory == null) {
-            for (String name : camelContext.getComponentNames()) {
-                Component comp = camelContext.getComponent(name);
-                if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
-                    factory = (RestApiResponseAdapterFactory) comp;
-                    break;
-                }
-            }
-        }
-
-        // lookup in registry
-        if (factory == null) {
-            Set<RestApiResponseAdapterFactory> factories = camelContext.getRegistry().findByType(RestApiResponseAdapterFactory.class);
-            if (factories != null && factories.size() == 1) {
-                factory = factories.iterator().next();
-            }
-        }
-
-        if (factory != null) {
-            return factory.newAdapter(exchange);
-        } else {
-            throw new IllegalStateException("Cannot find RestApiResponseAdapterFactory in Registry or as a Component to use");
-        }
-
-    }
-
-    @Override
-    protected void doStart() throws Exception {
-        // noop
-    }
-
-    @Override
-    protected void doStop() throws Exception {
-        // noop
-    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index e558ee3..5790651 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -32,11 +32,11 @@ import io.swagger.models.Contact;
 import io.swagger.models.Info;
 import io.swagger.models.License;
 import io.swagger.models.Swagger;
+import org.apache.camel.Exchange;
 import org.apache.camel.impl.DefaultClassResolver;
 import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.rest.RestDefinition;
 import org.apache.camel.model.rest.RestsDefinition;
-import org.apache.camel.spi.RestApiResponseAdapter;
 import org.apache.camel.util.CamelVersionHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -183,6 +183,10 @@ public class RestSwaggerSupport {
             mapper.enable(SerializationFeature.INDENT_OUTPUT);
             mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
             byte[] bytes = mapper.writeValueAsBytes(swagger);
+
+            int len = bytes.length;
+            response.addHeader(Exchange.CONTENT_LENGTH, "" + len);
+
             response.writeBytes(bytes);
         } else {
             response.noContent();
@@ -201,16 +205,25 @@ public class RestSwaggerSupport {
             response.addHeader("Access-Control-Allow-Origin", "*");
         }
 
+        response.addHeader(Exchange.CONTENT_TYPE, "application/json");
+
+        StringBuffer sb = new StringBuffer();
+
         List<String> contexts = findCamelContexts();
-        response.writeBytes("[\n".getBytes());
+        sb.append("[\n");
         for (int i = 0; i < contexts.size(); i++) {
             String name = contexts.get(i);
-            response.writeBytes(("{\"name\": \"" + name + "\"}").getBytes());
+            sb.append("{\"name\": \"").append(name).append("\"}");
             if (i < contexts.size() - 1) {
-                response.writeBytes(",\n".getBytes());
+                sb.append(",\n");
             }
         }
-        response.writeBytes("\n]".getBytes());
+        sb.append("\n]");
+
+        int len = sb.length();
+        response.addHeader(Exchange.CONTENT_LENGTH, "" + len);
+
+        response.writeBytes(sb.toString().getBytes());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
index 3159776..a5ca00a 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
@@ -29,7 +29,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import io.swagger.jaxrs.config.BeanConfig;
-import org.apache.camel.spi.RestApiResponseAdapter;
+import org.apache.camel.swagger.RestApiResponseAdapter;
 import org.apache.camel.swagger.RestSwaggerSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,11 +68,11 @@ public class RestSwaggerServlet extends HttpServlet {
             initBaseAndApiPaths(request);
         }
 
-        RestApiResponseAdapter adapter = new ServletRestApiResponseAdapter(response);
-
         String contextId;
         String route = request.getPathInfo();
 
+        RestApiResponseAdapter adapter = new ServletRestApiResponseAdapter(response);
+
         try {
 
             // render list of camel contexts as root

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
index bae3aec..5c022c3 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
@@ -19,7 +19,7 @@ package org.apache.camel.swagger.servlet;
 import java.io.IOException;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.camel.spi.RestApiResponseAdapter;
+import org.apache.camel.swagger.RestApiResponseAdapter;
 
 public class ServletRestApiResponseAdapter implements RestApiResponseAdapter {
 

http://git-wip-us.apache.org/repos/asf/camel/blob/e82caac4/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
new file mode 100644
index 0000000..6379515
--- /dev/null
+++ b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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.swagger;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.model.rest.RestParamType;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class RestApiNettyTest extends CamelTestSupport {
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry jndi = super.createRegistry();
+        jndi.bind("SwaggerRestApiProcessorFactory", new SwaggerRestApiProcessorFactory());
+        return jndi;
+    }
+
+    @Override
+    protected boolean useJmx() {
+        return true;
+    }
+
+    @Test
+    public void testApi() throws Exception {
+        Thread.sleep(999999);
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                restConfiguration().component("netty4-http").host("localhost").port(8080).apiContextPath("/api-doc");
+
+                rest("/hello").consumes("application/json").produces("application/json")
+                    .get("/hi/{name}").description("Saying hi")
+                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
+                        .to("log:hi")
+                    .get("/bye/{name}").description("Saying bye")
+                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
+                        .responseMessage().code(200).message("A reply message").endResponseMessage()
+                        .to("log:bye")
+                    .post("/bye").description("To update the greeting message").consumes("application/xml").produces("application/xml")
+                        .param().name("greeting").type(RestParamType.body).dataType("string").description("Message to use as greeting").endParam()
+                        .to("log:bye");
+            }
+        };
+    }
+}


[06/16] camel git commit: Polished

Posted by da...@apache.org.
Polished


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

Branch: refs/heads/master
Commit: aa3e1173e7e28d2c69e7d9dcf815c9564d4641da
Parents: e82caac
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 17:34:02 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:04 2015 +0200

----------------------------------------------------------------------
 components/camel-netty4-http/pom.xml | 139 +++++++++++++++---------------
 1 file changed, 70 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/aa3e1173/components/camel-netty4-http/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/pom.xml b/components/camel-netty4-http/pom.xml
index ad302f6..8e80b61 100644
--- a/components/camel-netty4-http/pom.xml
+++ b/components/camel-netty4-http/pom.xml
@@ -14,78 +14,79 @@
 	the specific language governing permissions and limitations under the
 	License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-	<modelVersion>4.0.0</modelVersion>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
 
-	<parent>
-		<groupId>org.apache.camel</groupId>
-		<artifactId>components</artifactId>
-		<version>2.16-SNAPSHOT</version>
-	</parent>
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>components</artifactId>
+    <version>2.16-SNAPSHOT</version>
+  </parent>
 
-	<artifactId>camel-netty4-http</artifactId>
-	<packaging>bundle</packaging>
-	<name>Camel :: Netty4 HTTP</name>
-	<description>Camel Netty4 HTTP support</description>
+  <artifactId>camel-netty4-http</artifactId>
+  <packaging>bundle</packaging>
+  <name>Camel :: Netty4 HTTP</name>
+  <description>Camel Netty4 HTTP support</description>
 
-	<properties>
-		<camel.osgi.export.pkg>
-			org.apache.camel.component.netty4.http.*
-		</camel.osgi.export.pkg>
-		<camel.osgi.export.service>org.apache.camel.spi.ComponentResolver;component=netty4-http</camel.osgi.export.service>
-	</properties>
+  <properties>
+    <camel.osgi.export.pkg>
+      org.apache.camel.component.netty4.http.*
+    </camel.osgi.export.pkg>
+    <camel.osgi.export.service>org.apache.camel.spi.ComponentResolver;component=netty4-http</camel.osgi.export.service>
+  </properties>
 
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.camel</groupId>
-            <artifactId>camel-netty4</artifactId>
-		</dependency>
-		<dependency>
-            <groupId>io.netty</groupId>
-            <artifactId>netty-codec-http</artifactId>
-            <version>${netty-version}</version>
-        </dependency>
-        
-		<!-- testing -->
-		<dependency>
-            <groupId>com.jcraft</groupId>
-            <artifactId>jzlib</artifactId>
-            <version>${jzlib-version}</version>
-            <scope>test</scope>
-        </dependency>
-		<dependency>
-			<groupId>org.apache.camel</groupId>
-			<artifactId>camel-test-spring</artifactId>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.camel</groupId>
-			<artifactId>camel-http</artifactId>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<scope>test</scope>
-		</dependency>
-        <!-- for testing rest-dsl -->
-        <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-jackson</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-jaxb</artifactId>
-            <scope>test</scope>
-        </dependency>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-netty4</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-codec-http</artifactId>
+      <version>${netty-version}</version>
+    </dependency>
+
+    <!-- testing -->
+    <dependency>
+      <groupId>com.jcraft</groupId>
+      <artifactId>jzlib</artifactId>
+      <version>${jzlib-version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-test-spring</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-http</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <!-- for testing rest-dsl -->
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-jackson</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-jaxb</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- logging -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
 
-		<!-- logging -->
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-log4j12</artifactId>
-			<scope>test</scope>
-		</dependency>
-    </dependencies>
-    
 </project>


[16/16] camel git commit: netty-all has all the http codec so no need for other JARs

Posted by da...@apache.org.
netty-all has all the http codec so no need for other JARs


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

Branch: refs/heads/master
Commit: e98a801b54699581046e12ac0157042553a6a29e
Parents: 17ef02f
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 11:36:11 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 11:36:11 2015 +0200

----------------------------------------------------------------------
 components/camel-netty4-http/pom.xml | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/e98a801b/components/camel-netty4-http/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/pom.xml b/components/camel-netty4-http/pom.xml
index f5942eb..ee2ae7b 100644
--- a/components/camel-netty4-http/pom.xml
+++ b/components/camel-netty4-http/pom.xml
@@ -41,12 +41,8 @@
       <groupId>org.apache.camel</groupId>
       <artifactId>camel-netty4</artifactId>
     </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-codec-http</artifactId>
-      <version>${netty-version}</version>
-    </dependency>
-
+    <!-- we use netty-all as dependency which has HTTP included -->
+ 
     <!-- testing -->
     <dependency>
       <groupId>com.jcraft</groupId>


[03/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: b88093cb668097c3e3cf7814e6286fd7ac5cf878
Parents: 3818efe
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 16:25:19 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:03 2015 +0200

----------------------------------------------------------------------
 .../camel/component/rest/RestApiComponent.java  |  46 +++++
 .../camel/component/rest/RestApiEndpoint.java   | 204 +++++++++++++++++++
 .../camel/component/rest/RestApiProducer.java   |  50 +++++
 .../model/rest/RestConfigurationDefinition.java |  33 +++
 .../apache/camel/model/rest/RestDefinition.java |  41 +++-
 .../camel/spi/RestApiConsumerFactory.java       |  43 ++++
 .../camel/spi/RestApiProcessorFactory.java      |  40 ++++
 .../camel/spi/RestApiResponseAdapter.java       |  33 +++
 .../org/apache/camel/spi/RestConfiguration.java |  17 ++
 .../org/apache/camel/component/rest-api         |  18 ++
 .../rest/DummyRestConsumerFactory.java          |  14 +-
 .../rest/DummyRestProcessorFactory.java         |  38 ++++
 .../camel/component/rest/FromRestApiTest.java   |  62 ++++++
 .../camel/swagger/RestSwaggerProcessor.java     |  81 ++++++++
 .../camel/swagger/RestSwaggerSupport.java       |  66 +++---
 .../swagger/SwaggerRestApiProcessorFactory.java |  32 +++
 .../swagger/servlet/RestSwaggerServlet.java     |  21 +-
 .../servlet/ServletRestApiResponseAdapter.java  |  48 +++++
 .../servlet/ServletSwaggerApiProvider.java      |  55 -----
 .../camel/swagger/spi/SwaggerApiProvider.java   |  32 ---
 20 files changed, 848 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/component/rest/RestApiComponent.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiComponent.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiComponent.java
new file mode 100644
index 0000000..9abaaac
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiComponent.java
@@ -0,0 +1,46 @@
+/**
+ * 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.rest;
+
+import java.util.Map;
+
+import org.apache.camel.Endpoint;
+import org.apache.camel.impl.UriEndpointComponent;
+
+public class RestApiComponent extends UriEndpointComponent {
+
+    public RestApiComponent() {
+        super(RestApiEndpoint.class);
+    }
+
+    @Override
+    protected Endpoint createEndpoint(String uri, String remaining, Map<String, Object> parameters) throws Exception {
+        RestApiEndpoint answer = new RestApiEndpoint(uri, this);
+        answer.setPath(remaining);
+
+        setProperties(answer, parameters);
+        answer.setParameters(parameters);
+
+        // if no explicit component name was given, then fallback and use default configured component name
+        if (answer.getComponentName() == null && getCamelContext().getRestConfiguration() != null) {
+            answer.setComponentName(getCamelContext().getRestConfiguration().getComponent());
+        }
+
+        return answer;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
new file mode 100644
index 0000000..a4ff9f5
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
@@ -0,0 +1,204 @@
+/**
+ * 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.rest;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.camel.Component;
+import org.apache.camel.Consumer;
+import org.apache.camel.NoSuchBeanException;
+import org.apache.camel.Processor;
+import org.apache.camel.Producer;
+import org.apache.camel.impl.DefaultEndpoint;
+import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.RestApiConsumerFactory;
+import org.apache.camel.spi.RestApiProcessorFactory;
+import org.apache.camel.spi.RestConfiguration;
+import org.apache.camel.spi.UriEndpoint;
+import org.apache.camel.spi.UriParam;
+import org.apache.camel.spi.UriPath;
+
+@UriEndpoint(scheme = "rest-api", title = "REST API", syntax = "rest-api:path", consumerOnly = true, label = "core,rest")
+public class RestApiEndpoint extends DefaultEndpoint {
+
+    @UriPath @Metadata(required = "true")
+    private String path;
+    @UriParam
+    private String componentName;
+
+    private Map<String, Object> parameters;
+
+    public RestApiEndpoint(String endpointUri, RestApiComponent component) {
+        super(endpointUri, component);
+    }
+
+    @Override
+    public RestApiComponent getComponent() {
+        return (RestApiComponent) super.getComponent();
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    /**
+     * The base path
+     */
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    public String getComponentName() {
+        return componentName;
+    }
+
+    /**
+     * The Camel Rest component to use for the REST transport, such as restlet, spark-rest.
+     * If no component has been explicit configured, then Camel will lookup if there is a Camel component
+     * that integrates with the Rest DSL, or if a org.apache.camel.spi.RestConsumerFactory is registered in the registry.
+     * If either one is found, then that is being used.
+     */
+    public void setComponentName(String componentName) {
+        this.componentName = componentName;
+    }
+
+    public Map<String, Object> getParameters() {
+        return parameters;
+    }
+
+    /**
+     * Additional parameters to configure the consumer of the REST transport for this REST service
+     */
+    public void setParameters(Map<String, Object> parameters) {
+        this.parameters = parameters;
+    }
+
+    @Override
+    public Producer createProducer() throws Exception {
+        RestApiProcessorFactory factory = null;
+
+        // lookup in registry
+        Set<RestApiProcessorFactory> factories = getCamelContext().getRegistry().findByType(RestApiProcessorFactory.class);
+        if (factories != null && factories.size() == 1) {
+            factory = factories.iterator().next();
+        }
+
+        if (factory != null) {
+
+            // calculate the url to the rest API service
+            String path = getPath();
+            if (path != null && !path.startsWith("/")) {
+                path = "/" + path;
+            }
+
+
+            Processor processor = factory.createApiProcessor(getCamelContext(), path, getParameters());
+            return new RestApiProducer(this, processor);
+        } else {
+            throw new IllegalStateException("Cannot find RestApiProcessorFactory in Registry");
+        }
+    }
+
+    @Override
+    public Consumer createConsumer(Processor processor) throws Exception {
+        RestApiConsumerFactory factory = null;
+        String cname = null;
+        if (getComponentName() != null) {
+            Object comp = getCamelContext().getRegistry().lookupByName(getComponentName());
+            if (comp != null && comp instanceof RestApiConsumerFactory) {
+                factory = (RestApiConsumerFactory) comp;
+            } else {
+                comp = getCamelContext().getComponent(getComponentName());
+                if (comp != null && comp instanceof RestApiConsumerFactory) {
+                    factory = (RestApiConsumerFactory) comp;
+                }
+            }
+
+            if (factory == null) {
+                if (comp != null) {
+                    throw new IllegalArgumentException("Component " + getComponentName() + " is not a RestApiConsumerFactory");
+                } else {
+                    throw new NoSuchBeanException(getComponentName(), RestApiConsumerFactory.class.getName());
+                }
+            }
+            cname = getComponentName();
+        }
+
+        // try all components
+        if (factory == null) {
+            for (String name : getCamelContext().getComponentNames()) {
+                Component comp = getCamelContext().getComponent(name);
+                if (comp != null && comp instanceof RestApiConsumerFactory) {
+                    factory = (RestApiConsumerFactory) comp;
+                    cname = name;
+                    break;
+                }
+            }
+        }
+
+        // lookup in registry
+        if (factory == null) {
+            Set<RestApiConsumerFactory> factories = getCamelContext().getRegistry().findByType(RestApiConsumerFactory.class);
+            if (factories != null && factories.size() == 1) {
+                factory = factories.iterator().next();
+            }
+        }
+
+        if (factory != null) {
+
+            RestConfiguration config = getCamelContext().getRestConfiguration(cname, true);
+
+            // calculate the url to the rest API service
+            String path = getPath();
+            if (path != null && !path.startsWith("/")) {
+                path = "/" + path;
+            }
+
+            // TODO: is this needed?
+            // there may be an optional context path configured to help Camel calculate the correct urls for the REST services
+            // this may be needed when using camel-serlvet where we cannot get the actual context-path or port number of the servlet engine
+            // during init of the servlet
+/*            String contextPath = config.getApiContextPath();
+            if (contextPath != null) {
+                if (!contextPath.startsWith("/")) {
+                    path = "/" + contextPath + path;
+                } else {
+                    path = contextPath + path;
+                }
+            }
+*/
+
+            Consumer consumer = factory.createApiConsumer(getCamelContext(), processor, path, getParameters());
+            configureConsumer(consumer);
+
+            return consumer;
+        } else {
+            throw new IllegalStateException("Cannot find RestApiConsumerFactory in Registry or as a Component to use");
+        }
+    }
+
+    @Override
+    public boolean isSingleton() {
+        return true;
+    }
+
+    @Override
+    public boolean isLenientProperties() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/component/rest/RestApiProducer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiProducer.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiProducer.java
new file mode 100644
index 0000000..77c994d
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiProducer.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
+ *
+ *      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.rest;
+
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.DefaultProducer;
+import org.apache.camel.util.ServiceHelper;
+
+public class RestApiProducer extends DefaultProducer {
+
+    private final Processor processor;
+
+    public RestApiProducer(Endpoint endpoint, Processor processor) {
+        super(endpoint);
+        this.processor = processor;
+    }
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+        processor.process(exchange);
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+        ServiceHelper.startService(processor);
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        super.doStop();
+        ServiceHelper.stopService(processor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
index f44f5ba..1ed3239 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
@@ -55,6 +55,9 @@ public class RestConfigurationDefinition {
     private String contextPath;
 
     @XmlAttribute
+    private String apiContextPath;
+
+    @XmlAttribute
     private RestHostNameResolver hostNameResolver;
 
     @XmlAttribute @Metadata(defaultValue = "auto")
@@ -154,6 +157,22 @@ public class RestConfigurationDefinition {
         this.contextPath = contextPath;
     }
 
+    public String getApiContextPath() {
+        return apiContextPath;
+    }
+
+    /**
+     * Sets a leading API context-path the REST API services will be using.
+     * <p/>
+     * This can be used when using components such as <tt>camel-servlet</tt> where the deployed web application
+     * is deployed using a context-path.
+     *
+     * @param contextPath the API context path
+     */
+    public void setApiContextPath(String contextPath) {
+        this.apiContextPath = contextPath;
+    }
+
     public RestHostNameResolver getHostNameResolver() {
         return hostNameResolver;
     }
@@ -339,6 +358,17 @@ public class RestConfigurationDefinition {
     }
 
     /**
+     * Sets a leading API context-path the REST API services will be using.
+     * <p/>
+     * This can be used when using components such as <tt>camel-servlet</tt> where the deployed web application
+     * is deployed using a context-path.
+     */
+    public RestConfigurationDefinition apiContextPath(String contextPath) {
+        setApiContextPath(contextPath);
+        return this;
+    }
+
+    /**
      * Sets a leading context-path the REST services will be using.
      * <p/>
      * This can be used when using components such as <tt>camel-servlet</tt> where the deployed web application
@@ -484,6 +514,9 @@ public class RestConfigurationDefinition {
         if (port != null) {
             answer.setPort(CamelContextHelper.parseInteger(context, port));
         }
+        if (apiContextPath != null) {
+            answer.setApiContextPath(CamelContextHelper.parseText(context, apiContextPath));
+        }
         if (contextPath != null) {
             answer.setContextPath(CamelContextHelper.parseText(context, contextPath));
         }

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
index 9e1a34d..7f93637 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -534,10 +533,48 @@ public class RestDefinition extends OptionalIdentifiedDefinition<RestDefinition>
         }
         for (RestConfiguration config : camelContext.getRestConfigurations()) {
             addRouteDefinition(camelContext, answer, config.getComponent());
+            if (config.getApiContextPath() != null) {
+                addApiRouteDefinition(camelContext, answer, config);
+            }
         }
         return answer;
     }
-    
+
+    private void addApiRouteDefinition(CamelContext camelContext, List<RouteDefinition> answer, RestConfiguration configuration) {
+        RouteDefinition route = new RouteDefinition();
+
+        // create the from endpoint uri which is using the rest-api component
+        String from = "rest-api:" + configuration.getApiContextPath();
+
+        // append options
+        Map<String, Object> options = new HashMap<String, Object>();
+
+        String routeId = "rest-api-" + route.idOrCreate(camelContext.getNodeIdFactory());
+        options.put("routeId", routeId);
+        if (configuration.getComponent() != null && !configuration.getComponent().isEmpty()) {
+            options.put("componentName", configuration.getComponent());
+        }
+
+        if (!options.isEmpty()) {
+            String query;
+            try {
+                query = URISupport.createQueryString(options);
+            } catch (URISyntaxException e) {
+                throw ObjectHelper.wrapRuntimeCamelException(e);
+            }
+            from = from + "?" + query;
+        }
+
+        // we use the same uri as the producer (so we have a little route for the rest api)
+        String to = from;
+
+        // the route should be from this rest endpoint
+        route.fromRest(from);
+        route.to(to);
+        route.setRestDefinition(this);
+        answer.add(route);
+    }
+
     private void addRouteDefinition(CamelContext camelContext, List<RouteDefinition> answer, String component) {
         for (VerbDefinition verb : getVerbs()) {
             // either the verb has a singular to or a embedded route

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
new file mode 100644
index 0000000..acad9a8
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
@@ -0,0 +1,43 @@
+/**
+ * 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.spi;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Consumer;
+import org.apache.camel.Processor;
+
+public interface RestApiConsumerFactory {
+
+    /**
+     * Creates a new REST API <a
+     * href="http://camel.apache.org/event-driven-consumer.html">Event
+     * Driven Consumer</a>, which provides API listing of the REST services
+     *
+     * @param camelContext the camel context
+     * @param processor    the processor
+     * @param contextPath  the context-path
+     * @param parameters   additional parameters
+     *
+     * @return a newly created REST API consumer
+     * @throws Exception can be thrown
+     */
+    Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception;
+
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
new file mode 100644
index 0000000..6650213
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
@@ -0,0 +1,40 @@
+/**
+ * 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.spi;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Processor;
+
+public interface RestApiProcessorFactory {
+
+    /**
+     * Creates a new REST API <a
+     * href="http://camel.apache.org/processor.html">Processor
+     * </a>, which provides API listing of the REST services
+     *
+     * @param camelContext the camel context
+     * @param contextPath  the context-path
+     * @param parameters   additional parameters
+     *
+     * @return a newly created REST API provider
+     * @throws Exception can be thrown
+     */
+    Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
new file mode 100644
index 0000000..3575d8d
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
@@ -0,0 +1,33 @@
+/**
+ * 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.spi;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * An adapter to allow Camel rest-api to use Camel components to render the api response.
+ */
+public interface RestApiResponseAdapter {
+
+    void addHeader(String name, String value);
+
+    OutputStream getOutputStream() throws IOException;
+
+    void noContent();
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
index 5daecd9..7e5e3fe 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
@@ -42,6 +42,7 @@ public class RestConfiguration {
     private String host;
     private int port;
     private String contextPath;
+    private String apiContextPath;
     private RestHostNameResolver restHostNameResolver = RestHostNameResolver.localHostName;
     private RestBindingMode bindingMode = RestBindingMode.off;
     private boolean skipBindingOnErrorCode = true;
@@ -147,6 +148,22 @@ public class RestConfiguration {
         this.contextPath = contextPath;
     }
 
+    public String getApiContextPath() {
+        return apiContextPath;
+    }
+
+    /**
+     * Sets a leading API context-path the REST API services will be using.
+     * <p/>
+     * This can be used when using components such as <tt>camel-servlet</tt> where the deployed web application
+     * is deployed using a context-path.
+     *
+     * @param contextPath the API context path
+     */
+    public void setApiContextPath(String contextPath) {
+        this.apiContextPath = contextPath;
+    }
+
     /**
      * Gets the resolver to use for resolving hostname
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/main/resources/META-INF/services/org/apache/camel/component/rest-api
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/META-INF/services/org/apache/camel/component/rest-api b/camel-core/src/main/resources/META-INF/services/org/apache/camel/component/rest-api
new file mode 100644
index 0000000..385496b
--- /dev/null
+++ b/camel-core/src/main/resources/META-INF/services/org/apache/camel/component/rest-api
@@ -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.rest.RestApiComponent
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
index 84819a4..d51d3d3 100644
--- a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
@@ -23,9 +23,10 @@ import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.component.seda.SedaEndpoint;
 import org.apache.camel.impl.ActiveMQUuidGenerator;
+import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestConsumerFactory;
 
-public class DummyRestConsumerFactory implements RestConsumerFactory {
+public class DummyRestConsumerFactory implements RestConsumerFactory, RestApiConsumerFactory {
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
@@ -45,4 +46,15 @@ public class DummyRestConsumerFactory implements RestConsumerFactory {
         return seda.createConsumer(processor);
     }
 
+    @Override
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
+        // just use a seda endpoint for testing purpose
+        String id = ActiveMQUuidGenerator.generateSanitizedId(contextPath);
+        // remove leading dash as we add that ourselves
+        if (id.startsWith("-")) {
+            id = id.substring(1);
+        }
+        SedaEndpoint seda = camelContext.getEndpoint("seda:api:" + "-" + id, SedaEndpoint.class);
+        return seda.createConsumer(processor);
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
new file mode 100644
index 0000000..ecb8c6c
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.rest;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.RestApiProcessorFactory;
+
+public class DummyRestProcessorFactory implements RestApiProcessorFactory {
+
+    @Override
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception {
+        return new Processor() {
+            @Override
+            public void process(Exchange exchange) throws Exception {
+                // noop;
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/camel-core/src/test/java/org/apache/camel/component/rest/FromRestApiTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/FromRestApiTest.java b/camel-core/src/test/java/org/apache/camel/component/rest/FromRestApiTest.java
new file mode 100644
index 0000000..9294381
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/FromRestApiTest.java
@@ -0,0 +1,62 @@
+/**
+ * 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.rest;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.model.ToDefinition;
+import org.apache.camel.model.rest.RestDefinition;
+
+public class FromRestApiTest extends ContextTestSupport {
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry jndi = super.createRegistry();
+        jndi.bind("dummy-rest", new DummyRestConsumerFactory());
+        jndi.bind("dummy-rest-api", new DummyRestProcessorFactory());
+        return jndi;
+    }
+
+    public void testFromRestModel() throws Exception {
+
+        assertEquals(1, context.getRestDefinitions().size());
+        RestDefinition rest = context.getRestDefinitions().get(0);
+        assertNotNull(rest);
+        assertEquals("/say/hello", rest.getPath());
+        assertEquals(1, rest.getVerbs().size());
+        ToDefinition to = assertIsInstanceOf(ToDefinition.class, rest.getVerbs().get(0).getTo());
+        assertEquals("log:hello", to.getUri());
+
+        // should be 2 routes
+        assertEquals(2, context.getRoutes().size());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                restConfiguration().host("localhost").component("dummy-rest").apiContextPath("/api");
+
+                rest("/say/hello")
+                    .get().to("log:hello");
+
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
new file mode 100644
index 0000000..d8b34db
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -0,0 +1,81 @@
+/**
+ * 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.swagger;
+
+import java.util.Map;
+
+import io.swagger.jaxrs.config.BeanConfig;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.RestApiResponseAdapter;
+import org.apache.camel.support.ServiceSupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RestSwaggerProcessor extends ServiceSupport implements Processor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerProcessor.class);
+    private final BeanConfig swaggerConfig;
+    private final RestSwaggerSupport support;
+
+    public RestSwaggerProcessor(Map<String, Object> parameters) {
+        support = new RestSwaggerSupport();
+        swaggerConfig = new BeanConfig();
+        support.initSwagger(swaggerConfig, parameters);
+    }
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+
+        String contextId;
+        String route = exchange.getIn().getHeader(Exchange.HTTP_PATH, String.class);
+
+        RestApiResponseAdapter adapter = null;
+
+        try {
+
+            // render list of camel contexts as root
+            if (route == null || route.equals("") || route.equals("/")) {
+                support.renderCamelContexts(adapter);
+            } else {
+                // first part is the camel context
+                if (route.startsWith("/")) {
+                    route = route.substring(1);
+                }
+                // the remainder is the route part
+                contextId = route.split("/")[0];
+                if (route.startsWith(contextId)) {
+                    route = route.substring(contextId.length());
+                }
+
+                support.renderResourceListing(adapter, swaggerConfig, contextId, route);
+            }
+        } catch (Exception e) {
+            LOG.warn("Error rendering Swagger API due " + e.getMessage(), e);
+        }
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        // noop
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index 6b7f56d..ebff12f 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -19,6 +19,7 @@ package org.apache.camel.swagger;
 import java.lang.management.ManagementFactory;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -35,36 +36,41 @@ import org.apache.camel.impl.DefaultClassResolver;
 import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.rest.RestDefinition;
 import org.apache.camel.model.rest.RestsDefinition;
-import org.apache.camel.swagger.spi.SwaggerApiProvider;
+import org.apache.camel.spi.RestApiResponseAdapter;
 import org.apache.camel.util.CamelVersionHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * A support class for that allows SPI to plugin
+ * and offer Swagger API service listings as part of the Camel component. This allows rest-dsl components
+ * such as servlet/jetty/netty4-http to offer Swagger API listings with minimal effort.
+ */
 public class RestSwaggerSupport {
 
     private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerSupport.class);
     private RestSwaggerReader reader = new RestSwaggerReader();
     private boolean cors;
 
-    public void initSwagger(BeanConfig swaggerConfig, SwaggerApiProvider config) {
+    public void initSwagger(BeanConfig swaggerConfig, Map<String, Object> config) {
         // configure swagger options
-        String s = config.getInitParameter("swagger.version");
+        String s = (String) config.get("swagger.version");
         if (s != null) {
             swaggerConfig.setVersion(s);
         }
-        s = config.getInitParameter("base.path");
+        s = (String) config.get("base.path");
         if (s != null) {
             swaggerConfig.setBasePath(s);
         }
-        s = config.getInitParameter("host");
+        s = (String) config.get("host");
         if (s != null) {
             swaggerConfig.setHost(s);
         }
-        s = config.getInitParameter("cors");
+        s = (String) config.get("cors");
         if (s != null) {
             cors = "true".equalsIgnoreCase(s);
         }
-        s = config.getInitParameter("schemas");
+        s = (String) config.get("schemas");
         if (s != null) {
             String[] schemas = s.split(",");
             swaggerConfig.setSchemes(schemas);
@@ -73,15 +79,15 @@ public class RestSwaggerSupport {
             swaggerConfig.setSchemes(new String[]{"http"});
         }
 
-        String version = config.getInitParameter("api.version");
-        String title = config.getInitParameter("api.title");
-        String description = config.getInitParameter("api.description");
-        String termsOfService = config.getInitParameter("api.termsOfService");
-        String licenseName = config.getInitParameter("api.license.name");
-        String licenseUrl = config.getInitParameter("api.license.url");
-        String contactName = config.getInitParameter("api.contact.name");
-        String contactUrl = config.getInitParameter("api.contact.url");
-        String contactEmail = config.getInitParameter("api.contact.email");
+        String version = (String) config.get("api.version");
+        String title = (String) config.get("api.title");
+        String description = (String) config.get("api.description");
+        String termsOfService = (String) config.get("api.termsOfService");
+        String licenseName = (String) config.get("api.license.name");
+        String licenseUrl = (String) config.get("api.license.url");
+        String contactName = (String) config.get("api.contact.name");
+        String contactUrl = (String) config.get("api.contact.url");
+        String contactEmail = (String) config.get("api.contact.email");
 
         Info info = new Info();
         info.setVersion(version);
@@ -159,13 +165,13 @@ public class RestSwaggerSupport {
         return answer;
     }
 
-    public void renderResourceListing(SwaggerApiProvider provider, BeanConfig swaggerConfig, String contextId, String route) throws Exception {
+    public void renderResourceListing(RestApiResponseAdapter response, BeanConfig swaggerConfig, String contextId, String route) throws Exception {
         LOG.trace("renderResourceListing");
 
         if (cors) {
-            provider.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
-            provider.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
-            provider.addHeader("Access-Control-Allow-Origin", "*");
+            response.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
+            response.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
+            response.addHeader("Access-Control-Allow-Origin", "*");
         }
 
         List<RestDefinition> rests = getRestDefinitions(contextId);
@@ -176,34 +182,34 @@ public class RestSwaggerSupport {
             ObjectMapper mapper = new ObjectMapper();
             mapper.enable(SerializationFeature.INDENT_OUTPUT);
             mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
-            mapper.writeValue(provider.getOutputStream(), swagger);
+            mapper.writeValue(response.getOutputStream(), swagger);
         } else {
-            provider.noContent();
+            response.noContent();
         }
     }
 
     /**
      * Renders a list of available CamelContexts in the JVM
      */
-    public void renderCamelContexts(SwaggerApiProvider provider) throws Exception {
+    public void renderCamelContexts(RestApiResponseAdapter response) throws Exception {
         LOG.trace("renderCamelContexts");
 
         if (cors) {
-            provider.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
-            provider.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
-            provider.addHeader("Access-Control-Allow-Origin", "*");
+            response.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
+            response.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
+            response.addHeader("Access-Control-Allow-Origin", "*");
         }
 
         List<String> contexts = findCamelContexts();
-        provider.getOutputStream().write("[\n".getBytes());
+        response.getOutputStream().write("[\n".getBytes());
         for (int i = 0; i < contexts.size(); i++) {
             String name = contexts.get(i);
-            provider.getOutputStream().write(("{\"name\": \"" + name + "\"}").getBytes());
+            response.getOutputStream().write(("{\"name\": \"" + name + "\"}").getBytes());
             if (i < contexts.size() - 1) {
-                provider.getOutputStream().write(",\n".getBytes());
+                response.getOutputStream().write(",\n".getBytes());
             }
         }
-        provider.getOutputStream().write("\n]".getBytes());
+        response.getOutputStream().write("\n]".getBytes());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
new file mode 100644
index 0000000..436ae45
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.swagger;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.RestApiProcessorFactory;
+
+public class SwaggerRestApiProcessorFactory implements RestApiProcessorFactory {
+
+    @Override
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception {
+        RestSwaggerProcessor processor = new RestSwaggerProcessor(parameters);
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
index 75d0290..3159776 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
@@ -19,6 +19,9 @@ package org.apache.camel.swagger.servlet;
 import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
 import javax.servlet.ServletConfig;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
@@ -26,8 +29,8 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import io.swagger.jaxrs.config.BeanConfig;
+import org.apache.camel.spi.RestApiResponseAdapter;
 import org.apache.camel.swagger.RestSwaggerSupport;
-import org.apache.camel.swagger.spi.SwaggerApiProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,8 +51,14 @@ public class RestSwaggerServlet extends HttpServlet {
     @Override
     public void init(final ServletConfig config) throws ServletException {
         super.init(config);
-
-        swagger.initSwagger(swaggerConfig, new ServletSwaggerApiProvider(config, null));
+        Map<String, Object> parameters = new HashMap<String, Object>();
+        Enumeration en = config.getInitParameterNames();
+        while (en.hasMoreElements()) {
+            String name = (String) en.nextElement();
+            Object value = config.getInitParameter(name);
+            parameters.put(name, value);
+        }
+        swagger.initSwagger(swaggerConfig, parameters);
     }
 
     @Override
@@ -59,7 +68,7 @@ public class RestSwaggerServlet extends HttpServlet {
             initBaseAndApiPaths(request);
         }
 
-        SwaggerApiProvider resp = new ServletSwaggerApiProvider(null, response);
+        RestApiResponseAdapter adapter = new ServletRestApiResponseAdapter(response);
 
         String contextId;
         String route = request.getPathInfo();
@@ -68,7 +77,7 @@ public class RestSwaggerServlet extends HttpServlet {
 
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
-                swagger.renderCamelContexts(resp);
+                swagger.renderCamelContexts(adapter);
             } else {
                 // first part is the camel context
                 if (route.startsWith("/")) {
@@ -80,7 +89,7 @@ public class RestSwaggerServlet extends HttpServlet {
                     route = route.substring(contextId.length());
                 }
 
-                swagger.renderResourceListing(resp, swaggerConfig, contextId, route);
+                swagger.renderResourceListing(adapter, swaggerConfig, contextId, route);
             }
         } catch (Exception e) {
             LOG.warn("Error rendering swagger due " + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
new file mode 100644
index 0000000..499b07d
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
@@ -0,0 +1,48 @@
+/**
+ * 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.swagger.servlet;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.camel.spi.RestApiResponseAdapter;
+
+public class ServletRestApiResponseAdapter implements RestApiResponseAdapter {
+
+    private final HttpServletResponse response;
+
+    public ServletRestApiResponseAdapter(HttpServletResponse response) {
+        this.response = response;
+    }
+
+    @Override
+    public void addHeader(String name, String value) {
+        response.addHeader(name, value);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        return response.getOutputStream();
+    }
+
+    @Override
+    public void noContent() {
+        response.setStatus(HttpServletResponse.SC_NO_CONTENT);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
deleted file mode 100644
index 3d39825..0000000
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
+++ /dev/null
@@ -1,55 +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.swagger.servlet;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import javax.servlet.ServletConfig;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.camel.swagger.spi.SwaggerApiProvider;
-
-public class ServletSwaggerApiProvider implements SwaggerApiProvider {
-
-    private final ServletConfig config;
-    private final HttpServletResponse response;
-
-    public ServletSwaggerApiProvider(ServletConfig config, HttpServletResponse response) {
-        this.config = config;
-        this.response = response;
-    }
-
-    @Override
-    public String getInitParameter(String key) {
-        return config.getInitParameter(key);
-    }
-
-    @Override
-    public void addHeader(String name, String value) {
-        response.addHeader(name, value);
-    }
-
-    @Override
-    public OutputStream getOutputStream() throws IOException {
-        return response.getOutputStream();
-    }
-
-    @Override
-    public void noContent() {
-        response.setStatus(HttpServletResponse.SC_NO_CONTENT);
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/b88093cb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
deleted file mode 100644
index 42e6a2d..0000000
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
+++ /dev/null
@@ -1,32 +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.swagger.spi;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-public interface SwaggerApiProvider {
-
-    String getInitParameter(String key);
-
-    void addHeader(String name, String value);
-
-    OutputStream getOutputStream() throws IOException;
-
-    void noContent();
-
-}


[02/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 646c9bcb2067070b9b88116dd3139af8ce3eaf00
Parents: b88093c
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 16:48:17 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:03 2015 +0200

----------------------------------------------------------------------
 .../camel/spi/RestApiResponseAdapter.java       |  3 +-
 .../spi/RestApiResponseAdapterFactory.java      | 34 +++++++++++
 .../netty4/http/NettyHttpComponent.java         | 14 ++++-
 .../http/NettyRestApiResponseAdapter.java       | 48 +++++++++++++++
 .../camel/swagger/RestSwaggerProcessor.java     | 62 +++++++++++++++++++-
 .../camel/swagger/RestSwaggerSupport.java       | 11 ++--
 .../swagger/SwaggerRestApiProcessorFactory.java |  3 +-
 .../servlet/ServletRestApiResponseAdapter.java  |  5 +-
 8 files changed, 165 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
index 3575d8d..01b1e4f 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapter.java
@@ -17,7 +17,6 @@
 package org.apache.camel.spi;
 
 import java.io.IOException;
-import java.io.OutputStream;
 
 /**
  * An adapter to allow Camel rest-api to use Camel components to render the api response.
@@ -26,7 +25,7 @@ public interface RestApiResponseAdapter {
 
     void addHeader(String name, String value);
 
-    OutputStream getOutputStream() throws IOException;
+    void writeBytes(byte[] bytes) throws IOException;
 
     void noContent();
 

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
new file mode 100644
index 0000000..71bffd8
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiResponseAdapterFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.spi;
+
+import org.apache.camel.Exchange;
+
+/**
+ * Factory to create {@link RestApiResponseAdapter} which allows Camel components
+ * to provide a response adapter to be used by the rest-dsl api support.
+ */
+public interface RestApiResponseAdapterFactory {
+
+    /**
+     * Creates a new {@link RestApiResponseAdapter}
+     *
+     * @param exchange  the exchange
+     * @return the adapter
+     */
+    RestApiResponseAdapter newAdapter(Exchange exchange);
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
index 0a4502c..d036cff 100644
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Consumer;
 import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.component.netty4.NettyComponent;
 import org.apache.camel.component.netty4.NettyConfiguration;
@@ -31,6 +32,8 @@ import org.apache.camel.component.netty4.NettyServerBootstrapConfiguration;
 import org.apache.camel.component.netty4.http.handlers.HttpServerMultiplexChannelHandler;
 import org.apache.camel.spi.HeaderFilterStrategy;
 import org.apache.camel.spi.HeaderFilterStrategyAware;
+import org.apache.camel.spi.RestApiResponseAdapter;
+import org.apache.camel.spi.RestApiResponseAdapterFactory;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
@@ -46,7 +49,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Netty HTTP based component.
  */
-public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory {
+public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory, RestApiResponseAdapterFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(NettyHttpComponent.class);
 
@@ -297,6 +300,15 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
     }
 
     @Override
+    public RestApiResponseAdapter newAdapter(Exchange exchange) {
+        NettyHttpMessage http = exchange.getIn(NettyHttpMessage.class);
+        if (http != null) {
+            return new NettyRestApiResponseAdapter(http.getHttpResponse());
+        }
+        return null;
+    }
+
+    @Override
     protected void doStop() throws Exception {
         super.doStop();
 

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
new file mode 100644
index 0000000..263f1e3
--- /dev/null
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyRestApiResponseAdapter.java
@@ -0,0 +1,48 @@
+/**
+ * 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.netty4.http;
+
+import java.io.IOException;
+
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.camel.spi.RestApiResponseAdapter;
+
+public class NettyRestApiResponseAdapter implements RestApiResponseAdapter {
+
+    private final FullHttpResponse httpResponse;
+
+    public NettyRestApiResponseAdapter(FullHttpResponse httpResponse) {
+        this.httpResponse = httpResponse;
+    }
+
+    @Override
+    public void addHeader(String name, String value) {
+        httpResponse.headers().set(name, value);
+    }
+
+    @Override
+    public void writeBytes(byte[] bytes) throws IOException {
+        httpResponse.content().writeBytes(bytes);
+    }
+
+    @Override
+    public void noContent() {
+        httpResponse.setStatus(HttpResponseStatus.NO_CONTENT);
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
index d8b34db..4b3b9cd 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -17,11 +17,16 @@
 package org.apache.camel.swagger;
 
 import java.util.Map;
+import java.util.Set;
 
 import io.swagger.jaxrs.config.BeanConfig;
+import org.apache.camel.CamelContext;
+import org.apache.camel.Component;
 import org.apache.camel.Exchange;
+import org.apache.camel.NoSuchBeanException;
 import org.apache.camel.Processor;
 import org.apache.camel.spi.RestApiResponseAdapter;
+import org.apache.camel.spi.RestApiResponseAdapterFactory;
 import org.apache.camel.support.ServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,11 +36,13 @@ public class RestSwaggerProcessor extends ServiceSupport implements Processor {
     private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerProcessor.class);
     private final BeanConfig swaggerConfig;
     private final RestSwaggerSupport support;
+    private final String componentName;
 
     public RestSwaggerProcessor(Map<String, Object> parameters) {
         support = new RestSwaggerSupport();
         swaggerConfig = new BeanConfig();
         support.initSwagger(swaggerConfig, parameters);
+        componentName = (String) parameters.get("componentName");
     }
 
     @Override
@@ -44,9 +51,8 @@ public class RestSwaggerProcessor extends ServiceSupport implements Processor {
         String contextId;
         String route = exchange.getIn().getHeader(Exchange.HTTP_PATH, String.class);
 
-        RestApiResponseAdapter adapter = null;
-
         try {
+            RestApiResponseAdapter adapter = lookupAdapter(exchange);
 
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
@@ -69,6 +75,58 @@ public class RestSwaggerProcessor extends ServiceSupport implements Processor {
         }
     }
 
+    protected RestApiResponseAdapter lookupAdapter(Exchange exchange) {
+        CamelContext camelContext = exchange.getContext();
+
+        RestApiResponseAdapterFactory factory = null;
+
+        if (componentName != null) {
+            Object comp = camelContext.getRegistry().lookupByName(componentName);
+            if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
+                factory = (RestApiResponseAdapterFactory) comp;
+            } else {
+                comp = camelContext.getComponent(componentName);
+                if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
+                    factory = (RestApiResponseAdapterFactory) comp;
+                }
+            }
+
+            if (factory == null) {
+                if (comp != null) {
+                    throw new IllegalArgumentException("Component " + componentName + " is not a RestApiResponseAdapterFactory");
+                } else {
+                    throw new NoSuchBeanException(componentName, RestApiResponseAdapterFactory.class.getName());
+                }
+            }
+        }
+
+        // try all components
+        if (factory == null) {
+            for (String name : camelContext.getComponentNames()) {
+                Component comp = camelContext.getComponent(name);
+                if (comp != null && comp instanceof RestApiResponseAdapterFactory) {
+                    factory = (RestApiResponseAdapterFactory) comp;
+                    break;
+                }
+            }
+        }
+
+        // lookup in registry
+        if (factory == null) {
+            Set<RestApiResponseAdapterFactory> factories = camelContext.getRegistry().findByType(RestApiResponseAdapterFactory.class);
+            if (factories != null && factories.size() == 1) {
+                factory = factories.iterator().next();
+            }
+        }
+
+        if (factory != null) {
+            return factory.newAdapter(exchange);
+        } else {
+            throw new IllegalStateException("Cannot find RestApiResponseAdapterFactory in Registry or as a Component to use");
+        }
+
+    }
+
     @Override
     protected void doStart() throws Exception {
         // noop

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index ebff12f..e558ee3 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -182,7 +182,8 @@ public class RestSwaggerSupport {
             ObjectMapper mapper = new ObjectMapper();
             mapper.enable(SerializationFeature.INDENT_OUTPUT);
             mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
-            mapper.writeValue(response.getOutputStream(), swagger);
+            byte[] bytes = mapper.writeValueAsBytes(swagger);
+            response.writeBytes(bytes);
         } else {
             response.noContent();
         }
@@ -201,15 +202,15 @@ public class RestSwaggerSupport {
         }
 
         List<String> contexts = findCamelContexts();
-        response.getOutputStream().write("[\n".getBytes());
+        response.writeBytes("[\n".getBytes());
         for (int i = 0; i < contexts.size(); i++) {
             String name = contexts.get(i);
-            response.getOutputStream().write(("{\"name\": \"" + name + "\"}").getBytes());
+            response.writeBytes(("{\"name\": \"" + name + "\"}").getBytes());
             if (i < contexts.size() - 1) {
-                response.getOutputStream().write(",\n".getBytes());
+                response.writeBytes(",\n".getBytes());
             }
         }
-        response.getOutputStream().write("\n]".getBytes());
+        response.writeBytes("\n]".getBytes());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
index 436ae45..19cd1a7 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
@@ -26,7 +26,6 @@ public class SwaggerRestApiProcessorFactory implements RestApiProcessorFactory {
 
     @Override
     public Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception {
-        RestSwaggerProcessor processor = new RestSwaggerProcessor(parameters);
-        return null;
+        return new RestSwaggerProcessor(parameters);
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/646c9bcb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
index 499b07d..bae3aec 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletRestApiResponseAdapter.java
@@ -17,7 +17,6 @@
 package org.apache.camel.swagger.servlet;
 
 import java.io.IOException;
-import java.io.OutputStream;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.camel.spi.RestApiResponseAdapter;
@@ -36,8 +35,8 @@ public class ServletRestApiResponseAdapter implements RestApiResponseAdapter {
     }
 
     @Override
-    public OutputStream getOutputStream() throws IOException {
-        return response.getOutputStream();
+    public void writeBytes(byte[] bytes) throws IOException {
+        response.getOutputStream().write(bytes);
     }
 
     @Override


[11/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 44a15c540f78858eb72e0679651dd92de727cf48
Parents: 9bd9e3e
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 09:19:37 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 09:19:37 2015 +0200

----------------------------------------------------------------------
 .../netty/http/NettyHttpComponent.java          | 22 +++++++++++++++--
 .../component/servlet/ServletComponent.java     | 23 ++++++++++++++++--
 .../component/undertow/UndertowComponent.java   | 25 +++++++++++++++++---
 3 files changed, 63 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/44a15c54/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
index 0e831f5..0ce17c6 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
@@ -31,6 +31,7 @@ import org.apache.camel.component.netty.NettyServerBootstrapConfiguration;
 import org.apache.camel.component.netty.http.handlers.HttpServerMultiplexChannelHandler;
 import org.apache.camel.spi.HeaderFilterStrategy;
 import org.apache.camel.spi.HeaderFilterStrategyAware;
+import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
@@ -46,7 +47,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Netty HTTP based component.
  */
-public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory {
+public class NettyHttpComponent extends NettyComponent implements HeaderFilterStrategyAware, RestConsumerFactory, RestApiConsumerFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(NettyHttpComponent.class);
 
@@ -226,6 +227,18 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
                                    String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, configuration, parameters, false);
+    }
+
+    @Override
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                                      RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        // reuse the createConsumer method we already have. The api need to use GET and match on uri prefix
+        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, configuration, parameters, true);
+    }
+
+    Consumer doCreateConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
+                              String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters, boolean api) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -278,7 +291,12 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
         String query = URISupport.createQueryString(map);
 
-        String url = "netty-http:%s://%s:%s/%s?httpMethodRestrict=%s";
+        String url;
+        if (api) {
+            url = "netty-http:%s://%s:%s/%s?matchOnUriPrefix=true&httpMethodRestrict=%s";
+        } else {
+            url = "netty-http:%s://%s:%s/%s?httpMethodRestrict=%s";
+        }
         
         // must use upper case for restrict
         String restrict = verb.toUpperCase(Locale.US);

http://git-wip-us.apache.org/repos/asf/camel/blob/44a15c54/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java b/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
index c70f81d..9aa1300 100644
--- a/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
+++ b/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
@@ -29,13 +29,14 @@ import org.apache.camel.http.common.HttpBinding;
 import org.apache.camel.http.common.HttpCommonComponent;
 import org.apache.camel.http.common.HttpConsumer;
 import org.apache.camel.spi.HeaderFilterStrategy;
+import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
 import org.apache.camel.util.URISupport;
 import org.apache.camel.util.UnsafeUriCharactersEncoder;
 
-public class ServletComponent extends HttpCommonComponent implements RestConsumerFactory {
+public class ServletComponent extends HttpCommonComponent implements RestConsumerFactory, RestApiConsumerFactory {
 
     private String servletName = "CamelServlet";
     private HttpRegistry httpRegistry;
@@ -160,6 +161,19 @@ public class ServletComponent extends HttpCommonComponent implements RestConsume
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
                                    String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, configuration, parameters, false);
+    }
+
+    @Override
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                                      RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        // reuse the createConsumer method we already have. The api need to use GET and match on uri prefix
+        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, configuration, parameters, true);
+    }
+
+    Consumer doCreateConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
+                              String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters, boolean api) throws Exception {
+
         String path = basePath;
         if (uriTemplate != null) {
             // make sure to avoid double slashes
@@ -185,7 +199,12 @@ public class ServletComponent extends HttpCommonComponent implements RestConsume
 
         String query = URISupport.createQueryString(map);
 
-        String url = "servlet:///%s?httpMethodRestrict=%s";
+        String url;
+        if (api) {
+            url = "servlet:///%s?matchOnUriPrefix=true&httpMethodRestrict=%s";
+        } else {
+            url = "servlet:///%s?httpMethodRestrict=%s";
+        }
         // must use upper case for restrict
         String restrict = verb.toUpperCase(Locale.US);
 

http://git-wip-us.apache.org/repos/asf/camel/blob/44a15c54/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
index 8274b07..3584819 100644
--- a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
+++ b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import io.undertow.Handlers;
 import io.undertow.Undertow;
 import io.undertow.server.handlers.PathHandler;
-
 import org.apache.camel.CamelContext;
 import org.apache.camel.Consumer;
 import org.apache.camel.Endpoint;
@@ -32,6 +31,7 @@ import org.apache.camel.Processor;
 import org.apache.camel.component.undertow.handlers.HttpCamelHandler;
 import org.apache.camel.component.undertow.handlers.NotFoundHandler;
 import org.apache.camel.impl.UriEndpointComponent;
+import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents the component that manages {@link UndertowEndpoint}.
  */
-public class UndertowComponent extends UriEndpointComponent implements RestConsumerFactory {
+public class UndertowComponent extends UriEndpointComponent implements RestConsumerFactory, RestApiConsumerFactory {
     private static final Logger LOG = LoggerFactory.getLogger(UndertowEndpoint.class);
 
     private UndertowHttpBinding undertowHttpBinding = new DefaultUndertowHttpBinding();
@@ -85,6 +85,18 @@ public class UndertowComponent extends UriEndpointComponent implements RestConsu
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
                                    String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, configuration, parameters, false);
+    }
+
+    @Override
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                                      RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        // reuse the createConsumer method we already have. The api need to use GET and match on uri prefix
+        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, configuration, parameters, true);
+    }
+
+    Consumer doCreateConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
+                              String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters, boolean api) throws Exception {
         String path = basePath;
         if (uriTemplate != null) {
             // make sure to avoid double slashes
@@ -125,8 +137,15 @@ public class UndertowComponent extends UriEndpointComponent implements RestConsu
 
         String query = URISupport.createQueryString(map);
 
-        String url = "undertow:%s://%s:%s/%s";
+        String url;
+        if (api) {
+            url = "undertow:%s://%s:%s/%s?matchOnUriPrefix=true";
+        } else {
+            url = "undertow:%s://%s:%s/%s";
+        }
+
         url = String.format(url, scheme, host, port, path);
+
         if (!query.isEmpty()) {
             url = url + "&" + query;
         }


[10/16] camel git commit: CAMEL-8545: Use provided rest configuration and only fallback to default if none provided.

Posted by da...@apache.org.
CAMEL-8545: Use provided rest configuration and only fallback to default if none provided.


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

Branch: refs/heads/master
Commit: bdbeb4a39a39984ca867bf8f7f78ec1fd49d4f6b
Parents: 8252a16
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 20:44:26 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:05 2015 +0200

----------------------------------------------------------------------
 .../java/org/apache/camel/CamelContext.java     |  4 +--
 .../camel/component/rest/RestApiEndpoint.java   |  8 ++---
 .../camel/component/rest/RestEndpoint.java      |  7 ++--
 .../apache/camel/impl/DefaultCamelContext.java  |  2 +-
 .../model/rest/RestConfigurationDefinition.java | 35 ++++++++++++++++++++
 .../camel/spi/RestApiConsumerFactory.java       |  4 +--
 .../camel/spi/RestApiProcessorFactory.java      |  3 +-
 .../org/apache/camel/spi/RestConfiguration.java | 14 ++++++++
 .../apache/camel/spi/RestConsumerFactory.java   |  2 +-
 .../rest/DummyRestConsumerFactory.java          |  8 +++--
 .../rest/DummyRestProcessorFactory.java         |  6 ++--
 .../org/apache/camel/coap/CoAPComponent.java    | 14 +++++---
 .../component/jetty/JettyHttpComponent.java     | 11 +++---
 .../netty/http/NettyHttpComponent.java          | 12 ++++---
 .../netty4/http/NettyHttpComponent.java         | 20 ++++++-----
 .../netty4/http/rest/RestApiNettyTest.java      |  5 ++-
 .../component/restlet/RestletComponent.java     | 11 +++---
 .../component/servlet/ServletComponent.java     |  9 +++--
 .../component/sparkrest/SparkComponent.java     | 10 +++---
 .../swagger/SwaggerRestApiProcessorFactory.java |  6 ++--
 .../camel/swagger/DummyRestConsumerFactory.java |  3 +-
 .../swagger/DummyRestConsumerFactory.java       |  3 +-
 .../rest/DummyRestConsumerFactory.java          | 10 +++---
 .../component/undertow/UndertowComponent.java   | 10 ++++--
 24 files changed, 152 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/CamelContext.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/CamelContext.java b/camel-core/src/main/java/org/apache/camel/CamelContext.java
index c820dff..9a1509a 100644
--- a/camel-core/src/main/java/org/apache/camel/CamelContext.java
+++ b/camel-core/src/main/java/org/apache/camel/CamelContext.java
@@ -500,6 +500,7 @@ public interface CamelContext extends SuspendableService, RuntimeConfiguration {
 
     /**
      * Gets the REST configuration for the given component
+     *
      * @param component the component name to get the configuration
      * @param defaultIfNotFound determine if the default configuration is returned if there isn't a 
      *        specific configuration for the given component  
@@ -508,8 +509,7 @@ public interface CamelContext extends SuspendableService, RuntimeConfiguration {
     RestConfiguration getRestConfiguration(String component, boolean defaultIfNotFound);
     
     /**
-     * Gets all the RestConfigurations 
-     * @return
+     * Gets all the RestConfiguration's
      */
     Collection<RestConfiguration> getRestConfigurations();
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
index a4ff9f5..56fe965 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
@@ -100,14 +100,15 @@ public class RestApiEndpoint extends DefaultEndpoint {
 
         if (factory != null) {
 
+            RestConfiguration config = getCamelContext().getRestConfiguration(componentName, true);
+
             // calculate the url to the rest API service
             String path = getPath();
             if (path != null && !path.startsWith("/")) {
                 path = "/" + path;
             }
 
-
-            Processor processor = factory.createApiProcessor(getCamelContext(), path, getParameters());
+            Processor processor = factory.createApiProcessor(getCamelContext(), path, config, getParameters());
             return new RestApiProducer(this, processor);
         } else {
             throw new IllegalStateException("Cannot find RestApiProcessorFactory in Registry");
@@ -182,8 +183,7 @@ public class RestApiEndpoint extends DefaultEndpoint {
                 }
             }
 */
-
-            Consumer consumer = factory.createApiConsumer(getCamelContext(), processor, path, getParameters());
+            Consumer consumer = factory.createApiConsumer(getCamelContext(), processor, path, config, getParameters());
             configureConsumer(consumer);
 
             return consumer;

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
index adb5f49..cf22ef0 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
@@ -245,9 +245,6 @@ public class RestEndpoint extends DefaultEndpoint {
         }
 
         if (factory != null) {
-            Consumer consumer = factory.createConsumer(getCamelContext(), processor, getMethod(), getPath(), getUriTemplate(), getConsumes(), getProduces(), getParameters());
-            configureConsumer(consumer);
-
             // if no explicit port/host configured, then use port from rest configuration
             String scheme = "http";
             String host = "";
@@ -305,6 +302,10 @@ public class RestEndpoint extends DefaultEndpoint {
                 }
             }
 
+            Consumer consumer = factory.createConsumer(getCamelContext(), processor, getMethod(), getPath(),
+                    getUriTemplate(), getConsumes(), getProduces(), config, getParameters());
+            configureConsumer(consumer);
+
             // add to rest registry so we can keep track of them, we will remove from the registry when the consumer is removed
             // the rest registry will automatic keep track when the consumer is removed,
             // and un-register the REST service from the registry

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/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 3cfa60e..1c5227f 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
@@ -2308,11 +2308,11 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
     public void setRestConfiguration(RestConfiguration restConfiguration) {
         restConfigurations.put("", restConfiguration);
     }
+
     public Collection<RestConfiguration> getRestConfigurations() {
         return restConfigurations.values();
     }
 
-
     public void addRestConfiguration(RestConfiguration restConfiguration) {
         restConfigurations.put(restConfiguration.getComponent(), restConfiguration);        
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
index 1ed3239..2c5906d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
@@ -87,6 +87,9 @@ public class RestConfigurationDefinition {
     @XmlElement(name = "dataFormatProperty")
     private List<RestPropertyDefinition> dataFormatProperties = new ArrayList<RestPropertyDefinition>();
 
+    @XmlElement(name = "apiProperty")
+    private List<RestPropertyDefinition> apiProperties = new ArrayList<RestPropertyDefinition>();
+
     @XmlElement(name = "corsHeaders")
     private List<RestPropertyDefinition> corsHeaders = new ArrayList<RestPropertyDefinition>();
 
@@ -303,6 +306,18 @@ public class RestConfigurationDefinition {
         this.dataFormatProperties = dataFormatProperties;
     }
 
+    public List<RestPropertyDefinition> getApiProperties() {
+        return apiProperties;
+    }
+
+    /**
+     * Allows to configure as many additional properties for the api documentation (swagger).
+     * For example set property api.title to my cool stuff
+     */
+    public void setApiProperties(List<RestPropertyDefinition> apiProperties) {
+        this.apiProperties = apiProperties;
+    }
+
     public List<RestPropertyDefinition> getCorsHeaders() {
         return corsHeaders;
     }
@@ -480,6 +495,17 @@ public class RestConfigurationDefinition {
     }
 
     /**
+     * For additional configuration options on data format level
+     */
+    public RestConfigurationDefinition apiProperty(String key, String value) {
+        RestPropertyDefinition prop = new RestPropertyDefinition();
+        prop.setKey(key);
+        prop.setValue(value);
+        getApiProperties().add(prop);
+        return this;
+    }
+
+    /**
      * For configuring CORS headers
      */
     public RestConfigurationDefinition corsHeaderProperty(String key, String value) {
@@ -574,6 +600,15 @@ public class RestConfigurationDefinition {
             }
             answer.setDataFormatProperties(props);
         }
+        if (!apiProperties.isEmpty()) {
+            Map<String, Object> props = new HashMap<String, Object>();
+            for (RestPropertyDefinition prop : apiProperties) {
+                String key = prop.getKey();
+                String value = CamelContextHelper.parseText(context, prop.getValue());
+                props.put(key, value);
+            }
+            answer.setApiProperties(props);
+        }
         if (!corsHeaders.isEmpty()) {
             Map<String, String> props = new HashMap<String, String>();
             for (RestPropertyDefinition prop : corsHeaders) {

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
index acad9a8..c000d93 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
@@ -37,7 +37,7 @@ public interface RestApiConsumerFactory {
      * @return a newly created REST API consumer
      * @throws Exception can be thrown
      */
-    Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception;
-
+    Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                               RestConfiguration configuration, Map<String, Object> parameters) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
index 6650213..a721ec5 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
@@ -35,6 +35,7 @@ public interface RestApiProcessorFactory {
      * @return a newly created REST API provider
      * @throws Exception can be thrown
      */
-    Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception;
+    Processor createApiProcessor(CamelContext camelContext, String contextPath,
+                                 RestConfiguration configuration, Map<String, Object> parameters) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
index 7e5e3fe..768ad42 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
@@ -53,6 +53,7 @@ public class RestConfiguration {
     private Map<String, Object> endpointProperties;
     private Map<String, Object> consumerProperties;
     private Map<String, Object> dataFormatProperties;
+    private Map<String, Object> apiProperties;
     private Map<String, String> corsHeaders;
 
     /**
@@ -378,6 +379,19 @@ public class RestConfiguration {
         this.dataFormatProperties = dataFormatProperties;
     }
 
+    public Map<String, Object> getApiProperties() {
+        return apiProperties;
+    }
+
+    /**
+     * Sets additional options on api level
+     *
+     * @param apiProperties the options
+     */
+    public void setApiProperties(Map<String, Object> apiProperties) {
+        this.apiProperties = apiProperties;
+    }
+
     /**
      * Gets the CORS headers to use if CORS has been enabled.
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/main/java/org/apache/camel/spi/RestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestConsumerFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestConsumerFactory.java
index ac07fb7..26f51a3 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestConsumerFactory.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestConsumerFactory.java
@@ -41,5 +41,5 @@ public interface RestConsumerFactory {
      * @throws Exception can be thrown
      */
     Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                            String consumes, String produces, Map<String, Object> parameters) throws Exception;
+                            String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
index d51d3d3..46572c8 100644
--- a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestConsumerFactory.java
@@ -24,13 +24,14 @@ import org.apache.camel.Processor;
 import org.apache.camel.component.seda.SedaEndpoint;
 import org.apache.camel.impl.ActiveMQUuidGenerator;
 import org.apache.camel.spi.RestApiConsumerFactory;
+import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 
 public class DummyRestConsumerFactory implements RestConsumerFactory, RestApiConsumerFactory {
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // just use a seda endpoint for testing purpose
         String id;
         if (uriTemplate != null) {
@@ -47,7 +48,8 @@ public class DummyRestConsumerFactory implements RestConsumerFactory, RestApiCon
     }
 
     @Override
-    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                                      RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // just use a seda endpoint for testing purpose
         String id = ActiveMQUuidGenerator.generateSanitizedId(contextPath);
         // remove leading dash as we add that ourselves
@@ -57,4 +59,6 @@ public class DummyRestConsumerFactory implements RestConsumerFactory, RestApiCon
         SedaEndpoint seda = camelContext.getEndpoint("seda:api:" + "-" + id, SedaEndpoint.class);
         return seda.createConsumer(processor);
     }
+
+
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
index ecb8c6c..1fbe87f 100644
--- a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
@@ -22,17 +22,17 @@ import org.apache.camel.CamelContext;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.spi.RestApiProcessorFactory;
+import org.apache.camel.spi.RestConfiguration;
 
 public class DummyRestProcessorFactory implements RestApiProcessorFactory {
 
     @Override
-    public Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception {
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         return new Processor() {
             @Override
             public void process(Exchange exchange) throws Exception {
-                // noop;
+                // noop
             }
         };
     }
-
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-coap/src/main/java/org/apache/camel/coap/CoAPComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-coap/src/main/java/org/apache/camel/coap/CoAPComponent.java b/components/camel-coap/src/main/java/org/apache/camel/coap/CoAPComponent.java
index 9ad4d42..32397f8 100644
--- a/components/camel-coap/src/main/java/org/apache/camel/coap/CoAPComponent.java
+++ b/components/camel-coap/src/main/java/org/apache/camel/coap/CoAPComponent.java
@@ -81,8 +81,14 @@ public class CoAPComponent extends UriEndpointComponent implements RestConsumerF
                                    String uriTemplate,
                                    String consumes, 
                                    String produces,
+                                   RestConfiguration configuration,
                                    Map<String, Object> parameters) throws Exception {
-        RestConfiguration config = getCamelContext().getRestConfiguration("coap", true);
+
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("coap", true);
+        }
+
         Map<String, Object> map = new HashMap<String, Object>();
         // setup endpoint options
         if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
@@ -90,8 +96,7 @@ public class CoAPComponent extends UriEndpointComponent implements RestConsumerF
         }
 
         String query = URISupport.createQueryString(map);
-        
-        
+
         String url = (config.getScheme() == null ? "coap" : config.getScheme())
             + "://" + config.getHost();
         if (config.getPort() != -1) {
@@ -110,8 +115,7 @@ public class CoAPComponent extends UriEndpointComponent implements RestConsumerF
         setProperties(endpoint, parameters);
         return endpoint.createConsumer(processor);
     }
-    
-    
+
     @Override
     protected void doStart() throws Exception {
         super.doStart();

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/JettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/JettyHttpComponent.java b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/JettyHttpComponent.java
index 0a6c454..03b5d17 100644
--- a/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/JettyHttpComponent.java
+++ b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/JettyHttpComponent.java
@@ -1003,7 +1003,7 @@ public abstract class JettyHttpComponent extends HttpCommonComponent implements
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -1021,7 +1021,10 @@ public abstract class JettyHttpComponent extends HttpCommonComponent implements
         int port = 0;
 
         // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("jetty", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("jetty", true);
+        }
         if (config.getScheme() != null) {
             scheme = config.getScheme();
         }
@@ -1044,7 +1047,7 @@ public abstract class JettyHttpComponent extends HttpCommonComponent implements
 
         Map<String, Object> map = new HashMap<String, Object>();
         // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("jetty"))) {
+        if (config.getComponent() == null || config.getComponent().equals("jetty")) {
             // setup endpoint options
             if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
                 map.putAll(config.getEndpointProperties());
@@ -1073,7 +1076,7 @@ public abstract class JettyHttpComponent extends HttpCommonComponent implements
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
index 23b1460..0e831f5 100644
--- a/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
+++ b/components/camel-netty-http/src/main/java/org/apache/camel/component/netty/http/NettyHttpComponent.java
@@ -225,7 +225,7 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -243,7 +243,10 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
         int port = 0;
 
         // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("netty-http", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("netty-http", true);
+        }
         if (config.getScheme() != null) {
             scheme = config.getScheme();
         }
@@ -266,7 +269,7 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
         Map<String, Object> map = new HashMap<String, Object>();
         // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("netty-http"))) {
+        if (config.getComponent() == null || config.getComponent().equals("netty-http")) {
             // setup endpoint options
             if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
                 map.putAll(config.getEndpointProperties());
@@ -286,13 +289,12 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
             url = url + "&" + query;
         }
 
-        
         NettyHttpEndpoint endpoint = camelContext.getEndpoint(url, NettyHttpEndpoint.class);
         setProperties(endpoint, parameters);
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
index 71fe02e..ae8e78c 100644
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
@@ -225,18 +225,19 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
-        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, parameters, false);
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, configuration, parameters, false);
     }
 
     @Override
-    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath,
+                                      RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // reuse the createConsumer method we already have. The api need to use GET and match on uri prefix
-        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, parameters, true);
+        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, configuration, parameters, true);
     }
 
     Consumer doCreateConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                              String consumes, String produces, Map<String, Object> parameters, boolean api) throws Exception {
+                              String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters, boolean api) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -254,7 +255,10 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
         int port = 0;
 
         // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("netty4-http", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("netty4-http", true);
+        }
         if (config.getScheme() != null) {
             scheme = config.getScheme();
         }
@@ -277,7 +281,7 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
         Map<String, Object> map = new HashMap<String, Object>();
         // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("netty4-http"))) {
+        if (config.getComponent() == null || config.getComponent().equals("netty4-http")) {
             // setup endpoint options
             if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
                 map.putAll(config.getEndpointProperties());
@@ -306,7 +310,7 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
index 2abf8f1..99d1665 100644
--- a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
+++ b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
@@ -50,6 +50,8 @@ public class RestApiNettyTest extends BaseNettyTest {
         assertNotNull(out);
         log.info(out);
 
+        assertTrue(out.contains("\"version\" : \"1.2.3\""));
+        assertTrue(out.contains("\"title\" : \"The hello rest thing\""));
         assertTrue(out.contains("\"/hello/bye/{name}\""));
         assertTrue(out.contains("\"/hello/hi/{name}\""));
         assertTrue(out.contains("\"summary\" : \"To update the greeting message\""));
@@ -60,7 +62,8 @@ public class RestApiNettyTest extends BaseNettyTest {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                restConfiguration().component("netty4-http").host("localhost").port(getPort()).apiContextPath("/api-doc");
+                restConfiguration().component("netty4-http").host("localhost").port(getPort()).apiContextPath("/api-doc")
+                    .apiProperty("cors", "true").apiProperty("api.title", "The hello rest thing").apiProperty("api.version", "1.2.3");
 
                 rest("/hello").consumes("application/json").produces("application/json")
                     .get("/hi/{name}").description("Saying hi")

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletComponent.java b/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletComponent.java
index 345d89d..9b91d90 100644
--- a/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletComponent.java
+++ b/components/camel-restlet/src/main/java/org/apache/camel/component/restlet/RestletComponent.java
@@ -664,7 +664,7 @@ public class RestletComponent extends HeaderFilterStrategyComponent implements R
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -683,7 +683,10 @@ public class RestletComponent extends HeaderFilterStrategyComponent implements R
         int port = this.getPort();
 
         // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("restlet", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("restlet", true);
+        }
         if (config.getScheme() != null) {
             scheme = config.getScheme();
         }
@@ -706,7 +709,7 @@ public class RestletComponent extends HeaderFilterStrategyComponent implements R
 
         Map<String, Object> map = new HashMap<String, Object>();
         // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("restlet"))) {
+        if (config.getComponent() == null || config.getComponent().equals("restlet")) {
             // setup endpoint options
             if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
                 map.putAll(config.getEndpointProperties());
@@ -736,7 +739,7 @@ public class RestletComponent extends HeaderFilterStrategyComponent implements R
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java b/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
index 1d573a8..c70f81d 100644
--- a/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
+++ b/components/camel-servlet/src/main/java/org/apache/camel/component/servlet/ServletComponent.java
@@ -159,7 +159,7 @@ public class ServletComponent extends HttpCommonComponent implements RestConsume
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         String path = basePath;
         if (uriTemplate != null) {
             // make sure to avoid double slashes
@@ -172,7 +172,10 @@ public class ServletComponent extends HttpCommonComponent implements RestConsume
         path = FileUtil.stripLeadingSeparator(path);
 
         // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("servlet", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("servlet", true);
+        }
 
         Map<String, Object> map = new HashMap<String, Object>();
         // setup endpoint options
@@ -204,7 +207,7 @@ public class ServletComponent extends HttpCommonComponent implements RestConsume
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-spark-rest/src/main/java/org/apache/camel/component/sparkrest/SparkComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-spark-rest/src/main/java/org/apache/camel/component/sparkrest/SparkComponent.java b/components/camel-spark-rest/src/main/java/org/apache/camel/component/sparkrest/SparkComponent.java
index eb5cba8..826db58 100644
--- a/components/camel-spark-rest/src/main/java/org/apache/camel/component/sparkrest/SparkComponent.java
+++ b/components/camel-spark-rest/src/main/java/org/apache/camel/component/sparkrest/SparkComponent.java
@@ -31,7 +31,6 @@ import org.apache.camel.spi.RestConsumerFactory;
 import org.apache.camel.util.FileUtil;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.URISupport;
-import spark.Spark;
 import spark.SparkBase;
 
 public class SparkComponent extends UriEndpointComponent implements RestConsumerFactory {
@@ -147,7 +146,7 @@ public class SparkComponent extends UriEndpointComponent implements RestConsumer
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -174,7 +173,10 @@ public class SparkComponent extends UriEndpointComponent implements RestConsumer
         }
 
         // build query string, and append any endpoint configuration properties
-        RestConfiguration config = getCamelContext().getRestConfiguration("spark-rest", true);
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("spark-rest", true);
+        }
         // setup endpoint options
         if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
             map.putAll(config.getEndpointProperties());
@@ -193,7 +195,7 @@ public class SparkComponent extends UriEndpointComponent implements RestConsumer
 
         // configure consumer properties
         Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
+        if (config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
             setProperties(consumer, config.getConsumerProperties());
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
index 19cd1a7..faba694 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
@@ -21,11 +21,13 @@ import java.util.Map;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Processor;
 import org.apache.camel.spi.RestApiProcessorFactory;
+import org.apache.camel.spi.RestConfiguration;
 
 public class SwaggerRestApiProcessorFactory implements RestApiProcessorFactory {
 
     @Override
-    public Processor createApiProcessor(CamelContext camelContext, String contextPath, Map<String, Object> parameters) throws Exception {
-        return new RestSwaggerProcessor(parameters);
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath,
+                                        RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+        return new RestSwaggerProcessor(configuration.getApiProperties());
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/DummyRestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/DummyRestConsumerFactory.java b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/DummyRestConsumerFactory.java
index d0d6efe..9946c4d 100644
--- a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/DummyRestConsumerFactory.java
+++ b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/DummyRestConsumerFactory.java
@@ -23,13 +23,14 @@ import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.component.seda.SedaEndpoint;
 import org.apache.camel.impl.ActiveMQUuidGenerator;
+import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 
 public class DummyRestConsumerFactory implements RestConsumerFactory {
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // just use a seda endpoint for testing purpose
         String id;
         if (uriTemplate != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-swagger/src/test/java/org/apache/camel/component/swagger/DummyRestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger/src/test/java/org/apache/camel/component/swagger/DummyRestConsumerFactory.java b/components/camel-swagger/src/test/java/org/apache/camel/component/swagger/DummyRestConsumerFactory.java
index eed969e..3392332 100644
--- a/components/camel-swagger/src/test/java/org/apache/camel/component/swagger/DummyRestConsumerFactory.java
+++ b/components/camel-swagger/src/test/java/org/apache/camel/component/swagger/DummyRestConsumerFactory.java
@@ -23,13 +23,14 @@ import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.component.seda.SedaEndpoint;
 import org.apache.camel.impl.ActiveMQUuidGenerator;
+import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 
 public class DummyRestConsumerFactory implements RestConsumerFactory {
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // just use a seda endpoint for testing purpose
         String id;
         if (uriTemplate != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-test-blueprint/src/test/java/org/apache/camel/test/blueprint/component/rest/DummyRestConsumerFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-test-blueprint/src/test/java/org/apache/camel/test/blueprint/component/rest/DummyRestConsumerFactory.java b/components/camel-test-blueprint/src/test/java/org/apache/camel/test/blueprint/component/rest/DummyRestConsumerFactory.java
index c87fae5..2236b10 100644
--- a/components/camel-test-blueprint/src/test/java/org/apache/camel/test/blueprint/component/rest/DummyRestConsumerFactory.java
+++ b/components/camel-test-blueprint/src/test/java/org/apache/camel/test/blueprint/component/rest/DummyRestConsumerFactory.java
@@ -23,19 +23,20 @@ import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.component.seda.SedaEndpoint;
 import org.apache.camel.impl.ActiveMQUuidGenerator;
+import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestConsumerFactory;
 
 public class DummyRestConsumerFactory implements RestConsumerFactory {
 
     @Override
-    public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String path, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+    public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         // just use a seda endpoint for testing purpose
         String id;
         if (uriTemplate != null) {
-            id = ActiveMQUuidGenerator.generateSanitizedId(path + uriTemplate);
+            id = ActiveMQUuidGenerator.generateSanitizedId(basePath + uriTemplate);
         } else {
-            id = ActiveMQUuidGenerator.generateSanitizedId(path);
+            id = ActiveMQUuidGenerator.generateSanitizedId(basePath);
         }
         // remove leading dash as we add that ourselves
         if (id.startsWith("-")) {
@@ -44,5 +45,4 @@ public class DummyRestConsumerFactory implements RestConsumerFactory {
         SedaEndpoint seda = camelContext.getEndpoint("seda:" + verb + "-" + id, SedaEndpoint.class);
         return seda.createConsumer(processor);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/bdbeb4a3/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
index 01b9684..8274b07 100644
--- a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
+++ b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/UndertowComponent.java
@@ -84,7 +84,7 @@ public class UndertowComponent extends UriEndpointComponent implements RestConsu
 
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
-                                   String consumes, String produces, Map<String, Object> parameters) throws Exception {
+                                   String consumes, String produces, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         String path = basePath;
         if (uriTemplate != null) {
             // make sure to avoid double slashes
@@ -98,7 +98,11 @@ public class UndertowComponent extends UriEndpointComponent implements RestConsu
         String scheme = "http";
         String host = "";
         int port = 0;
-        RestConfiguration config = getCamelContext().getRestConfiguration("undertow", true);
+
+        RestConfiguration config = configuration;
+        if (config == null) {
+            config = getCamelContext().getRestConfiguration("undertow", true);
+        }
         if (config.getScheme() != null) {
             scheme = config.getScheme();
         }
@@ -112,7 +116,7 @@ public class UndertowComponent extends UriEndpointComponent implements RestConsu
 
         Map<String, Object> map = new HashMap<String, Object>();
         // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("undertow"))) {
+        if (config.getComponent() == null || config.getComponent().equals("undertow")) {
             // setup endpoint options
             if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
                 map.putAll(config.getEndpointProperties());


[04/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 3818efe3ee2d1061134df37489d2f3a0dc100ea1
Parents: 8ce5a2b
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 14:07:39 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:03 2015 +0200

----------------------------------------------------------------------
 .../swagger/RestSwaggerAbstractServlet.java     | 241 -------------------
 .../camel/swagger/RestSwaggerCorsFilter.java    |  55 -----
 .../camel/swagger/RestSwaggerServlet.java       |  91 -------
 .../camel/swagger/RestSwaggerSupport.java       | 209 ++++++++++++++++
 .../swagger/servlet/RestSwaggerCorsFilter.java  |  55 +++++
 .../swagger/servlet/RestSwaggerServlet.java     | 127 ++++++++++
 .../servlet/ServletSwaggerApiProvider.java      |  55 +++++
 .../camel/swagger/spi/SwaggerApiProvider.java   |  32 +++
 .../src/main/webapp/WEB-INF/web.xml             |   6 +-
 9 files changed, 481 insertions(+), 390 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerAbstractServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerAbstractServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerAbstractServlet.java
deleted file mode 100644
index a0a1a76..0000000
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerAbstractServlet.java
+++ /dev/null
@@ -1,241 +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.swagger;
-
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.List;
-import javax.servlet.ServletConfig;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import io.swagger.jaxrs.config.BeanConfig;
-import io.swagger.models.Contact;
-import io.swagger.models.Info;
-import io.swagger.models.License;
-import io.swagger.models.Swagger;
-import org.apache.camel.impl.DefaultClassResolver;
-import org.apache.camel.model.rest.RestDefinition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.camel.swagger.SwaggerHelper.buildUrl;
-
-/**
- * Base class for rest-dsl swagger integration to use a servlet to service the swagger api.
- */
-public abstract class RestSwaggerAbstractServlet extends HttpServlet {
-
-    private Logger LOG = LoggerFactory.getLogger(RestSwaggerAbstractServlet.class);
-
-    private RestSwaggerReader reader = new RestSwaggerReader();
-    private BeanConfig swaggerConfig = new BeanConfig();
-    private boolean cors;
-    private volatile boolean initDone;
-
-    @Override
-    public void init(ServletConfig config) throws ServletException {
-        super.init(config);
-
-        // configure swagger options
-        String s = config.getInitParameter("swagger.version");
-        if (s != null) {
-            swaggerConfig.setVersion(s);
-        }
-        s = config.getInitParameter("base.path");
-        if (s != null) {
-            swaggerConfig.setBasePath(s);
-        }
-        s = config.getInitParameter("host");
-        if (s != null) {
-            swaggerConfig.setHost(s);
-        }
-        s = config.getInitParameter("cors");
-        if (s != null) {
-            cors = "true".equalsIgnoreCase(s);
-        }
-        s = config.getInitParameter("schemas");
-        if (s != null) {
-            String[] schemas = s.split(",");
-            swaggerConfig.setSchemes(schemas);
-        } else {
-            // assume http by default
-            swaggerConfig.setSchemes(new String[]{"http"});
-        }
-
-        String version = config.getInitParameter("api.version");
-        String title = config.getInitParameter("api.title");
-        String description = config.getInitParameter("api.description");
-        String termsOfService = config.getInitParameter("api.termsOfService");
-        String licenseName = config.getInitParameter("api.license.name");
-        String licenseUrl = config.getInitParameter("api.license.url");
-        String contactName = config.getInitParameter("api.contact.name");
-        String contactUrl = config.getInitParameter("api.contact.url");
-        String contactEmail = config.getInitParameter("api.contact.email");
-
-        Info info = new Info();
-        info.setVersion(version);
-        info.setTitle(title);
-        info.setDescription(description);
-        info.setTermsOfService(termsOfService);
-
-        if (licenseName != null || licenseUrl != null) {
-            License license = new License();
-            license.setName(licenseName);
-            license.setUrl(licenseUrl);
-            info.setLicense(license);
-        }
-
-        if (contactName != null || contactUrl != null || contactEmail != null) {
-            Contact contact = new Contact();
-            contact.setName(contactName);
-            contact.setUrl(contactUrl);
-            contact.setEmail(contactEmail);
-            info.setContact(contact);
-        }
-
-        swaggerConfig.setInfo(info);
-    }
-
-    public abstract List<RestDefinition> getRestDefinitions(String camelId) throws Exception;
-
-    public abstract List<String> findCamelContexts() throws Exception;
-
-    @Override
-    protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
-
-        if (!initDone) {
-            initBaseAndApiPaths(request);
-        }
-
-        String contextId;
-        String route = request.getPathInfo();
-
-        try {
-
-            // render list of camel contexts as root
-            if (route == null || route.equals("") || route.equals("/")) {
-                renderCamelContexts(request, response);
-            } else {
-                // first part is the camel context
-                if (route.startsWith("/")) {
-                    route = route.substring(1);
-                }
-                // the remainder is the route part
-                contextId = route.split("/")[0];
-                if (route.startsWith(contextId)) {
-                    route = route.substring(contextId.length());
-                }
-
-                renderResourceListing(request, response, contextId, route);
-            }
-        } catch (Exception e) {
-            LOG.warn("Error rendering swagger due " + e.getMessage(), e);
-        }
-    }
-
-    private void renderResourceListing(HttpServletRequest request, HttpServletResponse response, String contextId, String route) throws Exception {
-        LOG.trace("renderResourceListing");
-
-        if (cors) {
-            response.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
-            response.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
-            response.addHeader("Access-Control-Allow-Origin", "*");
-        }
-
-        List<RestDefinition> rests = getRestDefinitions(contextId);
-        if (rests != null) {
-            // read the rest-dsl into swagger model
-            Swagger swagger = reader.read(rests, route, swaggerConfig, new DefaultClassResolver());
-
-            ObjectMapper mapper = new ObjectMapper();
-            mapper.enable(SerializationFeature.INDENT_OUTPUT);
-            mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
-            mapper.writeValue(response.getOutputStream(), swagger);
-        } else {
-            response.setStatus(204);
-        }
-    }
-
-    private void initBaseAndApiPaths(HttpServletRequest request) throws MalformedURLException {
-        String base = swaggerConfig.getBasePath();
-        if (base == null || !base.startsWith("http")) {
-            // base path is configured using relative, so lets calculate the absolute url now we have the http request
-            URL url = new URL(request.getRequestURL().toString());
-            if (base == null) {
-                base = "";
-            }
-            String path = translateContextPath(request);
-            swaggerConfig.setHost(url.getHost());
-
-            if (url.getPort() != 80 && url.getPort() != -1) {
-                swaggerConfig.setHost(url.getHost() + ":" + url.getPort());
-            } else {
-                swaggerConfig.setHost(url.getHost());
-            }
-            swaggerConfig.setBasePath(buildUrl(path, base));
-        }
-        initDone = true;
-    }
-
-    /**
-     * We do only want the base context-path and not sub paths
-     */
-    private String translateContextPath(HttpServletRequest request) {
-        String path = request.getContextPath();
-        if (path.isEmpty() || path.equals("/")) {
-            return "";
-        } else {
-            int idx = path.lastIndexOf("/");
-            if (idx > 0) {
-                return path.substring(0, idx);
-            }
-        }
-        return path;
-    }
-
-    /**
-     * Renders a list of available CamelContexts in the JVM
-     */
-    private void renderCamelContexts(HttpServletRequest request, HttpServletResponse response) throws Exception {
-        LOG.trace("renderCamelContexts");
-
-        if (cors) {
-            response.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
-            response.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
-            response.addHeader("Access-Control-Allow-Origin", "*");
-        }
-
-        List<String> contexts = findCamelContexts();
-        response.getWriter().print("[\n");
-        for (int i = 0; i < contexts.size(); i++) {
-            String name = contexts.get(i);
-            response.getWriter().print("{\"name\": \"" + name + "\"}");
-            if (i < contexts.size() - 1) {
-                response.getWriter().print(",\n");
-            }
-        }
-        response.getWriter().print("\n]");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerCorsFilter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerCorsFilter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerCorsFilter.java
deleted file mode 100644
index 0cf7303..0000000
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerCorsFilter.java
+++ /dev/null
@@ -1,55 +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.swagger;
-
-import java.io.IOException;
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-
-/**
- * A simple CORS filter that can used to allow the swagger ui or other API browsers from remote origins to access the
- * Rest services exposes by this Camel swagger component.
- */
-public class RestSwaggerCorsFilter implements Filter {
-
-    @Override
-    public void init(FilterConfig filterConfig) throws ServletException {
-        // noop
-    }
-
-    @Override
-    public void destroy() {
-        // noop
-    }
-
-    @Override
-    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
-        HttpServletResponse res = (HttpServletResponse) response;
-
-        res.setHeader("Access-Control-Allow-Origin", "*");
-        res.setHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
-        res.setHeader("Access-Control-Max-Age", "3600");
-        res.setHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
-
-        chain.doFilter(request, response);
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerServlet.java
deleted file mode 100644
index 7d85e76..0000000
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerServlet.java
+++ /dev/null
@@ -1,91 +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.swagger;
-
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-import org.apache.camel.model.ModelHelper;
-import org.apache.camel.model.rest.RestDefinition;
-import org.apache.camel.model.rest.RestsDefinition;
-import org.apache.camel.util.CamelVersionHelper;
-
-/**
- * The default Camel swagger servlet to use when exposing the APIs of the rest-dsl using swagger.
- * <p/>
- * This requires Camel version 2.15 or better at runtime (and JMX to be enabled).
- */
-public class RestSwaggerServlet extends RestSwaggerAbstractServlet {
-
-    @Override
-    public List<RestDefinition> getRestDefinitions(String camelId) throws Exception {
-        ObjectName found = null;
-
-        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
-        Set<ObjectName> names = server.queryNames(new ObjectName("org.apache.camel:type=context,*"), null);
-        for (ObjectName on : names) {
-            String id = on.getKeyProperty("name");
-            if (id.startsWith("\"") && id.endsWith("\"")) {
-                id = id.substring(1, id.length() - 1);
-            }
-            if (camelId == null || camelId.equals(id)) {
-                // filter out older Camel versions as this requires Camel 2.15 or better (rest-dsl)
-                String version = (String) server.getAttribute(on, "CamelVersion");
-                if (CamelVersionHelper.isGE("2.15.0", version)) {
-                    found = on;
-                }
-            }
-        }
-
-        if (found != null) {
-            String xml = (String) server.invoke(found, "dumpRestsAsXml", null, null);
-            if (xml != null) {
-                RestsDefinition rests = ModelHelper.createModelFromXml(null, xml, RestsDefinition.class);
-                if (rests != null) {
-                    return rests.getRests();
-                }
-            }
-        }
-
-        return null;
-    }
-
-    @Override
-    public List<String> findCamelContexts() throws Exception {
-        List<String> answer = new ArrayList<>();
-
-        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
-        Set<ObjectName> names = server.queryNames(new ObjectName("*:type=context,*"), null);
-        for (ObjectName on : names) {
-            String id = on.getKeyProperty("name");
-            if (id.startsWith("\"") && id.endsWith("\"")) {
-                id = id.substring(1, id.length() - 1);
-            }
-
-            // filter out older Camel versions as this requires Camel 2.15 or better (rest-dsl)
-            String version = (String) server.getAttribute(on, "CamelVersion");
-            if (CamelVersionHelper.isGE("2.15.0", version)) {
-                answer.add(id);
-            }
-        }
-        return answer;
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
new file mode 100644
index 0000000..6b7f56d
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -0,0 +1,209 @@
+/**
+ * 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.swagger;
+
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import io.swagger.jaxrs.config.BeanConfig;
+import io.swagger.models.Contact;
+import io.swagger.models.Info;
+import io.swagger.models.License;
+import io.swagger.models.Swagger;
+import org.apache.camel.impl.DefaultClassResolver;
+import org.apache.camel.model.ModelHelper;
+import org.apache.camel.model.rest.RestDefinition;
+import org.apache.camel.model.rest.RestsDefinition;
+import org.apache.camel.swagger.spi.SwaggerApiProvider;
+import org.apache.camel.util.CamelVersionHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RestSwaggerSupport {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerSupport.class);
+    private RestSwaggerReader reader = new RestSwaggerReader();
+    private boolean cors;
+
+    public void initSwagger(BeanConfig swaggerConfig, SwaggerApiProvider config) {
+        // configure swagger options
+        String s = config.getInitParameter("swagger.version");
+        if (s != null) {
+            swaggerConfig.setVersion(s);
+        }
+        s = config.getInitParameter("base.path");
+        if (s != null) {
+            swaggerConfig.setBasePath(s);
+        }
+        s = config.getInitParameter("host");
+        if (s != null) {
+            swaggerConfig.setHost(s);
+        }
+        s = config.getInitParameter("cors");
+        if (s != null) {
+            cors = "true".equalsIgnoreCase(s);
+        }
+        s = config.getInitParameter("schemas");
+        if (s != null) {
+            String[] schemas = s.split(",");
+            swaggerConfig.setSchemes(schemas);
+        } else {
+            // assume http by default
+            swaggerConfig.setSchemes(new String[]{"http"});
+        }
+
+        String version = config.getInitParameter("api.version");
+        String title = config.getInitParameter("api.title");
+        String description = config.getInitParameter("api.description");
+        String termsOfService = config.getInitParameter("api.termsOfService");
+        String licenseName = config.getInitParameter("api.license.name");
+        String licenseUrl = config.getInitParameter("api.license.url");
+        String contactName = config.getInitParameter("api.contact.name");
+        String contactUrl = config.getInitParameter("api.contact.url");
+        String contactEmail = config.getInitParameter("api.contact.email");
+
+        Info info = new Info();
+        info.setVersion(version);
+        info.setTitle(title);
+        info.setDescription(description);
+        info.setTermsOfService(termsOfService);
+
+        if (licenseName != null || licenseUrl != null) {
+            License license = new License();
+            license.setName(licenseName);
+            license.setUrl(licenseUrl);
+            info.setLicense(license);
+        }
+
+        if (contactName != null || contactUrl != null || contactEmail != null) {
+            Contact contact = new Contact();
+            contact.setName(contactName);
+            contact.setUrl(contactUrl);
+            contact.setEmail(contactEmail);
+            info.setContact(contact);
+        }
+
+        swaggerConfig.setInfo(info);
+    }
+
+    public List<RestDefinition> getRestDefinitions(String camelId) throws Exception {
+        ObjectName found = null;
+
+        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+        Set<ObjectName> names = server.queryNames(new ObjectName("org.apache.camel:type=context,*"), null);
+        for (ObjectName on : names) {
+            String id = on.getKeyProperty("name");
+            if (id.startsWith("\"") && id.endsWith("\"")) {
+                id = id.substring(1, id.length() - 1);
+            }
+            if (camelId == null || camelId.equals(id)) {
+                // filter out older Camel versions as this requires Camel 2.15 or better (rest-dsl)
+                String version = (String) server.getAttribute(on, "CamelVersion");
+                if (CamelVersionHelper.isGE("2.15.0", version)) {
+                    found = on;
+                }
+            }
+        }
+
+        if (found != null) {
+            String xml = (String) server.invoke(found, "dumpRestsAsXml", null, null);
+            if (xml != null) {
+                RestsDefinition rests = ModelHelper.createModelFromXml(null, xml, RestsDefinition.class);
+                if (rests != null) {
+                    return rests.getRests();
+                }
+            }
+        }
+
+        return null;
+    }
+
+    public List<String> findCamelContexts() throws Exception {
+        List<String> answer = new ArrayList<>();
+
+        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+        Set<ObjectName> names = server.queryNames(new ObjectName("*:type=context,*"), null);
+        for (ObjectName on : names) {
+            String id = on.getKeyProperty("name");
+            if (id.startsWith("\"") && id.endsWith("\"")) {
+                id = id.substring(1, id.length() - 1);
+            }
+
+            // filter out older Camel versions as this requires Camel 2.15 or better (rest-dsl)
+            String version = (String) server.getAttribute(on, "CamelVersion");
+            if (CamelVersionHelper.isGE("2.15.0", version)) {
+                answer.add(id);
+            }
+        }
+        return answer;
+    }
+
+    public void renderResourceListing(SwaggerApiProvider provider, BeanConfig swaggerConfig, String contextId, String route) throws Exception {
+        LOG.trace("renderResourceListing");
+
+        if (cors) {
+            provider.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
+            provider.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
+            provider.addHeader("Access-Control-Allow-Origin", "*");
+        }
+
+        List<RestDefinition> rests = getRestDefinitions(contextId);
+        if (rests != null) {
+            // read the rest-dsl into swagger model
+            Swagger swagger = reader.read(rests, route, swaggerConfig, new DefaultClassResolver());
+
+            ObjectMapper mapper = new ObjectMapper();
+            mapper.enable(SerializationFeature.INDENT_OUTPUT);
+            mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+            mapper.writeValue(provider.getOutputStream(), swagger);
+        } else {
+            provider.noContent();
+        }
+    }
+
+    /**
+     * Renders a list of available CamelContexts in the JVM
+     */
+    public void renderCamelContexts(SwaggerApiProvider provider) throws Exception {
+        LOG.trace("renderCamelContexts");
+
+        if (cors) {
+            provider.addHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
+            provider.addHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
+            provider.addHeader("Access-Control-Allow-Origin", "*");
+        }
+
+        List<String> contexts = findCamelContexts();
+        provider.getOutputStream().write("[\n".getBytes());
+        for (int i = 0; i < contexts.size(); i++) {
+            String name = contexts.get(i);
+            provider.getOutputStream().write(("{\"name\": \"" + name + "\"}").getBytes());
+            if (i < contexts.size() - 1) {
+                provider.getOutputStream().write(",\n".getBytes());
+            }
+        }
+        provider.getOutputStream().write("\n]".getBytes());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerCorsFilter.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerCorsFilter.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerCorsFilter.java
new file mode 100644
index 0000000..4aca487
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerCorsFilter.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.swagger.servlet;
+
+import java.io.IOException;
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * A simple CORS filter that can used to allow the swagger ui or other API browsers from remote origins to access the
+ * Rest services exposes by this Camel swagger component.
+ */
+public class RestSwaggerCorsFilter implements Filter {
+
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+        // noop
+    }
+
+    @Override
+    public void destroy() {
+        // noop
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+        HttpServletResponse res = (HttpServletResponse) response;
+
+        res.setHeader("Access-Control-Allow-Origin", "*");
+        res.setHeader("Access-Control-Allow-Methods", "GET, HEAD, POST, PUT, DELETE, TRACE, OPTIONS, CONNECT, PATCH");
+        res.setHeader("Access-Control-Max-Age", "3600");
+        res.setHeader("Access-Control-Allow-Headers", "Origin, Accept, X-Requested-With, Content-Type, Access-Control-Request-Method, Access-Control-Request-Headers");
+
+        chain.doFilter(request, response);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
new file mode 100644
index 0000000..75d0290
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
@@ -0,0 +1,127 @@
+/**
+ * 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.swagger.servlet;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import javax.servlet.ServletConfig;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import io.swagger.jaxrs.config.BeanConfig;
+import org.apache.camel.swagger.RestSwaggerSupport;
+import org.apache.camel.swagger.spi.SwaggerApiProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.camel.swagger.SwaggerHelper.buildUrl;
+
+/**
+ * The default Camel swagger servlet to use when exposing the APIs of the rest-dsl using swagger.
+ * <p/>
+ * This requires Camel version 2.15 or better at runtime (and JMX to be enabled).
+ */
+public class RestSwaggerServlet extends HttpServlet {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerServlet.class);
+    private BeanConfig swaggerConfig = new BeanConfig();
+    private RestSwaggerSupport swagger = new RestSwaggerSupport();
+    private volatile boolean initDone;
+
+    @Override
+    public void init(final ServletConfig config) throws ServletException {
+        super.init(config);
+
+        swagger.initSwagger(swaggerConfig, new ServletSwaggerApiProvider(config, null));
+    }
+
+    @Override
+    protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
+
+        if (!initDone) {
+            initBaseAndApiPaths(request);
+        }
+
+        SwaggerApiProvider resp = new ServletSwaggerApiProvider(null, response);
+
+        String contextId;
+        String route = request.getPathInfo();
+
+        try {
+
+            // render list of camel contexts as root
+            if (route == null || route.equals("") || route.equals("/")) {
+                swagger.renderCamelContexts(resp);
+            } else {
+                // first part is the camel context
+                if (route.startsWith("/")) {
+                    route = route.substring(1);
+                }
+                // the remainder is the route part
+                contextId = route.split("/")[0];
+                if (route.startsWith(contextId)) {
+                    route = route.substring(contextId.length());
+                }
+
+                swagger.renderResourceListing(resp, swaggerConfig, contextId, route);
+            }
+        } catch (Exception e) {
+            LOG.warn("Error rendering swagger due " + e.getMessage(), e);
+        }
+    }
+
+    private void initBaseAndApiPaths(HttpServletRequest request) throws MalformedURLException {
+        String base = swaggerConfig.getBasePath();
+        if (base == null || !base.startsWith("http")) {
+            // base path is configured using relative, so lets calculate the absolute url now we have the http request
+            URL url = new URL(request.getRequestURL().toString());
+            if (base == null) {
+                base = "";
+            }
+            String path = translateContextPath(request);
+            swaggerConfig.setHost(url.getHost());
+
+            if (url.getPort() != 80 && url.getPort() != -1) {
+                swaggerConfig.setHost(url.getHost() + ":" + url.getPort());
+            } else {
+                swaggerConfig.setHost(url.getHost());
+            }
+            swaggerConfig.setBasePath(buildUrl(path, base));
+        }
+        initDone = true;
+    }
+
+    /**
+     * We do only want the base context-path and not sub paths
+     */
+    private String translateContextPath(HttpServletRequest request) {
+        String path = request.getContextPath();
+        if (path.isEmpty() || path.equals("/")) {
+            return "";
+        } else {
+            int idx = path.lastIndexOf("/");
+            if (idx > 0) {
+                return path.substring(0, idx);
+            }
+        }
+        return path;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
new file mode 100644
index 0000000..3d39825
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/ServletSwaggerApiProvider.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.swagger.servlet;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import javax.servlet.ServletConfig;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.camel.swagger.spi.SwaggerApiProvider;
+
+public class ServletSwaggerApiProvider implements SwaggerApiProvider {
+
+    private final ServletConfig config;
+    private final HttpServletResponse response;
+
+    public ServletSwaggerApiProvider(ServletConfig config, HttpServletResponse response) {
+        this.config = config;
+        this.response = response;
+    }
+
+    @Override
+    public String getInitParameter(String key) {
+        return config.getInitParameter(key);
+    }
+
+    @Override
+    public void addHeader(String name, String value) {
+        response.addHeader(name, value);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        return response.getOutputStream();
+    }
+
+    @Override
+    public void noContent() {
+        response.setStatus(HttpServletResponse.SC_NO_CONTENT);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
new file mode 100644
index 0000000..42e6a2d
--- /dev/null
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/spi/SwaggerApiProvider.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.swagger.spi;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public interface SwaggerApiProvider {
+
+    String getInitParameter(String key);
+
+    void addHeader(String name, String value);
+
+    OutputStream getOutputStream() throws IOException;
+
+    void noContent();
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/3818efe3/examples/camel-example-swagger-java/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-java/src/main/webapp/WEB-INF/web.xml b/examples/camel-example-swagger-java/src/main/webapp/WEB-INF/web.xml
index 35527aa..cfde29f 100755
--- a/examples/camel-example-swagger-java/src/main/webapp/WEB-INF/web.xml
+++ b/examples/camel-example-swagger-java/src/main/webapp/WEB-INF/web.xml
@@ -44,7 +44,7 @@
   <!-- to setup Camel Swagger servlet -->
   <servlet>
     <servlet-name>ApiDeclarationServlet</servlet-name>
-    <servlet-class>org.apache.camel.swagger.RestSwaggerServlet</servlet-class>
+    <servlet-class>org.apache.camel.swagger.servlet.RestSwaggerServlet</servlet-class>
     <init-param>
       <!-- we specify the base.path using relative notation, that means the actual path will be calculated at runtime as
            http://server:port/contextpath/rest -->
@@ -91,7 +91,7 @@
   <!-- enable CORS filter so people can use swagger ui to browse and test the apis -->
   <filter>
     <filter-name>RestSwaggerCorsFilter</filter-name>
-    <filter-class>org.apache.camel.swagger.RestSwaggerCorsFilter</filter-class>
+    <filter-class>org.apache.camel.swagger.servlet.RestSwaggerCorsFilter</filter-class>
   </filter>
 
   <filter-mapping>
@@ -101,7 +101,7 @@
   </filter-mapping>
 
   <welcome-file-list>
-      <welcome-file>home.html</welcome-file>
+    <welcome-file>home.html</welcome-file>
   </welcome-file-list>
 
 </web-app>
\ No newline at end of file


[05/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 8252a162061ac654cc3b7fd8d61334fee6ddfa3f
Parents: 818557e
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 17:52:18 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:04 2015 +0200

----------------------------------------------------------------------
 .../netty4/http/NettyHttpComponent.java         | 81 ++++----------------
 1 file changed, 17 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/8252a162/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
index 9012d93..71fe02e 100644
--- a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/NettyHttpComponent.java
@@ -226,6 +226,17 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
     @Override
     public Consumer createConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
                                    String consumes, String produces, Map<String, Object> parameters) throws Exception {
+        return doCreateConsumer(camelContext, processor, verb, basePath, uriTemplate, consumes, produces, parameters, false);
+    }
+
+    @Override
+    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
+        // reuse the createConsumer method we already have. The api need to use GET and match on uri prefix
+        return doCreateConsumer(camelContext, processor, "GET", contextPath, null, null, null, parameters, true);
+    }
+
+    Consumer doCreateConsumer(CamelContext camelContext, Processor processor, String verb, String basePath, String uriTemplate,
+                              String consumes, String produces, Map<String, Object> parameters, boolean api) throws Exception {
 
         String path = basePath;
         if (uriTemplate != null) {
@@ -275,7 +286,12 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
 
         String query = URISupport.createQueryString(map);
 
-        String url = "netty4-http:%s://%s:%s/%s?httpMethodRestrict=%s";
+        String url;
+        if (api) {
+            url = "netty4-http:%s://%s:%s/%s?matchOnUriPrefix=true&httpMethodRestrict=%s";
+        } else {
+            url = "netty4-http:%s://%s:%s/%s?httpMethodRestrict=%s";
+        }
         // must use upper case for restrict
         String restrict = verb.toUpperCase(Locale.US);
         // get the endpoint
@@ -298,69 +314,6 @@ public class NettyHttpComponent extends NettyComponent implements HeaderFilterSt
     }
 
     @Override
-    public Consumer createApiConsumer(CamelContext camelContext, Processor processor, String contextPath, Map<String, Object> parameters) throws Exception {
-        String path = FileUtil.stripLeadingSeparator(contextPath);
-
-        String scheme = "http";
-        String host = "";
-        int port = 0;
-
-        // if no explicit port/host configured, then use port from rest configuration
-        RestConfiguration config = getCamelContext().getRestConfiguration("netty4-http", true);
-        if (config.getScheme() != null) {
-            scheme = config.getScheme();
-        }
-        if (config.getHost() != null) {
-            host = config.getHost();
-        }
-        int num = config.getPort();
-        if (num > 0) {
-            port = num;
-        }
-
-        // if no explicit hostname set then resolve the hostname
-        if (ObjectHelper.isEmpty(host)) {
-            if (config.getRestHostNameResolver() == RestConfiguration.RestHostNameResolver.localHostName) {
-                host = HostUtils.getLocalHostName();
-            } else if (config.getRestHostNameResolver() == RestConfiguration.RestHostNameResolver.localIp) {
-                host = HostUtils.getLocalIp();
-            }
-        }
-
-        Map<String, Object> map = new HashMap<String, Object>();
-        // build query string, and append any endpoint configuration properties
-        if (config != null && (config.getComponent() == null || config.getComponent().equals("netty4-http"))) {
-            // setup endpoint options
-            if (config.getEndpointProperties() != null && !config.getEndpointProperties().isEmpty()) {
-                map.putAll(config.getEndpointProperties());
-            }
-        }
-
-        String query = URISupport.createQueryString(map);
-
-        String url = "netty4-http:%s://%s:%s/%s?httpMethodRestrict=%s&matchOnUriPrefix=true";
-        // must use upper case for restrict
-        String restrict = "GET";
-        // get the endpoint
-        url = String.format(url, scheme, host, port, path, restrict);
-
-        if (!query.isEmpty()) {
-            url = url + "&" + query;
-        }
-
-        NettyHttpEndpoint endpoint = camelContext.getEndpoint(url, NettyHttpEndpoint.class);
-        setProperties(endpoint, parameters);
-
-        // configure consumer properties
-        Consumer consumer = endpoint.createConsumer(processor);
-        if (config != null && config.getConsumerProperties() != null && !config.getConsumerProperties().isEmpty()) {
-            setProperties(consumer, config.getConsumerProperties());
-        }
-
-        return consumer;
-    }
-
-    @Override
     protected void doStop() throws Exception {
         super.doStop();
 


[09/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 26ae7724e1877645b1e36c7c3575760e0f83120a
Parents: bdbeb4a
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 21:14:50 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:05 2015 +0200

----------------------------------------------------------------------
 .../camel/component/rest/RestApiEndpoint.java   | 39 ++++++++++++++++++--
 .../model/rest/RestConfigurationDefinition.java | 25 +++++++++++++
 .../camel/spi/RestApiConsumerFactory.java       |  2 +
 .../org/apache/camel/spi/RestConfiguration.java | 19 ++++++++++
 .../netty4/http/rest/RestApiNettyTest.java      |  9 -----
 .../services/org/apache/camel/component/sql     | 18 ---------
 .../services/org/apache/camel/rest/swagger      | 18 +++++++++
 7 files changed, 100 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
index 56fe965..7ceb16b 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
@@ -16,15 +16,18 @@
  */
 package org.apache.camel.component.rest;
 
+import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.camel.CamelContext;
 import org.apache.camel.Component;
 import org.apache.camel.Consumer;
 import org.apache.camel.NoSuchBeanException;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
 import org.apache.camel.impl.DefaultEndpoint;
+import org.apache.camel.spi.FactoryFinder;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestApiProcessorFactory;
@@ -36,10 +39,15 @@ import org.apache.camel.spi.UriPath;
 @UriEndpoint(scheme = "rest-api", title = "REST API", syntax = "rest-api:path", consumerOnly = true, label = "core,rest")
 public class RestApiEndpoint extends DefaultEndpoint {
 
+    public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/rest/";
+    private FactoryFinder factoryFinder;
+
     @UriPath @Metadata(required = "true")
     private String path;
     @UriParam
     private String componentName;
+    @UriParam
+    private String apiComponentName;
 
     private Map<String, Object> parameters;
 
@@ -77,6 +85,17 @@ public class RestApiEndpoint extends DefaultEndpoint {
         this.componentName = componentName;
     }
 
+    public String getApiComponentName() {
+        return apiComponentName;
+    }
+
+    /**
+     * The Camel Rest API component to use for generating the API of the REST services, such as swagger.
+     */
+    public void setApiComponentName(String apiComponentName) {
+        this.apiComponentName = apiComponentName;
+    }
+
     public Map<String, Object> getParameters() {
         return parameters;
     }
@@ -88,19 +107,33 @@ public class RestApiEndpoint extends DefaultEndpoint {
         this.parameters = parameters;
     }
 
+    private Class<?> findApiProcessorFactory(String name, CamelContext context) throws ClassNotFoundException, IOException {
+        if (factoryFinder == null) {
+            factoryFinder = context.getFactoryFinder(RESOURCE_PATH);
+        }
+        return factoryFinder.findClass(name);
+    }
+
     @Override
     public Producer createProducer() throws Exception {
         RestApiProcessorFactory factory = null;
 
+        RestConfiguration config = getCamelContext().getRestConfiguration(componentName, true);
+
         // lookup in registry
         Set<RestApiProcessorFactory> factories = getCamelContext().getRegistry().findByType(RestApiProcessorFactory.class);
         if (factories != null && factories.size() == 1) {
             factory = factories.iterator().next();
         }
 
-        if (factory != null) {
+        // lookup on classpath using factory finder
+        String name = apiComponentName != null ? apiComponentName : config.getApiComponent();
+        Object instance = getCamelContext().getFactoryFinder(RESOURCE_PATH).newInstance(name);
+        if (instance instanceof RestApiProcessorFactory) {
+            factory = (RestApiProcessorFactory) instance;
+        }
 
-            RestConfiguration config = getCamelContext().getRestConfiguration(componentName, true);
+        if (factory != null) {
 
             // calculate the url to the rest API service
             String path = getPath();
@@ -111,7 +144,7 @@ public class RestApiEndpoint extends DefaultEndpoint {
             Processor processor = factory.createApiProcessor(getCamelContext(), path, config, getParameters());
             return new RestApiProducer(this, processor);
         } else {
-            throw new IllegalStateException("Cannot find RestApiProcessorFactory in Registry");
+            throw new IllegalStateException("Cannot find RestApiProcessorFactory in Registry or classpath");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
index 2c5906d..90a6138 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
@@ -42,6 +42,9 @@ public class RestConfigurationDefinition {
     @XmlAttribute
     private String component;
 
+    @XmlAttribute @Metadata(defaultValue = "swagger")
+    private String apiComponent;
+
     @XmlAttribute
     private String scheme;
 
@@ -107,6 +110,17 @@ public class RestConfigurationDefinition {
         this.component = component;
     }
 
+    public String getApiComponent() {
+        return apiComponent;
+    }
+
+    /**
+     * The name of the Camel component to use as the REST API (such as swagger)
+     */
+    public void setApiComponent(String apiComponent) {
+        this.apiComponent = apiComponent;
+    }
+
     public String getScheme() {
         return scheme;
     }
@@ -341,6 +355,14 @@ public class RestConfigurationDefinition {
     }
 
     /**
+     * To use a specific Camel rest API component
+     */
+    public RestConfigurationDefinition apiComponent(String componentId) {
+        setApiComponent(componentId);
+        return this;
+    }
+
+    /**
      * To use a specific scheme such as http/https
      */
     public RestConfigurationDefinition scheme(String scheme) {
@@ -531,6 +553,9 @@ public class RestConfigurationDefinition {
         if (component != null) {
             answer.setComponent(CamelContextHelper.parseText(context, component));
         }
+        if (apiComponent != null) {
+            answer.setApiComponent(CamelContextHelper.parseText(context, apiComponent));
+        }
         if (scheme != null) {
             answer.setScheme(CamelContextHelper.parseText(context, scheme));
         }

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
index c000d93..b051f29 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiConsumerFactory.java
@@ -24,6 +24,8 @@ import org.apache.camel.Processor;
 
 public interface RestApiConsumerFactory {
 
+    // TODO: merge this method to RestConsumerFactory
+
     /**
      * Creates a new REST API <a
      * href="http://camel.apache.org/event-driven-consumer.html">Event

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
index 768ad42..55a45d6 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
@@ -38,6 +38,7 @@ public class RestConfiguration {
     }
 
     private String component;
+    private String apiComponent;
     private String scheme;
     private String host;
     private int port;
@@ -75,6 +76,24 @@ public class RestConfiguration {
     }
 
     /**
+     * Gets the name of the Camel component to use as the REST API (such as swagger)
+     *
+     * @return the component name, or <tt>null</tt> to let Camel use the default name <tt>swagger</tt>
+     */
+    public String getApiComponent() {
+        return apiComponent;
+    }
+
+    /**
+     * Sets the name of the Camel component to use as the REST API (such as swagger)
+     *
+     * @param apiComponent the name of the component (such as swagger)
+     */
+    public void setApiComponent(String apiComponent) {
+        this.apiComponent = apiComponent;
+    }
+
+    /**
      * Gets the hostname to use by the REST consumer
      *
      * @return the hostname, or <tt>null</tt> to use default hostname

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
index 99d1665..41c22dc 100644
--- a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
+++ b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
@@ -18,21 +18,12 @@ package org.apache.camel.component.netty4.http.rest;
 
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.netty4.http.BaseNettyTest;
-import org.apache.camel.impl.JndiRegistry;
 import org.apache.camel.model.rest.RestParamType;
-import org.apache.camel.swagger.SwaggerRestApiProcessorFactory;
 import org.junit.Test;
 
 public class RestApiNettyTest extends BaseNettyTest {
 
     @Override
-    protected JndiRegistry createRegistry() throws Exception {
-        JndiRegistry jndi = super.createRegistry();
-        jndi.bind("SwaggerRestApiProcessorFactory", new SwaggerRestApiProcessorFactory());
-        return jndi;
-    }
-
-    @Override
     protected boolean useJmx() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/components/camel-sql/src/main/resources/META-INF/services/org/apache/camel/component/sql
----------------------------------------------------------------------
diff --git a/components/camel-sql/src/main/resources/META-INF/services/org/apache/camel/component/sql b/components/camel-sql/src/main/resources/META-INF/services/org/apache/camel/component/sql
deleted file mode 100755
index 3b9a254..0000000
--- a/components/camel-sql/src/main/resources/META-INF/services/org/apache/camel/component/sql
+++ /dev/null
@@ -1,18 +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.
-#
-
-class=org.apache.camel.component.sql.SqlComponent

http://git-wip-us.apache.org/repos/asf/camel/blob/26ae7724/components/camel-swagger-java/src/main/resources/META-INF/services/org/apache/camel/rest/swagger
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/resources/META-INF/services/org/apache/camel/rest/swagger b/components/camel-swagger-java/src/main/resources/META-INF/services/org/apache/camel/rest/swagger
new file mode 100755
index 0000000..999809c
--- /dev/null
+++ b/components/camel-swagger-java/src/main/resources/META-INF/services/org/apache/camel/rest/swagger
@@ -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.swagger.SwaggerRestApiProcessorFactory


[12/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 8fff52ad073d3cdd13099e3fd6c9346e504f3bbf
Parents: 44a15c5
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 09:29:08 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 09:29:08 2015 +0200

----------------------------------------------------------------------
 .../camel/component/rest/RestApiEndpoint.java     | 18 +++++++-----------
 1 file changed, 7 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/8fff52ad/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
index 7ceb16b..e6df668 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
@@ -16,18 +16,15 @@
  */
 package org.apache.camel.component.rest;
 
-import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.Component;
 import org.apache.camel.Consumer;
 import org.apache.camel.NoSuchBeanException;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
 import org.apache.camel.impl.DefaultEndpoint;
-import org.apache.camel.spi.FactoryFinder;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RestApiConsumerFactory;
 import org.apache.camel.spi.RestApiProcessorFactory;
@@ -39,8 +36,8 @@ import org.apache.camel.spi.UriPath;
 @UriEndpoint(scheme = "rest-api", title = "REST API", syntax = "rest-api:path", consumerOnly = true, label = "core,rest")
 public class RestApiEndpoint extends DefaultEndpoint {
 
+    public static final String DEFAULT_API_COMPONENT_NAME = "swagger";
     public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/rest/";
-    private FactoryFinder factoryFinder;
 
     @UriPath @Metadata(required = "true")
     private String path;
@@ -107,13 +104,6 @@ public class RestApiEndpoint extends DefaultEndpoint {
         this.parameters = parameters;
     }
 
-    private Class<?> findApiProcessorFactory(String name, CamelContext context) throws ClassNotFoundException, IOException {
-        if (factoryFinder == null) {
-            factoryFinder = context.getFactoryFinder(RESOURCE_PATH);
-        }
-        return factoryFinder.findClass(name);
-    }
-
     @Override
     public Producer createProducer() throws Exception {
         RestApiProcessorFactory factory = null;
@@ -128,6 +118,9 @@ public class RestApiEndpoint extends DefaultEndpoint {
 
         // lookup on classpath using factory finder
         String name = apiComponentName != null ? apiComponentName : config.getApiComponent();
+        if (name == null) {
+            name = DEFAULT_API_COMPONENT_NAME;
+        }
         Object instance = getCamelContext().getFactoryFinder(RESOURCE_PATH).newInstance(name);
         if (instance instanceof RestApiProcessorFactory) {
             factory = (RestApiProcessorFactory) instance;
@@ -152,6 +145,9 @@ public class RestApiEndpoint extends DefaultEndpoint {
     public Consumer createConsumer(Processor processor) throws Exception {
         RestApiConsumerFactory factory = null;
         String cname = null;
+
+        // we use the rest component as the HTTP consumer to service the API
+        // the API then uses the api component (eg usually camel-swagger-java) to build the API
         if (getComponentName() != null) {
             Object comp = getCamelContext().getRegistry().lookupByName(getComponentName());
             if (comp != null && comp instanceof RestApiConsumerFactory) {


[13/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: d975acd2030d672a8843b3fd78d3b6e108b4fd22
Parents: 8fff52a
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 09:54:38 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 09:54:38 2015 +0200

----------------------------------------------------------------------
 .../org/apache/camel/model/rest/RestDefinition.java    |  1 +
 .../org/apache/camel/model/rest/VerbDefinition.java    | 13 +++++++++++++
 .../org/apache/camel/swagger/RestSwaggerReader.java    | 10 +++++++---
 .../org/apache/camel/swagger/RestSwaggerSupport.java   |  2 +-
 .../camel/swagger/RestSwaggerReaderModelTest.java      |  2 +-
 .../apache/camel/swagger/RestSwaggerReaderTest.java    |  2 +-
 6 files changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
index 7f93637..c004af7 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
@@ -655,6 +655,7 @@ public class RestDefinition extends OptionalIdentifiedDefinition<RestDefinition>
                 }
             }
             String routeId = route.idOrCreate(camelContext.getNodeIdFactory());
+            verb.setRouteId(routeId);
             options.put("routeId", routeId);
             if (component != null && !component.isEmpty()) {
                 options.put("componentName", component);

http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/camel-core/src/main/java/org/apache/camel/model/rest/VerbDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/VerbDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/VerbDefinition.java
index 7e119c5..8d1e686 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/VerbDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/VerbDefinition.java
@@ -96,6 +96,8 @@ public class VerbDefinition extends OptionalIdentifiedDefinition<VerbDefinition>
     private RouteDefinition route;
     @XmlTransient
     private RestDefinition rest;
+    @XmlAttribute
+    private String routeId;
 
     @Override
     public String getLabel() {
@@ -239,6 +241,17 @@ public class VerbDefinition extends OptionalIdentifiedDefinition<VerbDefinition>
         this.outType = outType;
     }
 
+    public String getRouteId() {
+        return routeId;
+    }
+
+    /**
+     * The route id this rest-dsl is using (read-only)
+     */
+    public void setRouteId(String routeId) {
+        this.routeId = routeId;
+    }
+
     public RestDefinition getRest() {
         return rest;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerReader.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerReader.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerReader.java
index d5eac90..ea8a6a2 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerReader.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerReader.java
@@ -68,7 +68,7 @@ public class RestSwaggerReader {
      * @param classResolver     class resolver to use
      * @return the swagger model
      */
-    public Swagger read(List<RestDefinition> rests, String route, BeanConfig config, ClassResolver classResolver) {
+    public Swagger read(List<RestDefinition> rests, String route, BeanConfig config, String camelContextId, ClassResolver classResolver) {
         Swagger swagger = new Swagger();
 
         for (RestDefinition rest : rests) {
@@ -80,7 +80,7 @@ public class RestSwaggerReader {
                 }
             }
 
-            parse(swagger, rest, classResolver);
+            parse(swagger, rest, camelContextId, classResolver);
         }
 
         // configure before returning
@@ -88,7 +88,7 @@ public class RestSwaggerReader {
         return swagger;
     }
 
-    private void parse(Swagger swagger, RestDefinition rest, ClassResolver classResolver) {
+    private void parse(Swagger swagger, RestDefinition rest, String camelContextId, ClassResolver classResolver) {
         List<VerbDefinition> verbs = new ArrayList<>(rest.getVerbs());
         // must sort the verbs by uri so we group them together when an uri has multiple operations
         Collections.sort(verbs, new VerbOrdering());
@@ -144,6 +144,10 @@ public class RestSwaggerReader {
             // group in the same tag
             op.addTag(pathAsTag);
 
+            // add id as vendor extensions
+            op.getVendorExtensions().put("x-camelContextId", camelContextId);
+            op.getVendorExtensions().put("x-routeId", verb.getRouteId());
+
             Path path = swagger.getPath(opPath);
             if (path == null) {
                 path = new Path();

http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index 5790651..4e228a8 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -177,7 +177,7 @@ public class RestSwaggerSupport {
         List<RestDefinition> rests = getRestDefinitions(contextId);
         if (rests != null) {
             // read the rest-dsl into swagger model
-            Swagger swagger = reader.read(rests, route, swaggerConfig, new DefaultClassResolver());
+            Swagger swagger = reader.read(rests, route, swaggerConfig, contextId, new DefaultClassResolver());
 
             ObjectMapper mapper = new ObjectMapper();
             mapper.enable(SerializationFeature.INDENT_OUTPUT);

http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderModelTest.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderModelTest.java b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderModelTest.java
index d6293f5..72f8409 100644
--- a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderModelTest.java
+++ b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderModelTest.java
@@ -74,7 +74,7 @@ public class RestSwaggerReaderModelTest extends CamelTestSupport {
         config.setLicenseUrl("http://www.apache.org/licenses/LICENSE-2.0.html");
         RestSwaggerReader reader = new RestSwaggerReader();
 
-        Swagger swagger = reader.read(context.getRestDefinitions(), null, config, new DefaultClassResolver());
+        Swagger swagger = reader.read(context.getRestDefinitions(), null, config, context.getName(), new DefaultClassResolver());
         assertNotNull(swagger);
 
         ObjectMapper mapper = new ObjectMapper();

http://git-wip-us.apache.org/repos/asf/camel/blob/d975acd2/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderTest.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderTest.java b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderTest.java
index 20becb4..e1aa9e9 100644
--- a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderTest.java
+++ b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestSwaggerReaderTest.java
@@ -65,7 +65,7 @@ public class RestSwaggerReaderTest extends CamelTestSupport {
         config.setBasePath("/api");
         RestSwaggerReader reader = new RestSwaggerReader();
 
-        Swagger swagger = reader.read(context.getRestDefinitions(), null, config, new DefaultClassResolver());
+        Swagger swagger = reader.read(context.getRestDefinitions(), null, config, context.getName(), new DefaultClassResolver());
         assertNotNull(swagger);
 
         ObjectMapper mapper = new ObjectMapper();


[07/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 818557e56f9cd1b3f61b6963a2d3721497e19d2f
Parents: aa3e117
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 17:40:45 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:04 2015 +0200

----------------------------------------------------------------------
 components/camel-netty4-http/pom.xml            |  5 ++
 .../netty4/http/rest/RestApiNettyTest.java      | 80 ++++++++++++++++++++
 components/camel-swagger-java/pom.xml           |  5 --
 .../apache/camel/swagger/RestApiNettyTest.java  | 65 ----------------
 4 files changed, 85 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/818557e5/components/camel-netty4-http/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/pom.xml b/components/camel-netty4-http/pom.xml
index 8e80b61..f5942eb 100644
--- a/components/camel-netty4-http/pom.xml
+++ b/components/camel-netty4-http/pom.xml
@@ -80,6 +80,11 @@
       <artifactId>camel-jaxb</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-swagger-java</artifactId>
+      <scope>test</scope>
+    </dependency>
 
     <!-- logging -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/camel/blob/818557e5/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
new file mode 100644
index 0000000..2abf8f1
--- /dev/null
+++ b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/rest/RestApiNettyTest.java
@@ -0,0 +1,80 @@
+/**
+ * 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.netty4.http.rest;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.netty4.http.BaseNettyTest;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.model.rest.RestParamType;
+import org.apache.camel.swagger.SwaggerRestApiProcessorFactory;
+import org.junit.Test;
+
+public class RestApiNettyTest extends BaseNettyTest {
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry jndi = super.createRegistry();
+        jndi.bind("SwaggerRestApiProcessorFactory", new SwaggerRestApiProcessorFactory());
+        return jndi;
+    }
+
+    @Override
+    protected boolean useJmx() {
+        return true;
+    }
+
+    @Test
+    public void testApi() throws Exception {
+        String out = template.requestBody("netty4-http:http://localhost:{{port}}/api-doc/", null, String.class);
+        assertNotNull(out);
+        log.info(out);
+
+        String id = context.getName();
+        assertTrue(out.contains("{\"name\": \"" + id + "\"}"));
+
+        out = template.requestBody("netty4-http:http://localhost:{{port}}/api-doc/" + id, null, String.class);
+        assertNotNull(out);
+        log.info(out);
+
+        assertTrue(out.contains("\"/hello/bye/{name}\""));
+        assertTrue(out.contains("\"/hello/hi/{name}\""));
+        assertTrue(out.contains("\"summary\" : \"To update the greeting message\""));
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                restConfiguration().component("netty4-http").host("localhost").port(getPort()).apiContextPath("/api-doc");
+
+                rest("/hello").consumes("application/json").produces("application/json")
+                    .get("/hi/{name}").description("Saying hi")
+                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
+                        .to("log:hi")
+                    .get("/bye/{name}").description("Saying bye")
+                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
+                        .responseMessage().code(200).message("A reply message").endResponseMessage()
+                        .to("log:bye")
+                    .post("/bye").description("To update the greeting message").consumes("application/xml").produces("application/xml")
+                        .param().name("greeting").type(RestParamType.body).dataType("string").description("Message to use as greeting").endParam()
+                        .to("log:bye");
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/818557e5/components/camel-swagger-java/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/pom.xml b/components/camel-swagger-java/pom.xml
index 33072cf..742a9e5 100644
--- a/components/camel-swagger-java/pom.xml
+++ b/components/camel-swagger-java/pom.xml
@@ -114,11 +114,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.camel</groupId>
-      <artifactId>camel-netty4-http</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/camel/blob/818557e5/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java b/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
deleted file mode 100644
index 6379515..0000000
--- a/components/camel-swagger-java/src/test/java/org/apache/camel/swagger/RestApiNettyTest.java
+++ /dev/null
@@ -1,65 +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.swagger;
-
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.impl.JndiRegistry;
-import org.apache.camel.model.rest.RestParamType;
-import org.apache.camel.test.junit4.CamelTestSupport;
-import org.junit.Test;
-
-public class RestApiNettyTest extends CamelTestSupport {
-
-    @Override
-    protected JndiRegistry createRegistry() throws Exception {
-        JndiRegistry jndi = super.createRegistry();
-        jndi.bind("SwaggerRestApiProcessorFactory", new SwaggerRestApiProcessorFactory());
-        return jndi;
-    }
-
-    @Override
-    protected boolean useJmx() {
-        return true;
-    }
-
-    @Test
-    public void testApi() throws Exception {
-        Thread.sleep(999999);
-    }
-
-    @Override
-    protected RouteBuilder createRouteBuilder() throws Exception {
-        return new RouteBuilder() {
-            @Override
-            public void configure() throws Exception {
-                restConfiguration().component("netty4-http").host("localhost").port(8080).apiContextPath("/api-doc");
-
-                rest("/hello").consumes("application/json").produces("application/json")
-                    .get("/hi/{name}").description("Saying hi")
-                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
-                        .to("log:hi")
-                    .get("/bye/{name}").description("Saying bye")
-                        .param().name("name").type(RestParamType.path).dataType("string").description("Who is it").endParam()
-                        .responseMessage().code(200).message("A reply message").endResponseMessage()
-                        .to("log:bye")
-                    .post("/bye").description("To update the greeting message").consumes("application/xml").produces("application/xml")
-                        .param().name("greeting").type(RestParamType.body).dataType("string").description("Message to use as greeting").endParam()
-                        .to("log:bye");
-            }
-        };
-    }
-}


[14/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: f3aed3eb1ea9fce9dff9adf70ba6a5e4fd08986e
Parents: d975acd
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 11:17:41 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 11:17:41 2015 +0200

----------------------------------------------------------------------
 .../camel/component/rest/RestApiEndpoint.java   | 17 +++++++--
 .../model/rest/RestConfigurationDefinition.java | 37 ++++++++++++++++++++
 .../apache/camel/model/rest/RestDefinition.java |  3 ++
 .../camel/spi/RestApiProcessorFactory.java      | 10 +++---
 .../org/apache/camel/spi/RestConfiguration.java | 17 +++++++++
 .../rest/DummyRestProcessorFactory.java         |  3 +-
 .../camel/swagger/RestSwaggerProcessor.java     | 26 +++++++++++---
 .../camel/swagger/RestSwaggerSupport.java       | 15 +++++++-
 .../swagger/SwaggerRestApiProcessorFactory.java |  4 +--
 .../swagger/servlet/RestSwaggerServlet.java     |  2 +-
 .../camel/example/cdi/UserRouteBuilder.java     |  6 ++--
 11 files changed, 120 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
index e6df668..a5e7276 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestApiEndpoint.java
@@ -33,7 +33,7 @@ import org.apache.camel.spi.UriEndpoint;
 import org.apache.camel.spi.UriParam;
 import org.apache.camel.spi.UriPath;
 
-@UriEndpoint(scheme = "rest-api", title = "REST API", syntax = "rest-api:path", consumerOnly = true, label = "core,rest")
+@UriEndpoint(scheme = "rest-api", title = "REST API", syntax = "rest-api:path/contextId", consumerOnly = true, label = "core,rest")
 public class RestApiEndpoint extends DefaultEndpoint {
 
     public static final String DEFAULT_API_COMPONENT_NAME = "swagger";
@@ -41,6 +41,8 @@ public class RestApiEndpoint extends DefaultEndpoint {
 
     @UriPath @Metadata(required = "true")
     private String path;
+    @UriPath
+    private String contextIdPattern;
     @UriParam
     private String componentName;
     @UriParam
@@ -68,6 +70,17 @@ public class RestApiEndpoint extends DefaultEndpoint {
         this.path = path;
     }
 
+    public String getContextIdPattern() {
+        return contextIdPattern;
+    }
+
+    /**
+     * Optional CamelContext id pattern to only allow Rest APIs from rest services within CamelContext's which name matches the pattern.
+     */
+    public void setContextIdPattern(String contextIdPattern) {
+        this.contextIdPattern = contextIdPattern;
+    }
+
     public String getComponentName() {
         return componentName;
     }
@@ -134,7 +147,7 @@ public class RestApiEndpoint extends DefaultEndpoint {
                 path = "/" + path;
             }
 
-            Processor processor = factory.createApiProcessor(getCamelContext(), path, config, getParameters());
+            Processor processor = factory.createApiProcessor(getCamelContext(), path, getContextIdPattern(), config, getParameters());
             return new RestApiProducer(this, processor);
         } else {
             throw new IllegalStateException("Cannot find RestApiProcessorFactory in Registry or classpath");

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
index 90a6138..0467f21 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestConfigurationDefinition.java
@@ -61,6 +61,9 @@ public class RestConfigurationDefinition {
     private String apiContextPath;
 
     @XmlAttribute
+    private String apiContextIdPattern;
+
+    @XmlAttribute
     private RestHostNameResolver hostNameResolver;
 
     @XmlAttribute @Metadata(defaultValue = "auto")
@@ -190,6 +193,22 @@ public class RestConfigurationDefinition {
         this.apiContextPath = contextPath;
     }
 
+    public String getApiContextIdPattern() {
+        return apiContextIdPattern;
+    }
+
+    /**
+     * Sets an CamelContext id pattern to only allow Rest APIs from rest services within CamelContext's which name matches the pattern.
+     * <p/>
+     * The pattern <tt>#name#</tt> refers to the CamelContext name, to match on the current CamelContext only.
+     * For any other value, the pattern uses the rules from {@link org.apache.camel.util.EndpointHelper#matchPattern(String, String)}
+     *
+     * @param apiContextIdPattern  the pattern
+     */
+    public void setApiContextIdPattern(String apiContextIdPattern) {
+        this.apiContextIdPattern = apiContextIdPattern;
+    }
+
     public RestHostNameResolver getHostNameResolver() {
         return hostNameResolver;
     }
@@ -406,6 +425,16 @@ public class RestConfigurationDefinition {
     }
 
     /**
+     * Sets an CamelContext id pattern to only allow Rest APIs from rest services within CamelContext's which name matches the pattern.
+     * <p/>
+     * The pattern uses the rules from {@link org.apache.camel.util.EndpointHelper#matchPattern(String, String)}
+     */
+    public RestConfigurationDefinition apiContextIdPattern(String pattern) {
+        setApiContextIdPattern(pattern);
+        return this;
+    }
+
+    /**
      * Sets a leading context-path the REST services will be using.
      * <p/>
      * This can be used when using components such as <tt>camel-servlet</tt> where the deployed web application
@@ -568,6 +597,14 @@ public class RestConfigurationDefinition {
         if (apiContextPath != null) {
             answer.setApiContextPath(CamelContextHelper.parseText(context, apiContextPath));
         }
+        if (apiContextIdPattern != null) {
+            // special to allow #name# to refer to itself
+            if ("#name#".equals(apiComponent)) {
+                answer.setApiContextIdPattern(context.getName());
+            } else {
+                answer.setApiContextIdPattern(CamelContextHelper.parseText(context, apiContextIdPattern));
+            }
+        }
         if (contextPath != null) {
             answer.setContextPath(CamelContextHelper.parseText(context, contextPath));
         }

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
index c004af7..e2b1895 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestDefinition.java
@@ -554,6 +554,9 @@ public class RestDefinition extends OptionalIdentifiedDefinition<RestDefinition>
         if (configuration.getComponent() != null && !configuration.getComponent().isEmpty()) {
             options.put("componentName", configuration.getComponent());
         }
+        if (configuration.getApiContextIdPattern() != null) {
+            options.put("contextIdPattern", configuration.getApiContextIdPattern());
+        }
 
         if (!options.isEmpty()) {
             String query;

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
index a721ec5..8f5210d 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestApiProcessorFactory.java
@@ -28,14 +28,14 @@ public interface RestApiProcessorFactory {
      * href="http://camel.apache.org/processor.html">Processor
      * </a>, which provides API listing of the REST services
      *
-     * @param camelContext the camel context
-     * @param contextPath  the context-path
-     * @param parameters   additional parameters
-     *
+     * @param camelContext      the camel context
+     * @param contextPath       the context-path
+     * @param contextIdPattern  id pattern to only allow Rest APIs from rest services within CamelContext's which name matches the pattern.
+     * @param parameters        additional parameters
      * @return a newly created REST API provider
      * @throws Exception can be thrown
      */
-    Processor createApiProcessor(CamelContext camelContext, String contextPath,
+    Processor createApiProcessor(CamelContext camelContext, String contextPath, String contextIdPattern,
                                  RestConfiguration configuration, Map<String, Object> parameters) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
index 55a45d6..c8cbcff 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/RestConfiguration.java
@@ -44,6 +44,7 @@ public class RestConfiguration {
     private int port;
     private String contextPath;
     private String apiContextPath;
+    private String apiContextIdPattern;
     private RestHostNameResolver restHostNameResolver = RestHostNameResolver.localHostName;
     private RestBindingMode bindingMode = RestBindingMode.off;
     private boolean skipBindingOnErrorCode = true;
@@ -184,6 +185,22 @@ public class RestConfiguration {
         this.apiContextPath = contextPath;
     }
 
+    public String getApiContextIdPattern() {
+        return apiContextIdPattern;
+    }
+
+    /**
+     * Optional CamelContext id pattern to only allow Rest APIs from rest services within CamelContext's which name matches the pattern.
+     * <p/>
+     * The pattern <tt>#name#</tt> refers to the CamelContext name, to match on the current CamelContext only.
+     * For any other value, the pattern uses the rules from {@link org.apache.camel.util.EndpointHelper#matchPattern(String, String)}
+     *
+     * @param apiContextIdPattern  the pattern
+     */
+    public void setApiContextIdPattern(String apiContextIdPattern) {
+        this.apiContextIdPattern = apiContextIdPattern;
+    }
+
     /**
      * Gets the resolver to use for resolving hostname
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
index 1fbe87f..f2d2948 100644
--- a/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/DummyRestProcessorFactory.java
@@ -27,7 +27,8 @@ import org.apache.camel.spi.RestConfiguration;
 public class DummyRestProcessorFactory implements RestApiProcessorFactory {
 
     @Override
-    public Processor createApiProcessor(CamelContext camelContext, String contextPath, RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath, String contextIdPattern,
+                                        RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
         return new Processor() {
             @Override
             public void process(Exchange exchange) throws Exception {

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
index f2752a6..58d3205 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import io.swagger.jaxrs.config.BeanConfig;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
+import org.apache.camel.util.EndpointHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -30,11 +31,14 @@ public class RestSwaggerProcessor implements Processor {
     private static final Logger LOG = LoggerFactory.getLogger(RestSwaggerProcessor.class);
     private final BeanConfig swaggerConfig;
     private final RestSwaggerSupport support;
+    private final String contextIdPattern;
 
     @SuppressWarnings("unchecked")
-    public RestSwaggerProcessor(Map<String, Object> parameters) {
-        support = new RestSwaggerSupport();
-        swaggerConfig = new BeanConfig();
+    public RestSwaggerProcessor(String contextIdPattern, Map<String, Object> parameters) {
+        this.contextIdPattern = contextIdPattern;
+        this.support = new RestSwaggerSupport();
+        this.swaggerConfig = new BeanConfig();
+
         if (parameters == null) {
             parameters = Collections.EMPTY_MAP;
         }
@@ -52,7 +56,7 @@ public class RestSwaggerProcessor implements Processor {
         try {
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
-                support.renderCamelContexts(adapter);
+                support.renderCamelContexts(adapter, contextIdPattern);
             } else {
                 // first part is the camel context
                 if (route.startsWith("/")) {
@@ -64,7 +68,19 @@ public class RestSwaggerProcessor implements Processor {
                     route = route.substring(contextId.length());
                 }
 
-                support.renderResourceListing(adapter, swaggerConfig, contextId, route);
+                boolean match = true;
+                if (contextIdPattern != null) {
+                    match = EndpointHelper.matchPattern(contextId, contextIdPattern);
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Match contextId: {} with pattern: {} -> {}", new Object[]{contextId, contextIdPattern, match});
+                    }
+                }
+
+                if (!match) {
+                    adapter.noContent();
+                } else {
+                    support.renderResourceListing(adapter, swaggerConfig, contextId, route);
+                }
             }
         } catch (Exception e) {
             LOG.warn("Error rendering Swagger API due " + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index 4e228a8..5e9513d 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -18,6 +18,7 @@ package org.apache.camel.swagger;
 
 import java.lang.management.ManagementFactory;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -38,6 +39,7 @@ import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.rest.RestDefinition;
 import org.apache.camel.model.rest.RestsDefinition;
 import org.apache.camel.util.CamelVersionHelper;
+import org.apache.camel.util.EndpointHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -196,7 +198,7 @@ public class RestSwaggerSupport {
     /**
      * Renders a list of available CamelContexts in the JVM
      */
-    public void renderCamelContexts(RestApiResponseAdapter response) throws Exception {
+    public void renderCamelContexts(RestApiResponseAdapter response, String contextIdPattern) throws Exception {
         LOG.trace("renderCamelContexts");
 
         if (cors) {
@@ -210,6 +212,17 @@ public class RestSwaggerSupport {
         StringBuffer sb = new StringBuffer();
 
         List<String> contexts = findCamelContexts();
+
+        // filter non matched CamelContext's
+        Iterator<String> it = contexts.iterator();
+        while (it.hasNext()) {
+            String name = it.next();
+            boolean match = EndpointHelper.matchPattern(name, contextIdPattern);
+            if (!match) {
+                it.remove();
+            }
+        }
+
         sb.append("[\n");
         for (int i = 0; i < contexts.size(); i++) {
             String name = contexts.get(i);

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
index faba694..be70c60 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/SwaggerRestApiProcessorFactory.java
@@ -26,8 +26,8 @@ import org.apache.camel.spi.RestConfiguration;
 public class SwaggerRestApiProcessorFactory implements RestApiProcessorFactory {
 
     @Override
-    public Processor createApiProcessor(CamelContext camelContext, String contextPath,
+    public Processor createApiProcessor(CamelContext camelContext, String contextPath, String contextIdPattern,
                                         RestConfiguration configuration, Map<String, Object> parameters) throws Exception {
-        return new RestSwaggerProcessor(configuration.getApiProperties());
+        return new RestSwaggerProcessor(contextIdPattern, configuration.getApiProperties());
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
index a5ca00a..a1381ce 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
@@ -77,7 +77,7 @@ public class RestSwaggerServlet extends HttpServlet {
 
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
-                swagger.renderCamelContexts(adapter);
+                swagger.renderCamelContexts(adapter, null);
             } else {
                 // first part is the camel context
                 if (route.startsWith("/")) {

http://git-wip-us.apache.org/repos/asf/camel/blob/f3aed3eb/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
index 6061ad0..c7657d9 100644
--- a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
+++ b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
@@ -26,7 +26,7 @@ import static org.apache.camel.model.rest.RestParamType.path;
 /**
  * Define REST services using the Camel REST DSL
  */
-@ContextName
+@ContextName("myCamel")
 public class UserRouteBuilder extends RouteBuilder {
 
     @Override
@@ -39,8 +39,8 @@ public class UserRouteBuilder extends RouteBuilder {
             .dataFormatProperty("prettyPrint", "true")
             // setup context path and port number that netty will use
             .contextPath("/rest").port(8080)
-            // add swagger api-doc out of the box
-            .apiContextPath("/api-doc")
+            // add swagger api-doc out of the box, and only allow to docs for this CamelContext
+            .apiContextPath("/api-doc").apiContextIdPattern("#name#")
                 .apiProperty("api.title", "User API").apiProperty("api.version", "1.2.3")
                 // and enable CORS
                 .apiProperty("cors", "true");


[15/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 17ef02fe0ac73cff207d2576dee6082ed5b03b04
Parents: f3aed3e
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Sep 23 11:28:02 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 11:30:48 2015 +0200

----------------------------------------------------------------------
 .../camel/swagger/RestSwaggerProcessor.java     | 18 +++++++++-------
 .../camel/swagger/RestSwaggerSupport.java       | 22 +++++++++++++-------
 .../swagger/servlet/RestSwaggerServlet.java     |  2 +-
 .../camel/example/cdi/UserRouteBuilder.java     |  2 +-
 4 files changed, 28 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/17ef02fe/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
index 58d3205..a83fe25 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -48,7 +48,7 @@ public class RestSwaggerProcessor implements Processor {
     @Override
     public void process(Exchange exchange) throws Exception {
 
-        String contextId;
+        String contextId = exchange.getContext().getName();
         String route = exchange.getIn().getHeader(Exchange.HTTP_PATH, String.class);
 
         RestApiResponseAdapter adapter = new ExchangeRestApiResponseAdapter(exchange);
@@ -56,30 +56,34 @@ public class RestSwaggerProcessor implements Processor {
         try {
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
-                support.renderCamelContexts(adapter, contextIdPattern);
+                support.renderCamelContexts(adapter, contextId, contextIdPattern);
             } else {
                 // first part is the camel context
                 if (route.startsWith("/")) {
                     route = route.substring(1);
                 }
                 // the remainder is the route part
-                contextId = route.split("/")[0];
+                String name = route.split("/")[0];
                 if (route.startsWith(contextId)) {
-                    route = route.substring(contextId.length());
+                    route = route.substring(name.length());
                 }
 
                 boolean match = true;
                 if (contextIdPattern != null) {
-                    match = EndpointHelper.matchPattern(contextId, contextIdPattern);
+                    if ("#name#".equals(contextIdPattern)) {
+                        match = name.equals(contextId);
+                    } else {
+                        match = EndpointHelper.matchPattern(name, contextIdPattern);
+                    }
                     if (LOG.isDebugEnabled()) {
-                        LOG.debug("Match contextId: {} with pattern: {} -> {}", new Object[]{contextId, contextIdPattern, match});
+                        LOG.debug("Match contextId: {} with pattern: {} -> {}", new Object[]{name, contextIdPattern, match});
                     }
                 }
 
                 if (!match) {
                     adapter.noContent();
                 } else {
-                    support.renderResourceListing(adapter, swaggerConfig, contextId, route);
+                    support.renderResourceListing(adapter, swaggerConfig, name, route);
                 }
             }
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/camel/blob/17ef02fe/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
index 5e9513d..b3d33c3 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerSupport.java
@@ -198,7 +198,7 @@ public class RestSwaggerSupport {
     /**
      * Renders a list of available CamelContexts in the JVM
      */
-    public void renderCamelContexts(RestApiResponseAdapter response, String contextIdPattern) throws Exception {
+    public void renderCamelContexts(RestApiResponseAdapter response, String contextId, String contextIdPattern) throws Exception {
         LOG.trace("renderCamelContexts");
 
         if (cors) {
@@ -214,12 +214,20 @@ public class RestSwaggerSupport {
         List<String> contexts = findCamelContexts();
 
         // filter non matched CamelContext's
-        Iterator<String> it = contexts.iterator();
-        while (it.hasNext()) {
-            String name = it.next();
-            boolean match = EndpointHelper.matchPattern(name, contextIdPattern);
-            if (!match) {
-                it.remove();
+        if (contextIdPattern != null) {
+            Iterator<String> it = contexts.iterator();
+            while (it.hasNext()) {
+                String name = it.next();
+
+                boolean match;
+                if ("#name#".equals(contextIdPattern)) {
+                    match = name.equals(contextId);
+                } else {
+                    match = EndpointHelper.matchPattern(name, contextIdPattern);
+                }
+                if (!match) {
+                    it.remove();
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/17ef02fe/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
index a1381ce..345efc9 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/servlet/RestSwaggerServlet.java
@@ -77,7 +77,7 @@ public class RestSwaggerServlet extends HttpServlet {
 
             // render list of camel contexts as root
             if (route == null || route.equals("") || route.equals("/")) {
-                swagger.renderCamelContexts(adapter, null);
+                swagger.renderCamelContexts(adapter, null, null);
             } else {
                 // first part is the camel context
                 if (route.startsWith("/")) {

http://git-wip-us.apache.org/repos/asf/camel/blob/17ef02fe/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
index c7657d9..f037444 100644
--- a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
+++ b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
@@ -39,7 +39,7 @@ public class UserRouteBuilder extends RouteBuilder {
             .dataFormatProperty("prettyPrint", "true")
             // setup context path and port number that netty will use
             .contextPath("/rest").port(8080)
-            // add swagger api-doc out of the box, and only allow to docs for this CamelContext
+            // add swagger api-doc out of the box, and only allow the docs for this CamelContext (#name#)
             .apiContextPath("/api-doc").apiContextIdPattern("#name#")
                 .apiProperty("api.title", "User API").apiProperty("api.version", "1.2.3")
                 // and enable CORS


[08/16] camel git commit: CAMEL-8545: camel-swagger-java to run outside servlet - work in progress

Posted by da...@apache.org.
CAMEL-8545: camel-swagger-java to run outside servlet - work in progress


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

Branch: refs/heads/master
Commit: 9bd9e3e57855ba838ef6c0f9b9e7a2bd9c0a07cf
Parents: 26ae772
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Sep 22 21:30:14 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Sep 23 07:51:05 2015 +0200

----------------------------------------------------------------------
 .../camel/swagger/RestSwaggerProcessor.java     |   5 +
 examples/camel-example-swagger-cdi/pom.xml      | 108 ++++++++++
 .../java/org/apache/camel/example/cdi/User.java |  53 +++++
 .../camel/example/cdi/UserRouteBuilder.java     |  64 ++++++
 .../apache/camel/example/cdi/UserService.java   |  76 +++++++
 .../src/main/resources/META-INF/LICENSE.txt     | 203 +++++++++++++++++++
 .../src/main/resources/META-INF/NOTICE.txt      |  11 +
 .../src/main/resources/META-INF/beans.xml       |  18 ++
 .../src/main/resources/log4j.properties         |  29 +++
 examples/pom.xml                                |   1 +
 10 files changed, 568 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
----------------------------------------------------------------------
diff --git a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
index 9380d91..f2752a6 100644
--- a/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
+++ b/components/camel-swagger-java/src/main/java/org/apache/camel/swagger/RestSwaggerProcessor.java
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.swagger;
 
+import java.util.Collections;
 import java.util.Map;
 
 import io.swagger.jaxrs.config.BeanConfig;
@@ -30,9 +31,13 @@ public class RestSwaggerProcessor implements Processor {
     private final BeanConfig swaggerConfig;
     private final RestSwaggerSupport support;
 
+    @SuppressWarnings("unchecked")
     public RestSwaggerProcessor(Map<String, Object> parameters) {
         support = new RestSwaggerSupport();
         swaggerConfig = new BeanConfig();
+        if (parameters == null) {
+            parameters = Collections.EMPTY_MAP;
+        }
         support.initSwagger(swaggerConfig, parameters);
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/pom.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/pom.xml b/examples/camel-example-swagger-cdi/pom.xml
new file mode 100644
index 0000000..4163d79
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements. See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version
+    2.0 (the "License"); you may not use this file except in compliance
+    with the License. You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0 Unless required by
+    applicable law or agreed to in writing, software distributed under the
+    License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
+    CONDITIONS OF ANY KIND, either express or implied. See the License for
+    the specific language governing permissions and limitations under the
+    License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>examples</artifactId>
+    <version>2.16-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>camel-example-swagger-cdi</artifactId>
+  <packaging>jar</packaging>
+  <name>Camel :: Example :: Swagger CDI</name>
+  <description>An example using REST DSL and Swagger Java with CDI</description>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-cdi</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-swagger-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-netty4-http</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-jackson</artifactId>
+    </dependency>
+
+    <!-- cdi api -->
+    <dependency>
+      <groupId>javax.enterprise</groupId>
+      <artifactId>cdi-api</artifactId>
+      <version>1.2</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- need to use a CDI container such as JBoss Weld -->
+    <!-- lets use weld for CDI -->
+    <dependency>
+      <groupId>org.jboss.weld.se</groupId>
+      <artifactId>weld-se</artifactId>
+      <version>${weld2-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jboss.weld</groupId>
+      <artifactId>weld-core</artifactId>
+      <version>${weld2-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.deltaspike.cdictrl</groupId>
+      <artifactId>deltaspike-cdictrl-weld</artifactId>
+      <version>${deltaspike-version}</version>
+    </dependency>
+
+    <!-- use log4j as logger -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+
+    <plugins>
+      <!-- allows the routes to be run via 'mvn camel:run' -->
+      <plugin>
+        <groupId>org.apache.camel</groupId>
+        <artifactId>camel-maven-plugin</artifactId>
+        <version>${project.version}</version>
+        <configuration>
+          <useCDI>true</useCDI>
+        </configuration>
+      </plugin>
+    </plugins>
+
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/User.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/User.java b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/User.java
new file mode 100644
index 0000000..b347b37
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/User.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.example.cdi;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+@ApiModel(description = "Represents an user of the system")
+public class User {
+
+    private int id;
+    private String name;
+
+    public User() {
+    }
+
+    public User(int id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    @ApiModelProperty(value = "The id of the user", required = true)
+    public int getId() {
+        return id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    @ApiModelProperty(value = "The name of the user", required = true)
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
new file mode 100644
index 0000000..6061ad0
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserRouteBuilder.java
@@ -0,0 +1,64 @@
+/**
+ * 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.example.cdi;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.cdi.ContextName;
+import org.apache.camel.model.rest.RestBindingMode;
+
+import static org.apache.camel.model.rest.RestParamType.body;
+import static org.apache.camel.model.rest.RestParamType.path;
+
+/**
+ * Define REST services using the Camel REST DSL
+ */
+@ContextName
+public class UserRouteBuilder extends RouteBuilder {
+
+    @Override
+    public void configure() throws Exception {
+
+        // configure we want to use servlet as the component for the rest DSL
+        // and we enable json binding mode
+        restConfiguration().component("netty4-http").bindingMode(RestBindingMode.json)
+            // and output using pretty print
+            .dataFormatProperty("prettyPrint", "true")
+            // setup context path and port number that netty will use
+            .contextPath("/rest").port(8080)
+            // add swagger api-doc out of the box
+            .apiContextPath("/api-doc")
+                .apiProperty("api.title", "User API").apiProperty("api.version", "1.2.3")
+                // and enable CORS
+                .apiProperty("cors", "true");
+
+        // this user REST service is json only
+        rest("/user").description("User rest service")
+            .consumes("application/json").produces("application/json")
+
+            .get("/{id}").description("Find user by id").outType(User.class)
+                .param().name("id").type(path).description("The id of the user to get").dataType("int").endParam()
+                .to("bean:userService?method=getUser(${header.id})")
+
+            .put().description("Updates or create a user").type(User.class)
+                .param().name("body").type(body).description("The user to update or create").endParam()
+                .to("bean:userService?method=updateUser")
+
+            .get("/findAll").description("Find all users").outTypeList(User.class)
+                .to("bean:userService?method=listUsers");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserService.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserService.java b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserService.java
new file mode 100644
index 0000000..46bb240
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/java/org/apache/camel/example/cdi/UserService.java
@@ -0,0 +1,76 @@
+/**
+ * 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.example.cdi;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+
+/**
+ * A {@link org.apache.camel.example.cdi.User} service which we rest enable the routes defined in the XML file.
+ */
+public class UserService {
+
+    // use a tree map so they become sorted
+    private final Map<String, User> users = new TreeMap<String, User>();
+
+    private Random ran = new Random();
+
+    public UserService() {
+        users.put("123", new User(123, "John Doe"));
+        users.put("456", new User(456, "Donald Duck"));
+        users.put("789", new User(789, "Slow Turtle"));
+    }
+
+    /**
+     * Gets a user by the given id
+     *
+     * @param id  the id of the user
+     * @return the user, or <tt>null</tt> if no user exists
+     */
+    public User getUser(String id) {
+        if ("789".equals(id)) {
+            // simulate some cpu processing time when returning the slow turtle
+            int delay = 500 + ran.nextInt(1500);
+            try {
+                Thread.sleep(delay);
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+        return users.get(id);
+    }
+
+    /**
+     * List all users
+     *
+     * @return the list of all users
+     */
+    public Collection<User> listUsers() {
+        return users.values();
+    }
+
+    /**
+     * Updates or creates the given user
+     *
+     * @param user the user
+     */
+    public void updateUser(User user) {
+        users.put("" + user.getId(), user);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/resources/META-INF/LICENSE.txt
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/resources/META-INF/LICENSE.txt b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/LICENSE.txt
new file mode 100644
index 0000000..6b0b127
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/LICENSE.txt
@@ -0,0 +1,203 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/resources/META-INF/NOTICE.txt
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/resources/META-INF/NOTICE.txt b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/NOTICE.txt
new file mode 100644
index 0000000..2e215bf
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/NOTICE.txt
@@ -0,0 +1,11 @@
+   =========================================================================
+   ==  NOTICE file corresponding to the section 4 d of                    ==
+   ==  the Apache License, Version 2.0,                                   ==
+   ==  in this case for the Apache Camel distribution.                    ==
+   =========================================================================
+
+   This product includes software developed by
+   The Apache Software Foundation (http://www.apache.org/).
+
+   Please read the different LICENSE files present in the licenses directory of
+   this distribution.

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/resources/META-INF/beans.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/resources/META-INF/beans.xml b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/beans.xml
new file mode 100644
index 0000000..112d56d
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/resources/META-INF/beans.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<beans/>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/camel-example-swagger-cdi/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/examples/camel-example-swagger-cdi/src/main/resources/log4j.properties b/examples/camel-example-swagger-cdi/src/main/resources/log4j.properties
new file mode 100644
index 0000000..0057e0d
--- /dev/null
+++ b/examples/camel-example-swagger-cdi/src/main/resources/log4j.properties
@@ -0,0 +1,29 @@
+## ------------------------------------------------------------------------
+## 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.
+## ------------------------------------------------------------------------
+
+#
+# The logging properties used
+#
+log4j.rootLogger=INFO, stdout
+
+# uncomment the next line to debug Camel
+#log4j.logger.org.apache.camel=DEBUG
+
+# CONSOLE appender not used by default
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n

http://git-wip-us.apache.org/repos/asf/camel/blob/9bd9e3e5/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 1d41204..b684890 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -82,6 +82,7 @@
     <module>camel-example-ssh-security</module>
     <module>camel-example-sql</module>
     <module>camel-example-sql-blueprint</module>
+    <module>camel-example-swagger-cdi</module>
     <module>camel-example-swagger-java</module>
     <module>camel-example-tracer</module>
     <module>camel-example-twitter-websocket</module>