You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by lb...@apache.org on 2019/01/13 14:47:20 UTC

[camel-k] 06/09: Fix #312: refactor platform image lookup into its own package

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

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

commit 4390c77a9c9a79e1752dd838a2793f0d247c4f2c
Author: nferraro <ni...@gmail.com>
AuthorDate: Fri Jan 11 11:41:20 2019 +0100

    Fix #312: refactor platform image lookup into its own package
---
 cmd/util/publisher/publisher.go                    | 20 +++---
 pkg/builder/builder.go                             |  5 --
 pkg/builder/builder_steps.go                       | 49 -------------
 pkg/builder/kaniko/kaniko.go                       |  1 -
 pkg/builder/s2i/s2i.go                             |  1 -
 pkg/controller/integration/build_context.go        | 20 ++++--
 pkg/controller/integration/util.go                 | 44 ++++++++++--
 pkg/{builder/s2i/s2i.go => platform/images/doc.go} | 17 +----
 pkg/platform/images/images.go                      | 82 ++++++++++++++++++++++
 pkg/platform/images/images_test.go                 | 68 ++++++++++++++++++
 pkg/trait/builder_test.go                          |  4 +-
 11 files changed, 219 insertions(+), 92 deletions(-)

diff --git a/cmd/util/publisher/publisher.go b/cmd/util/publisher/publisher.go
index 857bbc0..e4dc8c3 100644
--- a/cmd/util/publisher/publisher.go
+++ b/cmd/util/publisher/publisher.go
@@ -33,6 +33,7 @@ import (
 	"github.com/apache/camel-k/pkg/apis"
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/builder"
+	"github.com/apache/camel-k/pkg/platform/images"
 	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
 	"github.com/pkg/errors"
@@ -60,7 +61,7 @@ func main() {
 
 	cmd.Flags().StringVar(&options.StartWith, "start-with", "", "The component to start with")
 	cmd.Flags().StringVar(&options.EndWith, "end-with", "", "The component to end with")
-	cmd.Flags().IntVar(&options.BuildAttempts, "attempts", 5, "The maximum number of build attempts")
+	cmd.Flags().IntVar(&options.BuildAttempts, "attempts", 5, "The maximum number of build attempts for each image")
 
 	panicIfErr(cmd.Execute())
 }
@@ -127,6 +128,13 @@ func (options *PublisherOptions) build(component string, camelVersion string) er
 	}
 	defer os.RemoveAll(dir)
 
+	dependencies := make([]string, 0)
+	for d := range images.StandardDependencies {
+		dependencies = append(dependencies, d)
+	}
+	dependencies = append(dependencies, images.BaseDependency)
+	dependencies = append(dependencies, "camel:"+component)
+
 	ctx := builder.Context{
 		C:    context.TODO(),
 		Path: dir,
@@ -136,13 +144,7 @@ func (options *PublisherOptions) build(component string, camelVersion string) er
 					CamelVersion: camelVersion,
 				},
 			},
-			Dependencies: []string{
-				"camel-k:knative",
-				"camel:core",
-				"runtime:jvm",
-				"runtime:yaml",
-				"camel:" + component,
-			},
+			Dependencies: dependencies,
 		},
 	}
 
@@ -170,7 +172,7 @@ func (options *PublisherOptions) build(component string, camelVersion string) er
 		return err
 	}
 
-	image := builder.PredefinedImageNameFor(component)
+	image := images.PredefinedImageNameFor(component)
 	buildCmd := exec.Command("docker", "build", "-t", image, archiveDir)
 	buildCmd.Stdout = os.Stdout
 	buildCmd.Stderr = os.Stderr
diff --git a/pkg/builder/builder.go b/pkg/builder/builder.go
index ae61eaa..9becccb 100644
--- a/pkg/builder/builder.go
+++ b/pkg/builder/builder.go
@@ -168,11 +168,6 @@ func (b *defaultBuilder) submit(request Request) {
 			break
 		}
 
