You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by lb...@apache.org on 2020/03/24 16:16:15 UTC

[camel-k-runtime] 01/07: create platform http service #265

This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 15e692a0f642f230ba6a8af3b680f162c9608aad
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Wed Mar 18 15:28:09 2020 +0100

    create platform http service #265
---
 camel-k-main/camel-k-runtime-health/pom.xml        |   2 +-
 .../camel/k/health/HealthContextCustomizer.java    |  17 +-
 .../camel/k/health/HealthCustomizerTest.java       |  20 +-
 .../pom.xml                                        |   6 +-
 .../apache/camel/k/http/PlatformHttpRouter.java    |  50 +++
 .../apache/camel/k/http/PlatformHttpServer.java    | 178 ++++++++++
 .../k/http/PlatformHttpServiceConfiguration.java   | 131 +++++++
 .../PlatformHttpServiceContextCustomizer.java}     |  56 +--
 .../camel/k/http/PlatformHttpServiceEndpoint.java  | 118 +++++++
 .../k/http/engine/RuntimePlatformHttpConsumer.java | 389 +++++++++++++++++++++
 .../k/http/engine/RuntimePlatformHttpEngine.java}  |  20 +-
 .../org/apache/camel/k/customizer/platform-http}   |   2 +-
 .../k/http/PlatformHttpServiceCustomizerTest.java  | 145 ++++++++
 .../src/test/resources/log4j2-test.xml             |   0
 .../camel/k/inspector/InspectorEndpoint.java       | 220 ------------
 .../camel/k/inspector/InspectorCustomizerTest.java |  74 ----
 camel-k-main/pom.xml                               |   2 +-
 camel-k-runtime-bom/pom.xml                        |   2 +-
 .../camel-k-runtime-example-health/README.adoc     |  25 --
 examples/camel-k-runtime-example-rest/README.adoc  |  58 +++
 .../data}/application.properties                   |   4 +-
 .../data}/routes.groovy                            |  17 +-
 .../pom.xml                                        |  18 +-
 examples/pom.xml                                   |   2 +-
 pom.xml                                            |   2 +-
 .../src/it/generate-catalog-main/verify.groovy     |   4 +-
 .../camel/k/tooling/maven/GenerateCatalogMojo.java |   2 +-
 .../maven/processors/CatalogProcessor3x.java       |   2 +-
 .../maven/processors/CatalogProcessor3Test.java    |   2 +-
 29 files changed, 1157 insertions(+), 411 deletions(-)

diff --git a/camel-k-main/camel-k-runtime-health/pom.xml b/camel-k-main/camel-k-runtime-health/pom.xml
index d203545..07624b3 100644
--- a/camel-k-main/camel-k-runtime-health/pom.xml
+++ b/camel-k-main/camel-k-runtime-health/pom.xml
@@ -45,7 +45,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.camel.k</groupId>
-            <artifactId>camel-k-runtime-inspector</artifactId>
+            <artifactId>camel-k-runtime-http</artifactId>
         </dependency>
 
         <dependency>
diff --git a/camel-k-main/camel-k-runtime-health/src/main/java/org/apache/camel/k/health/HealthContextCustomizer.java b/camel-k-main/camel-k-runtime-health/src/main/java/org/apache/camel/k/health/HealthContextCustomizer.java
index 8d70f10..f1308d2 100644
--- a/camel-k-main/camel-k-runtime-health/src/main/java/org/apache/camel/k/health/HealthContextCustomizer.java
+++ b/camel-k-main/camel-k-runtime-health/src/main/java/org/apache/camel/k/health/HealthContextCustomizer.java
@@ -23,6 +23,8 @@ import io.vertx.core.http.HttpMethod;
 import io.vertx.core.json.Json;
 import io.vertx.core.json.JsonArray;
 import io.vertx.core.json.JsonObject;
+import io.vertx.ext.web.Route;
+import io.vertx.ext.web.Router;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Ordered;
 import org.apache.camel.health.HealthCheck;
@@ -33,7 +35,7 @@ import org.apache.camel.impl.health.AbstractHealthCheck;
 import org.apache.camel.impl.health.ContextHealthCheck;
 import org.apache.camel.impl.health.RoutesHealthCheckRepository;
 import org.apache.camel.k.ContextCustomizer;
