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/05/29 12:33:33 UTC

[camel-k] branch main updated (1beec1a09 -> bac85e14a)

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 1beec1a09 chore(ci): support multi platform build
     new eaee49d3e feat(core): Support S2I for builder image generation
     new bac85e14a feat(core): Refactoring S2I BC and IS build code

The 2 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:
 pkg/builder/s2i.go                   |  72 +-------
 pkg/controller/build/build_pod.go    |   8 +-
 pkg/controller/catalog/initialize.go | 344 +++++++++++++++++++++++++++++++++--
 pkg/util/s2i/build.go                | 108 +++++++++++
 4 files changed, 454 insertions(+), 78 deletions(-)
 create mode 100644 pkg/util/s2i/build.go


[camel-k] 01/02: feat(core): Support S2I for builder image generation

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 eaee49d3ea8eb14b835f995f5573aad83238ed8e
Author: Gaelle Fournier <ga...@gmail.com>
AuthorDate: Thu May 25 15:12:45 2023 +0200

    feat(core): Support S2I for builder image generation
    
    * Add initialize builder image on catalog with imagestream and buildconfig resource who's owner is the CamelCatalog
    * Light refactoring of S2I code
    
    Ref #4297
---
 pkg/builder/s2i.go                   |  44 +----
 pkg/controller/build/build_pod.go    |   8 +-
 pkg/controller/catalog/initialize.go | 366 +++++++++++++++++++++++++++++++++--
 pkg/util/s2i/build.go                |  70 +++++++
 4 files changed, 434 insertions(+), 54 deletions(-)

