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/04 10:53:06 UTC

[camel-k] 08/09: fix(trait): run old native process for runtime pre 3.5.0

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 de0d1af6345a11106ef9207f32fa3e5debd53039
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Mon Oct 2 16:43:57 2023 +0200

    fix(trait): run old native process for runtime pre 3.5.0
---
 .../bases/camel.apache.org_integrationkits.yaml    |  1 -
 .../camel.apache.org_integrationplatforms.yaml     |  2 -
 .../crd/bases/camel.apache.org_integrations.yaml   |  1 -
 .../bases/camel.apache.org_kameletbindings.yaml    |  1 -
 config/crd/bases/camel.apache.org_pipes.yaml       |  1 -
 docs/modules/ROOT/pages/pipeline/pipeline.adoc     |  2 +-
 e2e/common/traits/builder_test.go                  |  5 +-
 .../catalog_builder_test.go                        |  6 +-
 e2e/native/native_test.go                          | 12 +--
 e2e/native/native_with_sources_test.go             | 12 ++-
 helm/camel-k/crds/crd-integration-kit.yaml         |  1 -
 helm/camel-k/crds/crd-integration-platform.yaml    |  2 -
 helm/camel-k/crds/crd-integration.yaml             |  1 -
 helm/camel-k/crds/crd-kamelet-binding.yaml         |  1 -
 helm/camel-k/crds/crd-pipe.yaml                    |  1 -
 pkg/apis/camel/v1/trait/quarkus.go                 |  2 +-
 pkg/builder/image.go                               |  2 +-
 pkg/builder/runtime_support.go                     | 91 ++++++++++++++++++++++
 pkg/trait/builder.go                               | 10 +--
 pkg/trait/builder_test.go                          |  9 ++-
 pkg/trait/quarkus.go                               |  9 ++-
 21 files changed, 127 insertions(+), 45 deletions(-)

diff --git a/config/crd/bases/camel.apache.org_integrationkits.yaml b/config/crd/bases/camel.apache.org_integrationkits.yaml
index f40c04244..b796bc8f4 100644
--- a/config/crd/bases/camel.apache.org_integrationkits.yaml
+++ b/config/crd/bases/camel.apache.org_integrationkits.yaml
@@ -350,7 +350,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/config/crd/bases/camel.apache.org_integrationplatforms.yaml b/config/crd/bases/camel.apache.org_integrationplatforms.yaml
index b11b568ed..39827cdd4 100644
--- a/config/crd/bases/camel.apache.org_integrationplatforms.yaml
+++ b/config/crd/bases/camel.apache.org_integrationplatforms.yaml
@@ -1643,7 +1643,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
@@ -3457,7 +3456,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/config/crd/bases/camel.apache.org_integrations.yaml b/config/crd/bases/camel.apache.org_integrations.yaml
index e59cb1c45..21eb27225 100644
--- a/config/crd/bases/camel.apache.org_integrations.yaml
+++ b/config/crd/bases/camel.apache.org_integrations.yaml
@@ -7560,7 +7560,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/config/crd/bases/camel.apache.org_kameletbindings.yaml b/config/crd/bases/camel.apache.org_kameletbindings.yaml
index abde245c9..f51b72726 100644
--- a/config/crd/bases/camel.apache.org_kameletbindings.yaml
+++ b/config/crd/bases/camel.apache.org_kameletbindings.yaml
@@ -7858,7 +7858,6 @@ spec:
                                 build packaging. Deprecated: use `QuarkusMode` instead.'
                               enum:
                               - fast-jar
-                              - native-sources
                               - native
                               type: string
                             type: array
diff --git a/config/crd/bases/camel.apache.org_pipes.yaml b/config/crd/bases/camel.apache.org_pipes.yaml
index a8a40da19..f58e4be83 100644
--- a/config/crd/bases/camel.apache.org_pipes.yaml
+++ b/config/crd/bases/camel.apache.org_pipes.yaml
@@ -7855,7 +7855,6 @@ spec:
                                 build packaging. Deprecated: use `QuarkusMode` instead.'
                               enum:
                               - fast-jar
