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/11/05 19:46:29 UTC

[camel-k] branch master updated (8ced01e -> 27e0385)

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

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


    from 8ced01e  Document operator-sdk version 0.0.7
     new bacb33c  Refactoring traits package
     new 9b9f743  Add ingress trait
     new 74c1f49  Fix roles for ingress
     new a7057af  Document the ingress trait
     new c028e80  Add host property to the route trait
     new 27e0385  Moved dependencies to a proper trait

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 deploy/cr-example.yaml                             |   2 -
 deploy/operator-role-kubernetes.yaml               |  13 ++
 deploy/operator-role-openshift.yaml                |   8 +
 deploy/resources.go                                |  23 ++-
 docs/traits.adoc                                   |  36 +++-
 pkg/apis/camel/v1alpha1/types.go                   |  13 +-
 pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go   |   5 -
 pkg/client/cmd/completion_bash.go                  |   2 +-
 pkg/client/cmd/run.go                              |  56 ++----
 pkg/discover/doc.go                                |  20 --
 pkg/{discover => metadata}/dependencies.go         |  62 ++----
 pkg/{trait => metadata}/doc.go                     |   4 +-
 .../language.go => metadata/languages.go}          |   7 +-
 pkg/{util/log/util.go => metadata/metadata.go}     |  28 ++-
 .../metadata_dependencies_test.go}                 |  22 +--
 .../metadata_languages_test.go}                    |  10 +-
 pkg/metadata/metadata_uri_test.go                  | 215 +++++++++++++++++++++
 pkg/{build/packager => metadata}/types.go          |  21 +-
 pkg/metadata/uris.go                               | 100 ++++++++++
 pkg/stub/action/integration/deploy.go              |   2 +-
 pkg/stub/action/integration/initialize.go          |  48 +----
 pkg/trait/catalog.go                               | 168 ++++++++++++----
 pkg/trait/dependencies.go                          |  69 +++++++
 pkg/trait/{base.go => deployment.go}               |  18 +-
 pkg/trait/ingress.go                               |  98 ++++++++++
 pkg/trait/owner.go                                 |  12 +-
 pkg/trait/route.go                                 |  44 +++--
 pkg/trait/service.go                               |  36 ++--
 pkg/trait/trait.go                                 |  32 ++-
 pkg/trait/trait_test.go                            |  56 +++---
 pkg/trait/types.go                                 | 117 +++++------
 pkg/trait/util.go                                  |  34 +---
 32 files changed, 948 insertions(+), 433 deletions(-)
 delete mode 100644 pkg/discover/doc.go
 rename pkg/{discover => metadata}/dependencies.go (50%)
 copy pkg/{trait => metadata}/doc.go (89%)
 rename pkg/{discover/language.go => metadata/languages.go} (84%)
 copy pkg/{util/log/util.go => metadata/metadata.go} (63%)
 rename pkg/{discover/dependencies_test.go => metadata/metadata_dependencies_test.go} (90%)
 rename pkg/{discover/languages_test.go => metadata/metadata_languages_test.go} (85%)
 create mode 100644 pkg/metadata/metadata_uri_test.go
 copy pkg/{build/packager => metadata}/types.go (61%)
 create mode 100644 pkg/metadata/uris.go
 create mode 100644 pkg/trait/dependencies.go
 rename pkg/trait/{base.go => deployment.go} (92%)
 create mode 100644 pkg/trait/ingress.go


[camel-k] 04/06: Document the ingress trait

Posted by lb...@apache.org.
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 a7057af584e55e2f703eb62e27226994d0f0ffa9
Author: nferraro <ni...@gmail.com>
AuthorDate: Mon Nov 5 15:10:28 2018 +0100

    Document the ingress trait
---
 docs/traits.adoc | 15 +++++++++++++++
 1 file changed, 15 insertions(+)

diff --git a/docs/traits.adoc b/docs/traits.adoc
index efe2aa1..c25e533 100644
--- a/docs/traits.adoc
+++ b/docs/traits.adoc
@@ -77,6 +77,21 @@ The following is a list of common traits that can be configured by the end users
   +
   It's enabled by default whenever a Service is added to the integration (through the `service` trait).
 
+| ingress
+| Kubernetes
+| Exposes the service associated with the integration to the outside world with a Kubernetes Ingress.
+  +
+  +
+  It's enabled by default whenever a Service is added to the integration (through the `service` trait).
+
+[cols="m,"]
+!===
+
+! ingress.host
+! **Required**. To configure the host exposed by the ingress.
+
+!===
+
 |=======================
 
 


[camel-k] 02/06: Add ingress trait

Posted by lb...@apache.org.
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 9b9f7437eae8872480c0e5facaf4cf5809912524
Author: nferraro <ni...@gmail.com>
AuthorDate: Wed Oct 31 00:17:20 2018 +0100

    Add ingress trait
---
 pkg/trait/catalog.go               |  4 +++
 pkg/trait/{route.go => ingress.go} | 55 +++++++++++++++++++++++---------------
 pkg/trait/route.go                 |  6 +++--
 3 files changed, 41 insertions(+), 24 deletions(-)

diff --git a/pkg/trait/catalog.go b/pkg/trait/catalog.go
index 049b1a6..d5cf865 100644
--- a/pkg/trait/catalog.go
+++ b/pkg/trait/catalog.go
@@ -30,6 +30,7 @@ type Catalog struct {
 	tDeployment ITrait
 	tService    ITrait
 	tRoute      ITrait
+	tIngress    ITrait
 	tOwner      ITrait
 }
 
@@ -39,6 +40,7 @@ func NewCatalog() *Catalog {
 		tDeployment: newDeploymentTrait(),
 		tService:    newServiceTrait(),
 		tRoute:      newRouteTrait(),
+		tIngress:    newIngressTrait(),
 		tOwner:      newOwnerTrait(),
 	}
 }
@@ -48,6 +50,7 @@ func (c *Catalog) allTraits() []ITrait {
 		c.tDeployment,
 		c.tService,
 		c.tRoute,
+		c.tIngress,
 		c.tOwner,
 	}
 }
