You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by pc...@apache.org on 2024/03/12 10:27:24 UTC

(camel-k) branch main updated: fix(traits): use Comparable matches

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 0725b40ad fix(traits): use Comparable matches
0725b40ad is described below

commit 0725b40ad7cc45d1a03f749505af2eafe3416059
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Thu Mar 7 17:34:16 2024 +0100

    fix(traits): use Comparable matches
    
    Reverting #4512 which introduced a function diverging the match from the original design
---
 docs/modules/ROOT/pages/architecture/traits.adoc | 11 ++-
 e2e/common/misc/pipe_test.go                     |  6 +-
 e2e/common/traits/builder_test.go                | 16 ++---
 pkg/controller/integration/kits.go               | 67 +-----------------
 pkg/controller/integration/kits_test.go          | 15 ----
 pkg/controller/integration/monitor.go            |  2 +-
 pkg/trait/builder.go                             | 31 +++++++-
 pkg/trait/builder_test.go                        | 51 ++++++++++++++
 pkg/trait/camel.go                               | 10 ++-
 pkg/trait/camel_test.go                          | 22 ++++++
 pkg/trait/quarkus.go                             | 20 +++---
 pkg/trait/quarkus_test.go                        | 28 ++++++++
 pkg/trait/registry.go                            |  5 --
 pkg/trait/trait_types.go                         | 13 +---
 pkg/trait/util.go                                | 89 ++++++++++++++++++-----
 pkg/trait/util_test.go                           | 90 ++++++++++++++++++++++++
 16 files changed, 332 insertions(+), 144 deletions(-)

diff --git a/docs/modules/ROOT/pages/architecture/traits.adoc b/docs/modules/ROOT/pages/architecture/traits.adoc
index fc56b1192..dee146ccb 100644
--- a/docs/modules/ROOT/pages/architecture/traits.adoc
+++ b/docs/modules/ROOT/pages/architecture/traits.adoc
@@ -49,6 +49,15 @@ type Trait interface {
 	IsAllowedInProfile(v1.TraitProfile) bool
 	Order() int
 }
+
+type Comparable interface {
+	Matches(trait Trait) bool
+}
+
+type ComparableTrait interface {
+	Trait
+	Comparable
+}
 ----
 
 Each trait will implement this interface. The most important methods that will be invoked by the xref:architecture/operator.adoc[Operator] are `Configure()` and `Apply()`. Basically, the `Configure()` method will set those inputs aforementioned (each trait has its own). The method is in charge to verify also the correctness of those expected parameters, where it makes sense (i.e., a well expected `Kubernetes` resource name). The function can return a `TraitCondition` object containing an [...]
@@ -59,6 +68,6 @@ The `Order()` method helps in resolving the order of execution of different trai
 
 The `InfluencesKit()`, `IsPlatformTrait()` and `RequiresIntegrationPlatform()` methods are easy to understand. They are used to determine if a trait has to influence an `IntegrationKit` build/initialization, if it's a platform trait (ie, needed by the platform itself) or are requiring the presence of an `IntegrationPlatform`.
 
-The presence of `InfluencesBuild()` will let specify the level of granularity of a trait down to its properties for a rebuild. So, if you need, you can compare the traits properties coming from the `prev` (previous) Integration to decide if it is worth to rebuild an Integration or the trait can reuse the one already provided in `this` version.
+For those traits that `InfluencesKit()` you may need to provide a `Matches(trait Trait)` func in order to specify those trait parameters that influences a build. This is required by the platform to decide if it is worth to rebuild an Integration or the trait can reuse the one already provided.
 
 Finally, through the `IsAllowedInProfile()` method we can override the default behavior (allow the trait for any profile). We must specify the profile we expect for this trait to be executed properly.
diff --git a/e2e/common/misc/pipe_test.go b/e2e/common/misc/pipe_test.go
index ef2f60936..e5e4db529 100644
--- a/e2e/common/misc/pipe_test.go
+++ b/e2e/common/misc/pipe_test.go
@@ -63,7 +63,7 @@ func TestPipe(t *testing.T) {
 					"--name", "throw-error-binding",
 				).Execute()).To(Succeed())
 
-				g.Eventually(IntegrationPodPhase(t, ns, "throw-error-binding"), TestTimeoutLong).Should(Equal(corev1.PodRunning))
+				g.Eventually(IntegrationPodPhase(t, ns, "throw-error-binding"), TestTimeoutMedium).Should(Equal(corev1.PodRunning))
 				g.Eventually(IntegrationLogs(t, ns, "throw-error-binding"), TestTimeoutShort).Should(ContainSubstring("[kameletErrorHandler] (Camel (camel-1) thread #1 - timer://tick)"))
 				g.Eventually(IntegrationLogs(t, ns, "throw-error-binding"), TestTimeoutShort).ShouldNot(ContainSubstring("[integrationLogger] (Camel (camel-1) thread #1 - timer://tick)"))
 