diff --git a/pkg/builder/s2i.go b/pkg/builder/s2i.go
index 99cbd00ea..043ba70cf 100644
--- a/pkg/builder/s2i.go
+++ b/pkg/builder/s2i.go
@@ -29,7 +29,6 @@ import (
 	"os"
 	"path/filepath"
 	"strings"
-	"time"
 
 	corev1 "k8s.io/api/core/v1"
 	apierrors "k8s.io/apimachinery/pkg/api/errors"
@@ -49,6 +48,7 @@ import (
 	"github.com/apache/camel-k/v2/pkg/client"
 	"github.com/apache/camel-k/v2/pkg/util"
 	"github.com/apache/camel-k/v2/pkg/util/log"
+	"github.com/apache/camel-k/v2/pkg/util/s2i"
 )
 
 type s2iTask struct {
@@ -203,11 +203,11 @@ func (t *s2iTask) Do(ctx context.Context) v1.BuildStatus {
 			return fmt.Errorf("cannot unmarshal instantiated binary response: %w", err)
 		}
 
-		err = t.waitForS2iBuildCompletion(ctx, t.c, &s2iBuild)
+		err = s2i.WaitForS2iBuildCompletion(ctx, t.c, &s2iBuild)
 		if err != nil {
 			if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
 				// nolint: contextcheck
-				if err := t.cancelBuild(context.Background(), &s2iBuild); err != nil {
+				if err := s2i.CancelBuild(context.Background(), t.c, &s2iBuild); err != nil {
 					log.Errorf(err, "cannot cancel s2i Build: %s/%s", s2iBuild.Namespace, s2iBuild.Name)
 				}
 			}
@@ -255,44 +255,6 @@ func (t *s2iTask) getControllerReference() metav1.Object {
 	return owner
 }
 
-func (t *s2iTask) waitForS2iBuildCompletion(ctx context.Context, c client.Client, build *buildv1.Build) error {
-	key := ctrl.ObjectKeyFromObject(build)
-	for {
-		select {
-
-		case <-ctx.Done():
-			return ctx.Err()
-
-		case <-time.After(1 * time.Second):
-			err := c.Get(ctx, key, build)
-			if err != nil {
-				if apierrors.IsNotFound(err) {
-					continue
-				}
-				return err
-			}
-
-			if build.Status.Phase == buildv1.BuildPhaseComplete {
-				return nil
-			} else if build.Status.Phase == buildv1.BuildPhaseCancelled ||
-				build.Status.Phase == buildv1.BuildPhaseFailed ||
-				build.Status.Phase == buildv1.BuildPhaseError {
-				return errors.New("build failed")
-			}
-		}
-	}
-}
-
-func (t *s2iTask) cancelBuild(ctx context.Context, build *buildv1.Build) error {
-	target := build.DeepCopy()
-	target.Status.Cancelled = true
-	if err := t.c.Patch(ctx, target, ctrl.MergeFrom(build)); err != nil {
-		return err
-	}
-	*build = *target
-	return nil
-}
-
 func tarDir(src string, writers ...io.Writer) error {
 	// ensure the src actually exists before trying to tar it
 	if _, err := os.Stat(src); err != nil {
diff --git a/pkg/controller/build/build_pod.go b/pkg/controller/build/build_pod.go
index 7f660edb8..ba996bb82 100644
--- a/pkg/controller/build/build_pod.go
+++ b/pkg/controller/build/build_pod.go
@@ -272,6 +272,12 @@ func addBuildTaskToPod(build *v1.Build, taskName string, pod *corev1.Pod) {
 		)
 	}
 
+	var envVars = proxyFromEnvironment()
+	envVars = append(envVars, corev1.EnvVar{
+		Name:  "HOME",
+		Value: filepath.Join(builderDir, build.Name),
+	})
+
 	container := corev1.Container{
 		Name:            taskName,
 		Image:           build.BuilderConfiguration().ToolImage,
@@ -287,7 +293,7 @@ func addBuildTaskToPod(build *v1.Build, taskName string, pod *corev1.Pod) {
 			taskName,
 		},
 		WorkingDir: filepath.Join(builderDir, build.Name),
-		Env:        proxyFromEnvironment(),
+		Env:        envVars,
 	}
 
 	configureResources(build, &container)
diff --git a/pkg/controller/catalog/initialize.go b/pkg/controller/catalog/initialize.go
index 98d4ddfbe..40a0618ab 100644
--- a/pkg/controller/catalog/initialize.go
+++ b/pkg/controller/catalog/initialize.go
@@ -18,11 +18,16 @@ limitations under the License.
 package catalog
 
 import (
+	"archive/tar"
 	"bufio"
+	"compress/gzip"
 	"context"
+	"encoding/json"
+	"errors"
 	"fmt"
 	"io"
 	"os"
+	"path/filepath"
 	"runtime"
 	"strings"
 	"time"
@@ -32,9 +37,19 @@ import (
 	"github.com/apache/camel-k/v2/pkg/client"
 	platformutil "github.com/apache/camel-k/v2/pkg/platform"
 	"github.com/apache/camel-k/v2/pkg/util"
+	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
+	"github.com/apache/camel-k/v2/pkg/util/s2i"
 	spectrum "github.com/container-tools/spectrum/pkg/builder"
 	gcrv1 "github.com/google/go-containerregistry/pkg/v1"
+	buildv1 "github.com/openshift/api/build/v1"
+	imagev1 "github.com/openshift/api/image/v1"
 	corev1 "k8s.io/api/core/v1"
+	k8serrors "k8s.io/apimachinery/pkg/api/errors"
+	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime/schema"
+	"k8s.io/apimachinery/pkg/runtime/serializer"
+	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
+	"sigs.k8s.io/controller-runtime/pkg/client/apiutil"
 )
 
 // NewInitializeAction returns a action that initializes the catalog configuration when not provided by the user.
@@ -66,16 +81,20 @@ func (action *initializeAction) Handle(ctx context.Context, catalog *v1.CamelCat
 		return catalog, nil
 	}
 
+	if platform.Status.Build.PublishStrategy == v1.IntegrationPlatformBuildPublishStrategyS2I {
+		return initializeS2i(ctx, action.client, platform, catalog)
+	}
+	// Default to spectrum
 	// Make basic options for building image in the registry
 	options, err := makeSpectrumOptions(ctx, action.client, platform.Namespace, platform.Status.Build.Registry)
 	if err != nil {
 		return catalog, err
 	}
+	return initializeSpectrum(options, platform, catalog)
 
-	return initialize(options, platform, catalog)
 }
 
-func initialize(options spectrum.Options, ip *v1.IntegrationPlatform, catalog *v1.CamelCatalog) (*v1.CamelCatalog, error) {
+func initializeSpectrum(options spectrum.Options, ip *v1.IntegrationPlatform, catalog *v1.CamelCatalog) (*v1.CamelCatalog, error) {
 	target := catalog.DeepCopy()
 	imageName := fmt.Sprintf(
 		"%s/camel-k-runtime-%s-builder:%s",
@@ -99,7 +118,7 @@ func initialize(options spectrum.Options, ip *v1.IntegrationPlatform, catalog *v
 	options.Stderr = newStdW
 	options.Stdout = newStdW
 
-	if !imageSnapshot(options) && imageExists(options) {
+	if !imageSnapshot(options.Base) && imageExistsSpectrum(options) {
 		target.Status.Phase = v1.CamelCatalogPhaseReady
 		target.Status.SetCondition(
 			v1.CamelCatalogConditionReady,
@@ -116,7 +135,7 @@ func initialize(options spectrum.Options, ip *v1.IntegrationPlatform, catalog *v
 	options.Base = catalog.Spec.GetQuarkusToolingImage()
 	options.Target = imageName
 
-	err := buildRuntimeBuilderWithTimeout(options, ip.Status.Build.GetBuildCatalogToolTimeout().Duration)
+	err := buildRuntimeBuilderWithTimeoutSpectrum(options, ip.Status.Build.GetBuildCatalogToolTimeout().Duration)
 
 	if err != nil {
 		target.Status.Phase = v1.CamelCatalogPhaseError
@@ -139,7 +158,254 @@ func initialize(options spectrum.Options, ip *v1.IntegrationPlatform, catalog *v
 	return target, nil
 }
 
-func imageExists(options spectrum.Options) bool {
+func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatform, catalog *v1.CamelCatalog) (*v1.CamelCatalog, error) {
+	target := catalog.DeepCopy()
+	// No registry in s2i
+	imageName := fmt.Sprintf(
+		"camel-k-runtime-%s-builder",
+		catalog.Spec.Runtime.Provider,
+	)
+	imageTag := strings.ToLower(catalog.Spec.Runtime.Version)
+
+	// Dockfile
+	dockerfile := string([]byte(`
+		FROM ` + catalog.Spec.GetQuarkusToolingImage() + `
+		USER 1000
+		ADD /usr/local/bin/kamel /usr/local/bin/kamel
+		ADD /usr/share/maven/mvnw/ /usr/share/maven/mvnw/
+	`))
+
+	// BuildConfig
+	bc := &buildv1.BuildConfig{
+		TypeMeta: metav1.TypeMeta{
+			APIVersion: buildv1.GroupVersion.String(),
+			Kind:       "BuildConfig",
+		},
+		ObjectMeta: metav1.ObjectMeta{
+			Name:      imageName,
+			Namespace: ip.Namespace,
+			Labels: map[string]string{
+				kubernetes.CamelCreatorLabelKind:      v1.CamelCatalogKind,
+				kubernetes.CamelCreatorLabelName:      catalog.Name,
+				kubernetes.CamelCreatorLabelNamespace: catalog.Namespace,
+				kubernetes.CamelCreatorLabelVersion:   catalog.ResourceVersion,
+				"camel.apache.org/runtime.version":    catalog.Spec.Runtime.Version,
+				"camel.apache.org/runtime.provider":   string(catalog.Spec.Runtime.Provider),
+			},
+			OwnerReferences: []metav1.OwnerReference{
+				{
+					APIVersion: catalog.APIVersion,
+					Kind:       catalog.Kind,
+					Name:       catalog.Name,
+					UID:        catalog.UID,
+				},
+			},
+		},
+		Spec: buildv1.BuildConfigSpec{
+			CommonSpec: buildv1.CommonSpec{
+				Source: buildv1.BuildSource{
+					Type:       buildv1.BuildSourceBinary,
+					Dockerfile: &dockerfile,
+				},
+				Strategy: buildv1.BuildStrategy{
+					DockerStrategy: &buildv1.DockerBuildStrategy{},
+				},
+				Output: buildv1.BuildOutput{
+					To: &corev1.ObjectReference{
+						Kind: "ImageStreamTag",
+						Name: imageName + ":" + imageTag,
+					},
+				},
+			},
+		},
+	}
+
+	// ImageStream
+	is := &imagev1.ImageStream{
+		TypeMeta: metav1.TypeMeta{
+			APIVersion: imagev1.GroupVersion.String(),
+			Kind:       "ImageStream",
+		},
+		ObjectMeta: metav1.ObjectMeta{
+			Name:      bc.Name,
+			Namespace: bc.Namespace,
+			Labels: map[string]string{
+				kubernetes.CamelCreatorLabelKind:      v1.CamelCatalogKind,
+				kubernetes.CamelCreatorLabelName:      catalog.Name,
+				kubernetes.CamelCreatorLabelNamespace: catalog.Namespace,
+				kubernetes.CamelCreatorLabelVersion:   catalog.ResourceVersion,
+				"camel.apache.org/runtime.provider":   string(catalog.Spec.Runtime.Provider),
+			},
+			OwnerReferences: []metav1.OwnerReference{
+				{
+					APIVersion: catalog.APIVersion,
+					Kind:       catalog.Kind,
+					Name:       catalog.Name,
+					UID:        catalog.UID,
+				},
+			},
+		},
+		Spec: imagev1.ImageStreamSpec{
+			LookupPolicy: imagev1.ImageLookupPolicy{
+				Local: true,
+			},
+		},
+	}
+
+	if !imageSnapshot(imageName+":"+imageTag) && imageExistsS2i(ctx, c, is) {
+		target.Status.Phase = v1.CamelCatalogPhaseReady
+		target.Status.SetCondition(
+			v1.CamelCatalogConditionReady,
+			corev1.ConditionTrue,
+			"Builder Image",
+			"Container image exists on registry (later)",
+		)
+		target.Status.Image = imageName
+		return target, nil
+	}
+
+	err := c.Delete(ctx, bc)
+	if err != nil && !k8serrors.IsNotFound(err) {
+		target.Status.Phase = v1.CamelCatalogPhaseError
+		target.Status.SetErrorCondition(
+			v1.CamelCatalogConditionReady,
+			"Builder Image",
+			err,
+		)
+		return target, err
+	}
+
+	err = c.Create(ctx, bc)
+	if err != nil {
+		target.Status.Phase = v1.CamelCatalogPhaseError
+		target.Status.SetErrorCondition(
+			v1.CamelCatalogConditionReady,
+			"Builder Image",
+			err,
+		)
+		return target, err
+	}
+
+	err = c.Delete(ctx, is)
+	if err != nil && !k8serrors.IsNotFound(err) {
+		target.Status.Phase = v1.CamelCatalogPhaseError
+		target.Status.SetErrorCondition(
+			v1.CamelCatalogConditionReady,
+			"Builder Image",
+			err,
+		)
+		return target, err
+	}
+
+	err = c.Create(ctx, is)
+	if err != nil {
+		target.Status.Phase = v1.CamelCatalogPhaseError
+		target.Status.SetErrorCondition(
+			v1.CamelCatalogConditionReady,
+			"Builder Image",
+			err,
+		)
+		return target, err
+	}
+
+	err = util.WithTempDir(imageName+"-s2i-", func(tmpDir string) error {
+		archive := filepath.Join(tmpDir, "archive.tar.gz")
+
+		archiveFile, err := os.Create(archive)
+		if err != nil {
+			return fmt.Errorf("cannot create tar archive: %w", err)
+		}
+
+		err = tarEntries(archiveFile, "/usr/local/bin/kamel:/usr/local/bin/kamel",
+			"/usr/share/maven/mvnw/:/usr/share/maven/mvnw/")
+		if err != nil {
+			return fmt.Errorf("cannot tar path entry: %w", err)
+		}
+
+		f, err := util.Open(archive)
+		if err != nil {
+			return err
+		}
+
+		restClient, err := apiutil.RESTClientForGVK(
+			schema.GroupVersionKind{Group: "build.openshift.io", Version: "v1"}, false,
+			c.GetConfig(), serializer.NewCodecFactory(c.GetScheme()))
+		if err != nil {
+			return err
+		}
+
+		r := restClient.Post().
+			Namespace(bc.Namespace).
+			Body(bufio.NewReader(f)).
+			Resource("buildconfigs").
+			Name(bc.Name).
+			SubResource("instantiatebinary").
+			Do(ctx)
+
+		if r.Error() != nil {
+			return fmt.Errorf("cannot instantiate binary: %w", err)
+		}
+
+		data, err := r.Raw()
+		if err != nil {
+			return fmt.Errorf("no raw data retrieved: %w", err)
+		}
+
+		s2iBuild := buildv1.Build{}
+		err = json.Unmarshal(data, &s2iBuild)
+		if err != nil {
+			return fmt.Errorf("cannot unmarshal instantiated binary response: %w", err)
+		}
+
+		err = s2i.WaitForS2iBuildCompletion(ctx, c, &s2iBuild)
+		if err != nil {
+			if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
+				// nolint: contextcheck
+				if err := s2i.CancelBuild(context.Background(), c, &s2iBuild); err != nil {
+					return fmt.Errorf("cannot cancel s2i Build: %s/%s", s2iBuild.Namespace, s2iBuild.Name)
+				}
+			}
+			return err
+		}
+		if s2iBuild.Status.Output.To != nil {
+			Log.Infof("Camel K builder container image %s:%s@%s created", imageName, imageTag, s2iBuild.Status.Output.To.ImageDigest)
+		}
+
+		err = c.Get(ctx, ctrl.ObjectKeyFromObject(is), is)
+		if err != nil {
+			return err
+		}
+
+		if is.Status.DockerImageRepository == "" {
+			return errors.New("dockerImageRepository not available in ImageStream")
+		}
+
+		target.Status.Phase = v1.CamelCatalogPhaseReady
+		target.Status.SetCondition(
+			v1.CamelCatalogConditionReady,
+			corev1.ConditionTrue,
+			"Builder Image",
+			"Container image successfully built",
+		)
+		target.Status.Image = is.Status.DockerImageRepository + ":" + imageTag
+
+		return f.Close()
+	})
+
+	if err != nil {
+		target.Status.Phase = v1.CamelCatalogPhaseError
+		target.Status.SetErrorCondition(
+			v1.CamelCatalogConditionReady,
+			"Builder Image",
+			err,
+		)
+		return target, err
+	}
+
+	return target, nil
+}
+
+func imageExistsSpectrum(options spectrum.Options) bool {
 	Log.Infof("Checking if Camel K builder container %s already exists...", options.Base)
 	ctrImg, err := spectrum.Pull(options)
 	if ctrImg != nil && err == nil {
@@ -156,18 +422,38 @@ func imageExists(options spectrum.Options) bool {
 	return false
 }
 
-func imageSnapshot(options spectrum.Options) bool {
-	return strings.HasSuffix(options.Base, "snapshot")
+func imageExistsS2i(ctx context.Context, c client.Client, is *imagev1.ImageStream) bool {
+	Log.Infof("Checking if Camel K builder container %s already exists...", is.Name)
+	key := ctrl.ObjectKey{
+		Namespace: is.Namespace,
+		Name:      is.Name,
+	}
+
+	err := c.Get(ctx, key, is)
+
+	if err != nil {
+		if !k8serrors.IsNotFound(err) {
+			Log.Infof("Couldn't pull image due to %s", err.Error())
+		}
+		Log.Info("Could not find Camel K builder container")
+		return false
+	}
+	Log.Info("Found Camel K builder container ")
+	return true
+}
+
+func imageSnapshot(imageName string) bool {
+	return strings.HasSuffix(imageName, "snapshot")
 }
 
-func buildRuntimeBuilderWithTimeout(options spectrum.Options, timeout time.Duration) error {
+func buildRuntimeBuilderWithTimeoutSpectrum(options spectrum.Options, timeout time.Duration) error {
 	// Backward compatibility with IP which had not a timeout field
 	if timeout == 0 {
-		return buildRuntimeBuilderImage(options)
+		return buildRuntimeBuilderImageSpectrum(options)
 	}
 	result := make(chan error, 1)
 	go func() {
-		result <- buildRuntimeBuilderImage(options)
+		result <- buildRuntimeBuilderImageSpectrum(options)
 	}()
 	select {
 	case <-time.After(timeout):
@@ -179,7 +465,7 @@ func buildRuntimeBuilderWithTimeout(options spectrum.Options, timeout time.Durat
 
 // This func will take care to dynamically build an image that will contain the tools required
 // by the catalog build plus kamel binary and a maven wrapper required for the build.
-func buildRuntimeBuilderImage(options spectrum.Options) error {
+func buildRuntimeBuilderImageSpectrum(options spectrum.Options) error {
 	if options.Base == "" {
 		return fmt.Errorf("missing base image, likely catalog is not compatible with this Camel K version")
 	}
@@ -189,7 +475,6 @@ func buildRuntimeBuilderImage(options spectrum.Options) error {
 		options.Jobs = jobs
 	}
 
-	// TODO support also S2I
 	_, err := spectrum.Build(options,
 		"/usr/local/bin/kamel:/usr/local/bin/",
 		"/usr/share/maven/mvnw/:/usr/share/maven/mvnw/")
@@ -227,3 +512,60 @@ func makeSpectrumOptions(ctx context.Context, c client.Client, platformNamespace
 
 	return options, nil
 }
+
+// Add entries (files or folders) into tar with the possibility to change its path.
+func tarEntries(writer io.Writer, files ...string) error {
+
+	gzw := gzip.NewWriter(writer)
+	defer util.CloseQuietly(gzw)
+
+	tw := tar.NewWriter(gzw)
+	defer util.CloseQuietly(tw)
+
+	// Iterate over files and and add them to the tar archive
+	for _, fileDetail := range files {
+		fileSource := strings.Split(fileDetail, ":")[0]
+		fileTarget := strings.Split(fileDetail, ":")[1]
+		// ensure the src actually exists before trying to tar it
+		if _, err := os.Stat(fileSource); err != nil {
+			return fmt.Errorf("unable to tar files: %w", err)
+		}
+
+		if err := filepath.Walk(fileSource, func(file string, fi os.FileInfo, err error) error {
+			if err != nil {
+				return err
+			}
+
+			if !fi.Mode().IsRegular() {
+				return nil
+			}
+
+			header, err := tar.FileInfoHeader(fi, fi.Name())
+			if err != nil {
+				return err
+			}
+
+			// update the name to correctly reflect the desired destination when un-taring
+			header.Name = strings.TrimPrefix(strings.ReplaceAll(file, fileSource, fileTarget), string(filepath.Separator))
+
+			if err := tw.WriteHeader(header); err != nil {
+				return err
+			}
+
+			f, err := util.Open(file)
+			if err != nil {
+				return err
+			}
+
+			if _, err := io.Copy(tw, f); err != nil {
+				return err
+			}
+
+			return f.Close()
+		}); err != nil {
+			return fmt.Errorf("unable to tar: %w", err)
+		}
+
+	}
+	return nil
+}
diff --git a/pkg/util/s2i/build.go b/pkg/util/s2i/build.go
new file mode 100644
index 000000000..d30207d59
--- /dev/null
+++ b/pkg/util/s2i/build.go
@@ -0,0 +1,70 @@
+/*
+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 s2i contains utilities for openshift s2i builds.
+package s2i
+
+import (
+	"context"
+	"errors"
+	"time"
+
+	"github.com/apache/camel-k/v2/pkg/client"
+	buildv1 "github.com/openshift/api/build/v1"
+	apierrors "k8s.io/apimachinery/pkg/api/errors"
+	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
+)
+
+// Cancel the s2i Build by updating its status.
+func CancelBuild(ctx context.Context, c client.Client, build *buildv1.Build) error {
+	target := build.DeepCopy()
+	target.Status.Cancelled = true
+	if err := c.Patch(ctx, target, ctrl.MergeFrom(build)); err != nil {
+		return err
+	}
+	*build = *target
+	return nil
+}
+
+// Wait for the s2i Build to complete with success or cancellation.
+func WaitForS2iBuildCompletion(ctx context.Context, c client.Client, build *buildv1.Build) error {
+	key := ctrl.ObjectKeyFromObject(build)
+	for {
+		select {
+
+		case <-ctx.Done():
+			return ctx.Err()
+
+		case <-time.After(1 * time.Second):
+			err := c.Get(ctx, key, build)
+			if err != nil {
+				if apierrors.IsNotFound(err) {
+					continue
+				}
+				return err
+			}
+
+			if build.Status.Phase == buildv1.BuildPhaseComplete {
+				return nil
+			} else if build.Status.Phase == buildv1.BuildPhaseCancelled ||
+				build.Status.Phase == buildv1.BuildPhaseFailed ||
+				build.Status.Phase == buildv1.BuildPhaseError {
+				return errors.New("build failed")
+			}
+		}
+	}
+}


[camel-k] 02/02: feat(core): Refactoring S2I BC and IS build code

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 bac85e14a9b2f350881a4c7daf80818afa6f33a4
Author: Gaelle Fournier <ga...@gmail.com>
AuthorDate: Fri May 26 17:33:35 2023 +0200

    feat(core): Refactoring S2I BC and IS build code
    
    Ref #4297
---
 pkg/builder/s2i.go                   | 28 +++--------------
 pkg/controller/catalog/initialize.go | 58 +++++++++++-------------------------
 pkg/util/s2i/build.go                | 38 +++++++++++++++++++++++
 3 files changed, 60 insertions(+), 64 deletions(-)

diff --git a/pkg/builder/s2i.go b/pkg/builder/s2i.go
index 043ba70cf..da6a66b36 100644
--- a/pkg/builder/s2i.go
+++ b/pkg/builder/s2i.go
@@ -31,7 +31,6 @@ import (
 	"strings"
 
 	corev1 "k8s.io/api/core/v1"
-	apierrors "k8s.io/apimachinery/pkg/api/errors"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
 	"k8s.io/apimachinery/pkg/runtime/schema"
@@ -39,7 +38,6 @@ import (
 
 	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
 	"sigs.k8s.io/controller-runtime/pkg/client/apiutil"
-	ctrlutil "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
 
 	buildv1 "github.com/openshift/api/build/v1"
 	imagev1 "github.com/openshift/api/image/v1"
@@ -90,11 +88,6 @@ func (t *s2iTask) Do(ctx context.Context) v1.BuildStatus {
 		},
 	}
 
-	err := t.c.Delete(ctx, bc)
-	if err != nil && !apierrors.IsNotFound(err) {
-		return status.Failed(fmt.Errorf("cannot delete build config: %w", err))
-	}
-
 	// Set the build controller as owner reference
 	owner := t.getControllerReference()
 	if owner == nil {
@@ -102,13 +95,9 @@ func (t *s2iTask) Do(ctx context.Context) v1.BuildStatus {
 		owner = t.build
 	}
 
-	if err := ctrlutil.SetOwnerReference(owner, bc, t.c.GetScheme()); err != nil {
-		return status.Failed(fmt.Errorf("cannot set owner reference on BuildConfig: %s: %w", bc.Name, err))
-	}
-
-	err = t.c.Create(ctx, bc)
+	err := s2i.BuildConfig(ctx, t.c, bc, owner)
 	if err != nil {
-		return status.Failed(fmt.Errorf("cannot create build config: %w", err))
+		return status.Failed(err)
 	}
 
 	is := &imagev1.ImageStream{
@@ -128,18 +117,9 @@ func (t *s2iTask) Do(ctx context.Context) v1.BuildStatus {
 		},
 	}
 
-	err = t.c.Delete(ctx, is)
-	if err != nil && !apierrors.IsNotFound(err) {
-		return status.Failed(fmt.Errorf("cannot delete image stream: %w", err))
-	}
-
-	if err := ctrlutil.SetOwnerReference(owner, is, t.c.GetScheme()); err != nil {
-		return status.Failed(fmt.Errorf("cannot set owner reference on ImageStream: %s: %w", is.Name, err))
-	}
-
-	err = t.c.Create(ctx, is)
+	err = s2i.ImageStream(ctx, t.c, is, owner)
 	if err != nil {
-		return status.Failed(fmt.Errorf("cannot create image stream: %w", err))
+		return status.Failed(err)
 	}
 
 	err = util.WithTempDir(t.build.Name+"-s2i-", func(tmpDir string) error {
diff --git a/pkg/controller/catalog/initialize.go b/pkg/controller/catalog/initialize.go
index 40a0618ab..64947d9de 100644
--- a/pkg/controller/catalog/initialize.go
+++ b/pkg/controller/catalog/initialize.go
@@ -39,15 +39,20 @@ import (
 	"github.com/apache/camel-k/v2/pkg/util"
 	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
 	"github.com/apache/camel-k/v2/pkg/util/s2i"
+
 	spectrum "github.com/container-tools/spectrum/pkg/builder"
 	gcrv1 "github.com/google/go-containerregistry/pkg/v1"
+
 	buildv1 "github.com/openshift/api/build/v1"
 	imagev1 "github.com/openshift/api/image/v1"
+
 	corev1 "k8s.io/api/core/v1"
 	k8serrors "k8s.io/apimachinery/pkg/api/errors"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
 	"k8s.io/apimachinery/pkg/runtime/schema"
 	"k8s.io/apimachinery/pkg/runtime/serializer"
+
 	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
 	"sigs.k8s.io/controller-runtime/pkg/client/apiutil"
 )
@@ -175,6 +180,8 @@ func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatf
 		ADD /usr/share/maven/mvnw/ /usr/share/maven/mvnw/
 	`))
 
+	owner := catalogReference(catalog)
+
 	// BuildConfig
 	bc := &buildv1.BuildConfig{
 		TypeMeta: metav1.TypeMeta{
@@ -192,14 +199,6 @@ func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatf
 				"camel.apache.org/runtime.version":    catalog.Spec.Runtime.Version,
 				"camel.apache.org/runtime.provider":   string(catalog.Spec.Runtime.Provider),
 			},
-			OwnerReferences: []metav1.OwnerReference{
-				{
-					APIVersion: catalog.APIVersion,
-					Kind:       catalog.Kind,
-					Name:       catalog.Name,
-					UID:        catalog.UID,
-				},
-			},
 		},
 		Spec: buildv1.BuildConfigSpec{
 			CommonSpec: buildv1.CommonSpec{
@@ -236,14 +235,6 @@ func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatf
 				kubernetes.CamelCreatorLabelVersion:   catalog.ResourceVersion,
 				"camel.apache.org/runtime.provider":   string(catalog.Spec.Runtime.Provider),
 			},
-			OwnerReferences: []metav1.OwnerReference{
-				{
-					APIVersion: catalog.APIVersion,
-					Kind:       catalog.Kind,
-					Name:       catalog.Name,
-					UID:        catalog.UID,
-				},
-			},
 		},
 		Spec: imagev1.ImageStreamSpec{
 			LookupPolicy: imagev1.ImageLookupPolicy{
@@ -264,18 +255,7 @@ func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatf
 		return target, nil
 	}
 
-	err := c.Delete(ctx, bc)
-	if err != nil && !k8serrors.IsNotFound(err) {
-		target.Status.Phase = v1.CamelCatalogPhaseError
-		target.Status.SetErrorCondition(
-			v1.CamelCatalogConditionReady,
-			"Builder Image",
-			err,
-		)
-		return target, err
-	}
-
-	err = c.Create(ctx, bc)
+	err := s2i.BuildConfig(ctx, c, bc, owner)
 	if err != nil {
 		target.Status.Phase = v1.CamelCatalogPhaseError
 		target.Status.SetErrorCondition(
@@ -286,18 +266,7 @@ func initializeS2i(ctx context.Context, c client.Client, ip *v1.IntegrationPlatf
 		return target, err
 	}
 
-	err = c.Delete(ctx, is)
-	if err != nil && !k8serrors.IsNotFound(err) {
-		target.Status.Phase = v1.CamelCatalogPhaseError
-		target.Status.SetErrorCondition(
-			v1.CamelCatalogConditionReady,
-			"Builder Image",
-			err,
-		)
-		return target, err
-	}
-
-	err = c.Create(ctx, is)
+	err = s2i.ImageStream(ctx, c, is, owner)
 	if err != nil {
 		target.Status.Phase = v1.CamelCatalogPhaseError
 		target.Status.SetErrorCondition(
@@ -569,3 +538,12 @@ func tarEntries(writer io.Writer, files ...string) error {
 	}
 	return nil
 }
+
+func catalogReference(catalog *v1.CamelCatalog) *unstructured.Unstructured {
+	owner := &unstructured.Unstructured{}
+	owner.SetName(catalog.Name)
+	owner.SetUID(catalog.UID)
+	owner.SetAPIVersion(catalog.APIVersion)
+	owner.SetKind(catalog.Kind)
+	return owner
+}
diff --git a/pkg/util/s2i/build.go b/pkg/util/s2i/build.go
index d30207d59..82e68d40e 100644
--- a/pkg/util/s2i/build.go
+++ b/pkg/util/s2i/build.go
@@ -21,12 +21,18 @@ package s2i
 import (
 	"context"
 	"errors"
+	"fmt"
 	"time"
 
 	"github.com/apache/camel-k/v2/pkg/client"
 	buildv1 "github.com/openshift/api/build/v1"
+	imagev1 "github.com/openshift/api/image/v1"
+
 	apierrors "k8s.io/apimachinery/pkg/api/errors"
+	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+
 	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
+	ctrlutil "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
 )
 
 // Cancel the s2i Build by updating its status.
@@ -68,3 +74,35 @@ func WaitForS2iBuildCompletion(ctx context.Context, c client.Client, build *buil
 		}
 	}
 }
+
+// Create the BuildConfig of the build with the right owner after having deleted it it already existed.
+func BuildConfig(ctx context.Context, c client.Client, bc *buildv1.BuildConfig, owner metav1.Object) error {
+	if err := c.Delete(ctx, bc); err != nil && !apierrors.IsNotFound(err) {
+		return fmt.Errorf("cannot delete build config: %w", err)
+	}
+
+	if err := ctrlutil.SetOwnerReference(owner, bc, c.GetScheme()); err != nil {
+		return fmt.Errorf("cannot set owner reference on BuildConfig: %s: %w", bc.Name, err)
+	}
+
+	if err := c.Create(ctx, bc); err != nil {
+		return fmt.Errorf("cannot create build config: %w", err)
+	}
+	return nil
+}
+
+// Create the ImageStream for the builded image with the right owner after having deleted it it already existed.
+func ImageStream(ctx context.Context, c client.Client, is *imagev1.ImageStream, owner metav1.Object) error {
+	if err := c.Delete(ctx, is); err != nil && !apierrors.IsNotFound(err) {
+		return fmt.Errorf("cannot delete image stream: %w", err)
+	}
+
+	if err := ctrlutil.SetOwnerReference(owner, is, c.GetScheme()); err != nil {
+		return fmt.Errorf("cannot set owner reference on ImageStream: %s: %w", is.Name, err)
+	}
+
+	if err := c.Create(ctx, is); err != nil {
+		return fmt.Errorf("cannot create image stream: %w", err)
+	}
+	return nil
+}