You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by cd...@apache.org on 2024/01/24 06:10:21 UTC

(camel-k) branch main updated: Create CamelCatalog on IntegrationPlatform controller

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 4264d13b3 Create CamelCatalog on IntegrationPlatform controller
4264d13b3 is described below

commit 4264d13b39ff6fcbf1d4f6b9c41047a7cf73d32b
Author: Christoph Deppisch <cd...@redhat.com>
AuthorDate: Mon Jan 22 15:18:59 2024 +0100

    Create CamelCatalog on IntegrationPlatform controller
    
    - Create CamelCatalog for new runtimeVersion that has been set on IntegrationPlatform
    - Improve CamelCatalog availability checks on IntegrationPlatform controller
    - Adds new condition on IntegrationPlatform representing the availability of the catalog
    - Adds new controller action and IntegrationPlatform phase "CreateCatalog" that handles the creation of the catalog
    - Sets IntegrationPlatform to error phase when catalog is not available
---
 pkg/apis/camel/v1/integrationplatform_types.go     |   9 +
 pkg/controller/integrationplatform/catalog.go      |  82 ++++++++
 pkg/controller/integrationplatform/catalog_test.go | 222 +++++++++++++++++++++
 .../integrationplatform_controller.go              |   1 +
 pkg/controller/integrationplatform/monitor.go      |  46 ++++-
 pkg/controller/integrationplatform/monitor_test.go | 191 ++++++++++++++++++
 pkg/platform/defaults.go                           |   2 +-
 pkg/trait/camel.go                                 |  54 +----
 pkg/trait/registry.go                              |   3 +-
 pkg/util/camel/camel_runtime.go                    |  62 ++++++
 pkg/util/defaults/defaults.go                      |   3 +
 pkg/util/maven/maven_command.go                    |  10 +-
 pkg/util/test/client.go                            |   7 +-
 pkg/util/util_test.go                              |   3 +-
 script/Makefile                                    |  10 +-
 15 files changed, 635 insertions(+), 70 deletions(-)