-                              - native-sources
                               - native
                               type: string
                             type: array
diff --git a/docs/modules/ROOT/pages/pipeline/pipeline.adoc b/docs/modules/ROOT/pages/pipeline/pipeline.adoc
index 85bbdf3a8..b204f338c 100644
--- a/docs/modules/ROOT/pages/pipeline/pipeline.adoc
+++ b/docs/modules/ROOT/pages/pipeline/pipeline.adoc
@@ -4,7 +4,7 @@ Since version 2.0, we've introduced the concept of `Pipeline` in order to provid
 
 In Camel K version 1 we used to have 2 static tasks: `builder` and `publisher` (named after the strategy adopted, ie, `spectrum`). Now you can include any further task in the middle by opportunely configuring the Build. Here a diagram illustrating the pipeline:
 
-image::camel_k_pipeline..png[Camel K Pipeline, width=1024]
+image::camel_k_pipeline.png[Camel K Pipeline, width=1024]
 
 We have 3 tasks which are required by Camel K to perform a build and provide a container image which will be used to start a Camel application. The **build** is a Maven process which is in charge to create a maven project based on the Integration sources provided. Then, after the Maven project is created and compiled, you can add any custom task (see section below). With this part we are introducing that level of flexibility required to accommodate any company build process. This part is [...]
 
