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 2023/10/16 12:27:59 UTC

[camel-k] branch main updated (782b75ec6 -> 167d6afb2)

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

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


    from 782b75ec6 chore: change deprecated yaml statements
     new f83f39b59 feat(trait): mount kamelets for runtime
     new 1cf62ec45 chore(e2e): kamelet trait test
     new e1a4dbbce chore(doc): kamelets deployment explained
     new 571928270 fix(trait): configmap identifier
     new 5f64ec6c9 chore(trait): keep kamelet default location
     new 167d6afb2 feat(trait): partition kamelets

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:
 config/crd/bases/camel.apache.org_builds.yaml      |   8 ++
 .../bases/camel.apache.org_integrationkits.yaml    |   3 +
 .../camel.apache.org_integrationplatforms.yaml     |   8 ++
 .../crd/bases/camel.apache.org_integrations.yaml   |  10 ++
 .../bases/camel.apache.org_kameletbindings.yaml    |   7 ++
 config/crd/bases/camel.apache.org_kamelets.yaml    |   6 ++
 config/crd/bases/camel.apache.org_pipes.yaml       |   7 ++
 .../images/architecture/kamelets_deployment.png    | Bin 0 -> 35011 bytes
 docs/modules/ROOT/nav-end.adoc                     |   1 +
 docs/modules/ROOT/pages/architecture/kamelets.adoc |  21 ++++
 .../ROOT/pages/kamelets/kamelets-distribution.adoc |  11 +-
 .../modules/ROOT/pages/kamelets/kamelets-user.adoc |  29 -----
 docs/modules/ROOT/partials/apis/camel-k-crds.adoc  |  14 +++
 docs/modules/traits/pages/kamelets.adoc            |   4 +
 .../common/traits/files/webhook.yaml               |   7 +-
 .../traits/{camel_test.go => kamelet_test.go}      |  40 ++++---
 e2e/support/test_support.go                        |  10 ++
 helm/camel-k/crds/crd-build.yaml                   |   8 ++
 helm/camel-k/crds/crd-integration-kit.yaml         |   3 +
 helm/camel-k/crds/crd-integration-platform.yaml    |   8 ++
 helm/camel-k/crds/crd-integration.yaml             |  10 ++
 helm/camel-k/crds/crd-kamelet-binding.yaml         |   7 ++
 helm/camel-k/crds/crd-kamelet.yaml                 |   6 ++
 helm/camel-k/crds/crd-pipe.yaml                    |   7 ++
 pkg/apis/camel/v1/common_types.go                  |   2 +
 pkg/apis/camel/v1/common_types_support.go          |  18 ++++
 pkg/apis/camel/v1/integration_types_support.go     |  13 ---
 pkg/apis/camel/v1/trait/kamelets.go                |   2 +
 .../applyconfiguration/camel/v1/sourcespec.go      |   9 ++
 pkg/trait/camel.go                                 |   7 +-
 pkg/trait/camel_test.go                            |   2 +-
 pkg/trait/kamelets.go                              | 109 ++++++++++---------
 pkg/trait/kamelets_support.go                      |  91 ++++++++++++++++
 pkg/trait/kamelets_support_test.go                 | 117 +++++++++++++++++++++
 pkg/trait/kamelets_test.go                         |   6 +-
 pkg/trait/trait_types.go                           |  74 +++++++------
 pkg/util/kubernetes/factory.go                     |   3 +
 resources/traits.yaml                              |   4 +
 38 files changed, 536 insertions(+), 156 deletions(-)
 create mode 100644 docs/modules/ROOT/images/architecture/kamelets_deployment.png
 create mode 100644 docs/modules/ROOT/pages/architecture/kamelets.adoc
 copy config/rbac/patch-role-to-clusterrole.yaml => e2e/common/traits/files/webhook.yaml (91%)
 copy e2e/common/traits/{camel_test.go => kamelet_test.go} (60%)
 create mode 100644 pkg/trait/kamelets_support.go
 create mode 100644 pkg/trait/kamelets_support_test.go


[camel-k] 05/06: chore(trait): keep kamelet default location

Posted by pc...@apache.org.
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

commit 5f64ec6c927df7e74c2f2d5315ee838071a5643d
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Fri Oct 13 12:03:43 2023 +0200

    chore(trait): keep kamelet default location
    
    To maintain backward compatibility
---
 pkg/trait/kamelets.go    | 2 +-
 pkg/trait/trait_types.go | 2 ++
 2 files changed, 3 insertions(+), 1 deletion(-)

