You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by "lburgazzoli (via GitHub)" <gi...@apache.org> on 2023/05/09 13:38:56 UTC

[GitHub] [camel-k] lburgazzoli opened a new pull request, #4350: feat: use contaier image as kamelet repository

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

   <!-- 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
   NONE
   ```
   


-- 
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 #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   I think that we'll probably have to manage that as soon as we introduce certain logic into the IP. Right now, the only possibility we have for an IP to be in error is that it cannot connect to the registry for any reason (although we are not doing that check).



-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   At this stage, the repository is only loaded once, so any new container image is not taken into account till the restart of the camel-k operator.



-- 
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] oscerd commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)

Review Comment:
   I would go with 3 for the moment. To avoid other complexities



-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   Right I was about to add pull policies but noticed that as today the way a repository is defined is limited as it is essentially an uri : https://github.com/apache/camel-k/blob/24172037c0a71caf96fa88afc5282d5edfddfde2/pkg/apis/camel/v1/integrationplatform_types.go#L146-L149
   
   
   So I wonder if this must be changed in something more type safe, in line to what has been done for traits, like:
   
   ```go
   type IntegrationPlatformKameletRepositorySpec struct {
       GitHub *GitHubKameletRepo `json:"github,omitempty"`
       OCI    *OCIKameletRepo    `json:"oci,omitempty"`
   }
   ```
   
   So we can add additional repo specific option, as example the authentication option for the oci repo.
   
   The only issue is that by definition, if the tag of the image is `latest`, then the pull policy should become `Always` so I was thinking to use a rate limiter alike apporach that would also cache and check the checksum/digest of the manifest before actually pulling the layers.
   



-- 
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


Re: [PR] feat: use contaier image as kamelet repository [camel-k]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #4350: feat: use contaier image as kamelet repository
URL: https://github.com/apache/camel-k/pull/4350


-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {

Review Comment:
   > If the image only contains kamelet files, the overhead sounds negligible
   
   correct, if we assume that the repository includes only kamelet files, then it won't be a big deal. 



-- 
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] astefanutti commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)

Review Comment:
   +1 for 3) to keep things simple 👍🏼.



-- 
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] johnpoth commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > > > I think I did some exploration with go-containerregistry in the past, but I have not seen an option to define a custom media type or annotations for the layers, do you know if that's possible ?
   > > 
   > > 
   > > Yes it does! BUT setting a custom media type comes at the cost of the image being rejected in most registries or clients. This is why I didn't set the custom media type when working on local dependencies. Although very appealing it wasn't worth the interoperability drawbacks IMO. This was ~1 years ago so maybe things have changed but looking at the list of supported registries it doesn't look like it...
   > 
   > Yeah, that is essentially also the conclusion in #2732, but I had hope for things to have become better, but looks like it is not. The alternative would be to use some annotations to help distinguish between the type of layers but, given the spec from the ORAS Artifacts have been merged to the OpenContainers specs I had an hope to get things more widely supported :)
   > 
   > Happy to switch to `go-containerregistry` at some point, @johnpoth do you by chance have some code I can take a look ?
   
   Sure! So for building the Image I would just use Spectrum or [Crane](https://github.com/google/go-containerregistry/blob/main/cmd/crane/recipes.md) tbh.. both are built on top of `go-containerregistry` anyway so:
   
   ```
   spectrum build -b scratch  -t docker.io/jpoth/camel-kamelets camel-kamelets/kamelets:/
   ```
   
   or Crane
   
   ```
   crane append  -f <(tar -f - -c camel-kamelets/kamelets/) -t docker.io/jpoth/camel-kamelets
   ```
   
   This will create an Image with just the Kamelet files in a single layer
   
   And for the Operator part I would use the equivalent of
   
   ```
   crane pull  jpoth/camel-kamelets kamelets.tar
   ```
   
   which is just a Crane [Pull](https://github.com/google/go-containerregistry/blob/a927d7c995a991edeb54e840f66d8cfffcf30d62/cmd/crane/cmd/export.go#L73) followed by an [Export](https://github.com/google/go-containerregistry/blob/a927d7c995a991edeb54e840f66d8cfffcf30d62/cmd/crane/cmd/export.go#LL79C21-L79C21). The [mutate.Export](https://github.com/google/go-containerregistry/blob/a927d7c995a991edeb54e840f66d8cfffcf30d62/pkg/v1/mutate/mutate.go#L249) is used which returns a _stream_. This allows you to not create a temporary file when extracting the Kamelets: for [example](https://github.com/google/go-containerregistry/blob/a927d7c995a991edeb54e840f66d8cfffcf30d62/pkg/v1/mutate/mutate_test.go#L57).
   
   **For authentication**, you can reuse the Spectrum auth we already [have](https://github.com/apache/camel-k/blob/main/pkg/builder/spectrum.go#LL116C1-L116C1)... 
   
   
   


-- 
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] johnpoth commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)

Review Comment:
   You could store a Kamelet in a layer and have it referenced in the Manifest. That way if you could pull a single Kamelet by getting it's address in the Manifest. The only drawback is the 127 layer limit so you could "only" store 127 Kamelets per image ...



-- 
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] astefanutti commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   +1 for structuring the repository definitions.
   
   +1 to comply with Kubernetes `imagePullPolicy` defaulting: https://kubernetes.io/docs/concepts/containers/images/#imagepullpolicy-defaulting.



-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {

Review Comment:
   This is not super optimized as it unpack the entire image in a temporary folder but I have not yet found a good way to filter out the content.



-- 
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] johnpoth commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {

Review Comment:
   If the image only contains kamelet files, the overhead sounds negligible  



-- 
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] lburgazzoli commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > Looks great! As Antonin said, eager to see everything in action!
   > 
   > I also think it would be cool if you could pull the image via docker/podman:
   > 
   > ```
   > docker pull lburgazzoli/camel-kamelets:latest
   > podman pull lburgazzoli/camel-kamelets:latest
   > ```
   > 
   > Right now I'm getting
   > 
   > ```
   > Error response from daemon: missing signature key
   > ```
   > 
   > I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   > 
   
   As today, I've blindly followed the `oras` tutorial so it could be possible that what they create is not compatible with containers tools. If you have any time to digg into the issue, I would really appreciate it. 


-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)

Review Comment:
   So theoretically it should be possible to inspect the manifest and lookup the layer i.e. by using some annotations, however I think the main decision factor would be how we would like to package the kamelets: 
   
   1. 1 kamelet per layer
   2. 1 kamelet group per layer
   3. all in one layer (as tar file)
   
   As today to keep things simpler, I would opt for option 3 



-- 
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


Re: [PR] feat: use contaier image as kamelet repository [camel-k]

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

   bunp


-- 
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] christophd commented on pull request #4350: feat: use contaier image as kamelet repository

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

   Awesome stuff!


-- 
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] lburgazzoli commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   Giving that, I believe the IntegrationPlatform reconcile loop should be in charge of resolving the container image & so on. 
   
   One question though, is how to deal with errors: should the integration platform have a related repository condition set to `false` and the Ready condition set to `false` ?



-- 
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] johnpoth commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > > > > Looks great! As Antonin said, eager to see everything in action!
   > > > > I also think it would be cool if you could pull the image via docker/podman:
   > > > > ```
   > > > > docker pull lburgazzoli/camel-kamelets:latest
   > > > > podman pull lburgazzoli/camel-kamelets:latest
   > > > > ```
   > > > > 
   > > > > 
   > > > >     
   > > > >       
   > > > >     
   > > > > 
   > > > >       
   > > > >     
   > > > > 
   > > > >     
   > > > >   
   > > > > Right now I'm getting
   > > > > ```
   > > > > Error response from daemon: missing signature key
   > > > > ```
   > > > > 
   > > > > 
   > > > >     
   > > > >       
   > > > >     
   > > > > 
   > > > >       
   > > > >     
   > > > > 
   > > > >     
   > > > >   
   > > > > I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   > > > 
   > > > 
   > > > As today, I've blindly followed the `oras` tutorial so it could be possible that what they create is not compatible with containers tools. If you have any time to digg into the issue, I would really appreciate it.
   > > 
   > > 
   > > For sure!
   > > IIRC that's one of the reasons I didn't use ORAS and went for spectrum and Google's go-containerregistry ...
   > > I think it boils down to the list of [registries](https://oras.land/docs/CLI/reference_types#registry-support) that support the ORAS spec is pretty limited. That being said there might be a workaround ...
   > 
   > I think I did some exploration with go-containerregistry in the past, but I have not seen an option to define a custom media type or annotations for the layers, do you know if that's possible ?
   
   Yes it does!
   BUT setting a custom media type comes at the cost of the image being rejected in most registries or clients. This is why I didn't set the custom media type when working on local dependencies. Although very appealing it wasn't worth the interoperability drawbacks IMO.  This was ~1 years ago so maybe things have changed but looking at the list of supported registries it doesn't look like it...


-- 
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] johnpoth commented on pull request #4350: feat: use contaier image as kamelet repository

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

   Looks great! As Antonin said, eager to see everything in action!
   
    I also think it would be cool if you could pull the image via docker/podman:
   
   ```
   docker pull lburgazzoli/camel-kamelets:latest
   podman pull lburgazzoli/camel-kamelets:latest
   ```
   
   Right now I'm getting 
   
   ```
   Error response from daemon: missing signature key
   ```
   
   I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   
   As a side note, since we are already publishing the Kamelets on [Maven](https://repo1.maven.org/maven2/org/apache/camel/kamelets/camel-kamelets/3.20.3/camel-kamelets-3.20.3.jar) it might be cool to support that too if we don't already.


-- 
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 #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {

Review Comment:
   Yeah. Well, we can always clean, keeping only what we need. As this would happen on operator startup, I guess it's not a big deal.



##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   We may eventually create a `KameletRepository` type which is reconciled and take care of the sync when any change happen. However, it would be a future optimization.



-- 
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] astefanutti commented on a diff in pull request #4350: feat: use contaier image as kamelet repository

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


##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)

Review Comment:
   I wonder what's the state of seekable / addressable image that would enable to lazily pull blobs / layers when needed. I think that would be applicable to our use case, but it doesn't seem it has landed into the OCI specs yet. Or maybe the repository image / manifest could already be built in a way that would serve that need.
   



##########
pkg/kamelet/repository/oci_repository.go:
##########
@@ -0,0 +1,170 @@
+package repository
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"github.com/containers/image/docker/reference"
+	"io/ioutil"
+	"path"
+	"path/filepath"
+	"sync"
+
+	camelv1 "github.com/apache/camel-k/v2/pkg/apis/camel/v1"
+	"k8s.io/apimachinery/pkg/util/yaml"
+
+	"os"
+	"strings"
+
+	"oras.land/oras-go/v2"
+	"oras.land/oras-go/v2/content/file"
+	"oras.land/oras-go/v2/registry/remote"
+	"oras.land/oras-go/v2/registry/remote/auth"
+	"oras.land/oras-go/v2/registry/remote/retry"
+)
+
+type ociKameletRepository struct {
+	once     sync.Once
+	image    string
+	relative string
+
+	kamelets map[string]string
+	pollRoot string
+	pollErr  error
+}
+
+func newOCIKameletRepository(image string) KameletRepository {
+	imageName := image
+	relativePath := ""
+
+	items := strings.Split(image, "?")
+	if len(items) == 2 {
+		imageName = items[0]
+		relativePath = items[1]
+	}
+
+	repo := ociKameletRepository{
+		image:    imageName,
+		relative: relativePath,
+		kamelets: make(map[string]string, 0),
+	}
+
+	return &repo
+}
+
+// Enforce type
+var _ KameletRepository = &ociKameletRepository{}
+
+func (r *ociKameletRepository) List(ctx context.Context) ([]string, error) {
+
+	r.pullAll(ctx)
+
+	if r.pollErr != nil {
+		return nil, r.pollErr
+	}
+
+	kamelets := make([]string, 0, len(r.kamelets))
+	for k := range r.kamelets {
+		kamelets = append(kamelets, k)
+	}
+
+	return kamelets, r.pollErr
+}
+
+func (r *ociKameletRepository) Get(ctx context.Context, name string) (*camelv1.Kamelet, error) {
+	r.pullAll(ctx)
+
+	filePath, ok := r.kamelets[name]
+	if !ok {
+		return nil, fmt.Errorf("cannot find kamelet %s", name)
+	}
+
+	content, err := ioutil.ReadFile(filePath)
+	if err != nil {
+		return nil, err
+	}
+
+	if strings.HasSuffix(filePath, ".yaml") || strings.HasSuffix(filePath, ".yml") {
+		content, err = yaml.ToJSON(content)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	var kamelet camelv1.Kamelet
+	if err := json.Unmarshal(content, &kamelet); err != nil {
+		return nil, err
+	}
+
+	return &kamelet, nil
+}
+
+func (r *ociKameletRepository) String() string {
+	return fmt.Sprintf("Image[%s]", r.image)
+}
+
+// Pull download an image from the given registry and copy the content to a local temporary folder.
+func (r *ociKameletRepository) pull(ctx context.Context, image string) (string, error) {
+	repo, err := reference.Parse(image)
+	if err != nil {
+		return "", err
+	}
+
+	nt, ok := repo.(reference.NamedTagged)
+	if !ok {
+		return "", fmt.Errorf("unable to determine image name and/or tag from %s", image)
+	}
+
+	or, err := remote.NewRepository(nt.Name())
+	if err != nil {
+		return "", err
+	}
+
+	or.Client = &auth.Client{
+		Client: retry.DefaultClient,
+		Cache:  auth.DefaultCache,
+	}
+
+	f, err := os.MkdirTemp("", "camel-")
+	if err != nil {
+		return "", err
+	}
+
+	store, err := file.New(f)
+	if err != nil {
+		return "", err
+	}
+
+	if _, err = oras.Copy(ctx, or, nt.Tag(), store, nt.Tag(), oras.DefaultCopyOptions); err != nil {
+		return "", err
+	}
+
+	return f, nil
+}
+
+func (r *ociKameletRepository) pullAll(ctx context.Context) {
+	r.once.Do(func() {

Review Comment:
   We could imagine to support different image pull policies, in a similar way Kubernetes does.
   
   A less advanced alternative would be to rely on a rate limiter to invalidate the cached data, e.g.:
   
   ```go
   rateLimiter = rate.NewLimiter(rate.Every(period), 1)
   
   if !rateLimiter.Allow() {
     return
   }
   
   ...
   ```



-- 
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] lburgazzoli commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > > 
   > > I think I did some exploration with go-containerregistry in the past, but I have not seen an option to define a custom media type or annotations for the layers, do you know if that's possible ?
   > 
   > Yes it does! BUT setting a custom media type comes at the cost of the image being rejected in most registries or clients. This is why I didn't set the custom media type when working on local dependencies. Although very appealing it wasn't worth the interoperability drawbacks IMO. This was ~1 years ago so maybe things have changed but looking at the list of supported registries it doesn't look like it...
   
   Yeah, that is essentially also the conclusion in https://github.com/apache/camel-k/issues/2732, but I had hope for things to have become better, but looks like it is not. The alternative would be to use some annotations to help distinguish between the type of layers but, given the spec from the ORAS Artifacts have been merged to the OpenContainers specs I had an hope to get things more widely supported :) 
   
   


-- 
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] lburgazzoli commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > > > Looks great! As Antonin said, eager to see everything in action!
   > > > I also think it would be cool if you could pull the image via docker/podman:
   > > > ```
   > > > docker pull lburgazzoli/camel-kamelets:latest
   > > > podman pull lburgazzoli/camel-kamelets:latest
   > > > ```
   > > > 
   > > > 
   > > >     
   > > >       
   > > >     
   > > > 
   > > >       
   > > >     
   > > > 
   > > >     
   > > >   
   > > > Right now I'm getting
   > > > ```
   > > > Error response from daemon: missing signature key
   > > > ```
   > > > 
   > > > 
   > > >     
   > > >       
   > > >     
   > > > 
   > > >       
   > > >     
   > > > 
   > > >     
   > > >   
   > > > I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   > > 
   > > 
   > > As today, I've blindly followed the `oras` tutorial so it could be possible that what they create is not compatible with containers tools. If you have any time to digg into the issue, I would really appreciate it.
   > 
   > For sure!
   > 
   > IIRC that's one of the reasons I didn't use ORAS and went for spectrum and Google's go-containerregistry ...
   > 
   > I think it boils down to the list of [registries](https://oras.land/docs/CLI/reference_types#registry-support) that support the ORAS spec is pretty limited. That being said there might be a workaround ...
   
   I think I did some exploration with go-containerregistry in the past, but I have not seen an option to define a custom media type or annotations for the layers, do you know if that's possible ? 
   


-- 
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


Re: [PR] feat: use contaier image as kamelet repository [camel-k]

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

   This PR has been automatically marked as stale due to 90 days of inactivity.
   It will be closed if no further activity occurs within 15 days.
   If you think that’s incorrect or the issue should never stale, please simply write any comment.
   Thanks for your contributions!


-- 
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] johnpoth commented on pull request #4350: feat: use contaier image as kamelet repository

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

   > > Looks great! As Antonin said, eager to see everything in action!
   > > I also think it would be cool if you could pull the image via docker/podman:
   > > ```
   > > docker pull lburgazzoli/camel-kamelets:latest
   > > podman pull lburgazzoli/camel-kamelets:latest
   > > ```
   > > 
   > > 
   > >     
   > >       
   > >     
   > > 
   > >       
   > >     
   > > 
   > >     
   > >   
   > > Right now I'm getting
   > > ```
   > > Error response from daemon: missing signature key
   > > ```
   > > 
   > > 
   > >     
   > >       
   > >     
   > > 
   > >       
   > >     
   > > 
   > >     
   > >   
   > > I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   > 
   > As today, I've blindly followed the `oras` tutorial so it could be possible that what they create is not compatible with containers tools. If you have any time to digg into the issue, I would really appreciate it.
   
   For sure! 
   
   IIRC that's one of the reasons I didn't use ORAS and went for spectrum and Google's go-containerregistry ...
   
   I think it boils down to the list of [registries](https://oras.land/docs/CLI/reference_types#registry-support) that support the ORAS spec is pretty limited. That being said there might be a workaround ...


-- 
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 pull request #4350: feat: use contaier image as kamelet repository

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

   > Looks great! As Antonin said, eager to see everything in action!
   > 
   > I also think it would be cool if you could pull the image via docker/podman:
   > 
   > ```
   > docker pull lburgazzoli/camel-kamelets:latest
   > podman pull lburgazzoli/camel-kamelets:latest
   > ```
   > 
   > Right now I'm getting
   > 
   > ```
   > Error response from daemon: missing signature key
   > ```
   > 
   > I think I stumbled this when working on custom built images and it might be the way the Manifest was built.
   > 
   > As a side note, since we are already publishing the Kamelets on [Maven](https://repo1.maven.org/maven2/org/apache/camel/kamelets/camel-kamelets/3.20.3/camel-kamelets-3.20.3.jar) it might be cool to support that too if we don't already.
   
   I started working some time ago on pulling the Kamelets from maven (see #3919). I stopped the development because it was unclear to me the way we wanted to support the bundling. If the effort is not that much, we may think to support several strategies as you suggest, I'd be +1 on that.


-- 
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] lburgazzoli commented on pull request #4350: feat: use contaier image as kamelet repository

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

   
   > 
   >     1. is about authentication to the registry. If we implement this, we will have to probably deal with it as well (but I am pretty sure it's something you are already taking into account).
   > 
   
   Yes this has intentional been skipped as some other changes may be required, so I didn't want to add to many stuffs before we agree it this is a good thing to do and how to move forward 
   


-- 
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


Re: [PR] feat: use contaier image as kamelet repository [camel-k]

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

   This PR has been automatically marked as stale due to 90 days of inactivity.
   It will be closed if no further activity occurs within 15 days.
   If you think that’s incorrect or the issue should never stale, please simply write any comment.
   Thanks for your contributions!


-- 
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