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/09/17 21:23:53 UTC

[camel-k-runtime] branch master updated: quarkus: improve runtime set-up

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


The following commit(s) were added to refs/heads/master by this push:
     new a1b2668  quarkus: improve runtime set-up
a1b2668 is described below

commit a1b2668c856a152828b476000ff08f00e31d3d28
Author: Luca Burgazzoli <lb...@gmail.com>
AuthorDate: Thu Sep 17 18:35:48 2020 +0200

    quarkus: improve runtime set-up
---
 .github/workflows/ci-build.yml                     |   1 +
 .../camel-k-quarkus-itests-cron/pom.xml            |  19 +++
 .../k/quarkus/cron/deployment/Application.java     |  66 ++++++++
 .../src/main/resources/application.properties      |   2 +
 .../camel/k/quarkus/cron/deployment/CronTest.java  |  24 ++-
 .../apache/camel/k/quarkus/master/Application.java |   5 +-
 .../src/main/resources/application.properties      |   2 +-
 .../pom.xml                                        |  19 ++-
 .../apache/camel/k/quarkus/it}/Application.java    |  42 +++--
 .../src/main/resources/application.properties      |   6 -
 .../org/apache/camel/k/quarkus/it/RuntimeIT.java}  |  18 +--
 .../org/apache/camel/k/quarkus/it/RuntimeTest.java |  62 ++++++++
 .../src/test/resources/test.properties}            |  12 +-
 camel-k-quarkus/camel-k-quarkus-itests/pom.xml     |   1 +
 .../k/quarkus/deployment/DeploymentProcessor.java  |  27 ++++
 .../org/apache/camel/k/quarkus/Application.java    | 170 +++++++++++++++++++++
 ...r.java => ApplicationConfigSourceProvider.java} |   2 +-
 .../k/quarkus/ApplicationListenerAdapter.java      | 140 -----------------
 .../camel/k/quarkus/ApplicationProducers.java}     |  55 ++++---
 .../camel/k/quarkus/ApplicationRecorder.java       |   8 +-
 ...se.microprofile.config.spi.ConfigSourceProvider |   2 +-
 .../camel/k/cron/CronSourceLoaderInterceptor.java  |   6 +-
 ...k.loader.knative.KnativeSourceLoaderInterceptor |   2 -
 .../data/application.properties                    |   2 +-
 pom.xml                                            |   6 +
 25 files changed, 475 insertions(+), 224 deletions(-)

diff --git a/.github/workflows/ci-build.yml b/.github/workflows/ci-build.yml
index b3126be..f7e591d 100644
--- a/.github/workflows/ci-build.yml
+++ b/.github/workflows/ci-build.yml
@@ -110,6 +110,7 @@ jobs:
       matrix:
         native-image-project:
           - :camel-k-quarkus-itests-core
+          - :camel-k-quarkus-itests-runtime
           - :camel-k-quarkus-itests-cron
           - :camel-k-quarkus-itests-master
           - :camel-k-quarkus-itests-kamelet
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml
index 291f5a3..032eb98 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml
@@ -30,8 +30,20 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-runtime-quarkus</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
             <artifactId>camel-k-quarkus-cron</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-quarkus-loader-yaml</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-timer</artifactId>
+        </dependency>
 
         <!-- quarkus dependencies -->
         <dependency>
@@ -68,6 +80,13 @@
             <artifactId>hamcrest-core</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.awaitility</groupId>
+            <artifactId>awaitility</artifactId>
+            <version>${awaitility-version}</version>
+        </dependency>
+
     </dependencies>
 
 
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java
index 3d8cb21..a70fd02 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java
@@ -16,22 +16,38 @@
  */
 package org.apache.camel.k.quarkus.cron.deployment;
 
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import javax.enterprise.context.ApplicationScoped;
 import javax.inject.Inject;
+import javax.inject.Singleton;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 
+import io.quarkus.arc.Unremovable;
 import org.apache.camel.CamelContext;
 import org.apache.camel.ExtendedCamelContext;
 import org.apache.camel.k.Constants;
