You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by nf...@apache.org on 2018/12/14 08:52:44 UTC

[camel-k] 06/08: runtime: use camel built-in service loader to find routes loaders

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

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

commit 6b33a811a1d4ddd4c672d37aa47145aa0a813363
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Thu Dec 13 14:03:09 2018 +0100

    runtime: use camel built-in service loader to find routes loaders
---
 .../src/main/java/org/apache/camel/k/Language.java |  16 +-
 .../services/org.apache.camel.k.RoutesLoader       |   1 -
 .../services/org/apache/camel/k/loader/groovy      |  18 ++
 .../org/apache/camel/k/groovy/LoaderTest.groovy    |   5 +-
 .../java/org/apache/camel/k/jvm/RoutesLoaders.java | 184 ---------------------
 .../main/java/org/apache/camel/k/jvm/Runtime.java  |   3 +-
 .../org/apache/camel/k/jvm/RuntimeSupport.java     |  16 ++
 .../apache/camel/k/jvm/loader/JavaClassLoader.java |  34 ++++
 .../camel/k/jvm/loader/JavaScriptLoader.java       |  58 +++++++
 .../camel/k/jvm/loader/JavaSourceLoader.java       |  45 +++++
 .../org/apache/camel/k/jvm/loader/XmlLoader.java   |  50 ++++++
 .../services/org.apache.camel.k.RoutesLoader       |   4 -
 .../services/org/apache/camel/k/loader/java-class  |  18 ++
 .../services/org/apache/camel/k/loader/java-source |  18 ++
 .../META-INF/services/org/apache/camel/k/loader/js |  18 ++
 .../services/org/apache/camel/k/loader/xml         |  18 ++
 .../org/apache/camel/k/jvm/RoutesLoadersTest.java  |  33 ++--
 .../services/javax.script.ScriptEngineFactory      |   1 -
 .../services/org.apache.camel.k.RoutesLoader       |   1 -
 .../services/org/apache/camel/k/loader/kotlin      |  18 ++
 .../kotlin/org/apache/camel/k/kotlin/LoaderTest.kt |   7 +-
 .../apache/camel/k/spring/boot/Application.java    |   3 +-
 22 files changed, 355 insertions(+), 214 deletions(-)

