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/14 11:13:10 UTC

[camel-k-runtime] 03/11: kamelets: create a camel-kamelet component #375

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 5452484afe08e8758b85cf497663e9b30b1c4f89
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Mon Aug 10 18:53:27 2020 +0200

    kamelets: create a camel-kamelet component #375
---
 .github/workflows/ci-build.yml                     |   1 +
 .../k/loader/groovy/GroovySourceLoaderTest.groovy  |  23 +-
 .../k/loader/groovy/support/TestRuntime.groovy     |   7 +-
 .../camel/k/loader/java/support/TestRuntime.groovy |   7 +-
 .../camel/k/loader/js/support/TestRuntime.groovy   |   7 +-
 .../kotlin/itests/KotlinSourceLoaderTest.java      |   4 +-
 .../camel/k/loader/kotlin/support/TestRuntime.kt   |   6 +-
 .../camel/k/loader/xml/support/TestRuntime.groovy  |   7 +-
 camel-k-main/camel-k-runtime-main/pom.xml          |   5 +
 .../apache/camel/k/main/ApplicationRuntime.java    |  15 +-
 .../java/org/apache/camel/k/main/RuntimeTest.java  |  68 ++++--
 .../org/apache/camel/k/main/TemplatesTest.java     |  89 ++++++++
 .../apache/camel/k/main/{ => support}/MyBean.java  |   2 +-
 .../camel/k/main/support/RuntimeTestSupport.java   |  48 ++++
 .../camel/k/main/{ => support}/TestCustomizer.java |   2 +-
 .../services/org/apache/camel/k/customizer/test    |   2 +-
 ...MyRoutesWithBeans.java => MyRouteTemplate.java} |  15 +-
 .../src/test/resources/MyRoutesWithBeans.java      |   5 +-
 ...tesWithBeans.java => MyRoutesWithKamelets.java} |  18 +-
 .../quarkus/deployment/DeploymentProcessor.java    |  63 +++---
 .../core/quarkus/deployment/DeploymentSupport.java |  62 +++++-
 .../camel/k/core/quarkus/deployment/CoreTest.java  |   4 +-
 .../camel-k-quarkus-itests-kamelet/pom.xml         | 155 +++++++++++++
 .../k/quarkus/kamelet/deployment/Application.java  |  83 +++++++
 .../src/main/resources}/application.properties     |  12 +-
 .../k/quarkus/kamelet/deployment/KameletIT.java    |  19 +-
 .../k/quarkus/kamelet/deployment/KameletTest.java} |  44 ++--
 camel-k-quarkus/camel-k-quarkus-itests/pom.xml     |   1 +
 .../camel-k-quarkus-kamelet/deployment}/pom.xml    |  55 ++---
 .../k/quarkus/kamelet/deployment/Feature.java      |  17 +-
 .../pom.xml                                        |  15 +-
 .../camel-k-quarkus-kamelet/runtime}/pom.xml       |  71 +++---
 .../k/quarkus/deployment/DeploymentProcessor.java  |  15 +-
 .../k/quarkus/ApplicationListenerAdapter.java      |   2 +-
 camel-k-quarkus/pom.xml                            |   1 +
 camel-k-runtime-bom/pom.xml                        |  10 +
 camel-k-runtime-core/pom.xml                       |  39 ++++
 .../apache/camel/k/SourceDefinitionConfigurer.java | 101 +++++++++
 .../k/listener/SourcesConfigurerConfigurer.java    |  45 ++++
 .../configurer/org.apache.camel.k.SourceDefinition |   2 +
 .../org.apache.camel.k.listener.SourcesConfigurer  |   2 +
 .../src/main/java/org/apache/camel/k/Source.java   |   5 +-
 .../java/org/apache/camel/k/SourceDefinition.java  | 247 +++++++++++++++++++++
 .../main/java/org/apache/camel/k/SourceLoader.java |   7 +-
 .../main/java/org/apache/camel/k/SourceType.java   |  22 +-
 .../src/main/java/org/apache/camel/k/Sources.java  | 194 ++++++----------
 .../apache/camel/k/listener/RoutesConfigurer.java  | 107 ---------
 .../apache/camel/k/listener/SourcesConfigurer.java |  77 +++++++
 .../apache/camel/k/support/PropertiesSupport.java  |  62 +++++-
 .../org/apache/camel/k/support/RuntimeSupport.java |  20 +-
 .../org/apache/camel/k/support/SourcesSupport.java | 164 ++++++++++++++
 .../org/apache/camel/k/support/StringSupport.java  |  40 ++--
 .../services/org.apache.camel.k.Runtime$Listener   |   2 +-
 .../test/java/org/apache/camel/k/SourceTest.java   |  10 +
 .../camel/k/support/PropertiesSupportTest.java     |  96 ++++++++
 .../java/org/apache/camel/k/cron/CronTest.java     |   4 +-
 .../KnativeSinkBindingCustomizerTest.java          |   4 +-
 .../knative/yaml/parser/KnativeConverterTest.java  |   4 +-
 .../knative/KnativeSourceRoutesLoaderTest.java     |   4 +-
 .../org/apache/camel/k/webhook/WebhookTest.java    |   8 +-
 camel-kamelet/pom.xml                              |  13 +-
 .../apache/camel/component/kamelet/kamelet.json    |   6 +-
 .../apache/camel/component/kamelet/Kamelet.java    |  67 ++++++
 .../camel/component/kamelet/KameletComponent.java  | 169 +++++---------
 .../camel/component/kamelet/KameletEndpoint.java   |  18 +-
 .../kamelet/KameletAddAfterCamelStartedTest.java   |  76 -------
 .../camel/component/kamelet/KameletBasicTest.java  |  94 ++++++++
 .../component/kamelet/KameletPropertiesTest.java   |  86 +++++++
 ...KameletTest.java => KameletValidationTest.java} |  45 +---
 camel-kamelet/src/test/resources/log4j2-test.xml   |   7 +-
 .../data/application.properties                    |  11 +-
 examples/camel-k-runtime-example-groovy/pom.xml    |   4 +-
 .../data/application.properties                    |  21 +-
 .../data/logger.yaml                               |   7 +-
 .../data/routes.groovy                             |  20 +-
 .../pom.xml                                        |  16 +-
 .../Dockerfile                                     |   7 +
 .../data/application.properties                    |  21 +-
 .../data/logger.yaml                               |   7 +-
 .../data/routes.yaml                               |   8 +-
 .../pom.xml                                        | 161 ++++++++++++++
 .../src/main/resources}/application.properties     |  17 +-
 examples/pom.xml                                   |   2 +
 pom.xml                                            |  38 +---
 .../maven/processors/CatalogProcessor3x.java       |  17 ++
 .../org/apache/camel/k/test/CamelKTestSupport.java |  45 ++--
 86 files changed, 2298 insertions(+), 911 deletions(-)

diff --git a/.github/workflows/ci-build.yml b/.github/workflows/ci-build.yml
index 44d4c56..7a89f03 100644
--- a/.github/workflows/ci-build.yml
+++ b/.github/workflows/ci-build.yml
@@ -78,6 +78,7 @@ jobs:
         native-image-project:
           - :camel-k-quarkus-itests-core
           - :camel-k-quarkus-itests-cron
+          - :camel-k-quarkus-itests-kamelet
           - :camel-k-quarkus-itests-knative
           - :camel-k-quarkus-itests-loader-xml
           - :camel-k-quarkus-itests-loader-yaml
diff --git a/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/GroovySourceLoaderTest.groovy b/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/GroovySourceLoaderTest.groovy
index 66ea733..5691bb9 100644
--- a/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/GroovySourceLoaderTest.groovy
+++ b/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/GroovySourceLoaderTest.groovy
@@ -22,13 +22,10 @@ import org.apache.camel.RuntimeCamelException
 import org.apache.camel.component.jackson.JacksonDataFormat
 import org.apache.camel.component.log.LogComponent
 import org.apache.camel.component.seda.SedaComponent
-import org.apache.camel.k.Sources
-import org.apache.camel.k.listener.RoutesConfigurer
 import org.apache.camel.k.loader.groovy.support.MyBean
 import org.apache.camel.k.loader.groovy.support.TestRuntime
 import org.apache.camel.language.bean.BeanLanguage
 import org.apache.camel.model.FromDefinition
-import org.apache.camel.model.ModelCamelContext
 import org.apache.camel.model.ToDefinition
 import org.apache.camel.model.rest.GetVerbDefinition
 import org.apache.camel.model.rest.PostVerbDefinition