+import org.apache.camel.k.Runtime;
+import org.apache.camel.k.Source;
+import org.apache.camel.k.SourceLoader;
+import org.apache.camel.k.Sources;
+import org.apache.camel.k.cron.CronSourceLoaderInterceptor;
+import org.apache.camel.k.loader.yaml.YamlSourceLoader;
 
 @Path("/test")
 @ApplicationScoped
 public class Application {
     @Inject
     CamelContext context;
+    @Inject
+    Runtime runtime;
+
+    private final AtomicBoolean stopped = new AtomicBoolean();
 
     @GET
     @Path("/find-cron-interceptor")
@@ -43,4 +59,54 @@ public class Application {
             .map(Class::getName)
             .orElse("");
     }
+
+    @GET
+    @Path("/load")
+    @Produces(MediaType.TEXT_PLAIN)
+    public String load() throws Exception {
+        final String code = ""
+            + "\n- from:"
+            + "\n    uri: \"timer:tick?period=1&delay=60000\""
+            + "\n    steps:"
+            + "\n      - log: \"${body}\"";
+
+        final SourceLoader loader = new YamlSourceLoader();
+        final Source source = Sources.fromBytes("my-cron", "yaml", null, List.of("cron"), code.getBytes(StandardCharsets.UTF_8));
+
+        final CronSourceLoaderInterceptor interceptor = new CronSourceLoaderInterceptor();
+        interceptor.setRuntime(runtime);
+        interceptor.setOverridableComponents("timer");
+
+        SourceLoader.Result result = interceptor.afterLoad(
+            loader,
+            source,
+            loader.load(runtime, source));
+
+        result.builder().ifPresent(b -> {
+            try {
+                context.addRoutes(b);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        });
+
+        return "" + context.getRoutesSize();
+    }
+
+    @GET
+    @Path("/stopped")
+    @Produces(MediaType.TEXT_PLAIN)
+    public String stopped()  {
+        return "" + stopped.get();
+    }
+
+    /*
+     * Override the default ShutdownTask for testing purpose.
+     */
+    @Unremovable
+    @Singleton
+    @javax.enterprise.inject.Produces
+    org.apache.camel.k.quarkus.Application.ShutdownTask shutdownTask() {
+        return () -> stopped.set(true);
+    }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
index 3f6864a..1882d46 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
@@ -26,3 +26,5 @@ quarkus.banner.enabled     = false
 #
 quarkus.camel.routes-discovery.enabled = false
 
+
+
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/test/java/org/apache/camel/k/quarkus/cron/deployment/CronTest.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/test/java/org/apache/camel/k/quarkus/cron/deployment/CronTest.java
index ff38d61..e0f9456 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/test/java/org/apache/camel/k/quarkus/cron/deployment/CronTest.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/test/java/org/apache/camel/k/quarkus/cron/deployment/CronTest.java
@@ -16,23 +16,43 @@
  */
 package org.apache.camel.k.quarkus.cron.deployment;
 
-import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 import io.quarkus.test.junit.QuarkusTest;
 import org.apache.camel.k.cron.CronSourceLoaderInterceptor;
 import org.junit.jupiter.api.Test;
 
 import static io.restassured.RestAssured.when;
+import static org.awaitility.Awaitility.await;
 import static org.hamcrest.Matchers.is;
 
 @QuarkusTest
 public class CronTest {
     @Test
-    public void cronInterceptorIsRegistered() throws IOException {
+    public void cronInterceptorIsRegistered() {
         when()
             .get("/test/find-cron-interceptor")
         .then()
             .statusCode(200)
             .body(is(CronSourceLoaderInterceptor.class.getName()));
     }
+
+    @Test
+    public void cronInvokesShutdown() {
+        when()
+            .get("/test/load")
+            .then()
+            .statusCode(200)
+            .body(is("1"));
+
+        await().atMost(10, TimeUnit.SECONDS).until(() -> {
+            String result = when()
+                .get("/test/stopped")
+                .then()
+                .statusCode(200)
+                .extract().body().asString();
+
+            return "true".equals(result);
+        });
+    }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java
index 3135739..d81ffd4 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java
@@ -37,13 +37,12 @@ public class Application {
     @GET
     @Path("/inspect")
     @Produces(MediaType.APPLICATION_JSON)
-    public JsonObject findCronInterceptor() {
-        KubernetesClusterService service = context.hasService(KubernetesClusterService.class);
+    public JsonObject inspect() {
+        var service = context.hasService(KubernetesClusterService.class);
 
         return Json.createObjectBuilder()
             .add("cluster-service", service != null ? service.getClass().getName() : "")
             .add("cluster-service-cm", service != null ? service.getConfigMapName() : "")
-
             .build();
     }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/resources/application.properties b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/resources/application.properties
index 9537015..e84134f 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/resources/application.properties
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/resources/application.properties
@@ -18,7 +18,7 @@
 #
 # Quarkus
 #
-quarkus.log.console.enable = true
+quarkus.log.console.enable = false
 quarkus.banner.enabled     = false
 
 #
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/pom.xml
similarity index 89%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/pom.xml
index 291f5a3..90f3326 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/pom.xml
@@ -25,15 +25,14 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>camel-k-quarkus-itests-cron</artifactId>
+    <artifactId>camel-k-quarkus-itests-runtime</artifactId>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.camel.k</groupId>
-            <artifactId>camel-k-quarkus-cron</artifactId>
+            <artifactId>camel-k-runtime-quarkus</artifactId>
         </dependency>
 
-        <!-- quarkus dependencies -->
         <dependency>
             <groupId>io.quarkus</groupId>
             <artifactId>quarkus-jsonb</artifactId>
@@ -63,11 +62,6 @@
             <artifactId>assertj-core</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.hamcrest</groupId>
-            <artifactId>hamcrest-core</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 
 
@@ -109,9 +103,13 @@
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-surefire-plugin</artifactId>
                 <configuration>
+                    <environmentVariables>
+                        <CAMEL_K_CONF>${project.basedir}/src/test/resources/test.properties</CAMEL_K_CONF>
+                    </environmentVariables>
                     <systemProperties>
                         <quarkus.http.test-port>${test.http.port.jvm}</quarkus.http.test-port>
                         <java.util.logging.manager>org.jboss.logmanager.LogManager</java.util.logging.manager>
+                        <camel.k.conf>${project.basedir}/src/test/resources/test.properties</camel.k.conf>
                     </systemProperties>
                 </configuration>
             </plugin>
@@ -128,7 +126,6 @@
             </activation>
             <properties>
                 <quarkus.package.type>native</quarkus.package.type>
-                <quarkus.native.additional-build-args>--language:js</quarkus.native.additional-build-args>
             </properties>
             <build>
                 <plugins>
@@ -142,6 +139,9 @@
                                     <goal>verify</goal>
                                 </goals>
                                 <configuration>
+                                    <environmentVariables>
+                                        <CAMEL_K_CONF>${project.basedir}/src/test/resources/test.properties</CAMEL_K_CONF>
+                                    </environmentVariables>
                                     <systemProperties>
                                         <quarkus.http.test-port>${test.http.port.native}</quarkus.http.test-port>
                                         <native.image.path>${project.build.directory}/${project.build.finalName}-runner</native.image.path>
@@ -155,5 +155,4 @@
         </profile>
     </profiles>
 
-
 </project>
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/java/org/apache/camel/k/quarkus/it/Application.java
similarity index 53%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/java/org/apache/camel/k/quarkus/it/Application.java
index 3135739..f3112a8 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-master/src/main/java/org/apache/camel/k/quarkus/master/Application.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/java/org/apache/camel/k/quarkus/it/Application.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.quarkus.master;
+package org.apache.camel.k.quarkus.it;
 
 import javax.enterprise.context.ApplicationScoped;
 import javax.inject.Inject;
@@ -22,28 +22,52 @@ import javax.json.Json;
 import javax.json.JsonObject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.component.kubernetes.cluster.KubernetesClusterService;
+import org.apache.camel.k.Runtime;
+import org.eclipse.microprofile.config.Config;
+
+import static org.apache.camel.k.quarkus.Application.instance;
 
 @Path("/test")
 @ApplicationScoped
 public class Application {
     @Inject
-    CamelContext context;
+    Config config;
 
     @GET
     @Path("/inspect")
     @Produces(MediaType.APPLICATION_JSON)
-    public JsonObject findCronInterceptor() {
-        KubernetesClusterService service = context.hasService(KubernetesClusterService.class);
-
+    public JsonObject inspect() {
         return Json.createObjectBuilder()
-            .add("cluster-service", service != null ? service.getClass().getName() : "")
-            .add("cluster-service-cm", service != null ? service.getConfigMapName() : "")
-
+            .add(
+                "camel-context",
+                instance(CamelContext.class)
+                    .map(Object::getClass)
+                    .map(Class::getName)
+                    .orElse(""))
+            .add(
+                "camel-k-runtime",
+                instance(Runtime.class)
+                    .map(Object::getClass)
+                    .map(Class::getName)
+                    .orElse(""))
+            .add(
+                "shutdown-task",
+                instance(org.apache.camel.k.quarkus.Application.ShutdownTask.class)
+                    .map(Object::getClass)
+                    .map(Class::getName)
+                    .orElse(""))
             .build();
     }
+
+    @GET
+    @Path("/property/{name}")
+    @Produces(MediaType.TEXT_PLAIN)
+    public String property(@PathParam("name") String name) {
+        return config.getValue(name, String.class);
+    }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/resources/application.properties
similarity index 93%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/resources/application.properties
index 3f6864a..0d67e06 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/main/resources/application.properties
@@ -20,9 +20,3 @@
 #
 quarkus.log.console.enable = false
 quarkus.banner.enabled     = false
-
-#
-# Quarkus :: Camel
-#
-quarkus.camel.routes-discovery.enabled = false
-
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeIT.java
similarity index 64%
copy from camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeIT.java
index 587dd9b..29f1c17 100644
--- a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeIT.java
@@ -14,18 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.quarkus;
+package org.apache.camel.k.quarkus.it;
 
-import java.util.List;
+import io.quarkus.test.junit.NativeImageTest;
 
-import io.quarkus.runtime.RuntimeValue;
-import io.quarkus.runtime.annotations.Recorder;
-import org.apache.camel.k.Runtime;
-import org.apache.camel.main.MainListener;
-
-@Recorder
-public class ApplicationRecorder {
-    public RuntimeValue<MainListener> createMainListener(List<Runtime.Listener> listeners) {
-        return new RuntimeValue<>(new ApplicationListenerAdapter(listeners));
-    }
-}
+@NativeImageTest
+public class RuntimeIT extends RuntimeTest {
+}
\ No newline at end of file
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeTest.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeTest.java
new file mode 100644
index 0000000..5689b73
--- /dev/null
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/java/org/apache/camel/k/quarkus/it/RuntimeTest.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.k.quarkus.it;
+
+import javax.ws.rs.core.MediaType;
+
+import io.quarkus.test.junit.QuarkusTest;
+import io.restassured.RestAssured;
+import io.restassured.path.json.JsonPath;
+import org.apache.camel.k.quarkus.Application;
+import org.apache.camel.k.quarkus.ApplicationProducers;
+import org.apache.camel.quarkus.core.FastCamelContext;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+@QuarkusTest
+public class RuntimeTest {
+    @Test
+    public void inspect() {
+        JsonPath p = RestAssured.given()
+            .accept(MediaType.APPLICATION_JSON)
+            .get("/test/inspect")
+            .then()
+                .statusCode(200)
+            .extract()
+                .body()
+                .jsonPath();
+
+        assertThat(p.getString("camel-context")).isEqualTo(FastCamelContext.class.getName());
+        assertThat(p.getString("camel-k-runtime")).isEqualTo(Application.Runtime.class.getName());
+        assertThat(p.getString("shutdown-task")).isEqualTo(ApplicationProducers.DefaultShutdownTask.class.getName());
+    }
+
+    @Test
+    public void configSourceProvider() {
+        String result = RestAssured.given()
+            .accept(MediaType.TEXT_PLAIN)
+            .get("/test/property/quarkus.my-property")
+            .then()
+                .statusCode(200)
+            .extract()
+                .body()
+                .asString();
+
+        assertThat(result).isEqualTo("my-test-value");
+    }
+}
\ No newline at end of file
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/resources/test.properties
similarity index 85%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/resources/test.properties
index 3f6864a..eb73887 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/resources/application.properties
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-runtime/src/test/resources/test.properties
@@ -15,14 +15,4 @@
 ## limitations under the License.
 ## ---------------------------------------------------------------------------
 
-#
-# Quarkus
-#
-quarkus.log.console.enable = false
-quarkus.banner.enabled     = false
-
-#
-# Quarkus :: Camel
-#
-quarkus.camel.routes-discovery.enabled = false
-
+quarkus.my-property = my-test-value
\ No newline at end of file
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/pom.xml b/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
index bc50525..dd4ddf1 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
@@ -30,6 +30,7 @@
 
     <modules>
         <module>camel-k-quarkus-itests-core</module>
+        <module>camel-k-quarkus-itests-runtime</module>
         <module>camel-k-quarkus-itests-cron</module>
         <module>camel-k-quarkus-itests-master</module>
         <module>camel-k-quarkus-itests-kamelet</module>
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/deployment/src/main/java/org/apache/camel/k/quarkus/deployment/DeploymentProcessor.java b/camel-k-quarkus/camel-k-runtime-quarkus/deployment/src/main/java/org/apache/camel/k/quarkus/deployment/DeploymentProcessor.java
index 076ca3c..f959492 100644
--- a/camel-k-quarkus/camel-k-runtime-quarkus/deployment/src/main/java/org/apache/camel/k/quarkus/deployment/DeploymentProcessor.java
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/deployment/src/main/java/org/apache/camel/k/quarkus/deployment/DeploymentProcessor.java
@@ -20,13 +20,20 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.ServiceLoader;
 
+import io.quarkus.arc.deployment.AdditionalBeanBuildItem;
+import io.quarkus.arc.deployment.BeanContainerBuildItem;
+import io.quarkus.arc.deployment.SyntheticBeansRuntimeInitBuildItem;
 import io.quarkus.deployment.annotations.BuildStep;
+import io.quarkus.deployment.annotations.Consume;
 import io.quarkus.deployment.annotations.ExecutionTime;
 import io.quarkus.deployment.annotations.Record;
 import io.quarkus.deployment.builditem.nativeimage.ReflectiveClassBuildItem;
 import org.apache.camel.k.Runtime;
+import org.apache.camel.k.quarkus.ApplicationProducers;
 import org.apache.camel.k.quarkus.ApplicationRecorder;
+import org.apache.camel.quarkus.core.deployment.spi.CamelRuntimeTaskBuildItem;
 import org.apache.camel.quarkus.main.CamelMainApplication;
+import org.apache.camel.quarkus.main.deployment.spi.CamelMainBuildItem;
 import org.apache.camel.quarkus.main.deployment.spi.CamelMainListenerBuildItem;
 
 public class DeploymentProcessor {
@@ -43,4 +50,24 @@ public class DeploymentProcessor {
 
         return new CamelMainListenerBuildItem(recorder.createMainListener(listeners));
     }
+
+    @Record(ExecutionTime.RUNTIME_INIT)
+    @BuildStep
+    @Consume(SyntheticBeansRuntimeInitBuildItem.class)
+    CamelRuntimeTaskBuildItem registerRuntime(
+            ApplicationRecorder recorder,
+            CamelMainBuildItem camelMain,
+            BeanContainerBuildItem beanContainer) {
+
+        recorder.publishRuntime(camelMain.getInstance(), beanContainer.getValue());
+
+        return new CamelRuntimeTaskBuildItem("camel-k-runtime");
+    }
+
+    @BuildStep
+    List<AdditionalBeanBuildItem> unremovableBeans() {
+       return List.of(
+           AdditionalBeanBuildItem.unremovableOf(ApplicationProducers.class)
+       );
+    }
 }
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/Application.java b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/Application.java
new file mode 100644
index 0000000..fb0392a
--- /dev/null
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/Application.java
@@ -0,0 +1,170 @@
+/*
+ * 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.quarkus;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import io.quarkus.arc.Arc;
+import io.quarkus.arc.ArcContainer;
+import io.quarkus.arc.InstanceHandle;
+import io.quarkus.runtime.Quarkus;
+import org.apache.camel.CamelContext;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.main.BaseMainSupport;
+import org.apache.camel.main.MainListener;
+
+public final class Application {
+    private Application() {
+    }
+
+    /**
+     * The camel-k runtime impl based on camel-quarkus
+     */
+    public static class Runtime implements org.apache.camel.k.Runtime {
+        private final BaseMainSupport main;
+        private final AtomicBoolean stopped;
+
+        public Runtime(BaseMainSupport main) {
+            this.main = main;
+            this.stopped = new AtomicBoolean();
+        }
+
+        @Override
+        public CamelContext getCamelContext() {
+            return main.getCamelContext();
+        }
+
+        @Override
+        public void addRoutes(RoutesBuilder builder) {
+            main.configure().addRoutesBuilder(builder);
+        }
+
+        @Override
+        public void addConfiguration(Object configuration) {
+            main.configure().addConfiguration(configuration);
+        }
+
+        @Override
+        public void setInitialProperties(Properties properties) {
+            main.setInitialProperties(properties);
+        }
+
+        @Override
+        public void setProperties(Properties properties) {
+            main.setOverrideProperties(properties);
+        }
+
+        @Override
+        public void stop() throws Exception {
+            if (!this.stopped.compareAndExchange(false, true)) {
+                instance(ShutdownTask.class).ifPresentOrElse(
+                    ShutdownTask::run,
+                    Quarkus::asyncExit);
+            }
+        }
+    }
+
+    /**
+     * Adapts main events to camel-k runtime lifecycle
+     */
+    public static class ListenerAdapter implements MainListener {
+        private final org.apache.camel.k.Runtime.Listener[] listeners;
+
+        public ListenerAdapter(List<org.apache.camel.k.Runtime.Listener> listeners) {
+            this.listeners = listeners.stream()
+                .sorted(Comparator.comparingInt(org.apache.camel.k.Runtime.Listener::getOrder))
+                .toArray(org.apache.camel.k.Runtime.Listener[]::new);
+        }
+
+        @Override
+        public void beforeInitialize(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.ConfigureProperties);
+        }
+
+        @Override
+        public void beforeConfigure(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.ConfigureRoutes);
+        }
+
+        @Override
+        public void afterConfigure(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.ConfigureContext);
+        }
+
+        @Override
+        public void configure(CamelContext context) {
+            // no-op
+        }
+
+        @Override
+        public void beforeStart(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.Starting);
+        }
+
+        @Override
+        public void afterStart(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.Started);
+        }
+
+        @Override
+        public void beforeStop(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.Stopping);
+        }
+
+        @Override
+        public void afterStop(BaseMainSupport main) {
+            invokeListeners(org.apache.camel.k.Runtime.Phase.Stopped);
+        }
+
+        private void invokeListeners(org.apache.camel.k.Runtime.Phase phase) {
+            org.apache.camel.k.Runtime runtime = instance(org.apache.camel.k.Runtime.class)
+                .orElseThrow(() -> new IllegalStateException("Unable to fine a Runtime instance"));
+
+            for (int i = 0; i < listeners.length; i ++) {
+                listeners[i].accept(phase, runtime);
+            }
+        }
+    }
+
+    /**
+     * Provide the task to be executed to shutdown the runtime
+     */
+    @FunctionalInterface
+    public interface ShutdownTask {
+        void run();
+    }
+
+    // *********************************
+    //
+    // Helpers
+    //
+    // *********************************
+
+    public static Optional<ArcContainer> container() {
+        return Optional.of(Arc.container());
+    }
+
+    public static <T> Optional<T> instance(Class<T> type) {
+        return container()
+            .map(container -> container.instance(type))
+            .map(InstanceHandle::get);
+    }
+}
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRuntimeConfigSourceProvider.java b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationConfigSourceProvider.java
similarity index 95%
rename from camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRuntimeConfigSourceProvider.java
rename to camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationConfigSourceProvider.java
index 302e0c8..cef2910 100644
--- a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRuntimeConfigSourceProvider.java
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationConfigSourceProvider.java
@@ -24,7 +24,7 @@ import org.apache.camel.k.support.PropertiesSupport;
 import org.eclipse.microprofile.config.spi.ConfigSource;
 import org.eclipse.microprofile.config.spi.ConfigSourceProvider;
 
-public class ApplicationRuntimeConfigSourceProvider implements ConfigSourceProvider {
+public class ApplicationConfigSourceProvider implements ConfigSourceProvider {
     @Override
     public Iterable<ConfigSource> getConfigSources(ClassLoader forClassLoader) {
         final Properties applicationProperties = PropertiesSupport.loadProperties();
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationListenerAdapter.java b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationListenerAdapter.java
deleted file mode 100644
index ccb01cc..0000000
--- a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationListenerAdapter.java
+++ /dev/null
@@ -1,140 +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.quarkus;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Properties;
-
-import io.quarkus.runtime.Quarkus;
-import org.apache.camel.CamelContext;
-import org.apache.camel.RoutesBuilder;
-import org.apache.camel.k.Runtime;
-import org.apache.camel.main.BaseMainSupport;
-import org.apache.camel.main.MainListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ApplicationListenerAdapter implements MainListener {
-    private static final Logger LOGGER = LoggerFactory.getLogger(ApplicationListenerAdapter.class);
-
-    private final List<Runtime.Listener> listeners;
-
-    public ApplicationListenerAdapter() {
-        this.listeners = new ArrayList<>();
-    }
-
-    public ApplicationListenerAdapter(List<Runtime.Listener> listeners) {
-        this.listeners = new ArrayList<>(listeners);
-    }
-
-    public void setListeners(List<Runtime.Listener> listeners) {
-        this.listeners.clear();
-        this.listeners.addAll(listeners);
-    }
-
-    @Override
-    public void beforeInitialize(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.ConfigureProperties);
-    }
-
-    @Override
-    public void beforeConfigure(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.ConfigureRoutes);
-    }
-
-    @Override
-    public void afterConfigure(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.ConfigureContext);
-    }
-
-    @Override
-    public void configure(CamelContext context) {
-        // no-op
-    }
-
-    @Override
-    public void beforeStart(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.Starting);
-    }
-
-    @Override
-    public void afterStart(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.Started);
-    }
-
-    @Override
-    public void beforeStop(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.Stopping);
-    }
-
-    @Override
-    public void afterStop(BaseMainSupport main) {
-        invokeListeners(listeners, on(main), Runtime.Phase.Stopped);
-    }
-
-    // ************************
-    //
-    // Helpers
-    //
-    // ************************
-
-    private static void invokeListeners(List<Runtime.Listener> listeners, Runtime runtime, Runtime.Phase phase) {
-        listeners.stream()
-            .sorted(Comparator.comparingInt(Runtime.Listener::getOrder))
-            .forEach(l -> {
-                if (l.accept(phase, runtime)) {
-                    LOGGER.debug("Listener {} executed in phase {}", l, phase);
-                }
-            });
-    }
-
-    private static Runtime on(BaseMainSupport main) {
-        return new Runtime() {
-            @Override
-            public CamelContext getCamelContext() {
-                return main.getCamelContext();
-            }
-
-            @Override
-            public void addRoutes(RoutesBuilder builder) {
-                main.configure().addRoutesBuilder(builder);
-            }
-
-            @Override
-            public void addConfiguration(Object configuration) {
-                main.configure().addConfiguration(configuration);
-            }
-
-            @Override
-            public void setInitialProperties(Properties properties) {
-                main.setInitialProperties(properties);
-            }
-
-            @Override
-            public void setProperties(Properties properties) {
-                main.setOverrideProperties(properties);
-            }
-
-            @Override
-            public void stop() throws Exception {
-                Quarkus.asyncExit();
-            }
-        };
-    }
-}
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationProducers.java
similarity index 51%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java
copy to camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationProducers.java
index 3d8cb21..f44f3c9 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-cron/src/main/java/org/apache/camel/k/quarkus/cron/deployment/Application.java
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationProducers.java
@@ -14,33 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.quarkus.cron.deployment;
+package org.apache.camel.k.quarkus;
 
 import javax.enterprise.context.ApplicationScoped;