diff --git a/runtime/core/src/main/java/org/apache/camel/k/Language.java b/runtime/core/src/main/java/org/apache/camel/k/Language.java
index bfeffa5..c338bd6 100644
--- a/runtime/core/src/main/java/org/apache/camel/k/Language.java
+++ b/runtime/core/src/main/java/org/apache/camel/k/Language.java
@@ -20,39 +20,53 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.camel.util.ObjectHelper;
 import org.apache.commons.lang3.StringUtils;
 
 public enum Language {
     Unknown(
+        "unknown",
         Collections.emptyList(),
         Collections.emptyList()),
     JavaClass(
+        "java-class",
         Collections.singletonList("class"),
         Collections.singletonList("class")),
     JavaSource(
+        "java-source",
         Collections.singletonList("java"),
         Collections.singletonList("java")),
     JavaScript(
+        "js",
         Arrays.asList("js", "javascript"),
         Collections.singletonList("js")),
     Groovy(
+        "groovy",
         Collections.singletonList("groovy"),
         Collections.singletonList("groovy")),
     Xml(
+        "xml",
         Collections.singletonList("xml"),
         Collections.singletonList("xml")),
     Kotlin(
+        "kotlin",
         Arrays.asList("kotlin", "kts"),
         Collections.singletonList("kts"));
 
+    private final String id;
     private final List<String> names;
     private final List<String> extensions;
 
-    Language(List<String> names, List<String> extensions) {
+    Language(String id, List<String> names, List<String> extensions) {
+        this.id = ObjectHelper.notNull(id, "id");
         this.names = names;
         this.extensions = extensions;
     }
 
+    public String getId() {
+        return id;
+    }
+
     public List<String> getNames() {
         return names;
     }
diff --git a/runtime/groovy/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader b/runtime/groovy/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
deleted file mode 100644
index db214e0..0000000
--- a/runtime/groovy/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.camel.k.groovy.GroovyRoutesLoader
\ No newline at end of file
diff --git a/runtime/groovy/src/main/resources/META-INF/services/org/apache/camel/k/loader/groovy b/runtime/groovy/src/main/resources/META-INF/services/org/apache/camel/k/loader/groovy
new file mode 100644
index 0000000..ba7720a
--- /dev/null
+++ b/runtime/groovy/src/main/resources/META-INF/services/org/apache/camel/k/loader/groovy
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.groovy.GroovyRoutesLoader
\ No newline at end of file
diff --git a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/LoaderTest.groovy b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/LoaderTest.groovy
index a9c8a0a..0bc194f 100644
--- a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/LoaderTest.groovy
+++ b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/LoaderTest.groovy
@@ -16,7 +16,8 @@
  */
 package org.apache.camel.k.groovy
 
-import org.apache.camel.k.jvm.RoutesLoaders
+import org.apache.camel.impl.DefaultCamelContext
+import org.apache.camel.k.jvm.RuntimeSupport
 import org.apache.camel.k.jvm.SimpleRuntimeRegistry
 import org.apache.camel.k.Source
 import org.apache.camel.model.ToDefinition
@@ -29,7 +30,7 @@ class LoaderTest extends Specification {
             def source = Source.create("classpath:routes.groovy")
 
         when:
-            def loader = RoutesLoaders.loaderFor(source)
+            def loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source)
             def builder = loader.load(new SimpleRuntimeRegistry(), source)
 
         then:
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RoutesLoaders.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RoutesLoaders.java
deleted file mode 100644
index 5702931..0000000
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RoutesLoaders.java
+++ /dev/null
@@ -1,184 +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.jvm;
-
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.List;
-import java.util.ServiceLoader;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import javax.script.Bindings;
-import javax.script.ScriptEngine;
-import javax.script.ScriptEngineManager;
-import javax.script.SimpleBindings;
-import javax.xml.bind.UnmarshalException;
-
-import org.apache.camel.CamelContext;
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.k.Constants;
-import org.apache.camel.k.Language;
-import org.apache.camel.k.RoutesLoader;
-import org.apache.camel.k.RuntimeRegistry;
-import org.apache.camel.k.Source;
-import org.apache.camel.k.jvm.dsl.Components;
-import org.apache.camel.model.RouteDefinition;
-import org.apache.camel.model.rest.RestConfigurationDefinition;
-import org.apache.camel.model.rest.RestDefinition;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.joor.Reflect;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public final class RoutesLoaders {
-    private static final Logger LOGGER = LoggerFactory.getLogger(RoutesLoaders.class);
-
-    private RoutesLoaders() {
-    }
-
-    public static class JavaClass implements RoutesLoader {
-        @Override
-        public List<Language> getSupportedLanguages() {
-            return Collections.singletonList(Language.JavaClass);
-        }
-
-        @Override
-        public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
-            String path = source.getLocation();
-            path = StringUtils.removeStart(path, Constants.SCHEME_CLASSPATH);
-            path = StringUtils.removeEnd(path, ".class");
-
-            Class<?> type = Class.forName(path);
-
-            if (!RouteBuilder.class.isAssignableFrom(type)) {
-                throw new IllegalStateException("The class provided (" + path + ") is not a org.apache.camel.builder.RouteBuilder");
-            }
-
-            return (RouteBuilder)type.newInstance();
-        }
-    }
-
-    public static class JavaSource implements RoutesLoader {
-        @Override
-        public List<Language> getSupportedLanguages() {
-            return Collections.singletonList(Language.JavaSource);
-        }
-
-        @Override
-        public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
-            return new RouteBuilder() {
-                @Override
-                public void configure() throws Exception {
-                    try (InputStream is = URIResolver.resolve(getContext(), source)) {
-                        String name = StringUtils.substringAfter(source.getLocation(), ":");
-                        name = StringUtils.removeEnd(name, ".java");
-
-                        if (name.contains("/")) {
-                            name = StringUtils.substringAfterLast(name, "/");
-                        }
-
-                        // Wrap routes builder
-                        includeRoutes(
-                            Reflect.compile(name, IOUtils.toString(is, StandardCharsets.UTF_8)).create().get()
-                        );
-                    }
-                }
-            };
-        }
-    }
-
-    public static class JavaScript implements RoutesLoader {
-        @Override
-        public List<Language> getSupportedLanguages() {
-            return Collections.singletonList(Language.JavaScript);
-        }
-
-        @Override
-        public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
-            return new RouteBuilder() {
-                @Override
-                public void configure() throws Exception {
-                    final CamelContext context = getContext();
-                    final ScriptEngineManager manager = new ScriptEngineManager();
-                    final ScriptEngine engine = manager.getEngineByName("nashorn");
-                    final Bindings bindings = new SimpleBindings();
-
-                    // Exposed to the underlying script, but maybe better to have
-                    // a nice dsl
-                    bindings.put("builder", this);
-                    bindings.put("context", context);
-                    bindings.put("components", new Components(context));
-                    bindings.put("registry", registry);
-                    bindings.put("from", (Function<String, RouteDefinition>) uri -> from(uri));
-                    bindings.put("rest", (Supplier<RestDefinition>) () -> rest());
-                    bindings.put("restConfiguration", (Supplier<RestConfigurationDefinition>) () -> restConfiguration());
-
-                    try (InputStream is = URIResolver.resolve(context, source)) {
-                        engine.eval(new InputStreamReader(is), bindings);
-                    }
-                }
-            };
-        }
-    }
-
-    public static class Xml implements RoutesLoader {
-        @Override
-        public List<Language> getSupportedLanguages() {
-            return Collections.singletonList(Language.Xml);
-        }
-
-        @Override
-        public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
-            return new RouteBuilder() {
-                @Override
-                public void configure() throws Exception {
-                    try (InputStream is = URIResolver.resolve(getContext(), source)) {
-                        try {
-                            setRouteCollection(
-                                getContext().loadRoutesDefinition(is)
-                            );
-                        } catch (UnmarshalException e) {
-                            LOGGER.debug("Unable to load RoutesDefinition: {}", e.getMessage());
-                        }
-
-                        try {
-                            setRestCollection(
-                                getContext().loadRestsDefinition(is)
-                            );
-                        } catch (UnmarshalException e) {
-                            LOGGER.debug("Unable to load RestsDefinition: {}", e.getMessage());
-                        }
-                    }
-                }
-            };
-        }
-    }
-
-
-    public static RoutesLoader loaderFor(Source source) {
-        for (RoutesLoader loader: ServiceLoader.load(RoutesLoader.class)) {
-            if (loader.getSupportedLanguages().contains(source.getLanguage())) {
-                return loader;
-            }
-        }
-
-        throw new IllegalArgumentException("Unable to find loader for: " + source);
-    }
-}
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Runtime.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Runtime.java
index 287ba69..3923e85 100644
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Runtime.java
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Runtime.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+
 import org.apache.camel.CamelContext;
 import org.apache.camel.ProducerTemplate;
 import org.apache.camel.builder.RouteBuilder;
