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

(camel-k) 03/03: feat(pipeline): support custom user id execution

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 2771247703e61f7e4511528e236bc8c0f50f95f7
Author: Pasquale Congiusti <pa...@gmail.com>
AuthorDate: Thu Jan 11 11:00:47 2024 +0100

    feat(pipeline): support custom user id execution
    
    Will add the container security context accordingly
---
 config/crd/bases/camel.apache.org_builds.yaml      |  4 ++
 docs/modules/ROOT/pages/pipeline/pipeline.adoc     | 61 ++++++++++++++++++----
 docs/modules/ROOT/partials/apis/camel-k-crds.adoc  |  7 +++
 helm/camel-k/crds/crd-build.yaml                   |  4 ++
 pkg/apis/camel/v1/build_types.go                   |  2 +
 pkg/apis/camel/v1/trait/builder.go                 |  2 +-
 pkg/apis/camel/v1/zz_generated.deepcopy.go         |  5 ++
 .../camel/applyconfiguration/camel/v1/usertask.go  |  9 ++++
 pkg/controller/build/build_pod.go                  |  6 +++
 pkg/controller/build/monitor_pod.go                | 58 +++++++++++---------
 pkg/resources/resources.go                         |  4 +-
 pkg/trait/builder.go                               | 56 +++++++++++---------
 pkg/trait/builder_test.go                          | 23 ++++++--
 13 files changed, 174 insertions(+), 67 deletions(-)

diff --git a/config/crd/bases/camel.apache.org_builds.yaml b/config/crd/bases/camel.apache.org_builds.yaml
index 02965a6bc..3cb3ea1b6 100644
--- a/config/crd/bases/camel.apache.org_builds.yaml
+++ b/config/crd/bases/camel.apache.org_builds.yaml
@@ -821,6 +821,10 @@ spec:
                         publishingImage:
                           description: the desired image build name
                           type: string
+                        userId:
+                          description: the user id used to run the container
+                          format: int64
+                          type: integer
                       type: object
                     jib:
                       description: a JibTask, for Jib strategy
diff --git a/docs/modules/ROOT/pages/pipeline/pipeline.adoc b/docs/modules/ROOT/pages/pipeline/pipeline.adoc
index b204f338c..6712c940d 100644
--- a/docs/modules/ROOT/pages/pipeline/pipeline.adoc
+++ b/docs/modules/ROOT/pages/pipeline/pipeline.adoc
@@ -1,22 +1,25 @@
 [[build-pipeline]]
 = Build Pipeline
-Since version 2.0, we've introduced the concept of `Pipeline` in order to provide a degree of flexibility for those user that want to customize the entire building process. We can think of a pipeline as a series of tasks that can be executed after the project generation and before the artifact publishing into the container registry.
 
-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:
+Since version 2.0, we've introduced the concept of `Pipeline` in order to provide a degree of flexibility for those user that want to customize the entire building process. We can think of a pipeline as a series of tasks that can be executed to generate a Camel application project, build it accordingly and publish into a container registry.
+
+Camel K operator creates a default pipeline containing an opinionated `build`, `package` and `publish` operations. Beside them you can include any additional task in order to enrich the building process:
 
 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 [...]
+The main 3 tasks 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.
+
+If you're building a Quarkus Native image, the quarkus trait influences the builder in order to include a custom task (named **quarkus-native**) which takes care of executing the native image compilation. But this is completely hidden to the final user.
 
-In reality we do make use of the custom task in one particular situation. In order to build a Quarkus Native image, the quarkus trait influences the builder in order to include a custom task which takes care of executing the native image compilation. But this is completely hidden to the final user.
+Once the build is over, the **package** task takes care to prepare the context later required by the publishing operations. Basically this task is in charge to copy all artifacts and also prepare a Dockerfile which could be used eventually by the rest of the process. Finally, the **publish** task is in charge to generate and push the container to the registry using all the artifacts generated in the package task.
 