-import javax.inject.Inject;
-import javax.ws.rs.GET;
-import javax.ws.rs.Path;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
+import javax.enterprise.inject.Produces;
+import javax.inject.Singleton;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.ExtendedCamelContext;
-import org.apache.camel.k.Constants;
+import io.quarkus.arc.DefaultBean;
+import io.quarkus.arc.Unremovable;
+import io.quarkus.runtime.Quarkus;
+import org.apache.camel.k.Runtime;
 
-@Path("/test")
 @ApplicationScoped
-public class Application {
-    @Inject
-    CamelContext context;
+public class ApplicationProducers {
+    private volatile Runtime runtime;
 
-    @GET
-    @Path("/find-cron-interceptor")
-    @Produces(MediaType.TEXT_PLAIN)
-    public String findCronInterceptor() {
-        return context.adapt(ExtendedCamelContext.class)
-            .getFactoryFinder(Constants.SOURCE_LOADER_INTERCEPTOR_RESOURCE_PATH)
-            .findClass("cron")
-            .map(Class::getName)
-            .orElse("");
+    public void setRuntime(Runtime runtime) {
+        this.runtime = runtime;
+    }
+
+    @Unremovable
+    @Singleton
+    @Produces
+    Runtime runtime() {
+        return this.runtime;
+    }
+
+    @Unremovable
+    @DefaultBean
+    @Singleton
+    @Produces
+    Application.ShutdownTask shutdownTask() {
+        return new DefaultShutdownTask();
+    }
+
+    public static class DefaultShutdownTask implements Application.ShutdownTask {
+        @Override
+        public void run() {
+            Quarkus.asyncExit();
+        }
     }
 }
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java
index 587dd9b..e4ed9a8 100644
--- a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/java/org/apache/camel/k/quarkus/ApplicationRecorder.java
@@ -18,14 +18,20 @@ package org.apache.camel.k.quarkus;
 
 import java.util.List;
 
+import io.quarkus.arc.runtime.BeanContainer;
 import io.quarkus.runtime.RuntimeValue;
 import io.quarkus.runtime.annotations.Recorder;
 import org.apache.camel.k.Runtime;
 import org.apache.camel.main.MainListener;
+import org.apache.camel.quarkus.main.CamelMain;
 
 @Recorder
 public class ApplicationRecorder {
     public RuntimeValue<MainListener> createMainListener(List<Runtime.Listener> listeners) {
-        return new RuntimeValue<>(new ApplicationListenerAdapter(listeners));
+        return new RuntimeValue<>(new Application.ListenerAdapter(listeners));
+    }
+
+    public void publishRuntime(RuntimeValue<CamelMain> main, BeanContainer container) {
+        container.instance(ApplicationProducers.class).setRuntime(new Application.Runtime(main.getValue()));
     }
 }
diff --git a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/resources/META-INF/services/org.eclipse.microprofile.config.spi.ConfigSourceProvider b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/resources/META-INF/services/org.eclipse.microprofile.config.spi.ConfigSourceProvider
index 30fe91e..1079ba1 100644
--- a/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/resources/META-INF/services/org.eclipse.microprofile.config.spi.ConfigSourceProvider
+++ b/camel-k-quarkus/camel-k-runtime-quarkus/runtime/src/main/resources/META-INF/services/org.eclipse.microprofile.config.spi.ConfigSourceProvider
@@ -14,4 +14,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-org.apache.camel.k.quarkus.ApplicationRuntimeConfigSourceProvider
\ No newline at end of file
+org.apache.camel.k.quarkus.ApplicationConfigSourceProvider
\ No newline at end of file
diff --git a/camel-k-runtime-cron/src/main/java/org/apache/camel/k/cron/CronSourceLoaderInterceptor.java b/camel-k-runtime-cron/src/main/java/org/apache/camel/k/cron/CronSourceLoaderInterceptor.java
index d3026d1..3c965ac 100644
--- a/camel-k-runtime-cron/src/main/java/org/apache/camel/k/cron/CronSourceLoaderInterceptor.java
+++ b/camel-k-runtime-cron/src/main/java/org/apache/camel/k/cron/CronSourceLoaderInterceptor.java
@@ -30,6 +30,7 @@ import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.spi.CamelEvent;
 import org.apache.camel.spi.Configurer;
 import org.apache.camel.support.EventNotifierSupport;
+import org.apache.camel.support.service.ServiceHelper;
 import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -100,7 +101,10 @@ public class CronSourceLoaderInterceptor implements SourceLoader.Interceptor, Ru
                             // Don't install the shutdown strategy more than once.
                             //
                             if (context.getManagementStrategy().getEventNotifiers().stream().noneMatch(CronShutdownStrategy.class::isInstance)) {
-                                context.getManagementStrategy().addEventNotifier(new CronShutdownStrategy(runtime));
+                                CronShutdownStrategy strategy = new CronShutdownStrategy(runtime);
+                                ServiceHelper.startService(strategy);
+
+                                context.getManagementStrategy().addEventNotifier(strategy);
                             }
                         }
                     }