diff --git a/pkg/apis/camel/v1/integrationplatform_types.go b/pkg/apis/camel/v1/integrationplatform_types.go
index 62470188c..6e480d3fa 100644
--- a/pkg/apis/camel/v1/integrationplatform_types.go
+++ b/pkg/apis/camel/v1/integrationplatform_types.go
@@ -193,6 +193,8 @@ const (
 	IntegrationPlatformPhaseReady IntegrationPlatformPhase = "Ready"
 	// IntegrationPlatformPhaseError when the IntegrationPlatform had some error (see Conditions).
 	IntegrationPlatformPhaseError IntegrationPlatformPhase = "Error"
+	// IntegrationPlatformPhaseCreateCatalog when the IntegrationPlatform creates a new CamelCatalog.
+	IntegrationPlatformPhaseCreateCatalog IntegrationPlatformPhase = "CreateCatalog"
 	// IntegrationPlatformPhaseDuplicate when the IntegrationPlatform is duplicated.
 	IntegrationPlatformPhaseDuplicate IntegrationPlatformPhase = "Duplicate"
 
@@ -205,8 +207,15 @@ const (
 	// IntegrationPlatformConditionTypeRegistryAvailable is the condition for the availability of a container registry.
 	IntegrationPlatformConditionTypeRegistryAvailable IntegrationPlatformConditionType = "RegistryAvailable"
 
+	// IntegrationPlatformConditionCamelCatalogAvailable is the condition for the availability of a container registry.
+	IntegrationPlatformConditionCamelCatalogAvailable IntegrationPlatformConditionType = "CamelCatalogAvailable"
+
 	// IntegrationPlatformConditionCreatedReason represents the reason that the IntegrationPlatform is created.
 	IntegrationPlatformConditionCreatedReason = "IntegrationPlatformCreated"
+	// IntegrationPlatformConditionTypeRegistryAvailableReason represents the reason that the IntegrationPlatform Registry is available.
+	IntegrationPlatformConditionTypeRegistryAvailableReason = "IntegrationPlatformRegistryAvailable"
+	// IntegrationPlatformConditionCamelCatalogAvailableReason represents the reason that the IntegrationPlatform is created.
+	IntegrationPlatformConditionCamelCatalogAvailableReason = "IntegrationPlatformCamelCatalogAvailable"
 )
 
 // IntegrationPlatformCondition describes the state of a resource at a certain point.
diff --git a/pkg/controller/integrationplatform/catalog.go b/pkg/controller/integrationplatform/catalog.go
new file mode 100644
index 000000000..91ab24c68
--- /dev/null
+++ b/pkg/controller/integrationplatform/catalog.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 integrationplatform
+
+import (
+	"context"
+	"fmt"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/util/camel"
+
+	corev1 "k8s.io/api/core/v1"
+)
+
+// NewCreateCatalogAction returns an action to create a new CamelCatalog.
+func NewCreateCatalogAction() Action {
+	return &catalogAction{}
+}
+
+type catalogAction struct {
+	baseAction
+}
+
+func (action *catalogAction) Name() string {
+	return "catalog"
+}
+
+func (action *catalogAction) CanHandle(platform *v1.IntegrationPlatform) bool {
+	return platform.Status.Phase == v1.IntegrationPlatformPhaseCreateCatalog
+}
+
+func (action *catalogAction) Handle(ctx context.Context, platform *v1.IntegrationPlatform) (*v1.IntegrationPlatform, error) {
+	// New runtime version set - check that catalog exists and create it if it does not exist
+	runtimeSpec := v1.RuntimeSpec{
+		Version:  platform.Status.Build.RuntimeVersion,
+		Provider: v1.RuntimeProviderQuarkus,
+	}
+
+	if catalog, err := camel.LoadCatalog(ctx, action.client, platform.Namespace, runtimeSpec); err != nil {
+		action.L.Error(err, "IntegrationPlatform unable to load Camel catalog",
+			"runtime-version", runtimeSpec.Version, "runtime-provider", runtimeSpec.Provider)
+		return platform, nil
+	} else if catalog == nil {
+		if _, err = camel.CreateCatalog(ctx, action.client, platform.Namespace, platform, runtimeSpec); err != nil {
+			action.L.Error(err, "IntegrationPlatform unable to create Camel catalog",
+				"runtime-version", runtimeSpec.Version, "runtime-provider", runtimeSpec.Provider)
+
+			platform.Status.Phase = v1.IntegrationPlatformPhaseError
+			platform.Status.SetCondition(
+				v1.IntegrationPlatformConditionCamelCatalogAvailable,
+				corev1.ConditionFalse,
+				v1.IntegrationPlatformConditionCamelCatalogAvailableReason,
+				fmt.Sprintf("camel catalog %s not available, please review given runtime version", runtimeSpec.Version))
+
+			return platform, nil
+		}
+	}
+
+	platform.Status.Phase = v1.IntegrationPlatformPhaseReady
+	platform.Status.SetCondition(
+		v1.IntegrationPlatformConditionCamelCatalogAvailable,
+		corev1.ConditionTrue,
+		v1.IntegrationPlatformConditionCamelCatalogAvailableReason,
+		fmt.Sprintf("camel catalog %s available", runtimeSpec.Version))
+
+	return platform, nil
+}
diff --git a/pkg/controller/integrationplatform/catalog_test.go b/pkg/controller/integrationplatform/catalog_test.go
new file mode 100644
index 000000000..e005f0e89
--- /dev/null
+++ b/pkg/controller/integrationplatform/catalog_test.go
@@ -0,0 +1,222 @@
+/*
+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 integrationplatform
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"os"
+	"strings"
+	"testing"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/platform"
+	"github.com/apache/camel-k/v2/pkg/resources"
+	"github.com/apache/camel-k/v2/pkg/util/defaults"
+	"github.com/apache/camel-k/v2/pkg/util/log"
+	"github.com/apache/camel-k/v2/pkg/util/test"
+	"github.com/rs/xid"
+	"github.com/stretchr/testify/assert"
+
+	corev1 "k8s.io/api/core/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	k8stesting "k8s.io/client-go/testing"
+	k8sclient "sigs.k8s.io/controller-runtime/pkg/client"
+)
+
+func TestCanHandlePhaseCreateCatalog(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Spec.Build.RuntimeVersion = defaults.DefaultRuntimeVersion
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseCreateCatalog
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	action := NewCreateCatalogAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer := action.CanHandle(&ip)
+	assert.True(t, answer)
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseError
+	answer = action.CanHandle(&ip)
+	assert.False(t, answer)
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseReady
+	answer = action.CanHandle(&ip)
+	assert.False(t, answer)
+}
+
+func TestCreateCatalog(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseCreateCatalog
+	ip.Spec.Build.RuntimeVersion = defaults.DefaultRuntimeVersion
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	// use local Maven executable in tests
+	t.Setenv("MAVEN_WRAPPER", "false")
+	_, ok := os.LookupEnv("MAVEN_CMD")
+	if !ok {
+		t.Setenv("MAVEN_CMD", "mvn")
+	}
+
+	fakeClient := c.(*test.FakeClient) //nolint
+	fakeClient.AddReactor("create", "*", func(action k8stesting.Action) (bool, runtime.Object, error) {
+		createAction := action.(k8stesting.CreateAction) //nolint
+
+		assert.Equal(t, "ns", createAction.GetNamespace())
+
+		return true, createAction.GetObject(), nil
+	})
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewCreateCatalogAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseReady, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+
+	list := v1.NewCamelCatalogList()
+	err = c.List(context.TODO(), &list, k8sclient.InNamespace(ip.Namespace))
+
+	assert.Nil(t, err)
+	assert.NotEmpty(t, list.Items)
+
+	items, err := resources.WithPrefix("/camel-catelog-")
+	assert.Nil(t, err)
+
+	for _, k := range items {
+		found := false
+
+		for _, c := range list.Items {
+			n := strings.TrimSuffix(k, ".yaml")
+			n = strings.ToLower(n)
+
+			if c.Name == n {
+				found = true
+			}
+		}
+
+		assert.True(t, found)
+	}
+}
+
+func TestCatalogAlreadyPresent(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseCreateCatalog
+
+	catalog := v1.NewCamelCatalog("ns", fmt.Sprintf("camel-catalog-%s", defaults.DefaultRuntimeVersion))
+	catalog.Spec.Runtime.Version = defaults.DefaultRuntimeVersion
+	catalog.Spec.Runtime.Provider = v1.RuntimeProviderQuarkus
+
+	c, err := test.NewFakeClient(&ip, &catalog)
+	assert.Nil(t, err)
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseReady, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+}
+
+func TestCreateCatalogError(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseCreateCatalog
+
+	// force catalog build to fail
+	ip.Spec.Build.RuntimeVersion = "0.0.0"
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	// use local Maven executable in tests
+	t.Setenv("MAVEN_WRAPPER", "false")
+	_, ok := os.LookupEnv("MAVEN_CMD")
+	if !ok {
+		t.Setenv("MAVEN_CMD", "mvn")
+	}
+
+	fakeClient := c.(*test.FakeClient) //nolint
+	fakeClient.AddReactor("create", "*", func(action k8stesting.Action) (bool, runtime.Object, error) {
+		createAction := action.(k8stesting.CreateAction) //nolint
+
+		assert.Equal(t, "ns", createAction.GetNamespace())
+
+		return true, nil, errors.New("failed to create catalog for some reason")
+	})
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewCreateCatalogAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseError, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionFalse, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+	assert.Equal(t, v1.IntegrationPlatformConditionCamelCatalogAvailableReason, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Reason)
+	assert.Equal(t, "camel catalog 0.0.0 not available, please review given runtime version", answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Message)
+}
diff --git a/pkg/controller/integrationplatform/integrationplatform_controller.go b/pkg/controller/integrationplatform/integrationplatform_controller.go
index 2c714af8d..3ae36d8d6 100644
--- a/pkg/controller/integrationplatform/integrationplatform_controller.go
+++ b/pkg/controller/integrationplatform/integrationplatform_controller.go
@@ -157,6 +157,7 @@ func (r *reconcileIntegrationPlatform) Reconcile(ctx context.Context, request re
 		NewInitializeAction(),
 		NewCreateAction(),
 		NewMonitorAction(),
+		NewCreateCatalogAction(),
 	}
 
 	var targetPhase v1.IntegrationPlatformPhase
diff --git a/pkg/controller/integrationplatform/monitor.go b/pkg/controller/integrationplatform/monitor.go
index cc849c162..a231cbf2e 100644
--- a/pkg/controller/integrationplatform/monitor.go
+++ b/pkg/controller/integrationplatform/monitor.go
@@ -23,6 +23,7 @@ import (
 
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	platformutil "github.com/apache/camel-k/v2/pkg/platform"
+	"github.com/apache/camel-k/v2/pkg/util/camel"
 	"github.com/apache/camel-k/v2/pkg/util/defaults"
 	"github.com/apache/camel-k/v2/pkg/util/openshift"
 	corev1 "k8s.io/api/core/v1"
@@ -52,6 +53,8 @@ func (action *monitorAction) Handle(ctx context.Context, platform *v1.Integratio
 		action.L.Info("IntegrationPlatform version updated", "version", platform.Status.Version)
 	}
 
+	platformPhase := v1.IntegrationPlatformPhaseReady
+
 	// Refresh applied configuration
 	if err := platformutil.ConfigureDefaults(ctx, action.client, platform, false); err != nil {
 		return nil, err
@@ -66,26 +69,59 @@ func (action *monitorAction) Handle(ctx context.Context, platform *v1.Integratio
 		platform.Status.SetCondition(
 			v1.IntegrationPlatformConditionTypeRegistryAvailable,
 			corev1.ConditionFalse,
-			"IntegrationPlatformRegistryAvailable",
+			v1.IntegrationPlatformConditionTypeRegistryAvailableReason,
 			"registry not available because provided by Openshift")
 	} else {
 		if platform.Status.Build.Registry.Address == "" {
 			// error, we need a registry if we're not on Openshift
-			platform.Status.Phase = v1.IntegrationPlatformPhaseError
+			platformPhase = v1.IntegrationPlatformPhaseError
 			platform.Status.SetCondition(
 				v1.IntegrationPlatformConditionTypeRegistryAvailable,
 				corev1.ConditionFalse,
-				"IntegrationPlatformRegistryAvailable",
+				v1.IntegrationPlatformConditionTypeRegistryAvailableReason,
 				"registry address not available, you need to set one")
 		} else {
-			platform.Status.Phase = v1.IntegrationPlatformPhaseReady
 			platform.Status.SetCondition(
 				v1.IntegrationPlatformConditionTypeRegistryAvailable,
 				corev1.ConditionTrue,
-				"IntegrationPlatformRegistryAvailable",
+				v1.IntegrationPlatformConditionTypeRegistryAvailableReason,
 				fmt.Sprintf("registry available at %s", platform.Status.Build.Registry.Address))
 		}
 	}
 
+	if platformPhase == v1.IntegrationPlatformPhaseReady {
+		// Camel catalog condition
+		runtimeSpec := v1.RuntimeSpec{
+			Version:  platform.Status.Build.RuntimeVersion,
+			Provider: v1.RuntimeProviderQuarkus,
+		}
+		if catalog, err := camel.LoadCatalog(ctx, action.client, platform.Namespace, runtimeSpec); err != nil {
+			action.L.Error(err, "IntegrationPlatform unable to load Camel catalog",
+				"runtime-version", runtimeSpec.Version, "runtime-provider", runtimeSpec.Provider)
+		} else if catalog == nil {
+			if platform.Status.Phase != v1.IntegrationPlatformPhaseError {
+				platformPhase = v1.IntegrationPlatformPhaseCreateCatalog
+			} else {
+				// IntegrationPlatform is in error phase for some reason - that error state must be resolved before we move into create catalog phase
+				// avoids to run into endless loop of error and catalog creation phase ping pong
+				platformPhase = v1.IntegrationPlatformPhaseError
+			}
+
+			platform.Status.SetCondition(
+				v1.IntegrationPlatformConditionCamelCatalogAvailable,
+				corev1.ConditionFalse,
+				v1.IntegrationPlatformConditionCamelCatalogAvailableReason,
+				fmt.Sprintf("camel catalog %s not available, please review given runtime version", runtimeSpec.Version))
+		} else {
+			platform.Status.SetCondition(
+				v1.IntegrationPlatformConditionCamelCatalogAvailable,
+				corev1.ConditionTrue,
+				v1.IntegrationPlatformConditionCamelCatalogAvailableReason,
+				fmt.Sprintf("camel catalog %s available", runtimeSpec.Version))
+		}
+	}
+
+	platform.Status.Phase = platformPhase
+
 	return platform, nil
 }
diff --git a/pkg/controller/integrationplatform/monitor_test.go b/pkg/controller/integrationplatform/monitor_test.go
new file mode 100644
index 000000000..ee22a9496
--- /dev/null
+++ b/pkg/controller/integrationplatform/monitor_test.go
@@ -0,0 +1,191 @@
+/*
+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 integrationplatform
+
+import (
+	"context"
+	"fmt"
+	"testing"
+
+	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"github.com/apache/camel-k/v2/pkg/platform"
+	"github.com/apache/camel-k/v2/pkg/util/defaults"
+	"github.com/apache/camel-k/v2/pkg/util/log"
+	"github.com/apache/camel-k/v2/pkg/util/test"
+	"github.com/rs/xid"
+	"github.com/stretchr/testify/assert"
+
+	corev1 "k8s.io/api/core/v1"
+)
+
+func TestCanHandlePhaseReadyOrError(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Spec.Build.RuntimeVersion = defaults.DefaultRuntimeVersion
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseReady
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer := action.CanHandle(&ip)
+	assert.True(t, answer)
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseError
+	answer = action.CanHandle(&ip)
+	assert.True(t, answer)
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseCreateCatalog
+	answer = action.CanHandle(&ip)
+	assert.False(t, answer)
+}
+
+func TestMonitor(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	catalog := v1.NewCamelCatalog("ns", fmt.Sprintf("camel-catalog-%s", defaults.DefaultRuntimeVersion))
+	catalog.Spec.Runtime.Version = defaults.DefaultRuntimeVersion
+	catalog.Spec.Runtime.Provider = v1.RuntimeProviderQuarkus
+
+	c, err := test.NewFakeClient(&ip, &catalog)
+	assert.Nil(t, err)
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseReady, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Status)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+}
+
+func TestMonitorTransitionToCreateCatalog(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Spec.Build.RuntimeVersion = defaults.DefaultRuntimeVersion
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseCreateCatalog, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Status)
+	assert.Equal(t, corev1.ConditionFalse, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+	assert.Equal(t, v1.IntegrationPlatformConditionCamelCatalogAvailableReason, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Reason)
+	assert.Equal(t, fmt.Sprintf("camel catalog %s not available, please review given runtime version", defaults.DefaultRuntimeVersion), answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Message)
+}
+
+func TestMonitorRetainErrorState(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+	ip.Spec.Build.Registry.Address = defaults.OpenShiftRegistryAddress
+
+	ip.Spec.Build.RuntimeVersion = defaults.DefaultRuntimeVersion
+
+	ip.Status.Phase = v1.IntegrationPlatformPhaseError
+
+	c, err := test.NewFakeClient(&ip)
+	assert.Nil(t, err)
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseError, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionTrue, answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Status)
+	assert.Equal(t, corev1.ConditionFalse, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Status)
+	assert.Equal(t, v1.IntegrationPlatformConditionCamelCatalogAvailableReason, answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Reason)
+	assert.Equal(t, fmt.Sprintf("camel catalog %s not available, please review given runtime version", defaults.DefaultRuntimeVersion), answer.Status.GetCondition(v1.IntegrationPlatformConditionCamelCatalogAvailable).Message)
+}
+
+func TestMonitorMissingRegistryError(t *testing.T) {
+	ip := v1.IntegrationPlatform{}
+	ip.Namespace = "ns"
+	ip.Name = xid.New().String()
+	ip.Spec.Cluster = v1.IntegrationPlatformClusterOpenShift
+	ip.Spec.Profile = v1.TraitProfileOpenShift
+
+	catalog := v1.NewCamelCatalog("ns", fmt.Sprintf("camel-catalog-%s", defaults.DefaultRuntimeVersion))
+	catalog.Spec.Runtime.Version = defaults.DefaultRuntimeVersion
+	catalog.Spec.Runtime.Provider = v1.RuntimeProviderQuarkus
+
+	c, err := test.NewFakeClient(&ip, &catalog)
+	assert.Nil(t, err)
+
+	err = platform.ConfigureDefaults(context.TODO(), c, &ip, false)
+	assert.Nil(t, err)
+
+	action := NewMonitorAction()
+	action.InjectLogger(log.Log)
+	action.InjectClient(c)
+
+	answer, err := action.Handle(context.TODO(), &ip)
+	assert.Nil(t, err)
+	assert.NotNil(t, answer)
+
+	assert.Equal(t, v1.IntegrationPlatformPhaseError, answer.Status.Phase)
+	assert.Equal(t, corev1.ConditionFalse, answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Status)
+	assert.Equal(t, v1.IntegrationPlatformConditionTypeRegistryAvailableReason, answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Reason)
+	assert.Equal(t, "registry address not available, you need to set one", answer.Status.GetCondition(v1.IntegrationPlatformConditionTypeRegistryAvailable).Message)
+}
diff --git a/pkg/platform/defaults.go b/pkg/platform/defaults.go
index b8a87db2a..67e2d3415 100644
--- a/pkg/platform/defaults.go
+++ b/pkg/platform/defaults.go
@@ -140,7 +140,7 @@ func configureRegistry(ctx context.Context, c client.Client, p *v1.IntegrationPl
 		p.Status.Build.Registry.Address == "" {
 		log.Debugf("Integration Platform %s [%s]: setting registry address", p.Name, p.Namespace)
 		// Default to using OpenShift internal container images registry when using a strategy other than S2I
-		p.Status.Build.Registry.Address = "image-registry.openshift-image-registry.svc:5000"
+		p.Status.Build.Registry.Address = defaults.OpenShiftRegistryAddress
 
 		// OpenShift automatically injects the service CA certificate into the service-ca.crt key on the ConfigMap
 		cm, err := createServiceCaBundleConfigMap(ctx, c, p)
diff --git a/pkg/trait/camel.go b/pkg/trait/camel.go
index 45e518954..237564077 100644
--- a/pkg/trait/camel.go
+++ b/pkg/trait/camel.go
@@ -18,23 +18,18 @@ limitations under the License.
 package trait
 
 import (
-	"context"
 	"errors"
 	"fmt"
 	"strconv"
-	"strings"
 
 	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"
 	ctrl "sigs.k8s.io/controller-runtime/pkg/client"
 
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	traitv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1/trait"
 	"github.com/apache/camel-k/v2/pkg/util/camel"
 	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
-	"github.com/apache/camel-k/v2/pkg/util/maven"
 	"github.com/apache/camel-k/v2/pkg/util/property"
 )
 
@@ -124,57 +119,10 @@ func (t *camelTrait) loadOrCreateCatalog(e *Environment, runtimeVersion string)
 		// the required versions (camel and runtime) are not expressed as
 		// semver constraints
 		if exactVersionRegexp.MatchString(runtimeVersion) {
-			ctx, cancel := context.WithTimeout(e.Ctx, e.Platform.Status.Build.GetTimeout().Duration)
-			defer cancel()
-			catalog, err = camel.GenerateCatalog(ctx, e.Client,
-				catalogNamespace, e.Platform.Status.Build.Maven, runtime, []maven.Dependency{})
+			catalog, err = camel.CreateCatalog(e.Ctx, e.Client, catalogNamespace, e.Platform, runtime)
 			if err != nil {
 				return err
 			}
-
-			// sanitize catalog name
-			catalogName := "camel-catalog-" + strings.ToLower(runtimeVersion)
-
-			cx := v1.NewCamelCatalogWithSpecs(catalogNamespace, catalogName, catalog.CamelCatalogSpec)
-			cx.Labels = make(map[string]string)
-			cx.Labels["app"] = "camel-k"
-			cx.Labels["camel.apache.org/runtime.version"] = runtime.Version
-			cx.Labels["camel.apache.org/runtime.provider"] = string(runtime.Provider)
-			cx.Labels["camel.apache.org/catalog.generated"] = True
-
-			if err := e.Client.Create(e.Ctx, &cx); err != nil {
-				if k8serrors.IsAlreadyExists(err) {
-					// It's still possible that catalog wasn't yet found at the time of loading
-					// but then created in the background before the client tries to create it.
-					// In this case, simply try loading again and reuse the existing catalog.
-					catalog, err = camel.LoadCatalog(e.Ctx, e.Client, catalogNamespace, runtime)
-					if err != nil {
-						// unexpected error
-						return fmt.Errorf("catalog %q already exists but unable to load: %w", catalogName, err)
-					}
-				} else {
-					return fmt.Errorf("unable to create catalog runtime=%s, provider=%s, name=%s: %w",
-						runtime.Version,
-						runtime.Provider,
-						catalogName, err)
-
-				}
-			}
-
-			// verify that the catalog has been generated
-			ct, err := kubernetes.GetUnstructured(
-				e.Ctx,
-				e.Client,
-				schema.GroupVersionKind{Group: "camel.apache.org", Version: "v1", Kind: "CamelCatalog"},
-				catalogName,
-				catalogNamespace,
-			)
-			if ct == nil || err != nil {
-				return fmt.Errorf("unable to create catalog runtime=%s, provider=%s, name=%s: %w",
-					runtime.Version,
-					runtime.Provider,
-					catalogName, err)
-			}
 		}
 	}
 
diff --git a/pkg/trait/registry.go b/pkg/trait/registry.go
index daa1bb8ff..3efd76510 100644
--- a/pkg/trait/registry.go
+++ b/pkg/trait/registry.go
@@ -29,6 +29,7 @@ import (
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	traitv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1/trait"
 	"github.com/apache/camel-k/v2/pkg/platform"
+	"github.com/apache/camel-k/v2/pkg/util/defaults"
 	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
 	"github.com/apache/camel-k/v2/pkg/util/registry"
 
@@ -70,7 +71,7 @@ func (t *registryTrait) Configure(e *Environment) (bool, *TraitCondition, error)
 func (t *registryTrait) Apply(e *Environment) error {
 	registryAddress := e.Platform.Status.Build.Registry.Address
 	if registryAddress == "" && e.Platform.Status.Cluster == v1.IntegrationPlatformClusterOpenShift {
-		registryAddress = "image-registry.openshift-image-registry.svc:5000"
+		registryAddress = defaults.OpenShiftRegistryAddress
 	}
 	if registryAddress == "" {
 		return errors.New("could not figure out Image Registry URL, please set it manually")
diff --git a/pkg/util/camel/camel_runtime.go b/pkg/util/camel/camel_runtime.go
index ed613690a..412875783 100644
--- a/pkg/util/camel/camel_runtime.go
+++ b/pkg/util/camel/camel_runtime.go
@@ -19,13 +19,75 @@ package camel
 
 import (
 	"context"
+	"fmt"
+	"strings"
 
+	"github.com/apache/camel-k/v2/pkg/util/kubernetes"
+	"github.com/apache/camel-k/v2/pkg/util/maven"
+	k8serrors "k8s.io/apimachinery/pkg/api/errors"
+	"k8s.io/apimachinery/pkg/runtime/schema"
 	k8sclient "sigs.k8s.io/controller-runtime/pkg/client"
 
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	"github.com/apache/camel-k/v2/pkg/client"
 )
 
+// CreateCatalog --.
+func CreateCatalog(ctx context.Context, client client.Client, namespace string, platform *v1.IntegrationPlatform, runtime v1.RuntimeSpec) (*RuntimeCatalog, error) {
+	ctx, cancel := context.WithTimeout(ctx, platform.Status.Build.GetTimeout().Duration)
+	defer cancel()
+	catalog, err := GenerateCatalog(ctx, client, namespace, platform.Status.Build.Maven, runtime, []maven.Dependency{})
+	if err != nil {
+		return nil, err
+	}
+
+	// sanitize catalog name
+	catalogName := "camel-catalog-" + strings.ToLower(runtime.Version)
+
+	cx := v1.NewCamelCatalogWithSpecs(namespace, catalogName, catalog.CamelCatalogSpec)
+	cx.Labels = make(map[string]string)
+	cx.Labels["app"] = "camel-k"
+	cx.Labels["camel.apache.org/runtime.version"] = runtime.Version
+	cx.Labels["camel.apache.org/runtime.provider"] = string(runtime.Provider)
+	cx.Labels["camel.apache.org/catalog.generated"] = "true"
+
+	if err := client.Create(ctx, &cx); err != nil {
+		if k8serrors.IsAlreadyExists(err) {
+			// It's still possible that catalog wasn't yet found at the time of loading
+			// but then created in the background before the client tries to create it.
+			// In this case, simply try loading again and reuse the existing catalog.
+			catalog, err = LoadCatalog(ctx, client, namespace, runtime)
+			if err != nil {
+				// unexpected error
+				return nil, fmt.Errorf("catalog %q already exists but unable to load: %w", catalogName, err)
+			}
+		} else {
+			return nil, fmt.Errorf("unable to create catalog runtime=%s, provider=%s, name=%s: %w",
+				runtime.Version,
+				runtime.Provider,
+				catalogName, err)
+
+		}
+	}
+
+	// verify that the catalog has been generated
+	ct, err := kubernetes.GetUnstructured(
+		ctx,
+		client,
+		schema.GroupVersionKind{Group: "camel.apache.org", Version: "v1", Kind: "CamelCatalog"},
+		catalogName,
+		namespace,
+	)
+	if ct == nil || err != nil {
+		return nil, fmt.Errorf("unable to create catalog runtime=%s, provider=%s, name=%s: %w",
+			runtime.Version,
+			runtime.Provider,
+			catalogName, err)
+	}
+
+	return catalog, nil
+}
+
 // LoadCatalog --.
 func LoadCatalog(ctx context.Context, client client.Client, namespace string, runtime v1.RuntimeSpec) (*RuntimeCatalog, error) {
 	options := []k8sclient.ListOption{
diff --git a/pkg/util/defaults/defaults.go b/pkg/util/defaults/defaults.go
index d1d6b84a8..d4bff2ec0 100644
--- a/pkg/util/defaults/defaults.go
+++ b/pkg/util/defaults/defaults.go
@@ -37,6 +37,9 @@ const (
 	// ImageName --
 	ImageName = "docker.io/apache/camel-k"
 
+	// OpenShiftRegistryAddress --
+	OpenShiftRegistryAddress = "image-registry.openshift-image-registry.svc:5000"
+
 	// installDefaultKamelets --
 	installDefaultKamelets = true
 )
diff --git a/pkg/util/maven/maven_command.go b/pkg/util/maven/maven_command.go
index 922421f98..fc4707573 100644
--- a/pkg/util/maven/maven_command.go
+++ b/pkg/util/maven/maven_command.go
@@ -46,10 +46,12 @@ func (c *Command) Do(ctx context.Context) error {
 		return err
 	}
 
-	// Prepare maven wrapper helps when running the builder as Pod as it makes
-	// the builder container, Maven agnostic
-	if err := c.prepareMavenWrapper(ctx); err != nil {
-		return err
+	if e, ok := os.LookupEnv("MAVEN_WRAPPER"); (ok && e == "true") || !ok {
+		// Prepare maven wrapper helps when running the builder as Pod as it makes
+		// the builder container, Maven agnostic
+		if err := c.prepareMavenWrapper(ctx); err != nil {
+			return err
+		}
 	}
 
 	mvnCmd := "./mvnw"
diff --git a/pkg/util/test/client.go b/pkg/util/test/client.go
index 081a6733a..82657df94 100644
--- a/pkg/util/test/client.go
+++ b/pkg/util/test/client.go
@@ -25,7 +25,6 @@ import (
 	"github.com/apache/camel-k/v2/pkg/apis"
 	v1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
 	"github.com/apache/camel-k/v2/pkg/client"
-	camel "github.com/apache/camel-k/v2/pkg/client/camel/clientset/versioned"
 	fakecamelclientset "github.com/apache/camel-k/v2/pkg/client/camel/clientset/versioned/fake"
 	camelv1 "github.com/apache/camel-k/v2/pkg/client/camel/clientset/versioned/typed/camel/v1"
 	camelv1alpha1 "github.com/apache/camel-k/v2/pkg/client/camel/clientset/versioned/typed/camel/v1alpha1"
@@ -137,12 +136,16 @@ func filterObjects(scheme *runtime.Scheme, input []runtime.Object, filter func(g
 type FakeClient struct {
 	controller.Client
 	kubernetes.Interface
-	camel            camel.Interface
+	camel            *fakecamelclientset.Clientset
 	scales           *fakescale.FakeScaleClient
 	disabledGroups   []string
 	enabledOpenshift bool
 }
 
+func (c *FakeClient) AddReactor(verb, resource string, reaction testing.ReactionFunc) {
+	c.camel.AddReactor(verb, resource, reaction)
+}
+
 func (c *FakeClient) CamelV1() camelv1.CamelV1Interface {
 	return c.camel.CamelV1()
 }
diff --git a/pkg/util/util_test.go b/pkg/util/util_test.go
index 6a428b234..945b865a2 100644
--- a/pkg/util/util_test.go
+++ b/pkg/util/util_test.go
@@ -23,12 +23,13 @@ import (
 	"path/filepath"
 	"testing"
 
+	"github.com/apache/camel-k/v2/pkg/util/defaults"
 	"github.com/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
 )
 
 func TestStringContainsPrefix(t *testing.T) {
-	args := []string{"install", "--operator-image=xxx/yyy:zzz", "--registry", "image-registry.openshift-image-registry.svc:5000"}
+	args := []string{"install", "--operator-image=xxx/yyy:zzz", "--registry", defaults.OpenShiftRegistryAddress}
 	assert.True(t, StringContainsPrefix(args, "--operator-image="))
 	assert.False(t, StringContainsPrefix(args, "--olm"))
 }
diff --git a/script/Makefile b/script/Makefile
index 3d1adbcce..08b26559d 100644
--- a/script/Makefile
+++ b/script/Makefile
@@ -49,9 +49,10 @@ METADATA_IMAGE_NAME := $(CUSTOM_IMAGE)-metadata
 BUNDLE_IMAGE_NAME ?= $(CUSTOM_IMAGE)-bundle
 RELEASE_GIT_REMOTE := origin
 GIT_COMMIT := $(shell if [ -d .git ]; then git rev-list -1 HEAD; else echo "$(CUSTOM_VERSION)"; fi)
-LINT_GOGC := 10
+LINT_GOGC := 20
 LINT_DEADLINE := 10m
 DEBUG_MODE ?= false
+OPENSHIFT_REGISTRY := image-registry.openshift-image-registry.svc:5000
 
 # olm bundle vars
 MANAGER := config/manager
@@ -187,6 +188,9 @@ codegen:
 	@echo "  // ImageName -- " >> $(VERSIONFILE)
 	@echo "  ImageName = \"$(CUSTOM_IMAGE)\"" >> $(VERSIONFILE)
 	@echo "" >> $(VERSIONFILE)
+	@echo "  // OpenShiftRegistryAddress -- " >> $(VERSIONFILE)
+	@echo "  OpenShiftRegistryAddress = \"$(OPENSHIFT_REGISTRY)\"" >> $(VERSIONFILE)
+	@echo "" >> $(VERSIONFILE)
 	@echo "  // installDefaultKamelets -- " >> $(VERSIONFILE)
 	@echo "  installDefaultKamelets = $(INSTALL_DEFAULT_KAMELETS)" >> $(VERSIONFILE)
 	@echo ")" >> $(VERSIONFILE)
@@ -404,10 +408,10 @@ OS_LOWER := $(shell echo $(OS) | tr '[:upper:]' '[:lower:]')
 endif
 
 lint:
-	GOGC=$(LINT_GOGC) golangci-lint run --config .golangci.yml --out-format tab --deadline $(LINT_DEADLINE) --verbose
+	GOGC=$(LINT_GOGC) golangci-lint run --config .golangci.yml --out-format colored-tab --deadline $(LINT_DEADLINE) --verbose
 
 lint-fix:
-	GOGC=$(LINT_GOGC) golangci-lint run --config .golangci.yml --out-format tab --deadline $(LINT_DEADLINE) --fix
+	GOGC=$(LINT_GOGC) golangci-lint run --config .golangci.yml --out-format colored-tab --deadline $(LINT_DEADLINE) --fix
 
 dir-licenses:
 	./script/vendor-license-directory.sh