@@ -46,7 +47,7 @@ public final class Runtime extends MainSupport {
     public void load(String[] routes) throws Exception {
         for (String route: routes) {
             final Source source = Source.create(route);
-            final RoutesLoader loader = RoutesLoaders.loaderFor(source);
+            final RoutesLoader loader = RuntimeSupport.loaderFor(getCamelContext(), source);
             final RouteBuilder builder = loader.load(registry, source);
 
             if (builder == null) {
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RuntimeSupport.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RuntimeSupport.java
index d6a005c..b3192b6 100644
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RuntimeSupport.java
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/RuntimeSupport.java
@@ -32,7 +32,9 @@ import org.apache.camel.CamelContext;
 import org.apache.camel.NoFactoryAvailableException;
 import org.apache.camel.component.properties.PropertiesComponent;
 import org.apache.camel.k.Constants;
+import org.apache.camel.k.RoutesLoader;
 import org.apache.camel.k.RuntimeTrait;
+import org.apache.camel.k.Source;
 import org.apache.camel.spi.FactoryFinder;
 import org.apache.camel.util.IntrospectionSupport;
 import org.apache.camel.util.ObjectHelper;
@@ -180,4 +182,18 @@ public final class RuntimeSupport {
                 }
             );
     }
+
+    public static RoutesLoader loaderFor(CamelContext context, Source source) {
+        final FactoryFinder finder;
+        final RoutesLoader loader;
+
+        try {
+            finder = context.getFactoryFinder(Constants.ROUTES_LOADER_RESOURCE_PATH);
+            loader = (RoutesLoader)finder.newInstance(source.getLanguage().getId());
+        } catch (NoFactoryAvailableException e) {
+            throw new IllegalArgumentException("Unable to find loader for: " + source, e);
+        }
+
+        return loader;
+    }
 }
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaClassLoader.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaClassLoader.java
new file mode 100644
index 0000000..bf73a82
--- /dev/null
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaClassLoader.java
@@ -0,0 +1,34 @@
+package org.apache.camel.k.jvm.loader;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.Constants;
+import org.apache.camel.k.Language;
+import org.apache.camel.k.RoutesLoader;
+import org.apache.camel.k.RuntimeRegistry;
+import org.apache.camel.k.Source;
+import org.apache.commons.lang3.StringUtils;
+
+public class JavaClassLoader implements RoutesLoader {
+    @Override
+    public List<Language> getSupportedLanguages() {
+        return Collections.singletonList(Language.JavaClass);
+    }
+
+    @Override
+    public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
+        String path = source.getLocation();
+        path = StringUtils.removeStart(path, Constants.SCHEME_CLASSPATH);
+        path = StringUtils.removeEnd(path, ".class");
+
+        Class<?> type = Class.forName(path);
+
+        if (!RouteBuilder.class.isAssignableFrom(type)) {
+            throw new IllegalStateException("The class provided (" + path + ") is not a org.apache.camel.builder.RouteBuilder");
+        }
+
+        return (RouteBuilder)type.newInstance();
+    }
+}
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaScriptLoader.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaScriptLoader.java
new file mode 100644
index 0000000..650a4d1
--- /dev/null
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaScriptLoader.java
@@ -0,0 +1,58 @@
+package org.apache.camel.k.jvm.loader;
+
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.script.Bindings;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.SimpleBindings;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.Language;
+import org.apache.camel.k.RoutesLoader;
+import org.apache.camel.k.RuntimeRegistry;
+import org.apache.camel.k.Source;
+import org.apache.camel.k.jvm.URIResolver;
+import org.apache.camel.k.jvm.dsl.Components;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.model.rest.RestConfigurationDefinition;
+import org.apache.camel.model.rest.RestDefinition;
+
+public class JavaScriptLoader implements RoutesLoader {
+    @Override
+    public List<Language> getSupportedLanguages() {
+        return Collections.singletonList(Language.JavaScript);
+    }
+
+    @Override
+    public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                final CamelContext context = getContext();
+                final ScriptEngineManager manager = new ScriptEngineManager();
+                final ScriptEngine engine = manager.getEngineByName("nashorn");
+                final Bindings bindings = new SimpleBindings();
+
+                // Exposed to the underlying script, but maybe better to have
+                // a nice dsl
+                bindings.put("builder", this);
+                bindings.put("context", context);
+                bindings.put("components", new Components(context));
+                bindings.put("registry", registry);
+                bindings.put("from", (Function<String, RouteDefinition>) uri -> from(uri));
+                bindings.put("rest", (Supplier<RestDefinition>) () -> rest());
+                bindings.put("restConfiguration", (Supplier<RestConfigurationDefinition>) () -> restConfiguration());
+
+                try (InputStream is = URIResolver.resolve(context, source)) {
+                    engine.eval(new InputStreamReader(is), bindings);
+                }
+            }
+        };
+    }
+}
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaSourceLoader.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaSourceLoader.java
new file mode 100644
index 0000000..767a6d4
--- /dev/null
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/JavaSourceLoader.java
@@ -0,0 +1,45 @@
+package org.apache.camel.k.jvm.loader;
+
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.Language;
+import org.apache.camel.k.RoutesLoader;
+import org.apache.camel.k.RuntimeRegistry;
+import org.apache.camel.k.Source;
+import org.apache.camel.k.jvm.URIResolver;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.joor.Reflect;
+
+public class JavaSourceLoader implements RoutesLoader {
+    @Override
+    public List<Language> getSupportedLanguages() {
+        return Collections.singletonList(Language.JavaSource);
+    }
+
+    @Override
+    public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                try (InputStream is = URIResolver.resolve(getContext(), source)) {
+                    String name = StringUtils.substringAfter(source.getLocation(), ":");
+                    name = StringUtils.removeEnd(name, ".java");
+
+                    if (name.contains("/")) {
+                        name = StringUtils.substringAfterLast(name, "/");
+                    }
+
+                    // Wrap routes builder
+                    includeRoutes(
+                        Reflect.compile(name, IOUtils.toString(is, StandardCharsets.UTF_8)).create().get()
+                    );
+                }
+            }
+        };
+    }
+}
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/XmlLoader.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/XmlLoader.java
new file mode 100644
index 0000000..7532f14
--- /dev/null
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/loader/XmlLoader.java
@@ -0,0 +1,50 @@
+package org.apache.camel.k.jvm.loader;
+
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.List;
+import javax.xml.bind.UnmarshalException;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.k.Language;
+import org.apache.camel.k.RoutesLoader;
+import org.apache.camel.k.RuntimeRegistry;
+import org.apache.camel.k.Source;
+import org.apache.camel.k.jvm.URIResolver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class XmlLoader implements RoutesLoader {
+    private static final Logger LOGGER = LoggerFactory.getLogger(XmlLoader.class);
+
+    @Override
+    public List<Language> getSupportedLanguages() {
+        return Collections.singletonList(Language.Xml);
+    }
+
+    @Override
+    public RouteBuilder load(RuntimeRegistry registry, Source source) throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                try (InputStream is = URIResolver.resolve(getContext(), source)) {
+                    try {
+                        setRouteCollection(
+                            getContext().loadRoutesDefinition(is)
+                        );
+                    } catch (UnmarshalException e) {
+                        LOGGER.debug("Unable to load RoutesDefinition: {}", e.getMessage());
+                    }
+
+                    try {
+                        setRestCollection(
+                            getContext().loadRestsDefinition(is)
+                        );
+                    } catch (UnmarshalException e) {
+                        LOGGER.debug("Unable to load RestsDefinition: {}", e.getMessage());
+                    }
+                }
+            }
+        };
+    }
+}
diff --git a/runtime/jvm/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader b/runtime/jvm/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
deleted file mode 100644
index 5a57927..0000000
--- a/runtime/jvm/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
+++ /dev/null
@@ -1,4 +0,0 @@
-org.apache.camel.k.jvm.RoutesLoaders$JavaClass
-org.apache.camel.k.jvm.RoutesLoaders$JavaSource
-org.apache.camel.k.jvm.RoutesLoaders$JavaScript
-org.apache.camel.k.jvm.RoutesLoaders$Xml
diff --git a/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-class b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-class
new file mode 100644
index 0000000..2d85f95
--- /dev/null
+++ b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-class
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.jvm.loader.JavaClassLoader
\ No newline at end of file
diff --git a/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-source b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-source
new file mode 100644
index 0000000..3bf8297
--- /dev/null
+++ b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/java-source
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.jvm.loader.JavaSourceLoader
\ No newline at end of file
diff --git a/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/js b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/js
new file mode 100644
index 0000000..45227e8
--- /dev/null
+++ b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/js
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.jvm.loader.JavaScriptLoader
\ No newline at end of file
diff --git a/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/xml b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/xml
new file mode 100644
index 0000000..e30ce1e
--- /dev/null
+++ b/runtime/jvm/src/main/resources/META-INF/services/org/apache/camel/k/loader/xml
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.jvm.loader.XmlLoader
\ No newline at end of file
diff --git a/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RoutesLoadersTest.java b/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RoutesLoadersTest.java
index cd32327..cc3b7c3 100644
--- a/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RoutesLoadersTest.java
+++ b/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RoutesLoadersTest.java
@@ -19,8 +19,13 @@ package org.apache.camel.k.jvm;
 import java.util.List;
 
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.k.RoutesLoader;
 import org.apache.camel.k.Source;