-Once the build and any customization are over, the **package** task takes care to prepare the context later required by the publishing operations. Basically this task is in charge to copy all artifacts and also prepare a Dockerfile which could be used eventually by the rest of the process. Finally, the **publish** task is in charge to generate and push the container to the registry using all the artifacts generated in the package task.
+The **custom tasks** can be provided at any point of the pipeline (read task sorting section). With them we are introducing that level of flexibility required to accommodate any company build process. This part is optional, by default, Camel K does not need any **custom task**.
 
 [[add-custom-tasks]]
 == Add custom user tasks
-As you have seen, the final user can include any optional task after a build operation is performed. We think this is the best moment when any custom operation can be performed as it is the immediate step after the Maven project is generated. And ideally, the context of any custom operation is the project.
+The final user can include any optional task, which, by default is run after a build operation is performed. We think this is the best moment when any custom operation can be performed as it is the immediate step after the Maven project is generated. And ideally, the context of any custom operation is the project.
 
-NOTE: Please, notice that, since the customization may require any tool not available in the operator container, you will need to run any additional task using builder `pod` strategy.
+Custom tasks are only available using builder `pod` strategy in order to let the user provide each task with the tools required to run the specific customization.
 
 Let's see an example:
 
@@ -29,6 +32,7 @@ spec:
       command: tree
       image: alpine
       name: custom1
+      userId: 0
   - custom:
       command: cat maven/pom.xml
       image: alpine
@@ -36,7 +40,7 @@ spec:
   - spectrum:
       ...
 ```
-The custom tasks will be executed in the directory where the Camel K runtime Maven project was generated. In this example we're creating 2 tasks to retrieve certain values from the project just for the scope of illustrating the feature. For each task you need to specify a name, the container image which you want to use to run and the command to execute.
+The custom tasks will be executed in the directory where the Camel K runtime Maven project was generated. In this example we're creating 2 tasks to retrieve certain values from the project just for the scope of illustrating the feature. For each task you need to specify a name, the container image which you want to use to run, the command to execute and (optionally) the user ID that has to run in the container.
 
 The goal is to let the user perform custom tasks which may result in a success or a failure. If the task executed results in a failure, then, the entire build is stopped and fails accordingly.
 
@@ -47,7 +51,7 @@ We are aware that configuring the `Build` type directly is something that the ma
 Maintaining the example above as a reference, configuring a custom task will be as easy as adding a trait property when running your Integration, for instance, via CLI:
 
 ```
-kamel run Test.java -t builder.tasks=custom1;alpine;tree -t builder.tasks="custom2;alpine;cat maven/pom.xml"
+kamel run Test.java -t builder.tasks=custom1;alpine;tree;0 -t builder.tasks="custom2;alpine;cat maven/pom.xml"
 ```
 
 Another interesting configuration you can provide via Builder trait is the (https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/)[Kubernetes requests and limits]. Each of the task you are providing in the pipeline, can be configured with the proper resource settings. You can use, for instance the `-t builder.request-cpu <task-name>:1000m` to configure the container executed by the `task-name`. This configuration works for all the tasks including `builder`, `pack [...]
