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 2018/09/19 08:45:35 UTC

[camel-k] 01/02: Implementation of auto-discovery

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.git

commit 25b9aa851cd8d4db670122239fe3218d33a1543e
Author: nferraro <ni...@gmail.com>
AuthorDate: Wed Sep 19 09:15:53 2018 +0200

    Implementation of auto-discovery
---
 pkg/apis/camel/v1alpha1/types.go                   | 33 ++++++--
 pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go   |  5 ++
 pkg/client/cmd/completion_bash.go                  |  2 +-
 pkg/client/cmd/run.go                              | 39 +++++----
 pkg/discover/dependencies.go                       | 96 ++++++++++++++++++++++
 pkg/discover/dependencies_test.go                  | 85 +++++++++++++++++++
 pkg/{util/camel/catalog.go => discover/doc.go}     | 35 +-------
 .../camel/catalog.go => discover/language.go}      | 46 +++++------
 .../catalog.go => discover/languages_test.go}      | 40 ++++-----
 pkg/stub/action/integration/deploy.go              |  4 +-
 pkg/stub/action/integration/initialize.go          | 39 ++++++++-
 pkg/util/camel/catalog.go                          | 23 +++++-
 pkg/util/camel/{catalog.go => catalog_test.go}     | 33 ++------
 runtime/examples/dns.js                            |  6 +-
 14 files changed, 342 insertions(+), 144 deletions(-)

