You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by "gansheer (via GitHub)" <gi...@apache.org> on 2023/08/21 16:22:46 UTC

[GitHub] [camel-k] gansheer opened a new pull request, #4680: feat(#1656) : Add Jib publish strategy

gansheer opened a new pull request, #4680:
URL: https://github.com/apache/camel-k/pull/4680

   * Add publish jib strategy compatible with incremental build and native build
   * Use google jib maven plugin
   * Manage container layers in generated image
   
   <!-- Description -->
   
   
   
   
   <!--
   Enter your extended release note in the below block. If the PR requires
   additional action from users switching to the new release, include the string
   "action required". If no release note is required, write "NONE". 
   
   You can (optionally) mark this PR with labels "kind/bug" or "kind/feature" to make sure
   the text is added to the right section of the release notes. 
   -->
   
   **Release Note**
   ```release-note
   feat(#1656) : Add Jib publish strategy
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1302641831


##########
pkg/util/maven/maven_command.go:
##########
@@ -142,7 +142,12 @@ func (c *Command) Do(ctx context.Context) error {
 
 	Log.WithValues("MAVEN_OPTS", mavenOptions).Infof("executing: %s", strings.Join(cmd.Args, " "))
 
-	return util.RunAndLog(ctx, cmd, mavenLogHandler, mavenLogHandler)
+	// generate maven file
+	if err := generateMavenContext(c.context.Path, args); err != nil {

Review Comment:
   It would be better, but I would like to dedicate another issue to this part as it needs a big refactoring in a sensitive part of the build task. It would also be an occasion to see if we can separate the maven code dedicated to commands execution from the project generation code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1301071913


##########
pkg/util/jib/configuration.go:
##########
@@ -0,0 +1,113 @@
+/*
+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 jib contains utilities for jib strategy builds.
+package jib
+
+import (
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+const JibMavenGoal = "jib:build"

Review Comment:
   The const are used in the jib task code `pkg/builder/jib.go`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1693508396

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] squakez commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "squakez (via GitHub)" <gi...@apache.org>.
squakez commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1302542435


##########
pkg/builder/quarkus.go:
##########
@@ -206,6 +207,7 @@ func GenerateQuarkusProjectCommon(runtimeVersion string, quarkusVersion string,
 				},
 			},
 		},
+		jib.JibMavenPlugin(),

Review Comment:
   We should do this one only when the publish strategy is JIB. However, in general, I think we should leverage the builder profile configuration to have a clean logic separation (ie, the quarkus trait should know nothing about the publishing strategy).



##########
pkg/builder/jib.go:
##########
@@ -0,0 +1,135 @@
+/*
+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 (
+	"context"
+	"os"
+	"os/exec"
+	"path/filepath"
+	"strings"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/client"
+	"github.com/apache/camel-k/v2/pkg/util"
+	"github.com/apache/camel-k/v2/pkg/util/jib"
+	"github.com/apache/camel-k/v2/pkg/util/log"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+type jibTask struct {
+	c     client.Client
+	build *v1.Build
+	task  *v1.JibTask
+}
+
+var _ Task = &jibTask{}
+
+func (t *jibTask) Do(ctx context.Context) v1.BuildStatus {
+	status := v1.BuildStatus{}
+
+	baseImage := t.build.Status.BaseImage
+	if baseImage == "" {
+		baseImage = t.task.BaseImage
+		status.BaseImage = baseImage
+	}
+
+	contextDir := t.task.ContextDir
+	if contextDir == "" {
+		// Use the working directory.
+		// This is useful when the task is executed in-container,
+		// so that its WorkingDir can be used to share state and
+		// coordinate with other tasks.
+		pwd, err := os.Getwd()
+		if err != nil {
+			return status.Failed(err)
+		}
+		contextDir = filepath.Join(pwd, ContextDir)
+	}
+
+	exists, err := util.DirectoryExists(contextDir)
+	if err != nil {
+		return status.Failed(err)
+	}
+	empty, err := util.DirectoryEmpty(contextDir)
+	if err != nil {
+		return status.Failed(err)
+	}
+	if !exists || empty {
+		// this can only indicate that there are no more resources to add to the base image,
+		// because transitive resolution is the same even if spec differs.
+		log.Infof("No new image to build, reusing existing image %s", baseImage)
+		status.Image = baseImage
+		return status
+	}
+	mavenDir := strings.ReplaceAll(contextDir, ContextDir, "maven")
+
+	log.Debugf("Registry address: %s", t.task.Registry.Address)
+	log.Debugf("Base image: %s", baseImage)
+
+	registryConfigDir := ""
+	if t.task.Registry.Secret != "" {
+		registryConfigDir, err = MountSecret(ctx, t.c, t.build.Namespace, t.task.Registry.Secret)
+		if err != nil {
+			return status.Failed(err)
+		}
+	}
+
+	if registryConfigDir != "" {
+		if err := os.RemoveAll(registryConfigDir); err != nil {
+			return status.Failed(err)
+		}
+	}
+
+	mavenCommand, err := util.ReadFile(filepath.Join(mavenDir, "MAVEN_CONTEXT"))
+	if err != nil {
+		return status.Failed(err)
+	}
+
+	mavenArgs := make([]string, 0)
+	mavenArgs = append(mavenArgs, jib.JibMavenGoal)
+	mavenArgs = append(mavenArgs, strings.Split(string(mavenCommand), " ")...)
+	mavenArgs = append(mavenArgs, jib.JibMavenToImageParam+t.task.Image)
+	mavenArgs = append(mavenArgs, jib.JibMavenFromImageParam+baseImage)
+	if t.task.Registry.Insecure {
+		mavenArgs = append(mavenArgs, jib.JibMavenInsecureRegistries+"true")
+	}
+
+	mvnCmd := "./mvnw"
+	if c, ok := os.LookupEnv("MAVEN_CMD"); ok {
+		mvnCmd = c
+	}
+	cmd := exec.CommandContext(ctx, mvnCmd, mavenArgs...)
+	cmd.Dir = mavenDir
+
+	myerror := util.RunAndLog(ctx, cmd, maven.MavenLogHandler, maven.MavenLogHandler)
+	if myerror != nil {
+		log.Errorf(myerror, "jib integration image containerization did not run successfully")
+		return status.Failed(myerror)
+	} else {
+		log.Info("jib integration image containerization did run successfully")

Review Comment:
   Better debug instead of info in this case.



##########
pkg/util/maven/maven_command.go:
##########
@@ -142,7 +142,12 @@ func (c *Command) Do(ctx context.Context) error {
 
 	Log.WithValues("MAVEN_OPTS", mavenOptions).Infof("executing: %s", strings.Join(cmd.Args, " "))
 
-	return util.RunAndLog(ctx, cmd, mavenLogHandler, mavenLogHandler)
+	// generate maven file
+	if err := generateMavenContext(c.context.Path, args); err != nil {

Review Comment:
   I am wondering if, instead of copying the execution config in a file, you could try to regenerate the same at the moment of calling the JIB (ie, creating a func which take care to generate and call it here and later when needed). It would be much cleaner IMO.



##########
pkg/builder/quarkus.go:
##########
@@ -206,6 +207,7 @@ func GenerateQuarkusProjectCommon(runtimeVersion string, quarkusVersion string,
 				},
 			},
 		},
+		jib.JibMavenPlugin(),

Review Comment:
   With the proposed approach, you should be able to create a configmap on the fly containing the required configuration for Jib and set to the build accordingly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1690132838

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.8% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] squakez commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "squakez (via GitHub)" <gi...@apache.org>.
squakez commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1301121125


##########
pkg/util/jib/configuration.go:
##########
@@ -0,0 +1,113 @@
+/*
+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 jib contains utilities for jib strategy builds.
+package jib
+
+import (
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+const JibMavenGoal = "jib:build"
+const JibMavenToImageParam = "-Djib.to.image="
+const JibMavenFromImageParam = "-Djib.from.image="
+const JibMavenInsecureRegistries = "-Djib.allowInsecureRegistries="
+const JibDigestFile = "target/jib-image.digest"
+
+// JibMavenPlugin generates the Jib Maven Plugin configuration with multiple layers or only one.
+func JibMavenPlugin(layers bool) maven.Plugin {
+
+	jibPlugin := maven.Plugin{
+		GroupID:    "com.google.cloud.tools",
+		ArtifactID: "jib-maven-plugin",
+		Version:    "3.3.2",

Review Comment:
   It does not matter, we can always add that, let's keep for a later dev.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] claudio4j commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "claudio4j (via GitHub)" <gi...@apache.org>.
claudio4j commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1305035712


##########
pkg/builder/project.go:
##########
@@ -195,13 +195,19 @@ func sanitizeDependencies(ctx *builderContext) error {
 	return camel.SanitizeIntegrationDependencies(ctx.Maven.Project.Dependencies)
 }
 
-func injectProfile(ctx *builderContext) error {
-	val, err := kubernetes.ResolveValueSource(ctx.C, ctx.Client, ctx.Namespace, &ctx.Build.Maven.Profile)
-	if err != nil {
-		return err
-	}
-	if val != "" {
-		ctx.Maven.Project.AddProfile(val)
+func injectProfiles(ctx *builderContext) error {
+	if ctx.Build.Maven.Profiles != nil {
+		profiles := ""
+		for i := range ctx.Build.Maven.Profiles {
+			val, err := kubernetes.ResolveValueSource(ctx.C, ctx.Client, ctx.Namespace, &ctx.Build.Maven.Profiles[i])
+			if err != nil {
+				return err

Review Comment:
   WDYT to return or print in the log a more descriptive error message about not able to load the configmap profile ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] squakez commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "squakez (via GitHub)" <gi...@apache.org>.
squakez commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1301052564


##########
pkg/util/jib/configuration.go:
##########
@@ -0,0 +1,113 @@
+/*
+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 jib contains utilities for jib strategy builds.
+package jib
+
+import (
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+const JibMavenGoal = "jib:build"

Review Comment:
   Are these consts used somewhere?



##########
pkg/builder/quarkus.go:
##########
@@ -206,6 +211,7 @@ func GenerateQuarkusProjectCommon(runtimeVersion string, quarkusVersion string,
 				},
 			},
 		},
+		jib.JibMavenPlugin(layers),

Review Comment:
   Not sure to understand this. It's returning a plugin object, but it seems nobody is using it.



##########
pkg/util/jib/configuration.go:
##########
@@ -0,0 +1,113 @@
+/*
+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 jib contains utilities for jib strategy builds.
+package jib
+
+import (
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+const JibMavenGoal = "jib:build"
+const JibMavenToImageParam = "-Djib.to.image="
+const JibMavenFromImageParam = "-Djib.from.image="
+const JibMavenInsecureRegistries = "-Djib.allowInsecureRegistries="
+const JibDigestFile = "target/jib-image.digest"
+
+// JibMavenPlugin generates the Jib Maven Plugin configuration with multiple layers or only one.
+func JibMavenPlugin(layers bool) maven.Plugin {
+
+	jibPlugin := maven.Plugin{
+		GroupID:    "com.google.cloud.tools",
+		ArtifactID: "jib-maven-plugin",
+		Version:    "3.3.2",

Review Comment:
   We should have this variable set in the Camel K Runtime BOM instead but I guess can be considered after the draft phase.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1687586828

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.2% --> 39.0% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1691877364

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1686670994

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.2% --> 39.0% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1692900969

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1688490422

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.8% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1691873932

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] squakez merged pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "squakez (via GitHub)" <gi...@apache.org>.
squakez merged PR #4680:
URL: https://github.com/apache/camel-k/pull/4680


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1693487085

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1686651262

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.2% --> 39.0% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1301073014


##########
pkg/builder/quarkus.go:
##########
@@ -206,6 +211,7 @@ func GenerateQuarkusProjectCommon(runtimeVersion string, quarkusVersion string,
 				},
 			},
 		},
+		jib.JibMavenPlugin(layers),

Review Comment:
   The plugin object is added to be used in the jib task code `pkg/builder/jib.go` at publish time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1302641831


##########
pkg/util/maven/maven_command.go:
##########
@@ -142,7 +142,12 @@ func (c *Command) Do(ctx context.Context) error {
 
 	Log.WithValues("MAVEN_OPTS", mavenOptions).Infof("executing: %s", strings.Join(cmd.Args, " "))
 
-	return util.RunAndLog(ctx, cmd, mavenLogHandler, mavenLogHandler)
+	// generate maven file
+	if err := generateMavenContext(c.context.Path, args); err != nil {

Review Comment:
   It would be better, but I would like to dedicate another issue to this part as it needs a big refactoring in a sensitive part of the build task. It would be also an occasion to see if we can separate the maven code dedicate to commands execution from the project generation code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1691892440

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.0% --> 38.9% (Coverage difference: **-.1%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1691886081

   I adapted to use a maven profile:
   
   * change the builder profile trait to accept multiple values
   * in case of JIB publish strategy, a configmap containing the default jib profile is created linked to the integration kit
   
   What still needs to be done in this Draft: tests
   
   What will be done in later PRs:
   * extract plugin version in camel-k-runtime
   * refactor the way we interact with maven to be able to reuse the code to generate a complete maven command in the publish task instead of using a temp file "MAVEN_CONTEXT"
   
   @squakez 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] github-actions[bot] commented on pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#issuecomment-1687573142

   :camel: **Thank you for contributing!**
   
   Code Coverage Report :warning: - Coverage changed: 39.2% --> 39.0% (Coverage difference: **-.2%**)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel-k] gansheer commented on a diff in pull request #4680: feat(#1656) : Add Jib publish strategy

Posted by "gansheer (via GitHub)" <gi...@apache.org>.
gansheer commented on code in PR #4680:
URL: https://github.com/apache/camel-k/pull/4680#discussion_r1301074869


##########
pkg/util/jib/configuration.go:
##########
@@ -0,0 +1,113 @@
+/*
+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 jib contains utilities for jib strategy builds.
+package jib
+
+import (
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+)
+
+const JibMavenGoal = "jib:build"
+const JibMavenToImageParam = "-Djib.to.image="
+const JibMavenFromImageParam = "-Djib.from.image="
+const JibMavenInsecureRegistries = "-Djib.allowInsecureRegistries="
+const JibDigestFile = "target/jib-image.digest"
+
+// JibMavenPlugin generates the Jib Maven Plugin configuration with multiple layers or only one.
+func JibMavenPlugin(layers bool) maven.Plugin {
+
+	jibPlugin := maven.Plugin{
+		GroupID:    "com.google.cloud.tools",
+		ArtifactID: "jib-maven-plugin",
+		Version:    "3.3.2",

Review Comment:
   Good idea, I would need to see how to make that work. Is it a good idea to do that now as we are moving the runtime code to a possible camel-quarkus extension ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org