@@ -45,16 +42,10 @@ class GroovySourceLoaderTest extends Specification {
     def runtime = new TestRuntime()
 
     def "load routes"() {
-        given:
-            def source = Sources.fromURI("classpath:routes.groovy")
-
         when:
-            def loader = RoutesConfigurer.load(runtime, source)
-
+            runtime.loadRoutes("classpath:routes.groovy")
         then:
-            loader instanceof GroovySourceLoader
-
-            with(runtime.getCamelContext(ModelCamelContext.class).routeDefinitions) {
+            with(runtime.context.routeDefinitions) {
                 it.size() == 1
 
                 it[0].outputs[0] instanceof ToDefinition
@@ -63,16 +54,10 @@ class GroovySourceLoaderTest extends Specification {
     }
 
     def "load routes with endpoint dsl"() {
-        given:
-            def source = Sources.fromURI("classpath:routes-with-endpoint-dsl.groovy")
-
         when:
-            def loader = RoutesConfigurer.load(runtime, source)
-
+            runtime.loadRoutes("classpath:routes-with-endpoint-dsl.groovy")
         then:
-            loader instanceof GroovySourceLoader
-
-            with(runtime.getCamelContext(ModelCamelContext.class).routeDefinitions) {
+            with(runtime.context.routeDefinitions) {
                 it.size() == 1
 
                 with(it[0].input, FromDefinition) {
diff --git a/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/support/TestRuntime.groovy b/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/support/TestRuntime.groovy
index 33ec1e9..53a41c6 100644
--- a/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/support/TestRuntime.groovy
+++ b/camel-k-loader-groovy/src/test/groovy/org/apache/camel/k/loader/groovy/support/TestRuntime.groovy
@@ -20,10 +20,9 @@ import org.apache.camel.CamelContext
 import org.apache.camel.RoutesBuilder
 import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.Runtime
+import org.apache.camel.k.support.SourcesSupport
 import org.apache.camel.model.ModelCamelContext
 
-import static org.apache.camel.k.listener.RoutesConfigurer.forRoutes
-
 class TestRuntime implements Runtime, AutoCloseable {
     public final ModelCamelContext context
     public final List<RoutesBuilder> builders
@@ -45,9 +44,7 @@ class TestRuntime implements Runtime, AutoCloseable {
     }
 
     void loadRoutes(String... routes) {
-        routes.each {
-            forRoutes(it).accept(Phase.ConfigureRoutes, this)
-        }
+        SourcesSupport.loadSources(this, routes)
     }
 
     void start() {
diff --git a/camel-k-loader-java/src/test/groovy/org/apache/camel/k/loader/java/support/TestRuntime.groovy b/camel-k-loader-java/src/test/groovy/org/apache/camel/k/loader/java/support/TestRuntime.groovy
index 21f16e1..6d8f2ea 100644
--- a/camel-k-loader-java/src/test/groovy/org/apache/camel/k/loader/java/support/TestRuntime.groovy
+++ b/camel-k-loader-java/src/test/groovy/org/apache/camel/k/loader/java/support/TestRuntime.groovy
@@ -21,10 +21,9 @@ import org.apache.camel.RoutesBuilder
 import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.CompositeClassloader
 import org.apache.camel.k.Runtime
+import org.apache.camel.k.support.SourcesSupport
 import org.apache.camel.model.ModelCamelContext
 
-import static org.apache.camel.k.listener.RoutesConfigurer.forRoutes
-
 class TestRuntime implements Runtime, AutoCloseable {
     final ModelCamelContext context
     final List<RoutesBuilder> builders
@@ -54,9 +53,7 @@ class TestRuntime implements Runtime, AutoCloseable {
     }
 
     void loadRoutes(String... routes) {
-        routes.each {
-            forRoutes(it).accept(Phase.ConfigureRoutes, this)
-        }
+        SourcesSupport.loadSources(this, routes)
     }
 
     void start() {
diff --git a/camel-k-loader-js/src/test/groovy/org/apache/camel/k/loader/js/support/TestRuntime.groovy b/camel-k-loader-js/src/test/groovy/org/apache/camel/k/loader/js/support/TestRuntime.groovy
index 8040d45..92a9ed7 100644
--- a/camel-k-loader-js/src/test/groovy/org/apache/camel/k/loader/js/support/TestRuntime.groovy
+++ b/camel-k-loader-js/src/test/groovy/org/apache/camel/k/loader/js/support/TestRuntime.groovy
@@ -22,10 +22,9 @@ import org.apache.camel.RoutesBuilder
 import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.CompositeClassloader
 import org.apache.camel.k.Runtime
+import org.apache.camel.k.support.SourcesSupport
 import org.apache.camel.model.ModelCamelContext
 
-import static org.apache.camel.k.listener.RoutesConfigurer.forRoutes
-
 class TestRuntime implements Runtime, AutoCloseable {
     final ModelCamelContext context
     final FluentProducerTemplate template
@@ -57,9 +56,7 @@ class TestRuntime implements Runtime, AutoCloseable {
     }
 
     void loadRoutes(String... routes) {
-        routes.each {
-            forRoutes(it).accept(Phase.ConfigureRoutes, this)
-        }
+        SourcesSupport.loadSources(this, routes)
     }
 
     void start() {
diff --git a/camel-k-loader-kotlin/camel-k-loader-kotlin-itests/src/test/java/org/apache/camel/k/loader/kotlin/itests/KotlinSourceLoaderTest.java b/camel-k-loader-kotlin/camel-k-loader-kotlin-itests/src/test/java/org/apache/camel/k/loader/kotlin/itests/KotlinSourceLoaderTest.java
index b0b5f0a..994d87f 100644
--- a/camel-k-loader-kotlin/camel-k-loader-kotlin-itests/src/test/java/org/apache/camel/k/loader/kotlin/itests/KotlinSourceLoaderTest.java
+++ b/camel-k-loader-kotlin/camel-k-loader-kotlin-itests/src/test/java/org/apache/camel/k/loader/kotlin/itests/KotlinSourceLoaderTest.java
@@ -21,7 +21,7 @@ import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.k.Runtime;
 import org.apache.camel.k.Source;
 import org.apache.camel.k.Sources;
-import org.apache.camel.k.listener.RoutesConfigurer;
+import org.apache.camel.k.support.SourcesSupport;
 import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
@@ -33,7 +33,7 @@ public class KotlinSourceLoaderTest {
         final Runtime runtime = Runtime.on(context);
         final Source source = Sources.fromURI("classpath:routes.kts");
 
-        RoutesConfigurer.load(runtime, source);
+        SourcesSupport.load(runtime, source);
 
         try {
             context.start();
diff --git a/camel-k-loader-kotlin/camel-k-loader-kotlin/src/test/kotlin/org/apache/camel/k/loader/kotlin/support/TestRuntime.kt b/camel-k-loader-kotlin/camel-k-loader-kotlin/src/test/kotlin/org/apache/camel/k/loader/kotlin/support/TestRuntime.kt
index eefa40f..85fa391 100644
--- a/camel-k-loader-kotlin/camel-k-loader-kotlin/src/test/kotlin/org/apache/camel/k/loader/kotlin/support/TestRuntime.kt
+++ b/camel-k-loader-kotlin/camel-k-loader-kotlin/src/test/kotlin/org/apache/camel/k/loader/kotlin/support/TestRuntime.kt
@@ -22,7 +22,7 @@ import org.apache.camel.FluentProducerTemplate
 import org.apache.camel.RoutesBuilder
 import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.CompositeClassloader
-import org.apache.camel.k.listener.RoutesConfigurer.forRoutes
+import org.apache.camel.k.support.SourcesSupport
 import org.apache.camel.model.ModelCamelContext
 import java.util.ArrayList
 
@@ -54,9 +54,7 @@ class TestRuntime : Runtime {
     }
 
     fun loadRoutes(vararg routes: String) {
-        for (route in routes) {
-            forRoutes(route).accept(Runtime.Phase.ConfigureRoutes, this)
-        }
+        SourcesSupport.loadSources(this, *routes)
     }
 
     fun start() {
diff --git a/camel-k-loader-xml/src/test/groovy/org/apache/camel/k/loader/xml/support/TestRuntime.groovy b/camel-k-loader-xml/src/test/groovy/org/apache/camel/k/loader/xml/support/TestRuntime.groovy
index b62e678..2d24ddf 100644
--- a/camel-k-loader-xml/src/test/groovy/org/apache/camel/k/loader/xml/support/TestRuntime.groovy
+++ b/camel-k-loader-xml/src/test/groovy/org/apache/camel/k/loader/xml/support/TestRuntime.groovy
@@ -22,10 +22,9 @@ import org.apache.camel.RoutesBuilder
 import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.CompositeClassloader
 import org.apache.camel.k.Runtime
+import org.apache.camel.k.support.SourcesSupport
 import org.apache.camel.model.ModelCamelContext
 
-import static org.apache.camel.k.listener.RoutesConfigurer.forRoutes
-
 class TestRuntime implements Runtime, AutoCloseable {
     final ModelCamelContext context
     final FluentProducerTemplate template
@@ -57,9 +56,7 @@ class TestRuntime implements Runtime, AutoCloseable {
     }
 
     void loadRoutes(String... routes) {
-        routes.each {
-            forRoutes(it).accept(Phase.ConfigureRoutes, this)
-        }
+        SourcesSupport.loadSources(this, routes)
     }
 
     void start() {
diff --git a/camel-k-main/camel-k-runtime-main/pom.xml b/camel-k-main/camel-k-runtime-main/pom.xml
index fb183c4..65cf40c 100644
--- a/camel-k-main/camel-k-runtime-main/pom.xml
+++ b/camel-k-main/camel-k-runtime-main/pom.xml
@@ -153,6 +153,11 @@
             <artifactId>camel-knative-test</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-kamelet</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/camel-k-main/camel-k-runtime-main/src/main/java/org/apache/camel/k/main/ApplicationRuntime.java b/camel-k-main/camel-k-runtime-main/src/main/java/org/apache/camel/k/main/ApplicationRuntime.java
index 93855b2..05cb0b2 100644
--- a/camel-k-main/camel-k-runtime-main/src/main/java/org/apache/camel/k/main/ApplicationRuntime.java
+++ b/camel-k-main/camel-k-runtime-main/src/main/java/org/apache/camel/k/main/ApplicationRuntime.java
@@ -30,14 +30,12 @@ import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.k.CompositeClassloader;
 import org.apache.camel.k.Runtime;
-import org.apache.camel.k.support.PropertiesSupport;
 import org.apache.camel.main.BaseMainSupport;
 import org.apache.camel.main.MainSupport;
 import org.apache.camel.main.RoutesCollector;
 import org.apache.camel.model.RouteTemplatesDefinition;
 import org.apache.camel.model.RoutesDefinition;
 import org.apache.camel.model.rest.RestsDefinition;
-import org.apache.camel.spi.HasId;
 import org.apache.camel.util.function.ThrowingConsumer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -103,16 +101,7 @@ public final class ApplicationRuntime implements Runtime {
     }
 
     public void addListener(Runtime.Listener listener) {
-        if (listener instanceof HasId) {
-            String id = ((HasId) listener).getId();
-            if (!id.endsWith(".")) {
-                id = id + ".";
-            }
-
-            PropertiesSupport.bindProperties(getCamelContext(), listener, id);
-        }
-
-        LOGGER.info("Add listener: {}", listener);
+        LOGGER.debug("Add listener: {}", listener);
 
         this.listeners.add(listener);
     }
@@ -178,7 +167,7 @@ public final class ApplicationRuntime implements Runtime {
                 .sorted(Comparator.comparingInt(Listener::getOrder))
                 .forEach(l -> {
                     if (l.accept(phase, ApplicationRuntime.this)) {
-                        LOGGER.info("Listener {} executed in phase {}", l, phase);
+                        LOGGER.debug("Listener {} executed in phase {}", l, phase);
                     }
                 });
         }
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/RuntimeTest.java b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/RuntimeTest.java
index 872de35..8197df6 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/RuntimeTest.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/RuntimeTest.java
@@ -27,7 +27,9 @@ import org.apache.camel.component.knative.test.KnativeEnvironmentSupport;
 import org.apache.camel.k.Runtime;
 import org.apache.camel.k.http.PlatformHttpServiceContextCustomizer;
 import org.apache.camel.k.listener.ContextConfigurer;
-import org.apache.camel.k.listener.RoutesConfigurer;
+import org.apache.camel.k.listener.SourcesConfigurer;
+import org.apache.camel.k.main.support.MyBean;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.k.test.AvailablePortFinder;
 import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.ToDefinition;
@@ -53,10 +55,11 @@ public class RuntimeTest {
             runtime.stop();
         }
     }
+
     @Test
     void testLoadMultipleRoutes() throws Exception {
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:r1.js", "classpath:r2.mytype?language=js"));
+        runtime.addListener(SourcesSupport.forRoutes("classpath:r1.js", "classpath:r2.mytype?language=js"));
         runtime.addListener(Runtime.Phase.Started, r -> {
             CamelContext context = r.getCamelContext();
             List<Route> routes = context.getRoutes();
@@ -74,12 +77,12 @@ public class RuntimeTest {
     @Test
     void testLoadRouteAndRest() throws Exception {
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:routes.xml", "classpath:rests.xml"));
+        runtime.addListener(SourcesSupport.forRoutes("classpath:routes.xml", "classpath:rests.xml"));
         runtime.addListener(Runtime.Phase.Started, r -> {
-            CamelContext context = r.getCamelContext();
+            ModelCamelContext context = r.getCamelContext(ModelCamelContext.class);
 
-            assertThat(context.adapt(ModelCamelContext.class).getRouteDefinitions()).isNotEmpty();
-            assertThat(context.adapt(ModelCamelContext.class).getRestDefinitions()).isNotEmpty();
+            assertThat(context.getRouteDefinitions()).isNotEmpty();
+            assertThat(context.getRestDefinitions()).isNotEmpty();
 
             runtime.stop();
         });
@@ -94,16 +97,55 @@ public class RuntimeTest {
         ));
 
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:routes-with-expression.xml"));
-        runtime.addListener(Runtime.Phase.Started, r -> runtime.stop());
+        runtime.addListener(SourcesSupport.forRoutes("classpath:routes-with-expression.xml"));
+        runtime.addListener(Runtime.Phase.Started, Runtime::stop);
         runtime.run();
     }
 
     @Test
     public void testLoadJavaSource() throws Exception {
-        ApplicationRuntime runtime = new ApplicationRuntime();
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:MyRoutesWithBeans.java", "classpath:MyRoutesConfig.java"));
-        runtime.addListener(Runtime.Phase.Started, r ->  runtime.stop());
+        runtime.addListener(SourcesSupport.forRoutes("classpath:MyRoutesWithBeans.java", "classpath:MyRoutesConfig.java"));
+        runtime.addListener(Runtime.Phase.Started, r -> {
+            assertThat(runtime.getCamelContext().getRoutes()).hasSize(1);
+            assertThat(runtime.getRegistry().lookupByName("my-processor")).isNotNull();
+            assertThat(runtime.getRegistry().lookupByName("my-bean")).isInstanceOfSatisfying(MyBean.class, b -> {
+                assertThat(b).hasFieldOrPropertyWithValue("name", "my-bean-name");
+            });
+            r.stop();
+        });
+        runtime.run();
+    }
+
+    @Test
+    public void testLoadJavaSourceFromProperties() throws Exception {
+        runtime.setInitialProperties(
+            "camel.k.sources[0].name", "MyRoutesWithBeans",
+            "camel.k.sources[0].location", "classpath:MyRoutesWithBeans.java",
+            "camel.k.sources[0].language", "java",
+            "camel.k.sources[1].name", "MyRoutesConfig",
+            "camel.k.sources[1].location", "classpath:MyRoutesConfig.java",
+            "camel.k.sources[1].language", "java"
+        );
+        runtime.addListener(new SourcesConfigurer());
+        runtime.addListener(Runtime.Phase.Started, r -> {
+            assertThat(runtime.getCamelContext().getRoutes()).hasSize(1);
+            assertThat(runtime.getRegistry().lookupByName("my-processor")).isNotNull();
+            assertThat(runtime.getRegistry().lookupByName("my-bean")).isInstanceOfSatisfying(MyBean.class, b -> {
+                assertThat(b).hasFieldOrPropertyWithValue("name", "my-bean-name");
+            });
+            r.stop();
+        });
+        runtime.run();
+    }
+
+    @Test
+    public void testLoadJavaSourceFromSimpleProperties() throws Exception {
+        runtime.setInitialProperties(
+            "camel.k.sources[0].location", "classpath:MyRoutesWithBeans.java",
+            "camel.k.sources[1].location", "classpath:MyRoutesConfig.java"
+        );
+        runtime.addListener(new SourcesConfigurer());
+        runtime.addListener(Runtime.Phase.Started, Runtime::stop);
         runtime.run();
 
         assertThat(runtime.getRegistry().lookupByName("my-processor")).isNotNull();
@@ -124,8 +166,8 @@ public class RuntimeTest {
         phsc.apply(runtime.getCamelContext());
 
         runtime.getCamelContext().addComponent("knative", component);
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:MyRoutesWithBeans.java?interceptors=knative-source"));
-        runtime.addListener(Runtime.Phase.Started, r ->  runtime.stop());
+        runtime.addListener(SourcesSupport.forRoutes("classpath:MyRoutesWithBeans.java?interceptors=knative-source"));
+        runtime.addListener(Runtime.Phase.Started, Runtime::stop);
         runtime.run();
 
         assertThat(runtime.getRegistry().lookupByName("my-bean")).isInstanceOfSatisfying(MyBean.class, b -> {
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TemplatesTest.java b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TemplatesTest.java
new file mode 100644
index 0000000..c7215a4
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TemplatesTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.main;
+
+import java.util.Map;
+
+import org.apache.camel.Route;
+import org.apache.camel.k.SourceType;
+import org.apache.camel.k.main.support.RuntimeTestSupport;
+import org.apache.camel.model.ModelCamelContext;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TemplatesTest {
+    @Test
+    public void templatesCanBeLoadedAndMaterialized() throws Exception {
+        RuntimeTestSupport.run(
+            Map.of(
+                "camel.k.sources[0].id", "my-template",
+                "camel.k.sources[0].location", "classpath:MyRouteTemplate.java",
+                "camel.k.sources[0].type", SourceType.template.name(),
+                "camel.k.sources[0].property-names[0]", "message"
+            ),
+            runtime -> {
+                var context = runtime.getCamelContext(ModelCamelContext.class);
+
+                assertThat(context.getRouteTemplateDefinitions()).hasSize(1);
+                assertThat(context.getRouteDefinitions()).isEmpty();
+
+                context.addRouteFromTemplate("myRoute", "my-template", Map.of("message", "test"));
+
+                assertThat(context.getRoutes())
+                    .hasSize(1)
+                    .first().hasFieldOrPropertyWithValue("id", "myRoute");
+            }
+        );
+    }
+
+    @Test
+    public void templatesCanBeLoadedAndMaterializedByKamelets() throws Exception {
+        RuntimeTestSupport.run(
+            Map.of(
+                // template
+                "camel.k.sources[0].id", "my-template",
+                "camel.k.sources[0].location", "classpath:MyRouteTemplate.java",
+                "camel.k.sources[0].type", SourceType.template.name(),
+                "camel.k.sources[0].property-names[0]", "message",
+                // route
+                "camel.k.sources[1].location", "classpath:MyRoutesWithKamelets.java",
+                "camel.k.sources[1].type", SourceType.source.name(),
+                // props
+                "camel.kamelet.my-template.message", "default-message"
+            ),
+            runtime -> {
+                var context = runtime.getCamelContext(ModelCamelContext.class);
+
+                // templates
+                assertThat(context.getRouteTemplateDefinitions()).hasSize(1);
+
+                // 2 routes defined in MyRoutesWithKamelets
+                // 2 routes materialized from templates by camel-kamelet
+                assertThat(context.getRouteDefinitions()).hasSize(4);
+
+                assertThat(context.getRoutes())
+                    .hasSize(4)
+                    .extracting(Route::getId)
+                    .containsExactlyInAnyOrder(
+                        "k1", "k2",  // routes from MyRoutesWithKamelets
+                        "myKamelet1", "myKamelet2" // routes from templates
+                    );
+            }
+        );
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/MyBean.java
similarity index 96%
copy from camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
copy to camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/MyBean.java
index 2069e08..650709d 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/MyBean.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.main;
+package org.apache.camel.k.main.support;
 
 public class MyBean {
     private final String name;
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/RuntimeTestSupport.java b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/RuntimeTestSupport.java
new file mode 100644
index 0000000..a753821
--- /dev/null
+++ b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/RuntimeTestSupport.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.k.main.support;
+
+import java.util.Map;
+
+import org.apache.camel.k.Runtime;
+import org.apache.camel.k.listener.SourcesConfigurer;
+import org.apache.camel.k.main.ApplicationRuntime;
+import org.junit.jupiter.api.function.ThrowingConsumer;
+
+public class RuntimeTestSupport {
+    private RuntimeTestSupport() {
+    }
+
+    public static void run(Map<String, String> properties, ThrowingConsumer<Runtime> consumer) throws Exception {
+        ApplicationRuntime runtime = new ApplicationRuntime();
+        try {
+            runtime.setInitialProperties(properties);
+            runtime.addListener(new SourcesConfigurer());
+            runtime.addListener(Runtime.Phase.Started, r -> {
+                try {
+                    consumer.accept(r);
+                } catch (Throwable e) {
+                    throw new RuntimeException(e);
+                } finally {
+                    r.stop();
+                }
+            });
+        } finally {
+            runtime.stop();
+        }
+    }
+}
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TestCustomizer.java b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/TestCustomizer.java
similarity index 97%
rename from camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TestCustomizer.java
rename to camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/TestCustomizer.java
index 4676101..c5e1770 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/TestCustomizer.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/support/TestCustomizer.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.main;
+package org.apache.camel.k.main.support;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.k.ContextCustomizer;
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test b/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
index a393d0e..26544f0 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
+++ b/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-class=org.apache.camel.k.main.TestCustomizer
\ No newline at end of file
+class=org.apache.camel.k.main.support.TestCustomizer
\ No newline at end of file
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRouteTemplate.java
similarity index 73%
copy from camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
copy to camel-k-main/camel-k-runtime-main/src/test/resources/MyRouteTemplate.java
index 0659a75..b70f309 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRouteTemplate.java
@@ -17,18 +17,13 @@
 
 import org.apache.camel.BindToRegistry;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.main.support.MyBean;
 
-public class MyRoutesWithBeans extends RouteBuilder {
+public class MyRouteTemplate extends RouteBuilder {
     @Override
     public void configure() throws Exception {
-        from("direct:start")
-            .routeId("my-route")
-            .bean("my-bean", "getName")
-            .to("log:info");
-    }
-
-    @BindToRegistry("my-bean")
-    public org.apache.camel.k.main.MyBean createMyBean() {
-        return new org.apache.camel.k.main.MyBean("my-bean-name");
+        from("direct:{{routeId}}")
+            .setBody().constant("{{message}}")
+            .to("log:{{routeId}}");
     }
 }
\ No newline at end of file
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
index 0659a75..003e97c 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
@@ -17,6 +17,7 @@
 
 import org.apache.camel.BindToRegistry;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.main.support.MyBean;
 
 public class MyRoutesWithBeans extends RouteBuilder {
     @Override
@@ -28,7 +29,7 @@ public class MyRoutesWithBeans extends RouteBuilder {
     }
 
     @BindToRegistry("my-bean")
-    public org.apache.camel.k.main.MyBean createMyBean() {
-        return new org.apache.camel.k.main.MyBean("my-bean-name");
+    public MyBean createMyBean() {
+        return new MyBean("my-bean-name");
     }
 }
\ No newline at end of file
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithKamelets.java
similarity index 73%
copy from camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
copy to camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithKamelets.java
index 0659a75..7e1f4b1 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithBeans.java
+++ b/camel-k-main/camel-k-runtime-main/src/test/resources/MyRoutesWithKamelets.java
@@ -17,18 +17,16 @@
 
 import org.apache.camel.BindToRegistry;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.main.support.MyBean;
 
-public class MyRoutesWithBeans extends RouteBuilder {
+public class MyRoutesWithKamelets extends RouteBuilder {
     @Override
     public void configure() throws Exception {
-        from("direct:start")
-            .routeId("my-route")
-            .bean("my-bean", "getName")
-            .to("log:info");
-    }
-
-    @BindToRegistry("my-bean")
-    public org.apache.camel.k.main.MyBean createMyBean() {
-        return new org.apache.camel.k.main.MyBean("my-bean-name");
+        from("direct:k1")
+            .routeId("k1")
+            .to("kamelet:my-template/myKamelet1?message=my-message");
+        from("direct:k2")
+            .routeId("k2")
+            .to("kamelet:my-template/myKamelet2");
     }
 }
\ No newline at end of file
diff --git a/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentProcessor.java b/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentProcessor.java
index 3f4a853..2884b8a 100644
--- a/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentProcessor.java
+++ b/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.camel.k.core.quarkus.deployment;
 
 import java.util.List;
+import java.util.stream.Collectors;
 
 import io.quarkus.deployment.annotations.BuildProducer;
 import io.quarkus.deployment.annotations.BuildStep;
@@ -26,6 +27,7 @@ import io.quarkus.deployment.builditem.CombinedIndexBuildItem;
 import io.quarkus.deployment.builditem.nativeimage.ReflectiveClassBuildItem;
 import io.quarkus.deployment.builditem.nativeimage.ServiceProviderBuildItem;
 import org.apache.camel.k.Constants;
+import org.apache.camel.k.SourceDefinition;
 import org.apache.camel.k.core.quarkus.RuntimeRecorder;
 import org.apache.camel.quarkus.core.deployment.spi.CamelContextCustomizerBuildItem;
 import org.apache.camel.quarkus.core.deployment.spi.CamelServiceDestination;
@@ -34,6 +36,8 @@ import org.apache.camel.spi.StreamCachingStrategy;
 import org.jboss.jandex.IndexView;
 
 import static org.apache.camel.k.core.quarkus.deployment.DeploymentSupport.getAllKnownImplementors;
+import static org.apache.camel.k.core.quarkus.deployment.DeploymentSupport.reflectiveClassBuildItem;
+import static org.apache.camel.k.core.quarkus.deployment.DeploymentSupport.stream;
 
 public class DeploymentProcessor {
     @BuildStep
@@ -53,59 +57,42 @@ public class DeploymentProcessor {
     }
 
     @BuildStep
-    void registerServices(
-            BuildProducer<ServiceProviderBuildItem> serviceProvider,
-            BuildProducer<ReflectiveClassBuildItem> reflectiveClass,
-            CombinedIndexBuildItem combinedIndexBuildItem) {
+    List<ReflectiveClassBuildItem> registerClasses() {
+       return List.of(
+           new ReflectiveClassBuildItem(true, false, SourceDefinition.class)
+       );
+    }
 
+    @BuildStep
+    List<ServiceProviderBuildItem> registerServices(CombinedIndexBuildItem combinedIndexBuildItem) {
         final IndexView view = combinedIndexBuildItem.getIndex();
         final String serviceType = "org.apache.camel.k.Runtime$Listener";
 
-        getAllKnownImplementors(view, serviceType).forEach(i -> {
-            serviceProvider.produce(
-                new ServiceProviderBuildItem(
-                    serviceType,
-                    i.name().toString())
-            );
-        });
+        return stream(getAllKnownImplementors(view, serviceType))
+            .map(i -> new ServiceProviderBuildItem(serviceType, i.name().toString()))
+            .collect(Collectors.toList());
     }
 
     @BuildStep
     void registerStreamCachingClasses(
             BuildProducer<ReflectiveClassBuildItem> reflectiveClass,
-            CombinedIndexBuildItem combinedIndexBuildItem) {
+            CombinedIndexBuildItem combinedIndex) {
 
-        final IndexView view = combinedIndexBuildItem.getIndex();
+        final IndexView view = combinedIndex.getIndex();
 
-        getAllKnownImplementors(view, StreamCachingStrategy.class).forEach(i-> {
-            reflectiveClass.produce(
-                new ReflectiveClassBuildItem(
-                    true,
-                    true,
-                    i.name().toString())
-            );
-        });
-        getAllKnownImplementors(view, StreamCachingStrategy.Statistics.class).forEach(i-> {
-            reflectiveClass.produce(
-                new ReflectiveClassBuildItem(
-                    true,
-                    true,
-                    i.name().toString())
-            );
-        });
-        getAllKnownImplementors(view, StreamCachingStrategy.SpoolRule.class).forEach(i-> {
-            reflectiveClass.produce(
-                new ReflectiveClassBuildItem(
-                    true,
-                    true,
-                    i.name().toString())
-            );
-        });
+        getAllKnownImplementors(view, StreamCachingStrategy.class)
+            .forEach(i-> reflectiveClass.produce(reflectiveClassBuildItem(i)));
+
+        getAllKnownImplementors(view, StreamCachingStrategy.Statistics.class)
+            .forEach(i-> reflectiveClass.produce(reflectiveClassBuildItem(i)));
+
+        getAllKnownImplementors(view, StreamCachingStrategy.SpoolRule.class)
+            .forEach(i-> reflectiveClass.produce(reflectiveClassBuildItem(i)));
 
         reflectiveClass.produce(
             new ReflectiveClassBuildItem(
                 true,
-                true,
+                false,
                 StreamCachingStrategy.SpoolRule.class)
         );
     }
diff --git a/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentSupport.java b/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentSupport.java
index 9803f9a..5715ba2 100644
--- a/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentSupport.java
+++ b/camel-k-quarkus/camel-k-quarkus-core/deployment/src/main/java/org/apache/camel/k/core/quarkus/deployment/DeploymentSupport.java
@@ -16,8 +16,10 @@
  */
 package org.apache.camel.k.core.quarkus.deployment;
 
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
 
 import io.quarkus.deployment.builditem.nativeimage.ReflectiveClassBuildItem;
 import org.jboss.jandex.AnnotationInstance;
@@ -33,34 +35,63 @@ public final class DeploymentSupport {
     public static Iterable<ClassInfo> getAllKnownImplementors(IndexView view, String name) {
         return view.getAllKnownImplementors(DotName.createSimple(name));
     }
+    public static <T> Iterable<T> getAllKnownImplementors(IndexView view, String name, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownImplementors(view, name)).map(mapper).collect(Collectors.toList());
+    }
+
 
     public static Iterable<ClassInfo> getAllKnownImplementors(IndexView view, Class<?> type) {
         return view.getAllKnownImplementors(DotName.createSimple(type.getName()));
     }
+    public static <T> Iterable<T> getAllKnownImplementors(IndexView view, Class<?> type, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownImplementors(view, type)).map(mapper).collect(Collectors.toList());
+    }
+
+
+    public static Iterable<ClassInfo> getAllKnownImplementors(IndexView view, DotName type) {
+        return view.getAllKnownImplementors(type);
+    }
+    public static <T> Iterable<T> getAllKnownImplementors(IndexView view, DotName type, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownImplementors(view, type)).map(mapper).collect(Collectors.toList());
+    }
+
 
     public static Iterable<ClassInfo> getAllKnownSubclasses(IndexView view, String name) {
         return view.getAllKnownSubclasses(DotName.createSimple(name));
     }
+    public static <T> Iterable<T> getAllKnownSubclasses(IndexView view, String name, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownSubclasses(view, name)).map(mapper).collect(Collectors.toList());
+    }
+
 
     public static Iterable<ClassInfo> getAllKnownSubclasses(IndexView view, Class<?> type) {
         return view.getAllKnownSubclasses(DotName.createSimple(type.getName()));
     }
-
-    public static Iterable<ClassInfo> getAllKnownImplementors(IndexView view, DotName type) {
-        return view.getAllKnownImplementors(type);
+    public static <T> Iterable<T> getAllKnownSubclasses(IndexView view, Class<?> type, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownSubclasses(view, type)).map(mapper).collect(Collectors.toList());
     }
 
     public static Iterable<ClassInfo> getAllKnownSubclasses(IndexView view, DotName type) {
         return view.getAllKnownSubclasses(type);
     }
+    public static <T> Iterable<T> getAllKnownSubclasses(IndexView view, DotName type, Function<ClassInfo, T> mapper) {
+        return stream(getAllKnownSubclasses(view, type)).map(mapper).collect(Collectors.toList());
+    }
+
 
     public static Iterable<ClassInfo> getAnnotated(IndexView view, String name) {
         return getAnnotated(view, DotName.createSimple(name));
     }
+    public static <T> Iterable<T> getAnnotated(IndexView view, String name, Function<ClassInfo, T> mapper) {
+        return stream(getAnnotated(view, name)).map(mapper).collect(Collectors.toList());
+    }
 
     public static Iterable<ClassInfo> getAnnotated(IndexView view, Class<?> type) {
         return getAnnotated(view, DotName.createSimple(type.getName()));
     }
+    public static <T> Iterable<T> getAnnotated(IndexView view, Class<?> type, Function<ClassInfo, T> mapper) {
+        return stream(getAnnotated(view, type)).map(mapper).collect(Collectors.toList());
+    }
 
     public static Iterable<ClassInfo> getAnnotated(IndexView view, DotName type) {
         return view.getAnnotations(type).stream()
@@ -69,16 +100,24 @@ public final class DeploymentSupport {
             .map(AnnotationTarget::asClass)
             .collect(Collectors.toList());
     }
+    public static <T> Iterable<T> getAnnotated(IndexView view, DotName type, Function<ClassInfo, T> mapper) {
+        return stream(getAnnotated(view, type)).map(mapper).collect(Collectors.toList());
+    }
 
 
     public static ReflectiveClassBuildItem reflectiveClassBuildItem(ClassInfo... classInfos) {
-        return new ReflectiveClassBuildItem(
-            true,
-            false,
-            Stream.of(classInfos)
-                .map(ClassInfo::name)
-                .map(DotName::toString)
-                .toArray(String[]::new)
+        return classInfos.length == 1
+            ? new ReflectiveClassBuildItem(
+                true,
+                false,
+                classInfos[0].name().toString())
+            : new ReflectiveClassBuildItem(
+                true,
+                false,
+                Stream.of(classInfos)
+                    .map(ClassInfo::name)
+                    .map(DotName::toString)
+                    .toArray(String[]::new)
         );
     }
 
@@ -93,4 +132,7 @@ public final class DeploymentSupport {
         );
     }
 
+    public static <T> Stream<T> stream(Iterable<T> iterable) {
+        return StreamSupport.stream(iterable.spliterator(), false);
+    }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java
index 3ac5bf2..cd3e13b 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java
@@ -24,7 +24,7 @@ import io.restassured.RestAssured;
 import io.restassured.path.json.JsonPath;
 import org.apache.camel.k.CompositeClassloader;
 import org.apache.camel.k.listener.ContextConfigurer;
-import org.apache.camel.k.listener.RoutesConfigurer;
+import org.apache.camel.k.listener.SourcesConfigurer;
 import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
@@ -45,7 +45,7 @@ public class CoreTest {
 
         assertThat(p.getList("services", String.class)).contains(
             ContextConfigurer.class.getName(),
-            RoutesConfigurer.class.getName()
+            SourcesConfigurer.class.getName()
         );
     }
 
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/pom.xml b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/pom.xml
new file mode 100644
index 0000000..91a09a3
--- /dev/null
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/pom.xml
@@ -0,0 +1,155 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.camel.k</groupId>
+        <artifactId>camel-k-quarkus-itests</artifactId>
+        <version>1.5.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>camel-k-quarkus-itests-kamelet</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-quarkus-kamelet</artifactId>
+        </dependency>
+
+        <!-- quarkus dependencies -->
+        <dependency>
+            <groupId>io.quarkus</groupId>
+            <artifactId>quarkus-resteasy</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.quarkus</groupId>
+            <artifactId>quarkus-resteasy-jsonb</artifactId>
+        </dependency>
+
+        <!-- test dependencies -->
+        <dependency>
+            <groupId>io.quarkus</groupId>
+            <artifactId>quarkus-junit5</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>io.rest-assured</groupId>
+            <artifactId>rest-assured</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.assertj</groupId>
+            <artifactId>assertj-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>${build-helper-maven-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <id>reserve-network-port</id>
+                        <goals>
+                            <goal>reserve-network-port</goal>
+                        </goals>
+                        <phase>process-resources</phase>
+                        <configuration>
+                            <portNames>
+                                <portName>test.http.port.jvm</portName>
+                                <portName>test.http.port.native</portName>
+                            </portNames>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>io.quarkus</groupId>
+                <artifactId>quarkus-maven-plugin</artifactId>
+                <version>${quarkus.version}</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>build</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <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>
+                    </systemProperties>
+                    <environmentVariables>
+                        <CAMEL_KNATIVE_CONFIGURATION>file:${project.basedir}/src/main/resources/env.json</CAMEL_KNATIVE_CONFIGURATION>
+                    </environmentVariables>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>native</id>
+            <activation>
+                <property>
+                    <name>native</name>
+                </property>
+            </activation>
+            <properties>
+                <quarkus.package.type>native</quarkus.package.type>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                                <configuration>
+                                    <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>
+                                    </systemProperties>
+                                    <environmentVariables>
+                                        <CAMEL_KNATIVE_CONFIGURATION>file:${project.basedir}/src/main/resources/env.json</CAMEL_KNATIVE_CONFIGURATION>
+                                    </environmentVariables>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+
+</project>
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Application.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Application.java
new file mode 100644
index 0000000..b9fd023
--- /dev/null
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Application.java
@@ -0,0 +1,83 @@
+/*
+ * 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.kamelet.deployment;
+
+import java.util.Locale;
+import java.util.stream.Collectors;
+import javax.enterprise.context.ApplicationScoped;
+import javax.inject.Inject;
+import javax.json.Json;
+import javax.json.JsonObject;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+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.FluentProducerTemplate;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.model.ModelCamelContext;
+import org.apache.camel.model.RouteTemplateDefinition;
+
+
+@Path("/test")
+@ApplicationScoped
+public class Application {
+    public static String TEMPLATE_ID = "to-upper";
+
+    @Inject
+    CamelContext context;
+    @Inject
+    FluentProducerTemplate template;
+
+    @GET
+    @Path("/inspect")
+    @Produces(MediaType.APPLICATION_JSON)
+    public JsonObject inspect() {
+        var templates = context.adapt(ModelCamelContext.class).getRouteTemplateDefinitions();
+        var ids = templates.stream().map(RouteTemplateDefinition::getId).collect(Collectors.toList());
+
+        return Json.createObjectBuilder()
+            .add("templates", Json.createArrayBuilder(ids))
+            .build();
+    }
+
+    @POST
+    @Path("/invoke/{templateId}")
+    @Consumes(MediaType.TEXT_PLAIN)
+    @Produces(MediaType.TEXT_PLAIN)
+    public String invoke(@PathParam("templateId") String templateId, String message) {
+        return template.toF("kamelet:%s/test?message=%s", templateId, message).request(String.class);
+    }
+
+    @javax.enterprise.inject.Produces
+    public RouteBuilder routes() {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                routeTemplate(TEMPLATE_ID)
+                    .templateParameter("message")
+                    .from("direct:{{routeId}}")
+                        .setBody().constant("{{message}}")
+                        .transform().body(String.class, b -> b.toUpperCase(Locale.US));
+            }
+        };
+    }
+}
diff --git a/examples/camel-k-runtime-example-groovy/data/application.properties b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/resources/application.properties
similarity index 81%
copy from examples/camel-k-runtime-example-groovy/data/application.properties
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/resources/application.properties
index ac98d6e..b51efa2 100644
--- a/examples/camel-k-runtime-example-groovy/data/application.properties
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/main/resources/application.properties
@@ -16,16 +16,12 @@
 ## ---------------------------------------------------------------------------
 
 #
-# Logging
+# Quarkus
 #
-logging.level.org.apache.camel.k = DEBUG
+quarkus.banner.enabled     = false
 
 #
-# camel - main
+# Quarkus :: Camel
 #
-camel.main.name = camel-k
-camel.main.stream-caching-enabled = true
-camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
+quarkus.camel.routes-discovery.enabled = false
 
-
-message = test
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletIT.java
similarity index 73%
copy from camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletIT.java
index 2069e08..80ff5d9 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletIT.java
@@ -14,21 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.main;
+package org.apache.camel.k.quarkus.kamelet.deployment;
 
-public class MyBean {
-    private final String name;
+import io.quarkus.test.junit.NativeImageTest;
 
-    public MyBean(String name) {
-        this.name = name;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    @Override
-    public String toString() {
-        return name;
-    }
+@NativeImageTest
+public class KameletIT extends KameletTest {
 }
\ No newline at end of file
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletTest.java
similarity index 62%
copy from camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java
copy to camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletTest.java
index 3ac5bf2..726ca37 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-core/src/test/java/org/apache/camel/k/core/quarkus/deployment/CoreTest.java
+++ b/camel-k-quarkus/camel-k-quarkus-itests/camel-k-quarkus-itests-kamelet/src/test/java/org/apache/camel/k/quarkus/kamelet/deployment/KameletTest.java
@@ -14,49 +14,51 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.core.quarkus.deployment;
+package org.apache.camel.k.quarkus.kamelet.deployment;
 
+import java.util.Locale;
 import javax.ws.rs.core.MediaType;
 
-import io.quarkus.test.junit.DisabledOnNativeImage;
 import io.quarkus.test.junit.QuarkusTest;
 import io.restassured.RestAssured;
 import io.restassured.path.json.JsonPath;
-import org.apache.camel.k.CompositeClassloader;
-import org.apache.camel.k.listener.ContextConfigurer;
-import org.apache.camel.k.listener.RoutesConfigurer;
+import org.apache.camel.Exchange;
 import org.junit.jupiter.api.Test;
 
+import static io.restassured.RestAssured.given;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.hamcrest.Matchers.is;
 
 @QuarkusTest
-public class CoreTest {
+public class KameletTest {
     @Test
-    public void testServices() {
+    public void inspect() {
         JsonPath p = RestAssured.given()
+            .contentType(MediaType.TEXT_PLAIN)
             .accept(MediaType.APPLICATION_JSON)
-            .get("/test/services")
+            .get("/test/inspect")
             .then()
                 .statusCode(200)
             .extract()
                 .body()
                 .jsonPath();
 
-        assertThat(p.getList("services", String.class)).contains(
-            ContextConfigurer.class.getName(),
-            RoutesConfigurer.class.getName()
-        );
+        assertThat(p.getList("templates", String.class)).contains(Application.TEMPLATE_ID);
     }
 
-    @DisabledOnNativeImage
     @Test
-    public void testClassLoader() {
-        RestAssured.given()
-            .accept(MediaType.TEXT_PLAIN)
-            .get("/test/application-classloader")
-            .then()
+    public void invoke() {
+        final String payload = "test";
+
+        String result = given()
+            .body(payload)
+            .header(Exchange.CONTENT_TYPE, "text/plain")
+        .when()
+            .post("/test/invoke/{templateId}", Application.TEMPLATE_ID)
+        .then()
             .statusCode(200)
-            .body(is(CompositeClassloader.class.getName()));
+            .extract()
+            .asString();
+
+        assertThat(result).isEqualTo(payload.toUpperCase(Locale.US));
     }
-}
\ 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 2ca69ea..1f0d076 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
@@ -31,6 +31,7 @@
     <modules>
         <module>camel-k-quarkus-itests-core</module>
         <module>camel-k-quarkus-itests-cron</module>
+        <module>camel-k-quarkus-itests-kamelet</module>
         <module>camel-k-quarkus-itests-knative</module>
         <module>camel-k-quarkus-itests-loader-support</module>
         <module>camel-k-quarkus-itests-loader-groovy</module>
diff --git a/examples/pom.xml b/camel-k-quarkus/camel-k-quarkus-kamelet/deployment/pom.xml
similarity index 54%
copy from examples/pom.xml
copy to camel-k-quarkus/camel-k-quarkus-kamelet/deployment/pom.xml
index 28bf5e9..328f788 100644
--- a/examples/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-kamelet/deployment/pom.xml
@@ -20,43 +20,48 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <groupId>org.apache.camel.k</groupId>
-        <artifactId>camel-k-runtime-parent</artifactId>
+        <artifactId>camel-k-quarkus-kamelet-parent</artifactId>
         <version>1.5.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <packaging>pom</packaging>
-    <artifactId>camel-k-runtime-examples</artifactId>
-
-    <properties>
-        <maven.install.skip>true</maven.install.skip>
-    </properties>
-
-    <modules>
-        <module>camel-k-runtime-example-api</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>
-        <module>camel-k-runtime-example-quarkus-js</module>
-        <module>camel-k-runtime-example-quarkus-yaml</module>
-        <module>camel-k-runtime-example-quarkus-knative</module>
-        <module>camel-k-runtime-example-quarkus-kotlin</module>
-        <module>camel-k-runtime-example-quarkus-groovy</module>
-        <module>camel-k-runtime-example-quarkus-java</module>
-        <module>camel-k-runtime-example-quarkus-xml</module>
-    </modules>
+    <artifactId>camel-k-quarkus-kamelet-deployment</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-quarkus-kamelet</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-quarkus-core-deployment</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-core-deployment</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-direct-deployment</artifactId>
+        </dependency>
+    </dependencies>
 
     <build>
         <plugins>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-deploy-plugin</artifactId>
-                <version>${maven-deploy-plugin-version}</version>
+                <artifactId>maven-compiler-plugin</artifactId>
                 <configuration>
-                    <skip>true</skip>
+                    <annotationProcessorPaths>
+                        <path>
+                            <groupId>io.quarkus</groupId>
+                            <artifactId>quarkus-extension-processor</artifactId>
+                            <version>${quarkus.version}</version>
+                        </path>
+                    </annotationProcessorPaths>
                 </configuration>
             </plugin>
         </plugins>
     </build>
+
 </project>
diff --git a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java b/camel-k-quarkus/camel-k-quarkus-kamelet/deployment/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Feature.java
similarity index 69%
copy from camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java
copy to camel-k-quarkus/camel-k-quarkus-kamelet/deployment/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Feature.java
index 043f44e..4427561 100644
--- a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java
+++ b/camel-k-quarkus/camel-k-quarkus-kamelet/deployment/src/main/java/org/apache/camel/k/quarkus/kamelet/deployment/Feature.java
@@ -14,17 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.component.kamelet;
+package org.apache.camel.k.quarkus.kamelet.deployment;
 
-import java.util.function.Predicate;
+import io.quarkus.deployment.annotations.BuildStep;
+import io.quarkus.deployment.builditem.FeatureBuildItem;
 
-public final class Kamelet {
-    public static final String SCHEME = "kamelet";
+public class Feature {
+    private static final String FEATURE = "camel-k-runtime-kamelet";
 
-    private Kamelet() {
-    }
-
-    public static Predicate<String> startsWith(String prefix) {
-        return item -> item.startsWith(prefix);
+    @BuildStep
+    FeatureBuildItem feature() {
+        return new FeatureBuildItem(FEATURE);
     }
 }
diff --git a/camel-k-quarkus/camel-k-quarkus-itests/pom.xml b/camel-k-quarkus/camel-k-quarkus-kamelet/pom.xml
similarity index 65%
copy from camel-k-quarkus/camel-k-quarkus-itests/pom.xml
copy to camel-k-quarkus/camel-k-quarkus-kamelet/pom.xml
index 2ca69ea..992c6f3 100644
--- a/camel-k-quarkus/camel-k-quarkus-itests/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-kamelet/pom.xml
@@ -26,20 +26,11 @@
     <modelVersion>4.0.0</modelVersion>
     <packaging>pom</packaging>
 
-    <artifactId>camel-k-quarkus-itests</artifactId>
+    <artifactId>camel-k-quarkus-kamelet-parent</artifactId>
 
     <modules>
-        <module>camel-k-quarkus-itests-core</module>
-        <module>camel-k-quarkus-itests-cron</module>
-        <module>camel-k-quarkus-itests-knative</module>
-        <module>camel-k-quarkus-itests-loader-support</module>
-        <module>camel-k-quarkus-itests-loader-groovy</module>
-        <module>camel-k-quarkus-itests-loader-kotlin</module>
-        <module>camel-k-quarkus-itests-loader-java</module>
-        <module>camel-k-quarkus-itests-loader-js</module>
-        <module>camel-k-quarkus-itests-loader-xml</module>
-        <module>camel-k-quarkus-itests-loader-yaml</module>
-        <module>camel-k-quarkus-itests-polyglot</module>
+        <module>runtime</module>
+        <module>deployment</module>
     </modules>
 
 </project>
diff --git a/camel-k-runtime-core/pom.xml b/camel-k-quarkus/camel-k-quarkus-kamelet/runtime/pom.xml
similarity index 52%
copy from camel-k-runtime-core/pom.xml
copy to camel-k-quarkus/camel-k-quarkus-kamelet/runtime/pom.xml
index b9bfeb0..57a4f75 100644
--- a/camel-k-runtime-core/pom.xml
+++ b/camel-k-quarkus/camel-k-quarkus-kamelet/runtime/pom.xml
@@ -20,65 +20,60 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <groupId>org.apache.camel.k</groupId>
-        <artifactId>camel-k-runtime-parent</artifactId>
+        <artifactId>camel-k-quarkus-kamelet-parent</artifactId>
         <version>1.5.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>camel-k-runtime-core</artifactId>
+    <artifactId>camel-k-quarkus-kamelet</artifactId>
 
     <dependencies>
-
-        <!-- ****************************** -->
-        <!--                                -->
-        <!-- RUNTIME                        -->
-        <!--                                -->
-        <!-- ****************************** -->
-
         <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-core-engine</artifactId>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-kamelet</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-core-languages</artifactId>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-quarkus-core</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.camel</groupId>
-            <artifactId>camel-endpointdsl</artifactId>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-core</artifactId>
         </dependency>
-
-        <!-- ****************************** -->
-        <!--                                -->
-        <!-- TESTS                          -->
-        <!--                                -->
-        <!-- ****************************** -->
-
         <dependency>
-            <groupId>org.apache.camel.k</groupId>
-            <artifactId>camel-k-test</artifactId>
-            <scope>test</scope>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-direct</artifactId>
         </dependency>
-
     </dependencies>
 
     <build>
         <plugins>
             <plugin>
+                <groupId>io.quarkus</groupId>
+                <artifactId>quarkus-bootstrap-maven-plugin</artifactId>
+                <version>${quarkus.version}</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>extension-descriptor</goal>
+                        </goals>
+                        <configuration>
+                            <deployment>${project.groupId}:${project.artifactId}-deployment:${project.version}</deployment>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-surefire-plugin</artifactId>
+                <artifactId>maven-compiler-plugin</artifactId>
                 <configuration>
-                    <environmentVariables>
-                        <CAMEL_K_RESOURCE_001>my env content</CAMEL_K_RESOURCE_001>
-                        <MY_ENV_RESOURCE_TXT>env:CAMEL_K_RESOURCE_001</MY_ENV_RESOURCE_TXT>
-                        <CAMEL_K_RESOURCE_002>H4sIAFlah1wAA8utVEjOzy0oSi0uTk1RSM0rA3LzSlLzSrgAUbkzGRoAAAA=</CAMEL_K_RESOURCE_002>
-                        <MY_COMPRESSED_ENV_RESOURCE_TXT>env:CAMEL_K_RESOURCE_002?compression=true</MY_COMPRESSED_ENV_RESOURCE_TXT>
-                    </environmentVariables>
-                    <systemPropertyVariables>
-                        <root>${project.basedir}</root>
-                        <camel.k.mount-path.configmaps>${project.basedir}/src/test/resources/configmaps</camel.k.mount-path.configmaps>
-                        <camel.k.mount-path.secrets>${project.basedir}/src/test/resources/secrets</camel.k.mount-path.secrets>
-                    </systemPropertyVariables>
+                    <annotationProcessorPaths>
+                        <path>
+                            <groupId>io.quarkus</groupId>
+                            <artifactId>quarkus-extension-processor</artifactId>
+                            <version>${quarkus.version}</version>
+                        </path>
+                    </annotationProcessorPaths>
                 </configuration>
             </plugin>
             <plugin>
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 787f1cc..076ca3c 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
@@ -28,7 +28,6 @@ import org.apache.camel.k.Runtime;
 import org.apache.camel.k.quarkus.ApplicationRecorder;
 import org.apache.camel.quarkus.main.CamelMainApplication;
 import org.apache.camel.quarkus.main.deployment.spi.CamelMainListenerBuildItem;
-import org.apache.camel.spi.HasId;
 
 public class DeploymentProcessor {
     @BuildStep
@@ -40,19 +39,7 @@ public class DeploymentProcessor {
     @BuildStep
     CamelMainListenerBuildItem registerListener(ApplicationRecorder recorder) {
         List<Runtime.Listener> listeners = new ArrayList<>();
-        ServiceLoader.load(Runtime.Listener.class).forEach(listener -> {
-            if (listener instanceof HasId) {
-                String id = ((HasId) listener).getId();
-                if (!id.endsWith(".")) {
-                    id = id + ".";
-                }
-
-                // TODO: this has to be done at runtime
-                //PropertiesSupport.bindProperties(getCamelContext(), listener, id);
-            }
-
-            listeners.add(listener);
-        });
+        ServiceLoader.load(Runtime.Listener.class).forEach(listeners::add);
 
         return new CamelMainListenerBuildItem(recorder.createMainListener(listeners));
     }
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
index 9c15503..ccb01cc 100644
--- 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
@@ -99,7 +99,7 @@ public class ApplicationListenerAdapter implements MainListener {
             .sorted(Comparator.comparingInt(Runtime.Listener::getOrder))
             .forEach(l -> {
                 if (l.accept(phase, runtime)) {
-                    LOGGER.info("Listener {} executed in phase {}", l, phase);
+                    LOGGER.debug("Listener {} executed in phase {}", l, phase);
                 }
             });
     }
diff --git a/camel-k-quarkus/pom.xml b/camel-k-quarkus/pom.xml
index 96f418f..e201c25 100644
--- a/camel-k-quarkus/pom.xml
+++ b/camel-k-quarkus/pom.xml
@@ -56,6 +56,7 @@
         <module>camel-k-quarkus-loader-kotlin</module>
 
         <module>camel-k-quarkus-core</module>
+        <module>camel-k-quarkus-kamelet</module>
         <module>camel-k-quarkus-knative</module>
         <module>camel-k-quarkus-cron</module>
 
diff --git a/camel-k-runtime-bom/pom.xml b/camel-k-runtime-bom/pom.xml
index b27eb44..7fe92ac 100644
--- a/camel-k-runtime-bom/pom.xml
+++ b/camel-k-runtime-bom/pom.xml
@@ -288,6 +288,16 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.camel.k</groupId>
+                <artifactId>camel-k-quarkus-kamelet</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.camel.k</groupId>
+                <artifactId>camel-k-quarkus-kamelet-deployment</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.camel.k</groupId>
                 <artifactId>camel-k-quarkus-cron</artifactId>
                 <version>${project.version}</version>
             </dependency>
diff --git a/camel-k-runtime-core/pom.xml b/camel-k-runtime-core/pom.xml
index b9bfeb0..608ff3e 100644
--- a/camel-k-runtime-core/pom.xml
+++ b/camel-k-runtime-core/pom.xml
@@ -93,6 +93,45 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <!-- we need to generate additional configurer classes -->
+                <groupId>org.apache.camel</groupId>
+                <artifactId>camel-package-maven-plugin</artifactId>
+                <version>${camel.version}</version>
+                <executions>
+                    <execution>
+                        <id>generate-configurer</id>
+                        <phase>process-classes</phase>
+                        <goals>
+                            <goal>generate-configurer</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>${build-helper-maven-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                            <goal>add-resource</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/generated/java</source>
+                            </sources>
+                            <resources>
+                                <resource>
+                                    <directory>src/generated/resources</directory>
+                                </resource>
+                            </resources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 
diff --git a/camel-k-runtime-core/src/generated/java/org/apache/camel/k/SourceDefinitionConfigurer.java b/camel-k-runtime-core/src/generated/java/org/apache/camel/k/SourceDefinitionConfigurer.java
new file mode 100644
index 0000000..e23c414
--- /dev/null
+++ b/camel-k-runtime-core/src/generated/java/org/apache/camel/k/SourceDefinitionConfigurer.java
@@ -0,0 +1,101 @@
+/* Generated by camel build tools - do NOT edit this file! */
+package org.apache.camel.k;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.spi.GeneratedPropertyConfigurer;
+import org.apache.camel.spi.PropertyConfigurerGetter;
+import org.apache.camel.util.CaseInsensitiveMap;
+import org.apache.camel.k.SourceDefinition;
+
+/**
+ * Generated by camel build tools - do NOT edit this file!
+ */
+@SuppressWarnings("unchecked")
+public class SourceDefinitionConfigurer extends org.apache.camel.support.component.PropertyConfigurerSupport implements GeneratedPropertyConfigurer, PropertyConfigurerGetter {
+
+    @Override
+    public boolean configure(CamelContext camelContext, Object obj, String name, Object value, boolean ignoreCase) {
+        org.apache.camel.k.SourceDefinition target = (org.apache.camel.k.SourceDefinition) obj;
+        switch (ignoreCase ? name.toLowerCase() : name) {
+        case "compressed":
+        case "Compressed": target.setCompressed(property(camelContext, boolean.class, value)); return true;
+        case "content":
+        case "Content": target.setContent(property(camelContext, byte[].class, value)); return true;
+        case "id":
+        case "Id": target.setId(property(camelContext, java.lang.String.class, value)); return true;
+        case "interceptors":
+        case "Interceptors": target.setInterceptors(property(camelContext, java.util.List.class, value)); return true;
+        case "language":
+        case "Language": target.setLanguage(property(camelContext, java.lang.String.class, value)); return true;
+        case "loader":
+        case "Loader": target.setLoader(property(camelContext, java.lang.String.class, value)); return true;
+        case "location":
+        case "Location": target.setLocation(property(camelContext, java.lang.String.class, value)); return true;
+        case "name":
+        case "Name": target.setName(property(camelContext, java.lang.String.class, value)); return true;
+        case "propertynames":
+        case "PropertyNames": target.setPropertyNames(property(camelContext, java.util.List.class, value)); return true;
+        case "type":
+        case "Type": target.setType(property(camelContext, org.apache.camel.k.SourceType.class, value)); return true;
+        default: return false;
+        }
+    }
+
+    @Override
+    public Map<String, Object> getAllOptions(Object target) {
+        Map<String, Object> answer = new CaseInsensitiveMap();
+        answer.put("Compressed", boolean.class);
+        answer.put("Content", byte[].class);
+        answer.put("Id", java.lang.String.class);
+        answer.put("Interceptors", java.util.List.class);
+        answer.put("Language", java.lang.String.class);
+        answer.put("Loader", java.lang.String.class);
+        answer.put("Location", java.lang.String.class);
+        answer.put("Name", java.lang.String.class);
+        answer.put("PropertyNames", java.util.List.class);
+        answer.put("Type", org.apache.camel.k.SourceType.class);
+        return answer;
+    }
+
+    @Override
+    public Object getOptionValue(Object obj, String name, boolean ignoreCase) {
+        org.apache.camel.k.SourceDefinition target = (org.apache.camel.k.SourceDefinition) obj;
+        switch (ignoreCase ? name.toLowerCase() : name) {
+        case "compressed":
+        case "Compressed": return target.isCompressed();
+        case "content":
+        case "Content": return target.getContent();
+        case "id":
+        case "Id": return target.getId();
+        case "interceptors":
+        case "Interceptors": return target.getInterceptors();
+        case "language":
+        case "Language": return target.getLanguage();
+        case "loader":
+        case "Loader": return target.getLoader();
+        case "location":
+        case "Location": return target.getLocation();
+        case "name":
+        case "Name": return target.getName();
+        case "propertynames":
+        case "PropertyNames": return target.getPropertyNames();
+        case "type":
+        case "Type": return target.getType();
+        default: return null;
+        }
+    }
+
+    @Override
+    public Object getCollectionValueType(Object target, String name, boolean ignoreCase) {
+        switch (ignoreCase ? name.toLowerCase() : name) {
+        case "interceptors":
+        case "Interceptors": return java.lang.String.class;
+        case "propertynames":
+        case "PropertyNames": return java.lang.String.class;
+        default: return null;
+        }
+    }
+}
+
diff --git a/camel-k-runtime-core/src/generated/java/org/apache/camel/k/listener/SourcesConfigurerConfigurer.java b/camel-k-runtime-core/src/generated/java/org/apache/camel/k/listener/SourcesConfigurerConfigurer.java
new file mode 100644
index 0000000..a9faffa
--- /dev/null
+++ b/camel-k-runtime-core/src/generated/java/org/apache/camel/k/listener/SourcesConfigurerConfigurer.java
@@ -0,0 +1,45 @@
+/* Generated by camel build tools - do NOT edit this file! */
+package org.apache.camel.k.listener;
+
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.spi.GeneratedPropertyConfigurer;
+import org.apache.camel.spi.PropertyConfigurerGetter;
+import org.apache.camel.util.CaseInsensitiveMap;
+import org.apache.camel.k.listener.SourcesConfigurer;
+
+/**
+ * Generated by camel build tools - do NOT edit this file!
+ */
+@SuppressWarnings("unchecked")
+public class SourcesConfigurerConfigurer extends org.apache.camel.support.component.PropertyConfigurerSupport implements GeneratedPropertyConfigurer, PropertyConfigurerGetter {
+
+    @Override
+    public boolean configure(CamelContext camelContext, Object obj, String name, Object value, boolean ignoreCase) {
+        org.apache.camel.k.listener.SourcesConfigurer target = (org.apache.camel.k.listener.SourcesConfigurer) obj;
+        switch (ignoreCase ? name.toLowerCase() : name) {
+        case "sources":
+        case "Sources": target.setSources(property(camelContext, org.apache.camel.k.SourceDefinition[].class, value)); return true;
+        default: return false;
+        }
+    }
+
+    @Override
+    public Map<String, Object> getAllOptions(Object target) {
+        Map<String, Object> answer = new CaseInsensitiveMap();
+        answer.put("Sources", org.apache.camel.k.SourceDefinition[].class);
+        return answer;
+    }
+
+    @Override
+    public Object getOptionValue(Object obj, String name, boolean ignoreCase) {
+        org.apache.camel.k.listener.SourcesConfigurer target = (org.apache.camel.k.listener.SourcesConfigurer) obj;
+        switch (ignoreCase ? name.toLowerCase() : name) {
+        case "sources":
+        case "Sources": return target.getSources();
+        default: return null;
+        }
+    }
+}
+
diff --git a/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.SourceDefinition b/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.SourceDefinition
new file mode 100644
index 0000000..ac318bd
--- /dev/null
+++ b/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.SourceDefinition
@@ -0,0 +1,2 @@
+# Generated by camel build tools - do NOT edit this file!
+class=org.apache.camel.k.SourceDefinitionConfigurer
diff --git a/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.listener.SourcesConfigurer b/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.listener.SourcesConfigurer
new file mode 100644
index 0000000..618e943
--- /dev/null
+++ b/camel-k-runtime-core/src/generated/resources/META-INF/services/org/apache/camel/configurer/org.apache.camel.k.listener.SourcesConfigurer
@@ -0,0 +1,2 @@
+# Generated by camel build tools - do NOT edit this file!
+class=org.apache.camel.k.listener.SourcesConfigurerConfigurer
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/Source.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/Source.java
index a14a17d..beae512 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/Source.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/Source.java
@@ -25,12 +25,15 @@ import java.util.List;
 import java.util.Optional;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.spi.HasId;
 
-public interface Source {
+public interface Source extends HasId {
     String getName();
     String getLanguage();
+    SourceType getType();
     Optional<String> getLoader();
     List<String> getInterceptors();
+    List<String> getPropertyNames();
     InputStream resolveAsInputStream(CamelContext ctx);
 
     default Reader resolveAsReader(CamelContext ctx) {
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceDefinition.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceDefinition.java
new file mode 100644
index 0000000..4844030
--- /dev/null
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceDefinition.java
@@ -0,0 +1,247 @@
+/*
+ * 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;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.camel.k.support.StringSupport;
+import org.apache.camel.spi.Configurer;
+import org.apache.camel.spi.IdAware;
+import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.URISupport;
+
+@Configurer
+public class SourceDefinition implements IdAware {
+    private String id;
+    private String name;
+    private String language;
+    private String loader;
+    private List<String> interceptors;
+    private SourceType type;
+    private List<String> propertyNames;
+    private String location;
+    private byte[] content;
+    private boolean compressed;
+
+    @Override
+    public String getId() {
+        return id;
+    }
+
+    /**
+     * Sets the id
+     */
+    @Override
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * The name of the source.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getLanguage() {
+        return language;
+    }
+
+    /**
+     * The language use to define the source.
+     */
+    public void setLanguage(String language) {
+        this.language = language;
+    }
+
+    public String getLoader() {
+        return loader;
+    }
+
+    /**
+     * The {@link SourceLoader} that should be used to load the content of the source.
+     */
+    public void setLoader(String loader) {
+        this.loader = loader;
+    }
+
+    public List<String> getInterceptors() {
+        return interceptors;
+    }
+
+    /**
+     * The {@link org.apache.camel.k.SourceLoader.Interceptor} that should be applied.
+     */
+    public void setInterceptors(List<String> interceptors) {
+        this.interceptors = interceptors;
+    }
+
+    public SourceType getType() {
+        return type;
+    }
+
+    /**
+     * The {@link SourceType} of the source.
+     */
+    public void setType(SourceType type) {
+        this.type = type;
+    }
+
+    public List<String> getPropertyNames() {
+        return propertyNames;
+    }
+
+    /**
+     * The list of properties names the source requires (used only for templates).
+     */
+    public void setPropertyNames(List<String> propertyNames) {
+        this.propertyNames = propertyNames;
+    }
+
+    public String getLocation() {
+        return location;
+    }
+
+    /**
+     * The location of the source.
+     */
+    public void setLocation(String location) {
+        this.location = location;
+    }
+
+    public byte[] getContent() {
+        return content;
+    }
+
+    /**
+     * The content of the source.
+     */
+    public void setContent(byte[] content) {
+        this.content = content;
+    }
+
+    public boolean isCompressed() {
+        return compressed;
+    }
+
+    /**
+     * If the content of the source is compressed.
+     */
+    public void setCompressed(boolean compressed) {
+        this.compressed = compressed;
+    }
+
+    @Override
+    public String toString() {
+        String answer = "";
+
+        if (name != null) {
+            answer += "name='" + name + "', ";
+        }
+        if (language != null) {
+            answer += "language='" + language + "', ";
+        }
+        if (loader != null) {
+            answer += "loader='" + loader + "', ";
+        }
+        if (interceptors != null) {
+            answer += "interceptors='" + interceptors + "', ";
+        }
+        if (type != null) {
+            answer += "type='" + type + "', ";
+        }
+        if (propertyNames != null) {
+            answer += "propertyNames='" + propertyNames + "', ";
+        }
+        if (location != null) {
+            answer += "location='" + location + "', ";
+        }
+        if (compressed) {
+            answer += "compressed='true', ";
+        }
+        if (content != null) {
+            answer += "<...>";
+        }
+
+        return "SourceDefinition{" + answer + '}';
+    }
+
+    // ***********************************
+    //
+    // Helpers
+    //
+    // ***********************************
+
+    public static SourceDefinition fromURI(String uri) throws Exception {
+        final String location = StringSupport.substringBefore(uri, "?");
+
+        if (!location.startsWith(Constants.SCHEME_PREFIX_CLASSPATH) && !location.startsWith(Constants.SCHEME_PREFIX_FILE)) {
+            throw new IllegalArgumentException("No valid resource format, expected scheme:path, found " + uri);
+        }
+
+        final String query = StringSupport.substringAfter(uri, "?");
+        final Map<String, Object> params = URISupport.parseQuery(query);
+
+        String language = (String) params.get("language");
+        if (ObjectHelper.isEmpty(language)) {
+            language = StringSupport.substringAfterLast(location, ":");
+            language = StringSupport.substringAfterLast(language, ".");
+        }
+        if (ObjectHelper.isEmpty(language)) {
+            throw new IllegalArgumentException("Unknown language " + language);
+        }
+
+        String name = (String) params.get("name");
+        if (name == null) {
+            name = StringSupport.substringAfter(location, ":");
+            name = StringSupport.substringBeforeLast(name, ".");
+
+            if (name.contains("/")) {
+                name = StringSupport.substringAfterLast(name, "/");
+            }
+        }
+
+        SourceDefinition answer = new SourceDefinition();
+        answer.id = (String) params.get("id");
+        answer.location = location;
+        answer.name = name;
+        answer.language = language;
+        answer.loader = (String) params.get("loader");
+        answer.interceptors = StringSupport.split((String) params.get("interceptors"), ",");
+        answer.compressed = Boolean.parseBoolean((String) params.get("compression"));
+
+        return answer;
+    }
+
+    public static SourceDefinition fromBytes(String id, String name, String language, String loader, List<String> interceptors, byte[] content) {
+        SourceDefinition answer = new SourceDefinition();
+        answer.id = id;
+        answer.name = name;
+        answer.language = language;
+        answer.loader = loader;
+        answer.interceptors = interceptors != null ? interceptors : Collections.emptyList();
+        answer.content = content;
+
+        return answer;
+    }
+}
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceLoader.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceLoader.java
index 8fe1333..923b3ee 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceLoader.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceLoader.java
@@ -101,11 +101,14 @@ public interface SourceLoader {
         /**
          * Invoked before the source is materialized top a RoutesBuilder.
          */
-        void beforeLoad(SourceLoader loader, Source source);
+        default void beforeLoad(SourceLoader loader, Source source) {
+        }
 
         /**
          * Invoked after the source is materialized and before is added to the runtime.
          */
-        Result afterLoad(SourceLoader loader, Source source, Result result);
+        default Result afterLoad(SourceLoader loader, Source source, Result result) {
+            return result;
+        }
     }
 }
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceType.java
similarity index 73%
copy from camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
copy to camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceType.java
index 2069e08..ed9717a 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/SourceType.java
@@ -14,21 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.main;
+package org.apache.camel.k;
 
-public class MyBean {
-    private final String name;
-
-    public MyBean(String name) {
-        this.name = name;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    @Override
-    public String toString() {
-        return name;
-    }
-}
\ No newline at end of file
+public enum SourceType {
+    source,
+    template
+}
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/Sources.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/Sources.java
index 8cd0615..239f505 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/Sources.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/Sources.java
@@ -18,177 +18,119 @@ package org.apache.camel.k;
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Base64;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Optional;
 import java.util.UUID;
 import java.util.zip.GZIPInputStream;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.k.support.StringSupport;
-import org.apache.camel.spi.ClassResolver;
 import org.apache.camel.support.ResourceHelper;
 import org.apache.camel.util.ObjectHelper;
-import org.apache.camel.util.URISupport;
 
 public final class  Sources {
     private Sources() {
     }
 
     public static Source fromURI(String uri) throws Exception {
-        return new URI(uri);
+        return fromDefinition(SourceDefinition.fromURI(uri));
     }
 
     public static Source fromBytes(String name, String language, String loader, List<String> interceptors, byte[] content) {
-        return new InMemory(name, language, loader, interceptors, content);
+        return fromDefinition(SourceDefinition.fromBytes(null, name, language, loader, interceptors, content));
     }
 
     public static Source fromBytes(String name, String language, String loader, byte[] content) {
-        return new InMemory(name, language, loader, content);
+        return fromDefinition(SourceDefinition.fromBytes(null, name, language, loader, null, content));
     }
 
     public static Source fromBytes(String language, byte[] content) {
-        return new InMemory(UUID.randomUUID().toString(), language, null, content);
+        return fromDefinition(SourceDefinition.fromBytes(null, UUID.randomUUID().toString(), language, null, null, content));
     }
 
-    private static final class InMemory implements Source {
-        private final String name;
-        private final String language;
-        private final String loader;
-        private final List<String> interceptors;
-        private final byte[] content;
-
-        public InMemory(String name, String language, String loader, byte[] content) {
-            this.name = name;
-            this.language = language;
-            this.loader = loader;
-            this.interceptors = Collections.emptyList();
-            this.content = Arrays.copyOf(content, content.length);
+    public static Source fromDefinition(SourceDefinition definition) {
+        if (definition.getLocation() == null && definition.getContent() == null) {
+            throw new IllegalArgumentException("Either the source location or the source content should be set");
         }
 
-        public InMemory(String name, String language, String loader, List<String> interceptors, byte[] content) {
-            this.name = name;
-            this.language = language;
-            this.loader = loader;
-            this.interceptors = new ArrayList<>(interceptors);
-            this.content = Arrays.copyOf(content, content.length);
-        }
-
-        @Override
-        public String getName() {
-            return name;
-        }
-
-        @Override
-        public String getLanguage() {
-            return language;
-        }
-
-        @Override
-        public Optional<String> getLoader() {
-            return Optional.ofNullable(loader);
-        }
-
-        @Override
-        public List<String> getInterceptors() {
-            return interceptors;
-        }
-
-        @Override
-        public InputStream resolveAsInputStream(CamelContext ctx) {
-            if (content == null) {
-                throw new IllegalArgumentException("No content defined");
+        return new Source() {
+            @Override
+            public String getId() {
+                return ObjectHelper.supplyIfEmpty(definition.getId(), this::getName);
             }
 
-            return new ByteArrayInputStream(this.content);
-        }
-    }
-
-    private static final class URI implements Source {
-        private final String location;
-        private final String name;
-        private final String language;
-        private final String loader;
-        private final String interceptors;
-        private final boolean compressed;
+            @Override
+            public String getName() {
+                String answer = definition.getName();
+                if (ObjectHelper.isEmpty(answer) && ObjectHelper.isNotEmpty(definition.getLocation())) {
+                    answer = StringSupport.substringAfter(definition.getLocation(), ":");
+                    answer = StringSupport.substringBeforeLast(answer, ".");
 
-        private URI(String uri) throws Exception {
-            final String location = StringSupport.substringBefore(uri, "?");
+                    if (answer.contains("/")) {
+                        answer = StringSupport.substringAfterLast(answer, "/");
+                    }
+                }
 
-            if (!location.startsWith(Constants.SCHEME_PREFIX_CLASSPATH) && !location.startsWith(Constants.SCHEME_PREFIX_FILE)) {
-                throw new IllegalArgumentException("No valid resource format, expected scheme:path, found " + uri);
+                return answer;
             }
 
-            final String query = StringSupport.substringAfter(uri, "?");
-            final Map<String, Object> params = URISupport.parseQuery(query);
+            @Override
+            public String getLanguage() {
+                String answer = definition.getLanguage();
+                if (ObjectHelper.isEmpty(answer) && ObjectHelper.isNotEmpty(definition.getLocation())) {
+                    answer = StringSupport.substringAfterLast(definition.getLocation(), ":");
+                    answer = StringSupport.substringAfterLast(answer, ".");
+                }
 
-            String language = (String) params.get("language");
-            if (ObjectHelper.isEmpty(language)) {
-                language = StringSupport.substringAfterLast(location, ":");
-                language = StringSupport.substringAfterLast(language, ".");
-            }
-            if (ObjectHelper.isEmpty(language)) {
-                throw new IllegalArgumentException("Unknown language " + language);
+                return answer;
             }
 
-            String name = (String) params.get("name");
-            if (name == null) {
-                name = StringSupport.substringAfter(location, ":");
-                name = StringSupport.substringBeforeLast(name, ".");
-
-                if (name.contains("/")) {
-                    name = StringSupport.substringAfterLast(name, "/");
-                }
+            @Override
+            public SourceType getType() {
+                return ObjectHelper.supplyIfEmpty(definition.getType(), () -> SourceType.source);
             }
 
-            this.location = location;
-            this.name = name;
-            this.language = language;
-            this.loader = (String) params.get("loader");
-            this.interceptors = (String) params.get("interceptors");
-            this.compressed = Boolean.parseBoolean((String) params.get("compression"));
-        }
-
-        @Override
-        public String getName() {
-            return name;
-        }
-
-        @Override
-        public String getLanguage() {
-            return language;
-        }
-
-        @Override
-        public Optional<String> getLoader() {
-            return Optional.ofNullable(loader);
-        }
-
-        @Override
-        public List<String> getInterceptors() {
-            return interceptors != null ? Arrays.asList(interceptors.split(",", -1)) : Collections.emptyList();
-        }
+            @Override
+            public Optional<String> getLoader() {
+                return Optional.ofNullable(definition.getLoader());
+            }
 
-        @Override
-        public InputStream resolveAsInputStream(CamelContext ctx) {
-            if (location == null) {
-                throw new IllegalArgumentException("Cannot resolve null URI");
+            @Override
+            public List<String> getInterceptors() {
+                return ObjectHelper.supplyIfEmpty(definition.getInterceptors(), Collections::emptyList);
             }
 
-            try {
-                final ClassResolver cr = ctx.getClassResolver();
-                final InputStream is = ResourceHelper.resolveResourceAsInputStream(cr, location);
+            @Override
+            public List<String> getPropertyNames() {
+                return ObjectHelper.supplyIfEmpty(definition.getPropertyNames(), Collections::emptyList);
+            }
 
-                return compressed
-                    ? new GZIPInputStream(Base64.getDecoder().wrap(is))
-                    : is;
-            } catch (Exception e) {
-                throw new RuntimeException(e);
+            /**
+             * Read the content of the source as {@link InputStream}.
+             *
+             * @param ctx the {@link CamelContext}
+             * @return the {@link InputStream} representing the source content
+             */
+            @Override
+            public InputStream resolveAsInputStream(CamelContext ctx) {
+                try {
+                    InputStream is;
+
+                    if (definition.getContent() != null) {
+                        is = new ByteArrayInputStream(definition.getContent());
+                    } else {
+                        is = ResourceHelper.resolveMandatoryResourceAsInputStream(ctx, definition.getLocation());
+                    }
+
+                    return definition.isCompressed()
+                        ? new GZIPInputStream(Base64.getDecoder().wrap(is))
+                        : is;
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
             }
-        }
+        };
     }
 }
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/RoutesConfigurer.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/RoutesConfigurer.java
deleted file mode 100644
index c74b5b8..0000000
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/RoutesConfigurer.java
+++ /dev/null
@@ -1,107 +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.listener;
-
-import java.util.List;
-
-import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.k.Constants;
-import org.apache.camel.k.Runtime;
-import org.apache.camel.k.RuntimeAware;
-import org.apache.camel.k.Source;
-import org.apache.camel.k.SourceLoader;
-import org.apache.camel.k.Sources;
-import org.apache.camel.k.support.RuntimeSupport;
-import org.apache.camel.util.ObjectHelper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class RoutesConfigurer extends AbstractPhaseListener {
-    private static final Logger LOGGER = LoggerFactory.getLogger(RoutesConfigurer.class);
-
-    public RoutesConfigurer() {
-        super(Runtime.Phase.ConfigureRoutes);
-    }
-
-    @Override
-    protected void accept(Runtime runtime) {
-        String routes = System.getProperty(Constants.PROPERTY_CAMEL_K_ROUTES);
-
-        if (ObjectHelper.isEmpty(routes)) {
-            routes = System.getenv(Constants.ENV_CAMEL_K_ROUTES);
-        }
-
-        if (ObjectHelper.isEmpty(routes)) {
-            LOGGER.warn("No routes found in {} environment variable", Constants.ENV_CAMEL_K_ROUTES);
-            return;
-        }
-
-        load(runtime, routes.split(",", -1));
-    }
-
-    protected void load(Runtime runtime, String... routes) {
-        for (String route: routes) {
-            if (ObjectHelper.isEmpty(route)) {
-                continue;
-            }
-
-            try {
-                load(runtime, Sources.fromURI(route));
-            } catch (Exception e) {
-                throw RuntimeCamelException.wrapRuntimeCamelException(e);
-            }
-
-            LOGGER.info("Loading routes from: {}", route);
-        }
-    }
-
-    public static RoutesConfigurer forRoutes(String... routes) {
-        return new RoutesConfigurer() {
-            @Override
-            protected void accept(Runtime runtime) {
-                load(runtime, routes);
-            }
-        };
-    }
-
-    public static SourceLoader load(Runtime runtime, Source source) {
-        final List<SourceLoader.Interceptor> interceptors = RuntimeSupport.loadInterceptors(runtime.getCamelContext(), source);
-        final SourceLoader loader = RuntimeSupport.loaderFor(runtime.getCamelContext(), source);
-
-        try {
-            for (SourceLoader.Interceptor interceptor: interceptors) {
-                if (interceptor instanceof RuntimeAware) {
-                    ((RuntimeAware) interceptor).setRuntime(runtime);
-                }
-
-                interceptor.beforeLoad(loader, source);
-            }
-
-            SourceLoader.Result result = loader.load(runtime, source);
-            for (SourceLoader.Interceptor interceptor: interceptors) {
-                result = interceptor.afterLoad(loader, source, result);
-            }
-
-            result.builder().ifPresent(runtime::addRoutes);
-            result.configuration().ifPresent(runtime::addConfiguration);
-        } catch (Exception e) {
-            throw RuntimeCamelException.wrapRuntimeCamelException(e);
-        }
-
-        return loader;
-    }
-}
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/SourcesConfigurer.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/SourcesConfigurer.java
new file mode 100644
index 0000000..5545947
--- /dev/null
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/listener/SourcesConfigurer.java
@@ -0,0 +1,77 @@
+/*
+ * 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.listener;
+
+import org.apache.camel.k.Constants;
+import org.apache.camel.k.Runtime;
+import org.apache.camel.k.SourceDefinition;
+import org.apache.camel.k.support.PropertiesSupport;
+import org.apache.camel.k.support.SourcesSupport;
+import org.apache.camel.spi.Configurer;
+import org.apache.camel.util.ObjectHelper;
+
+@Configurer
+public class SourcesConfigurer extends AbstractPhaseListener {
+    public static final String CAMEL_K_PREFIX = "camel.k.";
+    public static final String CAMEL_K_SOURCES_PREFIX = "camel.k.sources[";
+
+    private SourceDefinition[] sources;
+
+    public SourcesConfigurer() {
+        super(Runtime.Phase.ConfigureRoutes);
+    }
+
+    public SourceDefinition[] getSources() {
+        return sources;
+    }
+
+    public void setSources(SourceDefinition[] sources) {
+        this.sources = sources;
+    }
+
+    @Override
+    protected void accept(Runtime runtime) {
+        //
+        // load routes from env var for backward compatibility
+        //
+        String routes = System.getProperty(Constants.PROPERTY_CAMEL_K_ROUTES);
+        if (ObjectHelper.isEmpty(routes)) {
+            routes = System.getenv(Constants.ENV_CAMEL_K_ROUTES);
+        }
+
+        if (ObjectHelper.isNotEmpty(routes)) {
+            SourcesSupport.loadSources(runtime, routes.split(","));
+        }
+
+        //
+        // load routes from properties
+        //
+        // In order not to load any unwanted property, the filer remove any
+        // property that can't be bound to this configurer.
+        //
+        PropertiesSupport.bindProperties(
+            runtime.getCamelContext(),
+            this,
+            k -> k.startsWith(CAMEL_K_SOURCES_PREFIX),
+            CAMEL_K_PREFIX);
+
+        if (ObjectHelper.isNotEmpty(this.getSources())) {
+            SourcesSupport.loadSources(runtime, this.getSources());
+        }
+    }
+
+}
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/PropertiesSupport.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/PropertiesSupport.java
index e1e7996..113a7ef 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/PropertiesSupport.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/PropertiesSupport.java
@@ -26,34 +26,82 @@ import java.nio.file.Paths;
 import java.nio.file.SimpleFileVisitor;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.Component;
+import org.apache.camel.ExtendedCamelContext;
 import org.apache.camel.k.Constants;
 import org.apache.camel.spi.PropertiesComponent;
+import org.apache.camel.spi.PropertyConfigurer;
 import org.apache.camel.support.PropertyBindingSupport;
+import org.apache.camel.support.service.ServiceHelper;
 import org.apache.camel.util.ObjectHelper;
 
 public final class PropertiesSupport {
     private PropertiesSupport() {
     }
 
-    @SuppressWarnings("unchecked")
-    public static boolean bindProperties(CamelContext context, Object target, String prefix) {
+    public static <T> T bindProperties(CamelContext context, T target, String prefix) {
+        return bindProperties(context, target, prefix, false);
+    }
+
+    public static <T> T bindProperties(CamelContext context, T target, String prefix, boolean stripPrefix) {
+        return bindProperties(context, target, k -> k.startsWith(prefix), prefix, stripPrefix);
+    }
+
+    public static <T> T bindProperties(CamelContext context, T target, Predicate<String> filter, String prefix) {
+        return bindProperties(context, target, filter, prefix, false);
+    }
+
+    public static <T> T bindProperties(CamelContext context, T target, Predicate<String> filter, String prefix, boolean stripPrefix) {
         final PropertiesComponent component = context.getPropertiesComponent();
-        final Properties properties = component.loadProperties(k -> k.startsWith(prefix));
+        final Properties propertiesWithPrefix = component.loadProperties(filter);
+        final Map<String, Object> properties = new HashMap<>();
+
+        propertiesWithPrefix.stringPropertyNames().forEach(
+            name -> properties.put(
+                stripPrefix ? name.substring(prefix.length()) : name,
+                propertiesWithPrefix.getProperty(name))
+        );
+
+        PropertyConfigurer configurer = null;
+        if (target instanceof Component) {
+            // the component needs to be initialized to have the configurer ready
+            ServiceHelper.initService(target);
+            configurer = ((Component) target).getComponentPropertyConfigurer();
+        }
+
+        if (configurer == null) {
+            String name = target.getClass().getName();
+            if (target instanceof ExtendedCamelContext) {
+                // special for camel context itself as we have an extended configurer
+                name = ExtendedCamelContext.class.getName();
+            }
 
-        return PropertyBindingSupport.build()
+            // see if there is a configurer for it
+            configurer = context.adapt(ExtendedCamelContext.class)
+                .getConfigurerResolver()
+                .resolvePropertyConfigurer(name, context);
+        }
+
+        PropertyBindingSupport.build()
+            .withIgnoreCase(true)
             .withCamelContext(context)
             .withTarget(target)
-            .withProperties((Map)properties)
-            .withRemoveParameters(false)
-            .withOptionPrefix(prefix)
+            .withProperties(properties)
+            .withRemoveParameters(true)
+            .withOptionPrefix(stripPrefix ? null : prefix)
+            .withConfigurer(configurer)
             .bind();
+
+        return target;
     }
 
     public static String resolveApplicationPropertiesLocation() {
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/RuntimeSupport.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/RuntimeSupport.java
index f34134a..411a32e 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/RuntimeSupport.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/RuntimeSupport.java
@@ -66,7 +66,7 @@ public final class RuntimeSupport {
         customizers.entrySet().stream()
             .sorted(Map.Entry.comparingByValue())
             .forEach(e -> {
-                LOGGER.info("Apply ContextCustomizer with id={} and type={}", e.getKey(), e.getValue().getClass().getName());
+                LOGGER.debug("Apply ContextCustomizer with id={} and type={}", e.getKey(), e.getValue().getClass().getName());
 
                 PropertiesSupport.bindProperties(context, e.getValue(), Constants.CUSTOMIZER_PREFIX + e.getKey() + ".");
                 PropertiesSupport.bindProperties(context, e.getValue(), Constants.CUSTOMIZER_PREFIX_FALLBACK + e.getKey() + ".");
@@ -133,9 +133,9 @@ public final class RuntimeSupport {
                 .newInstance(customizerId, ContextCustomizer.class)
                 .orElseThrow(() -> new RuntimeException("Error creating instance for customizer: " + customizerId));
 
-            LOGGER.info("Found customizer {} with id {} from service definition", customizer, customizerId);
+            LOGGER.debug("Found customizer {} with id {} from service definition", customizer, customizerId);
         } else {
-            LOGGER.info("Found customizer {} with id {} from the registry", customizer, customizerId);
+            LOGGER.debug("Found customizer {} with id {} from the registry", customizer, customizerId);
         }
 
         return customizer;
@@ -178,11 +178,11 @@ public final class RuntimeSupport {
 
 
     public static SourceLoader lookupLoaderById(CamelContext context, String loaderId) {
-        LOGGER.info("Looking up loader for id: {}", loaderId);
+        LOGGER.debug("Looking up loader for id: {}", loaderId);
 
         SourceLoader loader = context.getRegistry().findByTypeWithName(SourceLoader.class).get(loaderId);
         if (loader != null) {
-            LOGGER.info("Found loader {} with id {} from the registry", loader, loaderId);
+            LOGGER.debug("Found loader {} with id {} from the registry", loader, loaderId);
             return loader;
         }
 
@@ -190,11 +190,11 @@ public final class RuntimeSupport {
     }
 
     public static SourceLoader lookupLoaderByLanguage(CamelContext context, String loaderId) {
-        LOGGER.info("Looking up loader for language: {}", loaderId);
+        LOGGER.debug("Looking up loader for language: {}", loaderId);
 
         for (SourceLoader loader: context.getRegistry().findByType(SourceLoader.class)) {
             if (loader.getSupportedLanguages().contains(loaderId)) {
-                LOGGER.info("Found loader {} for language {} from the registry", loader, loaderId);
+                LOGGER.debug("Found loader {} for language {} from the registry", loader, loaderId);
                 return loader;
             }
         }
@@ -208,7 +208,7 @@ public final class RuntimeSupport {
             .newInstance(loaderId, SourceLoader.class)
             .orElseThrow(() -> new RuntimeException("Error creating instance of loader: " + loaderId));
 
-        LOGGER.info("Found loader {} for language {} from service definition", loader, loaderId);
+        LOGGER.debug("Found loader {} for language {} from service definition", loader, loaderId);
 
         return loader;
     }
@@ -235,9 +235,9 @@ public final class RuntimeSupport {
                         .newInstance(id, SourceLoader.Interceptor.class)
                         .orElseThrow(() -> new IllegalArgumentException("Unable to find source loader interceptor for: " + id));
 
-                    LOGGER.info("Found source loader interceptor {} from service definition", id);
+                    LOGGER.debug("Found source loader interceptor {} from service definition", id);
                 } else {
-                    LOGGER.info("Found source loader interceptor {} from registry", id);
+                    LOGGER.debug("Found source loader interceptor {} from registry", id);
                 }
 
                 PropertiesSupport.bindProperties(context, interceptor, "loader.interceptor." + id + ".");
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/SourcesSupport.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/SourcesSupport.java
new file mode 100644
index 0000000..8432a54
--- /dev/null
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/SourcesSupport.java
@@ -0,0 +1,164 @@
+/*
+ * 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.support;
+
+import java.util.List;
+
+import org.apache.camel.RuntimeCamelException;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.builder.RouteBuilderLifecycleStrategy;
+import org.apache.camel.k.Runtime;
+import org.apache.camel.k.RuntimeAware;
+import org.apache.camel.k.Source;
+import org.apache.camel.k.SourceDefinition;
+import org.apache.camel.k.SourceLoader;
+import org.apache.camel.k.SourceType;
+import org.apache.camel.k.Sources;
+import org.apache.camel.k.listener.AbstractPhaseListener;
+import org.apache.camel.k.listener.SourcesConfigurer;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.model.RouteTemplateDefinition;
+import org.apache.camel.util.ObjectHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class SourcesSupport {
+    private static final Logger LOGGER = LoggerFactory.getLogger(SourcesConfigurer.class);
+
+    private SourcesSupport() {
+    }
+
+    public static Runtime.Listener forRoutes(String... sources) {
+        return new AbstractPhaseListener(Runtime.Phase.ConfigureRoutes) {
+            @Override
+            protected void accept(Runtime runtime) {
+                loadSources(runtime, sources);
+            }
+        };
+    }
+
+    public static Runtime.Listener forRoutes(SourceDefinition... definitions) {
+        return new AbstractPhaseListener(Runtime.Phase.ConfigureRoutes) {
+            @Override
+            protected void accept(Runtime runtime) {
+                loadSources(runtime, definitions);
+            }
+        };
+    }
+
+    public static void loadSources(Runtime runtime, String... routes) {
+        for (String route: routes) {
+            if (ObjectHelper.isEmpty(route)) {
+                continue;
+            }
+
+            LOGGER.info("Loading routes from: {}", route);
+
+            try {
+                load(runtime, Sources.fromURI(route));
+            } catch (Exception e) {
+                throw RuntimeCamelException.wrapRuntimeCamelException(e);
+            }
+        }
+    }
+
+    public static void loadSources(Runtime runtime, SourceDefinition... definitions) {
+        for (SourceDefinition definition: definitions) {
+            LOGGER.info("Loading routes from: {}", definition);
+
+            load(runtime, Sources.fromDefinition(definition));
+        }
+    }
+
+    public static SourceLoader load(Runtime runtime, Source source) {
+        final SourceLoader loader = RuntimeSupport.loaderFor(runtime.getCamelContext(), source);
+        final List<SourceLoader.Interceptor> interceptors = source.getType() == SourceType.source
+            ? sourceInterceptors(runtime, source)
+            : templateInterceptors(runtime, source);
+
+        try {
+            for (SourceLoader.Interceptor interceptor: interceptors) {
+                if (interceptor instanceof RuntimeAware) {
+                    ((RuntimeAware) interceptor).setRuntime(runtime);
+                }
+
+                interceptor.beforeLoad(loader, source);
+            }
+
+            SourceLoader.Result result = loader.load(runtime, source);
+
+            for (SourceLoader.Interceptor interceptor: interceptors) {
+                result = interceptor.afterLoad(loader, source, result);
+            }
+
+            result.builder().ifPresent(runtime::addRoutes);
+            result.configuration().ifPresent(runtime::addConfiguration);
+        } catch (Exception e) {
+            throw RuntimeCamelException.wrapRuntimeCamelException(e);
+        }
+
+        return loader;
+    }
+
+    private static List<SourceLoader.Interceptor> sourceInterceptors(Runtime runtime, Source source) {
+        return RuntimeSupport.loadInterceptors(runtime.getCamelContext(), source);
+    }
+
+    private static List<SourceLoader.Interceptor> templateInterceptors(Runtime runtime, Source source) {
+        if (!source.getInterceptors().isEmpty()) {
+            LOGGER.warn("Interceptors associated to the route template {} will be ignored", source.getName());
+        }
+
+        return List.of(
+            new SourceLoader.Interceptor() {
+                @Override
+                public SourceLoader.Result afterLoad(SourceLoader loader, Source source, SourceLoader.Result result) {
+                    RouteBuilder builder = result.builder()
+                        .map(RouteBuilder.class::cast)
+                        .orElseThrow(() -> new IllegalArgumentException("Unexpected routes builder type"));
+
+                    builder.addLifecycleInterceptor(new RouteBuilderLifecycleStrategy() {
+                        @Override
+                        public void afterConfigure(RouteBuilder builder) {
+                            List<RouteDefinition> routes = builder.getRouteCollection().getRoutes();
+                            List<RouteTemplateDefinition> templates = builder.getRouteTemplateCollection().getRouteTemplates();
+
+                            if (routes.size() != 1) {
+                                throw new IllegalArgumentException("There should be a single route definition, got " + routes.size());
+                            }
+                            if (!templates.isEmpty()) {
+                                throw new IllegalArgumentException("There should not be any template, got " + templates.size());
+                            }
+
+                            // create a new template from the source
+                            RouteTemplateDefinition templatesDefinition = builder.getRouteTemplateCollection().routeTemplate(source.getId());
+                            templatesDefinition.setRoute(routes.get(0));
+
+                            source.getPropertyNames().forEach(templatesDefinition::templateParameter);
+
+                            // remove all routes definitions as they have been translated
+                            // in the related route template
+                            routes.clear();
+                        }
+                    });
+
+                    return SourceLoader.Result.on(builder);
+                }
+            }
+        );
+    }
+}
diff --git a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/StringSupport.java b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/StringSupport.java
index 1ecaf2b..f850afb 100644
--- a/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/StringSupport.java
+++ b/camel-k-runtime-core/src/main/java/org/apache/camel/k/support/StringSupport.java
@@ -16,6 +16,9 @@
  */
 package org.apache.camel.k.support;
 
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.StringHelper;
 
@@ -23,47 +26,50 @@ public final class StringSupport {
     private StringSupport() {
     }
 
-    public static String substringBefore(final String str, final String separator) {
-        if (ObjectHelper.isEmpty(str) || separator == null) {
-            return str;
-        }
-        if (separator.isEmpty()) {
-            return "";
-        }
-        final int pos = str.indexOf(separator);
-        if (pos == -1) {
-            return str;
+    public static String substringBefore(String str, String separator) {
+        String answer = StringHelper.before(str, separator);
+        if (answer == null) {
+            answer = str;
         }
-        return str.substring(0, pos);
+
+        return answer;
     }
 
-    public static String substringAfter(final String str, final String separator) {
+    public static String substringAfter(String str, String separator) {
         String answer = StringHelper.after(str, separator);
-        return answer != null ? answer : "";
+        if (answer == null) {
+            answer = "";
+        }
+
+        return answer;
     }
 
-    public static String substringAfterLast(final String str, final String separator) {
+    public static String substringAfterLast(String str, String separator) {
         if (ObjectHelper.isEmpty(str)) {
             return str;
         }
         if (ObjectHelper.isEmpty(separator)) {
             return "";
         }
-        final int pos = str.lastIndexOf(separator);
+        int pos = str.lastIndexOf(separator);
         if (pos == -1 || pos == str.length() - separator.length()) {
             return "";
         }
         return str.substring(pos + separator.length());
     }
 
-    public static String substringBeforeLast(final String str, final String separator) {
+    public static String substringBeforeLast(String str, String separator) {
         if (ObjectHelper.isEmpty(str) || ObjectHelper.isEmpty(separator)) {
             return str;
         }
-        final int pos = str.lastIndexOf(separator);
+        int pos = str.lastIndexOf(separator);
         if (pos == -1) {
             return str;
         }
         return str.substring(0, pos);
     }
+
+    public static List<String> split(String input, String regex) {
+        return input != null ? List.of(input.split(regex)) : Collections.emptyList();
+    }
 }
diff --git a/camel-k-runtime-core/src/main/resources/META-INF/services/org.apache.camel.k.Runtime$Listener b/camel-k-runtime-core/src/main/resources/META-INF/services/org.apache.camel.k.Runtime$Listener
index 9caf5be..f0c50b7 100644
--- a/camel-k-runtime-core/src/main/resources/META-INF/services/org.apache.camel.k.Runtime$Listener
+++ b/camel-k-runtime-core/src/main/resources/META-INF/services/org.apache.camel.k.Runtime$Listener
@@ -16,5 +16,5 @@
 #
 
 org.apache.camel.k.listener.ContextConfigurer
-org.apache.camel.k.listener.RoutesConfigurer
+org.apache.camel.k.listener.SourcesConfigurer
 org.apache.camel.k.listener.PropertiesConfigurer
diff --git a/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java b/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java
index 4191de2..87279cf 100644
--- a/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java
+++ b/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java
@@ -18,6 +18,7 @@ package org.apache.camel.k;
 
 import org.junit.jupiter.api.Test;
 
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
 
 public class SourceTest {
@@ -42,4 +43,13 @@ public class SourceTest {
         );
     }
 
+    @Test
+    public void sourceCanBeContructedFromLocation() {
+        SourceDefinition definition = new SourceDefinition();
+        definition.setLocation("classpath:MyRoutes.java");
+
+        assertThat(Sources.fromDefinition(definition))
+            .hasFieldOrPropertyWithValue("name", "MyRoutes")
+            .hasFieldOrPropertyWithValue("language", "java");
+    }
 }
diff --git a/camel-k-runtime-core/src/test/java/org/apache/camel/k/support/PropertiesSupportTest.java b/camel-k-runtime-core/src/test/java/org/apache/camel/k/support/PropertiesSupportTest.java
new file mode 100644
index 0000000..6a91b58
--- /dev/null
+++ b/camel-k-runtime-core/src/test/java/org/apache/camel/k/support/PropertiesSupportTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.support;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.apache.camel.k.SourceDefinition;
+import org.apache.camel.k.listener.SourcesConfigurer;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.camel.k.test.CamelKTestSupport.asProperties;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class PropertiesSupportTest {
+    @Test
+    public void propertiesAreBoundToSourcesConfigurer() {
+        CamelContext context = new DefaultCamelContext();
+        context.getPropertiesComponent().setInitialProperties(asProperties(
+            "camel.k.sources[0].name", "MyRoutesWithBeans",
+            "camel.k.sources[0].location", "classpath:MyRoutesWithBeans.java",
+            "camel.k.sources[1].name", "MyRoutesConfig",
+            "camel.k.sources[1].location", "classpath:MyRoutesConfig.java",
+            "camel.k.sources[1].property-names[0]", "foo",
+            "camel.k.sources[1].property-names[1]", "bar"
+        ));
+
+        SourcesConfigurer configuration = new SourcesConfigurer();
+
+        PropertiesSupport.bindProperties(
+            context,
+            configuration,
+            k -> k.startsWith(SourcesConfigurer.CAMEL_K_SOURCES_PREFIX),
+            SourcesConfigurer.CAMEL_K_PREFIX);
+
+        assertThat(configuration.getSources())
+            .hasSize(2)
+            .anyMatch(byNameAndLocation("MyRoutesWithBeans", "classpath:MyRoutesWithBeans.java")
+                .and(d -> d.getPropertyNames() == null))
+            .anyMatch(byNameAndLocation("MyRoutesConfig", "classpath:MyRoutesConfig.java")
+                .and(d -> d.getPropertyNames() != null && d.getPropertyNames().containsAll(List.of("foo", "bar"))));
+    }
+
+    @Test
+    public void propertiesWithGapsAreBoundToSourcesConfigurer() {
+        CamelContext context = new DefaultCamelContext();
+        context.getPropertiesComponent().setInitialProperties(asProperties(
+            "camel.k.sources[0].name", "MyRoutesWithBeans",
+            "camel.k.sources[0].location", "classpath:MyRoutesWithBeans.java",
+            "camel.k.sources[2].name", "MyRoutesConfig",
+            "camel.k.sources[2].location", "classpath:MyRoutesConfig.java"
+        ));
+
+        SourcesConfigurer configuration = new SourcesConfigurer();
+
+        PropertiesSupport.bindProperties(
+            context,
+            configuration,
+            k -> k.startsWith(SourcesConfigurer.CAMEL_K_SOURCES_PREFIX),
+            SourcesConfigurer.CAMEL_K_PREFIX);
+
+        assertThat(configuration.getSources())
+            .hasSize(3)
+            .filteredOn(Objects::nonNull)
+            .hasSize(2)
+            .anyMatch(byNameAndLocation("MyRoutesWithBeans", "classpath:MyRoutesWithBeans.java"))
+            .anyMatch(byNameAndLocation("MyRoutesConfig", "classpath:MyRoutesConfig.java"));
+    }
+
+    // ***************************
+    //
+    // Helpers
+    //
+    // ***************************
+
+    private static Predicate<SourceDefinition> byNameAndLocation(String name, String location) {
+        return def -> Objects.equals(def.getName(), name) && Objects.equals(def.getLocation(), location);
+    }
+}
diff --git a/camel-k-runtime-cron/src/test/java/org/apache/camel/k/cron/CronTest.java b/camel-k-runtime-cron/src/test/java/org/apache/camel/k/cron/CronTest.java
index d90427a..5c30e15 100644
--- a/camel-k-runtime-cron/src/test/java/org/apache/camel/k/cron/CronTest.java
+++ b/camel-k-runtime-cron/src/test/java/org/apache/camel/k/cron/CronTest.java
@@ -22,8 +22,8 @@ import java.util.stream.Stream;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.k.listener.RoutesConfigurer;
 import org.apache.camel.k.main.ApplicationRuntime;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.support.LifecycleStrategySupport;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
@@ -39,7 +39,7 @@ public class CronTest {
         runtime.setProperties(
             "loader.interceptor.cron.overridable-components", cronOverride
         );
-        runtime.addListener(RoutesConfigurer.forRoutes(routes));
+        runtime.addListener(SourcesSupport.forRoutes(routes));
 
         // To check auto-termination of Camel context
         CountDownLatch termination = new CountDownLatch(1);
diff --git a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/customizer/KnativeSinkBindingCustomizerTest.java b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/customizer/KnativeSinkBindingCustomizerTest.java
index 7dab46d..c92199b 100644
--- a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/customizer/KnativeSinkBindingCustomizerTest.java
+++ b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/customizer/KnativeSinkBindingCustomizerTest.java
@@ -41,8 +41,8 @@ import org.apache.camel.k.Source;
 import org.apache.camel.k.SourceLoader;
 import org.apache.camel.k.Sources;
 import org.apache.camel.k.http.PlatformHttpServiceContextCustomizer;
-import org.apache.camel.k.listener.RoutesConfigurer;
 import org.apache.camel.k.support.RuntimeSupport;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.k.test.AvailablePortFinder;
 import org.junit.jupiter.api.Test;
 
@@ -106,7 +106,7 @@ public class KnativeSinkBindingCustomizerTest {
         RuntimeSupport.configureContextCustomizers(runtime);
 
         Source source = Sources.fromBytes("groovy", "from('direct:start').setBody().header('MyHeader').to('knative://endpoint/mySynk')".getBytes(StandardCharsets.UTF_8));
-        SourceLoader loader = RoutesConfigurer.load(runtime, source);
+        SourceLoader loader = SourcesSupport.load(runtime, source);
 
         assertThat(loader.getSupportedLanguages()).contains(source.getLanguage());
         assertThat(runtime.builders).hasSize(1);
diff --git a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/yaml/parser/KnativeConverterTest.java b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/yaml/parser/KnativeConverterTest.java
index fe818cd..88a265b 100644
--- a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/yaml/parser/KnativeConverterTest.java
+++ b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/knative/yaml/parser/KnativeConverterTest.java
@@ -27,8 +27,8 @@ 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.listener.RoutesConfigurer;
 import org.apache.camel.k.loader.yaml.YamlSourceLoader;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.model.FromDefinition;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.model.RouteDefinition;
@@ -43,7 +43,7 @@ public class KnativeConverterTest {
     public void testLoadRoutes() throws Exception {
         TestRuntime runtime = new TestRuntime();
         Source source = Sources.fromURI("classpath:route.yaml");
-        SourceLoader loader = RoutesConfigurer.load(runtime, source);
+        SourceLoader loader = SourcesSupport.load(runtime, source);
 
         assertThat(loader).isInstanceOf(YamlSourceLoader.class);
         assertThat(runtime.builders).hasSize(1);
diff --git a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/loader/knative/KnativeSourceRoutesLoaderTest.java b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/loader/knative/KnativeSourceRoutesLoaderTest.java
index 4a6e49c..2b7d4ff 100644
--- a/camel-k-runtime-knative/src/test/java/org/apache/camel/k/loader/knative/KnativeSourceRoutesLoaderTest.java
+++ b/camel-k-runtime-knative/src/test/java/org/apache/camel/k/loader/knative/KnativeSourceRoutesLoaderTest.java
@@ -37,7 +37,7 @@ import org.apache.camel.k.Source;
 import org.apache.camel.k.SourceLoader;
 import org.apache.camel.k.Sources;
 import org.apache.camel.k.http.PlatformHttpServiceContextCustomizer;
-import org.apache.camel.k.listener.RoutesConfigurer;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.k.test.AvailablePortFinder;
 import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.RouteDefinition;
@@ -80,7 +80,7 @@ public class KnativeSourceRoutesLoaderTest {
         context.addComponent(KnativeConstants.SCHEME, component);
 
         Source source = Sources.fromURI(uri);
-        SourceLoader loader = RoutesConfigurer.load(runtime, source);
+        SourceLoader loader = SourcesSupport.load(runtime, source);
 
         assertThat(loader.getSupportedLanguages()).contains(source.getLanguage());
         assertThat(runtime.builders).hasSize(1);
diff --git a/camel-k-runtime-webhook/src/test/java/org/apache/camel/k/webhook/WebhookTest.java b/camel-k-runtime-webhook/src/test/java/org/apache/camel/k/webhook/WebhookTest.java
index 768e4f6..010e85a 100644
--- a/camel-k-runtime-webhook/src/test/java/org/apache/camel/k/webhook/WebhookTest.java
+++ b/camel-k-runtime-webhook/src/test/java/org/apache/camel/k/webhook/WebhookTest.java
@@ -30,8 +30,8 @@ import org.apache.camel.FailedToCreateRouteException;
 import org.apache.camel.NamedNode;
 import org.apache.camel.Route;
 import org.apache.camel.k.listener.ContextConfigurer;
-import org.apache.camel.k.listener.RoutesConfigurer;
 import org.apache.camel.k.main.ApplicationRuntime;
+import org.apache.camel.k.support.SourcesSupport;
 import org.apache.camel.spi.RoutePolicy;
 import org.apache.camel.spi.RoutePolicyFactory;
 import org.apache.camel.support.RoutePolicySupport;
@@ -71,7 +71,7 @@ public class WebhookTest {
 
         AtomicBoolean routeStarted = new AtomicBoolean();
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:webhook.js"));
+        runtime.addListener(SourcesSupport.forRoutes("classpath:webhook.js"));
         runtime.getCamelContext().addRoutePolicyFactory(new RoutePolicyFactory() {
             @Override
             public RoutePolicy createRoutePolicy(CamelContext camelContext, String routeId, NamedNode route) {
@@ -132,7 +132,7 @@ public class WebhookTest {
         );
 
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:webhook.js"));
+        runtime.addListener(SourcesSupport.forRoutes("classpath:webhook.js"));
 
         Assertions.assertThrows(FailedToCreateRouteException.class, runtime::run);
     }
@@ -146,7 +146,7 @@ public class WebhookTest {
 
         runtime.getCamelContext().addComponent("dummy", new DummyWebhookComponent());
         runtime.addListener(new ContextConfigurer());
-        runtime.addListener(RoutesConfigurer.forRoutes("classpath:webhook.js"));
+        runtime.addListener(SourcesSupport.forRoutes("classpath:webhook.js"));
 
         Assertions.assertThrows(FailedToCreateRouteException.class, runtime::run);
     }
diff --git a/camel-kamelet/pom.xml b/camel-kamelet/pom.xml
index 33e09c3..33c0e76 100644
--- a/camel-kamelet/pom.xml
+++ b/camel-kamelet/pom.xml
@@ -21,11 +21,12 @@
     <parent>
         <groupId>org.apache.camel.k</groupId>
         <artifactId>camel-k-runtime-parent</artifactId>
-        <version>1.5.0-SNAPSHOT</version>
+        <version>1.5.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>camel-kamelet</artifactId>
+    <description>The Kamelet Component provides support for interacting with Knative</description>
 
     <dependencies>
 
@@ -76,6 +77,16 @@
             <artifactId>camel-log</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-test-junit5</artifactId>
+            <scope>test</scope>
+        </dependency>
 
     </dependencies>
 
diff --git a/camel-kamelet/src/generated/resources/org/apache/camel/component/kamelet/kamelet.json b/camel-kamelet/src/generated/resources/org/apache/camel/component/kamelet/kamelet.json
index 93ef496..2da7754 100644
--- a/camel-kamelet/src/generated/resources/org/apache/camel/component/kamelet/kamelet.json
+++ b/camel-kamelet/src/generated/resources/org/apache/camel/component/kamelet/kamelet.json
@@ -3,7 +3,7 @@
     "kind": "component",
     "name": "kamelet",
     "title": "Kamelet",
-    "description": "The Apache Software Foundation provides support for the Apache community of open-source software projects.\n    The Apache projects are characterized by a collaborative, consensus based development process, an open and\n    pragmatic software license, and a desire to create high quality software that leads the way in its field.\n    We consider ourselves not simply a group of projects sharing a server, but rather a community of developers\n    and users.",
+    "description": "The Kamelet Component provides support for interacting with Knative",
     "deprecated": false,
     "firstVersion": "3.5.0",
     "label": "camel-k",
@@ -11,14 +11,14 @@
     "supportLevel": "Preview",
     "groupId": "org.apache.camel.k",
     "artifactId": "camel-kamelet",
-    "version": "1.5.0-SNAPSHOT",
+    "version": "1.5.1-SNAPSHOT",
     "scheme": "kamelet",
     "extendsScheme": "",
     "syntax": "kamelet:templateId\/routeId",
     "async": false,
     "consumerOnly": false,
     "producerOnly": false,
-    "lenientProperties": false
+    "lenientProperties": true
   },
   "componentProperties": {
     "bridgeErrorHandler": { "kind": "property", "displayName": "Bridge Error Handler", "group": "consumer", "label": "consumer", "required": false, "type": "boolean", "javaType": "boolean", "deprecated": false, "secret": false, "defaultValue": false, "description": "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by [...]
diff --git a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java
index 043f44e..099c687 100644
--- a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java
+++ b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/Kamelet.java
@@ -16,10 +16,25 @@
  */
 package org.apache.camel.component.kamelet;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.function.Predicate;
 
+import org.apache.camel.CamelContext;
+import org.apache.camel.model.ModelCamelContext;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.spi.PropertiesComponent;
+import org.apache.camel.util.StringHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public final class Kamelet {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Kamelet.class);
+
     public static final String SCHEME = "kamelet";
+    public static final String PROPERTIES_PREFIX = "camel.kamelet.";
 
     private Kamelet() {
     }
@@ -27,4 +42,56 @@ public final class Kamelet {
     public static Predicate<String> startsWith(String prefix) {
         return item -> item.startsWith(prefix);
     }
+
+    public static void createRouteForEndpoint(KameletEndpoint endpoint) throws Exception {
+        LOGGER.debug("Creating route from template {}", endpoint.getTemplateId());
+
+        ModelCamelContext context = endpoint.getCamelContext().adapt(ModelCamelContext.class);
+        String id = context.addRouteFromTemplate(endpoint.getRouteId(), endpoint.getTemplateId(), endpoint.getKameletProperties());
+        RouteDefinition def = context.getRouteDefinition(id);
+        if (!def.isPrepared()) {
+            context.startRouteDefinitions(List.of(def));
+        }
+
+        LOGGER.debug("Route {} created from template {}", id, endpoint.getTemplateId());
+    }
+
+    public static  String extractTemplateId(CamelContext context, String remaining) {
+        String answer = StringHelper.before(remaining, "/");
+        if (answer == null) {
+            answer = remaining;
+        }
+
+        return answer;
+    }
+
+    public static  String extractRouteId(CamelContext context, String remaining) {
+        String answer = StringHelper.after(remaining, "/");
+        if (answer == null) {
+            answer = extractTemplateId(context, remaining) + "-" + context.getUuidGenerator().generateUuid();
+        }
+
+        return answer;
+    }
+
+    public static  Map<String, Object> extractKameletProperties(CamelContext context, String... elements) {
+        PropertiesComponent pc = context.getPropertiesComponent();
+        Map<String, Object> properties = new HashMap<>();
+        String prefix = Kamelet.PROPERTIES_PREFIX;
+
+        for (String element: elements) {
+            if (element == null) {
+                continue;
+            }
+
+            prefix = prefix + element + ".";
+
+            Properties prefixed = pc.loadProperties(Kamelet.startsWith(prefix));
+            for (String name : prefixed.stringPropertyNames()) {
+                properties.put(name.substring(prefix.length()), prefixed.getProperty(name));
+            }
+        }
+
+        return properties;
+    }
 }
diff --git a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletComponent.java b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletComponent.java
index 83898a2..7a97d9c 100644
--- a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletComponent.java
+++ b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletComponent.java
@@ -17,41 +17,39 @@
 package org.apache.camel.component.kamelet;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
 import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.model.ModelCamelContext;
-import org.apache.camel.model.RouteDefinition;
-import org.apache.camel.spi.CamelEvent;
+import org.apache.camel.VetoCamelContextStartException;
 import org.apache.camel.spi.annotations.Component;
 import org.apache.camel.support.DefaultComponent;
-import org.apache.camel.support.EventNotifierSupport;
-import org.apache.camel.support.service.ServiceHelper;
-import org.apache.camel.util.StringHelper;
+import org.apache.camel.support.LifecycleStrategySupport;
 
+/**
+ * The Kamelet Component provides support for interacting with <a href="https://knative.dev">Knative</a>.
+ */
 @Component(Kamelet.SCHEME)
 public class KameletComponent extends DefaultComponent {
+    private final LifecycleHandler lifecycleHandler;
+
     public KameletComponent() {
         this(null);
     }
 
     public KameletComponent(CamelContext context) {
         super(context);
-    }
 
-    // use as temporary to keep track of created kamelet endpoints during startup as we need to defer
-    // create routes from templates until camel context has finished loading all routes and whatnot
-    private final List<KameletEndpoint> endpoints = new ArrayList<>();
-    private volatile RouteTemplateEventNotifier notifier;
+        this.lifecycleHandler = new LifecycleHandler();
+    }
 
     @Override
     protected Endpoint createEndpoint(String uri, String remaining, Map<String, Object> parameters) throws Exception {
-        final String templateId = extractTemplateId(remaining);
-        final String routeId = extractRouteId(remaining);
+        final String templateId = Kamelet.extractTemplateId(getCamelContext(), remaining);
+        final String routeId = Kamelet.extractRouteId(getCamelContext(), remaining);
 
         //
         // The properties for the kamelets are determined by global properties
@@ -62,10 +60,10 @@ public class KameletComponent extends DefaultComponent {
         //   camel.kamelet." + templateId
         //   camel.kamelet." + templateId + "." routeId
         //
-        Map<String, Object> kameletProperties = extractKameletProperties(templateId, routeId);
+        Map<String, Object> kameletProperties = Kamelet.extractKameletProperties(getCamelContext(), templateId, routeId);
         kameletProperties.putAll(parameters);
-        kameletProperties.putIfAbsent("templateId", templateId);
-        kameletProperties.putIfAbsent("routeId", routeId);
+        kameletProperties.put("templateId", templateId);
+        kameletProperties.put("routeId", routeId);
 
         // Remaining parameter should be related to the route and to avoid the
         // parameters validation to fail, we need to clear the parameters map.
@@ -79,118 +77,73 @@ public class KameletComponent extends DefaultComponent {
         return endpoint;
     }
 
-    private String extractTemplateId(String remaining) {
-        String answer = StringHelper.before(remaining, "/");
-        if (answer == null) {
-            answer = remaining;
-        }
-
-        return answer;
-    }
-
-    private String extractRouteId(String remaining) {
-        String answer = StringHelper.after(remaining, "/");
-        if (answer == null) {
-            answer = extractTemplateId(remaining) + "-" + getCamelContext().getUuidGenerator().generateUuid();
-        }
-
-        return answer;
-    }
-
-    @SuppressWarnings("unchecked")
-    private Map<String, Object> extractKameletProperties(String... elements) {
-        Map<String, Object> properties = new HashMap<>();
-        String prefix = "camel.kamelet.";
-
-        for (String element: elements) {
-            if (element == null) {
-                continue;
-            }
-
-            prefix = prefix + element + ".";
-
-            properties.putAll(
-                (Map)getCamelContext().getPropertiesComponent().loadProperties(Kamelet.startsWith(prefix))
-            );
-
-        }
-
-        return properties;
-    }
-
     @Override
     protected void doInit() throws Exception {
-        super.doInit();
+        getCamelContext().addLifecycleStrategy(lifecycleHandler);
 
-        if (!getCamelContext().isRunAllowed()) {
-            // setup event listener which must be started to get triggered during initialization of camel context
-            notifier = new RouteTemplateEventNotifier(this);
-            ServiceHelper.startService(notifier);
-            getCamelContext().getManagementStrategy().addEventNotifier(notifier);
+        if (getCamelContext().isRunAllowed()) {
+            lifecycleHandler.setInitialized(true);
         }
+
+        super.doInit();
     }
 
     @Override
     protected void doStop() throws Exception {
-        if (notifier != null) {
-            ServiceHelper.stopService(notifier);
-            getCamelContext().getManagementStrategy().removeEventNotifier(notifier);
-            notifier = null;
-        }
+        getCamelContext().getLifecycleStrategies().remove(lifecycleHandler);
         super.doStop();
     }
 
     void onEndpointAdd(KameletEndpoint endpoint) {
-        if (notifier == null) {
-            try {
-                addRouteFromTemplate(endpoint);
-            } catch (Exception e) {
-                throw RuntimeCamelException.wrapRuntimeException(e);
-            }
-        } else {
-            // remember endpoints as we defer adding routes for them till later
-            this.endpoints.add(endpoint);
-        }
+        lifecycleHandler.track(endpoint);
     }
 
-    void addRouteFromTemplate(KameletEndpoint endpoint) throws Exception {
-        ModelCamelContext context = endpoint.getCamelContext().adapt(ModelCamelContext.class);
-        String id = context.addRouteFromTemplate(endpoint.getRouteId(), endpoint.getTemplateId(), endpoint.getKameletProperties());
-        RouteDefinition def = context.getRouteDefinition(id);
-        if (!def.isPrepared()) {
-            List<RouteDefinition> list = new ArrayList<>(1);
-            list.add(def);
-            context.startRouteDefinitions(list);
-        }
-    }
-
-    private static class RouteTemplateEventNotifier extends EventNotifierSupport {
-
-        private final KameletComponent component;
-
-        public RouteTemplateEventNotifier(KameletComponent component) {
-            this.component = component;
+    /*
+     * This LifecycleHandler is used to keep track of created kamelet endpoints during startup as
+     * we need to defer create routes from templates until camel context has finished loading
+     * all routes and whatnot.
+     *
+     * Once the camel context is initialized all the endpoint tracked by this LifecycleHandler will
+     * be used to create routes from templates.
+     */
+    private static class LifecycleHandler extends LifecycleStrategySupport {
+        private final List<KameletEndpoint> endpoints;
+        private final AtomicBoolean initialized;
+
+        public LifecycleHandler() {
+            this.endpoints = new ArrayList<>();
+            this.initialized = new AtomicBoolean();
         }
 
         @Override
-        public void notify(CamelEvent event) throws Exception {
-            for (KameletEndpoint endpoint : component.endpoints) {
-                component.addRouteFromTemplate(endpoint);
+        public void onContextInitialized(CamelContext context) throws VetoCamelContextStartException {
+            if (!this.initialized.compareAndExchange(false, true)) {
+                for (KameletEndpoint endpoint : endpoints) {
+                    try {
+                        Kamelet.createRouteForEndpoint(endpoint);
+                    } catch (Exception e) {
+                        throw new VetoCamelContextStartException("Failure creating route from template: " + endpoint.getTemplateId(), e, context);
+                    }
+                }
+
+                endpoints.clear();
             }
-            component.endpoints.clear();
-            // we were only needed during initializing/starting up camel, so remove after use
-            ServiceHelper.stopService(this);
-            component.getCamelContext().getManagementStrategy().removeEventNotifier(this);
-            component.notifier = null;
         }
 
-        @Override
-        public boolean isEnabled(CamelEvent event) {
-            // we only care about this event during startup as its triggered when
-            // all route and route template definitions have been added and prepared
-            // so this allows us to hook into the right moment
-            return event instanceof CamelEvent.CamelContextInitializedEvent;
+        public void setInitialized(boolean initialized) {
+            this.initialized.set(initialized);
         }
 
+        public void track(KameletEndpoint endpoint) {
+            if (this.initialized.get()) {
+                try {
+                    Kamelet.createRouteForEndpoint(endpoint);
+                } catch (Exception e) {
+                    throw RuntimeCamelException.wrapRuntimeException(e);
+                }
+            } else {
+                this.endpoints.add(endpoint);
+            }
+        }
     }
 }
diff --git a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletEndpoint.java b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletEndpoint.java
index 7609647..d8a98ec 100644
--- a/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletEndpoint.java
+++ b/camel-kamelet/src/main/java/org/apache/camel/component/kamelet/KameletEndpoint.java
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.component.kamelet;
 
+import java.util.Collections;
 import java.util.Map;
 
 import org.apache.camel.AsyncCallback;
@@ -32,12 +33,14 @@ import org.apache.camel.support.DefaultAsyncProducer;
 import org.apache.camel.support.DefaultConsumer;
 import org.apache.camel.support.DefaultEndpoint;
 import org.apache.camel.support.service.ServiceHelper;
+import org.apache.camel.util.ObjectHelper;
 
 @UriEndpoint(
     firstVersion = "3.5.0",
     scheme = "kamelet",
     syntax = "kamelet:templateId/routeId",
     title = "Kamelet",
+    lenientProperties = true,
     label = "camel-k")
 public class KameletEndpoint extends DefaultEndpoint {
     @Metadata(required = true)
@@ -50,6 +53,8 @@ public class KameletEndpoint extends DefaultEndpoint {
 
     private final Map<String, Object> kameletProperties;
     private final String kameletUri;
+    private Producer producer;
+    private Consumer consumer;
 
     public KameletEndpoint(
             String uri,
@@ -60,9 +65,13 @@ public class KameletEndpoint extends DefaultEndpoint {
 
         super(uri, component);
 
+        ObjectHelper.notNull(templateId, "template id");
+        ObjectHelper.notNull(routeId, "route id");
+        ObjectHelper.notNull(kameletProperties, "kamelet properties");
+
         this.templateId = templateId;
         this.routeId = routeId;
-        this.kameletProperties = kameletProperties;
+        this.kameletProperties = Collections.unmodifiableMap(kameletProperties);
         this.kameletUri = "direct:" + routeId;
     }
 
@@ -71,6 +80,11 @@ public class KameletEndpoint extends DefaultEndpoint {
         return (KameletComponent) super.getComponent();
     }
 
+    @Override
+    public boolean isLenientProperties() {
+        return true;
+    }
+
     public String getTemplateId() {
         return templateId;
     }
@@ -98,8 +112,8 @@ public class KameletEndpoint extends DefaultEndpoint {
     @Override
     protected void doInit() throws Exception {
         super.doInit();
-        // only need to add during init phase
         getComponent().onEndpointAdd(this);
+
     }
 
     // *********************************
diff --git a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletAddAfterCamelStartedTest.java b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletAddAfterCamelStartedTest.java
deleted file mode 100644
index fdc9dc6..0000000
--- a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletAddAfterCamelStartedTest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.camel.component.kamelet;
-
-import java.util.UUID;
-
-import org.apache.camel.CamelContext;
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.impl.DefaultCamelContext;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-public class KameletAddAfterCamelStartedTest {
-    private static final Logger LOGGER = LoggerFactory.getLogger(KameletAddAfterCamelStartedTest.class);
-
-    @Test
-    public void test() throws Exception {
-        String body = UUID.randomUUID().toString();
-
-        CamelContext context = new DefaultCamelContext();
-        context.addRoutes(new RouteBuilder() {
-            @Override
-            public void configure() throws Exception {
-                routeTemplate("setBody")
-                    .templateParameter("bodyValue")
-                    .from("direct:{{routeId}}")
-                    .setBody().constant("{{bodyValue}}");
-            }
-        });
-
-        /*
-        context.addRouteFromTemplate("setBody")
-            .routeId("test")
-            .parameter("routeId", "test")
-            .parameter("bodyValue", body)
-            .build();
-         */
-
-        // start camel here and add routes with kamelts later
-        context.start();
-
-        context.addRoutes(new RouteBuilder() {
-            @Override
-            public void configure() throws Exception {
-                // routes
-                from("direct:template")
-                    .toF("kamelet:setBody/test?bodyValue=%s", body)
-                    .to("log:1");
-            }
-        });
-
-        assertThat(
-            context.createFluentProducerTemplate().to("direct:template").withBody("test").request(String.class)
-        ).isEqualTo(body);
-
-        context.stop();
-    }
-}
diff --git a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletBasicTest.java b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletBasicTest.java
new file mode 100644
index 0000000..e023e07
--- /dev/null
+++ b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletBasicTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.kamelet;
+
+import java.util.UUID;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit5.CamelTestSupport;
+import org.apache.http.annotation.Obsolete;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class KameletBasicTest extends CamelTestSupport {
+    @Test
+    public void canProduceToKamelet()  {
+        String body = UUID.randomUUID().toString();
+
+        assertThat(
+            fluentTemplate.toF("kamelet:setBody/test?bodyValue=%s", body).request(String.class)
+        ).isEqualTo(body);
+    }
+
+    @Test
+    public void canConsumeFromKamelet() {
+        assertThat(
+            consumer.receiveBody("kamelet:tick", Integer.class)
+        ).isEqualTo(1);
+    }
+
+    @Test
+    public void kameletCanBeCreatedWhileContextIsStarting() {
+        assertThat(
+            fluentTemplate.to("direct:templateEmbedded").request(String.class)
+        ).isEqualTo("embedded");
+    }
+
+    @Test
+    public void kameletCanBeCreatedAfterContextIsStarted() throws Exception {
+        String body = UUID.randomUUID().toString();
+
+        RouteBuilder.addRoutes(context, b -> {
+            b.from("direct:templateAfter")
+                .toF("kamelet:setBody/test?bodyValue=%s", body);
+        });
+
+        assertThat(
+            fluentTemplate.to("direct:templateAfter").request(String.class)
+        ).isEqualTo(body);
+    }
+
+    // **********************************************
+    //
+    // test set-up
+    //
+    // **********************************************
+
+    @Obsolete
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                routeTemplate("setBody")
+                    .templateParameter("bodyValue")
+                    .from("direct:{{routeId}}")
+                    .setBody().constant("{{bodyValue}}");
+
+                routeTemplate("tick")
+                    .from("timer:{{routeId}}?repeatCount=1&delay=-1")
+                    .setBody().exchangeProperty(Exchange.TIMER_COUNTER)
+                    .to("direct:{{routeId}}");
+
+                from("direct:templateEmbedded")
+                    .toF("kamelet:setBody/embedded?bodyValue=embedded");
+            }
+        };
+    }
+}
diff --git a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletPropertiesTest.java b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletPropertiesTest.java
new file mode 100644
index 0000000..d33a15b
--- /dev/null
+++ b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletPropertiesTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.kamelet;
+
+import java.util.Properties;
+
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit5.CamelTestSupport;
+import org.apache.http.annotation.Obsolete;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.camel.k.test.CamelKTestSupport.asProperties;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class KameletPropertiesTest extends CamelTestSupport {
+    @Test
+    public void propertiesAreTakenFromRouteId() throws Exception {
+        assertThat(
+            fluentTemplate
+                .to("kamelet:setBody/test")
+                .request(String.class)
+        ).isEqualTo("from-route");
+    }
+
+    @Test
+    public void propertiesAreTakenFromTemplateId() throws Exception {
+        assertThat(
+            fluentTemplate
+                .to("kamelet:setBody")
+                .request(String.class)
+        ).isEqualTo("from-template");
+    }
+
+    @Test
+    public void propertiesAreTakenFromURI() {
+        assertThat(
+            fluentTemplate
+                .to("kamelet:setBody?bodyValue={{bodyValue}}")
+                .request(String.class)
+        ).isEqualTo("from-uri");
+    }
+
+    // **********************************************
+    //
+    // test set-up
+    //
+    // **********************************************
+
+    @Override
+    protected Properties useOverridePropertiesWithPropertiesComponent() {
+        return asProperties(
+            "bodyValue", "from-uri",
+            Kamelet.PROPERTIES_PREFIX + "setBody.bodyValue", "from-template",
+            Kamelet.PROPERTIES_PREFIX + "setBody.test.bodyValue", "from-route"
+        );
+    }
+
+    @Obsolete
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // template
+                routeTemplate("setBody")
+                    .templateParameter("bodyValue")
+                    .from("direct:{{routeId}}")
+                    .setBody().constant("{{bodyValue}}");
+            }
+        };
+    }
+}
diff --git a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletTest.java b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletValidationTest.java
similarity index 57%
rename from camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletTest.java
rename to camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletValidationTest.java
index 32634a0..e16cf10 100644
--- a/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletTest.java
+++ b/camel-kamelet/src/test/java/org/apache/camel/component/kamelet/KameletValidationTest.java
@@ -16,24 +16,18 @@
  */
 package org.apache.camel.component.kamelet;
 
-import java.util.UUID;
-
 import org.apache.camel.CamelContext;
+import org.apache.camel.RuntimeCamelException;
+import org.apache.camel.VetoCamelContextStartException;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.assertj.core.api.Assertions.assertThat;
 
-public class KameletTest {
-    private static final Logger LOGGER = LoggerFactory.getLogger(KameletTest.class);
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
 
+public class KameletValidationTest {
     @Test
-    public void test() throws Exception {
-        String body = UUID.randomUUID().toString();
-
+    public void validation() throws Exception {
         CamelContext context = new DefaultCamelContext();
         context.addRoutes(new RouteBuilder() {
             @Override
@@ -42,33 +36,14 @@ public class KameletTest {
                     .templateParameter("bodyValue")
                     .from("direct:{{routeId}}")
                     .setBody().constant("{{bodyValue}}");
-            }
-        });
-
-        /*
-        context.addRouteFromTemplate("setBody")
-            .routeId("test")
-            .parameter("routeId", "test")
-            .parameter("bodyValue", body)
-            .build();
-         */
 
-        context.addRoutes(new RouteBuilder() {
-            @Override
-            public void configure() throws Exception {
-                // routes
-                from("direct:template")
-                    .toF("kamelet:setBody/test?bodyValue=%s", body)
-                    .to("log:1");
+                from("direct:start")
+                    .to("kamelet:setBody/test");
             }
         });
 
-        context.start();
-
-        assertThat(
-            context.createFluentProducerTemplate().to("direct:template").withBody("test").request(String.class)
-        ).isEqualTo(body);
-
-        context.stop();
+        assertThatExceptionOfType(RuntimeCamelException.class)
+            .isThrownBy(context::start)
+            .withCauseExactlyInstanceOf(VetoCamelContextStartException.class);
     }
 }
diff --git a/camel-kamelet/src/test/resources/log4j2-test.xml b/camel-kamelet/src/test/resources/log4j2-test.xml
index 486a0f0..d5df1ad 100644
--- a/camel-kamelet/src/test/resources/log4j2-test.xml
+++ b/camel-kamelet/src/test/resources/log4j2-test.xml
@@ -22,15 +22,20 @@
     <Console name="STDOUT" target="SYSTEM_OUT">
       <PatternLayout pattern="%d{yyyy-MM-dd HH:mm:ss.SSS}|%-5level|%t|%c{1} - %msg%n"/>
     </Console>
+    <File name="FILE" filename="target/camel-kamelet-test.log">
+      <PatternLayout pattern="%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"/>
+    </File>
     <Null name="NONE"/>
   </Appenders>
 
   <Loggers>
+    <Logger name="org.apache.camel.component.kamelet" level="TRACE"/>
+
     <Root level="INFO">
       <!--
       <AppenderRef ref="STDOUT"/>
       -->
-      <AppenderRef ref="STDOUT"/>
+      <AppenderRef ref="FILE"/>
     </Root>
   </Loggers>
 
diff --git a/examples/camel-k-runtime-example-groovy/data/application.properties b/examples/camel-k-runtime-example-groovy/data/application.properties
index ac98d6e..3be9292 100644
--- a/examples/camel-k-runtime-example-groovy/data/application.properties
+++ b/examples/camel-k-runtime-example-groovy/data/application.properties
@@ -16,7 +16,7 @@
 ## ---------------------------------------------------------------------------
 
 #
-# Logging
+# logging
 #
 logging.level.org.apache.camel.k = DEBUG
 
@@ -27,5 +27,14 @@ camel.main.name = camel-k
 camel.main.stream-caching-enabled = true
 camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
 
+#
+# camel-k - sources
+#
+camel.k.sources[0].name     = routes
+camel.k.sources[0].language = groovy
+camel.k.sources[0].location = file:{{data.dir}}/routes.groovy
 
+#
+# misc
+#
 message = test
diff --git a/examples/camel-k-runtime-example-groovy/pom.xml b/examples/camel-k-runtime-example-groovy/pom.xml
index 48dd01f..129b306 100644
--- a/examples/camel-k-runtime-example-groovy/pom.xml
+++ b/examples/camel-k-runtime-example-groovy/pom.xml
@@ -69,8 +69,8 @@
                             <value>${project.basedir}/data/application.properties</value>
                         </systemProperty>
                         <systemProperty>
-                            <key>camel.k.routes</key>
-                            <value>file:${project.basedir}/data/routes.groovy</value>
+                            <key>data.dir</key>
+                            <value>${project.basedir}/data</value>
                         </systemProperty>
                     </systemProperties>
                 </configuration>
diff --git a/examples/camel-k-runtime-example-groovy/data/application.properties b/examples/camel-k-runtime-example-kamelets/data/application.properties
similarity index 72%
copy from examples/camel-k-runtime-example-groovy/data/application.properties
copy to examples/camel-k-runtime-example-kamelets/data/application.properties
index ac98d6e..3985ab1 100644
--- a/examples/camel-k-runtime-example-groovy/data/application.properties
+++ b/examples/camel-k-runtime-example-kamelets/data/application.properties
@@ -16,16 +16,31 @@
 ## ---------------------------------------------------------------------------
 
 #
-# Logging
+# logging
 #
-logging.level.org.apache.camel.k = DEBUG
+logging.level.org.apache.camel.k = INFO
 
 #
 # camel - main
 #
+camel.main.auto-configuration-log-summary = false
 camel.main.name = camel-k
 camel.main.stream-caching-enabled = true
 camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
 
+#
+# camel-k - sources (templates)
+#
+camel.k.sources[0].location = file:{{data.dir}}/logger.yaml
+camel.k.sources[0].type     = template
 
-message = test
+#
+# camel-k - sources (routes)
+#
+camel.k.sources[1].location = file:{{data.dir}}/routes.groovy
+camel.k.sources[1].type     = source
+
+#
+# misc
+#
+camel.kamelet.logger.message = default
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test b/examples/camel-k-runtime-example-kamelets/data/logger.yaml
similarity index 85%
copy from camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
copy to examples/camel-k-runtime-example-kamelets/data/logger.yaml
index a393d0e..9cbcafc 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
+++ b/examples/camel-k-runtime-example-kamelets/data/logger.yaml
@@ -6,7 +6,7 @@
 # (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
+#      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,
@@ -15,4 +15,7 @@
 # limitations under the License.
 #
 
-class=org.apache.camel.k.main.TestCustomizer
\ No newline at end of file
+- from:
+    uri: "direct:{{routeId}}"
+    steps:
+      - log: "{{message}}"
\ No newline at end of file
diff --git a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java b/examples/camel-k-runtime-example-kamelets/data/routes.groovy
similarity index 73%
rename from camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
rename to examples/camel-k-runtime-example-kamelets/data/routes.groovy
index 2069e08..f8fe0cb 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/java/org/apache/camel/k/main/MyBean.java
+++ b/examples/camel-k-runtime-example-kamelets/data/routes.groovy
@@ -14,21 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k.main;
 
-public class MyBean {
-    private final String name;
-
-    public MyBean(String name) {
-        this.name = name;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    @Override
-    public String toString() {
-        return name;
-    }
-}
\ No newline at end of file
+from('timer:tick')
+    .to('kamelet:logger/k1?message=logger-1')
+    .to('kamelet:logger/k2')
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-groovy/pom.xml b/examples/camel-k-runtime-example-kamelets/pom.xml
similarity index 84%
copy from examples/camel-k-runtime-example-groovy/pom.xml
copy to examples/camel-k-runtime-example-kamelets/pom.xml
index 48dd01f..68f0170 100644
--- a/examples/camel-k-runtime-example-groovy/pom.xml
+++ b/examples/camel-k-runtime-example-kamelets/pom.xml
@@ -25,7 +25,7 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>camel-k-runtime-example-groovy</artifactId>
+    <artifactId>camel-k-runtime-example-kamelets</artifactId>
 
     <dependencies>
         <dependency>
@@ -37,6 +37,14 @@
             <artifactId>camel-k-loader-groovy</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-k-loader-yaml</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.k</groupId>
+            <artifactId>camel-kamelet</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.camel</groupId>
             <artifactId>camel-log</artifactId>
         </dependency>
@@ -52,7 +60,7 @@
             <plugin>
                 <groupId>org.codehaus.mojo</groupId>
                 <artifactId>exec-maven-plugin</artifactId>
-                <version>${exec-maven-plugin-version}</version>
+                <version>${exec-maven-plugin.version}</version>
                 <executions>
                     <execution>
                         <goals>
@@ -69,8 +77,8 @@
                             <value>${project.basedir}/data/application.properties</value>
                         </systemProperty>
                         <systemProperty>
-                            <key>camel.k.routes</key>
-                            <value>file:${project.basedir}/data/routes.groovy</value>
+                            <key>data.dir</key>
+                            <value>${project.basedir}/data</value>
                         </systemProperty>
                     </systemProperties>
                 </configuration>
diff --git a/examples/camel-k-runtime-example-quarkus-kamelets/Dockerfile b/examples/camel-k-runtime-example-quarkus-kamelets/Dockerfile
new file mode 100644
index 0000000..29dede0
--- /dev/null
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/Dockerfile
@@ -0,0 +1,7 @@
+FROM registry.access.redhat.com/ubi8/ubi-minimal
+
+RUN mkdir -p /opt/camel-k
+COPY target/camel-k-runtime-example-quarkus-kamelets-runner /opt/camel-k/runner
+RUN chmod u+x /opt/camel-k/runner
+
+ENTRYPOINT [ "/opt/camel-k/runner" ]
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-groovy/data/application.properties b/examples/camel-k-runtime-example-quarkus-kamelets/data/application.properties
similarity index 72%
copy from examples/camel-k-runtime-example-groovy/data/application.properties
copy to examples/camel-k-runtime-example-quarkus-kamelets/data/application.properties
index ac98d6e..25628f5 100644
--- a/examples/camel-k-runtime-example-groovy/data/application.properties
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/data/application.properties
@@ -16,16 +16,31 @@
 ## ---------------------------------------------------------------------------
 
 #
-# Logging
+# logging
 #
-logging.level.org.apache.camel.k = DEBUG
+logging.level.org.apache.camel.k = INFO
 
 #
 # camel - main
 #
+camel.main.auto-configuration-log-summary = false
 camel.main.name = camel-k
 camel.main.stream-caching-enabled = true
 camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
 
+#
+# camel-k - sources (templates)
+#
+camel.k.sources[0].location = file:{{data.dir}}/logger.yaml
+camel.k.sources[0].type     = template
 
-message = test
+#
+# camel-k - sources (routes)
+#
+camel.k.sources[1].location = file:{{data.dir}}/routes.yaml
+camel.k.sources[1].type     = source
+
+#
+# misc
+#
+camel.kamelet.logger.message = default
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test b/examples/camel-k-runtime-example-quarkus-kamelets/data/logger.yaml
similarity index 85%
copy from camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
copy to examples/camel-k-runtime-example-quarkus-kamelets/data/logger.yaml
index a393d0e..9cbcafc 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/data/logger.yaml
@@ -6,7 +6,7 @@
 # (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
+#      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,
@@ -15,4 +15,7 @@
 # limitations under the License.
 #
 
-class=org.apache.camel.k.main.TestCustomizer
\ No newline at end of file
+- from:
+    uri: "direct:{{routeId}}"
+    steps:
+      - log: "{{message}}"
\ No newline at end of file
diff --git a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test b/examples/camel-k-runtime-example-quarkus-kamelets/data/routes.yaml
similarity index 81%
copy from camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
copy to examples/camel-k-runtime-example-quarkus-kamelets/data/routes.yaml
index a393d0e..a50acc2 100644
--- a/camel-k-main/camel-k-runtime-main/src/test/resources/META-INF/services/org/apache/camel/k/customizer/test
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/data/routes.yaml
@@ -6,7 +6,7 @@
 # (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
+#      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,
@@ -15,4 +15,8 @@
 # limitations under the License.
 #
 
-class=org.apache.camel.k.main.TestCustomizer
\ No newline at end of file
+- from:
+    uri: "timer:tick"
+    steps:
+      - to: "kamelet:logger/k1?message=logger-1"
+      - to: "kamelet:logger/k2"
\ No newline at end of file
diff --git a/examples/camel-k-runtime-example-quarkus-kamelets/pom.xml b/examples/camel-k-runtime-example-quarkus-kamelets/pom.xml
new file mode 100644
index 0000000..ba06a0a
--- /dev/null
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/pom.xml
@@ -0,0 +1,161 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <parent>
+        <groupId>org.apache.camel.k</groupId>
+        <artifactId>camel-k-runtime-examples</artifactId>
+        <version>1.5.1-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>camel-k-runtime-example-quarkus-kamelets</artifactId>
+
+    <properties>
+        <noDeps>true</noDeps>
+        <quarkus.camel.main.routes-discovery.enabled>false</quarkus.camel.main.routes-discovery.enabled>
+    </properties>
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.camel.quarkus</groupId>
+                <artifactId>camel-quarkus-bom</artifactId>
+                <version>${camel-quarkus.version}</version>
+                <type>pom</type>
+                <scope>import</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
+    <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-kamelet</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-log</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel.quarkus</groupId>
+            <artifactId>camel-quarkus-timer</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>io.quarkus</groupId>
+                <artifactId>quarkus-bootstrap-maven-plugin</artifactId>
+                <version>${quarkus.version}</version>
+            </plugin>
+            <plugin>
+                <groupId>io.quarkus</groupId>
+                <artifactId>quarkus-maven-plugin</artifactId>
+                <version>${quarkus.version}</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>build</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <finalName>${project.artifactId}</finalName>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <version>${exec-maven-plugin.version}</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>exec</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <executable>java</executable>
+                    <workingDirectory>${project.basedir}</workingDirectory>
+                    <arguments>
+                        <argument>-jar</argument>
+                        <argument>${project.build.directory}/${project.artifactId}-runner.jar</argument>
+                    </arguments>
+                    <environmentVariables>
+                        <CAMEL_K_CONF>${project.basedir}/data/application.properties</CAMEL_K_CONF>
+                        <DATA_DIR>${project.basedir}/data</DATA_DIR>
+                    </environmentVariables>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>native</id>
+            <activation>
+                <activeByDefault>false</activeByDefault>
+                <property>
+                    <name>native</name>
+                </property>
+            </activation>
+            <properties>
+                <quarkus.package.output-name>${project.artifactId}</quarkus.package.output-name>
+                <quarkus.package.type>native</quarkus.package.type>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <version>${exec-maven-plugin.version}</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>exec</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                        <configuration>
+                            <executable>${project.build.directory}/${project.artifactId}-runner</executable>
+                            <arguments combine.self="override" />
+                            <workingDirectory>${project.basedir}</workingDirectory>
+                            <environmentVariables>
+                                <CAMEL_K_CONF>${project.basedir}/data/application.properties</CAMEL_K_CONF>
+                                <DATA_DIR>${project.basedir}/data</DATA_DIR>
+                            </environmentVariables>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/examples/camel-k-runtime-example-groovy/data/application.properties b/examples/camel-k-runtime-example-quarkus-kamelets/src/main/resources/application.properties
similarity index 80%
copy from examples/camel-k-runtime-example-groovy/data/application.properties
copy to examples/camel-k-runtime-example-quarkus-kamelets/src/main/resources/application.properties
index ac98d6e..f56a6e1 100644
--- a/examples/camel-k-runtime-example-groovy/data/application.properties
+++ b/examples/camel-k-runtime-example-quarkus-kamelets/src/main/resources/application.properties
@@ -16,16 +16,9 @@
 ## ---------------------------------------------------------------------------
 
 #
-# Logging
+# quarkus
 #
-logging.level.org.apache.camel.k = DEBUG
-
-#
-# camel - main
-#
-camel.main.name = camel-k
-camel.main.stream-caching-enabled = true
-camel.main.stream-caching-spool-directory = ${java.io.tmpdir}/camel-k
-
-
-message = test
+quarkus.banner.enabled = false
+quarkus.log.console.enable = true
+quarkus.log.level = INFO
+quarkus.log.category."org.apache.camel".level = INFO
diff --git a/examples/pom.xml b/examples/pom.xml
index 28bf5e9..960d8f8 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -37,9 +37,11 @@
         <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-kamelets</module>
         <module>camel-k-runtime-example-knative</module>
         <module>camel-k-runtime-example-quarkus-js</module>
         <module>camel-k-runtime-example-quarkus-yaml</module>
+        <module>camel-k-runtime-example-quarkus-kamelets</module>
         <module>camel-k-runtime-example-quarkus-knative</module>
         <module>camel-k-runtime-example-quarkus-kotlin</module>
         <module>camel-k-runtime-example-quarkus-groovy</module>
diff --git a/pom.xml b/pom.xml
index d774404..b3b2ad6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -38,36 +38,10 @@
         <maven.compiler.source>11</maven.compiler.source>
         <maven.compiler.target>11</maven.compiler.target>
 
-        <camel.version>3.4.2</camel.version>
-        <camel-quarkus.version>1.0.0</camel-quarkus.version>
-        <junit.version>5.6.2</junit.version>
-        <junit-pioneer.version>0.9.0</junit-pioneer.version>
-        <joor.version>0.9.13</joor.version>
-        <assertj.version>3.17.1</assertj.version>
-        <log4j2.version>2.13.3</log4j2.version>
-        <groovy.version>3.0.5</groovy.version>
-        <kotlin.version>1.4.0</kotlin.version>
-        <snakeyaml.version>1.26</snakeyaml.version>
-        <spock.version>2.0-M3-groovy-3.0</spock.version>
-        <jackson.version>2.10.5</jackson.version>
-        <jaxb-api.version>2.3.1</jaxb-api.version>
-        <jaxb-core.version>2.3.0.1</jaxb-core.version>
-        <immutables.version>2.8.8</immutables.version>
-        <semver4j.version>3.1.0</semver4j.version>
-        <vertx.version>3.9.2</vertx.version>
-        <graalvm.version>20.1.0</graalvm.version>
-        <quarkus.version>1.7.0.Final</quarkus.version>
-        <jandex.version>2.2.1.Final</jandex.version>
-        <javapoet.version>1.13.0</javapoet.version>
-        <rest-assured.version>4.3.1</rest-assured.version>
-        <hamcrest.version>2.2</hamcrest.version>
-        <jaeger.version>1.3.2</jaeger.version>
-        <json-schema-validator>2.2.14</json-schema-validator>
-
         <camel-version>3.5.0-SNAPSHOT</camel-version>
 
         <!-- quarkus -->
-        <camel-quarkus-version>1.0.0</camel-quarkus-version>
+        <camel-quarkus-version>1.1.0-SNAPSHOT</camel-quarkus-version>
         <graalvm-version>20.1.0</graalvm-version>
         <quarkus-version>1.7.0.Final</quarkus-version>
 
@@ -549,6 +523,16 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.camel.k</groupId>
+                <artifactId>camel-k-quarkus-kamelet</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.camel.k</groupId>
+                <artifactId>camel-k-quarkus-kamelet-deployment</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.camel.k</groupId>
                 <artifactId>camel-k-quarkus-cron</artifactId>
                 <version>${project.version}</version>
             </dependency>
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 07d67be..aa5c45d 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
@@ -184,6 +184,23 @@ public class CatalogProcessor3x implements CatalogProcessor {
                     MavenArtifact.from("org.apache.camel.k", "camel-k-quarkus-knative"))
                 .build()
         );
+
+        specBuilder.putArtifact(
+            new CamelArtifact.Builder()
+                .groupId("org.apache.camel.k")
+                .artifactId("camel-kamelet")
+                .addScheme(new CamelScheme.Builder()
+                    .id("kamelet")
+                    .http(true)
+                    .build())
+                .addDependencies(
+                    () -> catalog.getRuntimeProvider() instanceof DefaultRuntimeProvider,
+                    MavenArtifact.from("org.apache.camel.k", "camel-kamelet"))
+                .addDependencies(
+                    () -> catalog.getRuntimeProvider() instanceof QuarkusRuntimeProvider,
+                    MavenArtifact.from("org.apache.camel.k", "camel-k-quarkus-kamelet"))
+                .build()
+        );
     }
 
     private static void processLoaders(CamelCatalog catalog, CamelCatalogSpec.Builder specBuilder) {
diff --git a/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java b/tooling/camel-k-test/src/main/java/org/apache/camel/k/test/CamelKTestSupport.java
similarity index 50%
copy from camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java
copy to tooling/camel-k-test/src/main/java/org/apache/camel/k/test/CamelKTestSupport.java
index 4191de2..05c6e64 100644
--- a/camel-k-runtime-core/src/test/java/org/apache/camel/k/SourceTest.java
+++ b/tooling/camel-k-test/src/main/java/org/apache/camel/k/test/CamelKTestSupport.java
@@ -14,32 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.k;
+package org.apache.camel.k.test;
 
-import org.junit.jupiter.api.Test;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
 
-import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
-
-public class SourceTest {
-    @Test
-    public void testResourceWithoutScheme() {
-        assertThatExceptionOfType(IllegalArgumentException.class).isThrownBy(
-            () -> Sources.fromURI("routes.js")
-        );
+public final class CamelKTestSupport {
+    private CamelKTestSupport() {
     }
 
-    @Test
-    public void testResourceWithIllegalScheme() {
-        assertThatExceptionOfType(IllegalArgumentException.class).isThrownBy(
-            () -> Sources.fromURI("http:routes.js")
-        );
-    }
+    public static Properties asProperties(String... properties) {
+        if ((properties.length & 1) != 0) {
+            throw new InternalError("length is odd");
+        }
+
+        Properties answer = new Properties();
+        for (int i = 0; i < properties.length; i += 2) {
+            answer.setProperty(
+                Objects.requireNonNull(properties[i]),
+                Objects.requireNonNull(properties[i + 1]));
+        }
 
-    @Test
-    public void testUnsupportedLanguage() {
-        assertThatExceptionOfType(IllegalArgumentException.class).isThrownBy(
-            () -> Sources.fromURI("  test")
-        );
+        return answer;
     }
 
+    public static Properties asProperties(Map<String, Object> properties) {
+        Properties answer = new Properties();
+        answer.putAll(properties);
+
+        return answer;
+    }
 }