-		if c.Image != "" && c.PublicImage != "" {
-			logrus.Info("image already computed: skipping following steps")
-			break
-		}
-
 		select {
 		case <-b.interrupt:
 			c.Error = errors.New("build canceled")
diff --git a/pkg/builder/builder_steps.go b/pkg/builder/builder_steps.go
index b5db1ca..79d98d8 100644
--- a/pkg/builder/builder_steps.go
+++ b/pkg/builder/builder_steps.go
@@ -152,55 +152,6 @@ func ComputeDependencies(ctx *Context) error {
 	return nil
 }
 
-// LookupPredefinedImage is used to find a suitable predefined image if available
-func LookupPredefinedImage(ctx *Context) error {
-	if !ctx.Request.Platform.Build.PredefinedImages {
-		// Usage of predefined images not enabled
-		return nil
-	}
-
-	standardDependencies := map[string]bool{
-		"camel:core":   true,
-		"runtime:jvm":  true,
-		"runtime:yaml": true,
-	}
-
-	realDependencies := make(map[string]bool)
-	for _, d := range ctx.Request.Dependencies {
-		if _, std := standardDependencies[d]; !std {
-			realDependencies[d] = true
-		}
-	}
-
-	knativeDep := "camel-k:knative"
-	if len(realDependencies) != 2 || !realDependencies[knativeDep] {
-		return nil
-	}
-
-	var otherDep string
-	for d := range realDependencies {
-		if d != knativeDep {
-			otherDep = d
-			break
-		}
-	}
-
-	camelPrefix := "camel:"
-	if !strings.HasPrefix(otherDep, camelPrefix) {
-		return nil
-	}
-
-	comp := strings.TrimPrefix(otherDep, camelPrefix)
-	ctx.Image = PredefinedImageNameFor(comp)
-	ctx.PublicImage = ctx.Image
-	return nil
-}
-
-// PredefinedImageNameFor --
-func PredefinedImageNameFor(comp string) string {
-	return fmt.Sprintf("camelk/camel-base-knative-%s:%s", comp, version.Version)
-}
-
 // ArtifactsSelector --
 type ArtifactsSelector func(ctx *Context) error
 
diff --git a/pkg/builder/kaniko/kaniko.go b/pkg/builder/kaniko/kaniko.go
index 0d80075..4b2d826 100644
--- a/pkg/builder/kaniko/kaniko.go
+++ b/pkg/builder/kaniko/kaniko.go
@@ -25,7 +25,6 @@ import (
 var DefaultSteps = []builder.Step{
 	builder.NewStep("generate", builder.ProjectGenerationPhase, builder.GenerateProject),
 	builder.NewStep("build/compute-dependencies", builder.ProjectBuildPhase, builder.ComputeDependencies),
-	builder.NewStep("lookup/predefined-image", builder.ProjectBuildPhase + 1, builder.LookupPredefinedImage),
 	builder.NewStep("packager", builder.ApplicationPackagePhase, builder.StandardPackager),
 	builder.NewStep("publisher/kaniko", builder.ApplicationPublishPhase, Publisher),
 	builder.NewStep("notify/context", builder.NotifyPhase, builder.NotifyIntegrationContext),
diff --git a/pkg/builder/s2i/s2i.go b/pkg/builder/s2i/s2i.go
index 9e105a0..8824e5d 100644
--- a/pkg/builder/s2i/s2i.go
+++ b/pkg/builder/s2i/s2i.go
@@ -25,7 +25,6 @@ import (
 var DefaultSteps = []builder.Step{
 	builder.NewStep("generate", builder.ProjectGenerationPhase, builder.GenerateProject),
 	builder.NewStep("build/compute-dependencies", builder.ProjectBuildPhase, builder.ComputeDependencies),
-	builder.NewStep("lookup/predefined-image", builder.ProjectBuildPhase + 1, builder.LookupPredefinedImage),
 	builder.NewStep("packager/incremental", builder.ApplicationPackagePhase, builder.IncrementalPackager),
 	builder.NewStep("publisher/s2i", builder.ApplicationPublishPhase, Publisher),
 	builder.NewStep("notify/context", builder.NotifyPhase, builder.NotifyIntegrationContext),
diff --git a/pkg/controller/integration/build_context.go b/pkg/controller/integration/build_context.go
index 427794c..3bbb4a9 100644
--- a/pkg/controller/integration/build_context.go
+++ b/pkg/controller/integration/build_context.go
@@ -21,16 +21,13 @@ import (
 	"context"
 	"fmt"
 
+	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/platform"
 	"github.com/apache/camel-k/pkg/trait"
-
-	"github.com/sirupsen/logrus"
-
 	"github.com/apache/camel-k/pkg/util"
 	"github.com/apache/camel-k/pkg/util/digest"
-
 	"github.com/rs/xid"
-
-	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/sirupsen/logrus"
 )
 
 // NewBuildContextAction create an action that handles integration context build
@@ -60,6 +57,17 @@ func (action *buildContextAction) Handle(ctx context.Context, integration *v1alp
 		return err
 	}
 
+	if ictx == nil {
+		// Try to create an external context if possible
+		pl, err := platform.GetCurrentPlatform(ctx, action.client, integration.Namespace)
+		if err != nil {
+			return nil
+		}
+		if pl.Spec.Build.PredefinedImages {
+			ictx, err = ImportPredefinedContextIfPresent(ctx, action.client, integration)
+		}
+	}
+
 	if ictx != nil {
 		if ictx.Labels["camel.apache.org/context.type"] == v1alpha1.IntegrationContextTypePlatform {
 			// This is a platform context and as it is auto generated it may get
diff --git a/pkg/controller/integration/util.go b/pkg/controller/integration/util.go
index 9c31ec8..9eecf11 100644
--- a/pkg/controller/integration/util.go
+++ b/pkg/controller/integration/util.go
@@ -19,14 +19,22 @@ package integration
 
 import (
 	"context"
-
-	"github.com/apache/camel-k/pkg/util"
-	k8sclient "sigs.k8s.io/controller-runtime/pkg/client"
+	"fmt"
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
+	"github.com/apache/camel-k/pkg/client"
+	"github.com/apache/camel-k/pkg/platform/images"
+	"github.com/apache/camel-k/pkg/util"
 	"github.com/pkg/errors"
+	"github.com/rs/xid"
+	k8sclient "sigs.k8s.io/controller-runtime/pkg/client"
 )
 
+var allowedLookupLabels = map[string]bool{
+	v1alpha1.IntegrationContextTypePlatform: true,
+	v1alpha1.IntegrationContextTypeExternal: true,
+}
+
 // LookupContextForIntegration --
 func LookupContextForIntegration(ctx context.Context, c k8sclient.Reader, integration *v1alpha1.Integration) (*v1alpha1.IntegrationContext, error) {
 	if integration.Status.Context != "" {
@@ -50,7 +58,7 @@ func LookupContextForIntegration(ctx context.Context, c k8sclient.Reader, integr
 
 	for _, ctx := range ctxList.Items {
 		ctx := ctx // pin
-		if ctx.Labels["camel.apache.org/context.type"] == v1alpha1.IntegrationContextTypePlatform {
+		if allowed, ok := allowedLookupLabels[ctx.Labels["camel.apache.org/context.type"]]; ok && allowed {
 			ideps := len(integration.Status.Dependencies)
 			cdeps := len(ctx.Spec.Dependencies)
 
@@ -66,3 +74,31 @@ func LookupContextForIntegration(ctx context.Context, c k8sclient.Reader, integr
 
 	return nil, nil
 }
+
+// ImportPredefinedContextIfPresent tries to create an external context from a predefined image
+func ImportPredefinedContextIfPresent(ctx context.Context, c client.Client, integration *v1alpha1.Integration) (*v1alpha1.IntegrationContext, error) {
+	image := images.LookupPredefinedImage(integration.Status.Dependencies)
+	if image == "" {
+		return nil, nil
+	}
+
+	externalCtxName := fmt.Sprintf("ctx-base-%s", xid.New())
+	externalCtx := v1alpha1.NewIntegrationContext(integration.Namespace, externalCtxName)
+
+	externalCtx.Labels = map[string]string{
+		"camel.apache.org/context.type":               v1alpha1.IntegrationContextTypeExternal,
+		"camel.apache.org/context.created.by.kind":    v1alpha1.IntegrationKind,
+		"camel.apache.org/context.created.by.name":    integration.Name,
+		"camel.apache.org/context.created.by.version": integration.ResourceVersion,
+	}
+
+	externalCtx.Spec = v1alpha1.IntegrationContextSpec{
+		Dependencies: integration.Status.Dependencies,
+		Image:        image,
+	}
+
+	if err := c.Create(ctx, &externalCtx); err != nil {
+		return nil, err
+	}
+	return &externalCtx, nil
+}
diff --git a/pkg/builder/s2i/s2i.go b/pkg/platform/images/doc.go
similarity index 52%
copy from pkg/builder/s2i/s2i.go
copy to pkg/platform/images/doc.go
index 9e105a0..cdf53c0 100644
--- a/pkg/builder/s2i/s2i.go
+++ b/pkg/platform/images/doc.go
@@ -15,18 +15,5 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package s2i
-
-import (
-	"github.com/apache/camel-k/pkg/builder"
-)
-
-// DefaultSteps --
-var DefaultSteps = []builder.Step{
-	builder.NewStep("generate", builder.ProjectGenerationPhase, builder.GenerateProject),
-	builder.NewStep("build/compute-dependencies", builder.ProjectBuildPhase, builder.ComputeDependencies),
-	builder.NewStep("lookup/predefined-image", builder.ProjectBuildPhase + 1, builder.LookupPredefinedImage),
-	builder.NewStep("packager/incremental", builder.ApplicationPackagePhase, builder.IncrementalPackager),
-	builder.NewStep("publisher/s2i", builder.ApplicationPublishPhase, Publisher),
-	builder.NewStep("notify/context", builder.NotifyPhase, builder.NotifyIntegrationContext),
-}
+// Package images contains information for retrieval of platform predefined images
+package images
diff --git a/pkg/platform/images/images.go b/pkg/platform/images/images.go
new file mode 100644
index 0000000..ab98a21
--- /dev/null
+++ b/pkg/platform/images/images.go
@@ -0,0 +1,82 @@
+/*
+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 images
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/apache/camel-k/version"
+)
+
+// BaseRepository is the docker repository that contains images
+const (
+	BaseRepository = "camelk"
+	ImagePrefix    = "camel-base-knative-"
+)
+
+// BaseDependency is a required dependency that must be found in the list
+var BaseDependency = "camel-k:knative"
+
+// StandardDependencies are common dependencies included in the image
+var StandardDependencies = map[string]bool{
+	"camel:core":   true,
+	"runtime:jvm":  true,
+	"runtime:yaml": true,
+}
+
+// LookupPredefinedImage is used to find a suitable predefined image if available
+func LookupPredefinedImage(dependencies []string) string {
+
+	realDependencies := make([]string, 0)
+	baseDependencyFound := false
+	for _, d := range dependencies {
+		if _, std := StandardDependencies[d]; std {
+			continue
+		}
+		if d == BaseDependency {
+			baseDependencyFound = true
+			continue
+		}
+		realDependencies = append(realDependencies, d)
+	}
+
+	if !baseDependencyFound {
+		return ""
+	}
+	if len(realDependencies) == 0 {
+		return PredefinedImageNameFor("core")
+	}
+	if len(realDependencies) != 1 {
+		return ""
+	}
+
+	otherDep := realDependencies[0]
+	camelPrefix := "camel:"
+	if !strings.HasPrefix(otherDep, camelPrefix) {
+		return ""
+	}
+
+	comp := strings.TrimPrefix(otherDep, camelPrefix)
+	return PredefinedImageNameFor(comp)
+}
+
+// PredefinedImageNameFor --
+func PredefinedImageNameFor(comp string) string {
+	return fmt.Sprintf("%s/%s%s:%s", BaseRepository, ImagePrefix, comp, version.Version)
+}
diff --git a/pkg/platform/images/images_test.go b/pkg/platform/images/images_test.go
new file mode 100644
index 0000000..53441e1
--- /dev/null
+++ b/pkg/platform/images/images_test.go
@@ -0,0 +1,68 @@
+/*
+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 images
+
+import (
+	"github.com/apache/camel-k/version"
+	"github.com/stretchr/testify/assert"
+	"strconv"
+	"testing"
+)
+
+func TestImageLookup(t *testing.T) {
+	cases := []struct {
+		dependencies []string
+		image        string
+	}{
+		{
+			dependencies: []string{"camel:telegram"},
+		},
+		{
+			dependencies: []string{"camel:telegram", "camel:core"},
+		},
+		{
+			dependencies: []string{"camel:telegram", "camel:core", "camel-k:knative"},
+			image:        BaseRepository + "/" + ImagePrefix + "telegram:" + version.Version,
+		},
+		{
+			dependencies: []string{"camel:core", "camel-k:knative"},
+			image:        BaseRepository + "/" + ImagePrefix + "core:" + version.Version,
+		},
+		{
+			dependencies: []string{"camel:dropbox", "camel:core", "camel-k:knative", "runtime:jvm"},
+			image:        BaseRepository + "/" + ImagePrefix + "dropbox:" + version.Version,
+		},
+		{
+			dependencies: []string{"camel:dropbox", "camel:core", "camel-k:knative", "runtime:jvm", "runtime:yaml"},
+			image:        BaseRepository + "/" + ImagePrefix + "dropbox:" + version.Version,
+		},
+		{
+			dependencies: []string{"camel:dropbox", "camel:core", "runtime:jvm", "runtime:yaml"},
+		},
+		{
+			dependencies: []string{"camel:dropbox", "camel:core", "camel-k:knative", "runtime:jvm", "runtime:groovy"},
+		},
+	}
+
+	for i, tc := range cases {
+		t.Run("case-"+strconv.Itoa(i), func(t *testing.T) {
+			assert.Equal(t, tc.image, LookupPredefinedImage(tc.dependencies))
+		})
+	}
+
+}
diff --git a/pkg/trait/builder_test.go b/pkg/trait/builder_test.go
index f67fad0..1afefab 100644
--- a/pkg/trait/builder_test.go
+++ b/pkg/trait/builder_test.go
@@ -82,7 +82,7 @@ func TestS2IBuilderTrait(t *testing.T) {
 	assert.NotEmpty(t, env.ExecutedTraits)
 	assert.NotNil(t, env.GetTrait(ID("builder")))
 	assert.NotEmpty(t, env.Steps)
-	assert.Len(t, env.Steps, 6)
+	assert.Len(t, env.Steps, 5)
 	assert.Condition(t, func() bool {
 		for _, s := range env.Steps {
 			if s.ID() == "publisher/s2i" && s.Phase() == builder.ApplicationPublishPhase {
@@ -102,7 +102,7 @@ func TestKanikoBuilderTrait(t *testing.T) {
 	assert.NotEmpty(t, env.ExecutedTraits)
 	assert.NotNil(t, env.GetTrait(ID("builder")))
 	assert.NotEmpty(t, env.Steps)
-	assert.Len(t, env.Steps, 6)
+	assert.Len(t, env.Steps, 5)
 	assert.Condition(t, func() bool {
 		for _, s := range env.Steps {
 			if s.ID() == "publisher/kaniko" && s.Phase() == builder.ApplicationPublishPhase {