@@ -82,7 +82,7 @@ func TestPipe(t *testing.T) {
 					"--name", "no-error-binding",
 				).Execute()).To(Succeed())
 
-				g.Eventually(IntegrationPodPhase(t, ns, "no-error-binding"), TestTimeoutLong).Should(Equal(corev1.PodRunning))
+				g.Eventually(IntegrationPodPhase(t, ns, "no-error-binding"), TestTimeoutMedium).Should(Equal(corev1.PodRunning))
 				g.Eventually(IntegrationLogs(t, ns, "no-error-binding"), TestTimeoutShort).ShouldNot(ContainSubstring("[kameletErrorHandler] (Camel (camel-1) thread #1 - timer://tick)"))
 				g.Eventually(IntegrationLogs(t, ns, "no-error-binding"), TestTimeoutShort).Should(ContainSubstring("[integrationLogger] (Camel (camel-1) thread #1 - timer://tick)"))
 
@@ -103,7 +103,7 @@ func TestPipe(t *testing.T) {
 				"--name", "kb-with-traits",
 			).Execute()).To(Succeed())
 
-			g.Eventually(IntegrationPodPhase(t, ns, "kb-with-traits"), TestTimeoutLong).Should(Equal(corev1.PodRunning))
+			g.Eventually(IntegrationPodPhase(t, ns, "kb-with-traits"), TestTimeoutMedium).Should(Equal(corev1.PodRunning))
 			g.Eventually(IntegrationLogs(t, ns, "kb-with-traits"), TestTimeoutShort).Should(ContainSubstring("hello from test"))
 			g.Eventually(IntegrationLogs(t, ns, "kb-with-traits"), TestTimeoutShort).Should(ContainSubstring("integrationLogger"))
 		})