diff --git a/e2e/common/traits/builder_test.go b/e2e/common/traits/builder_test.go
index 6b1419174..0062c401e 100644
--- a/e2e/common/traits/builder_test.go
+++ b/e2e/common/traits/builder_test.go
@@ -173,7 +173,7 @@ func TestBuilderTrait(t *testing.T) {
 		integrationKitNamespace := IntegrationKitNamespace(ns, name)()
 		builderKitName := fmt.Sprintf("camel-k-%s-builder", integrationKitName)
 		Eventually(BuilderPod(integrationKitNamespace, builderKitName), TestTimeoutShort).ShouldNot(BeNil())
-		Eventually(len(BuilderPod(integrationKitNamespace, builderKitName)().Spec.InitContainers), TestTimeoutShort).Should(Equal(3))
+		Eventually(len(BuilderPod(integrationKitNamespace, builderKitName)().Spec.InitContainers), TestTimeoutShort).Should(Equal(4))
 		Eventually(BuilderPod(integrationKitNamespace, builderKitName)().Spec.InitContainers[0].Name, TestTimeoutShort).Should(Equal("builder"))
 		Eventually(BuilderPod(integrationKitNamespace, builderKitName)().Spec.InitContainers[1].Name, TestTimeoutShort).Should(Equal("custom1"))
 		Eventually(BuilderPod(integrationKitNamespace, builderKitName)().Spec.InitContainers[2].Name, TestTimeoutShort).Should(Equal("custom2"))
@@ -249,9 +249,10 @@ func TestBuilderTrait(t *testing.T) {
 		integrationKitName := IntegrationKit(ns, name)()
 		builderKitName := fmt.Sprintf("camel-k-%s-builder", integrationKitName)
 		Eventually(BuilderPod(ns, builderKitName), TestTimeoutShort).ShouldNot(BeNil())
-		Eventually(len(BuilderPod(ns, builderKitName)().Spec.InitContainers), TestTimeoutShort).Should(Equal(2))
+		Eventually(len(BuilderPod(ns, builderKitName)().Spec.InitContainers), TestTimeoutShort).Should(Equal(3))
 		Eventually(BuilderPod(ns, builderKitName)().Spec.InitContainers[0].Name, TestTimeoutShort).Should(Equal("builder"))
 		Eventually(BuilderPod(ns, builderKitName)().Spec.InitContainers[1].Name, TestTimeoutShort).Should(Equal("custom1"))
+		Eventually(BuilderPod(ns, builderKitName)().Spec.InitContainers[2].Name, TestTimeoutShort).Should(Equal("package"))
 
 		// Check containers conditions
 		Eventually(Build(ns, integrationKitName), TestTimeoutShort).ShouldNot(BeNil())
diff --git a/e2e/commonwithcustominstall/catalog_builder_test.go b/e2e/commonwithcustominstall/catalog_builder_test.go
index bae648049..5440b0e51 100644
--- a/e2e/commonwithcustominstall/catalog_builder_test.go
+++ b/e2e/commonwithcustominstall/catalog_builder_test.go
@@ -92,9 +92,7 @@ func TestCamelCatalogBuilder(t *testing.T) {
 
 			Eventually(CamelCatalog(ns, compatibleCatalogName)).ShouldNot(BeNil())
 			Eventually(CamelCatalogPhase(ns, compatibleCatalogName)).Should(Equal(v1.CamelCatalogPhaseReady))
-			Eventually(CamelCatalogCondition(ns, compatibleCatalogName, v1.CamelCatalogConditionReady)().Message).Should(
-				Or(Equal("Container image successfully built"), Equal("Container image exists on registry")),
-			)
+			Eventually(CamelCatalogCondition(ns, compatibleCatalogName, v1.CamelCatalogConditionReady)().Message).Should(Equal("Container image tool found in catalog"))
 			Eventually(IntegrationPodPhase(ns, name), TestTimeoutMedium).Should(Equal(corev1.PodRunning))
 			Eventually(IntegrationConditionStatus(ns, name, v1.IntegrationConditionReady), TestTimeoutMedium).
 				Should(Equal(corev1.ConditionTrue))
@@ -121,7 +119,7 @@ func TestCamelCatalogBuilder(t *testing.T) {
 			Eventually(CamelCatalog(ns, compatibleCatalogName)).ShouldNot(BeNil())
 			Eventually(CamelCatalogPhase(ns, compatibleCatalogName)).Should(Equal(v1.CamelCatalogPhaseReady))
 			Eventually(CamelCatalogCondition(ns, compatibleCatalogName, v1.CamelCatalogConditionReady)().Message).Should(
-				Equal("Container image exists on registry"),
+				Equal("Container image tool found in catalog"),
 			)
 
 			Eventually(IntegrationKit(ns, name)).ShouldNot(Equal(""))
diff --git a/e2e/native/native_test.go b/e2e/native/native_test.go
index 739ddf5ba..2a60b36ea 100644
--- a/e2e/native/native_test.go
+++ b/e2e/native/native_test.go
@@ -46,7 +46,7 @@ func TestNativeIntegrations(t *testing.T) {
 			name := "unsupported-js"
 			Expect(KamelRunWithID(operatorID, ns, "files/JavaScript.js", "--name", name,
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=6.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:6.5Gi",
 			).Execute()).To(Succeed())
 
 			Eventually(IntegrationPhase(ns, name)).Should(Equal(v1.IntegrationPhaseError))
@@ -61,7 +61,7 @@ func TestNativeIntegrations(t *testing.T) {
 			name := "xml-native"
 			Expect(KamelRunWithID(operatorID, ns, "files/Xml.xml", "--name", name,
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=6.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:6.5Gi",
 			).Execute()).To(Succeed())
 
 			Eventually(IntegrationPodPhase(ns, name), TestTimeoutVeryLong).Should(Equal(corev1.PodRunning))
@@ -76,14 +76,14 @@ func TestNativeIntegrations(t *testing.T) {
 			Expect(Kamel("delete", name, "-n", ns).Execute()).To(Succeed())
 		})
 
-		t.Run("automatic rollout deployment from fast-jar to native kit", func(t *testing.T) {
+		t.Run("automatic rollout deployment from jvm to native kit", func(t *testing.T) {
 			// Let's make sure we start from a clean state
 			Expect(DeleteKits(ns)).To(Succeed())
 			name := "yaml-native"
 			Expect(KamelRunWithID(operatorID, ns, "files/yaml.yaml", "--name", name,
-				"-t", "quarkus.mode=fast-jar",
+				"-t", "quarkus.mode=jvm",
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=6.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:6.5Gi",
 			).Execute()).To(Succeed())
 
 			// Check that two Kits are created with distinct layout
@@ -107,7 +107,7 @@ func TestNativeIntegrations(t *testing.T) {
 			Eventually(IntegrationConditionStatus(ns, name, v1.IntegrationConditionReady), TestTimeoutShort).
 				Should(Equal(corev1.ConditionTrue))
 
-			Eventually(IntegrationLogs(ns, name), TestTimeoutShort).Should(ContainSubstring("Magicstring!"))
+			Eventually(IntegrationLogs(ns, name), TestTimeoutMedium).Should(ContainSubstring("Magicstring!"))
 
 			// ====================================
 			// !!! THE MOST TIME-CONSUMING PART !!!
diff --git a/e2e/native/native_with_sources_test.go b/e2e/native/native_with_sources_test.go
index 6478540f5..513479928 100644
--- a/e2e/native/native_with_sources_test.go
+++ b/e2e/native/native_with_sources_test.go
@@ -24,14 +24,12 @@ package native
 
 import (
 	"testing"
-	"time"
 
 	. "github.com/onsi/gomega"
 
 	. "github.com/apache/camel-k/v2/e2e/support"
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	corev1 "k8s.io/api/core/v1"
-	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 )
 
 func TestNativeHighMemoryIntegrations(t *testing.T) {
@@ -47,7 +45,7 @@ func TestNativeHighMemoryIntegrations(t *testing.T) {
 			name := "java-native"
 			Expect(KamelRunWithID(operatorID, ns, "files/Java.java", "--name", name,
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=9.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:9.5Gi",
 			).Execute()).To(Succeed())
 
 			Eventually(IntegrationPodPhase(ns, name), TestTimeoutVeryLong).Should(Equal(corev1.PodRunning))
@@ -61,7 +59,7 @@ func TestNativeHighMemoryIntegrations(t *testing.T) {
 				name := "java-native-clone"
 				Expect(KamelRunWithID(operatorID, ns, "files/Java.java", "--name", name,
 					"-t", "quarkus.mode=native",
-					"-t", "builder.tasks-limit-memory=quarkus-native:=9.5Gi",
+					"-t", "builder.tasks-limit-memory=quarkus-native:9.5Gi",
 				).Execute()).To(Succeed())
 
 				// This one should run quickly as it suppose to reuse an IntegrationKit
@@ -78,7 +76,7 @@ func TestNativeHighMemoryIntegrations(t *testing.T) {
 				name := "java-native-2"
 				Expect(KamelRunWithID(operatorID, ns, "files/Java2.java", "--name", name,
 					"-t", "quarkus.mode=native",
-					"-t", "builder.tasks-limit-memory=quarkus-native:=9.5Gi",
+					"-t", "builder.tasks-limit-memory=quarkus-native:9.5Gi",
 				).Execute()).To(Succeed())
 
 				Eventually(IntegrationPodPhase(ns, name), TestTimeoutVeryLong).Should(Equal(corev1.PodRunning))
@@ -98,7 +96,7 @@ func TestNativeHighMemoryIntegrations(t *testing.T) {
 			name := "groovy-native"
 			Expect(KamelRunWithID(operatorID, ns, "files/Groovy.groovy", "--name", name,
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=9.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:9.5Gi",
 			).Execute()).To(Succeed())
 
 			Eventually(IntegrationPodPhase(ns, name), TestTimeoutVeryLong).Should(Equal(corev1.PodRunning))
@@ -117,7 +115,7 @@ func TestNativeHighMemoryIntegrations(t *testing.T) {
 			name := "kotlin-native"
 			Expect(KamelRunWithID(operatorID, ns, "files/Kotlin.kts", "--name", name,
 				"-t", "quarkus.mode=native",
-				"-t", "builder.tasks-limit-memory=quarkus-native:=9.5Gi",
+				"-t", "builder.tasks-limit-memory=quarkus-native:9.5Gi",
 			).Execute()).To(Succeed())
 
 			Eventually(IntegrationPodPhase(ns, name), TestTimeoutVeryLong).Should(Equal(corev1.PodRunning))
diff --git a/helm/camel-k/crds/crd-integration-kit.yaml b/helm/camel-k/crds/crd-integration-kit.yaml
index f40c04244..b796bc8f4 100644
--- a/helm/camel-k/crds/crd-integration-kit.yaml
+++ b/helm/camel-k/crds/crd-integration-kit.yaml
@@ -350,7 +350,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/helm/camel-k/crds/crd-integration-platform.yaml b/helm/camel-k/crds/crd-integration-platform.yaml
index b11b568ed..39827cdd4 100644
--- a/helm/camel-k/crds/crd-integration-platform.yaml
+++ b/helm/camel-k/crds/crd-integration-platform.yaml
@@ -1643,7 +1643,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
@@ -3457,7 +3456,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/helm/camel-k/crds/crd-integration.yaml b/helm/camel-k/crds/crd-integration.yaml
index e59cb1c45..21eb27225 100644
--- a/helm/camel-k/crds/crd-integration.yaml
+++ b/helm/camel-k/crds/crd-integration.yaml
@@ -7560,7 +7560,6 @@ spec:
                             build packaging. Deprecated: use `QuarkusMode` instead.'
                           enum:
                           - fast-jar
-                          - native-sources
                           - native
                           type: string
                         type: array
diff --git a/helm/camel-k/crds/crd-kamelet-binding.yaml b/helm/camel-k/crds/crd-kamelet-binding.yaml
index abde245c9..f51b72726 100644
--- a/helm/camel-k/crds/crd-kamelet-binding.yaml
+++ b/helm/camel-k/crds/crd-kamelet-binding.yaml
@@ -7858,7 +7858,6 @@ spec:
                                 build packaging. Deprecated: use `QuarkusMode` instead.'
                               enum:
                               - fast-jar
-                              - native-sources
                               - native
                               type: string
                             type: array
diff --git a/helm/camel-k/crds/crd-pipe.yaml b/helm/camel-k/crds/crd-pipe.yaml
index a8a40da19..f58e4be83 100644
--- a/helm/camel-k/crds/crd-pipe.yaml
+++ b/helm/camel-k/crds/crd-pipe.yaml
@@ -7855,7 +7855,6 @@ spec:
                                 build packaging. Deprecated: use `QuarkusMode` instead.'
                               enum:
                               - fast-jar
-                              - native-sources
                               - native
                               type: string
                             type: array
diff --git a/pkg/apis/camel/v1/trait/quarkus.go b/pkg/apis/camel/v1/trait/quarkus.go
index 538f4e9df..fb1f36f89 100644
--- a/pkg/apis/camel/v1/trait/quarkus.go
+++ b/pkg/apis/camel/v1/trait/quarkus.go
@@ -55,7 +55,7 @@ const (
 
 // QuarkusPackageType is the type of Quarkus build packaging.
 // Deprecated: use `QuarkusMode` instead.
-// +kubebuilder:validation:Enum=fast-jar;native-sources;native
+// +kubebuilder:validation:Enum=fast-jar;native
 type QuarkusPackageType string
 
 const (
diff --git a/pkg/builder/image.go b/pkg/builder/image.go
index cedf72a82..9a7ba1bd0 100644
--- a/pkg/builder/image.go
+++ b/pkg/builder/image.go
@@ -69,7 +69,7 @@ func nativeImageContext(ctx *builderContext) error {
 		ctx.Artifacts = []v1.Artifact{
 			{
 				ID:       runner,
-				Location: filepath.Join(ctx.Path, "maven", "target", "native-sources", runner),
+				Location: QuarkusRuntimeSupport(ctx.Catalog.GetCamelQuarkusVersion()).TargetDirectory(ctx.Path, runner),
 				Target:   runner,
 			},
 		}
diff --git a/pkg/builder/runtime_support.go b/pkg/builder/runtime_support.go
new file mode 100644
index 000000000..f9da1ca00
--- /dev/null
+++ b/pkg/builder/runtime_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 builder
+
+import "path/filepath"
+
+// QuarkusRuntimeNativeAdapter is used to get the proper Quarkus native configuration which may be different
+// in Camel Quarkus version. It is known that before Camel Quarkus 3.5 there was no support to native-source,
+// and using this interface will adapt the configuration to build natively according the previous configuration.
+type QuarkusRuntimeNativeAdapter interface {
+	// The commands used to build a native application
+	BuildCommands() string
+	// The directory where to execute the command
+	Directory() string
+	// The directory where to expect the native compiled artifact
+	TargetDirectory(ctxPath, runner string) string
+	// The parameter to use for the maven project
+	NativeMavenProperty() string
+}
+
+// NativeSourcesAdapter used for Camel Quarkus runtime >= 3.5.0.
+type NativeSourcesAdapter struct {
+}
+
+// BuildCommands -- .
+func (n *NativeSourcesAdapter) BuildCommands() string {
+	return "cd " + n.Directory() + " && echo NativeImage version is $(native-image --version) && echo GraalVM expected version is $(cat graalvm.version) && echo WARN: Make sure they are compatible, otherwise the native compilation may results in error && native-image $(cat native-image.args)"
+}
+
+// Directory -- .
+func (n *NativeSourcesAdapter) Directory() string {
+	return filepath.Join("maven", "target", "native-sources")
+}
+
+// TargetDirectory -- .
+func (n *NativeSourcesAdapter) TargetDirectory(ctxPath, runner string) string {
+	return filepath.Join(ctxPath, "maven", "target", "native-sources", runner)
+}
+
+// NativeMavenProperty -- .
+func (n *NativeSourcesAdapter) NativeMavenProperty() string {
+	return "native-sources"
+}
+
+// NativeAdapter used for Camel Quarkus runtime < 3.5.0.
+type NativeAdapter struct {
+}
+
+// BuildCommands -- .
+func (n *NativeAdapter) BuildCommands() string {
+	return "cd " + n.Directory() + " && ./mvnw package -Dquarkus.package.type=native --global-settings settings.xml"
+}
+
+// Directory -- .
+func (n *NativeAdapter) Directory() string {
+	return "maven"
+}
+
+// TargetDirectory -- .
+func (n *NativeAdapter) TargetDirectory(ctxPath, runner string) string {
+	return filepath.Join(ctxPath, "maven", "target", runner)
+}
+
+// NativeMavenProperty -- .
+func (n *NativeAdapter) NativeMavenProperty() string {
+	// Empty on purpose. The parameter will be provided later by the command (see BuildCommands()).
+	return ""
+}
+
+// QuarkusRuntimeSupport is used to get the proper native configuration based on the Camel Quarkus version.
+func QuarkusRuntimeSupport(version string) QuarkusRuntimeNativeAdapter {
+	if version < "3.5.0" {
+		return &NativeAdapter{}
+	}
+	return &NativeSourcesAdapter{}
+}
diff --git a/pkg/trait/builder.go b/pkg/trait/builder.go
index c7cd7c7a7..f1c6a7f49 100644
--- a/pkg/trait/builder.go
+++ b/pkg/trait/builder.go
@@ -19,7 +19,6 @@ package trait
 
 import (
 	"fmt"
-	"path/filepath"
 	"regexp"
 	"sort"
 	"strings"
@@ -79,8 +78,9 @@ func (t *builderTrait) Configure(e *Environment) (bool, error) {
 				return false, err
 			}
 			if ok && pointer.BoolDeref(quarkus.Enabled, true) && (isNativeIntegration || isNativeKit) {
-				nativeArgsCd := filepath.Join("maven", "target", "native-sources")
-				command := "cd " + nativeArgsCd + " && echo NativeImage version is $(native-image --version) && echo GraalVM expected version is $(cat graalvm.version) && echo WARN: Make sure they are compatible, otherwise the native compilation may results in error && native-image $(cat native-image.args)"
+				// TODO expect maven repository in local repo (need to change builder pod accordingly!)
+				command := builder.QuarkusRuntimeSupport(e.CamelCatalog.GetCamelQuarkusVersion()).BuildCommands()
+
 				// it should be performed as the last custom task
 				t.Tasks = append(t.Tasks, fmt.Sprintf(`quarkus-native;%s;/bin/bash -c "%s"`, e.CamelCatalog.GetQuarkusToolingImage(), command))
 				// Force the build to run in a separate Pod and strictly sequential
@@ -494,8 +494,8 @@ func (t *builderTrait) parseTasksConf() (map[string]*v1.BuildConfiguration, erro
 // if however we have a command which is not quoted, then we leave it the way it is.
 func splitContainerCommand(command string) []string {
 	if !strings.Contains(command, "\"") {
-		// No quotes, just return
-		return []string{command}
+		// No quotes, then, splits all commands found
+		return strings.Split(command, " ")
 	}
 	matches := commandsRegexp.FindAllString(command, -1)
 	removeQuotes := make([]string, 0, len(matches))
diff --git a/pkg/trait/builder_test.go b/pkg/trait/builder_test.go
index 0c060177b..651da6a22 100644
--- a/pkg/trait/builder_test.go
+++ b/pkg/trait/builder_test.go
@@ -331,7 +331,9 @@ func TestBuilderCustomTasks(t *testing.T) {
 	assert.Equal(t, "ls", tasks[0].Custom.ContainerCommands[0])
 	assert.Equal(t, "test", tasks[1].Custom.Name)
 	assert.Equal(t, "alpine", tasks[1].Custom.ContainerImage)
-	assert.Equal(t, "mvn test", tasks[1].Custom.ContainerCommands[0])
+	assert.Equal(t, "mvn", tasks[1].Custom.ContainerCommands[0])
+	assert.Equal(t, "test", tasks[1].Custom.ContainerCommands[1])
+
 }
 
 func TestBuilderCustomTasksFailure(t *testing.T) {
@@ -399,8 +401,9 @@ func TestUserTaskSingleCommand(t *testing.T) {
 	command := `cat /path/to/a/resource`
 	podCommands := splitContainerCommand(command)
 
-	assert.Len(t, podCommands, 1)
-	assert.Equal(t, "cat /path/to/a/resource", podCommands[0])
+	assert.Len(t, podCommands, 2)
+	assert.Equal(t, "cat", podCommands[0])
+	assert.Equal(t, "/path/to/a/resource", podCommands[1])
 }
 
 func TestUserTaskMultiCommands(t *testing.T) {
diff --git a/pkg/trait/quarkus.go b/pkg/trait/quarkus.go
index 1db7f6f16..ad3744d8e 100644
--- a/pkg/trait/quarkus.go
+++ b/pkg/trait/quarkus.go
@@ -341,8 +341,13 @@ func (t *quarkusTrait) applyWhenBuildSubmitted(e *Environment) error {
 		return err
 	}
 
+	// The LoadCamelQuarkusCatalog is required to have catalog information available by the builder
+	packageSteps = append(packageSteps, builder.Quarkus.LoadCamelQuarkusCatalog)
+
 	if native {
-		buildTask.Maven.Properties["quarkus.package.type"] = string(nativeSourcesPackageType)
+		if nativePackagetType := builder.QuarkusRuntimeSupport(e.CamelCatalog.GetCamelQuarkusVersion()).NativeMavenProperty(); nativePackagetType != "" {
+			buildTask.Maven.Properties["quarkus.package.type"] = nativePackagetType
+		}
 		if len(e.IntegrationKit.Spec.Sources) > 0 {
 			buildTask.Sources = e.IntegrationKit.Spec.Sources
 			buildSteps = append(buildSteps, builder.Quarkus.PrepareProjectWithSources)
@@ -354,8 +359,6 @@ func (t *quarkusTrait) applyWhenBuildSubmitted(e *Environment) error {
 		// Default, if nothing is specified
 		buildTask.Maven.Properties["quarkus.package.type"] = string(fastJarPackageType)
 		packageSteps = append(packageSteps, builder.Quarkus.ComputeQuarkusDependencies)
-		// The LoadCamelQuarkusCatalog is required to have catalog information available by the builder
-		packageSteps = append(packageSteps, builder.Quarkus.LoadCamelQuarkusCatalog)
 		packageSteps = append(packageSteps, builder.Image.IncrementalImageContext)
 		// Create the dockerfile, regardless it's later used or not by the publish strategy
 		packageSteps = append(packageSteps, builder.Image.JvmDockerfile)