@@ -55,7 +59,7 @@ Another interesting configuration you can provide via Builder trait is the (http
 [[build-pipeline-result]]
 == Getting task execution status
 
-Altough the main goal of this custom task execution is to have a success/failure result, we thought it could be useful to get the log of each task to be consulted by the user. For this reason, you will be able to read it directly in the Build type. See the following example:
+Although the main goal of this custom task execution is to have a success/failure result, we thought it could be useful to get the log of each task to be consulted by the user. For this reason, you will be able to read it directly in the Build type. See the following example:
 
 ```
   conditions:
@@ -113,9 +117,44 @@ Given the limited space we can use in a Kubernetes custom resource, we are trunc
 
 If for any reason you still need to access the entire log of the execution, you can always access to the log of the builder `Pod` and the specific container that was executed, ie `kubectl logs camel-k-kit-chj2gpi9rcoc73cjfv2g-builder -c task1 -p`
 
+[[build-pipeline-sort]]
+=== Tasks filtering and sorting
+The default execution of tasks can be changed. You can include or remove tasks and provide your order of execution. When using the `builder` trait, this is done via `builder.tasks-filter` property. This parameter accepts a comma separated value list of tasks to execute. It must specify both operator tasks (ie, builder) and custom tasks.
+
+WARNING: altering the order of tasks may result in a disruptive build behavior.
+
+With this approach you can also provide your own publishing strategy. The pipeline run as a Kubernetes Pod. Each task is an `initContainer` and the last one is the final `container` which has to provide the published image digest into the `/dev/termination-log` in order to know the publish was completed. This is done out of the box when a supported publishing strategy is executed. If you want to provide your own publishing strategy, then, you need to make sure to follow this rule to have [...]
+
+There are certain environment variables we are injecting in each custom container in order to be able to get dynamic values required to perform certain operation (ie, publishing a container image):
+
+* INTEGRATION_KIT_IMAGE contains the name of the image expected to be used for the IntegrationKit generated during the pipeline execution
+
+We may add more if they are required and useful for a general case.
+
+[[build-pipeline-privileges]]
+=== Execution privileges
+The builder Pod will set a `PodSecurityContext` with a specific user ID privileges. This is a convention in order to maintain the same default user ID for the container images we use by default. Each container image will inherit this value unless specified with the user ID parameter. The value is by default 1001 unless you're using S2I publishing strategy (the default strategy for Openshift), where the value is defined by the platform.
+
+NOTE: you may run your task with root privileges using user ID 0. However, this is highly discouraged. You should instead make your container image being able to run with the default user ID privileges.
+
 [[build-pipeline-examples]]
 == Custom tasks examples
-
 As we are using container registry for execution, you will be able to execute virtually any kind of task. You can provide your own container with tools required by your company or use any one available in the OSS.
 
 As the target of the execution is the project, before the artifact is published to the registry, you can execute any task to validate the project. We can think of any vulnerability tool scanner, quality of code or any other action you tipically perform in your build pipeline.
+
+[[build-pipeline-examples-buildah]]
+=== Use your own publishing strategy
+We suggest to use the supported publishing strategy as they are working out of the box and optimized for the operator. However you may choose to adopt your own publishing strategy. Here an example to run an Integration using a Buildah publishing strategy:
+
+```bash
+$ kamel run test.yaml -t builder.strategy=pod -t builder.base-image=docker.io/library/eclipse-temurin:17 -t builder.tasks="my-buildah;quay.io/buildah/stable;/bin/bash -c \"cd context && buildah bud --storage-driver=vfs --tls-verify=false -t \$(INTEGRATION_KIT_IMAGE) . && buildah push --storage-driver=vfs --digestfile=/dev/termination-log --tls-verify=false \$(INTEGRATION_KIT_IMAGE) docker://\$(INTEGRATION_KIT_IMAGE)\";0" -t builder.tasks-filter=builder,package,my-buildah
+```
+
+You can see in this case we have created a custom task named `my-buildah`. This is expecting to run the build and publish operations. We must change directory to `/context/` where the **package** task has written a `Dockerfile`. We are using the INTEGRATION_KIT_IMAGE environment variable and we are writing the pushed image digest to `/dev/termination-log` as expected by the procedure.
+
+NOTE: depending on the strategy used you may not have a direct feature. It would suffice to add some command and just write the digest of the generated image via `echo $my-image-digest > /dev/termination-log`.
+
+Finally we filter the tasks we want to execute, making sure to include the **builder** and the **package** tasks which are provided by the operator and are in charge to do the building and the packaging.
+
+NOTE: we had to run the `my-buildah` task with root privileges as this is required by the default container image. We had also to override the `builder.base-image` as it also require a special treatment for Buildah to work.
\ No newline at end of file
diff --git a/docs/modules/ROOT/partials/apis/camel-k-crds.adoc b/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
index f2ec00ef0..16a74cf3a 100644
--- a/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
+++ b/docs/modules/ROOT/partials/apis/camel-k-crds.adoc
@@ -5659,6 +5659,13 @@ string
 
 the container image to use
 
+|`userId` +
+int64
+|
+
+
+the user id used to run the container
+
 |`command` +
 string
 |
diff --git a/helm/camel-k/crds/crd-build.yaml b/helm/camel-k/crds/crd-build.yaml
index 02965a6bc..3cb3ea1b6 100644
--- a/helm/camel-k/crds/crd-build.yaml
+++ b/helm/camel-k/crds/crd-build.yaml
@@ -821,6 +821,10 @@ spec:
                         publishingImage:
                           description: the desired image build name
                           type: string
+                        userId:
+                          description: the user id used to run the container
+                          format: int64
+                          type: integer
                       type: object
                     jib:
                       description: a JibTask, for Jib strategy
diff --git a/pkg/apis/camel/v1/build_types.go b/pkg/apis/camel/v1/build_types.go
index 076cf0426..a843eea5b 100644
--- a/pkg/apis/camel/v1/build_types.go
+++ b/pkg/apis/camel/v1/build_types.go
@@ -187,6 +187,8 @@ type UserTask struct {
 	BaseTask `json:",inline"`
 	// the container image to use
 	ContainerImage string `json:"image,omitempty"`
+	// the user id used to run the container
+	ContainerUserID *int64 `json:"userId,omitempty"`
 	// the command to execute
 	// Deprecated: use ContainerCommands
 	ContainerCommand string `json:"command,omitempty"`
diff --git a/pkg/apis/camel/v1/trait/builder.go b/pkg/apis/camel/v1/trait/builder.go
index 9dc6c361a..c98feafa6 100644
--- a/pkg/apis/camel/v1/trait/builder.go
+++ b/pkg/apis/camel/v1/trait/builder.go
@@ -60,7 +60,7 @@ type BuilderTrait struct {
 	Tasks []string `property:"tasks" json:"tasks,omitempty"`
 	// A list of tasks sorted by the order of execution in a csv format, ie, `<taskName1>,<taskName2>,...`.
 	// Mind that you must include also the operator tasks (`builder`, `quarkus-native`, `package`, `jib`, `spectrum`, `s2i`)
-	// if you need to execute them. Useful only wih `pod` strategy.
+	// if you need to execute them. Useful only with `pod` strategy.
 	TasksFilter string `property:"tasks-filter" json:"tasksFilter,omitempty"`
 	// A list of request cpu configuration for the specific task with format `<task-name>:<request-cpu-conf>`.
 	TasksRequestCPU []string `property:"tasks-request-cpu" json:"tasksRequestCPU,omitempty"`
diff --git a/pkg/apis/camel/v1/zz_generated.deepcopy.go b/pkg/apis/camel/v1/zz_generated.deepcopy.go
index e630aa257..f0d23440d 100644
--- a/pkg/apis/camel/v1/zz_generated.deepcopy.go
+++ b/pkg/apis/camel/v1/zz_generated.deepcopy.go
@@ -3090,6 +3090,11 @@ func (in *Traits) DeepCopy() *Traits {
 func (in *UserTask) DeepCopyInto(out *UserTask) {
 	*out = *in
 	in.BaseTask.DeepCopyInto(&out.BaseTask)
+	if in.ContainerUserID != nil {
+		in, out := &in.ContainerUserID, &out.ContainerUserID
+		*out = new(int64)
+		**out = **in
+	}
 	if in.ContainerCommands != nil {
 		in, out := &in.ContainerCommands, &out.ContainerCommands
 		*out = make([]string, len(*in))
diff --git a/pkg/client/camel/applyconfiguration/camel/v1/usertask.go b/pkg/client/camel/applyconfiguration/camel/v1/usertask.go
index 7a85991d1..5f396fcd5 100644
--- a/pkg/client/camel/applyconfiguration/camel/v1/usertask.go
+++ b/pkg/client/camel/applyconfiguration/camel/v1/usertask.go
@@ -24,6 +24,7 @@ package v1
 type UserTaskApplyConfiguration struct {
 	BaseTaskApplyConfiguration `json:",inline"`
 	ContainerImage             *string  `json:"image,omitempty"`
+	ContainerUserID            *int64   `json:"userId,omitempty"`
 	ContainerCommand           *string  `json:"command,omitempty"`
 	ContainerCommands          []string `json:"commands,omitempty"`
 	PublishingImage            *string  `json:"publishingImage,omitempty"`
@@ -59,6 +60,14 @@ func (b *UserTaskApplyConfiguration) WithContainerImage(value string) *UserTaskA
 	return b
 }
 
+// WithContainerUserID sets the ContainerUserID 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 ContainerUserID field is set to the value of the last call.
+func (b *UserTaskApplyConfiguration) WithContainerUserID(value int64) *UserTaskApplyConfiguration {
+	b.ContainerUserID = &value
+	return b
+}
+
 // WithContainerCommand sets the ContainerCommand 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 ContainerCommand field is set to the value of the last call.
diff --git a/pkg/controller/build/build_pod.go b/pkg/controller/build/build_pod.go
index 4914e771d..46fd34d55 100644
--- a/pkg/controller/build/build_pod.go
+++ b/pkg/controller/build/build_pod.go
@@ -573,6 +573,12 @@ func addCustomTaskToPod(build *v1.Build, task *v1.UserTask, pod *corev1.Pod) {
 		Env:             proxyFromEnvironment(),
 	}
 	container.Env = append(container.Env, corev1.EnvVar{Name: "INTEGRATION_KIT_IMAGE", Value: task.PublishingImage})
+	if task.ContainerUserID != nil {
+		container.SecurityContext = &corev1.SecurityContext{
+			RunAsUser:  task.ContainerUserID,
+			RunAsGroup: task.ContainerUserID,
+		}
+	}
 
 	configureResources(task.Name, build, &container)
 	addContainerToPod(build, container, pod)
diff --git a/pkg/controller/build/monitor_pod.go b/pkg/controller/build/monitor_pod.go
index 0fcf60a72..0af446104 100644
--- a/pkg/controller/build/monitor_pod.go
+++ b/pkg/controller/build/monitor_pod.go
@@ -147,19 +147,22 @@ func (action *monitorPodAction) Handle(ctx context.Context, build *v1.Build) (*v
 		observeBuildResult(build, build.Status.Phase, buildCreator, duration)
 
 		build.Status.Image = publishTaskImageName(build.Spec.Tasks)
-		build.Status.Digest = publishTaskDigest(build.Spec.Tasks, pod.Status.ContainerStatuses)
-		if build.Status.Digest == "" {
-			// Likely to happen for users provided publishing tasks and not providing the digest image among statuses
-			build.Status.Phase = v1.BuildPhaseError
-			build.Status.SetCondition(
-				"ImageDigestAvailable",
-				corev1.ConditionFalse,
-				"ImageDigestAvailable",
-				fmt.Sprintf(
-					"%s publishing task completed but no digest is available in container status. Make sure that the process successfully push the image to the registry and write its digest to /dev/termination-log",
-					publishTaskName(build.Spec.Tasks),
-				),
-			)
+		// operator supported publishing tasks should provide the digest in the builder command process execution
+		if !operatorSupportedPublishingStrategy(build.Spec.Tasks) {
+			build.Status.Digest = publishTaskDigest(build.Spec.Tasks, pod.Status.ContainerStatuses)
+			if build.Status.Digest == "" {
+				// Likely to happen for users provided publishing tasks and not providing the digest image among statuses
+				build.Status.Phase = v1.BuildPhaseError
+				build.Status.SetCondition(
+					"ImageDigestAvailable",
+					corev1.ConditionFalse,
+					"ImageDigestAvailable",
+					fmt.Sprintf(
+						"%s publishing task completed but no digest is available in container status. Make sure that the process successfully push the image to the registry and write its digest to /dev/termination-log",
+						publishTaskName(build.Spec.Tasks),
+					),
+				)
+			}
 		}
 
 	case corev1.PodFailed:
@@ -339,7 +342,7 @@ func (action *monitorPodAction) setConditionsFromTerminationMessages(ctx context
 }
 
 // we expect that the last task is any of the supported publishing task
-// or a custom user task
+// or a custom user task.
 func publishTask(tasks []v1.Task) *v1.Task {
 	if len(tasks) > 0 {
 		return &tasks[len(tasks)-1]
@@ -354,15 +357,16 @@ func publishTaskImageName(tasks []v1.Task) string {
 	if t == nil {
 		return ""
 	}
-	if t.Custom != nil {
+	switch {
+	case t.Custom != nil:
 		return t.Custom.PublishingImage
-	} else if t.Spectrum != nil {
+	case t.Spectrum != nil:
 		return t.Spectrum.Image
-	} else if t.Jib != nil {
+	case t.Jib != nil:
 		return t.Jib.Image
-	} else if t.Buildah != nil {
+	case t.Buildah != nil:
 		return t.Buildah.Image
-	} else if t.Kaniko != nil {
+	case t.Kaniko != nil:
 		return t.Kaniko.Image
 	}
 
@@ -374,15 +378,16 @@ func publishTaskName(tasks []v1.Task) string {
 	if t == nil {
 		return ""
 	}
-	if t.Custom != nil {
+	switch {
+	case t.Custom != nil:
 		return t.Custom.Name
-	} else if t.Spectrum != nil {
+	case t.Spectrum != nil:
 		return t.Spectrum.Name
-	} else if t.Jib != nil {
+	case t.Jib != nil:
 		return t.Jib.Name
-	} else if t.Buildah != nil {
+	case t.Buildah != nil:
 		return t.Buildah.Name
-	} else if t.Kaniko != nil {
+	case t.Kaniko != nil:
 		return t.Kaniko.Name
 	}
 
@@ -399,3 +404,8 @@ func publishTaskDigest(tasks []v1.Task, cntStates []corev1.ContainerStatus) stri
 	}
 	return ""
 }
+
+func operatorSupportedPublishingStrategy(tasks []v1.Task) bool {
+	taskName := publishTaskName(tasks)
+	return taskName == "jib" || taskName == "spectrum" || taskName == "s2i"
+}
diff --git a/pkg/resources/resources.go b/pkg/resources/resources.go
index 04bba969b..3d0b93b67 100644
--- a/pkg/resources/resources.go
+++ b/pkg/resources/resources.go
@@ -117,9 +117,9 @@ var assets = func() http.FileSystem {
 		"/crd/bases/camel.apache.org_builds.yaml": &vfsgen۰CompressedFileInfo{
 			name:             "camel.apache.org_builds.yaml",
 			modTime:          time.Time{},
-			uncompressedSize: 95542,
+			uncompressedSize: 95731,
 
-			compressedContent: []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xec\x7d\x6b\x73\x1b\x37\x96\xe8\x77\xfd\x8a\x53\xf1\x07\xcb\x55\x22\x35\xe3\x64\x67\xb3\xba\xb5\x75\x4b\x2b\x27\xb3\x1a\x27\xb6\xd7\x94\x3d\x99\xda\xda\x2a\x81\xdd\x87\x24\xc2\x6e\xa0\x2f\x80\x16\xcd\xdc\xba\xff\xfd\x16\x5e\xfd\x10\xd9\xdd\x00\x45\xda\x9e\x72\xe3\x4b\x62\xaa\x01\x9c\x83\xc7\x79\xe1\x3c\x9e\xc1\xe4\x78\xed\xec\x19\xfc\x42\x13\x64\x12\x53\x50\x1c\xd4\x0a\xe1\xba\x20\xc9\x0a\x61\xc6\x17\x6a\x43\x04\xc2\x [...]
+			compressedContent: []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xec\x7d\x6b\x73\x1b\x37\x96\xe8\x77\xfd\x8a\x53\xf1\x07\xcb\x55\x22\x35\xe3\x64\x67\xb3\xba\xb5\x75\x4b\x2b\x27\xb3\x1a\x27\xb6\xd7\x94\x3d\x99\xda\xda\x2a\x81\xdd\x87\x24\xc2\x6e\xa0\x2f\x80\x16\xcd\xdc\xba\xff\xfd\x16\x5e\xfd\x10\xd9\xdd\x00\x45\xda\x9e\x72\xe3\x4b\x62\xaa\x01\x9c\x83\xc7\x79\xe1\x3c\x9e\xc1\xe4\x78\xed\xec\x19\xfc\x42\x13\x64\x12\x53\x50\x1c\xd4\x0a\xe1\xba\x20\xc9\x0a\x61\xc6\x17\x6a\x43\x04\xc2\x [...]
 		},
 		"/crd/bases/camel.apache.org_camelcatalogs.yaml": &vfsgen۰CompressedFileInfo{
 			name:             "camel.apache.org_camelcatalogs.yaml",
diff --git a/pkg/trait/builder.go b/pkg/trait/builder.go
index 843deb2e4..a0da38970 100644
--- a/pkg/trait/builder.go
+++ b/pkg/trait/builder.go
@@ -21,6 +21,7 @@ import (
 	"fmt"
 	"regexp"
 	"sort"
+	"strconv"
 	"strings"
 
 	corev1 "k8s.io/api/core/v1"
@@ -478,22 +479,15 @@ func (t *builderTrait) getBaseImage(e *Environment) string {
 	return baseImage
 }
 
+// the format expected is "<task-name>;<task-image>;<task-container-command>[;<task-container-user-id>]".
 func (t *builderTrait) customTasks(tasksConf map[string]*v1.BuildConfiguration, imageName string) ([]v1.Task, error) {
 	customTasks := make([]v1.Task, len(t.Tasks))
 
 	for i, t := range t.Tasks {
 		splitted := strings.Split(t, ";")
 		if len(splitted) < 3 {
-			return nil, fmt.Errorf(`provide a custom task with at least 3 arguments, ie "my-task-name;my-image;echo 'hello', was %v"`, t)
+			return nil, fmt.Errorf(`provide a custom task with at least 3 arguments, ie "my-task-name;my-image;echo 'hello'", was %v`, t)
 		}
-		var containerCommand string
-		if len(splitted) > 3 {
-			// recompose in case of usage of separator char in the script
-			containerCommand = strings.Join(splitted[2:], ";")
-		} else {
-			containerCommand = splitted[2]
-		}
-		containerCommands := splitContainerCommand(containerCommand)
 		customTasks[i] = v1.Task{
 			Custom: &v1.UserTask{
 				BaseTask: v1.BaseTask{
@@ -502,9 +496,16 @@ func (t *builderTrait) customTasks(tasksConf map[string]*v1.BuildConfiguration,
 				},
 				PublishingImage:   imageName,
 				ContainerImage:    splitted[1],
-				ContainerCommands: containerCommands,
+				ContainerCommands: splitContainerCommand(splitted[2]),
 			},
 		}
+		if len(splitted) > 3 {
+			uid, err := strconv.ParseInt(splitted[3], 10, 64)
+			if err != nil {
+				return nil, fmt.Errorf(`provide a custom task with a correct numeric user id, was %v`, splitted[3])
+			}
+			customTasks[i].Custom.ContainerUserID = &uid
+		}
 	}
 	return customTasks, nil
 }
@@ -596,33 +597,36 @@ func filter(tasks []v1.Task, filterTasks []string) ([]v1.Task, error) {
 	var filteredTasks []v1.Task
 	for _, f := range filterTasks {
 		found := false
+
 		for _, t := range tasks {
-			if t.Builder != nil && t.Builder.Name == f {
+			switch {
+			case t.Builder != nil && t.Builder.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Custom != nil && t.Custom.Name == f {
+			case t.Custom != nil && t.Custom.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Package != nil && t.Package.Name == f {
+			case t.Package != nil && t.Package.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Spectrum != nil && t.Spectrum.Name == f {
+			case t.Spectrum != nil && t.Spectrum.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.S2i != nil && t.S2i.Name == f {
+			case t.S2i != nil && t.S2i.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Jib != nil && t.Jib.Name == f {
+			case t.Jib != nil && t.Jib.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Buildah != nil && t.Buildah.Name == f {
+			case t.Buildah != nil && t.Buildah.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
-			} else if t.Kaniko != nil && t.Kaniko.Name == f {
+			case t.Kaniko != nil && t.Kaniko.Name == f:
 				filteredTasks = append(filteredTasks, t)
 				found = true
 			}
 		}
+
 		if !found {
 			// If we reach this point it means no tasks exists for the name
 			return nil, fmt.Errorf("no task exist for %s name", f)
@@ -635,20 +639,20 @@ func filter(tasks []v1.Task, filterTasks []string) ([]v1.Task, error) {
 	return filteredTasks, nil
 }
 
-// return true if the task is either a publishing task or a custom user task
+// return true if the task is either a publishing task or a custom user task.
 func publishingOrUserTask(t v1.Task) bool {
-	if t.Custom != nil {
+	switch {
+	case t.Custom != nil:
 		return true
-	} else if t.Spectrum != nil {
+	case t.Spectrum != nil:
 		return true
-	} else if t.S2i != nil {
+	case t.Jib != nil:
 		return true
-	} else if t.Jib != nil {
+	case t.Buildah != nil:
 		return true
-	} else if t.Buildah != nil {
-		return true
-	} else if t.Kaniko != nil {
+	case t.Kaniko != nil:
 		return true
 	}
+
 	return false
 }
diff --git a/pkg/trait/builder_test.go b/pkg/trait/builder_test.go
index bb0e66b0d..0a8224516 100644
--- a/pkg/trait/builder_test.go
+++ b/pkg/trait/builder_test.go
@@ -340,11 +340,12 @@ func TestBuilderCustomTasksFailure(t *testing.T) {
 	_, err := builderTrait.customTasks(nil, "my-kit-img")
 
 	assert.NotNil(t, err)
+	assert.Equal(t, "provide a custom task with at least 3 arguments, ie \"my-task-name;my-image;echo 'hello'\", was test;alpine", err.Error())
 }
 
-func TestBuilderCustomTasksScript(t *testing.T) {
+func TestBuilderCustomTasksBashScript(t *testing.T) {
 	builderTrait := createNominalBuilderTraitTest()
-	builderTrait.Tasks = append(builderTrait.Tasks, "test;alpine;/bin/bash -c \"cd test && ls; echo 'helooo'\"")
+	builderTrait.Tasks = append(builderTrait.Tasks, "test;alpine;/bin/bash -c \"cd test && echo 'helooo'\"")
 
 	tasks, err := builderTrait.customTasks(nil, "my-kit-img")
 
@@ -354,7 +355,23 @@ func TestBuilderCustomTasksScript(t *testing.T) {
 	assert.Equal(t, "alpine", tasks[0].Custom.ContainerImage)
 	assert.Equal(t, "/bin/bash", tasks[0].Custom.ContainerCommands[0])
 	assert.Equal(t, "-c", tasks[0].Custom.ContainerCommands[1])
-	assert.Equal(t, "cd test && ls; echo 'helooo'", tasks[0].Custom.ContainerCommands[2])
+	assert.Equal(t, "cd test && echo 'helooo'", tasks[0].Custom.ContainerCommands[2])
+}
+
+func TestBuilderCustomTasksSecurityContextScript(t *testing.T) {
+	builderTrait := createNominalBuilderTraitTest()
+	builderTrait.Tasks = append(builderTrait.Tasks, "test;alpine;/bin/bash -c \"cd test && echo 'helooo'\";1000")
+
+	tasks, err := builderTrait.customTasks(nil, "my-kit-img")
+
+	assert.Nil(t, err)
+	assert.Equal(t, 1, len(tasks))
+	assert.Equal(t, "test", tasks[0].Custom.Name)
+	assert.Equal(t, "alpine", tasks[0].Custom.ContainerImage)
+	assert.Equal(t, "/bin/bash", tasks[0].Custom.ContainerCommands[0])
+	assert.Equal(t, "-c", tasks[0].Custom.ContainerCommands[1])
+	assert.Equal(t, "cd test && echo 'helooo'", tasks[0].Custom.ContainerCommands[2])
+	assert.Equal(t, int64(1000), *tasks[0].Custom.ContainerUserID)
 }
 
 func TestBuilderCustomTasksConfiguration(t *testing.T) {