diff --git a/e2e/common/traits/builder_test.go b/e2e/common/traits/builder_test.go
index 42dc5b1b7..0470b4bb0 100644
--- a/e2e/common/traits/builder_test.go
+++ b/e2e/common/traits/builder_test.go
@@ -30,7 +30,6 @@ import (
 
 	corev1 "k8s.io/api/core/v1"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
-	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
 
 	. "github.com/apache/camel-k/v2/e2e/support"
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
@@ -78,6 +77,8 @@ func TestBuilderTrait(t *testing.T) {
 			name := RandomizedSuffixName("java-dependencies-strategy")
 			g.Expect(KamelRunWithID(t, operatorID, ns, "files/Java.java",
 				"--name", name,
+				// This is required in order to avoid reusing a Kit already existing (which is the default behavior)
+				"--build-property", "strategy=dependencies",
 				"-t", "builder.order-strategy=dependencies").Execute()).To(Succeed())
 
 			g.Eventually(IntegrationPodPhase(t, ns, name), TestTimeoutLong).Should(Equal(corev1.PodRunning))
@@ -94,17 +95,8 @@ func TestBuilderTrait(t *testing.T) {
 			g.Eventually(BuildConfig(t, integrationKitNamespace, integrationKitName)().LimitCPU, TestTimeoutShort).Should(Equal(""))
 			g.Eventually(BuildConfig(t, integrationKitNamespace, integrationKitName)().RequestMemory, TestTimeoutShort).Should(Equal(""))
 			g.Eventually(BuildConfig(t, integrationKitNamespace, integrationKitName)().LimitMemory, TestTimeoutShort).Should(Equal(""))
-
 			g.Eventually(BuilderPod(t, integrationKitNamespace, builderKitName), TestTimeoutShort).Should(BeNil())
 
-			// check integration schema does not contains unwanted default trait value.
-			g.Eventually(UnstructuredIntegration(t, ns, name)).ShouldNot(BeNil())
-			unstructuredIntegration := UnstructuredIntegration(t, ns, name)()
-			builderTrait, _, _ := unstructured.NestedMap(unstructuredIntegration.Object, "spec", "traits", "builder")
-			g.Expect(builderTrait).NotTo(BeNil())
-			g.Expect(len(builderTrait)).To(Equal(1))
-			g.Expect(builderTrait["orderStrategy"]).To(Equal("dependencies"))
-
 			g.Expect(Kamel(t, "delete", "--all", "-n", ns).Execute()).To(Succeed())
 		})
 
@@ -112,6 +104,8 @@ func TestBuilderTrait(t *testing.T) {
 			name := RandomizedSuffixName("java-fifo-strategy")
 			g.Expect(KamelRunWithID(t, operatorID, ns, "files/Java.java",
 				"--name", name,
+				// This is required in order to avoid reusing a Kit already existing (which is the default behavior)
+				"--build-property", "strategy=fifo",
 				"-t", "builder.order-strategy=fifo").Execute()).To(Succeed())
 
 			g.Eventually(IntegrationPodPhase(t, ns, name), TestTimeoutLong).Should(Equal(corev1.PodRunning))
@@ -138,6 +132,8 @@ func TestBuilderTrait(t *testing.T) {
 			name := RandomizedSuffixName("java-resource-config")
 			g.Expect(KamelRunWithID(t, operatorID, ns, "files/Java.java",
 				"--name", name,
+				// This is required in order to avoid reusing a Kit already existing (which is the default behavior)
+				"--build-property", "resources=new-build",
 				"-t", "builder.tasks-request-cpu=builder:500m",
 				"-t", "builder.tasks-limit-cpu=builder:1000m",
 				"-t", "builder.tasks-request-memory=builder:2Gi",
diff --git a/pkg/controller/integration/kits.go b/pkg/controller/integration/kits.go
index 20df5c682..22adf0f03 100644
--- a/pkg/controller/integration/kits.go
+++ b/pkg/controller/integration/kits.go
@@ -19,8 +19,6 @@ package integration
 
 import (
 	"context"
-	"fmt"
-	"reflect"
 
 	k8serrors "k8s.io/apimachinery/pkg/api/errors"
 	"k8s.io/apimachinery/pkg/labels"
@@ -132,7 +130,7 @@ func integrationMatches(ctx context.Context, c client.Client, integration *v1.In
 		return false, err
 	}
 
-	if match, err := hasMatchingTraits(itc, ikc); !match || err != nil {
+	if match, err := trait.HasMatchingTraits(itc, ikc); !match || err != nil {
 		ilog.Debug("Integration and integration-kit traits do not match", "integration", integration.Name, "integration-kit", kit.Name, "namespace", integration.Namespace)
 		return false, err
 	}
@@ -195,7 +193,7 @@ func kitMatches(kit *v1.IntegrationKit, target *v1.IntegrationKit) (bool, error)
 		return false, err
 	}
 
-	if match, err := hasMatchingTraits(c1, c2); !match || err != nil {
+	if match, err := trait.HasMatchingTraits(c1, c2); !match || err != nil {
 		return false, err
 	}
 	if !util.StringSliceContains(kit.Spec.Dependencies, target.Spec.Dependencies) {
@@ -205,67 +203,6 @@ func kitMatches(kit *v1.IntegrationKit, target *v1.IntegrationKit) (bool, error)
 	return true, nil
 }
 
-func hasMatchingTraits(traitMap trait.Options, kitTraitMap trait.Options) (bool, error) {
-	catalog := trait.NewCatalog(nil)
-
-	for _, t := range catalog.AllTraits() {
-		if t == nil {
-			continue
-		}
-
-		id := string(t.ID())
-		it, ok1 := traitMap.Get(id)
-		kt, ok2 := kitTraitMap.Get(id)
-
-		if !ok1 && !ok2 {
-			continue
-		}
-
-		if t.InfluencesKit() && t.InfluencesBuild(it, kt) {
-			if ct, ok := t.(trait.ComparableTrait); ok {
-				// if it's match trait use its matches method to determine the match
-				if match, err := matchesComparableTrait(ct, it, kt); !match || err != nil {
-					return false, err
-				}
-			} else {
-				if !matchesTrait(it, kt) {
-					return false, nil
-				}
-			}
-		}
-	}
-
-	return true, nil
-}
-
-func matchesComparableTrait(ct trait.ComparableTrait, it map[string]interface{}, kt map[string]interface{}) (bool, error) {
-	t1 := reflect.New(reflect.TypeOf(ct).Elem()).Interface()
-	if err := trait.ToTrait(it, &t1); err != nil {
-		return false, err
-	}
-
-	t2 := reflect.New(reflect.TypeOf(ct).Elem()).Interface()
-	if err := trait.ToTrait(kt, &t2); err != nil {
-		return false, err
-	}
-
-	ct2, ok := t2.(trait.ComparableTrait)
-	if !ok {
-		return false, fmt.Errorf("type assertion failed: %v", t2)
-	}
-	tt1, ok := t1.(trait.Trait)
-	if !ok {
-		return false, fmt.Errorf("type assertion failed: %v", t1)
-	}
-
-	return ct2.Matches(tt1), nil
-}
-
-func matchesTrait(it map[string]interface{}, kt map[string]interface{}) bool {
-	// perform exact match on the two trait maps
-	return reflect.DeepEqual(it, kt)
-}
-
 func hasMatchingSourcesForNative(it *v1.Integration, kit *v1.IntegrationKit) bool {
 	if len(it.UserDefinedSources()) != len(kit.Spec.Sources) {
 		return false
diff --git a/pkg/controller/integration/kits_test.go b/pkg/controller/integration/kits_test.go
index 685f5d0e0..1f946aa35 100644
--- a/pkg/controller/integration/kits_test.go
+++ b/pkg/controller/integration/kits_test.go
@@ -27,7 +27,6 @@ import (
 	traitv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1/trait"
 
 	"github.com/apache/camel-k/v2/pkg/trait"
-	"github.com/apache/camel-k/v2/pkg/util/log"
 	"github.com/apache/camel-k/v2/pkg/util/test"
 
 	"github.com/stretchr/testify/assert"
@@ -94,10 +93,6 @@ func TestLookupKitForIntegration_DiscardKitsInError(t *testing.T) {
 
 	require.NoError(t, err)
 
-	a := buildKitAction{}
-	a.InjectLogger(log.Log)
-	a.InjectClient(c)
-
 	kits, err := lookupKitsForIntegration(context.TODO(), c, &v1.Integration{
 		TypeMeta: metav1.TypeMeta{
 			APIVersion: v1.SchemeGroupVersion.String(),
@@ -221,10 +216,6 @@ func TestLookupKitForIntegration_DiscardKitsWithIncompatibleTraits(t *testing.T)
 
 	require.NoError(t, err)
 
-	a := buildKitAction{}
-	a.InjectLogger(log.Log)
-	a.InjectClient(c)
-
 	kits, err := lookupKitsForIntegration(context.TODO(), c, &v1.Integration{
 		TypeMeta: metav1.TypeMeta{
 			APIVersion: v1.SchemeGroupVersion.String(),
@@ -288,9 +279,6 @@ func TestHasMatchingTraits_KitNoTraitShouldNotBePicked(t *testing.T) {
 		},
 	}
 
-	a := buildKitAction{}
-	a.InjectLogger(log.Log)
-
 	ok, err := integrationAndKitHaveSameTraits(integration, kit)
 	require.NoError(t, err)
 	assert.False(t, ok)
@@ -339,9 +327,6 @@ func TestHasMatchingTraits_KitSameTraitShouldBePicked(t *testing.T) {
 		},
 	}
 
-	a := buildKitAction{}
-	a.InjectLogger(log.Log)
-
 	ok, err := integrationAndKitHaveSameTraits(integration, kit)
 	require.NoError(t, err)
 	assert.True(t, ok)
diff --git a/pkg/controller/integration/monitor.go b/pkg/controller/integration/monitor.go
index 4bd465beb..2c526be94 100644
--- a/pkg/controller/integration/monitor.go
+++ b/pkg/controller/integration/monitor.go
@@ -237,7 +237,7 @@ func (action *monitorAction) checkDigestAndRebuild(ctx context.Context, integrat
 	}
 
 	if hash != integration.Status.Digest {
-		action.L.Info("Integration %s digest has changed: resetting its status. Will check if it needs to be rebuilt and restarted.", integration.Name)
+		action.L.Infof("Integration %s digest has changed: resetting its status. Will check if it needs to be rebuilt and restarted.", integration.Name)
 		if isIntegrationKitResetRequired(integration, kit) {
 			integration.SetIntegrationKit(nil)
 		}
diff --git a/pkg/trait/builder.go b/pkg/trait/builder.go
index 6b9375126..8c6a8d790 100644
--- a/pkg/trait/builder.go
+++ b/pkg/trait/builder.go
@@ -20,6 +20,7 @@ package trait
 import (
 	"fmt"
 	"regexp"
+	"slices"
 	"sort"
 	"strconv"
 	"strings"
@@ -56,9 +57,33 @@ func (t *builderTrait) InfluencesKit() bool {
 	return true
 }
 
-// InfluencesBuild overrides base class method.
-func (t *builderTrait) InfluencesBuild(this, prev map[string]interface{}) bool {
-	return true
+func (t *builderTrait) Matches(trait Trait) bool {
+	otherTrait, ok := trait.(*builderTrait)
+	if !ok {
+		return false
+	}
+	if t.BaseImage != otherTrait.BaseImage || len(t.Properties) != len(otherTrait.Properties) || len(t.Tasks) != len(otherTrait.Tasks) {
+		return false
+	}
+	// More sofisticated check if len is the same. Sort and compare via slices equal func.
+	// Although the Matches func is used as a support for comparison, it makes sense
+	// to copy the properties and avoid possible inconsistencies caused by the sorting operation.
+	srtThisProps := make([]string, len(t.Properties))
+	srtOtheProps := make([]string, len(otherTrait.Properties))
+	copy(srtThisProps, t.Properties)
+	copy(srtOtheProps, otherTrait.Properties)
+	slices.Sort(srtThisProps)
+	slices.Sort(srtOtheProps)
+	if !slices.Equal(srtThisProps, srtOtheProps) {
+		return false
+	}
+	srtThisTasks := make([]string, len(t.Tasks))
+	srtOtheTasks := make([]string, len(otherTrait.Tasks))
+	copy(srtThisTasks, t.Tasks)
+	copy(srtOtheTasks, otherTrait.Tasks)
+	slices.Sort(srtThisTasks)
+	slices.Sort(srtOtheTasks)
+	return slices.Equal(srtThisTasks, srtOtheTasks)
 }
 
 func (t *builderTrait) Configure(e *Environment) (bool, *TraitCondition, error) {
diff --git a/pkg/trait/builder_test.go b/pkg/trait/builder_test.go
index 2448a2877..7b3756719 100644
--- a/pkg/trait/builder_test.go
+++ b/pkg/trait/builder_test.go
@@ -28,6 +28,7 @@ import (
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	traitv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1/trait"
 	"github.com/apache/camel-k/v2/pkg/util/camel"
 	"github.com/apache/camel-k/v2/pkg/util/defaults"
 	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
@@ -603,3 +604,53 @@ func tasksByName(tasks []v1.Task) []string {
 	}
 	return pipelineTasks
 }
+
+func TestBuilderMatches(t *testing.T) {
+	t1 := builderTrait{
+		BasePlatformTrait: NewBasePlatformTrait("builder", 600),
+		BuilderTrait: traitv1.BuilderTrait{
+			OrderStrategy: "dependencies",
+		},
+	}
+	t2 := builderTrait{
+		BasePlatformTrait: NewBasePlatformTrait("builder", 600),
+		BuilderTrait: traitv1.BuilderTrait{
+			OrderStrategy: "dependencies",
+		},
+	}
+	assert.True(t, t1.Matches(&t2))
+	// This is a property that does not influence the build
+	t2.OrderStrategy = "fifo"
+	assert.True(t, t1.Matches(&t2))
+	// Changing properties which influences build
+	t1.Properties = []string{"hello=world"}
+	assert.False(t, t1.Matches(&t2))
+	t2.Properties = []string{"hello=world"}
+	assert.True(t, t1.Matches(&t2))
+	t1.Properties = []string{"hello=world", "weare=theworld"}
+	assert.False(t, t1.Matches(&t2))
+	// should detect swap
+	t2.Properties = []string{"weare=theworld", "hello=world"}
+	assert.True(t, t1.Matches(&t2))
+}
+
+func TestBuilderMatchesTasks(t *testing.T) {
+	t1 := builderTrait{
+		BasePlatformTrait: NewBasePlatformTrait("builder", 600),
+		BuilderTrait:      traitv1.BuilderTrait{},
+	}
+	t2 := builderTrait{
+		BasePlatformTrait: NewBasePlatformTrait("builder", 600),
+		BuilderTrait: traitv1.BuilderTrait{
+			Tasks: []string{"task1;my-task;do-something"},
+		},
+	}
+	t3 := builderTrait{
+		BasePlatformTrait: NewBasePlatformTrait("builder", 600),
+		BuilderTrait: traitv1.BuilderTrait{
+			Tasks: []string{"task1;my-task;do-something-else"},
+		},
+	}
+	assert.False(t, t1.Matches(&t2))
+	assert.False(t, t2.Matches(&t3))
+}
diff --git a/pkg/trait/camel.go b/pkg/trait/camel.go
index f56c2bd1f..3d2a991d8 100644
--- a/pkg/trait/camel.go
+++ b/pkg/trait/camel.go
@@ -53,9 +53,13 @@ func (t *camelTrait) InfluencesKit() bool {
 	return true
 }
 
-// InfluencesBuild only when the runtime has changed.
-func (t *camelTrait) InfluencesBuild(this, prev map[string]interface{}) bool {
-	return this["runtimeVersion"] != prev["runtimeVersion"]
+func (t *camelTrait) Matches(trait Trait) bool {
+	otherTrait, ok := trait.(*camelTrait)
+	if !ok {
+		return false
+	}
+
+	return otherTrait.RuntimeVersion == t.RuntimeVersion
 }
 
 func (t *camelTrait) Configure(e *Environment) (bool, *TraitCondition, error) {
diff --git a/pkg/trait/camel_test.go b/pkg/trait/camel_test.go
index e60e7cf99..a3fc3688d 100644
--- a/pkg/trait/camel_test.go
+++ b/pkg/trait/camel_test.go
@@ -25,6 +25,7 @@ import (
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	traitv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1/trait"
 	"github.com/apache/camel-k/v2/pkg/util/camel"
 	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
 	"github.com/apache/camel-k/v2/pkg/util/test"
@@ -208,3 +209,24 @@ func TestApplyCamelTraitWithSources(t *testing.T) {
 		"content": "XML Source Code",
 	}, sourceCm.Data)
 }
+
+func TestCamelMatches(t *testing.T) {
+	t1 := camelTrait{
+		BasePlatformTrait: NewBasePlatformTrait("camel", 600),
+		CamelTrait: traitv1.CamelTrait{
+			RuntimeVersion: "1.2.3",
+		},
+	}
+	t2 := camelTrait{
+		BasePlatformTrait: NewBasePlatformTrait("camel", 600),
+		CamelTrait: traitv1.CamelTrait{
+			RuntimeVersion: "1.2.3",
+		},
+	}
+
+	assert.True(t, t1.Matches(&t2))
+	t1.Properties = []string{"hello=world"}
+	assert.True(t, t1.Matches(&t2))
+	t2.RuntimeVersion = "3.2.1"
+	assert.False(t, t1.Matches(&t2))
+}
diff --git a/pkg/trait/quarkus.go b/pkg/trait/quarkus.go
index bf7fdcc68..3803cb24f 100644
--- a/pkg/trait/quarkus.go
+++ b/pkg/trait/quarkus.go
@@ -109,23 +109,14 @@ func (t *quarkusTrait) InfluencesKit() bool {
 	return true
 }
 
-// InfluencesBuild overrides base class method.
-func (t *quarkusTrait) InfluencesBuild(this, prev map[string]interface{}) bool {
-	return true
-}
-
-var _ ComparableTrait = &quarkusTrait{}
-
 func (t *quarkusTrait) Matches(trait Trait) bool {
 	qt, ok := trait.(*quarkusTrait)
 	if !ok {
 		return false
 	}
-
 	if len(t.Modes) == 0 && len(qt.Modes) != 0 && !qt.containsMode(traitv1.JvmQuarkusMode) {
 		return false
 	}
-
 	for _, md := range t.Modes {
 		if md == traitv1.JvmQuarkusMode && len(qt.Modes) == 0 {
 			continue
@@ -135,8 +126,17 @@ func (t *quarkusTrait) Matches(trait Trait) bool {
 		}
 		return false
 	}
+	// We need to check if the native base image used is the same
+	thisNativeBaseImage := t.NativeBaseImage
+	if thisNativeBaseImage == "" {
+		thisNativeBaseImage = QuarkusNativeDefaultBaseImageName
+	}
+	otherNativeBaseImage := qt.NativeBaseImage
+	if otherNativeBaseImage == "" {
+		otherNativeBaseImage = QuarkusNativeDefaultBaseImageName
+	}
 
-	return true
+	return thisNativeBaseImage == otherNativeBaseImage
 }
 
 func (t *quarkusTrait) Configure(e *Environment) (bool, *TraitCondition, error) {
diff --git a/pkg/trait/quarkus_test.go b/pkg/trait/quarkus_test.go
index 752e57fc0..5d681a309 100644
--- a/pkg/trait/quarkus_test.go
+++ b/pkg/trait/quarkus_test.go
@@ -225,3 +225,31 @@ func TestGetLanguageSettingsWithLoaders(t *testing.T) {
 	assert.Equal(t, languageSettings{native: false, sourcesRequiredAtBuildTime: true}, getLanguageSettings(environment, v1.LanguageKotlin))
 	assert.Equal(t, languageSettings{native: true, sourcesRequiredAtBuildTime: false}, getLanguageSettings(environment, v1.LanguageJavaShell))
 }
+
+func TestQuarkusMatches(t *testing.T) {
+	qt := quarkusTrait{
+		BasePlatformTrait: NewBasePlatformTrait("quarkus", 600),
+		QuarkusTrait: traitv1.QuarkusTrait{
+			Modes: []traitv1.QuarkusMode{traitv1.JvmQuarkusMode},
+		},
+	}
+	qt2 := quarkusTrait{
+		BasePlatformTrait: NewBasePlatformTrait("quarkus", 600),
+		QuarkusTrait: traitv1.QuarkusTrait{
+			Modes:           []traitv1.QuarkusMode{traitv1.JvmQuarkusMode},
+			NativeBaseImage: QuarkusNativeDefaultBaseImageName,
+		},
+	}
+
+	assert.True(t, qt.Matches(&qt2))
+	qt2.Modes = append(qt2.Modes, traitv1.NativeQuarkusMode)
+	assert.True(t, qt.Matches(&qt2))
+	qt2.Modes = []traitv1.QuarkusMode{traitv1.NativeQuarkusMode}
+	assert.False(t, qt.Matches(&qt2))
+	qt2.Modes = nil
+	assert.True(t, qt.Matches(&qt2))
+	qt2.Modes = []traitv1.QuarkusMode{}
+	assert.True(t, qt.Matches(&qt2))
+	qt2.NativeBaseImage = "docker.io/my-new-native-base"
+	assert.False(t, qt.Matches(&qt2))
+}
diff --git a/pkg/trait/registry.go b/pkg/trait/registry.go
index 98d9f88e2..906e6f7fa 100644
--- a/pkg/trait/registry.go
+++ b/pkg/trait/registry.go
@@ -54,11 +54,6 @@ func (t *registryTrait) InfluencesKit() bool {
 	return true
 }
 
-// InfluencesBuild overrides base class method.
-func (t *registryTrait) InfluencesBuild(this, prev map[string]interface{}) bool {
-	return true
-}
-
 func (t *registryTrait) Configure(e *Environment) (bool, *TraitCondition, error) {
 	// disabled by default
 	if e.IntegrationKit == nil || !pointer.BoolDeref(t.Enabled, false) {
diff --git a/pkg/trait/trait_types.go b/pkg/trait/trait_types.go
index 1d41da632..85d152e47 100644
--- a/pkg/trait/trait_types.go
+++ b/pkg/trait/trait_types.go
@@ -73,11 +73,6 @@ type Trait interface {
 	// InfluencesKit determines if the trait has any influence on Integration Kits
 	InfluencesKit() bool
 
-	// InfluencesBuild defines a low level of granularity for those traits which influences the build.
-	// The trait can specify if any particular trait configuration influences a build or not.
-	// Note: You must override this method if you override `InfluencesKit()`
-	InfluencesBuild(this, prev map[string]interface{}) bool
-
 	// IsPlatformTrait marks all fundamental traits that allow the platform to work
 	IsPlatformTrait() bool
 
@@ -91,10 +86,12 @@ type Trait interface {
 	Order() int
 }
 
+// Comparable is the interface exposing comparable funcs.
 type Comparable interface {
 	Matches(trait Trait) bool
 }
 
+// ComparableTrait is the interface used to compare two traits between them.
 type ComparableTrait interface {
 	Trait
 	Comparable
@@ -154,12 +151,6 @@ func (trait *BaseTrait) InfluencesKit() bool {
 	return false
 }
 
-// InfluencesBuild defines a low level of granularity for those traits which influences the build.
-// The trait can specify if any particular trait configuration influences a build or not.
-func (trait *BaseTrait) InfluencesBuild(this, prev map[string]interface{}) bool {
-	return false
-}
-
 // IsPlatformTrait marks all fundamental traits that allow the platform to work.
 func (trait *BaseTrait) IsPlatformTrait() bool {
 	return false
diff --git a/pkg/trait/util.go b/pkg/trait/util.go
index fe1190ea6..374d90e48 100644
--- a/pkg/trait/util.go
+++ b/pkg/trait/util.go
@@ -533,7 +533,6 @@ func NewTraitsOptionsForKameletBinding(kb *v1alpha1.KameletBinding) (Options, er
 
 func FromAnnotations(meta *metav1.ObjectMeta) (Options, error) {
 	options := make(Options)
-
 	for k, v := range meta.Annotations {
 		if strings.HasPrefix(k, v1.TraitAnnotationPrefix) {
 			configKey := strings.TrimPrefix(k, v1.TraitAnnotationPrefix)
@@ -544,22 +543,7 @@ func FromAnnotations(meta *metav1.ObjectMeta) (Options, error) {
 				if _, ok := options[id]; !ok {
 					options[id] = make(map[string]interface{})
 				}
-
-				propParts := util.ConfigTreePropertySplit(prop)
-				var current = options[id]
-				if len(propParts) > 1 {
-					c, err := util.NavigateConfigTree(current, propParts[0:len(propParts)-1])
-					if err != nil {
-						return options, err
-					}
-					if cc, ok := c.(map[string]interface{}); ok {
-						current = cc
-					} else {
-						return options, errors.New(`invalid array specification: to set an array value use the ["v1", "v2"] format`)
-					}
-				}
-				current[prop] = v
-
+				options[id][prop] = stringOrSlice(v)
 			} else {
 				return options, fmt.Errorf("wrong format for trait annotation %q: missing trait ID", k)
 			}
@@ -569,6 +553,24 @@ func FromAnnotations(meta *metav1.ObjectMeta) (Options, error) {
 	return options, nil
 }
 
+// stringOrSlice returns either a string or a slice with trimmed values when the input is
+// represented as an array style (ie, [a,b,c]).
+func stringOrSlice(val string) interface{} {
+	if val == "[]" {
+		// empty array
+		return []string{}
+	}
+	if strings.HasPrefix(val, "[") && strings.HasSuffix(val, "]") {
+		slice := strings.Split(val[1:len(val)-1], ",")
+		for i := range slice {
+			slice[i] = strings.Trim(slice[i], " ")
+		}
+		return slice
+	} else {
+		return val
+	}
+}
+
 // verify if the integration in the Environment contains an endpoint.
 func containsEndpoint(name string, e *Environment, c client.Client) (bool, error) {
 	sources, err := kubernetes.ResolveIntegrationSources(e.Ctx, c, e.Integration, e.Resources)
@@ -593,3 +595,56 @@ func containsEndpoint(name string, e *Environment, c client.Client) (bool, error
 	}
 	return hasKnativeEndpoint, nil
 }
+
+// HasMatchingTraits verifies if two traits options match.
+func HasMatchingTraits(traitMap Options, kitTraitMap Options) (bool, error) {
+	catalog := NewCatalog(nil)
+
+	for _, t := range catalog.AllTraits() {
+		if t == nil || !t.InfluencesKit() {
+			// We don't store the trait configuration if the trait cannot influence the kit behavior
+			continue
+		}
+		id := string(t.ID())
+		it, _ := traitMap.Get(id)
+		kt, _ := kitTraitMap.Get(id)
+		if ct, ok := t.(ComparableTrait); ok {
+			// if it's match trait use its matches method to determine the match
+			if match, err := matchesComparableTrait(ct, it, kt); !match || err != nil {
+				return false, err
+			}
+		} else {
+			if !matchesTrait(it, kt) {
+				return false, nil
+			}
+		}
+	}
+
+	return true, nil
+}
+
+func matchesComparableTrait(ct ComparableTrait, it map[string]interface{}, kt map[string]interface{}) (bool, error) {
+	t1 := reflect.New(reflect.TypeOf(ct).Elem()).Interface()
+	if err := ToTrait(it, &t1); err != nil {
+		return false, err
+	}
+	t2 := reflect.New(reflect.TypeOf(ct).Elem()).Interface()
+	if err := ToTrait(kt, &t2); err != nil {
+		return false, err
+	}
+	ct2, ok := t2.(ComparableTrait)
+	if !ok {
+		return false, fmt.Errorf("type assertion failed: %v", t2)
+	}
+	tt1, ok := t1.(Trait)
+	if !ok {
+		return false, fmt.Errorf("type assertion failed: %v", t1)
+	}
+
+	return ct2.Matches(tt1), nil
+}
+
+func matchesTrait(it map[string]interface{}, kt map[string]interface{}) bool {
+	// perform exact match on the two trait maps
+	return reflect.DeepEqual(it, kt)
+}
diff --git a/pkg/trait/util_test.go b/pkg/trait/util_test.go
index 69767785b..d89262bbf 100644
--- a/pkg/trait/util_test.go
+++ b/pkg/trait/util_test.go
@@ -345,3 +345,93 @@ func TestSameTraits(t *testing.T) {
 		assert.False(t, ok)
 	})
 }
+
+func TestHasMathchingTraitsEmpty(t *testing.T) {
+	opt1 := Options{
+		"builder": {},
+		"camel": {
+			"runtimeVersion": "1.2.3",
+		},
+		"quarkus": {},
+	}
+	opt2 := Options{
+		"camel": {
+			"runtimeVersion": "1.2.3",
+		},
+	}
+	opt3 := Options{
+		"camel": {
+			"runtimeVersion": "1.2.3",
+		},
+	}
+	opt4 := Options{
+		"camel": {
+			"runtimeVersion": "3.2.1",
+		},
+	}
+	b1, err := HasMatchingTraits(opt1, opt2)
+	assert.Nil(t, err)
+	assert.True(t, b1)
+	b2, err := HasMatchingTraits(opt1, opt4)
+	assert.Nil(t, err)
+	assert.False(t, b2)
+	b3, err := HasMatchingTraits(opt2, opt3)
+	assert.Nil(t, err)
+	assert.True(t, b3)
+}
+
+func TestHasMathchingTraitsMissing(t *testing.T) {
+	opt1 := Options{}
+	opt2 := Options{
+		"camel": {
+			"properties": []string{"a=1"},
+		},
+	}
+	b1, err := HasMatchingTraits(opt1, opt2)
+	assert.Nil(t, err)
+	assert.True(t, b1)
+}
+
+func TestFromAnnotationsPlain(t *testing.T) {
+	meta := metav1.ObjectMeta{
+		Annotations: map[string]string{
+			"trait.camel.apache.org/trait.prop1": "hello1",
+			"trait.camel.apache.org/trait.prop2": "hello2",
+		},
+	}
+	opt, err := FromAnnotations(&meta)
+	require.NoError(t, err)
+	tt, ok := opt.Get("trait")
+	assert.True(t, ok)
+	assert.Equal(t, "hello1", tt["prop1"])
+	assert.Equal(t, "hello2", tt["prop2"])
+}
+
+func TestFromAnnotationsArray(t *testing.T) {
+	meta := metav1.ObjectMeta{
+		Annotations: map[string]string{
+			"trait.camel.apache.org/trait.prop1": "[hello,world]",
+			// The func should trim empty spaces as well
+			"trait.camel.apache.org/trait.prop2": "[\"hello=1\", \"world=2\"]",
+		},
+	}
+	opt, err := FromAnnotations(&meta)
+	require.NoError(t, err)
+	tt, ok := opt.Get("trait")
+	assert.True(t, ok)
+	assert.Equal(t, []string{"hello", "world"}, tt["prop1"])
+	assert.Equal(t, []string{"\"hello=1\"", "\"world=2\""}, tt["prop2"])
+}
+
+func TestFromAnnotationsArrayEmpty(t *testing.T) {
+	meta := metav1.ObjectMeta{
+		Annotations: map[string]string{
+			"trait.camel.apache.org/trait.prop": "[]",
+		},
+	}
+	opt, err := FromAnnotations(&meta)
+	require.NoError(t, err)
+	tt, ok := opt.Get("trait")
+	assert.True(t, ok)
+	assert.Equal(t, []string{}, tt["prop"])
+}