+import org.apache.camel.k.jvm.loader.JavaClassLoader;
+import org.apache.camel.k.jvm.loader.JavaScriptLoader;
+import org.apache.camel.k.jvm.loader.JavaSourceLoader;
+import org.apache.camel.k.jvm.loader.XmlLoader;
 import org.apache.camel.model.ProcessDefinition;
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.model.SetBodyDefinition;
@@ -35,10 +40,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadClass() throws Exception {
         Source source = Source.create("classpath:" + MyRoutes.class.getName() + ".class");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaClass.class);
+        assertThat(loader).isInstanceOf(JavaClassLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -52,10 +57,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadJava() throws Exception {
         Source source = Source.create("classpath:MyRoutes.java");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaSource.class);
+        assertThat(loader).isInstanceOf(JavaSourceLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -69,10 +74,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadJavaWithNestedClass() throws Exception {
         Source source = Source.create("classpath:MyRoutesWithNestedClass.java");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaSource.class);
+        assertThat(loader).isInstanceOf(JavaSourceLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -88,10 +93,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadJavaScript() throws Exception {
         Source source = Source.create("classpath:routes.js");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaScript.class);
+        assertThat(loader).isInstanceOf(JavaScriptLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -105,10 +110,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadCompressedRoute() throws Exception {
         Source source = Source.create("classpath:routes-compressed.js.gz.b64?language=js&compression=true");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaScript.class);
+        assertThat(loader).isInstanceOf(JavaScriptLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -122,10 +127,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadJavaScriptWithCustomExtension() throws Exception {
         Source source = Source.create("classpath:routes.mytype?language=js");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.JavaScript.class);
+        assertThat(loader).isInstanceOf(JavaScriptLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
@@ -139,10 +144,10 @@ public class RoutesLoadersTest {
     @Test
     public void testLoadXml() throws Exception {
         Source source = Source.create("classpath:routes.xml");
-        RoutesLoader loader = RoutesLoaders.loaderFor(source);
+        RoutesLoader loader = RuntimeSupport.loaderFor(new DefaultCamelContext(), source);
         RouteBuilder builder = loader.load(new SimpleRuntimeRegistry(), source);
 
-        assertThat(loader).isInstanceOf(RoutesLoaders.Xml.class);
+        assertThat(loader).isInstanceOf(XmlLoader.class);
         assertThat(builder).isNotNull();
 
         builder.configure();
diff --git a/runtime/kotlin/src/main/resources/META-INF/services/javax.script.ScriptEngineFactory b/runtime/kotlin/src/main/resources/META-INF/services/javax.script.ScriptEngineFactory
deleted file mode 100644
index f8f5900..0000000
--- a/runtime/kotlin/src/main/resources/META-INF/services/javax.script.ScriptEngineFactory
+++ /dev/null
@@ -1 +0,0 @@
-org.jetbrains.kotlin.script.jsr223.KotlinJsr223JvmLocalScriptEngineFactory
\ No newline at end of file
diff --git a/runtime/kotlin/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader b/runtime/kotlin/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
deleted file mode 100644
index 83c3f09..0000000
--- a/runtime/kotlin/src/main/resources/META-INF/services/org.apache.camel.k.RoutesLoader
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.camel.k.kotlin.KotlinRoutesLoader
\ No newline at end of file
diff --git a/runtime/kotlin/src/main/resources/META-INF/services/org/apache/camel/k/loader/kotlin b/runtime/kotlin/src/main/resources/META-INF/services/org/apache/camel/k/loader/kotlin
new file mode 100644
index 0000000..d4bcde1
--- /dev/null
+++ b/runtime/kotlin/src/main/resources/META-INF/services/org/apache/camel/k/loader/kotlin
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+class=org.apache.camel.k.kotlin.KotlinRoutesLoader
\ No newline at end of file
diff --git a/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/LoaderTest.kt b/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/LoaderTest.kt
index c9fb931..c0f86c6 100644
--- a/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/LoaderTest.kt
+++ b/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/LoaderTest.kt
@@ -16,9 +16,10 @@
  */
 package org.apache.camel.k.kotlin
 
-import org.apache.camel.k.jvm.RoutesLoaders
-import org.apache.camel.k.jvm.SimpleRuntimeRegistry
+import org.apache.camel.impl.DefaultCamelContext
 import org.apache.camel.k.Source
+import org.apache.camel.k.jvm.RuntimeSupport
+import org.apache.camel.k.jvm.SimpleRuntimeRegistry
 import org.apache.camel.model.ProcessDefinition
 import org.apache.camel.model.ToDefinition
 import org.assertj.core.api.Assertions.assertThat
@@ -29,7 +30,7 @@ class LoaderTest {
     @Test
     fun `load route from classpath`() {
         var source = Source.create("classpath:routes.kts")
-        val loader = RoutesLoaders.loaderFor(source)
+        val loader = RuntimeSupport.loaderFor(DefaultCamelContext(), source)
         val builder = loader.load(SimpleRuntimeRegistry(), source)
 
         assertThat(loader).isInstanceOf(KotlinRoutesLoader::class.java)
diff --git a/runtime/spring-boot/src/main/java/org/apache/camel/k/spring/boot/Application.java b/runtime/spring-boot/src/main/java/org/apache/camel/k/spring/boot/Application.java
index eaf7c2b..427dd9e 100644
--- a/runtime/spring-boot/src/main/java/org/apache/camel/k/spring/boot/Application.java
+++ b/runtime/spring-boot/src/main/java/org/apache/camel/k/spring/boot/Application.java
@@ -25,7 +25,6 @@ import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.k.RuntimeRegistry;
 import org.apache.camel.k.Constants;
 import org.apache.camel.k.RoutesLoader;
-import org.apache.camel.k.jvm.RoutesLoaders;
 import org.apache.camel.k.jvm.RuntimeSupport;
 import org.apache.camel.k.Source;
 import org.apache.camel.spi.Registry;
@@ -95,7 +94,7 @@ public class Application {
                 try {
                     for (String route : routes.split(",")) {
                         final Source source = Source.create(route);
-                        final RoutesLoader loader = RoutesLoaders.loaderFor(source);
+                        final RoutesLoader loader = RuntimeSupport.loaderFor(context, source);
                         final RouteBuilder builder = loader.load(registry, source);
 
                         if (builder == null) {