@@ -65,6 +68,7 @@ func (c *Catalog) traitsFor(environment *environment) []ITrait {
 		return []ITrait{
 			c.tDeployment,
 			c.tService,
+			c.tIngress,
 			c.tOwner,
 		}
 		// case Knative: ...
diff --git a/pkg/trait/route.go b/pkg/trait/ingress.go
similarity index 54%
copy from pkg/trait/route.go
copy to pkg/trait/ingress.go
index 1553bae..bbd7017 100644
--- a/pkg/trait/route.go
+++ b/pkg/trait/ingress.go
@@ -18,41 +18,50 @@ limitations under the License.
 package trait
 
 import (
+	"errors"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
-	routev1 "github.com/openshift/api/route/v1"
 	corev1 "k8s.io/api/core/v1"
+	"k8s.io/api/extensions/v1beta1"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	"k8s.io/apimachinery/pkg/util/intstr"
 )
 
-type routeTrait struct {
+type ingressTrait struct {
 	BaseTrait `property:",squash"`
+	Host      string `property:"host"`
 }
 
-func newRouteTrait() *routeTrait {
-	return &routeTrait{
-		BaseTrait: newBaseTrait("route"),
+func newIngressTrait() *ingressTrait {
+	return &ingressTrait{
+		BaseTrait: newBaseTrait("ingress"),
+		Host:      "",
 	}
 }
 
-func (e *routeTrait) autoconfigure(environment *environment, resources *kubernetes.Collection) error {
+func (e *ingressTrait) autoconfigure(environment *environment, resources *kubernetes.Collection) error {
 	if e.Enabled == nil {
 		hasService := e.getTargetService(environment, resources) != nil
-		e.Enabled = &hasService
+		hasHost := e.Host != ""
+		enabled := hasService && hasHost
+		e.Enabled = &enabled
 	}
 	return nil
 }
 
-func (e *routeTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+func (e *ingressTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+	if e.Host == "" {
+		return errors.New("cannot apply ingress trait: no host defined")
+	}
 	service := e.getTargetService(environment, resources)
-	if service != nil {
-		resources.Add(e.getRouteFor(environment, service))
+	if service == nil {
+		return errors.New("cannot apply ingress trait: no target service")
 	}
 
+	resources.Add(e.getIngressFor(environment, service))
 	return nil
 }
 
-func (*routeTrait) getTargetService(e *environment, resources *kubernetes.Collection) (service *corev1.Service) {
+func (*ingressTrait) getTargetService(e *environment, resources *kubernetes.Collection) (service *corev1.Service) {
 	resources.VisitService(func(s *corev1.Service) {
 		if s.ObjectMeta.Labels != nil {
 			if intName, ok := s.ObjectMeta.Labels["camel.apache.org/integration"]; ok && intName == e.Integration.Name {
@@ -63,25 +72,27 @@ func (*routeTrait) getTargetService(e *environment, resources *kubernetes.Collec
 	return
 }
 
-func (*routeTrait) getRouteFor(e *environment, service *corev1.Service) *routev1.Route {
-	route := routev1.Route{
+func (e *ingressTrait) getIngressFor(env *environment, service *corev1.Service) *v1beta1.Ingress {
+	ingress := v1beta1.Ingress{
 		TypeMeta: metav1.TypeMeta{
-			Kind:       "Route",
-			APIVersion: routev1.SchemeGroupVersion.String(),
+			Kind:       "Ingress",
+			APIVersion: v1beta1.SchemeGroupVersion.String(),
 		},
 		ObjectMeta: metav1.ObjectMeta{
 			Name:      service.Name,
 			Namespace: service.Namespace,
 		},
-		Spec: routev1.RouteSpec{
-			Port: &routev1.RoutePort{
-				TargetPort: intstr.FromString("http"),
+		Spec: v1beta1.IngressSpec{
+			Backend: &v1beta1.IngressBackend{
+				ServiceName: service.Name,
+				ServicePort: intstr.FromString("http"),
 			},
-			To: routev1.RouteTargetReference{
-				Kind: "Service",
-				Name: service.Name,
+			Rules: []v1beta1.IngressRule{
+				{
+					Host: e.Host,
+				},
 			},
 		},
 	}
-	return &route
+	return &ingress
 }
diff --git a/pkg/trait/route.go b/pkg/trait/route.go
index 1553bae..b3efa1a 100644
--- a/pkg/trait/route.go
+++ b/pkg/trait/route.go
@@ -18,6 +18,7 @@ limitations under the License.
 package trait
 
 import (
+	"errors"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
 	routev1 "github.com/openshift/api/route/v1"
 	corev1 "k8s.io/api/core/v1"
@@ -45,10 +46,11 @@ func (e *routeTrait) autoconfigure(environment *environment, resources *kubernet
 
 func (e *routeTrait) customize(environment *environment, resources *kubernetes.Collection) error {
 	service := e.getTargetService(environment, resources)
-	if service != nil {
-		resources.Add(e.getRouteFor(environment, service))
+	if service == nil {
+		return errors.New("cannot apply route trait: no target service")
 	}
 
+	resources.Add(e.getRouteFor(environment, service))
 	return nil
 }
 


[camel-k] 01/06: Refactoring traits package

Posted by lb...@apache.org.
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 bacb33c132bd09ea0154633894c942a51d27e927
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Oct 26 00:04:53 2018 +0200

    Refactoring traits package
---
 docs/traits.adoc                                   |   6 +-
 pkg/client/cmd/completion_bash.go                  |   2 +-
 pkg/client/cmd/run.go                              |   3 +-
 pkg/{discover => metadata}/dependencies.go         |  62 ++----
 pkg/{discover => metadata}/doc.go                  |   5 +-
 .../language.go => metadata/languages.go}          |   7 +-
 .../languages_test.go => metadata/metadata.go}     |  30 ++-
 .../metadata_dependencies_test.go}                 |  22 +--
 .../metadata_languages_test.go}                    |  10 +-
 pkg/metadata/metadata_uri_test.go                  | 215 +++++++++++++++++++++
 pkg/{discover/doc.go => metadata/types.go}         |  18 +-
 pkg/metadata/uris.go                               | 100 ++++++++++
 pkg/stub/action/integration/initialize.go          |  11 +-
 pkg/trait/catalog.go                               | 139 +++++++++----
 pkg/trait/{base.go => deployment.go}               |  18 +-
 pkg/trait/owner.go                                 |  12 +-
 pkg/trait/route.go                                 |  38 ++--
 pkg/trait/service.go                               |  36 ++--
 pkg/trait/trait.go                                 |  13 +-
 pkg/trait/trait_test.go                            |  56 +++---
 pkg/trait/types.go                                 | 113 +++++------
 pkg/trait/util.go                                  |  31 ---
 22 files changed, 627 insertions(+), 320 deletions(-)

diff --git a/docs/traits.adoc b/docs/traits.adoc
index c5ada79..efe2aa1 100644
--- a/docs/traits.adoc
+++ b/docs/traits.adoc
@@ -23,6 +23,10 @@ The flag `--trait` can be also abbreviated with `-t`.
 The `enabled` property is available on all traits and can be used to enable/disable them. All traits have their own
 internal logic to determine if they need to be enabled when the user does not activate them explicitly.
 
+All traits share also a `auto` property that can be used to enable/disable auto-configuration of the trait based on the
+environment. The auto-configuration mechanism is able to enable/disable the trait when the `enabled` property is not explicitly
+set by the user and also change the trait configuration. The `auto` property is enabled by default.
+
 NOTE: Some traits are applicable only to specific platforms (see "profiles" in the table).
 
 A trait may have additional properties that can be configured by the end user.
@@ -83,6 +87,6 @@ There are also platform traits that **normally should not be configured** by the
 [options="header",cols="m,,"]
 |=======================
 | Trait      | Profiles 				| Description
-| base		 | Kubernetes, OpenShift	| Creates the basic Kubernetes resource needed for running the integration.
+| deployment | Kubernetes, OpenShift	| Creates the basic Kubernetes resource needed for running the integration.
 | owner      | Kubernetes, OpenShift	| Makes sure that every resource created by the traits belongs to the integration custom resource (so they are deleted when the integration is deleted).
 |=======================
diff --git a/pkg/client/cmd/completion_bash.go b/pkg/client/cmd/completion_bash.go
index df76a1e..ab390fe 100644
--- a/pkg/client/cmd/completion_bash.go
+++ b/pkg/client/cmd/completion_bash.go
@@ -69,7 +69,7 @@ __kamel_dependency_type() {
 }
 
 __kamel_traits() {
-    local type_list="` + strings.Join(trait.ComputeTraitsProperties(), " ") + `"
+    local type_list="` + strings.Join(trait.NewCatalog().ComputeTraitsProperties(), " ") + `"
     COMPREPLY=( $( compgen -W "${type_list}" -- "$cur") )
     compopt -o nospace
 }
diff --git a/pkg/client/cmd/run.go b/pkg/client/cmd/run.go
index 0778b07..3c59c84 100644
--- a/pkg/client/cmd/run.go
+++ b/pkg/client/cmd/run.go
@@ -127,7 +127,8 @@ func (o *runCmdOptions) validateArgs(cmd *cobra.Command, args []string) error {
 }
 
 func (o *runCmdOptions) run(cmd *cobra.Command, args []string) error {
-	tp := trait.ComputeTraitsProperties()
+	catalog := trait.NewCatalog()
+	tp := catalog.ComputeTraitsProperties()
 	for _, t := range o.Traits {
 		kv := strings.SplitN(t, "=", 2)
 
diff --git a/pkg/discover/dependencies.go b/pkg/metadata/dependencies.go
similarity index 50%
rename from pkg/discover/dependencies.go
rename to pkg/metadata/dependencies.go
index 8e402b2..d207228 100644
--- a/pkg/discover/dependencies.go
+++ b/pkg/metadata/dependencies.go
@@ -15,10 +15,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package discover
+package metadata
 
 import (
-	"regexp"
 	"sort"
 	"strings"
 
@@ -26,23 +25,14 @@ import (
 	"github.com/apache/camel-k/pkg/util/camel"
 )
 
-var (
-	singleQuotedURI *regexp.Regexp
-	doubleQuotedURI *regexp.Regexp
-)
-
-func init() {
-	singleQuotedURI = regexp.MustCompile("'([a-z0-9-]+):[^']+'")
-	doubleQuotedURI = regexp.MustCompile("\"([a-z0-9-]+):[^\"]+\"")
-}
-
-// Dependencies returns a list of dependencies required by the given source code
-func Dependencies(source v1alpha1.SourceSpec) []string {
+// discoverDependencies returns a list of dependencies required by the given source code
+func discoverDependencies(source v1alpha1.SourceSpec, fromURIs []string, toURIs []string) []string {
 	candidateMap := make(map[string]bool)
-	regexps := getRegexpsForLanguage(source.Language)
-	subMatches := findAllStringSubmatch(source.Content, regexps...)
-	for _, uriPrefix := range subMatches {
-		candidateComp := decodeComponent(uriPrefix)
+	uris := make([]string, 0, len(fromURIs)+len(toURIs))
+	uris = append(uris, fromURIs...)
+	uris = append(uris, toURIs...)
+	for _, uri := range uris {
+		candidateComp := decodeComponent(uri)
 		if candidateComp != "" {
 			candidateMap[candidateComp] = true
 		}
@@ -56,38 +46,12 @@ func Dependencies(source v1alpha1.SourceSpec) []string {
 	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}
-	case v1alpha1.LanguageKotlin:
-		return []*regexp.Regexp{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)
-				}
-			}
-		}
+func decodeComponent(uri string) string {
+	uriSplit := strings.SplitN(uri, ":", 2)
+	if len(uriSplit) < 2 {
+		return ""
 	}
-	return candidates
-}
-
-func decodeComponent(uriStart string) string {
+	uriStart := uriSplit[0]
 	if component := camel.Runtime.GetArtifactByScheme(uriStart); component != nil {
 		artifactID := component.ArtifactID
 		if strings.HasPrefix(artifactID, "camel-") {
diff --git a/pkg/discover/doc.go b/pkg/metadata/doc.go
similarity index 86%
copy from pkg/discover/doc.go
copy to pkg/metadata/doc.go
index 51cc065..e1b5958 100644
--- a/pkg/discover/doc.go
+++ b/pkg/metadata/doc.go
@@ -15,6 +15,5 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-// Package discover contains functions for extracting
-// information from user code before compilation
-package discover
+// Package metadata contains tools to discover metadata from Camel routes
+package metadata
diff --git a/pkg/discover/language.go b/pkg/metadata/languages.go
similarity index 84%
rename from pkg/discover/language.go
rename to pkg/metadata/languages.go
index cafb6c3..2d0d2da 100644
--- a/pkg/discover/language.go
+++ b/pkg/metadata/languages.go
@@ -15,8 +15,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-// Package discover contains functions for analyzing user code
-package discover
+package metadata
 
 import (
 	"strings"
@@ -24,8 +23,8 @@ import (
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 )
 
-// Language discovers the code language from file extension if not set
-func Language(source v1alpha1.SourceSpec) v1alpha1.Language {
+// discoverLanguage discovers the code language from file extension if not set
+func discoverLanguage(source v1alpha1.SourceSpec) v1alpha1.Language {
 	if source.Language != "" {
 		return source.Language
 	}
diff --git a/pkg/discover/languages_test.go b/pkg/metadata/metadata.go
similarity index 63%
copy from pkg/discover/languages_test.go
copy to pkg/metadata/metadata.go
index bbe0c45..46a7aef 100644
--- a/pkg/discover/languages_test.go
+++ b/pkg/metadata/metadata.go
@@ -15,28 +15,22 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package discover
+package metadata
 
 import (
-	"testing"
-
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
-	"github.com/stretchr/testify/assert"
 )
 
-func TestLanguageJavaSource(t *testing.T) {
-	code := v1alpha1.SourceSpec{
-		Name: "Request.java",
-	}
-	language := Language(code)
-	assert.Equal(t, v1alpha1.LanguageJavaSource, language)
-}
-
-func TestLanguageAlreadySet(t *testing.T) {
-	code := v1alpha1.SourceSpec{
-		Name:     "Request.java",
-		Language: v1alpha1.LanguageJavaScript,
+// Extract returns metadata information from the source code
+func Extract(source v1alpha1.SourceSpec) IntegrationMetadata {
+	language := discoverLanguage(source)
+	fromURIs := discoverFromURIs(source, language)
+	toURIs := discoverToURIs(source, language)
+	dependencies := discoverDependencies(source, fromURIs, toURIs)
+	return IntegrationMetadata{
+		Language:     language,
+		FromURIs:     fromURIs,
+		ToURIs:       toURIs,
+		Dependencies: dependencies,
 	}
-	language := Language(code)
-	assert.Equal(t, v1alpha1.LanguageJavaScript, language)
 }
diff --git a/pkg/discover/dependencies_test.go b/pkg/metadata/metadata_dependencies_test.go
similarity index 90%
rename from pkg/discover/dependencies_test.go
rename to pkg/metadata/metadata_dependencies_test.go
index a7934ef..8eea487 100644
--- a/pkg/discover/dependencies_test.go
+++ b/pkg/metadata/metadata_dependencies_test.go
@@ -15,7 +15,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package discover
+package metadata
 
 import (
 	"testing"
@@ -34,9 +34,9 @@ func TestDependenciesJavaSource(t *testing.T) {
 			from("ine:xistent").to("amqp:queue");
 		`,
 	}
-	dependencies := Dependencies(code)
+	meta := Extract(code)
 	// assert all dependencies are found and sorted (removing duplicates)
-	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, meta.Dependencies)
 }
 
 func TestDependenciesJavaClass(t *testing.T) {
@@ -49,8 +49,8 @@ func TestDependenciesJavaClass(t *testing.T) {
 			from("ine:xistent").to("amqp:queue");
 		`,
 	}
-	dependencies := Dependencies(code)
-	assert.Empty(t, dependencies)
+	meta := Extract(code)
+	assert.Empty(t, meta.Dependencies)
 }
 
 func TestDependenciesJavaScript(t *testing.T) {
@@ -64,9 +64,9 @@ func TestDependenciesJavaScript(t *testing.T) {
 			'"'
 		`,
 	}
-	dependencies := Dependencies(code)
+	meta := Extract(code)
 	// assert all dependencies are found and sorted (removing duplicates)
-	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, meta.Dependencies)
 }
 
 func TestDependenciesGroovy(t *testing.T) {
@@ -80,9 +80,9 @@ func TestDependenciesGroovy(t *testing.T) {
 			'"'
 		`,
 	}
-	dependencies := Dependencies(code)
+	meta := Extract(code)
 	// assert all dependencies are found and sorted (removing duplicates)
-	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, dependencies)
+	assert.Equal(t, []string{"camel:amqp", "camel:core", "camel:telegram"}, meta.Dependencies)
 }
 
 func TestDependencies(t *testing.T) {
@@ -95,7 +95,7 @@ func TestDependencies(t *testing.T) {
 			from("twitter-timeline:test").to("mock:end");
 		`,
 	}
-	dependencies := Dependencies(code)
+	meta := Extract(code)
 	// assert all dependencies are found and sorted (removing duplicates)
-	assert.Equal(t, []string{"camel:core", "camel:http4", "camel:twitter"}, dependencies)
+	assert.Equal(t, []string{"camel:core", "camel:http4", "camel:twitter"}, meta.Dependencies)
 }
diff --git a/pkg/discover/languages_test.go b/pkg/metadata/metadata_languages_test.go
similarity index 85%
rename from pkg/discover/languages_test.go
rename to pkg/metadata/metadata_languages_test.go
index bbe0c45..5382d38 100644
--- a/pkg/discover/languages_test.go
+++ b/pkg/metadata/metadata_languages_test.go
@@ -15,7 +15,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package discover
+package metadata
 
 import (
 	"testing"
@@ -28,8 +28,8 @@ func TestLanguageJavaSource(t *testing.T) {
 	code := v1alpha1.SourceSpec{
 		Name: "Request.java",
 	}
-	language := Language(code)
-	assert.Equal(t, v1alpha1.LanguageJavaSource, language)
+	meta := Extract(code)
+	assert.Equal(t, v1alpha1.LanguageJavaSource, meta.Language)
 }
 
 func TestLanguageAlreadySet(t *testing.T) {
@@ -37,6 +37,6 @@ func TestLanguageAlreadySet(t *testing.T) {
 		Name:     "Request.java",
 		Language: v1alpha1.LanguageJavaScript,
 	}
-	language := Language(code)
-	assert.Equal(t, v1alpha1.LanguageJavaScript, language)
+	meta := Extract(code)
+	assert.Equal(t, v1alpha1.LanguageJavaScript, meta.Language)
 }
diff --git a/pkg/metadata/metadata_uri_test.go b/pkg/metadata/metadata_uri_test.go
new file mode 100644
index 0000000..84da92a
--- /dev/null
+++ b/pkg/metadata/metadata_uri_test.go
@@ -0,0 +1,215 @@
+/*
+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 metadata
+
+import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/stretchr/testify/assert"
+	"testing"
+)
+
+func TestJava1(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageJavaSource,
+		Content: `
+			import org.apache.camel.builder.RouteBuilder;
+
+			public class Sample extends RouteBuilder {
+  				@Override
+  				public void configure() throws Exception {
+		  			from("timer:tick")
+		    			.setBody(constant("-\n             r\n             o\n             c\nHello! Camel K\n             s\n             !\n"))
+							.to("log:info?skipBodyLineSeparator=false");
+  				}
+			}
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Contains(t, metadata.FromURIs, "timer:tick")
+	assert.Len(t, metadata.FromURIs, 1)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Len(t, metadata.ToURIs, 1)
+}
+
+func TestJava2(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageJavaSource,
+		Content: `
+			import org.apache.camel.builder.RouteBuilder;
+
+			public class Sample extends RouteBuilder {
+  				@Override
+  				public void configure() throws Exception {
+		  			from("timer:tick")
+		    			.setBody(constant("!\n"))
+							.to           (
+
+											"log:info?skipBodyLineSeparator=false"
+
+							               )
+							.toD("uri:2")
+							.toF("uri:%s", "3");
+  				}
+			}
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Contains(t, metadata.FromURIs, "timer:tick")
+	assert.Len(t, metadata.FromURIs, 1)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Contains(t, metadata.ToURIs, "uri:2")
+	assert.Contains(t, metadata.ToURIs, "uri:%s") // resolution not supported yet
+	assert.Len(t, metadata.ToURIs, 3)
+}
+
+func TestGroovy1(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageGroovy,
+		Content: `
+			
+		  	from( "timer:tick")
+		    	.setBody().constant("aa")
+				.to   ('log:info?skipBodyLineSeparator=false').to(
+											'http://url' )
+  			
+			from("uri:2")
+		    	.setBody().constant("aa")
+				.to('uri:3')
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Contains(t, metadata.FromURIs, "timer:tick")
+	assert.Contains(t, metadata.FromURIs, "uri:2")
+	assert.Len(t, metadata.FromURIs, 2)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Contains(t, metadata.ToURIs, "http://url")
+	assert.Contains(t, metadata.ToURIs, "uri:3")
+	assert.Len(t, metadata.ToURIs, 3)
+}
+
+func TestGroovy2(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageGroovy,
+		Content: `
+			
+			rest().get("/")
+				.to   ('log:info?skipBodyLineSeparator=false').to( 'http://url' )
+						.toD('dyn:1')
+						.tony('thisisnot:anuri')
+						.toD( "dyn:2")
+						.toF( "f:%s", "2")
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Empty(t, metadata.FromURIs)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Contains(t, metadata.ToURIs, "http://url")
+	assert.Contains(t, metadata.ToURIs, "dyn:1")
+	assert.Contains(t, metadata.ToURIs, "dyn:2")
+	assert.Contains(t, metadata.ToURIs, "f:%s") // resolution not supported yet
+	assert.Len(t, metadata.ToURIs, 5)
+}
+
+func TestXml1(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageXML,
+		Content: `
+			<routes>
+			    <route id="hello">
+        			<from uri="timer:hello?period=3s"/>
+        			<setBody>
+            			<constant>Hello World!!!</constant>
+        			</setBody>
+        			<to uri="log:info"/>
+					<to uri="log:info2"/>
+					<toD uri="log:info3"/>
+    			</route>
+			</routes>
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Contains(t, metadata.FromURIs, "timer:hello?period=3s")
+	assert.Len(t, metadata.FromURIs, 1)
+	assert.Contains(t, metadata.ToURIs, "log:info")
+	assert.Contains(t, metadata.ToURIs, "log:info2")
+	assert.Contains(t, metadata.ToURIs, "log:info3")
+	assert.Len(t, metadata.ToURIs, 3)
+}
+
+func TestKotlin1(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageKotlin,
+		Content: `
+			
+		  	from( "timer:tick")
+		    	.setBody().constant("aa")
+				.to   ("log:info?skipBodyLineSeparator=false").to(
+											"http://url" )
+  			
+			from("uri:2")
+		    	.setBody().constant("aa")
+				.to("uri:3")
+				.toD("uri:4")
+				.toF("uri:%s", 5)
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Contains(t, metadata.FromURIs, "timer:tick")
+	assert.Contains(t, metadata.FromURIs, "uri:2")
+	assert.Len(t, metadata.FromURIs, 2)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Contains(t, metadata.ToURIs, "http://url")
+	assert.Contains(t, metadata.ToURIs, "uri:3")
+	assert.Contains(t, metadata.ToURIs, "uri:4")
+	assert.Contains(t, metadata.ToURIs, "uri:%s") // resolution not supported yet
+	assert.Len(t, metadata.ToURIs, 5)
+}
+
+func TestJavascript1(t *testing.T) {
+	source := v1alpha1.SourceSpec{
+		Name:     "test",
+		Language: v1alpha1.LanguageJavaScript,
+		Content: `
+			
+			rest().get("/")
+				.to   ('log:info?skipBodyLineSeparator=false').to( 'http://url' )
+				.toD("uri:2")
+				.toF("uri:%s", "3") 
+		`,
+	}
+
+	metadata := Extract(source)
+	assert.Empty(t, metadata.FromURIs)
+	assert.Contains(t, metadata.ToURIs, "log:info?skipBodyLineSeparator=false")
+	assert.Contains(t, metadata.ToURIs, "http://url")
+	assert.Contains(t, metadata.ToURIs, "uri:2")
+	assert.Contains(t, metadata.ToURIs, "uri:%s") // resolution not supported yet
+	assert.Len(t, metadata.ToURIs, 4)
+}
diff --git a/pkg/discover/doc.go b/pkg/metadata/types.go
similarity index 61%
rename from pkg/discover/doc.go
rename to pkg/metadata/types.go
index 51cc065..74ac309 100644
--- a/pkg/discover/doc.go
+++ b/pkg/metadata/types.go
@@ -15,6 +15,18 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-// Package discover contains functions for extracting
-// information from user code before compilation
-package discover
+package metadata
+
+import "github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+
+// IntegrationMetadata contains aggregate metadata about all Camel routes in a integrations
+type IntegrationMetadata struct {
+	// All starting URIs of defined routes
+	FromURIs []string
+	// All end URIs of defined routes
+	ToURIs   []string
+	// All inferred dependencies required to run the integration
+	Dependencies []string
+	// The language in which the integration is written
+	Language v1alpha1.Language
+}
diff --git a/pkg/metadata/uris.go b/pkg/metadata/uris.go
new file mode 100644
index 0000000..e9c17ab
--- /dev/null
+++ b/pkg/metadata/uris.go
@@ -0,0 +1,100 @@
+/*
+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 metadata
+
+import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"regexp"
+)
+
+var (
+	singleQuotedFrom = regexp.MustCompile("from\\s*\\(\\s*'([a-z0-9-]+:[^']+)'\\s*\\)")
+	doubleQuotedFrom = regexp.MustCompile("from\\s*\\(\\s*\"([a-z0-9-]+:[^\"]+)\"\\s*\\)")
+	singleQuotedTo   = regexp.MustCompile("\\.to\\s*\\(\\s*'([a-z0-9-]+:[^']+)'\\s*\\)")
+	singleQuotedToD  = regexp.MustCompile("\\.toD\\s*\\(\\s*'([a-z0-9-]+:[^']+)'\\s*\\)")
+	singleQuotedToF  = regexp.MustCompile("\\.toF\\s*\\(\\s*'([a-z0-9-]+:[^']+)'[^)]*\\)")
+	doubleQuotedTo   = regexp.MustCompile("\\.to\\s*\\(\\s*\"([a-z0-9-]+:[^\"]+)\"\\s*\\)")
+	doubleQuotedToD  = regexp.MustCompile("\\.toD\\s*\\(\\s*\"([a-z0-9-]+:[^\"]+)\"\\s*\\)")
+	doubleQuotedToF  = regexp.MustCompile("\\.toF\\s*\\(\\s*\"([a-z0-9-]+:[^\"]+)\"[^)]*\\)")
+	xmlTagFrom       = regexp.MustCompile("<\\s*from\\s+[^>]*uri\\s*=\\s*\"([a-z0-9-]+:[^\"]+)\"[^>]*>")
+	xmlTagTo         = regexp.MustCompile("<\\s*to\\s+[^>]*uri\\s*=\\s*\"([a-z0-9-]+:[^\"]+)\"[^>]*>")
+	xmlTagToD        = regexp.MustCompile("<\\s*toD\\s+[^>]*uri\\s*=\\s*\"([a-z0-9-]+:[^\"]+)\"[^>]*>")
+)
+
+// discoverFromURIs returns all uris used in a from clause
+func discoverFromURIs(source v1alpha1.SourceSpec, language v1alpha1.Language) []string {
+	fromRegexps := getFromRegexpsForLanguage(language)
+	return findAllDistinctStringSubmatch(source.Content, fromRegexps...)
+}
+
+// discoverToURIs returns all uris used in a to clause
+func discoverToURIs(source v1alpha1.SourceSpec, language v1alpha1.Language) []string {
+	toRegexps := getToRegexpsForLanguage(language)
+	return findAllDistinctStringSubmatch(source.Content, toRegexps...)
+}
+
+func getFromRegexpsForLanguage(language v1alpha1.Language) []*regexp.Regexp {
+	switch language {
+	case v1alpha1.LanguageJavaSource:
+		return []*regexp.Regexp{doubleQuotedFrom}
+	case v1alpha1.LanguageXML:
+		return []*regexp.Regexp{xmlTagFrom}
+	case v1alpha1.LanguageGroovy:
+		return []*regexp.Regexp{singleQuotedFrom, doubleQuotedFrom}
+	case v1alpha1.LanguageJavaScript:
+		return []*regexp.Regexp{singleQuotedFrom, doubleQuotedFrom}
+	case v1alpha1.LanguageKotlin:
+		return []*regexp.Regexp{doubleQuotedFrom}
+	}
+	return []*regexp.Regexp{}
+}
+
+func getToRegexpsForLanguage(language v1alpha1.Language) []*regexp.Regexp {
+	switch language {
+	case v1alpha1.LanguageJavaSource:
+		return []*regexp.Regexp{doubleQuotedTo, doubleQuotedToD, doubleQuotedToF}
+	case v1alpha1.LanguageXML:
+		return []*regexp.Regexp{xmlTagTo, xmlTagToD}
+	case v1alpha1.LanguageGroovy:
+		return []*regexp.Regexp{singleQuotedTo, doubleQuotedTo, singleQuotedToD, doubleQuotedToD, singleQuotedToF, doubleQuotedToF}
+	case v1alpha1.LanguageJavaScript:
+		return []*regexp.Regexp{singleQuotedTo, doubleQuotedTo, singleQuotedToD, doubleQuotedToD, singleQuotedToF, doubleQuotedToF}
+	case v1alpha1.LanguageKotlin:
+		return []*regexp.Regexp{doubleQuotedTo, doubleQuotedToD, doubleQuotedToF}
+	}
+	return []*regexp.Regexp{}
+}
+
+func findAllDistinctStringSubmatch(data string, regexps ...*regexp.Regexp) []string {
+	candidates := make([]string, 0)
+	alreadyFound := make(map[string]bool)
+	for _, reg := range regexps {
+		hits := reg.FindAllStringSubmatch(data, -1)
+		for _, hit := range hits {
+			if hit != nil && len(hit) > 1 {
+				for _, match := range hit[1:] {
+					if _, ok := alreadyFound[match]; !ok {
+						alreadyFound[match] = true
+						candidates = append(candidates, match)
+					}
+				}
+			}
+		}
+	}
+	return candidates
+}
\ No newline at end of file
diff --git a/pkg/stub/action/integration/initialize.go b/pkg/stub/action/integration/initialize.go
index f0b5e7f..3abf51f 100644
--- a/pkg/stub/action/integration/initialize.go
+++ b/pkg/stub/action/integration/initialize.go
@@ -18,6 +18,7 @@ limitations under the License.
 package integration
 
 import (
+	"github.com/apache/camel-k/pkg/metadata"
 	"github.com/apache/camel-k/pkg/platform"
 	"github.com/sirupsen/logrus"
 	"sort"
@@ -25,7 +26,6 @@ import (
 	"github.com/apache/camel-k/pkg/util"
 
 	"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"
 )
@@ -62,9 +62,11 @@ func (action *initializeAction) Handle(integration *v1alpha1.Integration) error
 		var defaultReplicas int32 = 1
 		target.Spec.Replicas = &defaultReplicas
 	}
+	// extract metadata
+	meta := metadata.Extract(target.Spec.Source)
+
 	// set the correct language
-	language := discover.Language(target.Spec.Source)
-	target.Spec.Source.Language = language
+	target.Spec.Source.Language = meta.Language
 
 	if !util.StringSliceExists(target.Spec.Dependencies, "camel:core") {
 		target.Spec.Dependencies = append(target.Spec.Dependencies, "camel:core")
@@ -76,8 +78,7 @@ func (action *initializeAction) Handle(integration *v1alpha1.Integration) error
 		target.Spec.DependenciesAutoDiscovery = &autoDiscoveryDependencies
 	}
 	if *target.Spec.DependenciesAutoDiscovery {
-		discovered := discover.Dependencies(target.Spec.Source)
-		target.Spec.Dependencies = action.mergeDependencies(target.Spec.Dependencies, discovered)
+		target.Spec.Dependencies = action.mergeDependencies(target.Spec.Dependencies, meta.Dependencies)
 	}
 	// sort the dependencies to get always the same list if they don't change
 	sort.Strings(target.Spec.Dependencies)
diff --git a/pkg/trait/catalog.go b/pkg/trait/catalog.go
index 4d94477..049b1a6 100644
--- a/pkg/trait/catalog.go
+++ b/pkg/trait/catalog.go
@@ -20,63 +20,126 @@ package trait
 import (
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
+	"github.com/fatih/structs"
+	"reflect"
+	"strings"
 )
 
-var (
-	tBase    = newBaseTrait()
-	tService = newServiceTrait()
-	tRoute   = newRouteTrait()
-	tOwner   = newOwnerTrait()
-)
+// Catalog collects all information about traits in one place
+type Catalog struct {
+	tDeployment ITrait
+	tService    ITrait
+	tRoute      ITrait
+	tOwner      ITrait
+}
+
+// NewCatalog creates a new trait Catalog
+func NewCatalog() *Catalog {
+	return &Catalog{
+		tDeployment: newDeploymentTrait(),
+		tService:    newServiceTrait(),
+		tRoute:      newRouteTrait(),
+		tOwner:      newOwnerTrait(),
+	}
+}
+
+func (c *Catalog) allTraits() []ITrait {
+	return []ITrait{
+		c.tDeployment,
+		c.tService,
+		c.tRoute,
+		c.tOwner,
+	}
+}
 
-// customizersFor returns a Catalog for the given integration details
-func customizersFor(environment *environment) customizer {
+func (c *Catalog) traitsFor(environment *environment) []ITrait {
 	switch environment.Platform.Spec.Cluster {
 	case v1alpha1.IntegrationPlatformClusterOpenShift:
-		return compose(
-			&tBase,
-			&tService,
-			&tRoute,
-			&tOwner,
-		)
+		return []ITrait{
+			c.tDeployment,
+			c.tService,
+			c.tRoute,
+			c.tOwner,
+		}
 	case v1alpha1.IntegrationPlatformClusterKubernetes:
-		return compose(
-			&tBase,
-			&tService,
-			&tOwner,
-		)
+		return []ITrait{
+			c.tDeployment,
+			c.tService,
+			c.tOwner,
+		}
 		// case Knative: ...
 	}
 	return nil
 }
 
-func compose(traits ...customizer) customizer {
-	return &chainedCustomizer{
-		customizers: traits,
+func (c *Catalog) customize(environment *environment, resources *kubernetes.Collection) error {
+	c.configure(environment)
+	traits := c.traitsFor(environment)
+	for _, trait := range traits {
+		if trait.IsAuto() {
+			if err := trait.autoconfigure(environment, resources); err != nil {
+				return err
+			}
+		}
+		if trait.IsEnabled() {
+			if err := trait.customize(environment, resources); err != nil {
+				return err
+			}
+			environment.ExecutedTraits = append(environment.ExecutedTraits, trait.ID())
+		}
 	}
+	return nil
 }
 
-// -------------------------------------------
+// GetTrait returns the trait with the given ID
+func (c *Catalog) GetTrait(id string) ITrait {
+	for _, t := range c.allTraits() {
+		if t.ID() == ID(id) {
+			return t
+		}
+	}
+	return nil
+}
 
-type chainedCustomizer struct {
-	customizers []customizer
+func (c *Catalog) configure(env *environment) {
+	if env.Integration == nil || env.Integration.Spec.Traits == nil {
+		return
+	}
+	for id, traitSpec := range env.Integration.Spec.Traits {
+		catTrait := c.GetTrait(id)
+		if catTrait != nil {
+			traitSpec.Decode(catTrait)
+		}
+	}
 }
 
-func (c *chainedCustomizer) ID() ID {
-	return ID("")
+// ComputeTraitsProperties returns all key/value configuration properties that can be used to configure traits
+func (c *Catalog) ComputeTraitsProperties() []string {
+	results := make([]string, 0)
+	for _, trait := range c.allTraits() {
+		c.processFields(structs.Fields(trait), func(name string) {
+			results = append(results, string(trait.ID())+"."+name)
+		})
+	}
+
+	return results
 }
 
-func (c *chainedCustomizer) customize(environment *environment, resources *kubernetes.Collection) (bool, error) {
-	atLeastOne := false
-	for _, custom := range c.customizers {
-		if environment.isEnabled(custom.ID()) || environment.isAutoDetectionMode(custom.ID()) {
-			if done, err := custom.customize(environment, resources); err != nil {
-				return false, err
-			} else if done && custom.ID() != "" {
-				environment.ExecutedCustomizers = append(environment.ExecutedCustomizers, custom.ID())
-				atLeastOne = atLeastOne || done
-			}
+func (c *Catalog) processFields(fields []*structs.Field, processor func(string)) {
+	for _, f := range fields {
+		if f.IsEmbedded() && f.IsExported() && f.Kind() == reflect.Struct {
+			c.processFields(f.Fields(), processor)
+		}
+
+		if f.IsEmbedded() {
+			continue
+		}
+
+		property := f.Tag("property")
+
+		if property != "" {
+			items := strings.Split(property, ",")
+			processor(items[0])
 		}
 	}
-	return atLeastOne, nil
 }
diff --git a/pkg/trait/base.go b/pkg/trait/deployment.go
similarity index 93%
rename from pkg/trait/base.go
rename to pkg/trait/deployment.go
index f146ce0..4225b99 100644
--- a/pkg/trait/base.go
+++ b/pkg/trait/deployment.go
@@ -28,20 +28,20 @@ import (
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 )
 
-type baseTrait struct {
-	Trait
+type deploymentTrait struct {
+	BaseTrait `property:",squash"`
 }
 
-func newBaseTrait() baseTrait {
-	return baseTrait{
-		Trait: NewTraitWithID("base"),
+func newDeploymentTrait() *deploymentTrait {
+	return &deploymentTrait{
+		BaseTrait: newBaseTrait("deployment"),
 	}
 }
 
-func (d *baseTrait) customize(environment *environment, resources *kubernetes.Collection) (bool, error) {
+func (d *deploymentTrait) customize(environment *environment, resources *kubernetes.Collection) error {
 	resources.Add(d.getConfigMapFor(environment))
 	resources.Add(d.getDeploymentFor(environment))
-	return true, nil
+	return nil
 }
 
 // **********************************
@@ -50,7 +50,7 @@ func (d *baseTrait) customize(environment *environment, resources *kubernetes.Co
 //
 // **********************************
 
-func (*baseTrait) getConfigMapFor(e *environment) *corev1.ConfigMap {
+func (*deploymentTrait) getConfigMapFor(e *environment) *corev1.ConfigMap {
 	// combine properties of integration with context, integration
 	// properties have the priority
 	properties := CombineConfigurationAsMap("property", e.Context, e.Integration)
@@ -86,7 +86,7 @@ func (*baseTrait) getConfigMapFor(e *environment) *corev1.ConfigMap {
 //
 // **********************************
 
-func (*baseTrait) getDeploymentFor(e *environment) *appsv1.Deployment {
+func (*deploymentTrait) getDeploymentFor(e *environment) *appsv1.Deployment {
 	sourceName := strings.TrimPrefix(e.Integration.Spec.Source.Name, "/")
 
 	// combine environment of integration with context, integration
diff --git a/pkg/trait/owner.go b/pkg/trait/owner.go
index a47f3c9..701b219 100644
--- a/pkg/trait/owner.go
+++ b/pkg/trait/owner.go
@@ -24,16 +24,16 @@ import (
 
 // ownerTrait ensures that all created resources belong to the integration being created
 type ownerTrait struct {
-	Trait
+	BaseTrait `property:",squash"`
 }
 
-func newOwnerTrait() ownerTrait {
-	return ownerTrait{
-		Trait: NewTraitWithID("owner"),
+func newOwnerTrait() *ownerTrait {
+	return &ownerTrait{
+		BaseTrait: newBaseTrait("owner"),
 	}
 }
 
-func (*ownerTrait) customize(e *environment, resources *kubernetes.Collection) (bool, error) {
+func (*ownerTrait) customize(e *environment, resources *kubernetes.Collection) error {
 	controller := true
 	blockOwnerDeletion := true
 	resources.VisitMetaObject(func(res metav1.Object) {
@@ -49,5 +49,5 @@ func (*ownerTrait) customize(e *environment, resources *kubernetes.Collection) (
 		}
 		res.SetOwnerReferences(references)
 	})
-	return true, nil
+	return nil
 }
diff --git a/pkg/trait/route.go b/pkg/trait/route.go
index 725b1a9..1553bae 100644
--- a/pkg/trait/route.go
+++ b/pkg/trait/route.go
@@ -26,31 +26,41 @@ import (
 )
 
 type routeTrait struct {
-	Trait
+	BaseTrait `property:",squash"`
 }
 
-func newRouteTrait() routeTrait {
-	return routeTrait{
-		Trait: NewTraitWithID("route"),
+func newRouteTrait() *routeTrait {
+	return &routeTrait{
+		BaseTrait: newBaseTrait("route"),
 	}
 }
 
-func (e *routeTrait) customize(environment *environment, resources *kubernetes.Collection) (bool, error) {
-	var service *corev1.Service
+func (e *routeTrait) autoconfigure(environment *environment, resources *kubernetes.Collection) error {
+	if e.Enabled == nil {
+		hasService := e.getTargetService(environment, resources) != nil
+		e.Enabled = &hasService
+	}
+	return nil
+}
+
+func (e *routeTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+	service := e.getTargetService(environment, resources)
+	if service != nil {
+		resources.Add(e.getRouteFor(environment, service))
+	}
+
+	return nil
+}
+
+func (*routeTrait) getTargetService(e *environment, resources *kubernetes.Collection) (service *corev1.Service) {
 	resources.VisitService(func(s *corev1.Service) {
 		if s.ObjectMeta.Labels != nil {
-			if intName, ok := s.ObjectMeta.Labels["camel.apache.org/integration"]; ok && intName == environment.Integration.Name {
+			if intName, ok := s.ObjectMeta.Labels["camel.apache.org/integration"]; ok && intName == e.Integration.Name {
 				service = s
 			}
 		}
 	})
-
-	if service != nil {
-		resources.Add(e.getRouteFor(environment, service))
-		return true, nil
-	}
-
-	return false, nil
+	return
 }
 
 func (*routeTrait) getRouteFor(e *environment, service *corev1.Service) *routev1.Route {
diff --git a/pkg/trait/service.go b/pkg/trait/service.go
index 2afe06c..619226b 100644
--- a/pkg/trait/service.go
+++ b/pkg/trait/service.go
@@ -35,36 +35,36 @@ var webComponents = map[string]bool{
 }
 
 type serviceTrait struct {
-	Trait
+	BaseTrait `property:",squash"`
 
 	Port int `property:"port"`
 }
 
-func newServiceTrait() serviceTrait {
-	return serviceTrait{
-		Trait: NewTraitWithID("service"),
-		Port:  8080,
+func newServiceTrait() *serviceTrait {
+	return &serviceTrait{
+		BaseTrait: newBaseTrait("service"),
+		Port:      8080,
 	}
 }
 
-func (s *serviceTrait) customize(environment *environment, resources *kubernetes.Collection) (bool, error) {
-	if environment.isAutoDetectionMode(s.ID()) && !s.requiresService(environment) {
-		return false, nil
+func (s *serviceTrait) autoconfigure(environment *environment, resources *kubernetes.Collection) error {
+	if s.Enabled == nil {
+		required := s.requiresService(environment)
+		s.Enabled = &required
 	}
-	svc, err := s.getServiceFor(environment)
-	if err != nil {
-		return false, err
+	return nil
+}
+
+func (s *serviceTrait) customize(environment *environment, resources *kubernetes.Collection) (err error) {
+	var svc *corev1.Service
+	if svc, err = s.getServiceFor(environment); err != nil {
+		return err
 	}
 	resources.Add(svc)
-	return true, nil
+	return nil
 }
 
 func (s *serviceTrait) getServiceFor(e *environment) (*corev1.Service, error) {
-	t := newServiceTrait()
-	if _, err := e.getTrait(s.ID(), &t); err != nil {
-		return nil, err
-	}
-
 	svc := corev1.Service{
 		TypeMeta: metav1.TypeMeta{
 			Kind:       "Service",
@@ -83,7 +83,7 @@ func (s *serviceTrait) getServiceFor(e *environment) (*corev1.Service, error) {
 					Name:       "http",
 					Port:       80,
 					Protocol:   corev1.ProtocolTCP,
-					TargetPort: intstr.FromInt(t.Port),
+					TargetPort: intstr.FromInt(s.Port),
 				},
 			},
 			Selector: map[string]string{
diff --git a/pkg/trait/trait.go b/pkg/trait/trait.go
index 8b317ab..bdb5b0e 100644
--- a/pkg/trait/trait.go
+++ b/pkg/trait/trait.go
@@ -32,9 +32,9 @@ func ComputeDeployment(integration *v1alpha1.Integration) ([]runtime.Object, err
 		return nil, err
 	}
 	resources := kubernetes.NewCollection()
-	customizers := customizersFor(environment)
+	catalog := NewCatalog()
 	// invoke the trait framework to determine the needed resources
-	if _, err = customizers.customize(environment, resources); err != nil {
+	if err := catalog.customize(environment, resources); err != nil {
 		return nil, errors.Wrap(err, "error during trait customization")
 	}
 	return resources.Items(), nil
@@ -52,9 +52,10 @@ func newEnvironment(integration *v1alpha1.Integration) (*environment, error) {
 	}
 
 	return &environment{
-		Platform:            pl,
-		Context:             ctx,
-		Integration:         integration,
-		ExecutedCustomizers: make([]ID, 0),
+		Platform:       pl,
+		Context:        ctx,
+		Integration:    integration,
+		ExecutedTraits: make([]ID, 0),
 	}, nil
 }
+
diff --git a/pkg/trait/trait_test.go b/pkg/trait/trait_test.go
index f447f84..6631c8c 100644
--- a/pkg/trait/trait_test.go
+++ b/pkg/trait/trait_test.go
@@ -32,10 +32,11 @@ import (
 func TestOpenShiftTraits(t *testing.T) {
 	env := createTestEnv(v1alpha1.IntegrationPlatformClusterOpenShift, "camel:core")
 	res := processTestEnv(t, env)
-	assert.Contains(t, env.ExecutedCustomizers, ID("base"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("service"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("route"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("owner"))
+	assert.NotEmpty(t, env.ExecutedTraits)
+	assert.Contains(t, env.ExecutedTraits, ID("deployment"))
+	assert.NotContains(t, env.ExecutedTraits, ID("service"))
+	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"
 	}))
@@ -47,10 +48,10 @@ func TestOpenShiftTraits(t *testing.T) {
 func TestOpenShiftTraitsWithWeb(t *testing.T) {
 	env := createTestEnv(v1alpha1.IntegrationPlatformClusterOpenShift, "camel:core", "camel:undertow")
 	res := processTestEnv(t, env)
-	assert.Contains(t, env.ExecutedCustomizers, ID("base"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("service"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("route"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("owner"))
+	assert.Contains(t, env.ExecutedTraits, ID("deployment"))
+	assert.Contains(t, env.ExecutedTraits, ID("service"))
+	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"
 	}))
@@ -74,8 +75,8 @@ func TestOpenShiftTraitsWithWebAndConfig(t *testing.T) {
 		},
 	}
 	res := processTestEnv(t, env)
-	assert.Contains(t, env.ExecutedCustomizers, ID("service"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("route"))
+	assert.Contains(t, env.ExecutedTraits, ID("service"))
+	assert.Contains(t, env.ExecutedTraits, ID("route"))
 	assert.NotNil(t, res.GetService(func(svc *corev1.Service) bool {
 		return svc.Name == "test" && svc.Spec.Ports[0].TargetPort.IntVal == int32(7071)
 	}))
@@ -91,8 +92,8 @@ func TestOpenShiftTraitsWithWebAndDisabledTrait(t *testing.T) {
 		},
 	}
 	res := processTestEnv(t, env)
-	assert.NotContains(t, env.ExecutedCustomizers, ID("service"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("route")) // No route without service
+	assert.NotContains(t, env.ExecutedTraits, ID("service"))
+	assert.NotContains(t, env.ExecutedTraits, ID("route")) // No route without service
 	assert.Nil(t, res.GetService(func(svc *corev1.Service) bool {
 		return true
 	}))
@@ -101,10 +102,10 @@ func TestOpenShiftTraitsWithWebAndDisabledTrait(t *testing.T) {
 func TestKubernetesTraits(t *testing.T) {
 	env := createTestEnv(v1alpha1.IntegrationPlatformClusterKubernetes, "camel:core")
 	res := processTestEnv(t, env)
-	assert.Contains(t, env.ExecutedCustomizers, ID("base"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("service"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("route"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("owner"))
+	assert.Contains(t, env.ExecutedTraits, ID("deployment"))
+	assert.NotContains(t, env.ExecutedTraits, ID("service"))
+	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"
 	}))
@@ -116,10 +117,10 @@ func TestKubernetesTraits(t *testing.T) {
 func TestKubernetesTraitsWithWeb(t *testing.T) {
 	env := createTestEnv(v1alpha1.IntegrationPlatformClusterKubernetes, "camel:core", "camel:servlet")
 	res := processTestEnv(t, env)
-	assert.Contains(t, env.ExecutedCustomizers, ID("base"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("service"))
-	assert.NotContains(t, env.ExecutedCustomizers, ID("route"))
-	assert.Contains(t, env.ExecutedCustomizers, ID("owner"))
+	assert.Contains(t, env.ExecutedTraits, ID("deployment"))
+	assert.Contains(t, env.ExecutedTraits, ID("service"))
+	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"
 	}))
@@ -134,26 +135,27 @@ func TestKubernetesTraitsWithWeb(t *testing.T) {
 func TestTraitDecode(t *testing.T) {
 	env := createTestEnv(v1alpha1.IntegrationPlatformClusterOpenShift)
 	env.Integration.Spec.Traits = make(map[string]v1alpha1.IntegrationTraitSpec)
-	env.Integration.Spec.Traits["service"] = v1alpha1.IntegrationTraitSpec{
+	svcTrait := v1alpha1.IntegrationTraitSpec{
 		Configuration: map[string]string{
 			"enabled": "false",
 			"port":    "7071",
+			"cippa":   "lippa",
 		},
 	}
+	env.Integration.Spec.Traits["service"] = svcTrait
 
 	svc := newServiceTrait()
-	ok, err := env.getTrait(ID("service"), &svc)
+	err := svcTrait.Decode(svc)
 
 	assert.Nil(t, err)
-	assert.True(t, ok)
 	assert.Equal(t, 7071, svc.Port)
-	assert.Equal(t, true, svc.Enabled)
+	assert.Equal(t, false, svc.IsEnabled())
 }
 
 func processTestEnv(t *testing.T, env *environment) *kubernetes.Collection {
 	resources := kubernetes.NewCollection()
-	customizers := customizersFor(env)
-	_, err := customizers.customize(env, resources)
+	catalog := NewCatalog()
+	err := catalog.customize(env, resources)
 	assert.Nil(t, err)
 	return resources
 }
@@ -175,6 +177,6 @@ func createTestEnv(cluster v1alpha1.IntegrationPlatformCluster, dependencies ...
 				Cluster: cluster,
 			},
 		},
-		ExecutedCustomizers: make([]ID, 0),
+		ExecutedTraits: make([]ID, 0),
 	}
 }
diff --git a/pkg/trait/types.go b/pkg/trait/types.go
index a825c6f..23aa6a2 100644
--- a/pkg/trait/types.go
+++ b/pkg/trait/types.go
@@ -18,14 +18,8 @@ limitations under the License.
 package trait
 
 import (
-	"fmt"
-	"reflect"
-
-	"github.com/sirupsen/logrus"
-
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
-	"github.com/pkg/errors"
 )
 
 // Identifiable represent an identifiable type
@@ -36,90 +30,69 @@ type Identifiable interface {
 // ID uniquely identifies a trait
 type ID string
 
-// Trait --
-type Trait struct {
+// ITrait TODO rename
+type ITrait interface {
 	Identifiable
-
-	id      ID
-	Enabled bool `property:"enabled"`
+	// enabled tells if the trait is enabled
+	IsEnabled() bool
+	// auto determine if the trait should be configured automatically
+	IsAuto() bool
+	// autoconfigure is called before any customization to ensure the trait is fully configured
+	autoconfigure(environment *environment, resources *kubernetes.Collection) error
+	// customize executes the trait customization on the resources and return true if the resources have been changed
+	customize(environment *environment, resources *kubernetes.Collection) error
 }
 
-// ID returns the trait ID
-func (trait *Trait) ID() ID {
-	return trait.id
-}
+/* Base trait */
 
-// NewTrait creates a new trait with defaults
-func NewTrait() Trait {
-	return Trait{
-		Enabled: true,
-	}
+// BaseTrait is the root trait with noop implementations for hooks
+type BaseTrait struct {
+	id      ID
+	Enabled *bool `property:"enabled"`
+	Auto    *bool `property:"auto"`
 }
 
-// NewTraitWithID creates a new trait with defaults and given ID
-func NewTraitWithID(traitID ID) Trait {
-	return Trait{
-		id:      traitID,
-		Enabled: true,
+func newBaseTrait(id string) BaseTrait {
+	return BaseTrait{
+		id: ID(id),
 	}
 }
 
-// A Customizer performs customization of the deployed objects
-type customizer interface {
-	Identifiable
-	// Customize executes the trait customization on the resources and return true if the resources have been changed
-	customize(environment *environment, resources *kubernetes.Collection) (bool, error)
+// ID returns the identifier of the trait
+func (trait *BaseTrait) ID() ID {
+	return trait.id
 }
 
-// A environment provides the context where the trait is executed
-type environment struct {
-	Platform            *v1alpha1.IntegrationPlatform
-	Context             *v1alpha1.IntegrationContext
-	Integration         *v1alpha1.Integration
-	ExecutedCustomizers []ID
+// IsAuto determines if we should apply automatic configuration
+func (trait *BaseTrait) IsAuto() bool {
+	if trait.Auto == nil {
+		return true
+	}
+	return *trait.Auto
 }
 
-func (e *environment) getTrait(traitID ID, target interface{}) (bool, error) {
-	if spec := e.getTraitSpec(traitID); spec != nil {
-		err := spec.Decode(&target)
-		if err != nil {
-			return false, errors.Wrap(err, fmt.Sprintf("unable to convert trait %s to the target struct %s", traitID, reflect.TypeOf(target).Name()))
-		}
-
-		return true, nil
+// IsEnabled is used to determine if the trait needs to be executed
+func (trait *BaseTrait) IsEnabled() bool {
+	if trait.Enabled == nil {
+		return true
 	}
-
-	return false, nil
+	return *trait.Enabled
 }
 
-func (e *environment) getTraitSpec(traitID ID) *v1alpha1.IntegrationTraitSpec {
-	if e.Integration.Spec.Traits == nil {
-		return nil
-	}
-	if conf, ok := e.Integration.Spec.Traits[string(traitID)]; ok {
-		return &conf
-	}
+func (trait *BaseTrait) autoconfigure(environment *environment, resources *kubernetes.Collection) error {
 	return nil
 }
 
-func (e *environment) isEnabled(traitID ID) bool {
-	t := NewTrait()
-	if _, err := e.getTrait(traitID, &t); err != nil {
-		logrus.Panic(err)
-	}
-
-	return t.Enabled
+func (trait *BaseTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+	return nil
 }
 
-func (e *environment) isAutoDetectionMode(traitID ID) bool {
-	spec := e.getTraitSpec(traitID)
-	if spec == nil {
-		return true
-	}
+/* Environment */
 
-	if spec.Configuration == nil {
-		return true
-	}
-
-	return spec.Configuration["enabled"] == ""
+// A environment provides the context where the trait is executed
+type environment struct {
+	Platform       *v1alpha1.IntegrationPlatform
+	Context        *v1alpha1.IntegrationContext
+	Integration    *v1alpha1.Integration
+	ExecutedTraits []ID
 }
diff --git a/pkg/trait/util.go b/pkg/trait/util.go
index 577bdb7..5b280a9 100644
--- a/pkg/trait/util.go
+++ b/pkg/trait/util.go
@@ -19,11 +19,9 @@ package trait
 
 import (
 	"fmt"
-	"reflect"
 	"strings"
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
-	"github.com/fatih/structs"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
 	"github.com/pkg/errors"
 	"k8s.io/api/core/v1"
@@ -119,32 +117,3 @@ func CombineConfigurationAsSlice(configurationType string, context *v1alpha1.Int
 	return keys
 }
 
-// ComputeTraitsProperties --
-func ComputeTraitsProperties() []string {
-	results := make([]string, 0)
-
-	processFields(structs.Fields(tService), func(name string) {
-		results = append(results, string(tService.ID())+"."+name)
-	})
-
-	return results
-}
-
-func processFields(fields []*structs.Field, processor func(string)) {
-	for _, f := range fields {
-		if f.IsEmbedded() && f.IsExported() && f.Kind() == reflect.Struct {
-			processFields(f.Fields(), processor)
-		}
-
-		if f.IsEmbedded() {
-			continue
-		}
-
-		property := f.Tag("property")
-
-		if property != "" {
-			items := strings.Split(property, ",")
-			processor(items[0])
-		}
-	}
-}


[camel-k] 03/06: Fix roles for ingress

Posted by lb...@apache.org.
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 74c1f49f995782766be3098922af5c15d6709303
Author: nferraro <ni...@gmail.com>
AuthorDate: Mon Nov 5 15:06:59 2018 +0100

    Fix roles for ingress
---
 deploy/operator-role-kubernetes.yaml | 13 +++++++++++++
 deploy/resources.go                  | 13 +++++++++++++
 2 files changed, 26 insertions(+)

diff --git a/deploy/operator-role-kubernetes.yaml b/deploy/operator-role-kubernetes.yaml
index d3865a8..c0c08cb 100644
--- a/deploy/operator-role-kubernetes.yaml
+++ b/deploy/operator-role-kubernetes.yaml
@@ -61,3 +61,16 @@ rules:
   - get
   - list
   - watch
+- apiGroups:
+  - extensions
+  resources:
+  - ingresses
+  verbs:
+  - create
+  - delete
+  - deletecollection
+  - get
+  - list
+  - patch
+  - update
+  - watch
diff --git a/deploy/resources.go b/deploy/resources.go
index f370a58..4fcbce0 100644
--- a/deploy/resources.go
+++ b/deploy/resources.go
@@ -2373,6 +2373,19 @@ rules:
   - get
   - list
   - watch
+- apiGroups:
+  - extensions
+  resources:
+  - ingresses
+  verbs:
+  - create
+  - delete
+  - deletecollection
+  - get
+  - list
+  - patch
+  - update
+  - watch
 
 `
 	Resources["operator-role-openshift.yaml"] =


[camel-k] 06/06: Moved dependencies to a proper trait

Posted by lb...@apache.org.
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 27e03854c670f01804cd7023db54afdcd0b40514
Author: nferraro <ni...@gmail.com>
AuthorDate: Mon Nov 5 18:33:35 2018 +0100

    Moved dependencies to a proper trait
---
 deploy/cr-example.yaml                           |  2 -
 deploy/resources.go                              |  2 -
 docs/traits.adoc                                 |  7 +++
 pkg/apis/camel/v1alpha1/types.go                 | 13 +++--
 pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go |  5 --
 pkg/client/cmd/run.go                            | 53 +++++++-----------
 pkg/stub/action/integration/deploy.go            |  2 +-
 pkg/stub/action/integration/initialize.go        | 41 +++-----------
 pkg/trait/catalog.go                             | 61 ++++++++++++++-------
 pkg/trait/dependencies.go                        | 69 ++++++++++++++++++++++++
 pkg/trait/deployment.go                          |  2 +-
 pkg/trait/ingress.go                             |  2 +-
 pkg/trait/owner.go                               |  2 +-
 pkg/trait/route.go                               |  2 +-
 pkg/trait/service.go                             |  2 +-
 pkg/trait/trait.go                               | 23 ++++++--
 pkg/trait/trait_test.go                          |  2 +-
 pkg/trait/types.go                               | 16 ++++--
 pkg/trait/util.go                                |  3 +-
 19 files changed, 186 insertions(+), 123 deletions(-)

diff --git a/deploy/cr-example.yaml b/deploy/cr-example.yaml
index c2851c7..253c402 100644
--- a/deploy/cr-example.yaml
+++ b/deploy/cr-example.yaml
@@ -3,8 +3,6 @@ kind: Integration
 metadata:
   name: example
 spec:
-  dependencies:
-    - camel:groovy
   source:
     content: |-
       // This is Camel K Groovy example route
diff --git a/deploy/resources.go b/deploy/resources.go
index b3e753e..de96acf 100644
--- a/deploy/resources.go
+++ b/deploy/resources.go
@@ -2188,8 +2188,6 @@ kind: Integration
 metadata:
   name: example
 spec:
-  dependencies:
-    - camel:groovy
   source:
     content: |-
       // This is Camel K Groovy example route
diff --git a/docs/traits.adoc b/docs/traits.adoc
index ff2ec93..27f90b2 100644
--- a/docs/traits.adoc
+++ b/docs/traits.adoc
@@ -55,6 +55,13 @@ The following is a list of common traits that can be configured by the end users
 |=======================
 | Trait      | Profiles 				| Description
 
+| dependencies
+| Kubernetes, OpenShift
+| Automatically adds dependencies required by the Camel routes by inspecting the user code.
+  +
+  +
+  It's enabled by default.
+
 | service
 | Kubernetes, OpenShift
 | Exposes the integration with a Service resource so that it can be accessed by other applications (or integrations) in the same namespace.
diff --git a/pkg/apis/camel/v1alpha1/types.go b/pkg/apis/camel/v1alpha1/types.go
index 06cccce..b79a0c5 100644
--- a/pkg/apis/camel/v1alpha1/types.go
+++ b/pkg/apis/camel/v1alpha1/types.go
@@ -49,13 +49,12 @@ 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"`
-	Traits                    map[string]IntegrationTraitSpec `json:"traits,omitempty"`
-	DependenciesAutoDiscovery *bool                           `json:"dependenciesAutoDiscovery,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"`
+	Traits        map[string]IntegrationTraitSpec `json:"traits,omitempty"`
+	Configuration []ConfigurationSpec             `json:"configuration,omitempty"`
 }
 
 // SourceSpec --
diff --git a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
index 1c13379..af2406c 100644
--- a/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
+++ b/pkg/apis/camel/v1alpha1/zz_generated.deepcopy.go
@@ -341,11 +341,6 @@ func (in *IntegrationSpec) DeepCopyInto(out *IntegrationSpec) {
 			(*out)[key] = *val.DeepCopy()
 		}
 	}
-	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/run.go b/pkg/client/cmd/run.go
index 3c59c84..6a52e03 100644
--- a/pkg/client/cmd/run.go
+++ b/pkg/client/cmd/run.go
@@ -63,7 +63,7 @@ func newCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 	}
 
 	cmd.Flags().StringVarP(&options.Language, "language", "l", "", "Programming Language used to write the file")
-	cmd.Flags().StringVarP(&options.Runtime, "runtime", "r", "jvm", "Runtime used by the integration")
+	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")
 	cmd.Flags().BoolVarP(&options.Wait, "wait", "w", false, "Waits for the integration to be running")
@@ -74,7 +74,6 @@ 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")
 	cmd.Flags().StringSliceVarP(&options.Traits, "trait", "t", nil, "Configure a trait. E.g. \"-t service.enabled=false\"")
 	cmd.Flags().StringSliceVar(&options.LoggingLevels, "logging-level", nil, "Configure the logging level. E.g. \"--logging-level org.apache.camel=DEBUG\"")
 
@@ -86,21 +85,20 @@ func newCmdRun(rootCmdOptions *RootCmdOptions) *cobra.Command {
 
 type runCmdOptions struct {
 	*RootCmdOptions
-	IntegrationContext        string
-	Language                  string
-	Runtime                   string
-	IntegrationName           string
-	Dependencies              []string
-	Properties                []string
-	ConfigMaps                []string
-	Secrets                   []string
-	Wait                      bool
-	Logs                      bool
-	Sync                      bool
-	Dev                       bool
-	DependenciesAutoDiscovery bool
-	Traits                    []string
-	LoggingLevels             []string
+	IntegrationContext string
+	Language           string
+	Runtime            string
+	IntegrationName    string
+	Dependencies       []string
+	Properties         []string
+	ConfigMaps         []string
+	Secrets            []string
+	Wait               bool
+	Logs               bool
+	Sync               bool
+	Dev                bool
+	Traits             []string
+	LoggingLevels      []string
 }
 
 func (o *runCmdOptions) validateArgs(cmd *cobra.Command, args []string) error {
@@ -273,10 +271,9 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 				Content:  code,
 				Language: v1alpha1.Language(o.Language),
 			},
-			Dependencies:              make([]string, 0, len(o.Dependencies)),
-			DependenciesAutoDiscovery: &o.DependenciesAutoDiscovery,
-			Context:                   o.IntegrationContext,
-			Configuration:             make([]v1alpha1.ConfigurationSpec, 0),
+			Dependencies:  make([]string, 0, len(o.Dependencies)),
+			Context:       o.IntegrationContext,
+			Configuration: make([]v1alpha1.ConfigurationSpec, 0),
 		},
 	}
 
@@ -290,24 +287,10 @@ func (o *runCmdOptions) updateIntegrationCode(filename string) (*v1alpha1.Integr
 		}
 	}
 
-	if o.Language == "groovy" || strings.HasSuffix(filename, ".groovy") {
-		util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:groovy")
-	}
-	if o.Language == "kotlin" || strings.HasSuffix(filename, ".kts") {
-		util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:kotlin")
-	}
-
-	// jvm runtime required by default
-	util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:jvm")
-
 	if o.Runtime != "" {
 		util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:"+o.Runtime)
 	}
 
-	switch o.Runtime {
-
-	}
-
 	for _, item := range o.Properties {
 		integration.Spec.Configuration = append(integration.Spec.Configuration, v1alpha1.ConfigurationSpec{
 			Type:  "property",
diff --git a/pkg/stub/action/integration/deploy.go b/pkg/stub/action/integration/deploy.go
index 2c814bc..d666a7b 100644
--- a/pkg/stub/action/integration/deploy.go
+++ b/pkg/stub/action/integration/deploy.go
@@ -42,7 +42,7 @@ func (action *deployAction) CanHandle(integration *v1alpha1.Integration) bool {
 }
 
 func (action *deployAction) Handle(integration *v1alpha1.Integration) error {
-	resources, err := trait.ComputeDeployment(integration)
+	resources, err := trait.BeforeDeployment(integration)
 	if err != nil {
 		return err
 	}
diff --git a/pkg/stub/action/integration/initialize.go b/pkg/stub/action/integration/initialize.go
index 3abf51f..949df34 100644
--- a/pkg/stub/action/integration/initialize.go
+++ b/pkg/stub/action/integration/initialize.go
@@ -18,16 +18,13 @@ limitations under the License.
 package integration
 
 import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/metadata"
 	"github.com/apache/camel-k/pkg/platform"
-	"github.com/sirupsen/logrus"
-	"sort"
-
-	"github.com/apache/camel-k/pkg/util"
-
-	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/trait"
 	"github.com/apache/camel-k/pkg/util/digest"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
+	"github.com/sirupsen/logrus"
 )
 
 // NewInitializeAction creates a new inititialize action
@@ -64,42 +61,16 @@ func (action *initializeAction) Handle(integration *v1alpha1.Integration) error
 	}
 	// extract metadata
 	meta := metadata.Extract(target.Spec.Source)
-
-	// set the correct language
 	target.Spec.Source.Language = meta.Language
 
-	if !util.StringSliceExists(target.Spec.Dependencies, "camel:core") {
-		target.Spec.Dependencies = append(target.Spec.Dependencies, "camel:core")
+	// execute custom initialization
+	if err := trait.BeforeInit(target); err != nil {
+		return err
 	}
 
-	// discover dependencies
-	if target.Spec.DependenciesAutoDiscovery == nil {
-		var autoDiscoveryDependencies = true
-		target.Spec.DependenciesAutoDiscovery = &autoDiscoveryDependencies
-	}
-	if *target.Spec.DependenciesAutoDiscovery {
-		target.Spec.Dependencies = action.mergeDependencies(target.Spec.Dependencies, meta.Dependencies)
-	}
-	// sort the dependencies to get always the same list if they don't change
-	sort.Strings(target.Spec.Dependencies)
 	// update the status
 	logrus.Info("Integration ", target.Name, " transitioning to state ", v1alpha1.IntegrationPhaseBuilding)
 	target.Status.Phase = v1alpha1.IntegrationPhaseBuilding
 	target.Status.Digest = digest.ComputeForIntegration(integration)
 	return sdk.Update(target)
 }
-
-func (action *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/trait/catalog.go b/pkg/trait/catalog.go
index d5cf865..956ed7e 100644
--- a/pkg/trait/catalog.go
+++ b/pkg/trait/catalog.go
@@ -27,26 +27,29 @@ import (
 
 // Catalog collects all information about traits in one place
 type Catalog struct {
-	tDeployment ITrait
-	tService    ITrait
-	tRoute      ITrait
-	tIngress    ITrait
-	tOwner      ITrait
+	tDependencies Trait
+	tDeployment   Trait
+	tService      Trait
+	tRoute        Trait
+	tIngress      Trait
+	tOwner        Trait
 }
 
 // NewCatalog creates a new trait Catalog
 func NewCatalog() *Catalog {
 	return &Catalog{
-		tDeployment: newDeploymentTrait(),
-		tService:    newServiceTrait(),
-		tRoute:      newRouteTrait(),
-		tIngress:    newIngressTrait(),
-		tOwner:      newOwnerTrait(),
+		tDependencies: newDependenciesTrait(),
+		tDeployment:   newDeploymentTrait(),
+		tService:      newServiceTrait(),
+		tRoute:        newRouteTrait(),
+		tIngress:      newIngressTrait(),
+		tOwner:        newOwnerTrait(),
 	}
 }
 
-func (c *Catalog) allTraits() []ITrait {
-	return []ITrait{
+func (c *Catalog) allTraits() []Trait {
+	return []Trait{
+		c.tDependencies,
 		c.tDeployment,
 		c.tService,
 		c.tRoute,
@@ -55,17 +58,19 @@ func (c *Catalog) allTraits() []ITrait {
 	}
 }
 
-func (c *Catalog) traitsFor(environment *environment) []ITrait {
+func (c *Catalog) traitsFor(environment *environment) []Trait {
 	switch environment.Platform.Spec.Cluster {
 	case v1alpha1.IntegrationPlatformClusterOpenShift:
-		return []ITrait{
+		return []Trait{
+			c.tDependencies,
 			c.tDeployment,
 			c.tService,
 			c.tRoute,
 			c.tOwner,
 		}
 	case v1alpha1.IntegrationPlatformClusterKubernetes:
-		return []ITrait{
+		return []Trait{
+			c.tDependencies,
 			c.tDeployment,
 			c.tService,
 			c.tIngress,
@@ -76,7 +81,7 @@ func (c *Catalog) traitsFor(environment *environment) []ITrait {
 	return nil
 }
 
-func (c *Catalog) customize(environment *environment, resources *kubernetes.Collection) error {
+func (c *Catalog) executeBeforeDeployment(environment *environment, resources *kubernetes.Collection) error {
 	c.configure(environment)
 	traits := c.traitsFor(environment)
 	for _, trait := range traits {
@@ -86,7 +91,27 @@ func (c *Catalog) customize(environment *environment, resources *kubernetes.Coll
 			}
 		}
 		if trait.IsEnabled() {
-			if err := trait.customize(environment, resources); err != nil {
+			if err := trait.beforeDeploy(environment, resources); err != nil {
+				return err
+			}
+			environment.ExecutedTraits = append(environment.ExecutedTraits, trait.ID())
+		}
+	}
+	return nil
+}
+
+func (c *Catalog) executeBeforeInit(environment *environment, integration *v1alpha1.Integration) error {
+	c.configure(environment)
+	traits := c.traitsFor(environment)
+	resources := kubernetes.NewCollection()
+	for _, trait := range traits {
+		if trait.IsAuto() {
+			if err := trait.autoconfigure(environment, resources); err != nil {
+				return err
+			}
+		}
+		if trait.IsEnabled() {
+			if err := trait.beforeInit(environment, integration); err != nil {
 				return err
 			}
 			environment.ExecutedTraits = append(environment.ExecutedTraits, trait.ID())
@@ -96,7 +121,7 @@ func (c *Catalog) customize(environment *environment, resources *kubernetes.Coll
 }
 
 // GetTrait returns the trait with the given ID
-func (c *Catalog) GetTrait(id string) ITrait {
+func (c *Catalog) GetTrait(id string) Trait {
 	for _, t := range c.allTraits() {
 		if t.ID() == ID(id) {
 			return t
diff --git a/pkg/trait/dependencies.go b/pkg/trait/dependencies.go
new file mode 100644
index 0000000..5bdb82d
--- /dev/null
+++ b/pkg/trait/dependencies.go
@@ -0,0 +1,69 @@
+/*
+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 trait
+
+import (
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/metadata"
+	"github.com/apache/camel-k/pkg/util"
+	"sort"
+)
+
+type dependenciesTrait struct {
+	BaseTrait `property:",squash"`
+}
+
+func newDependenciesTrait() *dependenciesTrait {
+	return &dependenciesTrait{
+		BaseTrait: newBaseTrait("dependencies"),
+	}
+}
+
+func (d *dependenciesTrait) beforeInit(environment *environment, integration *v1alpha1.Integration) error {
+	meta := metadata.Extract(integration.Spec.Source)
+
+	if meta.Language == v1alpha1.LanguageGroovy {
+		util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:groovy")
+	} else if meta.Language == v1alpha1.LanguageKotlin {
+		util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:kotlin")
+	}
+
+	// jvm runtime and camel-core required by default
+	util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "runtime:jvm")
+	util.StringSliceUniqueAdd(&integration.Spec.Dependencies, "camel:core")
+
+	integration.Spec.Dependencies = d.mergeDependencies(integration.Spec.Dependencies, meta.Dependencies)
+	// sort the dependencies to get always the same list if they don't change
+	sort.Strings(integration.Spec.Dependencies)
+	return nil
+}
+
+func (d *dependenciesTrait) 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/trait/deployment.go b/pkg/trait/deployment.go
index 4225b99..41f5e93 100644
--- a/pkg/trait/deployment.go
+++ b/pkg/trait/deployment.go
@@ -38,7 +38,7 @@ func newDeploymentTrait() *deploymentTrait {
 	}
 }
 
-func (d *deploymentTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+func (d *deploymentTrait) beforeDeploy(environment *environment, resources *kubernetes.Collection) error {
 	resources.Add(d.getConfigMapFor(environment))
 	resources.Add(d.getDeploymentFor(environment))
 	return nil
diff --git a/pkg/trait/ingress.go b/pkg/trait/ingress.go
index bbd7017..c166c43 100644
--- a/pkg/trait/ingress.go
+++ b/pkg/trait/ingress.go
@@ -48,7 +48,7 @@ func (e *ingressTrait) autoconfigure(environment *environment, resources *kubern
 	return nil
 }
 
-func (e *ingressTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+func (e *ingressTrait) beforeDeploy(environment *environment, resources *kubernetes.Collection) error {
 	if e.Host == "" {
 		return errors.New("cannot apply ingress trait: no host defined")
 	}
diff --git a/pkg/trait/owner.go b/pkg/trait/owner.go
index 701b219..c0a732f 100644
--- a/pkg/trait/owner.go
+++ b/pkg/trait/owner.go
@@ -33,7 +33,7 @@ func newOwnerTrait() *ownerTrait {
 	}
 }
 
-func (*ownerTrait) customize(e *environment, resources *kubernetes.Collection) error {
+func (*ownerTrait) beforeDeploy(e *environment, resources *kubernetes.Collection) error {
 	controller := true
 	blockOwnerDeletion := true
 	resources.VisitMetaObject(func(res metav1.Object) {
diff --git a/pkg/trait/route.go b/pkg/trait/route.go
index 02299b1..fef1786 100644
--- a/pkg/trait/route.go
+++ b/pkg/trait/route.go
@@ -45,7 +45,7 @@ func (e *routeTrait) autoconfigure(environment *environment, resources *kubernet
 	return nil
 }
 
-func (e *routeTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+func (e *routeTrait) beforeDeploy(environment *environment, resources *kubernetes.Collection) error {
 	service := e.getTargetService(environment, resources)
 	if service == nil {
 		return errors.New("cannot apply route trait: no target service")
diff --git a/pkg/trait/service.go b/pkg/trait/service.go
index 619226b..dcb77df 100644
--- a/pkg/trait/service.go
+++ b/pkg/trait/service.go
@@ -55,7 +55,7 @@ func (s *serviceTrait) autoconfigure(environment *environment, resources *kubern
 	return nil
 }
 
-func (s *serviceTrait) customize(environment *environment, resources *kubernetes.Collection) (err error) {
+func (s *serviceTrait) beforeDeploy(environment *environment, resources *kubernetes.Collection) (err error) {
 	var svc *corev1.Service
 	if svc, err = s.getServiceFor(environment); err != nil {
 		return err
diff --git a/pkg/trait/trait.go b/pkg/trait/trait.go
index bdb5b0e..9239b3a 100644
--- a/pkg/trait/trait.go
+++ b/pkg/trait/trait.go
@@ -25,8 +25,8 @@ import (
 	"k8s.io/apimachinery/pkg/runtime"
 )
 
-// ComputeDeployment generates all required resources for deploying the given integration
-func ComputeDeployment(integration *v1alpha1.Integration) ([]runtime.Object, error) {
+// BeforeDeployment generates all required resources for deploying the given integration
+func BeforeDeployment(integration *v1alpha1.Integration) ([]runtime.Object, error) {
 	environment, err := newEnvironment(integration)
 	if err != nil {
 		return nil, err
@@ -34,12 +34,26 @@ func ComputeDeployment(integration *v1alpha1.Integration) ([]runtime.Object, err
 	resources := kubernetes.NewCollection()
 	catalog := NewCatalog()
 	// invoke the trait framework to determine the needed resources
-	if err := catalog.customize(environment, resources); err != nil {
-		return nil, errors.Wrap(err, "error during trait customization")
+	if err := catalog.executeBeforeDeployment(environment, resources); err != nil {
+		return nil, errors.Wrap(err, "error during trait customization before deployment")
 	}
 	return resources.Items(), nil
 }
 
+// BeforeInit executes custom initializazion of the integration
+func BeforeInit(integration *v1alpha1.Integration) error {
+	environment, err := newEnvironment(integration)
+	if err != nil {
+		return err
+	}
+	catalog := NewCatalog()
+	// invoke the trait framework to determine the needed resources
+	if err := catalog.executeBeforeInit(environment, integration); err != nil {
+		return errors.Wrap(err, "error during trait customization before init")
+	}
+	return nil
+}
+
 // newEnvironment creates a environment from the given data
 func newEnvironment(integration *v1alpha1.Integration) (*environment, error) {
 	pl, err := platform.GetCurrentPlatform(integration.Namespace)
@@ -58,4 +72,3 @@ func newEnvironment(integration *v1alpha1.Integration) (*environment, error) {
 		ExecutedTraits: make([]ID, 0),
 	}, nil
 }
-
diff --git a/pkg/trait/trait_test.go b/pkg/trait/trait_test.go
index 6631c8c..d6f11aa 100644
--- a/pkg/trait/trait_test.go
+++ b/pkg/trait/trait_test.go
@@ -155,7 +155,7 @@ func TestTraitDecode(t *testing.T) {
 func processTestEnv(t *testing.T, env *environment) *kubernetes.Collection {
 	resources := kubernetes.NewCollection()
 	catalog := NewCatalog()
-	err := catalog.customize(env, resources)
+	err := catalog.executeBeforeDeployment(env, resources)
 	assert.Nil(t, err)
 	return resources
 }
diff --git a/pkg/trait/types.go b/pkg/trait/types.go
index 23aa6a2..3cb6cf5 100644
--- a/pkg/trait/types.go
+++ b/pkg/trait/types.go
@@ -30,8 +30,8 @@ type Identifiable interface {
 // ID uniquely identifies a trait
 type ID string
 
-// ITrait TODO rename
-type ITrait interface {
+// Trait is the interface of all traits
+type Trait interface {
 	Identifiable
 	// enabled tells if the trait is enabled
 	IsEnabled() bool
@@ -39,8 +39,10 @@ type ITrait interface {
 	IsAuto() bool
 	// autoconfigure is called before any customization to ensure the trait is fully configured
 	autoconfigure(environment *environment, resources *kubernetes.Collection) error
-	// customize executes the trait customization on the resources and return true if the resources have been changed
-	customize(environment *environment, resources *kubernetes.Collection) error
+	// beforeInit executes a customization of the integration before it's built
+	beforeInit(environment *environment, integration *v1alpha1.Integration) error
+	// beforeDeploy executes a customization of the gerenated resources before they are created
+	beforeDeploy(environment *environment, resources *kubernetes.Collection) error
 }
 
 /* Base trait */
@@ -83,7 +85,11 @@ func (trait *BaseTrait) autoconfigure(environment *environment, resources *kuber
 	return nil
 }
 
-func (trait *BaseTrait) customize(environment *environment, resources *kubernetes.Collection) error {
+func (trait *BaseTrait) beforeInit(environment *environment, integration *v1alpha1.Integration) error {
+	return nil
+}
+
+func (trait *BaseTrait) beforeDeploy(environment *environment, resources *kubernetes.Collection) error {
 	return nil
 }
 
diff --git a/pkg/trait/util.go b/pkg/trait/util.go
index 5b280a9..c6194b8 100644
--- a/pkg/trait/util.go
+++ b/pkg/trait/util.go
@@ -23,14 +23,13 @@ import (
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/operator-framework/operator-sdk/pkg/sdk"
-	"github.com/pkg/errors"
 	"k8s.io/api/core/v1"
 )
 
 // GetIntegrationContext retrieves the context set on the integration
 func GetIntegrationContext(integration *v1alpha1.Integration) (*v1alpha1.IntegrationContext, error) {
 	if integration.Spec.Context == "" {
-		return nil, errors.New("no context set on the integration")
+		return nil, nil
 	}
 
 	name := integration.Spec.Context


[camel-k] 05/06: Add host property to the route trait

Posted by lb...@apache.org.
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 c028e802db4ccddefede46afa25b21aed91f63cb
Author: nferraro <ni...@gmail.com>
AuthorDate: Mon Nov 5 15:39:19 2018 +0100

    Add host property to the route trait
---
 deploy/operator-role-openshift.yaml | 8 ++++++++
 deploy/resources.go                 | 8 ++++++++
 docs/traits.adoc                    | 8 ++++++++
 pkg/trait/route.go                  | 4 +++-
 4 files changed, 27 insertions(+), 1 deletion(-)

diff --git a/deploy/operator-role-openshift.yaml b/deploy/operator-role-openshift.yaml
index 788a7cc..41a5f72 100644
--- a/deploy/operator-role-openshift.yaml
+++ b/deploy/operator-role-openshift.yaml
@@ -119,3 +119,11 @@ rules:
   - patch
   - update
   - watch
+- apiGroups:
+  - ""
+  - route.openshift.io
+  resources:
+  - routes/custom-host
+  verbs:
+  - create
+
diff --git a/deploy/resources.go b/deploy/resources.go
index 4fcbce0..b3e753e 100644
--- a/deploy/resources.go
+++ b/deploy/resources.go
@@ -2511,6 +2511,14 @@ rules:
   - patch
   - update
   - watch
+- apiGroups:
+  - ""
+  - route.openshift.io
+  resources:
+  - routes/custom-host
+  verbs:
+  - create
+
 
 `
 	Resources["operator-service-account.yaml"] =
diff --git a/docs/traits.adoc b/docs/traits.adoc
index c25e533..ff2ec93 100644
--- a/docs/traits.adoc
+++ b/docs/traits.adoc
@@ -77,6 +77,14 @@ The following is a list of common traits that can be configured by the end users
   +
   It's enabled by default whenever a Service is added to the integration (through the `service` trait).
 
+[cols="m,"]
+!===
+
+! route.host
+! To configure the host exposed by the route.
+
+!===
+
 | ingress
 | Kubernetes
 | Exposes the service associated with the integration to the outside world with a Kubernetes Ingress.
diff --git a/pkg/trait/route.go b/pkg/trait/route.go
index b3efa1a..02299b1 100644
--- a/pkg/trait/route.go
+++ b/pkg/trait/route.go
@@ -28,6 +28,7 @@ import (
 
 type routeTrait struct {
 	BaseTrait `property:",squash"`
+	Host      string `property:"host"`
 }
 
 func newRouteTrait() *routeTrait {
@@ -65,7 +66,7 @@ func (*routeTrait) getTargetService(e *environment, resources *kubernetes.Collec
 	return
 }
 
-func (*routeTrait) getRouteFor(e *environment, service *corev1.Service) *routev1.Route {
+func (e *routeTrait) getRouteFor(env *environment, service *corev1.Service) *routev1.Route {
 	route := routev1.Route{
 		TypeMeta: metav1.TypeMeta{
 			Kind:       "Route",
@@ -83,6 +84,7 @@ func (*routeTrait) getRouteFor(e *environment, service *corev1.Service) *routev1
 				Kind: "Service",
 				Name: service.Name,
 			},
+			Host: e.Host,
 		},
 	}
 	return &route