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/11/22 13:50:30 UTC

[camel-k] 01/02: Support for multiple integration definitions #45

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 0eea5e78310fcc6f777207ec363cc3bdf12f5a37
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Wed Nov 21 23:53:09 2018 +0100

    Support for multiple integration definitions #45
---
 pkg/apis/camel/v1alpha1/types.go                   |   7 +-
 pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go   |  32 +++---
 pkg/client/cmd/run.go                              | 128 +++++++++++----------
 pkg/stub/action/integration/initialize.go          |  10 +-
 pkg/trait/dependencies.go                          |  26 +++--
 pkg/trait/deployment.go                            | 123 ++++++++++++++------
 pkg/trait/knative.go                               |  31 ++++-
 pkg/trait/trait_test.go                            |   8 +-
 pkg/util/digest/digest.go                          |   7 +-
 pkg/util/kubernetes/collection.go                  |   5 +
 runtime/examples/routes.js                         |   8 +-
 runtime/examples/simple.groovy                     |   6 +
 runtime/examples/simple.js                         |   6 +
 .../org/apache/camel/k/groovy/LoaderTest.groovy    |   4 +-
 .../camel/k/groovy/dsl/IntegrationTest.groovy      |   6 +-
 .../k/groovy/dsl/extension/LogExtensionTest.groovy |   2 +-
 .../java/org/apache/camel/k/jvm/Application.java   |   9 +-
 .../java/org/apache/camel/k/jvm/Constants.java     |   5 +-
 .../java/org/apache/camel/k/jvm/RoutesLoaders.java |  38 +++---
 .../main/java/org/apache/camel/k/jvm/Runtime.java  |  27 +++--
 .../org/apache/camel/k/jvm/RuntimeSupport.java     |   5 +-
 .../java/org/apache/camel/k/jvm/URIResolver.java   |  29 +++--
 .../java/org/apache/camel/k/jvm/RuntimeTest.java   |  51 ++++++++
 runtime/jvm/src/test/resources/r1.js               |   4 +
 runtime/jvm/src/test/resources/r2.mytype           |   4 +
 .../apache/camel/k/kotlin/KotlinRoutesLoader.kt    |  10 +-
 .../apache/camel/k/kotlin/dsl/IntegrationTest.kt   |   6 +-
 27 files changed, 391 insertions(+), 206 deletions(-)