diff --git a/pkg/apis/camel/v1alpha1/types.go b/pkg/apis/camel/v1alpha1/types.go
index 90effe8..3a91be6 100644
--- a/pkg/apis/camel/v1alpha1/types.go
+++ b/pkg/apis/camel/v1alpha1/types.go
@@ -48,20 +48,37 @@ type Integration struct {
 
 // IntegrationSpec --
 type IntegrationSpec struct {
-	Replicas      *int32              `json:"replicas,omitempty"`
-	Source        SourceSpec          `json:"source,omitempty"`
-	Context       string              `json:"context,omitempty"`
-	Dependencies  []string            `json:"dependencies,omitempty"`
-	Configuration []ConfigurationSpec `json:"configuration,omitempty"`
+	Replicas                  *int32              `json:"replicas,omitempty"`
+	Source                    SourceSpec          `json:"source,omitempty"`
+	Context                   string              `json:"context,omitempty"`
+	Dependencies              []string            `json:"dependencies,omitempty"`
+	DependenciesAutoDiscovery *bool               `json:"dependenciesAutoDiscovery,omitempty"`
+	Configuration             []ConfigurationSpec `json:"configuration,omitempty"`
 }
 
 // SourceSpec --
 type SourceSpec struct {
-	Name     string `json:"name,omitempty"`
-	Content  string `json:"content,omitempty"`
-	Language string `json:"language,omitempty"`
+	Name     string   `json:"name,omitempty"`
+	Content  string   `json:"content,omitempty"`
+	Language Language `json:"language,omitempty"`
 }
 
+// Language --
+type Language string
+
+const (
+	// LanguageJavaSource --
+	LanguageJavaSource Language = "java"
+	// LanguageJavaClass --
+	LanguageJavaClass Language = "class"
+	// LanguageGroovy --
+	LanguageGroovy Language = "groovy"
+	// LanguageJavaScript --
+	LanguageJavaScript Language = "js"
+	// LanguageXML --
+	LanguageXML Language = "xml"
+)
+
 // IntegrationStatus --
 type IntegrationStatus struct {
 	Phase  IntegrationPhase `json:"phase,omitempty"`
diff --git a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
index ec6db14..9feb415 100644
--- a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
+++ b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
@@ -219,6 +219,11 @@ func (in *IntegrationSpec) DeepCopyInto(out *IntegrationSpec) {
 		*out = make([]string, len(*in))
 		copy(*out, *in)
 	}
+	if in.DependenciesAutoDiscovery != nil {
+		in, out := &in.DependenciesAutoDiscovery, &out.DependenciesAutoDiscovery
+		*out = new(bool)
+		**out = **in
+	}
 	if in.Configuration != nil {
 		in, out := &in.Configuration, &out.Configuration
 		*out = make([]ConfigurationSpec, len(*in))
diff --git a/pkg/client/cmd/completion_bash.go b/pkg/client/cmd/completion_bash.go
index 5d0d0d6..07274e9 100644
--- a/pkg/client/cmd/completion_bash.go
+++ b/pkg/client/cmd/completion_bash.go
@@ -129,7 +129,7 @@ func configureKnownBashCompletions(command *cobra.Command) {
 func computeCamelDependencies() string {
 	result := ""
 
-	for k := range catalog.Runtime.Artifact {
+	for k := range camel.Runtime.Artifact {
 		if result != "" {
 			result = result + " " + k
 		} else {
diff --git a/pkg/client/cmd/run.go b/pkg/client/cmd/run.go
index ac90449..47e23fe 100644
--- a/pkg/client/cmd/run.go
+++ b/pkg/client/cmd/run.go
@@ -64,6 +64,7 @@ func NewCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 	cmd.Flags().BoolVar(&options.Logs, "logs", false, "Print integration logs")
 	cmd.Flags().BoolVar(&options.Sync, "sync", false, "Synchronize the local source file with the cluster, republishing at each change")
 	cmd.Flags().BoolVar(&options.Dev, "dev", false, "Enable Dev mode (equivalent to \"-w --logs --sync\")")
+	cmd.Flags().BoolVar(&options.DependenciesAutoDiscovery, "auto-discovery", true, "Automatically discover Camel modules by analyzing user code")
 
 	// completion support
 	configureKnownCompletions(&cmd)
@@ -73,17 +74,18 @@ func NewCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 
 type runCmdOptions struct {
 	*RootCmdOptions
-	IntegrationContext string
-	Language           string
-	IntegrationName    string
-	Dependencies       []string
-	Properties         []string
-	ConfigMaps         []string
-	Secrets            []string
-	Wait               bool
-	Logs               bool
-	Sync               bool
-	Dev                bool
+	IntegrationContext        string
+	Language                  string
+	IntegrationName           string
+	Dependencies              []string
+	Properties                []string
+	ConfigMaps                []string
+	Secrets                   []string
+	Wait                      bool
+	Logs                      bool
+	Sync                      bool
+	Dev                       bool
+	DependenciesAutoDiscovery bool
 }
 
 func (*runCmdOptions) validateArgs(cmd *cobra.Command, args []string) error {
@@ -125,7 +127,7 @@ func (o *runCmdOptions) run(cmd *cobra.Command, args []string) error {
 
 	if o.Sync && !o.Logs && !o.Dev {
 		// Let's add a wait point, otherwise the script terminates
-		<- o.Context.Done()
+		<-o.Context.Done()
 	}
 	return nil
 }
@@ -191,9 +193,9 @@ func (o *runCmdOptions) syncIntegration(file string) error {
 	go func() {
 		for {
 			select {
-			case <- o.Context.Done():
+			case <-o.Context.Done():
 				return
-			case <- changes:
+			case <-changes:
 				_, err := o.updateIntegrationCode(file)
 				if err != nil {
 					logrus.Error("Unable to sync integration: ", err)
@@ -246,11 +248,12 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 			Source: v1alpha1.SourceSpec{
 				Name:     codeName,
 				Content:  code,
-				Language: o.Language,
+				Language: v1alpha1.Language(o.Language),
 			},
-			Dependencies:  make([]string, 0, len(o.Dependencies)),
-			Context:       o.IntegrationContext,
-			Configuration: make([]v1alpha1.ConfigurationSpec, 0),
+			Dependencies:              make([]string, 0, len(o.Dependencies)),
+			DependenciesAutoDiscovery: &o.DependenciesAutoDiscovery,
+			Context:                   o.IntegrationContext,
+			Configuration:             make([]v1alpha1.ConfigurationSpec, 0),
 		},
 	}
 
diff --git a/pkg/discover/dependencies.go b/pkg/discover/dependencies.go
new file mode 100644
index 0000000..82ef757
--- /dev/null
+++ b/pkg/discover/dependencies.go
@@ -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 discover
+
+import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/util/camel"
+	"regexp"
+	"sort"
+	"strings"
+)
+
+var (
+	singleQuotedURI *regexp.Regexp
+	doubleQuotedURI *regexp.Regexp
+)
+
+func init() {
+	singleQuotedURI = regexp.MustCompile("'([a-z-]+):[^']+'")
+	doubleQuotedURI = regexp.MustCompile("\"([a-z-]+):[^\"]+\"")
+}
+
+// Dependencies returns a list of dependencies required by the given source code
+func Dependencies(source v1alpha1.SourceSpec) []string {
+	candidateMap := make(map[string]bool)
+	regexps := getRegexpsForLanguage(source.Language)
+	subMatches := findAllStringSubmatch(source.Content, regexps...)
+	for _, uriPrefix := range subMatches {
+		candidateComp := decodeComponent(uriPrefix)
+		if candidateComp != "" {
+			candidateMap[candidateComp] = true
+		}
+	}
+	// Remove duplicates and sort
+	candidateComponents := make([]string, 0, len(candidateMap))
+	for cmp := range candidateMap {
+		candidateComponents = append(candidateComponents, cmp)
+	}
+	sort.Strings(candidateComponents)
+	return candidateComponents
+}
+
+func getRegexpsForLanguage(language v1alpha1.Language) []*regexp.Regexp {
+	switch language {
+	case v1alpha1.LanguageJavaSource:
+		return []*regexp.Regexp{doubleQuotedURI}
+	case v1alpha1.LanguageXML:
+		return []*regexp.Regexp{doubleQuotedURI}
+	case v1alpha1.LanguageGroovy:
+		return []*regexp.Regexp{singleQuotedURI, doubleQuotedURI}
+	case v1alpha1.LanguageJavaScript:
+		return []*regexp.Regexp{singleQuotedURI, doubleQuotedURI}
+	}
+	return []*regexp.Regexp{}
+}
+
+func findAllStringSubmatch(data string, regexps ...*regexp.Regexp) []string {
+	candidates := make([]string, 0)
+	for _, reg := range regexps {
+		hits := reg.FindAllStringSubmatch(data, -1)
+		for _, hit := range hits {
+			if hit != nil && len(hit) > 1 {
+				for _, match := range hit[1:] {
+					candidates = append(candidates, match)
+				}
+			}
+		}
+	}
+	return candidates
+}
+
+func decodeComponent(uriStart string) string {
+	if component := camel.Runtime.GetArtifactByScheme(uriStart); component != nil {
+		artifactID := component.ArtifactID
+		if strings.HasPrefix(artifactID, "camel-") {
+			return "camel:" + artifactID[6:]
+		}
+		return "mvn:" + component.GroupID + ":" + artifactID + ":" + component.Version
+	}
+	return ""
+}
diff --git a/pkg/discover/dependencies_test.go b/pkg/discover/dependencies_test.go
new file mode 100644
index 0000000..ca5fff2
--- /dev/null
+++ b/pkg/discover/dependencies_test.go
@@ -0,0 +1,85 @@
+/*
+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 discover
+
+import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/stretchr/testify/assert"
+	"testing"
+)
+
+func TestDependenciesJavaSource(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name: "Source.java",
+		Language: v1alpha1.LanguageJavaSource,
+		Content: `
+			from("telegram:bots/cippa").to("log:stash");
+			from("timer:tick").to("amqp:queue");
+			from("ine:xistent").to("amqp:queue");
+		`,
+	}
+	dependencies := Dependencies(code)
+	// assert all dependencies are found and sorted (removing duplicates)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+}
+
+func TestDependenciesJavaClass(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name: "Source.class",
+		Language: v1alpha1.LanguageJavaClass,
+		Content: `
+			from("telegram:bots/cippa").to("log:stash");
+			from("timer:tick").to("amqp:queue");
+			from("ine:xistent").to("amqp:queue");
+		`,
+	}
+	dependencies := Dependencies(code)
+	assert.Empty(t, dependencies)
+}
+
+func TestDependenciesJavaScript(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name: "source.js",
+		Language: v1alpha1.LanguageJavaScript,
+		Content: `
+			from('telegram:bots/cippa').to("log:stash");
+			from('timer:tick').to("amqp:queue");
+			from("ine:xistent").to("amqp:queue");
+			'"'
+		`,
+	}
+	dependencies := Dependencies(code)
+	// assert all dependencies are found and sorted (removing duplicates)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+}
+
+func TestDependenciesGroovy(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name: "source.groovy",
+		Language: v1alpha1.LanguageGroovy,
+		Content: `
+			from('telegram:bots/cippa').to("log:stash");
+			from('timer:tick').to("amqp:queue");
+			from("ine:xistent").to("amqp:queue");
+			'"'
+		`,
+	}
+	dependencies := Dependencies(code)
+	// assert all dependencies are found and sorted (removing duplicates)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+}
\ No newline at end of file
diff --git a/pkg/util/camel/catalog.go b/pkg/discover/doc.go
similarity index 52%
copy from pkg/util/camel/catalog.go
copy to pkg/discover/doc.go
index cf6bc99..51cc065 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/discover/doc.go
@@ -15,35 +15,6 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package catalog
-
-import (
-	"github.com/apache/camel-k/deploy"
-	"gopkg.in/yaml.v2"
-)
-
-// Catalog --
-type Catalog struct {
-	Version  string              `yaml:"version"`
-	Artifact map[string]Artifact `yaml:"artifacts"`
-}
-
-// Artifact --
-type Artifact struct {
-	GroupID     string   `yaml:"groupId"`
-	ArtifactID  string   `yaml:"artifactId"`
-	Version     string   `yaml:"version"`
-	Schemes     []string `yaml:"schemes"`
-	Languages   []string `yaml:"languages"`
-	DataFormats []string `yaml:"dataformats"`
-}
-
-func init() {
-	data := deploy.Resources["camel-catalog.yaml"]
-	if err := yaml.Unmarshal([]byte(data), &Runtime); err != nil {
-		panic(err)
-	}
-}
-
-// Runtime --
-var Runtime Catalog
+// Package discover contains functions for extracting
+// information from user code before compilation
+package discover
diff --git a/pkg/util/camel/catalog.go b/pkg/discover/language.go
similarity index 54%
copy from pkg/util/camel/catalog.go
copy to pkg/discover/language.go
index cf6bc99..e38d583 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/discover/language.go
@@ -15,35 +15,29 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package catalog
+// Package discover contains functions for analyzing user code
+package discover
 
 import (
-	"github.com/apache/camel-k/deploy"
-	"gopkg.in/yaml.v2"
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"strings"
 )
 
-// Catalog --
-type Catalog struct {
-	Version  string              `yaml:"version"`
-	Artifact map[string]Artifact `yaml:"artifacts"`
-}
-
-// Artifact --
-type Artifact struct {
-	GroupID     string   `yaml:"groupId"`
-	ArtifactID  string   `yaml:"artifactId"`
-	Version     string   `yaml:"version"`
-	Schemes     []string `yaml:"schemes"`
-	Languages   []string `yaml:"languages"`
-	DataFormats []string `yaml:"dataformats"`
-}
-
-func init() {
-	data := deploy.Resources["camel-catalog.yaml"]
-	if err := yaml.Unmarshal([]byte(data), &Runtime); err != nil {
-		panic(err)
+// Language discovers the code language from file extension if not set
+func Language(source v1alpha1.SourceSpec) v1alpha1.Language {
+	if source.Language != "" {
+		return source.Language
+	}
+	for _, l := range []v1alpha1.Language{
+		v1alpha1.LanguageJavaSource,
+		v1alpha1.LanguageJavaClass,
+		v1alpha1.LanguageJavaScript,
+		v1alpha1.LanguageGroovy,
+		v1alpha1.LanguageJavaScript} {
+
+		if strings.HasSuffix(source.Name, "."+string(l)) {
+			return l
+		}
 	}
+	return ""
 }
-
-// Runtime --
-var Runtime Catalog
diff --git a/pkg/util/camel/catalog.go b/pkg/discover/languages_test.go
similarity index 54%
copy from pkg/util/camel/catalog.go
copy to pkg/discover/languages_test.go
index cf6bc99..919de9d 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/discover/languages_test.go
@@ -15,35 +15,27 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package catalog
+package discover
 
 import (
-	"github.com/apache/camel-k/deploy"
-	"gopkg.in/yaml.v2"
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/stretchr/testify/assert"
+	"testing"
 )
 
-// Catalog --
-type Catalog struct {
-	Version  string              `yaml:"version"`
-	Artifact map[string]Artifact `yaml:"artifacts"`
-}
-
-// Artifact --
-type Artifact struct {
-	GroupID     string   `yaml:"groupId"`
-	ArtifactID  string   `yaml:"artifactId"`
-	Version     string   `yaml:"version"`
-	Schemes     []string `yaml:"schemes"`
-	Languages   []string `yaml:"languages"`
-	DataFormats []string `yaml:"dataformats"`
+func TestLanguageJavaSource(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name: "Source.java",
+	}
+	language := Language(code)
+	assert.Equal(t, v1alpha1.LanguageJavaSource, language)
 }
 
-func init() {
-	data := deploy.Resources["camel-catalog.yaml"]
-	if err := yaml.Unmarshal([]byte(data), &Runtime); err != nil {
-		panic(err)
+func TestLanguageAlreadySet(t *testing.T) {
+	code := v1alpha1.SourceSpec{
+		Name:     "Source.java",
+		Language: v1alpha1.LanguageJavaScript,
 	}
+	language := Language(code)
+	assert.Equal(t, v1alpha1.LanguageJavaScript, language)
 }
-
-// Runtime --
-var Runtime Catalog
diff --git a/pkg/stub/action/integration/deploy.go b/pkg/stub/action/integration/deploy.go
index 99545f6..8d873ea 100644
--- a/pkg/stub/action/integration/deploy.go
+++ b/pkg/stub/action/integration/deploy.go
@@ -85,7 +85,7 @@ func getConfigMapFor(ctx *v1alpha1.IntegrationContext, integration *v1alpha1.Int
 			Namespace: integration.Namespace,
 			Labels:    integration.Labels,
 			Annotations: map[string]string{
-				"camel.apache.org/source.language": integration.Spec.Source.Language,
+				"camel.apache.org/source.language": string(integration.Spec.Source.Language),
 				"camel.apache.org/source.name":     integration.Spec.Source.Name,
 			},
 			OwnerReferences: []metav1.OwnerReference{
@@ -143,7 +143,7 @@ func getDeploymentFor(ctx *v1alpha1.IntegrationContext, integration *v1alpha1.In
 	// set env vars needed by the runtime
 	environment["JAVA_MAIN_CLASS"] = "org.apache.camel.k.jvm.Application"
 	environment["CAMEL_K_ROUTES_URI"] = "file:/etc/camel/conf/" + sourceName
-	environment["CAMEL_K_ROUTES_LANGUAGE"] = integration.Spec.Source.Language
+	environment["CAMEL_K_ROUTES_LANGUAGE"] = string(integration.Spec.Source.Language)
 	environment["CAMEL_K_CONF"] = "/etc/camel/conf/application.properties"
 	environment["CAMEL_K_CONF_D"] = "/etc/camel/conf.d"
 
diff --git a/pkg/stub/action/integration/initialize.go b/pkg/stub/action/integration/initialize.go
index f2747eb..25322fd 100644
--- a/pkg/stub/action/integration/initialize.go
+++ b/pkg/stub/action/integration/initialize.go
@@ -19,35 +19,70 @@ package action
 
 import (
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/discover"
 	"github.com/apache/camel-k/pkg/util/digest"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
+	"sort"
 )
 
-// initializes the integration status to trigger the deployment
+// InitializeAction initializes the integration status to trigger the deployment
 type InitializeAction struct {
 }
 
+// NewInitializeAction creates a new inititialize action
 func NewInitializeAction() IntegrationAction {
 	return &InitializeAction{}
 }
 
+// Name returns a common name of the action
 func (b *InitializeAction) Name() string {
 	return "initialize"
 }
 
+// CanHandle tells whether this action can handle the integration
 func (b *InitializeAction) CanHandle(integration *v1alpha1.Integration) bool {
 	return integration.Status.Phase == ""
 }
 
+// Handle handles the integratios
 func (b *InitializeAction) Handle(integration *v1alpha1.Integration) error {
 	target := integration.DeepCopy()
 	// set default values
-	var defaultReplicas int32 = 1
 	if target.Spec.Replicas == nil {
+		var defaultReplicas int32 = 1
 		target.Spec.Replicas = &defaultReplicas
 	}
+	// set the correct language
+	language := discover.Language(target.Spec.Source)
+	target.Spec.Source.Language = language
+	// discover dependencies
+	if target.Spec.DependenciesAutoDiscovery == nil {
+		var autoDiscoveryDependencies = true
+		target.Spec.DependenciesAutoDiscovery = &autoDiscoveryDependencies
+	}
+	if *target.Spec.DependenciesAutoDiscovery {
+		discovered := discover.Dependencies(target.Spec.Source)
+		target.Spec.Dependencies = b.mergeDependencies(target.Spec.Dependencies, discovered)
+	}
+	// sort the dependencies to get always the same list if they don't change
+	sort.Strings(target.Spec.Dependencies)
 	// update the status
 	target.Status.Phase = v1alpha1.IntegrationPhaseBuilding
 	target.Status.Digest = digest.ComputeForIntegration(integration)
 	return sdk.Update(target)
 }
+
+func (b *InitializeAction) mergeDependencies(list1 []string, list2 []string) []string {
+	set := make(map[string]bool, 0)
+	for _, d := range list1 {
+		set[d] = true
+	}
+	for _, d := range list2 {
+		set[d] = true
+	}
+	ret := make([]string, 0, len(set))
+	for d := range set {
+		ret = append(ret, d)
+	}
+	return ret
+}
diff --git a/pkg/util/camel/catalog.go b/pkg/util/camel/catalog.go
index cf6bc99..2e81e95 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/util/camel/catalog.go
@@ -15,7 +15,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package catalog
+package camel
 
 import (
 	"github.com/apache/camel-k/deploy"
@@ -24,8 +24,9 @@ import (
 
 // Catalog --
 type Catalog struct {
-	Version  string              `yaml:"version"`
-	Artifact map[string]Artifact `yaml:"artifacts"`
+	Version          string              `yaml:"version"`
+	Artifact         map[string]Artifact `yaml:"artifacts"`
+	artifactByScheme map[string]string   `yaml:"-"`
 }
 
 // Artifact --
@@ -43,6 +44,22 @@ func init() {
 	if err := yaml.Unmarshal([]byte(data), &Runtime); err != nil {
 		panic(err)
 	}
+	Runtime.artifactByScheme = make(map[string]string)
+	for id, artifact := range Runtime.Artifact {
+		for _, scheme := range artifact.Schemes {
+			Runtime.artifactByScheme[scheme] = id
+		}
+	}
+}
+
+// GetArtifactByScheme returns the artifact corresponding to the given component scheme
+func (c Catalog) GetArtifactByScheme(scheme string) *Artifact {
+	if id, ok := c.artifactByScheme[scheme]; ok {
+		if artifact, present := c.Artifact[id]; present {
+			return &artifact
+		}
+	}
+	return nil
 }
 
 // Runtime --
diff --git a/pkg/util/camel/catalog.go b/pkg/util/camel/catalog_test.go
similarity index 53%
copy from pkg/util/camel/catalog.go
copy to pkg/util/camel/catalog_test.go
index cf6bc99..f59ff8c 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/util/camel/catalog_test.go
@@ -15,35 +15,14 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package catalog
+package camel
 
 import (
-	"github.com/apache/camel-k/deploy"
-	"gopkg.in/yaml.v2"
+	"github.com/stretchr/testify/assert"
+	"testing"
 )
 
-// Catalog --
-type Catalog struct {
-	Version  string              `yaml:"version"`
-	Artifact map[string]Artifact `yaml:"artifacts"`
+func TestCatalog(t *testing.T) {
+	assert.NotNil(t, Runtime)
+	assert.NotEmpty(t, Runtime.Artifact)
 }
-
-// Artifact --
-type Artifact struct {
-	GroupID     string   `yaml:"groupId"`
-	ArtifactID  string   `yaml:"artifactId"`
-	Version     string   `yaml:"version"`
-	Schemes     []string `yaml:"schemes"`
-	Languages   []string `yaml:"languages"`
-	DataFormats []string `yaml:"dataformats"`
-}
-
-func init() {
-	data := deploy.Resources["camel-catalog.yaml"]
-	if err := yaml.Unmarshal([]byte(data), &Runtime); err != nil {
-		panic(err)
-	}
-}
-
-// Runtime --
-var Runtime Catalog
diff --git a/runtime/examples/dns.js b/runtime/examples/dns.js
index 337682d..f7c90d2 100644
--- a/runtime/examples/dns.js
+++ b/runtime/examples/dns.js
@@ -3,10 +3,14 @@
 //
 //     kamel run -d camel:dns runtime/examples/dns.js
 //
+// Or simply (since dependency auto-detection is enabled by default):
+//
+//     kamel run runtime/examples/dns.js
+//
 
 from('timer:dns?period=1s')
     .routeId('dns')
     .setHeader('dns.domain')
         .constant('www.google.com')
     .to('dns:ip')
-    .to('log:dns')
\ No newline at end of file
+    .to('log:dns');
\ No newline at end of file