-import org.apache.camel.k.inspector.InspectorCustomizer;
+import org.apache.camel.k.http.PlatformHttpRouter;
 
 public class HealthContextCustomizer implements ContextCustomizer {
     public static final String DEFAULT_PATH = "/health";
@@ -83,7 +85,7 @@ public class HealthContextCustomizer implements ContextCustomizer {
 
     @Override
     public int getOrder() {
-        return Ordered.HIGHEST;
+        return Ordered.LOWEST;
     }
 
     @Override
@@ -105,14 +107,15 @@ public class HealthContextCustomizer implements ContextCustomizer {
             throw new RuntimeException(e);
         }
 
-        camelContext.getRegistry().bind(
-            "health-route",
-            customizer(camelContext)
+        // add health route
+        addRoute(
+            camelContext,
+            PlatformHttpRouter.lookup(camelContext).get()
         );
     }
 
-    private InspectorCustomizer customizer(CamelContext camelContext) {
-        return router -> router.route(HttpMethod.GET, path).handler(routingContext -> {
+    private Route addRoute(CamelContext camelContext, Router router) {
+        return router.route(HttpMethod.GET, path).handler(routingContext -> {
             int code = 200;
 
             Collection<HealthCheck.Result> results = HealthCheckHelper.invoke(
diff --git a/camel-k-main/camel-k-runtime-health/src/test/java/org/apache/camel/k/health/HealthCustomizerTest.java b/camel-k-main/camel-k-runtime-health/src/test/java/org/apache/camel/k/health/HealthCustomizerTest.java
index 1a7fb06..caf512e 100644
--- a/camel-k-main/camel-k-runtime-health/src/test/java/org/apache/camel/k/health/HealthCustomizerTest.java
+++ b/camel-k-main/camel-k-runtime-health/src/test/java/org/apache/camel/k/health/HealthCustomizerTest.java
@@ -21,7 +21,7 @@ import java.net.URL;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.k.Runtime;
-import org.apache.camel.k.inspector.InspectorContextCustomizer;
+import org.apache.camel.k.http.PlatformHttpServiceContextCustomizer;
 import org.apache.camel.test.AvailablePortFinder;
 import org.apache.camel.util.ObjectHelper;
 import org.junit.jupiter.params.ParameterizedTest;
@@ -46,22 +46,22 @@ public class HealthCustomizerTest {
             }
         });
 
-        HealthContextCustomizer healthCustomizer = new HealthContextCustomizer();
-        healthCustomizer.apply(runtime.getCamelContext());
-
-        InspectorContextCustomizer inspectorCustomizer = new InspectorContextCustomizer();
-        inspectorCustomizer.setBindPort(AvailablePortFinder.getNextAvailable());
+        PlatformHttpServiceContextCustomizer phsc = new PlatformHttpServiceContextCustomizer();
+        phsc.setBindPort(AvailablePortFinder.getNextAvailable());
 
         String url;
         if (ObjectHelper.isEmpty(path)) {
-            url = "http://localhost:" + inspectorCustomizer.getBindPort() + HealthContextCustomizer.DEFAULT_PATH;
+            url = "http://localhost:" + phsc.getBindPort() + HealthContextCustomizer.DEFAULT_PATH;
         } else {
-            inspectorCustomizer.setPath(path);
+            phsc.setPath(path);
 
-            url = "http://localhost:" + inspectorCustomizer.getBindPort() + path + HealthContextCustomizer.DEFAULT_PATH;
+            url = "http://localhost:" + phsc.getBindPort() + path + HealthContextCustomizer.DEFAULT_PATH;
         }
 
-        inspectorCustomizer.apply(runtime.getCamelContext());
+        phsc.apply(runtime.getCamelContext());
+
+        HealthContextCustomizer healthCustomizer = new HealthContextCustomizer();
+        healthCustomizer.apply(runtime.getCamelContext());
 
         try {
             runtime.getCamelContext().start();
diff --git a/camel-k-main/camel-k-runtime-inspector/pom.xml b/camel-k-main/camel-k-runtime-http/pom.xml
similarity index 96%
rename from camel-k-main/camel-k-runtime-inspector/pom.xml
rename to camel-k-main/camel-k-runtime-http/pom.xml
index 1e57d08..934c70e 100644
--- a/camel-k-main/camel-k-runtime-inspector/pom.xml
+++ b/camel-k-main/camel-k-runtime-http/pom.xml
@@ -25,7 +25,7 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>camel-k-runtime-inspector</artifactId>
+    <artifactId>camel-k-runtime-http</artifactId>
 
     <dependencies>
 
@@ -40,6 +40,10 @@
             <artifactId>camel-core-engine</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-platform-http</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.camel.k</groupId>
             <artifactId>camel-k-runtime-core</artifactId>
         </dependency>
diff --git a/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpRouter.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpRouter.java
new file mode 100644
index 0000000..5f33f2f
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpRouter.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.k.http;
+
+import io.vertx.ext.web.Router;
+import org.apache.camel.CamelContext;
+import org.apache.camel.component.platform.http.PlatformHttpConstants;
+import org.apache.camel.support.CamelContextHelper;
+
+public class PlatformHttpRouter {
+    public static final String PLATFORM_HTTP_ROUTER_NAME = PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME + "-router";
+
+    private final Router router;
+
+    public PlatformHttpRouter(Router router) {
+        this.router = router;
+    }
+
+    public Router get() {
+        return router;
+    }
+
+    // **********************
+    //
+    // Helpers
+    //
+    // **********************
+
+    public static PlatformHttpRouter lookup(CamelContext camelContext) {
+        return CamelContextHelper.mandatoryLookup(
+            camelContext,
+            PlatformHttpRouter.PLATFORM_HTTP_ROUTER_NAME,
+            PlatformHttpRouter.class
+        );
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServer.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServer.java
new file mode 100644
index 0000000..6d1bd7d
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServer.java
@@ -0,0 +1,178 @@
+/*
+ * 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.k.http;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+
+import io.vertx.core.Handler;
+import io.vertx.core.Vertx;
+import io.vertx.core.http.HttpServer;
+import io.vertx.ext.web.Router;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.BodyHandler;
+import org.apache.camel.CamelContext;
+import org.apache.camel.component.platform.http.PlatformHttpConstants;
+import org.apache.camel.support.service.ServiceSupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+final class PlatformHttpServer extends ServiceSupport {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PlatformHttpServer.class);
+
+    private final CamelContext context;
+    private final PlatformHttpServiceConfiguration configuration;
+    private final Vertx vertx;
+    private final ExecutorService executor;
+
+    private HttpServer server;
+
+    public PlatformHttpServer(CamelContext context, PlatformHttpServiceConfiguration configuration, Vertx vertx, ExecutorService executor) {
+        this.context = context;
+        this.configuration = configuration;
+        this.vertx = vertx;
+        this.executor = executor;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        startAsync().toCompletableFuture().join();
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        try {
+            if (server != null) {
+                stopAsync().toCompletableFuture().join();
+            }
+        } finally {
+            this.server = null;
+        }
+    }
+
+    private CompletionStage<Void> startAsync() {
+        final Router router = Router.router(vertx);
+        final Router subRouter = Router.router(vertx);
+
+        router.route()
+            .order(Integer.MIN_VALUE)
+            .handler(ctx -> {
+                ctx.request().resume();
+                createBodyHandler().handle(ctx);
+            });
+
+        router.mountSubRouter(configuration.getPath(), subRouter);
+
+        context.getRegistry().bind(PlatformHttpRouter.PLATFORM_HTTP_ROUTER_NAME, new PlatformHttpRouter(subRouter));
+
+        //HttpServerOptions options = new HttpServerOptions();
+        if (configuration.getSslContextParameters() != null) {
+            // TODO: add ssl support
+            throw new UnsupportedOperationException("Not yet implemented");
+        }
+
+        server = vertx.createHttpServer();
+
+        return CompletableFuture.runAsync(
+            () -> {
+                CountDownLatch latch = new CountDownLatch(1);
+                server.requestHandler(router).listen(configuration.getBindPort(), configuration.getBindHost(), result -> {
+                    try {
+                        if (result.failed()) {
+                            LOGGER.warn("Failed to start Vert.x HttpServer on {}:{}, reason: {}",
+                                configuration.getBindHost(),
+                                configuration.getBindPort(),
+                                result.cause().getMessage()
+                            );
+
+                            throw new RuntimeException(result.cause());
+                        }
+
+                        LOGGER.info("Vert.x HttpServer started on {}:{}", configuration.getBindHost(), configuration.getBindPort());
+                    } finally {
+                        latch.countDown();
+                    }
+                });
+
+                try {
+                    latch.await();
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+            },
+            executor
+        );
+    }
+
+    protected CompletionStage<Void> stopAsync() {
+        return CompletableFuture.runAsync(
+            () -> {
+                CountDownLatch latch = new CountDownLatch(1);
+
+                // remove the platform-http component
+                context.removeComponent(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME);
+
+                server.close(result -> {
+                    try {
+                        if (result.failed()) {
+                            LOGGER.warn("Failed to close Vert.x HttpServer reason: {}",
+                                result.cause().getMessage()
+                            );
+
+                            throw new RuntimeException(result.cause());
+                        }
+
+                        LOGGER.info("Vert.x HttpServer stopped");
+                    } finally {
+                        latch.countDown();
+                    }
+                });
+
+                try {
+                    latch.await();
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+            },
+            executor
+        );
+    }
+
+    private Handler<RoutingContext> createBodyHandler() {
+        BodyHandler bodyHandler = BodyHandler.create();
+
+        if (configuration.getMaxBodySize() != null) {
+            bodyHandler.setBodyLimit(configuration.getMaxBodySize().longValueExact());
+        }
+
+        bodyHandler.setHandleFileUploads(configuration.getBodyHandler().isHandleFileUploads());
+        bodyHandler.setUploadsDirectory(configuration.getBodyHandler().getUploadsDirectory());
+        bodyHandler.setDeleteUploadedFilesOnEnd(configuration.getBodyHandler().isDeleteUploadedFilesOnEnd());
+        bodyHandler.setMergeFormAttributes(configuration.getBodyHandler().isMergeFormAttributes());
+        bodyHandler.setPreallocateBodyBuffer(configuration.getBodyHandler().isPreallocateBodyBuffer());
+
+        return new Handler<RoutingContext>() {
+            @Override
+            public void handle(RoutingContext event) {
+                event.request().resume();
+                bodyHandler.handle(event);
+            }
+        };
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceConfiguration.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceConfiguration.java
new file mode 100644
index 0000000..659ecab
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceConfiguration.java
@@ -0,0 +1,131 @@
+/*
+ * 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.k.http;
+
+import java.math.BigInteger;
+
+import org.apache.camel.support.jsse.SSLContextParameters;
+
+public class PlatformHttpServiceConfiguration {
+    public static final String DEFAULT_BIND_HOST = "0.0.0.0";
+    public static final int DEFAULT_BIND_PORT = 8081;
+    public static final String DEFAULT_PATH = "/";
+
+    private String bindHost = DEFAULT_BIND_HOST;
+    private int bindPort = DEFAULT_BIND_PORT;
+    private String path = DEFAULT_PATH;
+    private BigInteger maxBodySize;
+
+    private BodyHandler bodyHandler = new BodyHandler();
+    private SSLContextParameters sslContextParameters;
+
+    public String getBindHost() {
+        return bindHost;
+    }
+
+    public void setBindHost(String bindHost) {
+        this.bindHost = bindHost;
+    }
+
+    public int getBindPort() {
+        return bindPort;
+    }
+
+    public void setBindPort(int bindPort) {
+        this.bindPort = bindPort;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    public BigInteger getMaxBodySize() {
+        return maxBodySize;
+    }
+
+    public void setMaxBodySize(BigInteger maxBodySize) {
+        this.maxBodySize = maxBodySize;
+    }
+
+    public BodyHandler getBodyHandler() {
+        return bodyHandler;
+    }
+
+    public void setBodyHandler(BodyHandler bodyHandler) {
+        this.bodyHandler = bodyHandler;
+    }
+
+    public SSLContextParameters getSslContextParameters() {
+        return sslContextParameters;
+    }
+
+    public void setSslContextParameters(SSLContextParameters sslContextParameters) {
+        this.sslContextParameters = sslContextParameters;
+    }
+
+    public static class BodyHandler {
+        private boolean handleFileUploads = true;
+        private String uploadsDirectory = "file-uploads";
+        private boolean mergeFormAttributes = true;
+        private boolean deleteUploadedFilesOnEnd;
+        private boolean preallocateBodyBuffer = true;
+
+        public boolean isHandleFileUploads() {
+            return handleFileUploads;
+        }
+
+        public void setHandleFileUploads(boolean handleFileUploads) {
+            this.handleFileUploads = handleFileUploads;
+        }
+
+        public String getUploadsDirectory() {
+            return uploadsDirectory;
+        }
+
+        public void setUploadsDirectory(String uploadsDirectory) {
+            this.uploadsDirectory = uploadsDirectory;
+        }
+
+        public boolean isMergeFormAttributes() {
+            return mergeFormAttributes;
+        }
+
+        public void setMergeFormAttributes(boolean mergeFormAttributes) {
+            this.mergeFormAttributes = mergeFormAttributes;
+        }
+
+        public boolean isDeleteUploadedFilesOnEnd() {
+            return deleteUploadedFilesOnEnd;
+        }
+
+        public void setDeleteUploadedFilesOnEnd(boolean deleteUploadedFilesOnEnd) {
+            this.deleteUploadedFilesOnEnd = deleteUploadedFilesOnEnd;
+        }
+
+        public boolean isPreallocateBodyBuffer() {
+            return preallocateBodyBuffer;
+        }
+
+        public void setPreallocateBodyBuffer(boolean preallocateBodyBuffer) {
+            this.preallocateBodyBuffer = preallocateBodyBuffer;
+        }
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorContextCustomizer.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceContextCustomizer.java
similarity index 52%
rename from camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorContextCustomizer.java
rename to camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceContextCustomizer.java
index 5bcc396..cc100ae 100644
--- a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorContextCustomizer.java
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceContextCustomizer.java
@@ -14,65 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.inspector;
+package org.apache.camel.k.http;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Ordered;
+import org.apache.camel.component.platform.http.PlatformHttpComponent;
+import org.apache.camel.component.platform.http.PlatformHttpConstants;
 import org.apache.camel.k.ContextCustomizer;
+import org.apache.camel.k.http.engine.RuntimePlatformHttpEngine;
 
-public class InspectorContextCustomizer implements ContextCustomizer {
-    public static final String DEFAULT_BIND_HOST = "0.0.0.0";
-    public static final int DEFAULT_BIND_PORT = 8081;
-    public static final String DEFAULT_PATH = "/";
-
-    private String bindHost;
-    private int bindPort;
-    private String path;
-    private InspectorEndpoint endpoint;
-
-    public InspectorContextCustomizer() {
-        this.bindHost = DEFAULT_BIND_HOST;
-        this.bindPort = DEFAULT_BIND_PORT;
-        this.path = DEFAULT_PATH;
-    }
-
-    public String getBindHost() {
-        return bindHost;
-    }
-
-    public void setBindHost(String bindHost) {
-        this.bindHost = bindHost;
-    }
-
-    public int getBindPort() {
-        return bindPort;
-    }
-
-    public void setBindPort(int bindPort) {
-        this.bindPort = bindPort;
-    }
-
-    public String getPath() {
-        return path;
-    }
-
-    public void setPath(String path) {
-        this.path = path;
-    }
+public class PlatformHttpServiceContextCustomizer extends PlatformHttpServiceConfiguration implements ContextCustomizer {
+    private PlatformHttpServiceEndpoint endpoint;
 
     @Override
     public int getOrder() {
-        return Ordered.LOWEST;
+        return Ordered.HIGHEST;
     }
 
     @Override
     public void apply(CamelContext camelContext) {
-        endpoint = new InspectorEndpoint(camelContext, bindHost, bindPort, path);
+        endpoint = new PlatformHttpServiceEndpoint(camelContext, this);
 
         try {
             camelContext.addService(endpoint, true, true);
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
+
+        // add the platform-http component
+        PlatformHttpComponent component = new PlatformHttpComponent();
+        component.setEngine(new RuntimePlatformHttpEngine());
+
+        camelContext.addComponent(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME, component);
     }
 }
diff --git a/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceEndpoint.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceEndpoint.java
new file mode 100644
index 0000000..1f904e9
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/PlatformHttpServiceEndpoint.java
@@ -0,0 +1,118 @@
+/*
+ * 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.k.http;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import io.vertx.core.Vertx;
+import io.vertx.core.VertxOptions;
+import org.apache.camel.CamelContext;
+import org.apache.camel.support.CamelContextHelper;
+import org.apache.camel.support.service.ServiceSupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PlatformHttpServiceEndpoint extends ServiceSupport {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PlatformHttpServiceEndpoint.class);
+
+    private final CamelContext context;
+    private final PlatformHttpServiceConfiguration configuration;
+
+    private Vertx vertx;
+    private boolean localVertx;
+    private PlatformHttpServer vertxHttpServer;
+    private ExecutorService executor;
+
+    public PlatformHttpServiceEndpoint(CamelContext context, PlatformHttpServiceConfiguration configuration) {
+        this.context = context;
+        this.configuration = configuration;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        vertx = CamelContextHelper.findByType(context, Vertx.class);
+        executor = context.getExecutorServiceManager().newSingleThreadExecutor(this, "platform-http-service");
+
+        if (vertx != null) {
+            LOGGER.info("Found Vert.x instance in registry: {}", vertx);
+        } else {
+            VertxOptions options = CamelContextHelper.findByType(context, VertxOptions.class);
+            if (options == null) {
+                options = new VertxOptions();
+            }
+
+            LOGGER.info("Creating new Vert.x instance");
+
+            vertx = Vertx.vertx(options);
+            localVertx = true;
+        }
+
+        vertxHttpServer = new PlatformHttpServer(context, configuration, vertx, executor);
+        vertxHttpServer.start();
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        if (vertxHttpServer != null) {
+            vertxHttpServer.stop();
+        }
+
+        if (vertx != null && localVertx) {
+            Future<?> future = executor.submit(
+                () -> {
+                    CountDownLatch latch = new CountDownLatch(1);
+
+                    vertx.close(result -> {
+                        try {
+                            if (result.failed()) {
+                                LOGGER.warn("Failed to close Vert.x reason: {}",
+                                    result.cause().getMessage()
+                                );
+
+                                throw new RuntimeException(result.cause());
+                            }
+
+                            LOGGER.info("Vert.x stopped");
+                        } finally {
+                            latch.countDown();
+                        }
+                    });
+
+                    try {
+                        latch.await();
+                    } catch (InterruptedException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+            );
+
+            try {
+                future.get();
+            } finally {
+                vertx = null;
+                localVertx = false;
+            }
+        }
+
+        if (executor != null) {
+            context.getExecutorServiceManager().shutdown(executor);
+            executor = null;
+        }
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpConsumer.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpConsumer.java
new file mode 100644
index 0000000..e438c32
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpConsumer.java
@@ -0,0 +1,389 @@
+/*
+ * 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.k.http.engine;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import io.vertx.core.MultiMap;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.core.http.HttpMethod;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.http.HttpServerResponse;
+import io.vertx.ext.web.Route;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.Message;
+import org.apache.camel.NoTypeConversionAvailableException;
+import org.apache.camel.Processor;
+import org.apache.camel.TypeConversionException;
+import org.apache.camel.TypeConverter;
+import org.apache.camel.component.platform.http.PlatformHttpEndpoint;
+import org.apache.camel.component.platform.http.spi.Method;
+import org.apache.camel.k.http.PlatformHttpRouter;
+import org.apache.camel.spi.HeaderFilterStrategy;
+import org.apache.camel.support.DefaultConsumer;
+import org.apache.camel.support.DefaultMessage;
+import org.apache.camel.support.ExchangeHelper;
+import org.apache.camel.support.MessageHelper;
+import org.apache.camel.support.ObjectHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RuntimePlatformHttpConsumer extends DefaultConsumer {
+    private static final Logger LOG = LoggerFactory.getLogger(RuntimePlatformHttpConsumer.class);
+    private static final Pattern PATH_PARAMETER_PATTERN = Pattern.compile("\\{([^/}]+)\\}");
+
+    private Route route;
+
+    public RuntimePlatformHttpConsumer(PlatformHttpEndpoint endpoint, Processor processor) {
+        super(endpoint, processor);
+    }
+
+    @Override
+    public PlatformHttpEndpoint getEndpoint() {
+        return (PlatformHttpEndpoint) super.getEndpoint();
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+
+        final PlatformHttpEndpoint endpoint = getEndpoint();
+        final PlatformHttpRouter router = PlatformHttpRouter.lookup(endpoint.getCamelContext());
+        final String path = endpoint.getPath();
+        final String vertxPathParamPath = PATH_PARAMETER_PATTERN.matcher(path).replaceAll(":$1");
+        final Route newRoute = router.get().route(vertxPathParamPath);
+
+        final Set<Method> methods = Method.parseList(endpoint.getHttpMethodRestrict());
+        if (!methods.equals(Method.getAll())) {
+            methods.stream().forEach(m -> newRoute.method(HttpMethod.valueOf(m.name())));
+        }
+        if (endpoint.getConsumes() != null) {
+            newRoute.consumes(endpoint.getConsumes());
+        }
+        if (endpoint.getProduces() != null) {
+            newRoute.produces(endpoint.getProduces());
+        }
+
+        newRoute.handler(
+            ctx -> {
+                Exchange exchg = null;
+                try {
+                    final Exchange exchange = exchg = toExchange(ctx);
+                    createUoW(exchange);
+                    getAsyncProcessor().process(
+                        exchange,
+                        doneSync -> writeResponse(ctx, exchange, getEndpoint().getHeaderFilterStrategy()));
+                } catch (Exception e) {
+                    ctx.fail(e);
+                    getExceptionHandler().handleException("Failed handling platform-http endpoint " + path, exchg, e);
+                } finally {
+                    if (exchg != null) {
+                        doneUoW(exchg);
+                    }
+                }
+            });
+
+        this.route = newRoute;
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        if (route != null) {
+            route.remove();
+            route = null;
+        }
+        super.doStop();
+    }
+
+    @Override
+    protected void doSuspend() throws Exception {
+        if (route != null) {
+            route.disable();
+        }
+        super.doSuspend();
+    }
+
+    @Override
+    protected void doResume() throws Exception {
+        if (route != null) {
+            route.enable();
+        }
+        super.doResume();
+    }
+
+    static Object toHttpResponse(HttpServerResponse response, Message message, HeaderFilterStrategy headerFilterStrategy) {
+        final Exchange exchange = message.getExchange();
+        final TypeConverter tc = exchange.getContext().getTypeConverter();
+        final int code = determineResponseCode(exchange, message.getBody());
+
+        response.setStatusCode(code);
+
+        // copy headers from Message to Response
+        if (headerFilterStrategy != null) {
+            for (Map.Entry<String, Object> entry : message.getHeaders().entrySet()) {
+                final String key = entry.getKey();
+                final Object value = entry.getValue();
+                // use an iterator as there can be multiple values. (must not use a delimiter)
+                final Iterator<?> it = ObjectHelper.createIterator(value, null);
+
+                String firstValue = null;
+                List<String> values = null;
+
+                while (it.hasNext()) {
+                    final String headerValue = tc.convertTo(String.class, it.next());
+                    if (headerValue != null
+                        && !headerFilterStrategy.applyFilterToCamelHeaders(key, headerValue, exchange)) {
+                        if (firstValue == null) {
+                            firstValue = headerValue;
+                        } else {
+                            if (values == null) {
+                                values = new ArrayList<>();
+                                values.add(firstValue);
+                            }
+                            values.add(headerValue);
+                        }
+                    }
+                }
+                if (values != null) {
+                    response.putHeader(key, values);
+                } else if (firstValue != null) {
+                    response.putHeader(key, firstValue);
+                }
+            }
+        }
+
+        Object body = message.getBody();
+        final Exception exception = exchange.getException();
+
+        if (exception != null) {
+            // we failed due an exception so print it as plain text
+            final StringWriter sw = new StringWriter();
+            final PrintWriter pw = new PrintWriter(sw);
+
+            exception.printStackTrace(pw);
+
+            // the body should then be the stacktrace
+            body = ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
+            // force content type to be text/plain as that is what the stacktrace is
+            message.setHeader(Exchange.CONTENT_TYPE, "text/plain; charset=utf-8");
+
+            // and mark the exception as failure handled, as we handled it by returning it as the response
+            ExchangeHelper.setFailureHandled(exchange);
+        }
+
+        // set the content type in the response.
+        final String contentType = MessageHelper.getContentType(message);
+        if (contentType != null) {
+            // set content-type
+            response.putHeader("Content-Type", contentType);
+        }
+        return body;
+    }
+
+    /*
+     * Copied from org.apache.camel.http.common.DefaultHttpBinding.determineResponseCode(Exchange, Object)
+     * If DefaultHttpBinding.determineResponseCode(Exchange, Object) is moved to a module without the servlet-api
+     * dependency we could eventually consume it from there.
+     */
+    static int determineResponseCode(Exchange camelExchange, Object body) {
+        boolean failed = camelExchange.isFailed();
+        int defaultCode = failed ? 500 : 200;
+
+        Message message = camelExchange.getMessage();
+        Integer currentCode = message.getHeader(Exchange.HTTP_RESPONSE_CODE, Integer.class);
+        int codeToUse = currentCode == null ? defaultCode : currentCode;
+
+        if (codeToUse != 500) {
+            if ((body == null) || (body instanceof String && ((String) body).trim().isEmpty())) {
+                // no content
+                codeToUse = currentCode == null ? 204 : currentCode;
+            }
+        }
+
+        return codeToUse;
+    }
+
+    static void writeResponse(RoutingContext ctx, Exchange camelExchange, HeaderFilterStrategy headerFilterStrategy) {
+        final Object body = toHttpResponse(ctx.response(), camelExchange.getMessage(), headerFilterStrategy);
+
+        final HttpServerResponse response = ctx.response();
+        if (body == null) {
+            LOG.trace("No payload to send as reply for exchange: {}", camelExchange);
+            response.end();
+        } else if (body instanceof String) {
+            response.end((String) body);
+        } else if (body instanceof InputStream) {
+            final byte[] bytes = new byte[4096];
+            try (InputStream in = (InputStream) body) {
+                int len;
+                while ((len = in.read(bytes)) >= 0) {
+                    response.write(Buffer.buffer(len).appendBytes(bytes, 0, len));
+                }
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+            response.end();
+        } else {
+            final TypeConverter tc = camelExchange.getContext().getTypeConverter();
+            try {
+                final ByteBuffer bb = tc.mandatoryConvertTo(ByteBuffer.class, body);
+                response.end(Buffer.buffer(bb.capacity()).setBytes(0, bb));
+            } catch (TypeConversionException | NoTypeConversionAvailableException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+    }
+
+    private Exchange toExchange(RoutingContext ctx) {
+        final Exchange exchange = getEndpoint().createExchange();
+        Message in = toCamelMessage(ctx, exchange);
+
+        final String charset = ctx.parsedHeaders().contentType().parameter("charset");
+        if (charset != null) {
+            exchange.setProperty(Exchange.CHARSET_NAME, charset);
+            in.setHeader(Exchange.HTTP_CHARACTER_ENCODING, charset);
+        }
+
+        exchange.setIn(in);
+        return exchange;
+    }
+
+    private Message toCamelMessage(RoutingContext ctx, Exchange exchange) {
+        final Message result = new DefaultMessage(exchange);
+
+        final HeaderFilterStrategy headerFilterStrategy = getEndpoint().getHeaderFilterStrategy();
+        populateCamelHeaders(ctx, result.getHeaders(), exchange, headerFilterStrategy);
+        final String mimeType = ctx.parsedHeaders().contentType().value();
+        final boolean isMultipartFormData = "multipart/form-data".equals(mimeType);
+        if ("application/x-www-form-urlencoded".equals(mimeType) || isMultipartFormData) {
+            final MultiMap formData = ctx.request().formAttributes();
+            final Map<String, Object> body = new HashMap<>();
+            for (String key : formData.names()) {
+                for (String value : formData.getAll(key)) {
+                    if (headerFilterStrategy != null
+                        && !headerFilterStrategy.applyFilterToExternalHeaders(key, value, exchange)) {
+                        appendHeader(result.getHeaders(), key, value);
+                        appendHeader(body, key, value);
+                    }
+                }
+            }
+            result.setBody(body);
+            if (isMultipartFormData) {
+                //populateAttachments(ctx.fileUploads(), result);
+            }
+        } else {
+            // extract body by myself if undertow parser didn't handle and the method is allowed to have one
+            // body is extracted as byte[] then auto TypeConverter kicks in
+            Method m = Method.valueOf(ctx.request().method().name());
+            if (m.canHaveBody()) {
+                final Buffer body = ctx.getBody();
+                if (body != null) {
+                    result.setBody(body.getBytes());
+                }
+            } else {
+                result.setBody(null);
+            }
+        }
+        return result;
+    }
+
+    static void populateCamelHeaders(
+        RoutingContext ctx,
+        Map<String, Object> headersMap,
+        Exchange exchange,
+        HeaderFilterStrategy headerFilterStrategy) {
+
+        final HttpServerRequest request = ctx.request();
+        headersMap.put(Exchange.HTTP_PATH, request.path());
+
+        if (headerFilterStrategy != null) {
+            final MultiMap requestHeaders = request.headers();
+            final String authz = requestHeaders.get("authorization");
+            // store a special header that this request was authenticated using HTTP Basic
+            if (authz != null && authz.trim().startsWith("Basic")) {
+                if (!headerFilterStrategy.applyFilterToExternalHeaders(Exchange.AUTHENTICATION, "Basic", exchange)) {
+                    appendHeader(headersMap, Exchange.AUTHENTICATION, "Basic");
+                }
+            }
+            for (String name : requestHeaders.names()) {
+                // add the headers one by one, and use the header filter strategy
+                for (String value : requestHeaders.getAll(name)) {
+                    if (!headerFilterStrategy.applyFilterToExternalHeaders(name.toString(), value, exchange)) {
+                        appendHeader(headersMap, name.toString(), value);
+                    }
+                }
+            }
+
+            // process uri parameters as headers
+            final MultiMap pathParameters = ctx.queryParams();
+            // continue if the map is not empty, otherwise there are no params
+            if (!pathParameters.isEmpty()) {
+                for (String name : pathParameters.names()) {
+                    for (String value : pathParameters.getAll(name)) {
+                        if (!headerFilterStrategy.applyFilterToExternalHeaders(name, value, exchange)) {
+                            appendHeader(headersMap, name, value);
+                        }
+                    }
+                }
+            }
+        }
+
+        // Path parameters
+        ctx.pathParams().forEach((k, v) ->  appendHeader(headersMap, k, v));
+
+        // NOTE: these headers is applied using the same logic as camel-http/camel-jetty to be consistent
+        headersMap.put(Exchange.HTTP_METHOD, request.method().toString());
+        // strip query parameters from the uri
+        headersMap.put(Exchange.HTTP_URL, request.absoluteURI());
+        // uri is without the host and port
+        headersMap.put(Exchange.HTTP_URI, request.uri());
+        headersMap.put(Exchange.HTTP_QUERY, request.query());
+        headersMap.put(Exchange.HTTP_RAW_QUERY, request.query());
+    }
+
+    @SuppressWarnings("unchecked")
+    static void appendHeader(Map<String, Object> headers, String key, Object value) {
+        if (headers.containsKey(key)) {
+            Object existing = headers.get(key);
+            List<Object> list;
+            if (existing instanceof List) {
+                list = (List<Object>) existing;
+            } else {
+                list = new ArrayList<>();
+                list.add(existing);
+            }
+            list.add(value);
+            value = list;
+        }
+
+        headers.put(key, value);
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorCustomizer.java b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpEngine.java
similarity index 60%
rename from camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorCustomizer.java
rename to camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpEngine.java
index 06b4cbb..05f8111 100644
--- a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorCustomizer.java
+++ b/camel-k-main/camel-k-runtime-http/src/main/java/org/apache/camel/k/http/engine/RuntimePlatformHttpEngine.java
@@ -14,16 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.inspector;
+package org.apache.camel.k.http.engine;
 
-import java.util.function.Consumer;
+import org.apache.camel.Consumer;
+import org.apache.camel.Processor;
+import org.apache.camel.component.platform.http.PlatformHttpEndpoint;
+import org.apache.camel.component.platform.http.spi.PlatformHttpEngine;
 
-import io.vertx.ext.web.Router;
-
-/**
- * Marker interface to easily lookup Vertx customizer from the
- * camel {@link org.apache.camel.spi.Registry}.
- */
-@FunctionalInterface
-public interface InspectorCustomizer extends Consumer<Router> {
+public class RuntimePlatformHttpEngine implements PlatformHttpEngine {
+    @Override
+    public Consumer createConsumer(PlatformHttpEndpoint platformHttpEndpoint, Processor processor) {
+        return new RuntimePlatformHttpConsumer(platformHttpEndpoint, processor);
+    }
 }
diff --git a/camel-k-main/camel-k-runtime-inspector/src/main/resources/META-INF/services/org/apache/camel/k/customizer/inspector b/camel-k-main/camel-k-runtime-http/src/main/resources/META-INF/services/org/apache/camel/k/customizer/platform-http
similarity index 92%
rename from camel-k-main/camel-k-runtime-inspector/src/main/resources/META-INF/services/org/apache/camel/k/customizer/inspector
rename to camel-k-main/camel-k-runtime-http/src/main/resources/META-INF/services/org/apache/camel/k/customizer/platform-http
index 6fa86ed..9be183b 100644
--- a/camel-k-main/camel-k-runtime-inspector/src/main/resources/META-INF/services/org/apache/camel/k/customizer/inspector
+++ b/camel-k-main/camel-k-runtime-http/src/main/resources/META-INF/services/org/apache/camel/k/customizer/platform-http
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-class=org.apache.camel.k.inspector.InspectorContextCustomizer
+class=org.apache.camel.k.http.PlatformHttpServiceContextCustomizer
diff --git a/camel-k-main/camel-k-runtime-http/src/test/java/org/apache/camel/k/http/PlatformHttpServiceCustomizerTest.java b/camel-k-main/camel-k-runtime-http/src/test/java/org/apache/camel/k/http/PlatformHttpServiceCustomizerTest.java
new file mode 100644
index 0000000..5756d37
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-http/src/test/java/org/apache/camel/k/http/PlatformHttpServiceCustomizerTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.k.http;
+
+import io.vertx.core.http.HttpMethod;
+import io.vertx.core.json.Json;
+import io.vertx.core.json.JsonObject;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.platform.http.PlatformHttpComponent;
+import org.apache.camel.component.platform.http.PlatformHttpConstants;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.apache.camel.k.Runtime;
+import org.apache.camel.k.http.engine.RuntimePlatformHttpEngine;
+import org.apache.camel.test.AvailablePortFinder;
+import org.apache.camel.util.ObjectHelper;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static io.restassured.RestAssured.given;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+
+public class PlatformHttpServiceCustomizerTest {
+
+    @ParameterizedTest
+    @ValueSource(strings = { "", "/test", "/test/nested" })
+    public void testPlatformHttpServiceCustomizer(String path) throws Exception {
+        Runtime runtime = Runtime.on(new DefaultCamelContext());
+
+        PlatformHttpServiceContextCustomizer httpService = new PlatformHttpServiceContextCustomizer();
+        httpService.setBindPort(AvailablePortFinder.getNextAvailable());
+
+        if (ObjectHelper.isNotEmpty(path)) {
+            httpService.setPath(path);
+        }
+
+        httpService.apply(runtime.getCamelContext());
+
+        PlatformHttpRouter.lookup(runtime.getCamelContext()).get().route(HttpMethod.GET, "/my/path")
+            .handler(routingContext -> {
+                JsonObject response = new JsonObject();
+                response.put("status", "UP");
+
+                routingContext.response()
+                    .putHeader("content-type", "application/json")
+                    .setStatusCode(200)
+                    .end(Json.encodePrettily(response));
+            });
+
+        given()
+            .port(httpService.getBindPort())
+        .when()
+            .get(path + "/my/path")
+        .then()
+            .statusCode(200)
+            .body("status", equalTo("UP"));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = { "/", "/test", "/test/nested" })
+    public void testPlatformHttpComponent(String path) throws Exception {
+        Runtime runtime = Runtime.on(new DefaultCamelContext());
+        runtime.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                fromF("platform-http:%s", path)
+                    .setBody().constant(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME);
+            }
+        });
+
+        PlatformHttpServiceContextCustomizer httpService = new PlatformHttpServiceContextCustomizer();
+        httpService.setBindPort(AvailablePortFinder.getNextAvailable());
+        httpService.apply(runtime.getCamelContext());
+
+        PlatformHttpComponent c = runtime.getCamelContext().getComponent(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME, PlatformHttpComponent.class);
+
+        assertThat(c).isNotNull();
+        assertThat(c.getEngine()).isInstanceOf(RuntimePlatformHttpEngine.class);
+
+        try {
+            runtime.getCamelContext().start();
+
+            given()
+                .port(httpService.getBindPort())
+            .when()
+                .get(path)
+            .then()
+                .statusCode(200)
+                .body(equalTo(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME));
+        } finally {
+            runtime.getCamelContext().stop();
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = { "/", "/test", "/test/nested" })
+    public void testPlatformHttpComponentPost(String path) throws Exception {
+        Runtime runtime = Runtime.on(new DefaultCamelContext());
+        runtime.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                fromF("platform-http:%s", path)
+                    .transform().body(String.class, b -> b.toUpperCase());
+            }
+        });
+
+        PlatformHttpServiceContextCustomizer httpService = new PlatformHttpServiceContextCustomizer();
+        httpService.setBindPort(AvailablePortFinder.getNextAvailable());
+        httpService.apply(runtime.getCamelContext());
+
+        PlatformHttpComponent c = runtime.getCamelContext().getComponent(PlatformHttpConstants.PLATFORM_HTTP_COMPONENT_NAME, PlatformHttpComponent.class);
+
+        assertThat(c).isNotNull();
+        assertThat(c.getEngine()).isInstanceOf(RuntimePlatformHttpEngine.class);
+
+        try {
+            runtime.getCamelContext().start();
+
+            given()
+                .port(httpService.getBindPort())
+                .body("test")
+            .when()
+                .post(path)
+            .then()
+                .statusCode(200)
+                .body(equalTo("TEST"));
+        } finally {
+            runtime.getCamelContext().stop();
+        }
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-inspector/src/test/resources/log4j2-test.xml b/camel-k-main/camel-k-runtime-http/src/test/resources/log4j2-test.xml
similarity index 100%
rename from camel-k-main/camel-k-runtime-inspector/src/test/resources/log4j2-test.xml
rename to camel-k-main/camel-k-runtime-http/src/test/resources/log4j2-test.xml
diff --git a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorEndpoint.java b/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorEndpoint.java
deleted file mode 100644
index 71eafde..0000000
--- a/camel-k-main/camel-k-runtime-inspector/src/main/java/org/apache/camel/k/inspector/InspectorEndpoint.java
+++ /dev/null
@@ -1,220 +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.k.inspector;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionStage;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
-import io.vertx.core.Vertx;
-import io.vertx.core.VertxOptions;
-import io.vertx.core.http.HttpServer;
-import io.vertx.ext.web.Router;
-import org.apache.camel.CamelContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.support.service.ServiceSupport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class InspectorEndpoint extends ServiceSupport {
-    private static final Logger LOGGER = LoggerFactory.getLogger(InspectorEndpoint.class);
-
-    private final CamelContext context;
-    private final String bindHost;
-    private final int bindPort;
-    private final String path;
-
-    private Vertx vertx;
-    private boolean localVertx;
-    private ExecutorService executor;
-    private HttpServerWrapper vertxHttpServer;
-
-    public InspectorEndpoint(CamelContext context, String bindHost, int bindPort, String path) {
-        this.context = context;
-        this.bindHost = bindHost;
-        this.bindPort = bindPort;
-        this.path = path;
-    }
-
-    @Override
-    protected void doStart() throws Exception {
-        this.executor = context.getExecutorServiceManager().newSingleThreadExecutor(this, "main-actuator");
-        this.vertx = CamelContextHelper.findByType(context, Vertx.class);
-
-        if (this.vertx != null) {
-            LOGGER.info("Found Vert.x instance in registry: {}", this.vertx);
-        } else {
-            VertxOptions options = CamelContextHelper.findByType(context, VertxOptions.class);
-            if (options == null) {
-                options = new VertxOptions();
-            }
-
-            LOGGER.info("Creating new Vert.x instance");
-
-            this.vertx = Vertx.vertx(options);
-            this.localVertx = true;
-        }
-
-        vertxHttpServer = new HttpServerWrapper();
-        vertxHttpServer.start();
-    }
-
-    @Override
-    protected void doStop() throws Exception {
-        if (this.vertxHttpServer != null) {
-            vertxHttpServer.stop();
-        }
-
-        if (this.vertx != null && this.localVertx) {
-            Future<?> future = this.executor.submit(
-                () -> {
-                    CountDownLatch latch = new CountDownLatch(1);
-
-                    this.vertx.close(result -> {
-                        try {
-                            if (result.failed()) {
-                                LOGGER.warn("Failed to close Vert.x reason: {}",
-                                    result.cause().getMessage()
-                                );
-
-                                throw new RuntimeException(result.cause());
-                            }
-
-                            LOGGER.info("Vert.x stopped");
-                        } finally {
-                            latch.countDown();
-                        }
-                    });
-
-                    try {
-                        latch.await();
-                    } catch (InterruptedException e) {
-                        throw new RuntimeException(e);
-                    }
-                }
-            );
-
-            try {
-                future.get();
-            } finally {
-                this.vertx = null;
-                this.localVertx = false;
-            }
-        }
-
-        if (this.executor != null) {
-            context.getExecutorServiceManager().shutdownNow(this.executor);
-        }
-    }
-
-    private final class HttpServerWrapper extends ServiceSupport {
-        private HttpServer server;
-
-        @Override
-        protected void doStart() throws Exception {
-            startAsync().toCompletableFuture().join();
-        }
-
-        @Override
-        protected void doStop() throws Exception {
-            try {
-                if (server != null) {
-                    stopAsync().toCompletableFuture().join();
-                }
-            } finally {
-                this.server = null;
-            }
-        }
-
-        private CompletionStage<Void> startAsync() {
-            final Router router = Router.router(vertx);
-            final Router subRouter = Router.router(vertx);
-
-            context.getRegistry().findByType(InspectorCustomizer.class).forEach(customizer -> {
-                LOGGER.debug("InspectorCustomizer: {}", customizer);
-                customizer.accept(subRouter);
-            });
-
-            router.mountSubRouter(path, subRouter);
-
-            server = vertx.createHttpServer();
-            return CompletableFuture.runAsync(
-                () -> {
-                    CountDownLatch latch = new CountDownLatch(1);
-
-                    server.requestHandler(router).listen(bindPort, bindHost, result -> {
-                        try {
-                            if (result.failed()) {
-                                LOGGER.warn("Failed to start Vert.x HttpServer on {}:{}, reason: {}",
-                                    bindHost,
-                                    bindPort,
-                                    result.cause().getMessage()
-                                );
-
-                                throw new RuntimeException(result.cause());
-                            }
-
-                            LOGGER.info("Vert.x HttpServer started on {}:{}", bindHost, bindPort);
-                        } finally {
-                            latch.countDown();
-                        }
-                    });
-
-                    try {
-                        latch.await();
-                    } catch (InterruptedException e) {
-                        throw new RuntimeException(e);
-                    }
-                },
-                executor
-            );
-        }
-
-        protected CompletionStage<Void> stopAsync() {
-            return CompletableFuture.runAsync(
-                () -> {
-                    CountDownLatch latch = new CountDownLatch(1);
-
-                    server.close(result -> {
-                        try {
-                            if (result.failed()) {
-                                LOGGER.warn("Failed to close Vert.x HttpServer reason: {}",
-                                    result.cause().getMessage()
-                                );
-
-                                throw new RuntimeException(result.cause());
-                            }
-
-                            LOGGER.info("Vert.x HttpServer stopped");
-                        } finally {
-                            latch.countDown();
-                        }
-                    });
-
-                    try {
-                        latch.await();
-                    } catch (InterruptedException e) {
-                        throw new RuntimeException(e);
-                    }
-                },
-                executor
-            );
-        }
-    }
-}
diff --git a/camel-k-main/camel-k-runtime-inspector/src/test/java/org/apache/camel/k/inspector/InspectorCustomizerTest.java b/camel-k-main/camel-k-runtime-inspector/src/test/java/org/apache/camel/k/inspector/InspectorCustomizerTest.java
deleted file mode 100644
index d103b1f..0000000
--- a/camel-k-main/camel-k-runtime-inspector/src/test/java/org/apache/camel/k/inspector/InspectorCustomizerTest.java
+++ /dev/null
@@ -1,74 +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.k.inspector;
-
-import java.net.URL;
-
-import io.vertx.core.http.HttpMethod;
-import io.vertx.core.json.Json;
-import io.vertx.core.json.JsonObject;
-import org.apache.camel.impl.DefaultCamelContext;
-import org.apache.camel.k.Runtime;
-import org.apache.camel.test.AvailablePortFinder;
-import org.apache.camel.util.ObjectHelper;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
-
-import static io.restassured.RestAssured.when;
-import static org.hamcrest.Matchers.equalTo;
-
-public class InspectorCustomizerTest {
-
-    @ParameterizedTest
-    @ValueSource(strings = { "", "/test", "/test/nested" })
-    public void testInspectorConfigurer(String path) throws Exception {
-        Runtime runtime = Runtime.on(new DefaultCamelContext());
-
-        runtime.getRegistry().bind("my-customizer", (InspectorCustomizer) router ->
-            router.route(HttpMethod.GET, "/my/path")
-                .handler(routingContext -> {
-                    JsonObject response = new JsonObject();
-                    response.put("status", "UP");
-
-                    routingContext.response()
-                        .putHeader("content-type", "application/json")
-                        .setStatusCode(200)
-                        .end(Json.encodePrettily(response));
-                })
-        );
-
-        InspectorContextCustomizer inspectorCustomizer = new InspectorContextCustomizer();
-        inspectorCustomizer.setBindPort(AvailablePortFinder.getNextAvailable());
-
-        String url;
-        if (ObjectHelper.isEmpty(path)) {
-            url = "http://localhost:" + inspectorCustomizer.getBindPort() + "/my/path";
-        } else {
-            inspectorCustomizer.setPath(path);
-
-            url = "http://localhost:" + inspectorCustomizer.getBindPort() + path + "/my/path";
-        }
-
-        inspectorCustomizer.apply(runtime.getCamelContext());
-
-        when()
-            .get(new URL(url))
-        .then()
-            .statusCode(200)
-            .body("status", equalTo("UP"));
-    }
-}
diff --git a/camel-k-main/pom.xml b/camel-k-main/pom.xml
index d9d5762..eb217ef 100644
--- a/camel-k-main/pom.xml
+++ b/camel-k-main/pom.xml
@@ -31,7 +31,7 @@
     <modules>
         <module>camel-k-runtime-main</module>
         <module>camel-k-runtime-health</module>
-        <module>camel-k-runtime-inspector</module>
+        <module>camel-k-runtime-http</module>
     </modules>
 
 </project>
diff --git a/camel-k-runtime-bom/pom.xml b/camel-k-runtime-bom/pom.xml
index ca51c48..57fc9a2 100644
--- a/camel-k-runtime-bom/pom.xml
+++ b/camel-k-runtime-bom/pom.xml
@@ -135,7 +135,7 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.camel.k</groupId>
-                <artifactId>camel-k-runtime-inspector</artifactId>
+                <artifactId>camel-k-runtime-http</artifactId>
                 <version>${project.version}</version>
             </dependency>
             <dependency>
diff --git a/examples/camel-k-runtime-example-health/README.adoc b/examples/camel-k-runtime-example-health/README.adoc
deleted file mode 100644
index 1c19469..0000000
--- a/examples/camel-k-runtime-example-health/README.adoc
+++ /dev/null
@@ -1,25 +0,0 @@
-Health Apache Camel K Runtime example
-======================================
-
-This repository contains an Apache Camel-K Runtime application that expose `/health` endpoint to gather Camel context status.
-
-In order to run it:
-```bash
-mvn clean exec:java
-```
-while it is running (from another terminal) you can access the rest camel route exposed through the servlet with:
-```bash
-curl http://localhost:8082/health
-```
-if the camel context has started properly that should get:
-```json
-{
-    "checks": [
-        {
-            "name": "route:my-route",
-            "status": "UP"
-        }
-    ],
-    "status": "UP"
-}
-```
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-rest/README.adoc b/examples/camel-k-runtime-example-rest/README.adoc
new file mode 100644
index 0000000..bc2024f
--- /dev/null
+++ b/examples/camel-k-runtime-example-rest/README.adoc
@@ -0,0 +1,58 @@
+Rest Apache Camel K Runtime example
+======================================
+
+This repository contains an Apache Camel-K Runtime application that expose a `rest` endpoint and and `/health` endpoint to gather Camel context status.
+
+In order to run it:
+[code,shell]
+----
+mvn clean exec:java
+----
+
+while it is running (from another terminal) you can access to:
+
+* health info with:
++
+[code,shell]
+----
+http http://localhost:8080/health
+----
++
+if the camel context has started properly that should get:
++
+[code,json]
+----
+{
+    "checks": [
+        {
+            "name": "context",
+            "status": "UP"
+        },
+        {
+            "name": "route:route1",
+            "status": "UP"
+        },
+        {
+            "name": "route:route2",
+            "status": "UP"
+        }
+    ],
+    "status": "UP"
+}
+----
+
+* rest endpoint info with:
++
+[code,shell]
+----
+http POST http://localhost:8080/rest a=b
+----
++
+if the route has started properly that should get:
++
+[code,json]
+----
+{
+    "A": "B"
+}
+----
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-health/src/main/resources/application.properties b/examples/camel-k-runtime-example-rest/data/application.properties
similarity index 91%
rename from examples/camel-k-runtime-example-health/src/main/resources/application.properties
rename to examples/camel-k-runtime-example-rest/data/application.properties
index cf08a83..3977fbf 100644
--- a/examples/camel-k-runtime-example-health/src/main/resources/application.properties
+++ b/examples/camel-k-runtime-example-rest/data/application.properties
@@ -30,6 +30,6 @@ camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
 #
 # Camel K
 #
-customizer.inspector.enabled = true
-customizer.inspector.bind-port = 8082
+customizer.platform-http.enabled = true
+customizer.platform-http.bind-port = 8080
 customizer.health.enabled = true
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-health/src/main/resources/routes.groovy b/examples/camel-k-runtime-example-rest/data/routes.groovy
similarity index 78%
rename from examples/camel-k-runtime-example-health/src/main/resources/routes.groovy
rename to examples/camel-k-runtime-example-rest/data/routes.groovy
index a1818bd..28a8fad 100644
--- a/examples/camel-k-runtime-example-health/src/main/resources/routes.groovy
+++ b/examples/camel-k-runtime-example-rest/data/routes.groovy
@@ -15,7 +15,16 @@
  * limitations under the License.
  */
 
-from('netty-http:http:0.0.0.0:8080//test')
-    .routeId('my-route')
-    .convertBodyTo(String.class)
-    .to('log:info')
\ No newline at end of file
+rest {
+    configuration {
+        component 'platform-http'
+    }
+
+    post('/rest') {
+        to 'direct:post'
+    }
+}
+
+from('direct:post')
+    .transform().body(String.class, b -> b.toUpperCase())
+    .to('log:post')
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-health/pom.xml b/examples/camel-k-runtime-example-rest/pom.xml
similarity index 84%
rename from examples/camel-k-runtime-example-health/pom.xml
rename to examples/camel-k-runtime-example-rest/pom.xml
index e4aa9ff..1245993 100644
--- a/examples/camel-k-runtime-example-health/pom.xml
+++ b/examples/camel-k-runtime-example-rest/pom.xml
@@ -25,7 +25,7 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>camel-k-runtime-example-health</artifactId>
+    <artifactId>camel-k-runtime-example-rest</artifactId>
 
     <dependencies>
         <dependency>
@@ -41,13 +41,21 @@
             <artifactId>camel-k-runtime-health</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-netty-http</artifactId>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-runtime-http</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.camel</groupId>
             <artifactId>camel-log</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-rest</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-direct</artifactId>
+        </dependency>
     </dependencies>
 
     <build>
@@ -70,11 +78,11 @@
                     <systemProperties>
                         <systemProperty>
                             <key>camel.k.conf</key>
-                            <value>${project.basedir}/src/main/resources/application.properties</value>
+                            <value>${project.basedir}/data/application.properties</value>
                         </systemProperty>
                         <systemProperty>
                             <key>camel.k.routes</key>
-                            <value>file:${project.basedir}/src/main/resources/routes.groovy</value>
+                            <value>file:${project.basedir}/data/routes.groovy</value>
                         </systemProperty>
                     </systemProperties>
                 </configuration>
diff --git a/examples/pom.xml b/examples/pom.xml
index 47ec202..80a7d73 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -33,7 +33,7 @@
     </properties>
 
     <modules>
-        <module>camel-k-runtime-example-health</module>
+        <module>camel-k-runtime-example-rest</module>
         <module>camel-k-runtime-example-groovy</module>
         <module>camel-k-runtime-example-yaml</module>
         <module>camel-k-runtime-example-knative</module>
diff --git a/pom.xml b/pom.xml
index 6d6fa00..d6a58fe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -318,7 +318,7 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.camel.k</groupId>
-                <artifactId>camel-k-runtime-inspector</artifactId>
+                <artifactId>camel-k-runtime-http</artifactId>
                 <version>${project.version}</version>
             </dependency>
             <dependency>
diff --git a/tooling/camel-k-maven-plugin/src/it/generate-catalog-main/verify.groovy b/tooling/camel-k-maven-plugin/src/it/generate-catalog-main/verify.groovy
index fbad5a9..f9ebcef 100644
--- a/tooling/camel-k-maven-plugin/src/it/generate-catalog-main/verify.groovy
+++ b/tooling/camel-k-maven-plugin/src/it/generate-catalog-main/verify.groovy
@@ -24,8 +24,8 @@ assert catalog.spec.runtime.capabilities['health'].dependencies[0].groupId == 'o
 assert catalog.spec.runtime.capabilities['health'].dependencies[0].artifactId == 'camel-k-runtime-health'
 assert catalog.spec.runtime.capabilities['rest'].dependencies[0].groupId == 'org.apache.camel'
 assert catalog.spec.runtime.capabilities['rest'].dependencies[0].artifactId == 'camel-rest'
-assert catalog.spec.runtime.capabilities['rest'].dependencies[1].groupId == 'org.apache.camel'
-assert catalog.spec.runtime.capabilities['rest'].dependencies[1].artifactId == 'camel-undertow'
+assert catalog.spec.runtime.capabilities['rest'].dependencies[1].groupId == 'org.apache.camel.k'
+assert catalog.spec.runtime.capabilities['rest'].dependencies[1].artifactId == 'camel-k-runtime-http'
 
 assert catalog.metadata.labels['camel.apache.org/runtime.version'] == runtimeVersion
 
diff --git a/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/GenerateCatalogMojo.java b/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/GenerateCatalogMojo.java
index c3e5341..b1637a2 100644
--- a/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/GenerateCatalogMojo.java
+++ b/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/GenerateCatalogMojo.java
@@ -133,7 +133,7 @@ public class GenerateCatalogMojo extends AbstractMojo {
                         "rest",
                         new CamelCapability.Builder()
                             .addDependency("org.apache.camel", "camel-rest")
-                            .addDependency("org.apache.camel", "camel-undertow")
+                            .addDependency("org.apache.camel.k", "camel-k-runtime-http")
                             .build());
                     break;
                 case "quarkus":
diff --git a/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3x.java b/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3x.java
index e235443..60ed03c 100644
--- a/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3x.java
+++ b/tooling/camel-k-maven-plugin/src/main/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3x.java
@@ -132,7 +132,7 @@ public class CatalogProcessor3x implements CatalogProcessor {
         specBuilder.putArtifact(
             new CamelArtifact.Builder()
                 .groupId("org.apache.camel.k")
-                .artifactId("camel-k-runtime-inspector")
+                .artifactId("camel-k-runtime-http")
                 .build()
         );
         specBuilder.putArtifact(
diff --git a/tooling/camel-k-maven-plugin/src/test/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3Test.java b/tooling/camel-k-maven-plugin/src/test/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3Test.java
index 9a953bc..4496d42 100644
--- a/tooling/camel-k-maven-plugin/src/test/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3Test.java
+++ b/tooling/camel-k-maven-plugin/src/test/java/org/apache/camel/k/tooling/maven/processors/CatalogProcessor3Test.java
@@ -80,7 +80,7 @@ public class CatalogProcessor3Test extends AbstractCatalogProcessorTest {
         Map<String, CamelArtifact> artifactMap = spec.getArtifacts();
 
         assertThat(artifactMap).containsKeys("camel-k-runtime-health");
-        assertThat(artifactMap).containsKeys("camel-k-runtime-inspector");
+        assertThat(artifactMap).containsKeys("camel-k-runtime-http");
         assertThat(artifactMap).containsKeys("camel-k-runtime-webhook");
 
         assertThat(artifactMap.get("camel-k-runtime-knative")).satisfies(a -> {