diff --git a/pkg/apis/camel/v1alpha1/types.go b/pkg/apis/camel/v1alpha1/types.go
index b872ac3..691b528 100644
--- a/pkg/apis/camel/v1alpha1/types.go
+++ b/pkg/apis/camel/v1alpha1/types.go
@@ -50,7 +50,7 @@ type Integration struct {
 // IntegrationSpec --
 type IntegrationSpec struct {
 	Replicas      *int32                          `json:"replicas,omitempty"`
-	Source        SourceSpec                      `json:"source,omitempty"`
+	Sources       []SourceSpec                    `json:"sources,omitempty"`
 	Context       string                          `json:"context,omitempty"`
 	Dependencies  []string                        `json:"dependencies,omitempty"`
 	Profile       TraitProfile                    `json:"profile,omitempty"`
@@ -58,6 +58,11 @@ type IntegrationSpec struct {
 	Configuration []ConfigurationSpec             `json:"configuration,omitempty"`
 }
 
+// AddSource --
+func (is *IntegrationSpec) AddSource(name string, content string, language Language) {
+	is.Sources = append(is.Sources, SourceSpec{Name: name, Content: content, Language: language})
+}
+
 // SourceSpec --
 type SourceSpec struct {
 	Name     string   `json:"name,omitempty"`
diff --git a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
index 202fd25..4aa466b 100644
--- a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
+++ b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
@@ -337,28 +337,32 @@ func (in *IntegrationPlatformStatus) DeepCopy() *IntegrationPlatformStatus {
 }
 
 // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
-func (in *IntegrationSpec) DeepCopyInto(out *IntegrationSpec) {
-	*out = *in
-	if in.Replicas != nil {
-		in, out := &in.Replicas, &out.Replicas
+func (is *IntegrationSpec) DeepCopyInto(out *IntegrationSpec) {
+	*out = *is
+	if is.Replicas != nil {
+		in, out := &is.Replicas, &out.Replicas
 		*out = new(int32)
 		**out = **in
 	}
-	out.Source = in.Source
-	if in.Dependencies != nil {
-		in, out := &in.Dependencies, &out.Dependencies
+	if is.Sources != nil {
+		in, out := &is.Sources, &out.Sources
+		*out = make([]SourceSpec, len(*in))
+		copy(*out, *in)
+	}
+	if is.Dependencies != nil {
+		in, out := &is.Dependencies, &out.Dependencies
 		*out = make([]string, len(*in))
 		copy(*out, *in)
 	}
-	if in.Traits != nil {
-		in, out := &in.Traits, &out.Traits
+	if is.Traits != nil {
+		in, out := &is.Traits, &out.Traits
 		*out = make(map[string]IntegrationTraitSpec, len(*in))
 		for key, val := range *in {
 			(*out)[key] = *val.DeepCopy()
 		}
 	}
-	if in.Configuration != nil {
-		in, out := &in.Configuration, &out.Configuration
+	if is.Configuration != nil {
+		in, out := &is.Configuration, &out.Configuration
 		*out = make([]ConfigurationSpec, len(*in))
 		copy(*out, *in)
 	}
@@ -366,12 +370,12 @@ func (in *IntegrationSpec) DeepCopyInto(out *IntegrationSpec) {
 }
 
 // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new IntegrationSpec.
-func (in *IntegrationSpec) DeepCopy() *IntegrationSpec {
-	if in == nil {
+func (is *IntegrationSpec) DeepCopy() *IntegrationSpec {
+	if is == nil {
 		return nil
 	}
 	out := new(IntegrationSpec)
-	in.DeepCopyInto(out)
+	is.DeepCopyInto(out)
 	return out
 }
 
diff --git a/pkg/client/cmd/run.go b/pkg/client/cmd/run.go
index bf2c268..bfc0445 100644
--- a/pkg/client/cmd/run.go
+++ b/pkg/client/cmd/run.go
@@ -20,29 +20,31 @@ package cmd
 import (
 	"encoding/json"
 	"fmt"
-	"github.com/operator-framework/operator-sdk/pkg/util/k8sutil"
-	"gopkg.in/yaml.v2"
 	"io/ioutil"
-	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
-	"k8s.io/apimachinery/pkg/runtime"
 	"net/http"
 	"os"
 	"os/signal"
+	"path"
 	"regexp"
 	"strconv"
 	"strings"
 	"syscall"
 
+	"github.com/operator-framework/operator-sdk/pkg/util/k8sutil"
+	"gopkg.in/yaml.v2"
+	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
+	"k8s.io/apimachinery/pkg/runtime"
+
 	"github.com/apache/camel-k/pkg/trait"
 	"github.com/apache/camel-k/pkg/util"
 
-	"github.com/apache/camel-k/pkg/util/sync"
 	"github.com/pkg/errors"
 	"github.com/sirupsen/logrus"
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
 	"github.com/apache/camel-k/pkg/util/log"
+	"github.com/apache/camel-k/pkg/util/sync"
 	"github.com/apache/camel-k/pkg/util/watch"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
 	"github.com/spf13/cobra"
@@ -67,7 +69,6 @@ func newCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 		RunE:  options.run,
 	}
 
-	cmd.Flags().StringVarP(&options.Language, "language", "l", "", "Programming Language used to write the file")
 	cmd.Flags().StringVarP(&options.Runtime, "runtime", "r", "", "Runtime used by the integration")
 	cmd.Flags().StringVar(&options.IntegrationName, "name", "", "The integration name")
 	cmd.Flags().StringSliceVarP(&options.Dependencies, "dependency", "d", nil, "The integration dependency")
@@ -93,7 +94,6 @@ func newCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 type runCmdOptions struct {
 	*RootCmdOptions
 	IntegrationContext string
-	Language           string
 	Runtime            string
 	IntegrationName    string
 	Dependencies       []string
@@ -111,22 +111,27 @@ type runCmdOptions struct {
 }
 
 func (o *runCmdOptions) validateArgs(cmd *cobra.Command, args []string) error {
-	if len(args) != 1 {
-		return errors.New("accepts 1 arg, received " + strconv.Itoa(len(args)))
+	if len(args) < 1 {
+		return errors.New("accepts at least 1 arg, received 0")
 	}
-	fileName := args[0]
-	if !strings.HasPrefix(fileName, "http://") && !strings.HasPrefix(fileName, "https://") {
-		if _, err := os.Stat(fileName); err != nil && os.IsNotExist(err) {
-			return errors.Wrap(err, "file "+fileName+" does not exist")
-		} else if err != nil {
-			return errors.Wrap(err, "error while accessing file "+fileName)
-		}
-	} else {
-		resp, err := http.Get(fileName)
-		if err != nil {
-			return errors.Wrap(err, "The URL provided is not reachable")
-		} else if resp.StatusCode != 200 {
-			return errors.New("The URL provided is not reachable " + fileName + " The error code returned is " + strconv.Itoa(resp.StatusCode))
+	if len(args) > 1 && o.IntegrationName == "" {
+		return errors.New("integration name is mandatory when loading multiple integrations")
+	}
+
+	for _, fileName := range args {
+		if !strings.HasPrefix(fileName, "http://") && !strings.HasPrefix(fileName, "https://") {
+			if _, err := os.Stat(fileName); err != nil && os.IsNotExist(err) {
+				return errors.Wrap(err, "file "+fileName+" does not exist")
+			} else if err != nil {
+				return errors.Wrap(err, "error while accessing file "+fileName)
+			}
+		} else {
+			resp, err := http.Get(fileName)
+			if err != nil {
+				return errors.Wrap(err, "The URL provided is not reachable")
+			} else if resp.StatusCode != 200 {
+				return errors.New("The URL provided is not reachable " + fileName + " The error code returned is " + strconv.Itoa(resp.StatusCode))
+			}
 		}
 	}
 
@@ -165,7 +170,7 @@ func (o *runCmdOptions) run(cmd *cobra.Command, args []string) error {
 	}
 
 	if o.Sync || o.Dev {
-		err = o.syncIntegration(args[0])
+		err = o.syncIntegration(args)
 		if err != nil {
 			return err
 		}
@@ -215,36 +220,35 @@ func (o *runCmdOptions) waitForIntegrationReady(integration *v1alpha1.Integratio
 	return watch.HandleStateChanges(o.Context, integration, handler)
 }
 
-func (o *runCmdOptions) syncIntegration(file string) error {
-	changes, err := sync.File(o.Context, file)
-	if err != nil {
-		return err
-	}
-	go func() {
-		for {
-			select {
-			case <-o.Context.Done():
-				return
-			case <-changes:
-				_, err := o.updateIntegrationCode(file)
-				if err != nil {
-					logrus.Error("Unable to sync integration: ", err)
+func (o *runCmdOptions) syncIntegration(sources []string) error {
+	for _, s := range sources {
+		changes, err := sync.File(o.Context, s)
+		if err != nil {
+			return err
+		}
+		go func() {
+			for {
+				select {
+				case <-o.Context.Done():
+					return
+				case <-changes:
+					_, err := o.updateIntegrationCode(sources)
+					if err != nil {
+						logrus.Error("Unable to sync integration: ", err)
+					}
 				}
 			}
-		}
-	}()
+		}()
+	}
+
 	return nil
 }
 
 func (o *runCmdOptions) createIntegration(cmd *cobra.Command, args []string) (*v1alpha1.Integration, error) {
-	return o.updateIntegrationCode(args[0])
+	return o.updateIntegrationCode(args)
 }
 
-func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integration, error) {
-	code, err := o.loadCode(filename)
-	if err != nil {
-		return nil, err
-	}
+func (o *runCmdOptions) updateIntegrationCode(sources []string) (*v1alpha1.Integration, error) {
 
 	namespace := o.Namespace
 
@@ -252,17 +256,13 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 	if o.IntegrationName != "" {
 		name = o.IntegrationName
 		name = kubernetes.SanitizeName(name)
-	} else {
-		name = kubernetes.SanitizeName(filename)
+	} else if len(sources) == 1 {
+		name = kubernetes.SanitizeName(sources[0])
 		if name == "" {
 			name = "integration"
 		}
-	}
-
-	codeName := filename
-
-	if idx := strings.LastIndexByte(filename, os.PathSeparator); idx > -1 {
-		codeName = codeName[idx+1:]
+	} else {
+		return nil, errors.New("invalid argument combination")
 	}
 
 	integration := v1alpha1.Integration{
@@ -275,11 +275,6 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 			Name:      name,
 		},
 		Spec: v1alpha1.IntegrationSpec{
-			Source: v1alpha1.SourceSpec{
-				Name:     codeName,
-				Content:  code,
-				Language: v1alpha1.Language(o.Language),
-			},
 			Dependencies:  make([]string, 0, len(o.Dependencies)),
 			Context:       o.IntegrationContext,
 			Configuration: make([]v1alpha1.ConfigurationSpec, 0),
@@ -287,6 +282,15 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 		},
 	}
 
+	for _, source := range sources {
+		code, err := o.loadCode(source)
+		if err != nil {
+			return nil, err
+		}
+
+		integration.Spec.AddSource(path.Base(source), code, "")
+	}
+
 	for _, item := range o.Dependencies {
 		if strings.HasPrefix(item, "mvn:") {
 			integration.Spec.Dependencies = append(integration.Spec.Dependencies, item)
@@ -336,7 +340,7 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 	case "":
 		// continue..
 	case "yaml":
-		jsondata, err := toJson(&integration)
+		jsondata, err := toJSON(&integration)
 		if err != nil {
 			return nil, err
 		}
@@ -348,7 +352,7 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 		return nil, nil
 
 	case "json":
-		data, err := toJson(&integration)
+		data, err := toJSON(&integration)
 		if err != nil {
 			return nil, err
 		}
@@ -360,7 +364,7 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 	}
 
 	existed := false
-	err = sdk.Create(&integration)
+	err := sdk.Create(&integration)
 	if err != nil && k8serrors.IsAlreadyExists(err) {
 		existed = true
 		clone := integration.DeepCopy()
@@ -384,7 +388,7 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 	return &integration, nil
 }
 
-func toJson(value runtime.Object) ([]byte, error) {
+func toJSON(value runtime.Object) ([]byte, error) {
 	u, err := k8sutil.UnstructuredFromRuntimeObject(value)
 	if err != nil {
 		return nil, fmt.Errorf("error creating unstructured data: %v", err)
diff --git a/pkg/stub/action/integration/initialize.go b/pkg/stub/action/integration/initialize.go
index fa02315..b495b11 100644
--- a/pkg/stub/action/integration/initialize.go
+++ b/pkg/stub/action/integration/initialize.go
@@ -59,9 +59,13 @@ func (action *initializeAction) Handle(integration *v1alpha1.Integration) error
 		var defaultReplicas int32 = 1
 		target.Spec.Replicas = &defaultReplicas
 	}
-	// extract metadata
-	meta := metadata.Extract(target.Spec.Source)
-	target.Spec.Source.Language = meta.Language
+	for i := range target.Spec.Sources {
+		// extract metadata
+		s := &target.Spec.Sources[i]
+
+		meta := metadata.Extract(*s)
+		s.Language = meta.Language
+	}
 
 	// execute custom initialization
 	if _, err := trait.Apply(target, nil); err != nil {
diff --git a/pkg/trait/dependencies.go b/pkg/trait/dependencies.go
index f284678..b57d097 100644
--- a/pkg/trait/dependencies.go
+++ b/pkg/trait/dependencies.go
@@ -39,21 +39,23 @@ func (*dependenciesTrait) appliesTo(e *Environment) bool {
 	return e.Integration != nil && e.Integration.Status.Phase == ""
 }
 
-func (d *dependenciesTrait) apply(e *Environment) error {
-	meta := metadata.Extract(e.Integration.Spec.Source)
+func (*dependenciesTrait) apply(e *Environment) error {
+	for _, s := range e.Integration.Spec.Sources {
+		meta := metadata.Extract(s)
 
-	if meta.Language == v1alpha1.LanguageGroovy {
-		util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:groovy")
-	} else if meta.Language == v1alpha1.LanguageKotlin {
-		util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:kotlin")
-	}
+		if meta.Language == v1alpha1.LanguageGroovy {
+			util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:groovy")
+		} else if meta.Language == v1alpha1.LanguageKotlin {
+			util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:kotlin")
+		}
 
-	// jvm runtime and camel-core required by default
-	util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:jvm")
-	util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "camel:core")
+		// jvm runtime and camel-core required by default
+		util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "runtime:jvm")
+		util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, "camel:core")
 
-	for _, d := range meta.Dependencies {
-		util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, d)
+		for _, d := range meta.Dependencies {
+			util.StringSliceUniqueAdd(&e.Integration.Spec.Dependencies, d)
+		}
 	}
 
 	// sort the dependencies to get always the same list if they don't change
diff --git a/pkg/trait/deployment.go b/pkg/trait/deployment.go
index 8023050..cc16ef1 100644
--- a/pkg/trait/deployment.go
+++ b/pkg/trait/deployment.go
@@ -25,6 +25,7 @@ import (
 	appsv1 "k8s.io/api/apps/v1"
 	corev1 "k8s.io/api/core/v1"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
 )
 
 type deploymentTrait struct {
@@ -42,8 +43,8 @@ func (d *deploymentTrait) appliesTo(e *Environment) bool {
 }
 
 func (d *deploymentTrait) apply(e *Environment) error {
-	e.Resources.Add(getConfigMapFor(e))
-	e.Resources.Add(getDeploymentFor(e))
+	e.Resources.AddAll(d.getConfigMapFor(e))
+	e.Resources.Add(d.getDeploymentFor(e))
 	return nil
 }
 
@@ -53,34 +54,60 @@ func (d *deploymentTrait) apply(e *Environment) error {
 //
 // **********************************
 
-func getConfigMapFor(e *Environment) *corev1.ConfigMap {
+func (*deploymentTrait) getConfigMapFor(e *Environment) []runtime.Object {
+	maps := make([]runtime.Object, 0, len(e.Integration.Spec.Sources)+1)
+
 	// combine properties of integration with context, integration
 	// properties have the priority
 	properties := CombineConfigurationAsMap("property", e.Context, e.Integration)
 
-	cm := corev1.ConfigMap{
-		TypeMeta: metav1.TypeMeta{
-			Kind:       "ConfigMap",
-			APIVersion: "v1",
-		},
-		ObjectMeta: metav1.ObjectMeta{
-			Name:      e.Integration.Name,
-			Namespace: e.Integration.Namespace,
-			Labels: map[string]string{
-				"camel.apache.org/integration": e.Integration.Name,
+	maps = append(
+		maps,
+		&corev1.ConfigMap{
+			TypeMeta: metav1.TypeMeta{
+				Kind:       "ConfigMap",
+				APIVersion: "v1",
 			},
-			Annotations: map[string]string{
-				"camel.apache.org/source.language": string(e.Integration.Spec.Source.Language),
-				"camel.apache.org/source.name":     e.Integration.Spec.Source.Name,
+			ObjectMeta: metav1.ObjectMeta{
+				Name:      e.Integration.Name + "-properties",
+				Namespace: e.Integration.Namespace,
+				Labels: map[string]string{
+					"camel.apache.org/integration": e.Integration.Name,
+				},
+			},
+			Data: map[string]string{
+				"properties": PropertiesString(properties),
 			},
 		},
-		Data: map[string]string{
-			"integration": e.Integration.Spec.Source.Content,
-			"properties":  PropertiesString(properties),
-		},
+	)
+
+	for i, s := range e.Integration.Spec.Sources {
+		maps = append(
+			maps,
+			&corev1.ConfigMap{
+				TypeMeta: metav1.TypeMeta{
+					Kind:       "ConfigMap",
+					APIVersion: "v1",
+				},
+				ObjectMeta: metav1.ObjectMeta{
+					Name:      fmt.Sprintf("%s-source-%03d", e.Integration.Name, i),
+					Namespace: e.Integration.Namespace,
+					Labels: map[string]string{
+						"camel.apache.org/integration": e.Integration.Name,
+					},
+					Annotations: map[string]string{
+						"camel.apache.org/source.language": string(s.Language),
+						"camel.apache.org/source.name":     s.Name,
+					},
+				},
+				Data: map[string]string{
+					"integration": s.Content,
+				},
+			},
+		)
 	}
 
-	return &cm
+	return maps
 }
 
 // **********************************
@@ -89,8 +116,16 @@ func getConfigMapFor(e *Environment) *corev1.ConfigMap {
 //
 // **********************************
 
-func getDeploymentFor(e *Environment) *appsv1.Deployment {
-	sourceName := strings.TrimPrefix(e.Integration.Spec.Source.Name, "/")
+func (*deploymentTrait) getDeploymentFor(e *Environment) *appsv1.Deployment {
+	sources := make([]string, 0, len(e.Integration.Spec.Sources))
+	for i, s := range e.Integration.Spec.Sources {
+		src := fmt.Sprintf("file:/etc/camel/integrations/%03d/%s", i, strings.TrimPrefix(s.Name, "/"))
+		if s.Language != "" {
+			src = src + "?language=" + string(s.Language)
+		}
+
+		sources = append(sources, src)
+	}
 
 	// combine Environment of integration with context, integration
 	// Environment has the priority
@@ -100,8 +135,7 @@ func getDeploymentFor(e *Environment) *appsv1.Deployment {
 	environment["JAVA_MAIN_CLASS"] = "org.apache.camel.k.jvm.Application"
 
 	// camel-k runtime
-	environment["CAMEL_K_ROUTES_URI"] = "file:/etc/camel/conf/" + sourceName
-	environment["CAMEL_K_ROUTES_LANGUAGE"] = string(e.Integration.Spec.Source.Language)
+	environment["CAMEL_K_ROUTES"] = strings.Join(sources, ",")
 	environment["CAMEL_K_CONF"] = "/etc/camel/conf/application.properties"
 	environment["CAMEL_K_CONF_D"] = "/etc/camel/conf.d"
 
@@ -159,21 +193,18 @@ func getDeploymentFor(e *Environment) *appsv1.Deployment {
 	cnt := 0
 
 	//
-	// Volumes :: Defaults
+	// Volumes :: Properties
 	//
 
 	vols = append(vols, corev1.Volume{
-		Name: "integration",
+		Name: "integration-properties",
 		VolumeSource: corev1.VolumeSource{
 			ConfigMap: &corev1.ConfigMapVolumeSource{
 				LocalObjectReference: corev1.LocalObjectReference{
-					Name: e.Integration.Name,
+					Name: e.Integration.Name + "-properties",
 				},
 				Items: []corev1.KeyToPath{
 					{
-						Key:  "integration",
-						Path: sourceName,
-					}, {
 						Key:  "properties",
 						Path: "application.properties",
 					},
@@ -183,11 +214,39 @@ func getDeploymentFor(e *Environment) *appsv1.Deployment {
 	})
 
 	mnts = append(mnts, corev1.VolumeMount{
-		Name:      "integration",
+		Name:      "integration-properties",
 		MountPath: "/etc/camel/conf",
 	})
 
 	//
+	// Volumes :: Sources
+	//
+
+	for i, s := range e.Integration.Spec.Sources {
+		vols = append(vols, corev1.Volume{
+			Name: fmt.Sprintf("integration-source-%03d", i),
+			VolumeSource: corev1.VolumeSource{
+				ConfigMap: &corev1.ConfigMapVolumeSource{
+					LocalObjectReference: corev1.LocalObjectReference{
+						Name: fmt.Sprintf("%s-source-%03d", e.Integration.Name, i),
+					},
+					Items: []corev1.KeyToPath{
+						{
+							Key:  "integration",
+							Path: strings.TrimPrefix(s.Name, "/"),
+						},
+					},
+				},
+			},
+		})
+
+		mnts = append(mnts, corev1.VolumeMount{
+			Name:      fmt.Sprintf("integration-source-%03d", i),
+			MountPath: fmt.Sprintf("/etc/camel/integrations/%03d", i),
+		})
+	}
+
+	//
 	// Volumes :: Additional ConfigMaps
 	//
 
diff --git a/pkg/trait/knative.go b/pkg/trait/knative.go
index b122f8d..0b93d87 100644
--- a/pkg/trait/knative.go
+++ b/pkg/trait/knative.go
@@ -19,6 +19,7 @@ package trait
 
 import (
 	"encoding/json"
+	"fmt"
 	"strings"
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
@@ -72,14 +73,26 @@ func (t *knativeTrait) getServiceFor(e *Environment) *serving.Service {
 	// Environment has the priority
 	environment := CombineConfigurationAsMap("env", e.Context, e.Integration)
 
+	sources := make([]string, 0, len(e.Integration.Spec.Sources))
+	for i, s := range e.Integration.Spec.Sources {
+		envName := fmt.Sprintf("KAMEL_K_ROUTE_%03d", i)
+		environment[envName] = s.Content
+
+		src := fmt.Sprintf("env:%s", envName)
+		if s.Language != "" {
+			src = src + "?language=" + string(s.Language)
+		}
+
+		sources = append(sources, src)
+	}
+
 	// set env vars needed by the runtime
 	environment["JAVA_MAIN_CLASS"] = "org.apache.camel.k.jvm.Application"
 
 	// camel-k runtime
-	environment["CAMEL_K_ROUTES_URI"] = "inline:" + e.Integration.Spec.Source.Content
-	environment["CAMEL_K_ROUTES_LANGUAGE"] = string(e.Integration.Spec.Source.Language)
-	environment["CAMEL_K_CONF"] = "inline:" + PropertiesString(properties)
-	environment["CAMEL_K_CONF_D"] = "/etc/camel/conf.d"
+	environment["CAMEL_K_ROUTES"] = strings.Join(sources, ",")
+	environment["CAMEL_K_CONF"] = "env:CAMEL_K_PROPERTIES"
+	environment["CAMEL_K_PROPERTIES"] = PropertiesString(properties)
 
 	// add a dummy env var to trigger deployment if everything but the code
 	// has been changed
@@ -214,6 +227,12 @@ func (t *knativeTrait) getConfiguredSourceChannels() []string {
 }
 
 func (*knativeTrait) getSourceChannels(e *Environment) []string {
-	meta := metadata.Extract(e.Integration.Spec.Source)
-	return knativeutil.ExtractChannelNames(meta.FromURIs)
+	channels := make([]string, 0)
+
+	for _, s := range e.Integration.Spec.Sources {
+		meta := metadata.Extract(s)
+		channels = append(channels, knativeutil.ExtractChannelNames(meta.FromURIs)...)
+	}
+
+	return channels
 }
diff --git a/pkg/trait/trait_test.go b/pkg/trait/trait_test.go
index 17d8909..cb3acd5 100644
--- a/pkg/trait/trait_test.go
+++ b/pkg/trait/trait_test.go
@@ -38,7 +38,7 @@ func TestOpenShiftTraits(t *testing.T) {
 	assert.NotContains(t, env.ExecutedTraits, ID("route"))
 	assert.Contains(t, env.ExecutedTraits, ID("owner"))
 	assert.NotNil(t, res.GetConfigMap(func(cm *corev1.ConfigMap) bool {
-		return cm.Name == "test"
+		return cm.Name == "test-properties"
 	}))
 	assert.NotNil(t, res.GetDeployment(func(deployment *appsv1.Deployment) bool {
 		return deployment.Name == "test"
@@ -53,7 +53,7 @@ func TestOpenShiftTraitsWithWeb(t *testing.T) {
 	assert.Contains(t, env.ExecutedTraits, ID("route"))
 	assert.Contains(t, env.ExecutedTraits, ID("owner"))
 	assert.NotNil(t, res.GetConfigMap(func(cm *corev1.ConfigMap) bool {
-		return cm.Name == "test"
+		return cm.Name == "test-properties"
 	}))
 	assert.NotNil(t, res.GetDeployment(func(deployment *appsv1.Deployment) bool {
 		return deployment.Name == "test"
@@ -107,7 +107,7 @@ func TestKubernetesTraits(t *testing.T) {
 	assert.NotContains(t, env.ExecutedTraits, ID("route"))
 	assert.Contains(t, env.ExecutedTraits, ID("owner"))
 	assert.NotNil(t, res.GetConfigMap(func(cm *corev1.ConfigMap) bool {
-		return cm.Name == "test"
+		return cm.Name == "test-properties"
 	}))
 	assert.NotNil(t, res.GetDeployment(func(deployment *appsv1.Deployment) bool {
 		return deployment.Name == "test"
@@ -122,7 +122,7 @@ func TestKubernetesTraitsWithWeb(t *testing.T) {
 	assert.NotContains(t, env.ExecutedTraits, ID("route"))
 	assert.Contains(t, env.ExecutedTraits, ID("owner"))
 	assert.NotNil(t, res.GetConfigMap(func(cm *corev1.ConfigMap) bool {
-		return cm.Name == "test"
+		return cm.Name == "test-properties"
 	}))
 	assert.NotNil(t, res.GetDeployment(func(deployment *appsv1.Deployment) bool {
 		return deployment.Name == "test"
diff --git a/pkg/util/digest/digest.go b/pkg/util/digest/digest.go
index 265c664..2be054f 100644
--- a/pkg/util/digest/digest.go
+++ b/pkg/util/digest/digest.go
@@ -37,9 +37,12 @@ func ComputeForIntegration(integration *v1alpha1.Integration) string {
 	hash.Write([]byte(integration.Spec.Context))
 
 	// Integration code
-	if integration.Spec.Source.Content != "" {
-		hash.Write([]byte(integration.Spec.Source.Content))
+	for _, s := range integration.Spec.Sources {
+		if s.Content != "" {
+			hash.Write([]byte(s.Content))
+		}
 	}
+
 	// Integration dependencies
 	for _, item := range integration.Spec.Dependencies {
 		hash.Write([]byte(item))
diff --git a/pkg/util/kubernetes/collection.go b/pkg/util/kubernetes/collection.go
index 54b6ba7..fbc9098 100644
--- a/pkg/util/kubernetes/collection.go
+++ b/pkg/util/kubernetes/collection.go
@@ -47,6 +47,11 @@ func (c *Collection) Add(resource runtime.Object) {
 	c.items = append(c.items, resource)
 }
 
+// AddAll adds all resources to the collection
+func (c *Collection) AddAll(resource []runtime.Object) {
+	c.items = append(c.items, resource...)
+}
+
 // VisitDeployment executes the visitor function on all Deployment resources
 func (c *Collection) VisitDeployment(visitor func(*appsv1.Deployment)) {
 	c.Visit(func(res runtime.Object) {
diff --git a/runtime/examples/routes.js b/runtime/examples/routes.js
index a020511..edb7806 100644
--- a/runtime/examples/routes.js
+++ b/runtime/examples/routes.js
@@ -5,10 +5,10 @@
 //
 // ****************
 
-l = components.get('log')
-l.exchangeFormatter = function(e) {
-    return "log - body=" + e.in.body + ", headers=" + e.in.headers
-}
+//l = components.get('log')
+//l.exchangeFormatter = function(e) {
+//    return "log - body=" + e.in.body + ", headers=" + e.in.headers
+//}
 
 // ****************
 //
diff --git a/runtime/examples/simple.groovy b/runtime/examples/simple.groovy
new file mode 100644
index 0000000..7546207
--- /dev/null
+++ b/runtime/examples/simple.groovy
@@ -0,0 +1,6 @@
+
+from('timer:groovy?period=1s')
+    .routeId('groovy')
+    .setBody()
+        .simple('Hello Camel K from ${routeId}')
+    .to('log:info?showAll=false')
diff --git a/runtime/examples/simple.js b/runtime/examples/simple.js
new file mode 100644
index 0000000..918c544
--- /dev/null
+++ b/runtime/examples/simple.js
@@ -0,0 +1,6 @@
+
+from('timer:js?period=1s')
+    .routeId('js')
+    .setBody()
+        .simple('Hello Camel K from ${routeId}')
+    .to('log:info?multiline=true')
\ 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 e24e0cf..16b4f0f 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
@@ -29,8 +29,8 @@ class LoaderTest extends Specification {
             def resource = "classpath:routes.groovy"
 
         when:
-            def loader = RoutesLoaders.loaderFor(resource, null);
-            def builder = loader.load(new RuntimeRegistry(), resource);
+            def loader = RoutesLoaders.loaderFor(resource, null)
+            def builder = loader.load(new RuntimeRegistry(), resource)
 
         then:
             loader instanceof GroovyRoutesLoader
diff --git a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/IntegrationTest.groovy b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/IntegrationTest.groovy
index 1b6a1e6..32904f1 100644
--- a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/IntegrationTest.groovy
+++ b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/IntegrationTest.groovy
@@ -32,7 +32,7 @@ class IntegrationTest extends Specification {
         when:
         def runtime = new Runtime()
         runtime.setDuration(5)
-        runtime.load('classpath:routes-with-rest.groovy', null)
+        runtime.load(['classpath:routes-with-rest.groovy'])
         runtime.addMainListener(new MainListenerSupport() {
             @Override
             void afterStart(MainSupport main) {
@@ -55,7 +55,7 @@ class IntegrationTest extends Specification {
         when:
         def runtime = new Runtime()
         runtime.setDuration(5)
-        runtime.load('classpath:routes-with-bindings.groovy', null)
+        runtime.load(['classpath:routes-with-bindings.groovy'])
         runtime.addMainListener(new MainListenerSupport() {
             @Override
             void afterStart(MainSupport main) {
@@ -82,7 +82,7 @@ class IntegrationTest extends Specification {
         when:
         def runtime = new Runtime()
         runtime.setDuration(5)
-        runtime.load('classpath:routes-with-component-configuration.groovy', null)
+        runtime.load(['classpath:routes-with-component-configuration.groovy'])
         runtime.addMainListener(new MainListenerSupport() {
             @Override
             void afterStart(MainSupport main) {
diff --git a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/extension/LogExtensionTest.groovy b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/extension/LogExtensionTest.groovy
index 68bad92..fee6475 100644
--- a/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/extension/LogExtensionTest.groovy
+++ b/runtime/groovy/src/test/groovy/org/apache/camel/k/groovy/dsl/extension/LogExtensionTest.groovy
@@ -30,7 +30,7 @@ class LogExtensionTest extends Specification {
         when:
         def log = new LogComponent()
         log.formatter {
-            "body: " + it.in.body
+            "body: $it.in.body"
         }
 
         def ex = new DefaultExchange(ctx)
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Application.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Application.java
index 0c22f03..d26ddef 100644
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Application.java
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Application.java
@@ -52,15 +52,14 @@ public class Application {
     // *******************************
 
     public static void main(String[] args) throws Exception {
-        final String resource = System.getenv(Constants.ENV_CAMEL_K_ROUTES_URI);
-        final String language = System.getenv(Constants.ENV_CAMEL_K_ROUTES_LANGUAGE);
+        final String routes = System.getenv(Constants.ENV_CAMEL_K_ROUTES);
 
-        if (ObjectHelper.isEmpty(resource)) {
-            throw new IllegalStateException("No valid resource found in " + Constants.ENV_CAMEL_K_ROUTES_URI + " environment variable");
+        if (ObjectHelper.isEmpty(routes)) {
+            throw new IllegalStateException("No valid routes found in " + Constants.ENV_CAMEL_K_ROUTES + " environment variable");
         }
 
         Runtime runtime = new Runtime();
-        runtime.load(resource, language);
+        runtime.load(routes.split(",", -1));
         runtime.addMainListener(new ComponentPropertiesBinder());
         runtime.run();
     }
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Constants.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Constants.java
index e5a09ed..d3cb4b7 100644
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Constants.java
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/Constants.java
@@ -17,13 +17,12 @@
 package org.apache.camel.k.jvm;
 
 public final class Constants {
-    public static final String ENV_CAMEL_K_ROUTES_URI = "CAMEL_K_ROUTES_URI";
-    public static final String ENV_CAMEL_K_ROUTES_LANGUAGE = "CAMEL_K_ROUTES_LANGUAGE";
+    public static final String ENV_CAMEL_K_ROUTES = "CAMEL_K_ROUTES";
     public static final String ENV_CAMEL_K_CONF = "CAMEL_K_CONF";
     public static final String ENV_CAMEL_K_CONF_D = "CAMEL_K_CONF_D";
     public static final String SCHEME_CLASSPATH = "classpath:";
     public static final String SCHEME_FILE = "file:";
-    public static final String SCHEME_INLINE = "inline:";
+    public static final String SCHEME_ENV = "env:";
     public static final String LOGGING_LEVEL_PREFIX = "logging.level.";
 
     private Constants() {
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
index 35cb47d..0d81738 100644
--- 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
@@ -16,6 +16,20 @@
  */
 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.jvm.dsl.Components;
@@ -29,24 +43,6 @@ import org.joor.Reflect;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.script.Bindings;
-import javax.script.ScriptEngine;
-import javax.script.ScriptEngineManager;
-import javax.script.SimpleBindings;
-import javax.xml.bind.UnmarshalException;
-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 static org.apache.camel.k.jvm.Constants.SCHEME_CLASSPATH;
-import static org.apache.camel.k.jvm.Constants.SCHEME_FILE;
-import static org.apache.camel.k.jvm.Constants.SCHEME_INLINE;
-
 public final class RoutesLoaders {
     private static final Logger LOGGER = LoggerFactory.getLogger(RoutesLoaders.class);
 
@@ -62,7 +58,7 @@ public final class RoutesLoaders {
         @Override
         public RouteBuilder load(RuntimeRegistry registry, String resource) throws Exception {
             String path = resource;
-            path = StringUtils.removeStart(path, SCHEME_CLASSPATH);
+            path = StringUtils.removeStart(path, Constants.SCHEME_CLASSPATH);
             path = StringUtils.removeEnd(path, ".class");
 
             Class<?> type = Class.forName(path);
@@ -173,7 +169,9 @@ public final class RoutesLoaders {
 
 
     public static RoutesLoader loaderFor(String resource, String languageName) {
-        if (!resource.startsWith(SCHEME_CLASSPATH) && !resource.startsWith(SCHEME_FILE) && !resource.startsWith(SCHEME_INLINE)) {
+        if (!resource.startsWith(Constants.SCHEME_CLASSPATH) &&
+            !resource.startsWith(Constants.SCHEME_FILE) &&
+            !resource.startsWith(Constants.SCHEME_ENV)) {
             throw new IllegalArgumentException("No valid resource format, expected scheme:path, found " + resource);
         }
 
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 72753f7..d4a755c 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
@@ -27,6 +27,8 @@ import org.apache.camel.component.properties.PropertiesComponent;
 import org.apache.camel.impl.CompositeRegistry;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.main.MainSupport;
+import org.apache.camel.util.URISupport;
+import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,18 +42,25 @@ public final class Runtime extends MainSupport {
         this.contextMap = new ConcurrentHashMap<>();
     }
 
-    public void load(String resource, String language) throws Exception {
-        final RoutesLoader loader = RoutesLoaders.loaderFor(resource, language);
-        final RouteBuilder routes = loader.load(registry, resource);
+    public void load(String[] routes) throws Exception {
+        for (String route: routes) {
+            // determine location and language
+            final String location = StringUtils.substringBefore(route, "?");
+            final String query = StringUtils.substringAfter(route, "?");
+            final String language = (String)URISupport.parseQuery(query).get("language");
 
-        if (routes == null) {
-            throw new IllegalStateException("Unable to load route from: " + resource);
-        }
+            // load routes
+            final RoutesLoader loader = RoutesLoaders.loaderFor(location, language);
+            final RouteBuilder builder = loader.load(registry, location);
+
+            if (routes == null) {
+                throw new IllegalStateException("Unable to load route from: " + route);
+            }
 
-        LOGGER.info("Routes: {}", resource);
-        LOGGER.info("Language: {}", language);
+            LOGGER.info("Routes: {}", route);
 
-        addRouteBuilder(routes);
+            addRouteBuilder(builder);
+        }
     }
 
     public RuntimeRegistry getRegistry() {
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 44dc3a1..8b4db47 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
@@ -36,7 +36,6 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.LoggerConfig;
 
-import static org.apache.camel.k.jvm.Constants.SCHEME_INLINE;
 
 public final class RuntimeSupport {
     private RuntimeSupport() {
@@ -49,8 +48,8 @@ public final class RuntimeSupport {
 
         // Main location
         if (ObjectHelper.isNotEmpty(conf)) {
-            if (conf.startsWith(SCHEME_INLINE)) {
-                try (Reader reader = URIResolver.resolveInline(conf)) {
+            if (conf.startsWith(Constants.SCHEME_ENV)) {
+                try (Reader reader = URIResolver.resolveEnv(conf)) {
                     properties.load(reader);
                 } catch (IOException e) {
                     throw new RuntimeException(e);
diff --git a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/URIResolver.java b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/URIResolver.java
index b52e81c..12ade89 100644
--- a/runtime/jvm/src/main/java/org/apache/camel/k/jvm/URIResolver.java
+++ b/runtime/jvm/src/main/java/org/apache/camel/k/jvm/URIResolver.java
@@ -16,36 +16,43 @@
  */
 package org.apache.camel.k.jvm;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.util.ResourceHelper;
-
 import java.io.ByteArrayInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
 import java.io.StringReader;
 
-import static org.apache.camel.k.jvm.Constants.SCHEME_INLINE;
+import org.apache.camel.CamelContext;
+import org.apache.camel.util.ResourceHelper;
+import org.apache.camel.util.StringHelper;
+
 
 public class URIResolver {
 
-    public static InputStream resolve(CamelContext ctx, String uri) throws IOException {
+    public static InputStream resolve(CamelContext ctx, String uri) throws Exception {
         if (uri == null) {
             throw new IllegalArgumentException("Cannot resolve null URI");
         }
-        if (uri.startsWith(SCHEME_INLINE)) {
+
+        if (uri.startsWith(Constants.SCHEME_ENV)) {
+            final String envvar = StringHelper.after(uri, ":");
+            final String content = System.getenv(envvar);
+
             // Using platform encoding on purpose
-            return new ByteArrayInputStream(uri.substring(SCHEME_INLINE.length()).getBytes());
+            return new ByteArrayInputStream(content.getBytes());
         }
 
         return ResourceHelper.resolveMandatoryResourceAsInputStream(ctx, uri);
     }
 
-    public static Reader resolveInline(String uri) {
-        if (!uri.startsWith(SCHEME_INLINE)) {
+    public static Reader resolveEnv(String uri) {
+        if (!uri.startsWith(Constants.SCHEME_ENV)) {
             throw new IllegalArgumentException("The provided content is not inline: " + uri);
         }
-        return new StringReader(uri.substring(SCHEME_INLINE.length()));
+
+        final String envvar = StringHelper.after(uri, ":");
+        final String content = System.getenv(envvar);
+
+        return new StringReader(content);
     }
 
 }
diff --git a/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RuntimeTest.java b/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RuntimeTest.java
new file mode 100644
index 0000000..909ae23
--- /dev/null
+++ b/runtime/jvm/src/test/java/org/apache/camel/k/jvm/RuntimeTest.java
@@ -0,0 +1,51 @@
+/**
+ * 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.util.List;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Route;
+import org.apache.camel.util.ObjectHelper;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+
+public class RuntimeTest {
+
+    @Test
+    void testLoadMultipleRoutes() throws Exception {
+        Runtime runtime = new Runtime();
+        runtime.load(new String[]{
+            "classpath:r1.js",
+            "classpath:r2.mytype?language=js",
+        });
+
+        try {
+            runtime.start();
+
+            CamelContext context = runtime.getCamelContext();
+            List<Route> routes = context.getRoutes();
+
+            assertThat(routes).hasSize(2);
+            assertThat(routes).anyMatch(p -> ObjectHelper.equal("r1", p.getId()));
+            assertThat(routes).anyMatch(p -> ObjectHelper.equal("r2", p.getId()));
+        } finally {
+            runtime.stop();
+        }
+    }
+}
diff --git a/runtime/jvm/src/test/resources/r1.js b/runtime/jvm/src/test/resources/r1.js
new file mode 100644
index 0000000..f6ca425
--- /dev/null
+++ b/runtime/jvm/src/test/resources/r1.js
@@ -0,0 +1,4 @@
+
+from('timer:tick1')
+    .id('r1')
+    .to('log:info1')
\ No newline at end of file
diff --git a/runtime/jvm/src/test/resources/r2.mytype b/runtime/jvm/src/test/resources/r2.mytype
new file mode 100644
index 0000000..a0b33cc
--- /dev/null
+++ b/runtime/jvm/src/test/resources/r2.mytype
@@ -0,0 +1,4 @@
+
+from('timer:tick2')
+    .id('r2')
+    .to('log:info2')
\ No newline at end of file
diff --git a/runtime/kotlin/src/main/kotlin/org/apache/camel/k/kotlin/KotlinRoutesLoader.kt b/runtime/kotlin/src/main/kotlin/org/apache/camel/k/kotlin/KotlinRoutesLoader.kt
index 8602709..afbeb4f 100644
--- a/runtime/kotlin/src/main/kotlin/org/apache/camel/k/kotlin/KotlinRoutesLoader.kt
+++ b/runtime/kotlin/src/main/kotlin/org/apache/camel/k/kotlin/KotlinRoutesLoader.kt
@@ -85,12 +85,10 @@ class KotlinRoutesLoader : RoutesLoader {
                     )
 
                     for (report in result.reports) {
-                        if (report.severity == ScriptDiagnostic.Severity.ERROR) {
-                            LOGGER.error("{}", report.message, report.exception)
-                        } else if (report.severity == ScriptDiagnostic.Severity.WARNING) {
-                            LOGGER.warn("{}", report.message, report.exception)
-                        } else {
-                            LOGGER.info("{}", report.message)
+                        when {
+                            report.severity == ScriptDiagnostic.Severity.ERROR -> LOGGER.error("{}", report.message, report.exception)
+                            report.severity == ScriptDiagnostic.Severity.WARNING -> LOGGER.warn("{}", report.message, report.exception)
+                            else -> LOGGER.info("{}", report.message)
                         }
                     }
                 }
diff --git a/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/dsl/IntegrationTest.kt b/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/dsl/IntegrationTest.kt
index 734f278..4b6e85f 100644
--- a/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/dsl/IntegrationTest.kt
+++ b/runtime/kotlin/src/test/kotlin/org/apache/camel/k/kotlin/dsl/IntegrationTest.kt
@@ -16,7 +16,7 @@ class IntegrationTest {
     fun `load integration with rest`() {
         var runtime = org.apache.camel.k.jvm.Runtime()
         runtime.duration = 5
-        runtime.load("classpath:routes-with-rest.kts", null)
+        runtime.load(arrayOf("classpath:routes-with-rest.kts"))
         runtime.addMainListener(object: MainListenerSupport() {
             override fun afterStart(main: MainSupport) {
                 main.stop()
@@ -37,7 +37,7 @@ class IntegrationTest {
     fun `load integration with binding`() {
         var runtime = org.apache.camel.k.jvm.Runtime()
         runtime.duration = 5
-        runtime.load("classpath:routes-with-bindings.kts", null)
+        runtime.load(arrayOf("classpath:routes-with-bindings.kts"))
         runtime.addMainListener(object: MainListenerSupport() {
             override fun afterStart(main: MainSupport) {
                 main.stop()
@@ -60,7 +60,7 @@ class IntegrationTest {
 
         var runtime = org.apache.camel.k.jvm.Runtime()
         runtime.duration = 5
-        runtime.load("classpath:routes-with-component-configuration.kts", null)
+        runtime.load(arrayOf("classpath:routes-with-component-configuration.kts"))
         runtime.addMainListener(object : MainListenerSupport() {
             override fun afterStart(main: MainSupport) {
                 val seda = runtime.camelContext.getComponent("seda", SedaComponent::class.java)