You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by as...@apache.org on 2019/11/04 16:10:31 UTC

[camel-k] 07/38: feat(quarkus): Support loading Camel Quarkus catalog

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

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

commit e6f028156593395e098eeec0a38b14902ee3bae0
Author: Antonin Stefanutti <an...@stefanutti.fr>
AuthorDate: Tue Oct 22 15:35:22 2019 +0200

    feat(quarkus): Support loading Camel Quarkus catalog
---
 pkg/builder/builder_steps.go           |  18 -----
 pkg/builder/runtime/main.go            |  13 ++++
 pkg/builder/runtime/quarkus.go         |  15 +++-
 pkg/builder/runtime/runtime.go         |  10 +++
 pkg/controller/integrationkit/build.go |  16 ++--
 pkg/trait/camel.go                     | 132 ++++++++++++++++++---------------
 pkg/trait/camel_test.go                |   9 +--
 pkg/trait/quarkus.go                   |  23 ++++++
 pkg/util/camel/camel_runtime.go        |  30 ++++++--
 pkg/util/camel/camel_util.go           |  35 ++++++++-
 pkg/util/camel/camel_util_test.go      |  20 ++---
 pkg/util/camel/catalog.go              |   2 +-
 12 files changed, 210 insertions(+), 113 deletions(-)