diff --git a/pkg/trait/kamelets.go b/pkg/trait/kamelets.go
index 705026397..574a8b327 100644
--- a/pkg/trait/kamelets.go
+++ b/pkg/trait/kamelets.go
@@ -200,7 +200,7 @@ func (t *kameletsTrait) addKamelets(e *Environment) error {
 		if e.ApplicationProperties == nil {
 			e.ApplicationProperties = map[string]string{}
 		}
-		e.ApplicationProperties[KameletLocationProperty] = fmt.Sprintf("file:%s", t.MountPoint)
+		e.ApplicationProperties[KameletLocationProperty] = fmt.Sprintf("file:%s,classpath:/kamelets", t.MountPoint)
 		e.Resources.Add(&kameletsBundleConfigmap)
 		// resort dependencies
 		sort.Strings(e.Integration.Status.Dependencies)
diff --git a/pkg/trait/trait_types.go b/pkg/trait/trait_types.go
index 05ee587ce..fca860d74 100644
--- a/pkg/trait/trait_types.go
+++ b/pkg/trait/trait_types.go
@@ -538,6 +538,8 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 			} else if configMap.Labels[kubernetes.ConfigMapTypeLabel] == "kamelets-bundle" {
 				// Kamelets bundle configmap
 				kameletMountPoint := strings.ReplaceAll(e.ApplicationProperties[KameletLocationProperty], "file:", "")
+				// We need also to remove the default location provided
+				kameletMountPoint = strings.ReplaceAll(kameletMountPoint, ",classpath:/kamelets", "")
 				refName := "kamelets-bundle"
 				vol := getVolume(refName, "configmap", configMap.Name, "", "")
 				mnt := getMount(refName, kameletMountPoint, "", true)


[camel-k] 02/06: chore(e2e): kamelet trait test

Posted by pc...@apache.org.
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

commit 1cf62ec451405fcb7ad21a5bccf479976ebc6b3f
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Tue Oct 10 11:38:27 2023 +0200

    chore(e2e): kamelet trait test
---
 e2e/common/traits/files/webhook.yaml | 21 +++++++++++
 e2e/common/traits/kamelet_test.go    | 67 ++++++++++++++++++++++++++++++++++++
 e2e/support/test_support.go          | 10 ++++++
 3 files changed, 98 insertions(+)

diff --git a/e2e/common/traits/files/webhook.yaml b/e2e/common/traits/files/webhook.yaml
new file mode 100644
index 000000000..c6deb4c8b
--- /dev/null
+++ b/e2e/common/traits/files/webhook.yaml
@@ -0,0 +1,21 @@
+# ---------------------------------------------------------------------------
+# 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.
+# ---------------------------------------------------------------------------
+
+- from:
+    uri: "kamelet:capabilities-webhook-source"
+    steps:
+      - log: "${body}"
diff --git a/e2e/common/traits/kamelet_test.go b/e2e/common/traits/kamelet_test.go
new file mode 100644
index 000000000..bd6ec96af
--- /dev/null
+++ b/e2e/common/traits/kamelet_test.go
@@ -0,0 +1,67 @@
+//go:build integration
+// +build integration
+
+// To enable compilation of this file in Goland, go to "Settings -> Go -> Vendoring & Build Tags -> Custom Tags" and add "integration"
+
+/*
+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 traits
+
+import (
+	"testing"
+
+	. "github.com/onsi/gomega"
+
+	corev1 "k8s.io/api/core/v1"
+
+	. "github.com/apache/camel-k/v2/e2e/support"
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+)
+
+func TestKameletTrait(t *testing.T) {
+	RegisterTestingT(t)
+
+	t.Run("discover kamelet capabilities", func(t *testing.T) {
+		template := map[string]interface{}{
+			"from": map[string]interface{}{
+				"uri": "platform-http:///webhook",
+				"steps": []map[string]interface{}{
+					{
+						"to": "kamelet:sink",
+					},
+				},
+			},
+		}
+		Expect(CreateKamelet(ns, "capabilities-webhook-source", template, nil, nil)()).To(Succeed())
+
+		name := "webhook"
+		Expect(KamelRunWithID(operatorID, ns, "files/webhook.yaml", "--name", name).Execute()).To(Succeed())
+		Eventually(IntegrationPodPhase(ns, name), TestTimeoutLong).Should(Equal(corev1.PodRunning))
+		Eventually(IntegrationConditionStatus(ns, name, v1.IntegrationConditionReady), TestTimeoutShort).Should(Equal(corev1.ConditionTrue))
+		Eventually(IntegrationLogs(ns, name), TestTimeoutShort).Should(ContainSubstring("Started capabilities-webhook-source-1 (platform-http:///webhook)"))
+		// Verify Integration capabilities
+		Eventually(IntegrationStatusCapabilities(ns, name), TestTimeoutShort).Should(ContainElements("platform-http"))
+		// Verify expected resources from Kamelet (Service in this case)
+		service := Service(ns, name)
+		Eventually(service, TestTimeoutShort).ShouldNot(BeNil())
+	})
+
+	// Clean-up
+	Expect(DeleteKamelet(ns, "capabilities-webhook-source")).To(Succeed())
+	Expect(Kamel("delete", "--all", "-n", ns).Execute()).To(Succeed())
+}
diff --git a/e2e/support/test_support.go b/e2e/support/test_support.go
index 7b57a711b..91b4228ad 100644
--- a/e2e/support/test_support.go
+++ b/e2e/support/test_support.go
@@ -998,6 +998,16 @@ func IntegrationSpecProfile(ns string, name string) func() v1.TraitProfile {
 	}
 }
 
+func IntegrationStatusCapabilities(ns string, name string) func() []string {
+	return func() []string {
+		it := Integration(ns, name)()
+		if it == nil || &it.Status == nil {
+			return nil
+		}
+		return it.Status.Capabilities
+	}
+}
+
 func IntegrationSpecSA(ns string, name string) func() string {
 	return func() string {
 		it := Integration(ns, name)()


[camel-k] 03/06: chore(doc): kamelets deployment explained

Posted by pc...@apache.org.
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

commit e1a4dbbce8cb7f809272970b8b12bee6a27f6628
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Thu Oct 5 12:33:13 2023 +0200

    chore(doc): kamelets deployment explained
---
 .../images/architecture/kamelets_deployment.png    | Bin 0 -> 35011 bytes
 docs/modules/ROOT/nav-end.adoc                     |   1 +
 docs/modules/ROOT/pages/architecture/kamelets.adoc |  19 ++++++++++++++
 .../ROOT/pages/kamelets/kamelets-distribution.adoc |  11 +++++++-
 .../modules/ROOT/pages/kamelets/kamelets-user.adoc |  29 ---------------------
 pkg/apis/camel/v1/common_types_support.go          |  15 ++++++++++-
 pkg/apis/camel/v1/integration_types_support.go     |  13 ---------
 7 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/docs/modules/ROOT/images/architecture/kamelets_deployment.png b/docs/modules/ROOT/images/architecture/kamelets_deployment.png
new file mode 100644
index 000000000..dbb184f07
Binary files /dev/null and b/docs/modules/ROOT/images/architecture/kamelets_deployment.png differ
diff --git a/docs/modules/ROOT/nav-end.adoc b/docs/modules/ROOT/nav-end.adoc
index 69cf0c052..3f57cc06f 100644
--- a/docs/modules/ROOT/nav-end.adoc
+++ b/docs/modules/ROOT/nav-end.adoc
@@ -83,6 +83,7 @@
 *** xref:architecture/cr/camel-catalog.adoc[CamelCatalog]
 ** xref:architecture/runtime.adoc[Runtime]
 ** xref:architecture/traits.adoc[Traits]
+** xref:architecture/kamelets.adoc[Kamelets]
 ** xref:architecture/incremental-image.adoc[Incremental Image]
 * API
 ** xref:apis/camel-k.adoc[Camel K API]
diff --git a/docs/modules/ROOT/pages/architecture/kamelets.adoc b/docs/modules/ROOT/pages/architecture/kamelets.adoc
new file mode 100644
index 000000000..d4006e072
--- /dev/null
+++ b/docs/modules/ROOT/pages/architecture/kamelets.adoc
@@ -0,0 +1,19 @@
+= Kamelets architecture in Camel K
+
+xref:kamelets/kamelets.adoc[Kamelets] are a technology which were originally developed as a Camel K side resource but moved into Camel framework as Kamelet component. From an design point of view, a Kamelet is a specification that is provided into the cluster and which can be used at any point by an Integration or a Pipe, in order to reuse the connector style approach.
+
+In Camel framework, a Kamelet is nothing than a component which can be used as any other component with the `kamelet` uri scheme. This is translated to one or more Route Templates. What's important for Camel runtime is to have the Kamelet spec available somewhere when running the application making reference to it.
+
+[[deployment-model]]
+== Deployment model
+
+In Camel K, it is worth to explain how the Kamelets are deployed in order to make Camel runtime application to correctly use the Kamelet referenced in the Integration. First of all, the operator uses a xref:traits:kamelets.adoc[Camel K trait] which is in charge to discover the Kamelets used in your route. This is due to get all the specification and dependencies required.
+
+image::architecture/kamelets_deployment.png[Kamelets deployment model]
+
+The operator creates a ConfigMap in order to bundle all the Kamelets which are eventually required by the application runtime. The Kamelets spec has to be available and in ready phase status. Once the application is created and ready to start, the operator mounts such a ConfigMap in a known location (default `/etc/camel/kamelets`) so that the Camel application will be able to read the definition from such location and run them according the logic expected in the same Camel framework.
+
+[[kamelet-parsing]]
+=== Parsing capabilities defined in a Kamelet
+
+The operator is in charge to perform one important hidden operation. The Kamelet specification may contains Camel components and capabilities which the user should be in charge to define explicitly. However, the operator extract the Kamelet source and parses its content as a generated Integration source. In this way you will be able to get all the Kubernetes resources which are required to run your Integration (ie, a Kamelet using rest or exposing http services).
diff --git a/docs/modules/ROOT/pages/kamelets/kamelets-distribution.adoc b/docs/modules/ROOT/pages/kamelets/kamelets-distribution.adoc
index 617039bc3..77e5a99fb 100644
--- a/docs/modules/ROOT/pages/kamelets/kamelets-distribution.adoc
+++ b/docs/modules/ROOT/pages/kamelets/kamelets-distribution.adoc
@@ -18,4 +18,13 @@ kamel kamelet add-repo <git-platform>:<owner>/<repo>[/path_to_kamelets_folder][@
 ```
 Mind that `<git-platform>` is the repository of your choice, for instance `github` and `[@version]` is the tag to use (by default it will use `main` branch).
 
-With this approach you can dynamically include any repository where your Kamelets are hosted. They will be lazily initialized as soon as they are required by any of the Integration or Pipes which will make use of them.
\ No newline at end of file
+With this approach you can dynamically include any repository where your Kamelets are hosted. They will be lazily initialized as soon as they are required by any of the Integration or Pipes which will make use of them.
+
+[[kamelets-as-dependency]]
+== Kamelets as a dependency
+
+The Camel K has an opinionated way to use Kamelets which is the one exposed above. Here the Kamelet spec resource is expected to be available in the cluster.
+
+However, you may find situations where you want to bundle a Kamelet in a dependency (ie, some external catalog containing all Kamelets spec). As Kamelets are a Camel thing, then, you can use such dependency and let the runtime use the Kamelets available in the classpath.
+
+If you're using such an approach, then, you will need to make sure to skip the Kamelet trait (which is in charge to discover them and get required dependencies), and provide all the dependencies which may be required by your Kamelet. Additionally, you may need to specify a Camel property to tell the runtime where to expect to find the Kamelets, `camel.component.kamelet.location` (default `classpath:/kamelets`).
\ No newline at end of file
diff --git a/docs/modules/ROOT/pages/kamelets/kamelets-user.adoc b/docs/modules/ROOT/pages/kamelets/kamelets-user.adoc
index cdb00a68f..10eec3afe 100644
--- a/docs/modules/ROOT/pages/kamelets/kamelets-user.adoc
+++ b/docs/modules/ROOT/pages/kamelets/kamelets-user.adoc
@@ -205,35 +205,6 @@ from('timer:tick')
 You can run this integration without specifying other parameters, the Kamelet endpoint will be implicitly configured by the Camel K operator that will
 automatically mount the secret into the integration Pod.
 
-[[kamelets-default-catalog]]
-== Kamelets Catalog
-
-When you install a Camel K Operator, you will have automatically a wide set of canned Kamelets available to be used. The catalog and the related documentation is available at link:/camel-kamelets/next/index.html[Apache Kamelets Catalog]. There is a CLI command very useful to have a quick list of the available Kamelets:
-[source,shell]
-----
-$ kamel kamelet get
-----
-
-These Kamelets are installed by default when you install Camel K operator. You will be able to use them without worrying to install each of them, just look at the documentation and be ready to experiment how easy they are.
-
-[[kamelets-custom-catalog]]
-=== Use a custom Catalog
-
-You can overwrite the default catalog by adding certain configuration to the `IntegrationPlatform`. In order to add a new repository, you must edit the `IntegrationPlatfom` and edit the `.spec.kamelet.repositories[].uri` field, which expects an array of repository URIs where you can host your catalog of Kamelets, ie:
-[source,yaml]
-----
-spec:
-  ...
-  kamelet:
-    repositories:
-    - uri: github:my-org/my-repo
-  ...
-----
-
-By default the value is null, which means they fallback to the Apache Kamelets Catalog.
-
-NOTE: this option is experimental and may be subjected to changes in future versions.
-
 [[kamelets-usage-binding]]
 == Binding Kamelets
 
diff --git a/pkg/apis/camel/v1/common_types_support.go b/pkg/apis/camel/v1/common_types_support.go
index c8adc7809..4e7b54dc8 100644
--- a/pkg/apis/camel/v1/common_types_support.go
+++ b/pkg/apis/camel/v1/common_types_support.go
@@ -212,7 +212,20 @@ func DecodeValueSource(input string, defaultKey string, errorMessage string) (Va
 	return ValueSource{}, fmt.Errorf(errorMessage)
 }
 
-// IsGeneratedFromKamelet determines is a source spec is derived from a Kamelet
+// IsGeneratedFromKamelet determines is a source spec is derived from a Kamelet.
 func (s *SourceSpec) IsGeneratedFromKamelet() bool {
 	return s.FromKamelet
 }
+
+// InferLanguage returns the language of the source or discovers it from file extension if not set.
+func (s *SourceSpec) InferLanguage() Language {
+	if s.Language != "" {
+		return s.Language
+	}
+	for _, l := range Languages {
+		if strings.HasSuffix(s.Name, "."+string(l)) {
+			return l
+		}
+	}
+	return ""
+}
diff --git a/pkg/apis/camel/v1/integration_types_support.go b/pkg/apis/camel/v1/integration_types_support.go
index 23c50f9e9..4ad79634f 100644
--- a/pkg/apis/camel/v1/integration_types_support.go
+++ b/pkg/apis/camel/v1/integration_types_support.go
@@ -192,19 +192,6 @@ func NewSourceSpec(name string, content string, language Language) SourceSpec {
 	}
 }
 
-// InferLanguage returns the language of the source or discovers it from file extension if not set.
-func (in *SourceSpec) InferLanguage() Language {
-	if in.Language != "" {
-		return in.Language
-	}
-	for _, l := range Languages {
-		if strings.HasSuffix(in.Name, "."+string(l)) {
-			return l
-		}
-	}
-	return ""
-}
-
 // SetOperatorID sets the given operator id as an annotation.
 func (in *Integration) SetOperatorID(operatorID string) {
 	SetAnnotation(&in.ObjectMeta, OperatorIDAnnotation, operatorID)


[camel-k] 01/06: feat(trait): mount kamelets for runtime

Posted by pc...@apache.org.
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

commit f83f39b59df2095ab00ae8c691819a1502e9b5a1
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Tue Oct 10 10:54:20 2023 +0200

    feat(trait): mount kamelets for runtime
    
    * Add a configmap as a bundle for all kamelets used by the Integration
    * Mount the bundle in /etc/camel/kamelets
    * Let the runtime use the Kamelets instead of trasforming into RouteTemplates
    * Let the operator use the Kamelet template for capability parsing
    
    Closes #4618
---
 config/crd/bases/camel.apache.org_builds.yaml      |   8 ++
 .../bases/camel.apache.org_integrationkits.yaml    |   3 +
 .../camel.apache.org_integrationplatforms.yaml     |   8 ++
 .../crd/bases/camel.apache.org_integrations.yaml   |  10 ++
 .../bases/camel.apache.org_kameletbindings.yaml    |   7 ++
 config/crd/bases/camel.apache.org_kamelets.yaml    |   6 ++
 config/crd/bases/camel.apache.org_pipes.yaml       |   7 ++
 docs/modules/ROOT/partials/apis/camel-k-crds.adoc  |  14 +++
 docs/modules/traits/pages/kamelets.adoc            |   4 +
 helm/camel-k/crds/crd-build.yaml                   |   8 ++
 helm/camel-k/crds/crd-integration-kit.yaml         |   3 +
 helm/camel-k/crds/crd-integration-platform.yaml    |   8 ++
 helm/camel-k/crds/crd-integration.yaml             |  10 ++
 helm/camel-k/crds/crd-kamelet-binding.yaml         |   7 ++
 helm/camel-k/crds/crd-kamelet.yaml                 |   6 ++
 helm/camel-k/crds/crd-pipe.yaml                    |   7 ++
 pkg/apis/camel/v1/common_types.go                  |   2 +
 pkg/apis/camel/v1/common_types_support.go          |   5 +
 pkg/apis/camel/v1/trait/kamelets.go                |   2 +
 .../applyconfiguration/camel/v1/sourcespec.go      |   9 ++
 pkg/trait/camel.go                                 |   1 +
 pkg/trait/kamelets.go                              | 117 +++++++++++----------
 pkg/trait/kamelets_test.go                         |   6 +-
 pkg/trait/trait_types.go                           |  74 +++++++------
 pkg/util/kubernetes/factory.go                     |   3 +
 resources/traits.yaml                              |   4 +
 26 files changed, 251 insertions(+), 88 deletions(-)

diff --git a/config/crd/bases/camel.apache.org_builds.yaml b/config/crd/bases/camel.apache.org_builds.yaml
index 08a6cc213..caacc731d 100644
--- a/config/crd/bases/camel.apache.org_builds.yaml
+++ b/config/crd/bases/camel.apache.org_builds.yaml
@@ -686,6 +686,10 @@ spec:
                               contentType:
                                 description: the content type (tipically text or binary)
                                 type: string
+                              from-kamelet:
+                                description: True if the spec is generated from a
+                                  Kamelet
+                                type: boolean
                               interceptors:
                                 description: Interceptors are optional identifiers
                                   the org.apache.camel.k.RoutesLoader uses to pre/post
@@ -1424,6 +1428,10 @@ spec:
                               contentType:
                                 description: the content type (tipically text or binary)
                                 type: string
+                              from-kamelet:
+                                description: True if the spec is generated from a
+                                  Kamelet
+                                type: boolean
                               interceptors:
                                 description: Interceptors are optional identifiers
                                   the org.apache.camel.k.RoutesLoader uses to pre/post
diff --git a/config/crd/bases/camel.apache.org_integrationkits.yaml b/config/crd/bases/camel.apache.org_integrationkits.yaml
index 3d56b6ab8..d3e061b62 100644
--- a/config/crd/bases/camel.apache.org_integrationkits.yaml
+++ b/config/crd/bases/camel.apache.org_integrationkits.yaml
@@ -143,6 +143,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/config/crd/bases/camel.apache.org_integrationplatforms.yaml b/config/crd/bases/camel.apache.org_integrationplatforms.yaml
index d34c2f432..011141207 100644
--- a/config/crd/bases/camel.apache.org_integrationplatforms.yaml
+++ b/config/crd/bases/camel.apache.org_integrationplatforms.yaml
@@ -1190,6 +1190,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
@@ -3004,6 +3008,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
diff --git a/config/crd/bases/camel.apache.org_integrations.yaml b/config/crd/bases/camel.apache.org_integrations.yaml
index 608118fef..0ea61efce 100644
--- a/config/crd/bases/camel.apache.org_integrations.yaml
+++ b/config/crd/bases/camel.apache.org_integrations.yaml
@@ -189,6 +189,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
@@ -7107,6 +7110,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
@@ -7916,6 +7923,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/config/crd/bases/camel.apache.org_kameletbindings.yaml b/config/crd/bases/camel.apache.org_kameletbindings.yaml
index b40d8c9a8..5e137abd8 100644
--- a/config/crd/bases/camel.apache.org_kameletbindings.yaml
+++ b/config/crd/bases/camel.apache.org_kameletbindings.yaml
@@ -189,6 +189,9 @@ spec:
                         contentType:
                           description: the content type (tipically text or binary)
                           type: string
+                        from-kamelet:
+                          description: True if the spec is generated from a Kamelet
+                          type: boolean
                         interceptors:
                           description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                             uses to pre/post process sources
@@ -7393,6 +7396,10 @@ spec:
                             description: Comma separated list of Kamelet names to
                               load into the current integration
                             type: string
+                          mountPoint:
+                            description: The directory where the application mounts
+                              and reads Kamelet spec (default `/etc/camel/kamelets`)
+                            type: string
                         type: object
                       keda:
                         description: 'Deprecated: for backward compatibility.'
diff --git a/config/crd/bases/camel.apache.org_kamelets.yaml b/config/crd/bases/camel.apache.org_kamelets.yaml
index 3131ac0e2..aeb9739e2 100644
--- a/config/crd/bases/camel.apache.org_kamelets.yaml
+++ b/config/crd/bases/camel.apache.org_kamelets.yaml
@@ -466,6 +466,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
@@ -1163,6 +1166,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/config/crd/bases/camel.apache.org_pipes.yaml b/config/crd/bases/camel.apache.org_pipes.yaml
index ca6c71c0e..a30364531 100644
--- a/config/crd/bases/camel.apache.org_pipes.yaml
+++ b/config/crd/bases/camel.apache.org_pipes.yaml
@@ -187,6 +187,9 @@ spec:
                         contentType:
                           description: the content type (tipically text or binary)
                           type: string
+                        from-kamelet:
+                          description: True if the spec is generated from a Kamelet
+                          type: boolean
                         interceptors:
                           description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                             uses to pre/post process sources
@@ -7391,6 +7394,10 @@ spec:
                             description: Comma separated list of Kamelet names to
                               load into the current integration
                             type: string
+                          mountPoint:
+                            description: The directory where the application mounts
+                              and reads Kamelet spec (default `/etc/camel/kamelets`)
+                            type: string
                         type: object
                       keda:
                         description: 'Deprecated: for backward compatibility.'
diff --git a/docs/modules/ROOT/partials/apis/camel-k-crds.adoc b/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
index d280492a7..a6d669c43 100644
--- a/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
+++ b/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
@@ -5064,6 +5064,13 @@ Type defines the kind of source described by this object
 
 List of property names defined in the source (e.g. if type is "template")
 
+|`from-kamelet` +
+bool
+|
+
+
+True if the spec is generated from a Kamelet
+
 
 |===
 
@@ -6932,6 +6939,13 @@ string
 
 Comma separated list of Kamelet names to load into the current integration
 
+|`mountPoint` +
+string
+|
+
+
+The directory where the application mounts and reads Kamelet spec (default `/etc/camel/kamelets`)
+
 
 |===
 
diff --git a/docs/modules/traits/pages/kamelets.adoc b/docs/modules/traits/pages/kamelets.adoc
index 0336e0ff4..5fbf1f197 100755
--- a/docs/modules/traits/pages/kamelets.adoc
+++ b/docs/modules/traits/pages/kamelets.adoc
@@ -33,6 +33,10 @@ The following configuration options are available:
 | string
 | Comma separated list of Kamelet names to load into the current integration
 
+| kamelets.mount-point
+| string
+| The directory where the application mounts and reads Kamelet spec (default `/etc/camel/kamelets`)
+
 |===
 
 // End of autogenerated code - DO NOT EDIT! (configuration)
diff --git a/helm/camel-k/crds/crd-build.yaml b/helm/camel-k/crds/crd-build.yaml
index 08a6cc213..caacc731d 100644
--- a/helm/camel-k/crds/crd-build.yaml
+++ b/helm/camel-k/crds/crd-build.yaml
@@ -686,6 +686,10 @@ spec:
                               contentType:
                                 description: the content type (tipically text or binary)
                                 type: string
+                              from-kamelet:
+                                description: True if the spec is generated from a
+                                  Kamelet
+                                type: boolean
                               interceptors:
                                 description: Interceptors are optional identifiers
                                   the org.apache.camel.k.RoutesLoader uses to pre/post
@@ -1424,6 +1428,10 @@ spec:
                               contentType:
                                 description: the content type (tipically text or binary)
                                 type: string
+                              from-kamelet:
+                                description: True if the spec is generated from a
+                                  Kamelet
+                                type: boolean
                               interceptors:
                                 description: Interceptors are optional identifiers
                                   the org.apache.camel.k.RoutesLoader uses to pre/post
diff --git a/helm/camel-k/crds/crd-integration-kit.yaml b/helm/camel-k/crds/crd-integration-kit.yaml
index 3d56b6ab8..d3e061b62 100644
--- a/helm/camel-k/crds/crd-integration-kit.yaml
+++ b/helm/camel-k/crds/crd-integration-kit.yaml
@@ -143,6 +143,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/helm/camel-k/crds/crd-integration-platform.yaml b/helm/camel-k/crds/crd-integration-platform.yaml
index d34c2f432..011141207 100644
--- a/helm/camel-k/crds/crd-integration-platform.yaml
+++ b/helm/camel-k/crds/crd-integration-platform.yaml
@@ -1190,6 +1190,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
@@ -3004,6 +3008,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
diff --git a/helm/camel-k/crds/crd-integration.yaml b/helm/camel-k/crds/crd-integration.yaml
index 608118fef..0ea61efce 100644
--- a/helm/camel-k/crds/crd-integration.yaml
+++ b/helm/camel-k/crds/crd-integration.yaml
@@ -189,6 +189,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
@@ -7107,6 +7110,10 @@ spec:
                         description: Comma separated list of Kamelet names to load
                           into the current integration
                         type: string
+                      mountPoint:
+                        description: The directory where the application mounts and
+                          reads Kamelet spec (default `/etc/camel/kamelets`)
+                        type: string
                     type: object
                   keda:
                     description: 'Deprecated: for backward compatibility.'
@@ -7916,6 +7923,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/helm/camel-k/crds/crd-kamelet-binding.yaml b/helm/camel-k/crds/crd-kamelet-binding.yaml
index b40d8c9a8..5e137abd8 100644
--- a/helm/camel-k/crds/crd-kamelet-binding.yaml
+++ b/helm/camel-k/crds/crd-kamelet-binding.yaml
@@ -189,6 +189,9 @@ spec:
                         contentType:
                           description: the content type (tipically text or binary)
                           type: string
+                        from-kamelet:
+                          description: True if the spec is generated from a Kamelet
+                          type: boolean
                         interceptors:
                           description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                             uses to pre/post process sources
@@ -7393,6 +7396,10 @@ spec:
                             description: Comma separated list of Kamelet names to
                               load into the current integration
                             type: string
+                          mountPoint:
+                            description: The directory where the application mounts
+                              and reads Kamelet spec (default `/etc/camel/kamelets`)
+                            type: string
                         type: object
                       keda:
                         description: 'Deprecated: for backward compatibility.'
diff --git a/helm/camel-k/crds/crd-kamelet.yaml b/helm/camel-k/crds/crd-kamelet.yaml
index 3131ac0e2..aeb9739e2 100644
--- a/helm/camel-k/crds/crd-kamelet.yaml
+++ b/helm/camel-k/crds/crd-kamelet.yaml
@@ -466,6 +466,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
@@ -1163,6 +1166,9 @@ spec:
                     contentType:
                       description: the content type (tipically text or binary)
                       type: string
+                    from-kamelet:
+                      description: True if the spec is generated from a Kamelet
+                      type: boolean
                     interceptors:
                       description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                         uses to pre/post process sources
diff --git a/helm/camel-k/crds/crd-pipe.yaml b/helm/camel-k/crds/crd-pipe.yaml
index ca6c71c0e..a30364531 100644
--- a/helm/camel-k/crds/crd-pipe.yaml
+++ b/helm/camel-k/crds/crd-pipe.yaml
@@ -187,6 +187,9 @@ spec:
                         contentType:
                           description: the content type (tipically text or binary)
                           type: string
+                        from-kamelet:
+                          description: True if the spec is generated from a Kamelet
+                          type: boolean
                         interceptors:
                           description: Interceptors are optional identifiers the org.apache.camel.k.RoutesLoader
                             uses to pre/post process sources
@@ -7391,6 +7394,10 @@ spec:
                             description: Comma separated list of Kamelet names to
                               load into the current integration
                             type: string
+                          mountPoint:
+                            description: The directory where the application mounts
+                              and reads Kamelet spec (default `/etc/camel/kamelets`)
+                            type: string
                         type: object
                       keda:
                         description: 'Deprecated: for backward compatibility.'
diff --git a/pkg/apis/camel/v1/common_types.go b/pkg/apis/camel/v1/common_types.go
index f63f4db35..21c202f1a 100644
--- a/pkg/apis/camel/v1/common_types.go
+++ b/pkg/apis/camel/v1/common_types.go
@@ -414,6 +414,8 @@ type SourceSpec struct {
 	Type SourceType `json:"type,omitempty"`
 	// List of property names defined in the source (e.g. if type is "template")
 	PropertyNames []string `json:"property-names,omitempty"`
+	// True if the spec is generated from a Kamelet
+	FromKamelet bool `json:"from-kamelet,omitempty"`
 }
 
 // SourceType represents an available source type.
diff --git a/pkg/apis/camel/v1/common_types_support.go b/pkg/apis/camel/v1/common_types_support.go
index a7a956f98..c8adc7809 100644
--- a/pkg/apis/camel/v1/common_types_support.go
+++ b/pkg/apis/camel/v1/common_types_support.go
@@ -211,3 +211,8 @@ func DecodeValueSource(input string, defaultKey string, errorMessage string) (Va
 
 	return ValueSource{}, fmt.Errorf(errorMessage)
 }
+
+// IsGeneratedFromKamelet determines is a source spec is derived from a Kamelet
+func (s *SourceSpec) IsGeneratedFromKamelet() bool {
+	return s.FromKamelet
+}
diff --git a/pkg/apis/camel/v1/trait/kamelets.go b/pkg/apis/camel/v1/trait/kamelets.go
index 9467b4883..8b9cfd6b0 100644
--- a/pkg/apis/camel/v1/trait/kamelets.go
+++ b/pkg/apis/camel/v1/trait/kamelets.go
@@ -26,4 +26,6 @@ type KameletsTrait struct {
 	Auto *bool `property:"auto" json:"auto,omitempty"`
 	// Comma separated list of Kamelet names to load into the current integration
 	List string `property:"list" json:"list,omitempty"`
+	// The directory where the application mounts and reads Kamelet spec (default `/etc/camel/kamelets`)
+	MountPoint string `property:"mount-point" json:"mountPoint,omitempty"`
 }
diff --git a/pkg/client/camel/applyconfiguration/camel/v1/sourcespec.go b/pkg/client/camel/applyconfiguration/camel/v1/sourcespec.go
index 8c844b189..0cf5fdac3 100644
--- a/pkg/client/camel/applyconfiguration/camel/v1/sourcespec.go
+++ b/pkg/client/camel/applyconfiguration/camel/v1/sourcespec.go
@@ -32,6 +32,7 @@ type SourceSpecApplyConfiguration struct {
 	Interceptors               []string            `json:"interceptors,omitempty"`
 	Type                       *camelv1.SourceType `json:"type,omitempty"`
 	PropertyNames              []string            `json:"property-names,omitempty"`
+	FromKamelet                *bool               `json:"from-kamelet,omitempty"`
 }
 
 // SourceSpecApplyConfiguration constructs an declarative configuration of the SourceSpec type for use with
@@ -149,3 +150,11 @@ func (b *SourceSpecApplyConfiguration) WithPropertyNames(values ...string) *Sour
 	}
 	return b
 }
+
+// WithFromKamelet sets the FromKamelet field in the declarative configuration to the given value
+// and returns the receiver, so that objects can be built by chaining "With" function invocations.
+// If called multiple times, the FromKamelet field is set to the value of the last call.
+func (b *SourceSpecApplyConfiguration) WithFromKamelet(value bool) *SourceSpecApplyConfiguration {
+	b.FromKamelet = &value
+	return b
+}
diff --git a/pkg/trait/camel.go b/pkg/trait/camel.go
index 6065a1fb0..e96dd6fcb 100644
--- a/pkg/trait/camel.go
+++ b/pkg/trait/camel.go
@@ -219,6 +219,7 @@ func (t *camelTrait) computeConfigMaps(e *Environment) []ctrl.Object {
 					Labels: map[string]string{
 						v1.IntegrationLabel:                e.Integration.Name,
 						"camel.apache.org/properties.type": "user",
+						kubernetes.ConfigMapTypeLabel:      "camel-properties",
 					},
 				},
 				Data: map[string]string{
diff --git a/pkg/trait/kamelets.go b/pkg/trait/kamelets.go
index b2c410a59..705026397 100644
--- a/pkg/trait/kamelets.go
+++ b/pkg/trait/kamelets.go
@@ -20,6 +20,7 @@ package trait
 import (
 	"errors"
 	"fmt"
+	"path/filepath"
 	"sort"
 	"strconv"
 	"strings"
@@ -35,14 +36,16 @@ import (
 	"github.com/apache/camel-k/v2/pkg/kamelet/repository"
 	"github.com/apache/camel-k/v2/pkg/platform"
 	"github.com/apache/camel-k/v2/pkg/util"
+	"github.com/apache/camel-k/v2/pkg/util/camel"
 	"github.com/apache/camel-k/v2/pkg/util/digest"
 	"github.com/apache/camel-k/v2/pkg/util/dsl"
 	"github.com/apache/camel-k/v2/pkg/util/kamelets"
+	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
 )
 
 const (
-	contentKey = "content"
-
+	contentKey                = "content"
+	KameletLocationProperty   = "camel.component.kamelet.location"
 	kameletLabel              = "camel.apache.org/kamelet"
 	kameletConfigurationLabel = "camel.apache.org/kamelet.configuration"
 )
@@ -89,6 +92,10 @@ func (t *kameletsTrait) Configure(e *Environment) (bool, error) {
 			sort.Strings(kamelets)
 			t.List = strings.Join(kamelets, ",")
 		}
+
+		if t.MountPoint == "" {
+			t.MountPoint = filepath.Join(camel.BasePath, "kamelets")
+		}
 	}
 
 	return len(t.getKameletKeys()) > 0, nil
@@ -102,8 +109,6 @@ func (t *kameletsTrait) Apply(e *Environment) error {
 	}
 	if e.IntegrationInPhase(v1.IntegrationPhaseInitialization) {
 		return t.addConfigurationSecrets(e)
-	} else if e.IntegrationInRunningPhases() {
-		return t.configureApplicationProperties(e)
 	}
 
 	return nil
@@ -129,7 +134,6 @@ func (t *kameletsTrait) collectKamelets(e *Environment) (map[string]*v1.Kamelet,
 			missingKamelets = append(missingKamelets, key)
 		} else {
 			availableKamelets = append(availableKamelets, key)
-
 			// Initialize remote kamelets
 			kamelets[key], err = kameletutils.Initialize(kamelet)
 			if err != nil {
@@ -142,7 +146,7 @@ func (t *kameletsTrait) collectKamelets(e *Environment) (map[string]*v1.Kamelet,
 	sort.Strings(missingKamelets)
 
 	if len(missingKamelets) > 0 {
-		message := fmt.Sprintf("kamelets [%s] found, [%s] not found in repositories: %s",
+		message := fmt.Sprintf("kamelets [%s] found, kamelets [%s] not found in %s repositories",
 			strings.Join(availableKamelets, ","),
 			strings.Join(missingKamelets, ","),
 			repo.String())
@@ -161,7 +165,7 @@ func (t *kameletsTrait) collectKamelets(e *Environment) (map[string]*v1.Kamelet,
 		v1.IntegrationConditionKameletsAvailable,
 		corev1.ConditionTrue,
 		v1.IntegrationConditionKameletsAvailableReason,
-		fmt.Sprintf("kamelets [%s] found in repositories: %s", strings.Join(availableKamelets, ","), repo.String()),
+		fmt.Sprintf("kamelets [%s] found in %s repositories", strings.Join(availableKamelets, ","), repo.String()),
 	)
 
 	return kamelets, nil
@@ -173,59 +177,61 @@ func (t *kameletsTrait) addKamelets(e *Environment) error {
 		if err != nil {
 			return err
 		}
-
+		kameletsBundleConfigmap := initializeConfigmapBundle(e.Integration.Name, e.Integration.Namespace)
 		for _, key := range t.getKameletKeys() {
 			kamelet := kamelets[key]
-
 			if kamelet.Status.Phase != v1.KameletPhaseReady {
 				return fmt.Errorf("kamelet %q is not %s: %s", key, v1.KameletPhaseReady, kamelet.Status.Phase)
 			}
-
+			// Add source for parsing capabilities
 			if err := t.addKameletAsSource(e, kamelet); err != nil {
 				return err
 			}
-
-			// Adding dependencies from Kamelets
+			// Adding explicit dependencies from Kamelets
 			util.StringSliceUniqueConcat(&e.Integration.Status.Dependencies, kamelet.Spec.Dependencies)
+			// Add to Kamelet bundle configmap
+			serialized, err := kubernetes.ToYAMLNoManagedFields(kamelet)
+			if err != nil {
+				return err
+			}
+			kameletsBundleConfigmap.Data[fmt.Sprintf("%s.kamelet.yaml", kamelet.Name)] = string(serialized)
+		}
+		// set kamelets runtime location
+		if e.ApplicationProperties == nil {
+			e.ApplicationProperties = map[string]string{}
 		}
+		e.ApplicationProperties[KameletLocationProperty] = fmt.Sprintf("file:%s", t.MountPoint)
+		e.Resources.Add(&kameletsBundleConfigmap)
 		// resort dependencies
 		sort.Strings(e.Integration.Status.Dependencies)
 	}
 	return nil
 }
 
-func (t *kameletsTrait) configureApplicationProperties(e *Environment) error {
-	if len(t.getKameletKeys()) > 0 {
-		kamelets, err := t.collectKamelets(e)
-		if err != nil {
-			return err
-		}
-
-		for _, key := range t.getKameletKeys() {
-			kamelet := kamelets[key]
-			// Configuring defaults from Kamelet
-			for _, prop := range kamelet.Status.Properties {
-				if prop.Default != "" {
-					// Check whether user specified a value
-					userDefined := false
-					propName := fmt.Sprintf("camel.kamelet.%s.%s", kamelet.Name, prop.Name)
-					propPrefix := propName + "="
-					for _, userProp := range e.Integration.Spec.Configuration {
-						if strings.HasPrefix(userProp.Value, propPrefix) {
-							userDefined = true
-							break
-						}
-					}
-					if !userDefined {
-						e.ApplicationProperties[propName] = prop.Default
-					}
-				}
-			}
-		}
+func initializeConfigmapBundle(name, namespace string) corev1.ConfigMap {
+	return corev1.ConfigMap{
+		TypeMeta: metav1.TypeMeta{
+			Kind:       "ConfigMap",
+			APIVersion: "v1",
+		},
+		ObjectMeta: metav1.ObjectMeta{
+			Name:      fmt.Sprintf("kamelets-bundle-%s", name),
+			Namespace: namespace,
+			Labels: map[string]string{
+				v1.IntegrationLabel:           name,
+				kubernetes.ConfigMapTypeLabel: "kamelets-bundle",
+			},
+			Annotations: map[string]string{
+				kubernetes.ConfigMapAutogenLabel: "true",
+			},
+		},
+		Data: map[string]string{},
 	}
-	return nil
 }
 
+// This func will add a Kamelet as a generated Integration source. The source included here is going to be used in order to parse the Kamelet
+// for any component or capability (ie, rest) which is included in the Kamelet spec itself. However, the generated source is marked as coming `FromKamelet`.
+// When mounting the sources, these generated sources won't be mounted as sources but as Kamelet instead.
 func (t *kameletsTrait) addKameletAsSource(e *Environment, kamelet *v1.Kamelet) error {
 	sources := make([]v1.SourceSpec, 0)
 
@@ -241,7 +247,8 @@ func (t *kameletsTrait) addKameletAsSource(e *Environment, kamelet *v1.Kamelet)
 				Name:    fmt.Sprintf("%s.yaml", kamelet.Name),
 				Content: string(flowData),
 			},
-			Language: v1.LanguageYaml,
+			Language:    v1.LanguageYaml,
+			FromKamelet: true,
 		}
 		flowSource, err = integrationSourceFromKameletSource(e, kamelet, flowSource, fmt.Sprintf("%s-kamelet-%s-template", e.Integration.Name, kamelet.Name))
 		if err != nil {
@@ -363,18 +370,28 @@ func integrationSourceFromKameletSource(e *Environment, kamelet *v1.Kamelet, sou
 	if err != nil {
 		return v1.SourceSpec{}, err
 	}
+	cm := initializeConfigmapKameletSource(source, hash, name, e.Integration.Namespace, e.Integration.Name, kamelet.Name)
+	e.Resources.Add(&cm)
 
-	cm := corev1.ConfigMap{
+	target := source.DeepCopy()
+	target.Content = ""
+	target.ContentRef = name
+	target.ContentKey = contentKey
+	return *target, nil
+}
+
+func initializeConfigmapKameletSource(source v1.SourceSpec, hash, name, namespace, itName, kamName string) corev1.ConfigMap {
+	return corev1.ConfigMap{
 		TypeMeta: metav1.TypeMeta{
 			Kind:       "ConfigMap",
 			APIVersion: "v1",
 		},
 		ObjectMeta: metav1.ObjectMeta{
 			Name:      name,
-			Namespace: e.Integration.Namespace,
+			Namespace: namespace,
 			Labels: map[string]string{
-				"camel.apache.org/integration": e.Integration.Name,
-				"camel.apache.org/kamelet":     kamelet.Name,
+				"camel.apache.org/integration": itName,
+				"camel.apache.org/kamelet":     kamName,
 			},
 			Annotations: map[string]string{
 				"camel.apache.org/source.language":    string(source.Language),
@@ -389,12 +406,4 @@ func integrationSourceFromKameletSource(e *Environment, kamelet *v1.Kamelet, sou
 			contentKey: source.Content,
 		},
 	}
-
-	e.Resources.Add(&cm)
-
-	target := source.DeepCopy()
-	target.Content = ""
-	target.ContentRef = name
-	target.ContentKey = contentKey
-	return *target, nil
 }
diff --git a/pkg/trait/kamelets_test.go b/pkg/trait/kamelets_test.go
index a91bc9c71..96a80b5a7 100644
--- a/pkg/trait/kamelets_test.go
+++ b/pkg/trait/kamelets_test.go
@@ -109,7 +109,7 @@ func TestKameletLookup(t *testing.T) {
 	require.NoError(t, err)
 	cm := environment.Resources.GetConfigMap(func(_ *corev1.ConfigMap) bool { return true })
 	assert.NotNil(t, cm)
-	assert.Equal(t, "it-kamelet-timer-template", cm.Name)
+	assert.Equal(t, "kamelets-bundle-it", cm.Name)
 	assert.Equal(t, "test", cm.Namespace)
 
 	assert.Len(t, environment.Integration.Status.GeneratedSources, 1)
@@ -209,7 +209,7 @@ func TestNonYAMLKameletLookup(t *testing.T) {
 	require.NoError(t, err)
 	cm := environment.Resources.GetConfigMap(func(_ *corev1.ConfigMap) bool { return true })
 	assert.NotNil(t, cm)
-	assert.Equal(t, "it-kamelet-timer-000", cm.Name)
+	assert.Equal(t, "kamelets-bundle-it", cm.Name)
 	assert.Equal(t, "test", cm.Namespace)
 
 	assert.Len(t, environment.Integration.Status.GeneratedSources, 1)
@@ -480,7 +480,7 @@ func TestKameletConditionFalse(t *testing.T) {
 	assert.Equal(t, corev1.ConditionFalse, cond.Status)
 	assert.Equal(t, v1.IntegrationConditionKameletsAvailableReason, cond.Reason)
 	assert.Contains(t, cond.Message, "[timer] found")
-	assert.Contains(t, cond.Message, "[none] not found")
+	assert.Contains(t, cond.Message, "kamelets [none] not found")
 }
 
 func TestKameletConditionTrue(t *testing.T) {
diff --git a/pkg/trait/trait_types.go b/pkg/trait/trait_types.go
index 614f6fdc8..05ee587ce 100644
--- a/pkg/trait/trait_types.go
+++ b/pkg/trait/trait_types.go
@@ -419,6 +419,7 @@ func (e *Environment) computeApplicationProperties() (*corev1.ConfigMap, error)
 				Labels: map[string]string{
 					v1.IntegrationLabel:                e.Integration.Name,
 					"camel.apache.org/properties.type": "application",
+					kubernetes.ConfigMapTypeLabel:      "camel-properties",
 				},
 			},
 			Data: map[string]string{
@@ -436,7 +437,8 @@ func (e *Environment) addSourcesProperties() {
 	}
 	idx := 0
 	for _, s := range e.Integration.Sources() {
-		if e.isEmbedded(s) {
+		// We don't process routes embedded (native) or Kamelets
+		if e.isEmbedded(s) || s.IsGeneratedFromKamelet() {
 			continue
 		}
 		srcName := strings.TrimPrefix(filepath.ToSlash(s.Name), "/")
@@ -481,14 +483,14 @@ func (e *Environment) addSourcesProperties() {
 }
 
 func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]corev1.VolumeMount) {
-	//
-	// Volumes :: Sources
-	//
+	// Sources
 	idx := 0
 	for _, s := range e.Integration.Sources() {
-		if e.isEmbedded(s) {
+		// We don't process routes embedded (native) or Kamelets
+		if e.isEmbedded(s) || s.IsGeneratedFromKamelet() {
 			continue
 		}
+		// Routes are copied under /etc/camel/sources and discovered by the runtime accordingly
 		cmName := fmt.Sprintf("%s-source-%03d", e.Integration.Name, idx)
 		if s.ContentRef != "" {
 			cmName = s.ContentRef
@@ -507,24 +509,38 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 		*mnts = append(*mnts, *mnt)
 		idx++
 	}
-
+	// Resources (likely application properties or kamelets)
 	if e.Resources != nil {
 		e.Resources.VisitConfigMap(func(configMap *corev1.ConfigMap) {
-			propertiesType := configMap.Labels["camel.apache.org/properties.type"]
-			resName := propertiesType + ".properties"
-
-			var mountPath string
-			switch propertiesType {
-			case "application":
-				mountPath = filepath.Join(camel.BasePath, resName)
-			case "user":
-				mountPath = filepath.Join(camel.ConfDPath, resName)
-			}
-
-			if propertiesType != "" {
-				refName := propertiesType + "-properties"
-				vol := getVolume(refName, "configmap", configMap.Name, "application.properties", resName)
-				mnt := getMount(refName, mountPath, resName, true)
+			// Camel properties
+			if configMap.Labels[kubernetes.ConfigMapTypeLabel] == "camel-properties" {
+				propertiesType := configMap.Labels["camel.apache.org/properties.type"]
+				resName := propertiesType + ".properties"
+
+				var mountPath string
+				switch propertiesType {
+				case "application":
+					mountPath = filepath.Join(camel.BasePath, resName)
+				case "user":
+					mountPath = filepath.Join(camel.ConfDPath, resName)
+				}
+
+				if propertiesType != "" {
+					refName := propertiesType + "-properties"
+					vol := getVolume(refName, "configmap", configMap.Name, "application.properties", resName)
+					mnt := getMount(refName, mountPath, resName, true)
+
+					*vols = append(*vols, *vol)
+					*mnts = append(*mnts, *mnt)
+				} else {
+					log.WithValues("Function", "trait.configureVolumesAndMounts").Infof("Warning: could not determine camel properties type %s", propertiesType)
+				}
+			} else if configMap.Labels[kubernetes.ConfigMapTypeLabel] == "kamelets-bundle" {
+				// Kamelets bundle configmap
+				kameletMountPoint := strings.ReplaceAll(e.ApplicationProperties[KameletLocationProperty], "file:", "")
+				refName := "kamelets-bundle"
+				vol := getVolume(refName, "configmap", configMap.Name, "", "")
+				mnt := getMount(refName, kameletMountPoint, "", true)
 
 				*vols = append(*vols, *vol)
 				*mnts = append(*mnts, *mnt)
@@ -532,9 +548,8 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 		})
 	}
 
-	//
-	// Volumes :: Additional ConfigMaps
-	//
+	// Deprecated - should use mount trait
+	// User provided configmaps
 	for _, configmaps := range e.collectConfigurations("configmap") {
 		refName := kubernetes.SanitizeLabel(configmaps["value"])
 		mountPath := getMountPoint(configmaps["value"], configmaps["resourceMountPoint"], "configmap", configmaps["resourceType"])
@@ -545,9 +560,8 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 		*mnts = append(*mnts, *mnt)
 	}
 
-	//
-	// Volumes :: Additional Secrets
-	//
+	// Deprecated - should use mount trait
+	// User provided secrets
 	for _, secret := range e.collectConfigurations("secret") {
 		refName := kubernetes.SanitizeLabel(secret["value"])
 		mountPath := getMountPoint(secret["value"], secret["resourceMountPoint"], "secret", secret["resourceType"])
@@ -568,10 +582,8 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 		*vols = append(*vols, *vol)
 		*mnts = append(*mnts, *mnt)
 	}
-
-	//
-	// Volumes :: Additional user provided volumes
-	//
+	// Deprecated - should use mount trait
+	// User provided volumes
 	for _, volumeConfig := range e.collectConfigurationValues("volume") {
 		configParts := strings.Split(volumeConfig, ":")
 
diff --git a/pkg/util/kubernetes/factory.go b/pkg/util/kubernetes/factory.go
index 1915c76d9..5bf26ce12 100644
--- a/pkg/util/kubernetes/factory.go
+++ b/pkg/util/kubernetes/factory.go
@@ -39,6 +39,9 @@ const ConfigMapAutogenLabel = "camel.apache.org/generated"
 // ConfigMapOriginalFileNameLabel -- .
 const ConfigMapOriginalFileNameLabel = "camel.apache.org/filename"
 
+// ConfigMapTypeLabel -- .
+const ConfigMapTypeLabel = "camel.apache.org/config.type"
+
 // NewTolerations build an array of Tolerations from an array of string.
 func NewTolerations(taints []string) ([]corev1.Toleration, error) {
 	tolerations := make([]corev1.Toleration, 0)
diff --git a/resources/traits.yaml b/resources/traits.yaml
index 4489f2040..6b9357beb 100755
--- a/resources/traits.yaml
+++ b/resources/traits.yaml
@@ -909,6 +909,10 @@ traits:
   - name: list
     type: string
     description: Comma separated list of Kamelet names to load into the current integration
+  - name: mount-point
+    type: string
+    description: The directory where the application mounts and reads Kamelet spec
+      (default `/etc/camel/kamelets`)
 - name: keda
   platform: false
   profiles:


[camel-k] 04/06: fix(trait): configmap identifier

Posted by pc...@apache.org.
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

commit 571928270d5311767ac8d2684df02a060c3ebed7
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Tue Oct 10 16:36:10 2023 +0200

    fix(trait): configmap identifier
---
 pkg/trait/camel.go      | 6 ++++--
 pkg/trait/camel_test.go | 2 +-
 2 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/pkg/trait/camel.go b/pkg/trait/camel.go
index e96dd6fcb..36e3d74b4 100644
--- a/pkg/trait/camel.go
+++ b/pkg/trait/camel.go
@@ -229,8 +229,9 @@ func (t *camelTrait) computeConfigMaps(e *Environment) []ctrl.Object {
 		)
 	}
 
-	for i, s := range sources {
-		if s.ContentRef != "" || e.isEmbedded(s) {
+	i := 0
+	for _, s := range sources {
+		if s.ContentRef != "" || e.isEmbedded(s) || s.IsGeneratedFromKamelet() {
 			continue
 		}
 
@@ -258,6 +259,7 @@ func (t *camelTrait) computeConfigMaps(e *Environment) []ctrl.Object {
 		}
 
 		maps = append(maps, &cm)
+		i++
 	}
 
 	return maps
diff --git a/pkg/trait/camel_test.go b/pkg/trait/camel_test.go
index f6edf5eaa..692c5df21 100644
--- a/pkg/trait/camel_test.go
+++ b/pkg/trait/camel_test.go
@@ -195,7 +195,7 @@ func TestApplyCamelTraitWithSources(t *testing.T) {
 
 	assert.Equal(t, 1, environment.Resources.Size())
 	sourceCm := environment.Resources.GetConfigMap(func(cm *corev1.ConfigMap) bool {
-		return cm.Name == "some-integration-source-001" && cm.Annotations["camel.apache.org/source.language"] == "xml" && cm.Annotations["camel.apache.org/source.name"] == "source2.xml"
+		return cm.Name == "some-integration-source-000" && cm.Annotations["camel.apache.org/source.language"] == "xml" && cm.Annotations["camel.apache.org/source.name"] == "source2.xml"
 	})
 	assert.NotNil(t, sourceCm)
 	assert.Equal(t, map[string]string{


[camel-k] 06/06: feat(trait): partition kamelets

Posted by pc...@apache.org.
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

commit 167d6afb27fac407160695189790da3a5e9f331a
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Fri Oct 13 15:21:04 2023 +0200

    feat(trait): partition kamelets
    
    Configmap limit is 1 MB. We partition into more configmaps if the bundle exceed such limit.
---
 docs/modules/ROOT/pages/architecture/kamelets.adoc |   2 +
 pkg/trait/kamelets.go                              |  56 ++++------
 pkg/trait/kamelets_support.go                      |  91 ++++++++++++++++
 pkg/trait/kamelets_support_test.go                 | 117 +++++++++++++++++++++
 pkg/trait/kamelets_test.go                         |   4 +-
 pkg/trait/trait_types.go                           |   4 +-
 6 files changed, 235 insertions(+), 39 deletions(-)

diff --git a/docs/modules/ROOT/pages/architecture/kamelets.adoc b/docs/modules/ROOT/pages/architecture/kamelets.adoc
index d4006e072..6efa7cd8d 100644
--- a/docs/modules/ROOT/pages/architecture/kamelets.adoc
+++ b/docs/modules/ROOT/pages/architecture/kamelets.adoc
@@ -13,6 +13,8 @@ image::architecture/kamelets_deployment.png[Kamelets deployment model]
 
 The operator creates a ConfigMap in order to bundle all the Kamelets which are eventually required by the application runtime. The Kamelets spec has to be available and in ready phase status. Once the application is created and ready to start, the operator mounts such a ConfigMap in a known location (default `/etc/camel/kamelets`) so that the Camel application will be able to read the definition from such location and run them according the logic expected in the same Camel framework.
 
+NOTE: as the Configmap resource is limited to 1 MiB, the operator may split into more than a single Configmap bundle.
+
 [[kamelet-parsing]]
 === Parsing capabilities defined in a Kamelet
 
diff --git a/pkg/trait/kamelets.go b/pkg/trait/kamelets.go
index 574a8b327..46629b24f 100644
--- a/pkg/trait/kamelets.go
+++ b/pkg/trait/kamelets.go
@@ -40,14 +40,14 @@ import (
 	"github.com/apache/camel-k/v2/pkg/util/digest"
 	"github.com/apache/camel-k/v2/pkg/util/dsl"
 	"github.com/apache/camel-k/v2/pkg/util/kamelets"
-	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
 )
 
 const (
-	contentKey                = "content"
-	KameletLocationProperty   = "camel.component.kamelet.location"
-	kameletLabel              = "camel.apache.org/kamelet"
-	kameletConfigurationLabel = "camel.apache.org/kamelet.configuration"
+	contentKey                  = "content"
+	KameletLocationProperty     = "camel.component.kamelet.location"
+	kameletLabel                = "camel.apache.org/kamelet"
+	kameletConfigurationLabel   = "camel.apache.org/kamelet.configuration"
+	kameletMountPointAnnotation = "camel.apache.org/kamelet.mount-point"
 )
 
 type configurationKey struct {
@@ -177,7 +177,7 @@ func (t *kameletsTrait) addKamelets(e *Environment) error {
 		if err != nil {
 			return err
 		}
-		kameletsBundleConfigmap := initializeConfigmapBundle(e.Integration.Name, e.Integration.Namespace)
+		kb := newKameletBundle()
 		for _, key := range t.getKameletKeys() {
 			kamelet := kamelets[key]
 			if kamelet.Status.Phase != v1.KameletPhaseReady {
@@ -190,45 +190,33 @@ func (t *kameletsTrait) addKamelets(e *Environment) error {
 			// Adding explicit dependencies from Kamelets
 			util.StringSliceUniqueConcat(&e.Integration.Status.Dependencies, kamelet.Spec.Dependencies)
 			// Add to Kamelet bundle configmap
-			serialized, err := kubernetes.ToYAMLNoManagedFields(kamelet)
-			if err != nil {
-				return err
-			}
-			kameletsBundleConfigmap.Data[fmt.Sprintf("%s.kamelet.yaml", kamelet.Name)] = string(serialized)
+			kb.add(kamelet)
+		}
+		bundleConfigmaps, err := kb.toConfigmaps(e.Integration.Name, e.Integration.Namespace)
+		if err != nil {
+			return err
 		}
 		// set kamelets runtime location
 		if e.ApplicationProperties == nil {
 			e.ApplicationProperties = map[string]string{}
 		}
-		e.ApplicationProperties[KameletLocationProperty] = fmt.Sprintf("file:%s,classpath:/kamelets", t.MountPoint)
-		e.Resources.Add(&kameletsBundleConfigmap)
+		for _, cm := range bundleConfigmaps {
+			kameletMountPoint := fmt.Sprintf("%s/%s", t.MountPoint, cm.Name)
+			cm.Annotations[kameletMountPointAnnotation] = kameletMountPoint
+			e.Resources.Add(cm)
+			if e.ApplicationProperties[KameletLocationProperty] == "" {
+				e.ApplicationProperties[KameletLocationProperty] = fmt.Sprintf("file:%s", kameletMountPoint)
+			} else {
+				e.ApplicationProperties[KameletLocationProperty] += fmt.Sprintf(",file:%s", kameletMountPoint)
+			}
+		}
+		e.ApplicationProperties[KameletLocationProperty] += ",classpath:/kamelets"
 		// resort dependencies
 		sort.Strings(e.Integration.Status.Dependencies)
 	}
 	return nil
 }
 
-func initializeConfigmapBundle(name, namespace string) corev1.ConfigMap {
-	return corev1.ConfigMap{
-		TypeMeta: metav1.TypeMeta{
-			Kind:       "ConfigMap",
-			APIVersion: "v1",
-		},
-		ObjectMeta: metav1.ObjectMeta{
-			Name:      fmt.Sprintf("kamelets-bundle-%s", name),
-			Namespace: namespace,
-			Labels: map[string]string{
-				v1.IntegrationLabel:           name,
-				kubernetes.ConfigMapTypeLabel: "kamelets-bundle",
-			},
-			Annotations: map[string]string{
-				kubernetes.ConfigMapAutogenLabel: "true",
-			},
-		},
-		Data: map[string]string{},
-	}
-}
-
 // This func will add a Kamelet as a generated Integration source. The source included here is going to be used in order to parse the Kamelet
 // for any component or capability (ie, rest) which is included in the Kamelet spec itself. However, the generated source is marked as coming `FromKamelet`.
 // When mounting the sources, these generated sources won't be mounted as sources but as Kamelet instead.
diff --git a/pkg/trait/kamelets_support.go b/pkg/trait/kamelets_support.go
new file mode 100644
index 000000000..147a0c9c4
--- /dev/null
+++ b/pkg/trait/kamelets_support.go
@@ -0,0 +1,91 @@
+/*
+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 (
+	"fmt"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/cmd/source"
+	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
+	corev1 "k8s.io/api/core/v1"
+	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+)
+
+type kameletBundle struct {
+	kamelets []*v1.Kamelet
+}
+
+func newKameletBundle() *kameletBundle {
+	return &kameletBundle{
+		kamelets: make([]*v1.Kamelet, 0),
+	}
+}
+
+func (kb *kameletBundle) add(k *v1.Kamelet) {
+	kb.kamelets = append(kb.kamelets, k)
+}
+
+// Split the contents of the Kamelets into one ore more configmap, making sure not to overpass the 1 MB limit.
+func (kb *kameletBundle) toConfigmaps(itName, itNamespace string) ([]*corev1.ConfigMap, error) {
+	configmaps := make([]*corev1.ConfigMap, 0)
+	cmSize := 0
+	cmID := 1
+	cm := newBundleConfigmap(itName, itNamespace, cmID)
+	for _, k := range kb.kamelets {
+		serialized, err := kubernetes.ToYAMLNoManagedFields(k)
+		if err != nil {
+			return nil, err
+		}
+		// Add if it fits into a configmap, otherwise, create a new configmap
+		if cmSize+len(serialized) > source.Megabyte {
+			configmaps = append(configmaps, cm)
+			// create a new configmap
+			cmSize = 0
+			cmID++
+			cm = newBundleConfigmap(itName, itNamespace, cmID)
+		}
+		cm.Data[fmt.Sprintf("%s.kamelet.yaml", k.Name)] = string(serialized)
+		cmSize += len(serialized)
+	}
+	// Add the last configmap
+	configmaps = append(configmaps, cm)
+
+	return configmaps, nil
+}
+
+func newBundleConfigmap(name, namespace string, id int) *corev1.ConfigMap {
+	return &corev1.ConfigMap{
+		TypeMeta: metav1.TypeMeta{
+			Kind:       "ConfigMap",
+			APIVersion: "v1",
+		},
+		ObjectMeta: metav1.ObjectMeta{
+			Name:      fmt.Sprintf("kamelets-bundle-%s-%03d", name, id),
+			Namespace: namespace,
+			Labels: map[string]string{
+				v1.IntegrationLabel:           name,
+				kubernetes.ConfigMapTypeLabel: "kamelets-bundle",
+			},
+			Annotations: map[string]string{
+				kubernetes.ConfigMapAutogenLabel: "true",
+			},
+		},
+		Data: map[string]string{},
+	}
+}
diff --git a/pkg/trait/kamelets_support_test.go b/pkg/trait/kamelets_support_test.go
new file mode 100644
index 000000000..2d3894154
--- /dev/null
+++ b/pkg/trait/kamelets_support_test.go
@@ -0,0 +1,117 @@
+/*
+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 (
+	"fmt"
+	"testing"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestKameletBundleSingle(t *testing.T) {
+	kb := newKameletBundle()
+	kb.add(kamelet("my-ns", "test"))
+	cmBundle, err := kb.toConfigmaps("my-it", "my-ns")
+	assert.Nil(t, err)
+	assert.NotNil(t, cmBundle)
+	assert.Len(t, cmBundle, 1)
+	assert.Equal(t, "my-ns", cmBundle[0].Namespace)
+	assert.Equal(t, "kamelets-bundle-my-it-001", cmBundle[0].Name)
+	assert.Equal(t, "my-it", cmBundle[0].Labels[v1.IntegrationLabel])
+	assert.NotNil(t, cmBundle[0].Data["test.kamelet.yaml"])
+}
+
+func TestKameletBundleMultiKameletsSingleConfigmap(t *testing.T) {
+	kb := newKameletBundle()
+	kb.add(kamelet("default", "test1"))
+	kb.add(kamelet("default", "test2"))
+	kb.add(kamelet("default", "test3"))
+	kb.add(kamelet("default", "test4"))
+	kb.add(kamelet("default", "test5"))
+	kb.add(kamelet("default", "test6"))
+	cmBundle, err := kb.toConfigmaps("my-it", "default")
+	assert.Nil(t, err)
+	assert.NotNil(t, cmBundle)
+	assert.Len(t, cmBundle, 1)
+	assert.Equal(t, "default", cmBundle[0].Namespace)
+	assert.Equal(t, "kamelets-bundle-my-it-001", cmBundle[0].Name)
+	assert.Equal(t, "my-it", cmBundle[0].Labels[v1.IntegrationLabel])
+	assert.Len(t, cmBundle[0].Data, 6)
+	assert.NotNil(t, cmBundle[0].Data["test1.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[0].Data["test2.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[0].Data["test3.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[0].Data["test4.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[0].Data["test5.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[0].Data["test6.kamelet.yaml"])
+}
+
+func TestKameletBundleMultiKameletsMultiConfigmap(t *testing.T) {
+	kb := newKameletBundle()
+	for i := 0; i < 2000; i++ {
+		kb.add(kamelet("default", fmt.Sprintf("test%d", i)))
+	}
+	cmBundle, err := kb.toConfigmaps("my-it", "default")
+	assert.Nil(t, err)
+	assert.NotNil(t, cmBundle)
+	assert.Len(t, cmBundle, 2)
+	assert.Equal(t, "default", cmBundle[0].Namespace)
+	assert.Equal(t, "kamelets-bundle-my-it-001", cmBundle[0].Name)
+	assert.Equal(t, "my-it", cmBundle[0].Labels[v1.IntegrationLabel])
+	assert.Equal(t, "default", cmBundle[1].Namespace)
+	assert.Equal(t, "kamelets-bundle-my-it-002", cmBundle[1].Name)
+	assert.Equal(t, "my-it", cmBundle[1].Labels[v1.IntegrationLabel])
+	assert.Equal(t, 2000, len(cmBundle[0].Data)+len(cmBundle[1].Data))
+	assert.NotNil(t, cmBundle[0].Data["test1.kamelet.yaml"])
+	assert.NotNil(t, cmBundle[1].Data["test1999.kamelet.yaml"])
+}
+
+func kamelet(ns, name string) *v1.Kamelet {
+	kamelet := v1.NewKamelet(ns, name)
+	kamelet.Spec = v1.KameletSpec{
+		Sources: []v1.SourceSpec{
+			{
+				DataSpec: v1.DataSpec{
+					Name: "mykamelet.groovy",
+					Content: `from("timer1").to("log:info")
+					from("timer2").to("log:info")
+					from("timer3").to("log:info")
+					from("timer4").to("log:info")
+					from("timer5").to("log:info")
+					from("timer6").to("log:info")
+					from("timer7").to("log:info")
+					from("timer8").to("log:info")
+					from("timer9").to("log:info")
+					from("timer10").to("log:info")
+					from("timer11").to("log:info")
+					from("timer12").to("log:info")
+					from("timer13").to("log:info")
+					from("timer14").to("log:info")
+					from("timer15").to("log:info")
+					from("timer16").to("log:info")
+					from("timer17").to("log:info")`,
+				},
+				Type: v1.SourceTypeTemplate,
+			},
+		},
+	}
+	kamelet.Status = v1.KameletStatus{Phase: v1.KameletPhaseReady}
+
+	return &kamelet
+}
diff --git a/pkg/trait/kamelets_test.go b/pkg/trait/kamelets_test.go
index 96a80b5a7..483fcddc9 100644
--- a/pkg/trait/kamelets_test.go
+++ b/pkg/trait/kamelets_test.go
@@ -109,7 +109,7 @@ func TestKameletLookup(t *testing.T) {
 	require.NoError(t, err)
 	cm := environment.Resources.GetConfigMap(func(_ *corev1.ConfigMap) bool { return true })
 	assert.NotNil(t, cm)
-	assert.Equal(t, "kamelets-bundle-it", cm.Name)
+	assert.Equal(t, "kamelets-bundle-it-001", cm.Name)
 	assert.Equal(t, "test", cm.Namespace)
 
 	assert.Len(t, environment.Integration.Status.GeneratedSources, 1)
@@ -209,7 +209,7 @@ func TestNonYAMLKameletLookup(t *testing.T) {
 	require.NoError(t, err)
 	cm := environment.Resources.GetConfigMap(func(_ *corev1.ConfigMap) bool { return true })
 	assert.NotNil(t, cm)
-	assert.Equal(t, "kamelets-bundle-it", cm.Name)
+	assert.Equal(t, "kamelets-bundle-it-001", cm.Name)
 	assert.Equal(t, "test", cm.Namespace)
 
 	assert.Len(t, environment.Integration.Status.GeneratedSources, 1)
diff --git a/pkg/trait/trait_types.go b/pkg/trait/trait_types.go
index fca860d74..9471ddb66 100644
--- a/pkg/trait/trait_types.go
+++ b/pkg/trait/trait_types.go
@@ -537,9 +537,7 @@ func (e *Environment) configureVolumesAndMounts(vols *[]corev1.Volume, mnts *[]c
 				}
 			} else if configMap.Labels[kubernetes.ConfigMapTypeLabel] == "kamelets-bundle" {
 				// Kamelets bundle configmap
-				kameletMountPoint := strings.ReplaceAll(e.ApplicationProperties[KameletLocationProperty], "file:", "")
-				// We need also to remove the default location provided
-				kameletMountPoint = strings.ReplaceAll(kameletMountPoint, ",classpath:/kamelets", "")
+				kameletMountPoint := configMap.Annotations[kameletMountPointAnnotation]
 				refName := "kamelets-bundle"
 				vol := getVolume(refName, "configmap", configMap.Name, "", "")
 				mnt := getMount(refName, kameletMountPoint, "", true)