diff --git a/camel-k-runtime-knative/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.loader.knative.KnativeSourceLoaderInterceptor b/camel-k-runtime-knative/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.loader.knative.KnativeSourceLoaderInterceptor
deleted file mode 100644
index 50833dc..0000000
--- a/camel-k-runtime-knative/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.loader.knative.KnativeSourceLoaderInterceptor
+++ /dev/null
@@ -1,2 +0,0 @@
-# Generated by camel build tools - do NOT edit this file!
-class=org.apache.camel.k.loader.knative.KnativeSourceLoaderInterceptorConfigurer
diff --git a/examples/camel-k-runtime-example-quarkus-cron/data/application.properties b/examples/camel-k-runtime-example-quarkus-cron/data/application.properties
index 339b6c4..c32de63 100644
--- a/examples/camel-k-runtime-example-quarkus-cron/data/application.properties
+++ b/examples/camel-k-runtime-example-quarkus-cron/data/application.properties
@@ -30,6 +30,6 @@ camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-q
 #
 # Camel K
 #
-camel.k.loader.interceptor.cron.overridable-components = timer_
+camel.k.loader.interceptor.cron.overridable-components = timer
 
 
diff --git a/pom.xml b/pom.xml
index fb3c0fe..a7febd7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -619,6 +619,12 @@
             </dependency>
 
             <dependency>
+                <groupId>org.awaitility</groupId>
+                <artifactId>awaitility</artifactId>
+                <version>${awaitility-version}</version>
+            </dependency>
+
+            <dependency>
                 <groupId>org.codehaus.groovy</groupId>
                 <artifactId>groovy</artifactId>
                 <version>${groovy-version}</version>