diff --git a/pkg/builder/builder_steps.go b/pkg/builder/builder_steps.go
index d6b1f3f..32a54fa 100644
--- a/pkg/builder/builder_steps.go
+++ b/pkg/builder/builder_steps.go
@@ -27,7 +27,6 @@ import (
 	k8sclient "sigs.k8s.io/controller-runtime/pkg/client"
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
-	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
 	"github.com/apache/camel-k/pkg/util/maven"
 	"github.com/apache/camel-k/pkg/util/tar"
@@ -40,7 +39,6 @@ func init() {
 }
 
 type steps struct {
-	LoadCatalog             Step
 	GenerateProjectSettings Step
 	InjectDependencies      Step
 	SanitizeDependencies    Step
@@ -50,10 +48,6 @@ type steps struct {
 
 // Steps --
 var Steps = steps{
-	LoadCatalog: NewStep(
-		InitPhase,
-		loadCatalog,
-	),
 	GenerateProjectSettings: NewStep(
 		ProjectGenerationPhase+1,
 		generateProjectSettings,
@@ -78,7 +72,6 @@ var Steps = steps{
 
 // DefaultSteps --
 var DefaultSteps = []Step{
-	Steps.LoadCatalog,
 	Steps.GenerateProjectSettings,
 	Steps.InjectDependencies,
 	Steps.SanitizeDependencies,
@@ -111,17 +104,6 @@ func registerStep(steps ...Step) {
 	}
 }
 
-func loadCatalog(ctx *Context) error {
-	catalog, err := camel.LoadCatalog(ctx.C, ctx.Client, ctx.Build.Meta.Namespace, ctx.Build.CamelVersion, ctx.Build.RuntimeVersion)
-	if err != nil {
-		return err
-	}
-
-	ctx.Catalog = catalog
-
-	return nil
-}
-
 func generateProjectSettings(ctx *Context) error {
 	val, err := kubernetes.ResolveValueSource(ctx.C, ctx.Client, ctx.Namespace, &ctx.Build.Platform.Build.Maven.Settings)
 	if err != nil {
diff --git a/pkg/builder/runtime/main.go b/pkg/builder/runtime/main.go
index 4b363a0..03fddda 100644
--- a/pkg/builder/runtime/main.go
+++ b/pkg/builder/runtime/main.go
@@ -27,16 +27,29 @@ import (
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/builder"
+	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/defaults"
 	"github.com/apache/camel-k/pkg/util/maven"
 )
 
 // MainSteps --
 var MainSteps = []builder.Step{
+	Steps.LoadCamelCatalog,
 	Steps.GenerateProject,
 	Steps.ComputeDependencies,
 }
 
+func loadCamelCatalog(ctx *builder.Context) error {
+	catalog, err := camel.LoadCatalog(ctx.C, ctx.Client, ctx.Build.Meta.Namespace, ctx.Build.CamelVersion, ctx.Build.RuntimeVersion, nil)
+	if err != nil {
+		return err
+	}
+
+	ctx.Catalog = catalog
+
+	return nil
+}
+
 func generateProject(ctx *builder.Context) error {
 	p := maven.NewProjectWithGAV("org.apache.camel.k.integration", "camel-k-integration", defaults.Version)
 	p.Properties = ctx.Build.Platform.Build.Properties
diff --git a/pkg/builder/runtime/quarkus.go b/pkg/builder/runtime/quarkus.go
index df0faf5..c3c734a 100644
--- a/pkg/builder/runtime/quarkus.go
+++ b/pkg/builder/runtime/quarkus.go
@@ -10,16 +10,29 @@ import (
 
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/builder"
+	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/defaults"
 	"github.com/apache/camel-k/pkg/util/maven"
 )
 
 // QuarkusSteps --
 var QuarkusSteps = []builder.Step{
+	Steps.LoadCamelQuarkusCatalog,
 	Steps.GenerateQuarkusProject,
 	Steps.ComputeQuarkusDependencies,
 }
 
+func loadCamelQuarkusCatalog(ctx *builder.Context) error {
+	catalog, err := camel.LoadCatalog(ctx.C, ctx.Client, ctx.Build.Meta.Namespace, ctx.Build.CamelVersion, ctx.Build.RuntimeVersion, ctx.Build.RuntimeProvider.Quarkus)
+	if err != nil {
+		return err
+	}
+
+	ctx.Catalog = catalog
+
+	return nil
+}
+
 func generateQuarkusProject(ctx *builder.Context) error {
 	p := maven.NewProjectWithGAV("org.apache.camel.k.integration", "camel-k-integration", defaults.Version)
 	p.Properties = ctx.Build.Platform.Build.Properties
@@ -82,7 +95,7 @@ func computeQuarkusDependencies(ctx *builder.Context) error {
 
 	// Build the project, as the quarkus-bootstrap plugin build-tree goal
 	// requires the artifact to be installed
-	mc.AddArgument("package")
+	mc.AddArgument("install")
 	if err := maven.Run(mc); err != nil {
 		return errors.Wrap(err, "failure while building project")
 	}
diff --git a/pkg/builder/runtime/runtime.go b/pkg/builder/runtime/runtime.go
index 22106e0..ebd2162 100644
--- a/pkg/builder/runtime/runtime.go
+++ b/pkg/builder/runtime/runtime.go
@@ -28,15 +28,21 @@ func init() {
 // TODO: organise runtime steps into nested structs
 type steps struct {
 	// Main
+	LoadCamelCatalog    builder.Step
 	GenerateProject     builder.Step
 	ComputeDependencies builder.Step
 	// Quarkus
+	LoadCamelQuarkusCatalog    builder.Step
 	GenerateQuarkusProject     builder.Step
 	ComputeQuarkusDependencies builder.Step
 }
 
 var Steps = steps{
 	// Main
+	LoadCamelCatalog: builder.NewStep(
+		builder.InitPhase,
+		loadCamelCatalog,
+	),
 	GenerateProject: builder.NewStep(
 		builder.ProjectGenerationPhase,
 		generateProject,
@@ -46,6 +52,10 @@ var Steps = steps{
 		computeDependencies,
 	),
 	// Quarkus
+	LoadCamelQuarkusCatalog: builder.NewStep(
+		builder.InitPhase,
+		loadCamelQuarkusCatalog,
+	),
 	GenerateQuarkusProject: builder.NewStep(
 		builder.ProjectGenerationPhase,
 		generateQuarkusProject,
diff --git a/pkg/controller/integrationkit/build.go b/pkg/controller/integrationkit/build.go
index 7f68382..cbc1471 100644
--- a/pkg/controller/integrationkit/build.go
+++ b/pkg/controller/integrationkit/build.go
@@ -92,13 +92,15 @@ func (action *buildAction) handleBuildSubmitted(ctx context.Context, kit *v1alph
 				Name:      kit.Name,
 			},
 			Spec: v1alpha1.BuildSpec{
-				Meta:           kit.ObjectMeta,
-				CamelVersion:   env.CamelCatalog.Version,
-				RuntimeVersion: env.RuntimeVersion,
-				Platform:       env.Platform.Spec,
-				Dependencies:   kit.Spec.Dependencies,
-				Steps:          builder.StepIDsFor(env.Steps...),
-				BuildDir:       env.BuildDir,
+				Meta:            kit.ObjectMeta,
+				CamelVersion:    env.CamelCatalog.Version,
+				RuntimeVersion:  env.RuntimeVersion,
+				RuntimeProvider: env.CamelCatalog.RuntimeProvider,
+				Platform:        env.Platform.Spec,
+				Dependencies:    kit.Spec.Dependencies,
+				// TODO: sort for easy read
+				Steps:    builder.StepIDsFor(env.Steps...),
+				BuildDir: env.BuildDir,
 			},
 		}
 
diff --git a/pkg/trait/camel.go b/pkg/trait/camel.go
index b776d9c..c7c8a61 100644
--- a/pkg/trait/camel.go
+++ b/pkg/trait/camel.go
@@ -25,15 +25,17 @@ import (
 	"regexp"
 	"strings"
 
+	yaml2 "gopkg.in/yaml.v2"
+
+	"github.com/pkg/errors"
+
+	k8serrors "k8s.io/apimachinery/pkg/api/errors"
+
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/defaults"
 	"github.com/apache/camel-k/pkg/util/kubernetes"
 	"github.com/apache/camel-k/pkg/util/maven"
-	"github.com/pkg/errors"
-
-	yaml2 "gopkg.in/yaml.v2"
-	k8serrors "k8s.io/apimachinery/pkg/api/errors"
 )
 
 type camelTrait struct {
@@ -57,11 +59,6 @@ func (t *camelTrait) Configure(e *Environment) (bool, error) {
 }
 
 func (t *camelTrait) Apply(e *Environment) error {
-	ns := e.DetermineNamespace()
-	if ns == "" {
-		return errors.New("unable to determine namespace")
-	}
-
 	cv := e.DetermineCamelVersion()
 	rv := e.DetermineRuntimeVersion()
 
@@ -73,52 +70,21 @@ func (t *camelTrait) Apply(e *Environment) error {
 	}
 
 	if e.CamelCatalog == nil {
-		c, err := camel.LoadCatalog(e.C, e.Client, ns, cv, rv)
+		var catalog *camel.RuntimeCatalog
+		var err error
+		quarkus := e.Catalog.GetTrait("quarkus").(*quarkusTrait)
+		if quarkus.isEnabled() {
+			catalog, err = quarkus.loadOrCreateCatalog(e, cv, rv)
+		} else {
+			catalog, err = t.loadOrCreateCatalog(e, cv, rv)
+		}
 		if err != nil {
 			return err
-		}
-		if c == nil {
-			// if the catalog is not found in the cluster, try to create it if
-			// the required versions (camel and runtime) are not expressed as
-			// semver constraints
-			cvHasFixedVersion, err := regexp.MatchString(`^(\d+)\.(\d+)\.([\w-\.]+)$`, cv)
-			if err != nil {
-				return err
-			}
-			rvHasFixedVersion, err := regexp.MatchString(`^(\d+)\.(\d+)\.([\w-\.]+)$`, rv)
-			if err != nil {
-				return err
-			}
-
-			if cvHasFixedVersion && rvHasFixedVersion {
-				c, err = t.GenerateCatalog(e, cv, rv)
-				if err != nil {
-					return err
-				}
-
-				// sanitize catalog name
-				catalogName := "camel-catalog-" + strings.ToLower(cv+"-"+rv)
-
-				cx := v1alpha1.NewCamelCatalogWithSpecs(ns, catalogName, c.CamelCatalogSpec)
-				cx.Labels = make(map[string]string)
-				cx.Labels["app"] = "camel-k"
-				cx.Labels["camel.apache.org/catalog.version"] = cv
-				cx.Labels["camel.apache.org/catalog.loader.version"] = cv
-				cx.Labels["camel.apache.org/runtime.version"] = rv
-				cx.Labels["camel.apache.org/catalog.generated"] = True
-
-				err = e.Client.Create(e.C, &cx)
-				if err != nil && !k8serrors.IsAlreadyExists(err) {
-					return err
-				}
-			}
-		}
-
-		if c == nil {
+		} else if catalog == nil {
 			return fmt.Errorf("unable to find catalog for: %s", cv)
 		}
 
-		e.CamelCatalog = c
+		e.CamelCatalog = catalog
 	}
 
 	e.RuntimeVersion = rv
@@ -126,17 +92,68 @@ func (t *camelTrait) Apply(e *Environment) error {
 	if e.Integration != nil {
 		e.Integration.Status.CamelVersion = e.CamelCatalog.Version
 		e.Integration.Status.RuntimeVersion = rv
+		e.Integration.Status.RuntimeProvider = e.CamelCatalog.RuntimeProvider
 	}
 	if e.IntegrationKit != nil {
 		e.IntegrationKit.Status.CamelVersion = e.CamelCatalog.Version
 		e.IntegrationKit.Status.RuntimeVersion = rv
+		e.IntegrationKit.Status.RuntimeProvider = e.CamelCatalog.RuntimeProvider
 	}
 
 	return nil
 }
 
-// GenerateCatalog --
-func (t *camelTrait) GenerateCatalog(e *Environment, camelVersion string, runtimeVersion string) (*camel.RuntimeCatalog, error) {
+func (t *camelTrait) loadOrCreateCatalog(e *Environment, camelVersion string, runtimeVersion string) (*camel.RuntimeCatalog, error) {
+	ns := e.DetermineNamespace()
+	if ns == "" {
+		return nil, errors.New("unable to determine namespace")
+	}
+
+	c, err := camel.LoadCatalog(e.C, e.Client, ns, camelVersion, runtimeVersion, nil)
+	if err != nil {
+		return nil, err
+	}
+
+	if c == nil {
+		// if the catalog is not found in the cluster, try to create it if
+		// the required versions (camel and runtime) are not expressed as
+		// semver constraints
+		cvHasFixedVersion, err := regexp.MatchString(`^(\d+)\.(\d+)\.([\w-\.]+)$`, camelVersion)
+		if err != nil {
+			return nil, err
+		}
+		rvHasFixedVersion, err := regexp.MatchString(`^(\d+)\.(\d+)\.([\w-\.]+)$`, runtimeVersion)
+		if err != nil {
+			return nil, err
+		}
+
+		if cvHasFixedVersion && rvHasFixedVersion {
+			c, err = t.generateCatalog(e, camelVersion, runtimeVersion)
+			if err != nil {
+				return nil, err
+			}
+
+			// sanitize catalog name
+			catalogName := "camel-catalog-" + strings.ToLower(camelVersion+"-"+runtimeVersion)
+
+			cx := v1alpha1.NewCamelCatalogWithSpecs(ns, catalogName, c.CamelCatalogSpec)
+			cx.Labels = make(map[string]string)
+			cx.Labels["app"] = "camel-k"
+			cx.Labels["camel.apache.org/catalog.version"] = camelVersion
+			cx.Labels["camel.apache.org/catalog.loader.version"] = camelVersion
+			cx.Labels["camel.apache.org/runtime.version"] = runtimeVersion
+			cx.Labels["camel.apache.org/catalog.generated"] = True
+
+			err = e.Client.Create(e.C, &cx)
+			if err != nil && !k8serrors.IsAlreadyExists(err) {
+				return nil, err
+			}
+		}
+	}
+	return c, nil
+}
+
+func (t *camelTrait) generateCatalog(e *Environment, camelVersion string, runtimeVersion string) (*camel.RuntimeCatalog, error) {
 	root := os.TempDir()
 	tmpDir, err := ioutil.TempDir(root, "camel-catalog")
 	if err != nil {
@@ -149,10 +166,7 @@ func (t *camelTrait) GenerateCatalog(e *Environment, camelVersion string, runtim
 		return nil, err
 	}
 
-	project, err := t.GenerateMavenProject(camelVersion, runtimeVersion)
-	if err != nil {
-		return nil, err
-	}
+	project := t.generateMavenProject(camelVersion, runtimeVersion)
 
 	mc := maven.NewContext(tmpDir, project)
 	mc.LocalRepository = e.Platform.Spec.Build.LocalRepository
@@ -191,9 +205,9 @@ func (t *camelTrait) GenerateCatalog(e *Environment, camelVersion string, runtim
 	return camel.NewRuntimeCatalog(catalog.Spec), nil
 }
 
-// GenerateCatalogMavenProject --
-func (t *camelTrait) GenerateMavenProject(camelVersion string, runtimeVersion string) (maven.Project, error) {
+func (t *camelTrait) generateMavenProject(camelVersion string, runtimeVersion string) maven.Project {
 	p := maven.NewProjectWithGAV("org.apache.camel.k.integration", "camel-k-catalog-generator", defaults.Version)
+
 	p.Build = &maven.Build{
 		DefaultGoal: "generate-resources",
 		Plugins: []maven.Plugin{
@@ -220,5 +234,5 @@ func (t *camelTrait) GenerateMavenProject(camelVersion string, runtimeVersion st
 		},
 	}
 
-	return p, nil
+	return p
 }
diff --git a/pkg/trait/camel_test.go b/pkg/trait/camel_test.go
index ee35856..24c5032 100644
--- a/pkg/trait/camel_test.go
+++ b/pkg/trait/camel_test.go
@@ -72,8 +72,7 @@ func TestApplyCamelTraitWithoutEnvironmentCatalogAndUnmatchableVersionFails(t *t
 func TestCamelTraitGenerateMavenProjectSucceeds(t *testing.T) {
 	trait, _ := createNominalCamelTest()
 
-	mvnProject, err := trait.GenerateMavenProject("1.23.0", "1.0.0")
-	assert.Nil(t, err)
+	mvnProject := trait.generateMavenProject("1.23.0", "1.0.0")
 	assert.NotNil(t, mvnProject)
 	assert.Equal(t, "org.apache.camel.k.integration", mvnProject.GroupID)
 	assert.Equal(t, "camel-k-catalog-generator", mvnProject.ArtifactID)
@@ -96,7 +95,6 @@ func TestCamelTraitGenerateMavenProjectSucceeds(t *testing.T) {
 }
 
 func createNominalCamelTest() (*camelTrait, *Environment) {
-
 	client, _ := test.NewFakeClient()
 
 	trait := newCamelTrait()
@@ -109,8 +107,9 @@ func createNominalCamelTest() (*camelTrait, *Environment) {
 				Version: "1.23.0",
 			},
 		},
-		C:      context.TODO(),
-		Client: client,
+		Catalog: NewEnvironmentTestCatalog(),
+		C:       context.TODO(),
+		Client:  client,
 		Integration: &v1alpha1.Integration{
 			ObjectMeta: metav1.ObjectMeta{
 				Namespace: "namespace",
diff --git a/pkg/trait/quarkus.go b/pkg/trait/quarkus.go
index 8b70b11..2be0cac 100644
--- a/pkg/trait/quarkus.go
+++ b/pkg/trait/quarkus.go
@@ -21,10 +21,13 @@ import (
 	"fmt"
 	"strings"
 
+	"github.com/pkg/errors"
+
 	"github.com/apache/camel-k/pkg/apis/camel/v1alpha1"
 	"github.com/apache/camel-k/pkg/builder/runtime"
 	"github.com/apache/camel-k/pkg/metadata"
 	"github.com/apache/camel-k/pkg/util"
+	"github.com/apache/camel-k/pkg/util/camel"
 	"github.com/apache/camel-k/pkg/util/defaults"
 	"github.com/apache/camel-k/pkg/util/envvar"
 )
@@ -51,6 +54,26 @@ func (t *quarkusTrait) Apply(e *Environment) error {
 	return nil
 }
 
+func (t *quarkusTrait) loadOrCreateCatalog(e *Environment, camelVersion string, runtimeVersion string) (*camel.RuntimeCatalog, error) {
+	ns := e.DetermineNamespace()
+	if ns == "" {
+		return nil, errors.New("unable to determine namespace")
+	}
+
+	c, err := camel.LoadCatalog(e.C, e.Client, ns, camelVersion, runtimeVersion, v1alpha1.QuarkusRuntimeProvider{
+		// FIXME
+		CamelQuarkusVersion: "0.2.0",
+		QuarkusVersion:      "0.21.2",
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	// TODO: generate a catalog if nil
+
+	return c, nil
+}
+
 func (t *quarkusTrait) addBuildSteps(e *Environment) {
 	e.Steps = append(e.Steps, runtime.QuarkusSteps...)
 }
diff --git a/pkg/util/camel/camel_runtime.go b/pkg/util/camel/camel_runtime.go
index a0d502a..331d648 100644
--- a/pkg/util/camel/camel_runtime.go
+++ b/pkg/util/camel/camel_runtime.go
@@ -28,24 +28,38 @@ import (
 )
 
 // LoadCatalog --
-func LoadCatalog(ctx context.Context, client client.Client, namespace string, camelVersion string, runtimeVersion string) (*RuntimeCatalog, error) {
-	var catalog *RuntimeCatalog
-	var err error
+func LoadCatalog(ctx context.Context, client client.Client, namespace string, camelVersion string, runtimeVersion string, provider interface{}) (*RuntimeCatalog, error) {
+	options := []k8sclient.ListOption{
+		k8sclient.InNamespace(namespace),
+	}
+
+	if _, ok := provider.(v1alpha1.QuarkusRuntimeProvider); ok {
+		options = append(options, k8sclient.MatchingLabels{
+			"camel.apache.org/runtime.provider": "quarkus",
+		})
+	}
 
 	list := v1alpha1.NewCamelCatalogList()
-	err = client.List(ctx, &list, k8sclient.InNamespace(namespace))
+	err := client.List(ctx, &list, options...)
 	if err != nil {
 		return nil, err
 	}
 
-	catalog, err = findBestMatch(list.Items, camelVersion, runtimeVersion)
+	catalog, err := findBestMatch(list.Items, camelVersion, runtimeVersion, provider)
 	if err != nil {
 		return nil, err
 	}
 
-	if catalog == nil {
-		return nil, fmt.Errorf("unable to find catalog matching version requirement: camel=%s, runtime=%s", camelVersion, runtimeVersion)
+	if catalog != nil {
+		return catalog, nil
 	}
 
-	return catalog, nil
+	switch provider := provider.(type) {
+	case v1alpha1.QuarkusRuntimeProvider:
+		return nil, fmt.Errorf("unable to find catalog matching version requirement: camel=%s, runtime=%s, camel-quarkus=%s, quarkus=%s",
+			camelVersion, runtimeVersion, provider.CamelQuarkusVersion, provider.QuarkusVersion)
+	default:
+		return nil, fmt.Errorf("unable to find catalog matching version requirement: camel=%s, runtime=%s",
+			camelVersion, runtimeVersion)
+	}
 }
diff --git a/pkg/util/camel/camel_util.go b/pkg/util/camel/camel_util.go
index ae80ac7..6aecdfe 100644
--- a/pkg/util/camel/camel_util.go
+++ b/pkg/util/camel/camel_util.go
@@ -26,7 +26,8 @@ import (
 	"github.com/apache/camel-k/pkg/util/log"
 )
 
-func findBestMatch(catalogs []v1alpha1.CamelCatalog, camelVersion string, runtimeVersion string) (*RuntimeCatalog, error) {
+func findBestMatch(catalogs []v1alpha1.CamelCatalog, camelVersion string, runtimeVersion string, provider interface{}) (*RuntimeCatalog, error) {
+	// FIXME: take the provider into account for exact match
 	for _, catalog := range catalogs {
 		if catalog.Spec.Version == camelVersion && catalog.Spec.RuntimeVersion == runtimeVersion {
 			return NewRuntimeCatalog(catalog.Spec), nil
@@ -40,9 +41,35 @@ func findBestMatch(catalogs []v1alpha1.CamelCatalog, camelVersion string, runtim
 	}
 
 	cc := newCatalogVersionCollection(catalogs)
-	for _, x := range cc {
-		if vc.Check(x.Version) && rc.Check(x.RuntimeVersion) {
-			return NewRuntimeCatalog(x.Catalog.Spec), nil
+
+	switch provider := provider.(type) {
+	case v1alpha1.QuarkusRuntimeProvider:
+		qc := newSemVerConstraint(provider.QuarkusVersion)
+		cqc := newSemVerConstraint(provider.CamelQuarkusVersion)
+		if qc == nil || cqc == nil {
+			return nil, nil
+		}
+		for _, c := range cc {
+			qv, err := semver.NewVersion(c.Catalog.Spec.RuntimeProvider.Quarkus.QuarkusVersion)
+			if err != nil {
+				log.Debugf("Invalid semver version (quarkus) %s", qv)
+				continue
+			}
+			cqv, err := semver.NewVersion(c.Catalog.Spec.RuntimeProvider.Quarkus.CamelQuarkusVersion)
+			if err != nil {
+				log.Debugf("Invalid semver version (camel quarkus) %s", cqv)
+				continue
+			}
+			if vc.Check(c.Version) && rc.Check(c.RuntimeVersion) && qc.Check(qv) && cqc.Check(cqv) {
+				return NewRuntimeCatalog(c.Catalog.Spec), nil
+			}
+		}
+
+	default:
+		for _, c := range cc {
+			if vc.Check(c.Version) && rc.Check(c.RuntimeVersion) {
+				return NewRuntimeCatalog(c.Catalog.Spec), nil
+			}
 		}
 	}
 
diff --git a/pkg/util/camel/camel_util_test.go b/pkg/util/camel/camel_util_test.go
index 456556a..2e02c50 100644
--- a/pkg/util/camel/camel_util_test.go
+++ b/pkg/util/camel/camel_util_test.go
@@ -32,7 +32,7 @@ func TestFindBestMatch_Camel(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "~2.23.x", "1.0.0")
+	c, err := findBestMatch(catalogs, "~2.23.x", "1.0.0", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1", c.Version)
@@ -45,7 +45,7 @@ func TestFindBestMatch_Runtime(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.0", "~1.0.x")
+	c, err := findBestMatch(catalogs, "2.23.0", "~1.0.x", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.0", c.Version)
@@ -60,7 +60,7 @@ func TestFindBestMatch(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "~2.23.x", "~1.0.x")
+	c, err := findBestMatch(catalogs, "~2.23.x", "~1.0.x", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1", c.Version)
@@ -74,7 +74,7 @@ func TestFindExactSemVerMatch_Camel(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.0", "1.0.0")
+	c, err := findBestMatch(catalogs, "2.23.0", "1.0.0", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.0", c.Version)
@@ -87,7 +87,7 @@ func TestFindExactSemVerMatch_Runtime(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.0", "1.0.0")
+	c, err := findBestMatch(catalogs, "2.23.0", "1.0.0", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.0", c.Version)
@@ -102,7 +102,7 @@ func TestFindExactMatch_Camel(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.1-tag-00001", "1.0.0")
+	c, err := findBestMatch(catalogs, "2.23.1-tag-00001", "1.0.0", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1-tag-00001", c.Version)
@@ -116,7 +116,7 @@ func TestFindExactMatch_Runtime(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.1-tag-00001", "1.0.1")
+	c, err := findBestMatch(catalogs, "2.23.1-tag-00001", "1.0.1", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1-tag-00001", c.Version)
@@ -131,7 +131,7 @@ func TestFindRangeMatch_Camel(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, ">= 2.23.0, < 2.23.2", "1.0.0")
+	c, err := findBestMatch(catalogs, ">= 2.23.0, < 2.23.2", "1.0.0", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1", c.Version)
@@ -146,7 +146,7 @@ func TestFindRangeMatch_Runtime(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, "2.23.0", "> 1.0.1, < 1.0.3")
+	c, err := findBestMatch(catalogs, "2.23.0", "> 1.0.1, < 1.0.3", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.0", c.Version)
@@ -163,7 +163,7 @@ func TestFindRangeMatch(t *testing.T) {
 		{Spec: v1alpha1.CamelCatalogSpec{Version: "2.22.1", RuntimeVersion: "1.0.0"}},
 	}
 
-	c, err := findBestMatch(catalogs, ">= 2.23.0, < 2.23.2", "> 1.0.1, < 1.0.3")
+	c, err := findBestMatch(catalogs, ">= 2.23.0, < 2.23.2", "> 1.0.1, < 1.0.3", nil)
 	assert.Nil(t, err)
 	assert.NotNil(t, c)
 	assert.Equal(t, "2.23.1", c.Version)
diff --git a/pkg/util/camel/catalog.go b/pkg/util/camel/catalog.go
index 4b8df56..a5a6d1e 100644
--- a/pkg/util/camel/catalog.go
+++ b/pkg/util/camel/catalog.go
@@ -42,5 +42,5 @@ func DefaultCatalog() (*RuntimeCatalog, error) {
 		}
 	}
 
-	return findBestMatch(catalogs, defaults.DefaultCamelVersion, defaults.DefaultRuntimeVersion)
+	return findBestMatch(catalogs, defaults.DefaultCamelVersion, defaults.DefaultRuntimeVersion, nil)
 }