Move transfer and unpack packages

Packages related to transfer and unpacking provide core interfaces which
use other core interfaces and part of common functionality.

Signed-off-by: Derek McGowan <derek@mcg.dev>
This commit is contained in:
Derek McGowan
2024-02-07 22:40:15 -08:00
parent f5ed7b84e9
commit f46aea6187
35 changed files with 62 additions and 62 deletions

View File

@@ -0,0 +1,168 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package archive
import (
"context"
"io"
"github.com/containerd/typeurl/v2"
v1 "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/containerd/containerd/v2/api/types"
transfertypes "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/images/archive"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/containerd/v2/core/transfer/plugins"
tstreaming "github.com/containerd/containerd/v2/core/transfer/streaming"
"github.com/containerd/log"
"github.com/containerd/platforms"
)
func init() {
// TODO: Move this to separate package?
plugins.Register(&transfertypes.ImageExportStream{}, &ImageExportStream{})
plugins.Register(&transfertypes.ImageImportStream{}, &ImageImportStream{})
}
type ExportOpt func(*ImageExportStream)
func WithPlatform(p v1.Platform) ExportOpt {
return func(s *ImageExportStream) {
s.platforms = append(s.platforms, p)
}
}
func WithAllPlatforms(s *ImageExportStream) {
s.allPlatforms = true
}
func WithSkipCompatibilityManifest(s *ImageExportStream) {
s.skipCompatibilityManifest = true
}
func WithSkipNonDistributableBlobs(s *ImageExportStream) {
s.skipNonDistributable = true
}
// NewImageExportStream returns an image exporter via tar stream
func NewImageExportStream(stream io.WriteCloser, mediaType string, opts ...ExportOpt) *ImageExportStream {
s := &ImageExportStream{
stream: stream,
mediaType: mediaType,
}
for _, opt := range opts {
opt(s)
}
return s
}
type ImageExportStream struct {
stream io.WriteCloser
mediaType string
platforms []v1.Platform
allPlatforms bool
skipCompatibilityManifest bool
skipNonDistributable bool
}
func (iis *ImageExportStream) ExportStream(context.Context) (io.WriteCloser, string, error) {
return iis.stream, iis.mediaType, nil
}
func (iis *ImageExportStream) Export(ctx context.Context, cs content.Store, imgs []images.Image) error {
opts := []archive.ExportOpt{
archive.WithImages(imgs),
}
if len(iis.platforms) > 0 {
opts = append(opts, archive.WithPlatform(platforms.Ordered(iis.platforms...)))
} else {
opts = append(opts, archive.WithPlatform(platforms.DefaultStrict()))
}
if iis.allPlatforms {
opts = append(opts, archive.WithAllPlatforms())
}
if iis.skipCompatibilityManifest {
opts = append(opts, archive.WithSkipDockerManifest())
}
if iis.skipNonDistributable {
opts = append(opts, archive.WithSkipNonDistributableBlobs())
}
return archive.Export(ctx, cs, iis.stream, opts...)
}
func (iis *ImageExportStream) MarshalAny(ctx context.Context, sm streaming.StreamCreator) (typeurl.Any, error) {
sid := tstreaming.GenerateID("export")
stream, err := sm.Create(ctx, sid)
if err != nil {
return nil, err
}
// Receive stream and copy to writer
go func() {
if _, err := io.Copy(iis.stream, tstreaming.ReceiveStream(ctx, stream)); err != nil {
log.G(ctx).WithError(err).WithField("streamid", sid).Errorf("error copying stream")
}
iis.stream.Close()
}()
var specified []*types.Platform
for _, p := range iis.platforms {
specified = append(specified, &types.Platform{
OS: p.OS,
Architecture: p.Architecture,
Variant: p.Variant,
})
}
s := &transfertypes.ImageExportStream{
Stream: sid,
MediaType: iis.mediaType,
Platforms: specified,
AllPlatforms: iis.allPlatforms,
SkipCompatibilityManifest: iis.skipCompatibilityManifest,
SkipNonDistributable: iis.skipNonDistributable,
}
return typeurl.MarshalAny(s)
}
func (iis *ImageExportStream) UnmarshalAny(ctx context.Context, sm streaming.StreamGetter, anyType typeurl.Any) error {
var s transfertypes.ImageExportStream
if err := typeurl.UnmarshalTo(anyType, &s); err != nil {
return err
}
stream, err := sm.Get(ctx, s.Stream)
if err != nil {
log.G(ctx).WithError(err).WithField("stream", s.Stream).Debug("failed to get export stream")
return err
}
specified := types.OCIPlatformFromProto(s.Platforms)
iis.stream = tstreaming.WriteByteStream(ctx, stream)
iis.mediaType = s.MediaType
iis.platforms = specified
iis.allPlatforms = s.AllPlatforms
iis.skipCompatibilityManifest = s.SkipCompatibilityManifest
iis.skipNonDistributable = s.SkipNonDistributable
return nil
}

View File

@@ -0,0 +1,104 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package archive
import (
"context"
"io"
"github.com/containerd/typeurl/v2"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
transferapi "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images/archive"
"github.com/containerd/containerd/v2/core/streaming"
tstreaming "github.com/containerd/containerd/v2/core/transfer/streaming"
"github.com/containerd/log"
)
type ImportOpt func(*ImageImportStream)
func WithForceCompression(s *ImageImportStream) {
s.forceCompress = true
}
// NewImageImportStream returns a image importer via tar stream
func NewImageImportStream(stream io.Reader, mediaType string, opts ...ImportOpt) *ImageImportStream {
s := &ImageImportStream{
stream: stream,
mediaType: mediaType,
}
for _, opt := range opts {
opt(s)
}
return s
}
type ImageImportStream struct {
stream io.Reader
mediaType string
forceCompress bool
}
func (iis *ImageImportStream) ImportStream(context.Context) (io.Reader, string, error) {
return iis.stream, iis.mediaType, nil
}
func (iis *ImageImportStream) Import(ctx context.Context, store content.Store) (ocispec.Descriptor, error) {
var opts []archive.ImportOpt
if iis.forceCompress {
opts = append(opts, archive.WithImportCompression())
}
return archive.ImportIndex(ctx, store, iis.stream, opts...)
}
func (iis *ImageImportStream) MarshalAny(ctx context.Context, sm streaming.StreamCreator) (typeurl.Any, error) {
sid := tstreaming.GenerateID("import")
stream, err := sm.Create(ctx, sid)
if err != nil {
return nil, err
}
tstreaming.SendStream(ctx, iis.stream, stream)
s := &transferapi.ImageImportStream{
Stream: sid,
MediaType: iis.mediaType,
ForceCompress: iis.forceCompress,
}
return typeurl.MarshalAny(s)
}
func (iis *ImageImportStream) UnmarshalAny(ctx context.Context, sm streaming.StreamGetter, anyType typeurl.Any) error {
var s transferapi.ImageImportStream
if err := typeurl.UnmarshalTo(anyType, &s); err != nil {
return err
}
stream, err := sm.Get(ctx, s.Stream)
if err != nil {
log.G(ctx).WithError(err).WithField("stream", s.Stream).Debug("failed to get import stream")
return err
}
iis.stream = tstreaming.ReceiveStream(ctx, stream)
iis.mediaType = s.MediaType
iis.forceCompress = s.ForceCompress
return nil
}

View File

@@ -0,0 +1,466 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package image
import (
"context"
"fmt"
"github.com/containerd/typeurl/v2"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/containerd/containerd/v2/api/types"
transfertypes "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/images/archive"
"github.com/containerd/containerd/v2/core/remotes"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/transfer/plugins"
"github.com/containerd/errdefs"
"github.com/containerd/platforms"
)
func init() {
// TODO: Move this to separate package?
plugins.Register(&transfertypes.ImageStore{}, &Store{}) // TODO: Rename ImageStoreDestination
}
type Store struct {
imageName string
imageLabels map[string]string
platforms []ocispec.Platform
allMetadata bool
labelMap func(ocispec.Descriptor) []string
manifestLimit int
// extraReferences are used to store or lookup multiple references
extraReferences []Reference
unpacks []transfer.UnpackConfiguration
}
// Reference is used to create or find a reference for an image
type Reference struct {
Name string
// IsPrefix determines whether the Name should be considered
// a prefix (without tag or digest).
// For lookup, this may allow matching multiple tags.
// For store, this must have a tag or digest added.
IsPrefix bool
// AllowOverwrite allows overwriting or ignoring the name if
// another reference is provided (such as through an annotation).
// Only used if IsPrefix is true.
AllowOverwrite bool
// AddDigest adds the manifest digest to the reference.
// For lookup, this allows matching tags with any digest.
// For store, this allows adding the digest to the name.
// Only used if IsPrefix is true.
AddDigest bool
// SkipNamedDigest only considers digest references which do not
// have a non-digested named reference.
// For lookup, this will deduplicate digest references when there is a named match.
// For store, this only adds this digest reference when there is no matching full
// name reference from the prefix.
// Only used if IsPrefix is true.
SkipNamedDigest bool
}
// StoreOpt defines options when configuring an image store source or destination
type StoreOpt func(*Store)
// WithImageLabels are the image labels to apply to a new image
func WithImageLabels(labels map[string]string) StoreOpt {
return func(s *Store) {
s.imageLabels = labels
}
}
// WithPlatforms specifies which platforms to fetch content for
func WithPlatforms(p ...ocispec.Platform) StoreOpt {
return func(s *Store) {
s.platforms = append(s.platforms, p...)
}
}
// WithManifestLimit defines the max number of manifests to fetch
func WithManifestLimit(limit int) StoreOpt {
return func(s *Store) {
s.manifestLimit = limit
}
}
func WithAllMetadata(s *Store) {
s.allMetadata = true
}
// WithNamedPrefix uses a named prefix to references images which only have a tag name
// reference in the annotation or check full references annotations against. Images
// with no reference resolved from matching annotations will not be stored.
// - name: image name prefix to append a tag to or check full name references with
// - allowOverwrite: allows the tag to be overwritten by full name reference inside
// the image which does not have name as the prefix
func WithNamedPrefix(name string, allowOverwrite bool) StoreOpt {
ref := Reference{
Name: name,
IsPrefix: true,
AllowOverwrite: allowOverwrite,
}
return func(s *Store) {
s.extraReferences = append(s.extraReferences, ref)
}
}
// WithDigestRef uses a named prefix to references images which only have a tag name
// reference in the annotation or check full references annotations against and
// additionally may add a digest reference. Images with no references resolved
// from matching annotations may be stored by digest.
// - name: image name prefix to append a tag to or check full name references with
// - allowOverwrite: allows the tag to be overwritten by full name reference inside
// the image which does not have name as the prefix
// - skipNamed: is set if no digest reference should be created if a named reference
// is successfully resolved from the annotations.
func WithDigestRef(name string, allowOverwrite bool, skipNamed bool) StoreOpt {
ref := Reference{
Name: name,
IsPrefix: true,
AllowOverwrite: allowOverwrite,
AddDigest: true,
SkipNamedDigest: skipNamed,
}
return func(s *Store) {
s.extraReferences = append(s.extraReferences, ref)
}
}
func WithExtraReference(name string) StoreOpt {
ref := Reference{
Name: name,
}
return func(s *Store) {
s.extraReferences = append(s.extraReferences, ref)
}
}
// WithUnpack specifies a platform to unpack for and an optional snapshotter to use
func WithUnpack(p ocispec.Platform, snapshotter string) StoreOpt {
return func(s *Store) {
s.unpacks = append(s.unpacks, transfer.UnpackConfiguration{
Platform: p,
Snapshotter: snapshotter,
})
}
}
// NewStore creates a new image store source or Destination
func NewStore(image string, opts ...StoreOpt) *Store {
s := &Store{
imageName: image,
}
for _, opt := range opts {
opt(s)
}
return s
}
func (is *Store) String() string {
return fmt.Sprintf("Local Image Store (%s)", is.imageName)
}
func (is *Store) ImageFilter(h images.HandlerFunc, cs content.Store) images.HandlerFunc {
var p platforms.MatchComparer
if len(is.platforms) == 0 {
p = platforms.All
} else {
p = platforms.Ordered(is.platforms...)
}
h = images.SetChildrenMappedLabels(cs, h, is.labelMap)
if is.allMetadata {
// Filter manifests by platforms but allow to handle manifest
// and configuration for not-target platforms
h = remotes.FilterManifestByPlatformHandler(h, p)
} else {
// Filter children by platforms if specified.
h = images.FilterPlatforms(h, p)
}
// Sort and limit manifests if a finite number is needed
if is.manifestLimit > 0 {
h = images.LimitManifests(h, p, is.manifestLimit)
}
return h
}
func (is *Store) Store(ctx context.Context, desc ocispec.Descriptor, store images.Store) ([]images.Image, error) {
var imgs []images.Image
// If import ref type, store references from annotation or prefix
if refSource, ok := desc.Annotations["io.containerd.import.ref-source"]; ok {
switch refSource {
case "annotation":
for _, ref := range is.extraReferences {
// Only use prefix references for annotation matching
if !ref.IsPrefix {
continue
}
var nameT func(string) string
if ref.AllowOverwrite {
nameT = archive.AddRefPrefix(ref.Name)
} else {
nameT = archive.FilterRefPrefix(ref.Name)
}
name := imageName(desc.Annotations, nameT)
if name == "" {
// If digested, add digest reference
if ref.AddDigest {
imgs = append(imgs, images.Image{
Name: fmt.Sprintf("%s@%s", ref.Name, desc.Digest),
Target: desc,
Labels: is.imageLabels,
})
}
continue
}
imgs = append(imgs, images.Image{
Name: name,
Target: desc,
Labels: is.imageLabels,
})
// If a named reference was found and SkipNamedDigest is true, do
// not use this reference
if ref.AddDigest && !ref.SkipNamedDigest {
imgs = append(imgs, images.Image{
Name: fmt.Sprintf("%s@%s", ref.Name, desc.Digest),
Target: desc,
Labels: is.imageLabels,
})
}
}
default:
return nil, fmt.Errorf("ref source not supported: %w", errdefs.ErrInvalidArgument)
}
delete(desc.Annotations, "io.containerd.import.ref-source")
} else {
if is.imageName != "" {
imgs = append(imgs, images.Image{
Name: is.imageName,
Target: desc,
Labels: is.imageLabels,
})
}
// If extra references, store all complete references (skip prefixes)
for _, ref := range is.extraReferences {
if ref.IsPrefix {
continue
}
name := ref.Name
if ref.AddDigest {
name = fmt.Sprintf("%s@%s", name, desc.Digest)
}
imgs = append(imgs, images.Image{
Name: name,
Target: desc,
Labels: is.imageLabels,
})
}
}
if len(imgs) == 0 {
return nil, fmt.Errorf("no image name found: %w", errdefs.ErrNotFound)
}
for i := 0; i < len(imgs); {
if created, err := store.Create(ctx, imgs[i]); err != nil {
if !errdefs.IsAlreadyExists(err) {
return nil, err
}
updated, err := store.Update(ctx, imgs[i])
if err != nil {
// if image was removed, try create again
if errdefs.IsNotFound(err) {
// Keep trying same image
continue
}
return nil, err
}
imgs[i] = updated
} else {
imgs[i] = created
}
i++
}
return imgs, nil
}
func (is *Store) Get(ctx context.Context, store images.Store) (images.Image, error) {
return store.Get(ctx, is.imageName)
}
func (is *Store) Lookup(ctx context.Context, store images.Store) ([]images.Image, error) {
var imgs []images.Image
if is.imageName != "" {
img, err := store.Get(ctx, is.imageName)
if err != nil {
return nil, err
}
imgs = append(imgs, img)
}
for _, ref := range is.extraReferences {
if ref.IsPrefix {
return nil, fmt.Errorf("prefix lookup on export not implemented: %w", errdefs.ErrNotImplemented)
}
img, err := store.Get(ctx, ref.Name)
if err != nil {
return nil, err
}
imgs = append(imgs, img)
}
return imgs, nil
}
func (is *Store) UnpackPlatforms() []transfer.UnpackConfiguration {
unpacks := make([]transfer.UnpackConfiguration, len(is.unpacks))
for i, uc := range is.unpacks {
unpacks[i].Snapshotter = uc.Snapshotter
unpacks[i].Platform = uc.Platform
}
return unpacks
}
func (is *Store) MarshalAny(context.Context, streaming.StreamCreator) (typeurl.Any, error) {
s := &transfertypes.ImageStore{
Name: is.imageName,
Labels: is.imageLabels,
ManifestLimit: uint32(is.manifestLimit),
AllMetadata: is.allMetadata,
Platforms: types.OCIPlatformToProto(is.platforms),
ExtraReferences: referencesToProto(is.extraReferences),
Unpacks: unpackToProto(is.unpacks),
}
return typeurl.MarshalAny(s)
}
func (is *Store) UnmarshalAny(ctx context.Context, sm streaming.StreamGetter, a typeurl.Any) error {
var s transfertypes.ImageStore
if err := typeurl.UnmarshalTo(a, &s); err != nil {
return err
}
is.imageName = s.Name
is.imageLabels = s.Labels
is.manifestLimit = int(s.ManifestLimit)
is.allMetadata = s.AllMetadata
is.platforms = types.OCIPlatformFromProto(s.Platforms)
is.extraReferences = referencesFromProto(s.ExtraReferences)
is.unpacks = unpackFromProto(s.Unpacks)
return nil
}
func referencesToProto(references []Reference) []*transfertypes.ImageReference {
ir := make([]*transfertypes.ImageReference, len(references))
for i := range references {
r := transfertypes.ImageReference{
Name: references[i].Name,
IsPrefix: references[i].IsPrefix,
AllowOverwrite: references[i].AllowOverwrite,
AddDigest: references[i].AddDigest,
SkipNamedDigest: references[i].SkipNamedDigest,
}
ir[i] = &r
}
return ir
}
func referencesFromProto(references []*transfertypes.ImageReference) []Reference {
or := make([]Reference, len(references))
for i := range references {
or[i].Name = references[i].Name
or[i].IsPrefix = references[i].IsPrefix
or[i].AllowOverwrite = references[i].AllowOverwrite
or[i].AddDigest = references[i].AddDigest
or[i].SkipNamedDigest = references[i].SkipNamedDigest
}
return or
}
func unpackToProto(uc []transfer.UnpackConfiguration) []*transfertypes.UnpackConfiguration {
auc := make([]*transfertypes.UnpackConfiguration, len(uc))
for i := range uc {
p := types.Platform{
OS: uc[i].Platform.OS,
Architecture: uc[i].Platform.Architecture,
Variant: uc[i].Platform.Variant,
}
auc[i] = &transfertypes.UnpackConfiguration{
Platform: &p,
Snapshotter: uc[i].Snapshotter,
}
}
return auc
}
func unpackFromProto(auc []*transfertypes.UnpackConfiguration) []transfer.UnpackConfiguration {
uc := make([]transfer.UnpackConfiguration, len(auc))
for i := range auc {
uc[i].Snapshotter = auc[i].Snapshotter
if auc[i].Platform != nil {
uc[i].Platform.OS = auc[i].Platform.OS
uc[i].Platform.Architecture = auc[i].Platform.Architecture
uc[i].Platform.Variant = auc[i].Platform.Variant
}
}
return uc
}
func imageName(annotations map[string]string, cleanup func(string) string) string {
name := annotations[images.AnnotationImageName]
if name != "" {
if cleanup != nil {
// containerd reference name should be full reference and not
// modified, if it is incomplete or does not match a specified
// prefix, do not use the reference
if cleanName := cleanup(name); cleanName != name {
name = ""
}
}
return name
}
name = annotations[ocispec.AnnotationRefName]
if name != "" {
if cleanup != nil {
name = cleanup(name)
}
}
return name
}

View File

@@ -0,0 +1,418 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package image
import (
"context"
"errors"
"sort"
"sync"
"testing"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/errdefs"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
func TestStore(t *testing.T) {
for _, testCase := range []struct {
Name string
ImageStore *Store
// Annotations are the different references annotations to run the test with,
// the possible values:
// - "OCI": Uses the OCI defined annotation "org.opencontainers.image.ref.name"
// This annotation may be a full reference or tag only
// - "containerd": Uses the containerd defined annotation "io.containerd.image.name"
// This annotation is always a full reference as used by containerd
// - "Annotation": Sets the annotation flag but does not set a reference annotation
// Use this case to test the default where no reference is provided
// - "NoAnnotation": Does not set the annotation flag
// Use this case to test storing of the index images by reference
Annotations []string
ImageName string
Images []string
Err error
}{
{
Name: "Prefix",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/image",
IsPrefix: true,
},
},
},
Annotations: []string{"OCI", "containerd"},
ImageName: "registry.test/image:latest",
Images: []string{"registry.test/image:latest"},
},
{
Name: "Overwrite",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "placeholder",
IsPrefix: true,
AllowOverwrite: true,
},
},
},
Annotations: []string{"OCI", "containerd"},
ImageName: "registry.test/image:latest",
Images: []string{"registry.test/image:latest"},
},
{
Name: "TagOnly",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/image",
IsPrefix: true,
},
},
},
Annotations: []string{"OCI"},
ImageName: "latest",
Images: []string{"registry.test/image:latest"},
},
{
Name: "AddDigest",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/base",
IsPrefix: true,
AddDigest: true,
},
},
},
Annotations: []string{"Annotation"},
Images: []string{"registry.test/base@"},
},
{
Name: "NameAndDigest",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/base",
IsPrefix: true,
AddDigest: true,
},
},
},
Annotations: []string{"OCI"},
ImageName: "latest",
Images: []string{"registry.test/base:latest", "registry.test/base@"},
},
{
Name: "NameSkipDigest",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/base",
IsPrefix: true,
AddDigest: true,
SkipNamedDigest: true,
},
},
},
Annotations: []string{"OCI"},
ImageName: "latest",
Images: []string{"registry.test/base:latest"},
},
{
Name: "OverwriteNameDigest",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "base-name",
IsPrefix: true,
AllowOverwrite: true,
AddDigest: true,
},
},
},
Annotations: []string{"OCI", "containerd"},
ImageName: "registry.test/base:latest",
Images: []string{"registry.test/base:latest", "base-name@"},
},
{
Name: "OverwriteNameSkipDigest",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "base-name",
IsPrefix: true,
AllowOverwrite: true,
AddDigest: true,
SkipNamedDigest: true,
},
},
},
Annotations: []string{"OCI", "containerd"},
ImageName: "registry.test/base:latest",
Images: []string{"registry.test/base:latest"},
},
{
Name: "ReferenceNotFound",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.test/image",
IsPrefix: true,
},
},
},
Annotations: []string{"OCI", "containerd"},
ImageName: "registry.test/base:latest",
Err: errdefs.ErrNotFound,
},
{
Name: "NoReference",
ImageStore: &Store{},
Annotations: []string{"Annotation", "NoAnnotation"},
Err: errdefs.ErrNotFound,
},
{
Name: "ImageName",
ImageStore: &Store{
imageName: "registry.test/index:latest",
},
Annotations: []string{"NoAnnotation"},
Images: []string{"registry.test/index:latest"},
},
} {
testCase := testCase
for _, a := range testCase.Annotations {
name := testCase.Name + "_" + a
dgst := digest.Canonical.FromString(name)
desc := ocispec.Descriptor{
Digest: dgst,
Annotations: map[string]string{},
}
expected := make([]string, len(testCase.Images))
for i, img := range testCase.Images {
if img[len(img)-1] == '@' {
img = img + dgst.String()
}
expected[i] = img
}
switch a {
case "containerd":
desc.Annotations["io.containerd.import.ref-source"] = "annotation"
desc.Annotations[images.AnnotationImageName] = testCase.ImageName
case "OCI":
desc.Annotations["io.containerd.import.ref-source"] = "annotation"
desc.Annotations[ocispec.AnnotationRefName] = testCase.ImageName
case "Annotation":
desc.Annotations["io.containerd.import.ref-source"] = "annotation"
}
t.Run(name, func(t *testing.T) {
imgs, err := testCase.ImageStore.Store(context.Background(), desc, newSimpleImageStore())
if err != nil {
if testCase.Err == nil {
t.Fatal(err)
}
if !errors.Is(err, testCase.Err) {
t.Fatalf("unexpected error %v: expeceted %v", err, testCase.Err)
}
return
} else if testCase.Err != nil {
t.Fatalf("succeeded but expected error: %v", testCase.Err)
}
if len(imgs) != len(expected) {
t.Fatalf("mismatched array length\nexpected:\n\t%v\nactual\n\t%v", expected, imgs)
}
for i, name := range expected {
if imgs[i].Name != name {
t.Fatalf("wrong image name %q, expected %q", imgs[i].Name, name)
}
if imgs[i].Target.Digest != dgst {
t.Fatalf("wrong image digest %s, expected %s", imgs[i].Target.Digest, dgst)
}
}
})
}
}
}
func TestLookup(t *testing.T) {
ctx := context.Background()
is := newSimpleImageStore()
for _, name := range []string{
"registry.io/test1:latest",
"registry.io/test1:v1",
} {
is.Create(ctx, images.Image{
Name: name,
})
}
for _, testCase := range []struct {
Name string
ImageStore *Store
Expected []string
Err error
}{
{
Name: "SingleImage",
ImageStore: &Store{
imageName: "registry.io/test1:latest",
},
Expected: []string{"registry.io/test1:latest"},
},
{
Name: "MultipleReferences",
ImageStore: &Store{
imageName: "registry.io/test1:latest",
extraReferences: []Reference{
{
Name: "registry.io/test1:v1",
},
},
},
Expected: []string{"registry.io/test1:latest", "registry.io/test1:v1"},
},
{
Name: "OnlyReferences",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.io/test1:latest",
},
{
Name: "registry.io/test1:v1",
},
},
},
Expected: []string{"registry.io/test1:latest", "registry.io/test1:v1"},
},
{
Name: "UnsupportedPrefix",
ImageStore: &Store{
extraReferences: []Reference{
{
Name: "registry.io/test1:latest",
IsPrefix: true,
},
},
},
Err: errdefs.ErrNotImplemented,
},
} {
t.Run(testCase.Name, func(t *testing.T) {
images, err := testCase.ImageStore.Lookup(ctx, is)
if err != nil {
if !errors.Is(err, testCase.Err) {
t.Errorf("unexpected error %v, expected %v", err, testCase.Err)
}
return
} else if testCase.Err != nil {
t.Fatal("expected error")
}
imageNames := make([]string, len(images))
for i, img := range images {
imageNames[i] = img.Name
}
sort.Strings(imageNames)
sort.Strings(testCase.Expected)
if len(images) != len(testCase.Expected) {
t.Fatalf("unexpected images:\n\t%v\nexpected:\n\t%v", imageNames, testCase.Expected)
}
for i := range imageNames {
if imageNames[i] != testCase.Expected[i] {
t.Fatalf("unexpected images:\n\t%v\nexpected:\n\t%v", imageNames, testCase.Expected)
}
}
})
}
}
// simpleImageStore is for testing images in memory,
// no filter support
type simpleImageStore struct {
l sync.Mutex
images map[string]images.Image
}
func newSimpleImageStore() images.Store {
return &simpleImageStore{
images: make(map[string]images.Image),
}
}
func (is *simpleImageStore) Get(ctx context.Context, name string) (images.Image, error) {
is.l.Lock()
defer is.l.Unlock()
img, ok := is.images[name]
if !ok {
return images.Image{}, errdefs.ErrNotFound
}
return img, nil
}
func (is *simpleImageStore) List(ctx context.Context, filters ...string) ([]images.Image, error) {
is.l.Lock()
defer is.l.Unlock()
var imgs []images.Image
// filters not supported, return all
for _, img := range is.images {
imgs = append(imgs, img)
}
return imgs, nil
}
func (is *simpleImageStore) Create(ctx context.Context, image images.Image) (images.Image, error) {
is.l.Lock()
defer is.l.Unlock()
if _, ok := is.images[image.Name]; ok {
return images.Image{}, errdefs.ErrAlreadyExists
}
is.images[image.Name] = image
return image, nil
}
func (is *simpleImageStore) Update(ctx context.Context, image images.Image, fieldpaths ...string) (images.Image, error) {
is.l.Lock()
defer is.l.Unlock()
if _, ok := is.images[image.Name]; !ok {
return images.Image{}, errdefs.ErrNotFound
}
// fieldpaths no supported, update entire image
is.images[image.Name] = image
return image, nil
}
func (is *simpleImageStore) Delete(ctx context.Context, name string, opts ...images.DeleteOpt) error {
is.l.Lock()
defer is.l.Unlock()
if _, ok := is.images[name]; !ok {
return errdefs.ErrNotFound
}
delete(is.images, name)
return nil
}

View File

@@ -0,0 +1,64 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/transfer"
)
func (ts *localTransferService) exportStream(ctx context.Context, ig transfer.ImageGetter, is transfer.ImageExporter, tops *transfer.Config) error {
ctx, done, err := ts.withLease(ctx)
if err != nil {
return err
}
defer done(ctx)
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: "Exporting",
})
}
var imgs []images.Image
if il, ok := ig.(transfer.ImageLookup); ok {
imgs, err = il.Lookup(ctx, ts.images)
if err != nil {
return err
}
} else {
img, err := ig.Get(ctx, ts.images)
if err != nil {
return err
}
imgs = append(imgs, img)
}
err = is.Export(ctx, ts.content, imgs)
if err != nil {
return err
}
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: "Completed export",
})
}
return nil
}

View File

@@ -0,0 +1,170 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"encoding/json"
"fmt"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/unpack"
"github.com/containerd/errdefs"
"github.com/containerd/log"
)
func (ts *localTransferService) importStream(ctx context.Context, i transfer.ImageImporter, is transfer.ImageStorer, tops *transfer.Config) error {
ctx, done, err := ts.withLease(ctx)
if err != nil {
return err
}
defer done(ctx)
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: "Importing",
})
}
index, err := i.Import(ctx, ts.content)
if err != nil {
return err
}
var (
descriptors []ocispec.Descriptor
handler images.Handler
unpacker *unpack.Unpacker
)
// If save index, add index
descriptors = append(descriptors, index)
var handlerFunc images.HandlerFunc = func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
// Only save images at top level
if desc.Digest != index.Digest {
return images.Children(ctx, ts.content, desc)
}
p, err := content.ReadBlob(ctx, ts.content, desc)
if err != nil {
return nil, err
}
var idx ocispec.Index
if err := json.Unmarshal(p, &idx); err != nil {
return nil, err
}
for _, m := range idx.Manifests {
m.Annotations = mergeMap(m.Annotations, map[string]string{"io.containerd.import.ref-source": "annotation"})
descriptors = append(descriptors, m)
}
return idx.Manifests, nil
}
if f, ok := is.(transfer.ImageFilterer); ok {
handlerFunc = f.ImageFilter(handlerFunc, ts.content)
}
handler = images.Handlers(handlerFunc)
// First find suitable platforms to unpack into
// If image storer is also an unpacker type, i.e implemented UnpackPlatforms() func
if iu, ok := is.(transfer.ImageUnpacker); ok {
unpacks := iu.UnpackPlatforms()
if len(unpacks) > 0 {
uopts := []unpack.UnpackerOpt{}
for _, u := range unpacks {
matched, mu := getSupportedPlatform(u, ts.config.UnpackPlatforms)
if matched {
uopts = append(uopts, unpack.WithUnpackPlatform(mu))
}
}
if ts.config.DuplicationSuppressor != nil {
uopts = append(uopts, unpack.WithDuplicationSuppressor(ts.config.DuplicationSuppressor))
}
unpacker, err = unpack.NewUnpacker(ctx, ts.content, uopts...)
if err != nil {
return fmt.Errorf("unable to initialize unpacker: %w", err)
}
handler = unpacker.Unpack(handler)
}
}
if err := images.WalkNotEmpty(ctx, handler, index); err != nil {
if unpacker != nil {
// wait for unpacker to cleanup
unpacker.Wait()
}
// TODO: Handle Not Empty as a special case on the input
return err
}
if unpacker != nil {
if _, err = unpacker.Wait(); err != nil {
return err
}
}
for _, desc := range descriptors {
desc := desc
imgs, err := is.Store(ctx, desc, ts.images)
if err != nil {
if errdefs.IsNotFound(err) {
log.G(ctx).Infof("No images store for %s", desc.Digest)
continue
}
return err
}
if tops.Progress != nil {
for _, img := range imgs {
tops.Progress(transfer.Progress{
Event: "saved",
Name: img.Name,
Desc: &desc,
})
}
}
}
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: "Completed import",
})
}
return nil
}
func mergeMap(m1, m2 map[string]string) map[string]string {
merged := make(map[string]string, len(m1)+len(m2))
for k, v := range m1 {
merged[k] = v
}
for k, v := range m2 {
merged[k] = v
}
return merged
}

View File

@@ -0,0 +1,281 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"sort"
"sync"
"time"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/remotes"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/log"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
type ProgressTracker struct {
root string
transferState string
added chan jobUpdate
waitC chan struct{}
parents map[digest.Digest][]ocispec.Descriptor
parentL sync.Mutex
}
type jobState uint8
const (
jobAdded jobState = iota
jobInProgress
jobComplete
)
type jobStatus struct {
state jobState
name string
parents []string
progress int64
desc ocispec.Descriptor
}
type jobUpdate struct {
desc ocispec.Descriptor
exists bool
//children []ocispec.Descriptor
}
type ActiveJobs interface {
Status(string) (content.Status, bool)
}
type StatusTracker interface {
Active(context.Context, ...string) (ActiveJobs, error)
Check(context.Context, digest.Digest) (bool, error)
}
// NewProgressTracker tracks content download progress
func NewProgressTracker(root, transferState string) *ProgressTracker {
return &ProgressTracker{
root: root,
transferState: transferState,
added: make(chan jobUpdate, 1),
waitC: make(chan struct{}),
parents: map[digest.Digest][]ocispec.Descriptor{},
}
}
func (j *ProgressTracker) HandleProgress(ctx context.Context, pf transfer.ProgressFunc, pt StatusTracker) {
defer close(j.waitC)
// Instead of ticker, just delay
jobs := map[digest.Digest]*jobStatus{}
tc := time.NewTicker(time.Millisecond * 300)
defer tc.Stop()
update := func() {
// TODO: Filter by references
active, err := pt.Active(ctx)
if err != nil {
log.G(ctx).WithError(err).Error("failed to get statuses for progress")
}
for dgst, job := range jobs {
if job.state != jobComplete {
status, ok := active.Status(job.name)
if ok {
if status.Offset > job.progress {
pf(transfer.Progress{
Event: j.transferState,
Name: job.name,
Parents: job.parents,
Progress: status.Offset,
Total: status.Total,
Desc: &job.desc,
})
job.progress = status.Offset
job.state = jobInProgress
jobs[dgst] = job
}
} else {
ok, err := pt.Check(ctx, job.desc.Digest)
if err != nil {
log.G(ctx).WithError(err).Error("failed to get statuses for progress")
} else if ok {
pf(transfer.Progress{
Event: "complete",
Name: job.name,
Parents: job.parents,
Progress: job.desc.Size,
Total: job.desc.Size,
Desc: &job.desc,
})
}
job.state = jobComplete
jobs[dgst] = job
}
}
}
}
for {
select {
case update := <-j.added:
job, ok := jobs[update.desc.Digest]
if !ok {
// Only captures the parents defined before,
// could handle parent updates in same thread
// if there is a synchronization issue
var parents []string
j.parentL.Lock()
for _, parent := range j.parents[update.desc.Digest] {
parents = append(parents, remotes.MakeRefKey(ctx, parent))
}
j.parentL.Unlock()
if len(parents) == 0 {
parents = []string{j.root}
}
name := remotes.MakeRefKey(ctx, update.desc)
job = &jobStatus{
state: jobAdded,
name: name,
parents: parents,
desc: update.desc,
}
jobs[update.desc.Digest] = job
pf(transfer.Progress{
Event: "waiting",
Name: name,
Parents: parents,
//Digest: desc.Digest.String(),
Progress: 0,
Total: update.desc.Size,
Desc: &job.desc,
})
}
if update.exists {
pf(transfer.Progress{
Event: "already exists",
Name: remotes.MakeRefKey(ctx, update.desc),
Progress: update.desc.Size,
Total: update.desc.Size,
Desc: &job.desc,
})
job.state = jobComplete
job.progress = job.desc.Size
}
case <-tc.C:
update()
// Next timer?
case <-ctx.Done():
update()
return
}
}
}
// Add adds a descriptor to be tracked
func (j *ProgressTracker) Add(desc ocispec.Descriptor) {
if j == nil {
return
}
j.added <- jobUpdate{
desc: desc,
}
}
func (j *ProgressTracker) MarkExists(desc ocispec.Descriptor) {
if j == nil {
return
}
j.added <- jobUpdate{
desc: desc,
exists: true,
}
}
// AddChildren adds hierarchy information
func (j *ProgressTracker) AddChildren(desc ocispec.Descriptor, children []ocispec.Descriptor) {
if j == nil || len(children) == 0 {
return
}
j.parentL.Lock()
defer j.parentL.Unlock()
for _, child := range children {
j.parents[child.Digest] = append(j.parents[child.Digest], desc)
}
}
func (j *ProgressTracker) Wait() {
// timeout rather than rely on cancel
timeout := time.After(10 * time.Second)
select {
case <-timeout:
case <-j.waitC:
}
}
type contentActive struct {
active []content.Status
}
func (c *contentActive) Status(ref string) (content.Status, bool) {
idx := sort.Search(len(c.active), func(i int) bool { return c.active[i].Ref >= ref })
if idx < len(c.active) && c.active[idx].Ref == ref {
return c.active[idx], true
}
return content.Status{}, false
}
type contentStatusTracker struct {
cs content.Store
}
func NewContentStatusTracker(cs content.Store) StatusTracker {
return &contentStatusTracker{
cs: cs,
}
}
func (c *contentStatusTracker) Active(ctx context.Context, _ ...string) (ActiveJobs, error) {
active, err := c.cs.ListStatuses(ctx)
if err != nil {
log.G(ctx).WithError(err).Error("failed to list statuses for progress")
}
sort.Slice(active, func(i, j int) bool {
return active[i].Ref < active[j].Ref
})
return &contentActive{
active: active,
}, nil
}
func (c *contentStatusTracker) Check(ctx context.Context, dgst digest.Digest) (bool, error) {
_, err := c.cs.Info(ctx, dgst)
if err == nil {
return true, nil
}
return false, nil
}

295
core/transfer/local/pull.go Normal file
View File

@@ -0,0 +1,295 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"fmt"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/remotes"
"github.com/containerd/containerd/v2/core/remotes/docker"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/unpack"
"github.com/containerd/containerd/v2/defaults"
"github.com/containerd/errdefs"
"github.com/containerd/log"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/sirupsen/logrus"
)
func (ts *localTransferService) pull(ctx context.Context, ir transfer.ImageFetcher, is transfer.ImageStorer, tops *transfer.Config) error {
ctx, done, err := ts.withLease(ctx)
if err != nil {
return err
}
defer done(ctx)
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: fmt.Sprintf("Resolving from %s", ir),
})
}
name, desc, err := ir.Resolve(ctx)
if err != nil {
return fmt.Errorf("failed to resolve image: %w", err)
}
if desc.MediaType == images.MediaTypeDockerSchema1Manifest {
// Explicitly call out schema 1 as deprecated and not supported
return fmt.Errorf("schema 1 image manifests are no longer supported: %w", errdefs.ErrInvalidArgument)
}
// Verify image before pulling.
for vfName, vf := range ts.verifiers {
log := log.G(ctx).WithFields(logrus.Fields{
"name": name,
"digest": desc.Digest.String(),
"verifier": vfName,
})
log.Debug("Verifying image pull")
jdg, err := vf.VerifyImage(ctx, name, desc)
if err != nil {
log.WithError(err).Error("No judgement received from verifier")
return fmt.Errorf("blocking pull of %v with digest %v: image verifier %v returned error: %w", name, desc.Digest.String(), vfName, err)
}
log = log.WithFields(logrus.Fields{
"ok": jdg.OK,
"reason": jdg.Reason,
})
if !jdg.OK {
log.Warn("Image verifier blocked pull")
return fmt.Errorf("image verifier %s blocked pull of %v with digest %v for reason: %v", vfName, name, desc.Digest.String(), jdg.Reason)
}
log.Debug("Image verifier allowed pull")
}
// TODO: Handle already exists
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: fmt.Sprintf("Pulling from %s", ir),
})
tops.Progress(transfer.Progress{
Event: "fetching image content",
Name: name,
//Digest: img.Target.Digest.String(),
})
}
fetcher, err := ir.Fetcher(ctx, name)
if err != nil {
return fmt.Errorf("failed to get fetcher for %q: %w", name, err)
}
var (
handler images.Handler
baseHandlers []images.Handler
unpacker *unpack.Unpacker
// has a config media type bug (distribution#1622)
hasMediaTypeBug1622 bool
store = ts.content
progressTracker *ProgressTracker
)
ctx, cancel := context.WithCancel(ctx)
if tops.Progress != nil {
progressTracker = NewProgressTracker(name, "downloading") //Pass in first name as root
go progressTracker.HandleProgress(ctx, tops.Progress, NewContentStatusTracker(store))
defer progressTracker.Wait()
}
defer cancel()
// Get all the children for a descriptor
childrenHandler := images.ChildrenHandler(store)
if f, ok := is.(transfer.ImageFilterer); ok {
childrenHandler = f.ImageFilter(childrenHandler, store)
}
checkNeedsFix := images.HandlerFunc(
func(_ context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
// set to true if there is application/octet-stream media type
if desc.MediaType == docker.LegacyConfigMediaType {
hasMediaTypeBug1622 = true
}
return []ocispec.Descriptor{}, nil
},
)
appendDistSrcLabelHandler, err := docker.AppendDistributionSourceLabel(store, name)
if err != nil {
return err
}
// Set up baseHandlers from service configuration if present or create a new one
if ts.config.BaseHandlers != nil {
baseHandlers = ts.config.BaseHandlers
} else {
baseHandlers = []images.Handler{}
}
if tops.Progress != nil {
baseHandlers = append(baseHandlers, images.HandlerFunc(
func(_ context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
progressTracker.Add(desc)
return []ocispec.Descriptor{}, nil
},
))
baseChildrenHandler := childrenHandler
childrenHandler = images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) (children []ocispec.Descriptor, err error) {
children, err = baseChildrenHandler(ctx, desc)
if err != nil {
return
}
progressTracker.AddChildren(desc, children)
return
})
}
handler = images.Handlers(append(baseHandlers,
fetchHandler(store, fetcher, progressTracker),
checkNeedsFix,
childrenHandler, // List children to track hierarchy
appendDistSrcLabelHandler,
)...)
// First find suitable platforms to unpack into
// If image storer is also an unpacker type, i.e implemented UnpackPlatforms() func
if iu, ok := is.(transfer.ImageUnpacker); ok {
unpacks := iu.UnpackPlatforms()
if len(unpacks) > 0 {
uopts := []unpack.UnpackerOpt{}
// Only unpack if requested unpackconfig matches default/supported unpackconfigs
for _, u := range unpacks {
matched, mu := getSupportedPlatform(u, ts.config.UnpackPlatforms)
if matched {
uopts = append(uopts, unpack.WithUnpackPlatform(mu))
}
}
if ts.limiterD != nil {
uopts = append(uopts, unpack.WithLimiter(ts.limiterD))
}
if ts.config.DuplicationSuppressor != nil {
uopts = append(uopts, unpack.WithDuplicationSuppressor(ts.config.DuplicationSuppressor))
}
unpacker, err = unpack.NewUnpacker(ctx, ts.content, uopts...)
if err != nil {
return fmt.Errorf("unable to initialize unpacker: %w", err)
}
handler = unpacker.Unpack(handler)
}
}
if err := images.Dispatch(ctx, handler, ts.limiterD, desc); err != nil {
if unpacker != nil {
// wait for unpacker to cleanup
unpacker.Wait()
}
return err
}
// NOTE(fuweid): unpacker defers blobs download. before create image
// record in ImageService, should wait for unpacking(including blobs
// download).
if unpacker != nil {
if _, err = unpacker.Wait(); err != nil {
return err
}
// TODO: Check results to make sure unpack was successful
}
if hasMediaTypeBug1622 {
if desc, err = docker.ConvertManifest(ctx, store, desc); err != nil {
return err
}
}
imgs, err := is.Store(ctx, desc, ts.images)
if err != nil {
return err
}
if tops.Progress != nil {
for _, img := range imgs {
tops.Progress(transfer.Progress{
Event: "saved",
Name: img.Name,
})
}
}
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: fmt.Sprintf("Completed pull from %s", ir),
})
}
return nil
}
func fetchHandler(ingester content.Ingester, fetcher remotes.Fetcher, pt *ProgressTracker) images.HandlerFunc {
return func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
ctx = log.WithLogger(ctx, log.G(ctx).WithFields(log.Fields{
"digest": desc.Digest,
"mediatype": desc.MediaType,
"size": desc.Size,
}))
if desc.MediaType == images.MediaTypeDockerSchema1Manifest {
return nil, fmt.Errorf("%v not supported", desc.MediaType)
}
err := remotes.Fetch(ctx, ingester, fetcher, desc)
if errdefs.IsAlreadyExists(err) {
pt.MarkExists(desc)
return nil, nil
}
return nil, err
}
}
// getSupportedPlatform returns a matched platform comparing input UnpackConfiguration to the supported platform/snapshotter combinations
// If input platform didn't specify snapshotter, default will be used if there is a match on platform.
func getSupportedPlatform(uc transfer.UnpackConfiguration, supportedPlatforms []unpack.Platform) (bool, unpack.Platform) {
var u unpack.Platform
for _, sp := range supportedPlatforms {
// If both platform and snapshotter match, return the supportPlatform
// If platform matched and SnapshotterKey is empty, we assume client didn't pass SnapshotterKey
// use default Snapshotter
if sp.Platform.Match(uc.Platform) {
// Assume sp.SnapshotterKey is not empty
if uc.Snapshotter == sp.SnapshotterKey {
return true, sp
} else if uc.Snapshotter == "" && sp.SnapshotterKey == defaults.DefaultSnapshotter {
return true, sp
}
}
}
return false, u
}

View File

@@ -0,0 +1,153 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"testing"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/unpack"
"github.com/containerd/containerd/v2/defaults"
"github.com/containerd/platforms"
)
func TestGetSupportedPlatform(t *testing.T) {
supportedPlatforms := []unpack.Platform{
{
Platform: platforms.OnlyStrict(platforms.MustParse("linux/amd64")),
SnapshotterKey: "native",
},
{
Platform: platforms.OnlyStrict(platforms.MustParse("linux/amd64")),
SnapshotterKey: "devmapper",
},
{
Platform: platforms.OnlyStrict(platforms.MustParse("linux/arm64")),
SnapshotterKey: "native",
},
{
Platform: platforms.OnlyStrict(platforms.MustParse("linux/arm")),
SnapshotterKey: "native",
},
{
Platform: platforms.DefaultStrict(),
SnapshotterKey: defaults.DefaultSnapshotter,
},
}
for _, testCase := range []struct {
// Name is the name of the test
Name string
// Input
UnpackConfig transfer.UnpackConfiguration
SupportedPlatforms []unpack.Platform
// Expected
Match bool
ExpectedPlatform transfer.UnpackConfiguration
}{
{
Name: "No match on input linux/arm64 and devmapper snapshotter",
UnpackConfig: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/arm64"),
Snapshotter: "devmapper",
},
SupportedPlatforms: supportedPlatforms,
Match: false,
ExpectedPlatform: transfer.UnpackConfiguration{},
},
{
Name: "No match on input linux/386 and native snapshotter",
UnpackConfig: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/386"),
Snapshotter: "native",
},
SupportedPlatforms: supportedPlatforms,
Match: false,
ExpectedPlatform: transfer.UnpackConfiguration{},
},
{
Name: "Match linux/amd64 and native snapshotter",
UnpackConfig: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/amd64"),
Snapshotter: "native",
},
SupportedPlatforms: supportedPlatforms,
Match: true,
ExpectedPlatform: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/amd64"),
Snapshotter: "native",
},
},
{
Name: "Match linux/arm64 and native snapshotter",
UnpackConfig: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/arm64"),
Snapshotter: "native",
},
SupportedPlatforms: supportedPlatforms,
Match: true,
ExpectedPlatform: transfer.UnpackConfiguration{
Platform: platforms.MustParse("linux/arm64"),
Snapshotter: "native",
},
},
{
Name: "Default platform input only match with defaultSnapshotter",
UnpackConfig: transfer.UnpackConfiguration{
Platform: platforms.DefaultSpec(),
},
SupportedPlatforms: supportedPlatforms,
Match: true,
ExpectedPlatform: transfer.UnpackConfiguration{
Platform: platforms.DefaultSpec(),
Snapshotter: defaults.DefaultSnapshotter,
},
},
} {
testCase := testCase
t.Run(testCase.Name, func(t *testing.T) {
m, sp := getSupportedPlatform(testCase.UnpackConfig, testCase.SupportedPlatforms)
// Match result should match expected
if m != testCase.Match {
t.Fatalf("Expect match result %v, but got %v", testCase.Match, m)
}
// If match result is false, the Platform should be nil too
if !m && sp.Platform != nil {
t.Fatalf("Expect nil Platform when we don't have a match")
}
// Snapshotter should match, empty string can be compared too
if sp.SnapshotterKey != testCase.ExpectedPlatform.Snapshotter {
t.Fatalf("Expect SnapshotterKey %v, but got %v", testCase.ExpectedPlatform.Snapshotter, sp.SnapshotterKey)
}
// If the matched Platform is not nil, it should match the expected Platform
if sp.Platform != nil && !sp.Platform.Match(testCase.ExpectedPlatform.Platform) {
t.Fatalf("Expect Platform %v doesn't match", testCase.ExpectedPlatform.Platform)
}
// If the ExectedPlatform is not empty, the matched Platform shoule not be nil either
if sp.Platform == nil && testCase.ExpectedPlatform.Platform.OS != "" {
t.Fatalf("Expect Platform %v doesn't match", testCase.ExpectedPlatform.Platform)
}
})
}
}

268
core/transfer/local/push.go Normal file
View File

@@ -0,0 +1,268 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"fmt"
"sync"
"time"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/remotes"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/errdefs"
"github.com/containerd/platforms"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
func (ts *localTransferService) push(ctx context.Context, ig transfer.ImageGetter, p transfer.ImagePusher, tops *transfer.Config) error {
/*
// TODO: Platform matching
if pushCtx.PlatformMatcher == nil {
if len(pushCtx.Platforms) > 0 {
ps, err := platforms.ParseAll(pushCtx.Platforms)
if err != nil {
return err
}
pushCtx.PlatformMatcher = platforms.Any(ps...)
} else {
pushCtx.PlatformMatcher = platforms.All
}
}
*/
matcher := platforms.All
// Filter push
img, err := ig.Get(ctx, ts.images)
if err != nil {
return err
}
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: fmt.Sprintf("Pushing to %s", p),
})
tops.Progress(transfer.Progress{
Event: "pushing content",
Name: img.Name,
//Digest: img.Target.Digest.String(),
Desc: &img.Target,
})
}
var pusher remotes.Pusher
pusher, err = p.Pusher(ctx, img.Target)
if err != nil {
return err
}
var wrapper func(images.Handler) images.Handler
ctx, cancel := context.WithCancel(ctx)
if tops.Progress != nil {
progressTracker := NewProgressTracker(img.Name, "uploading") //Pass in first name as root
p := newProgressPusher(pusher, progressTracker)
go progressTracker.HandleProgress(ctx, tops.Progress, p)
defer progressTracker.Wait()
wrapper = p.WrapHandler
pusher = p
}
defer cancel()
// TODO: Add handler to track parents
/*
// TODO: Add handlers
if len(pushCtx.BaseHandlers) > 0 {
wrapper = func(h images.Handler) images.Handler {
h = images.Handlers(append(pushCtx.BaseHandlers, h)...)
if pushCtx.HandlerWrapper != nil {
h = pushCtx.HandlerWrapper(h)
}
return h
}
} else if pushCtx.HandlerWrapper != nil {
wrapper = pushCtx.HandlerWrapper
}
*/
if err := remotes.PushContent(ctx, pusher, img.Target, ts.content, ts.limiterU, matcher, wrapper); err != nil {
return err
}
if tops.Progress != nil {
tops.Progress(transfer.Progress{
Event: "pushed content",
Name: img.Name,
//Digest: img.Target.Digest.String(),
Desc: &img.Target,
})
tops.Progress(transfer.Progress{
Event: fmt.Sprintf("Completed push to %s", p),
Desc: &img.Target,
})
}
return nil
}
type progressPusher struct {
remotes.Pusher
progress *ProgressTracker
status *pushStatus
}
type pushStatus struct {
l sync.Mutex
statuses map[string]content.Status
complete map[digest.Digest]struct{}
}
func newProgressPusher(pusher remotes.Pusher, progress *ProgressTracker) *progressPusher {
return &progressPusher{
Pusher: pusher,
progress: progress,
status: &pushStatus{
statuses: map[string]content.Status{},
complete: map[digest.Digest]struct{}{},
},
}
}
func (p *progressPusher) WrapHandler(h images.Handler) images.Handler {
return images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) (subdescs []ocispec.Descriptor, err error) {
p.progress.Add(desc)
subdescs, err = h.Handle(ctx, desc)
p.progress.AddChildren(desc, subdescs)
return
})
}
func (p *progressPusher) Push(ctx context.Context, d ocispec.Descriptor) (content.Writer, error) {
ref := remotes.MakeRefKey(ctx, d)
p.status.add(ref, d)
cw, err := p.Pusher.Push(ctx, d)
if err != nil {
if errdefs.IsAlreadyExists(err) {
p.progress.MarkExists(d)
p.status.markComplete(ref, d)
}
return nil, err
}
return &progressWriter{
Writer: cw,
ref: ref,
desc: d,
status: p.status,
progress: p.progress,
}, nil
}
func (ps *pushStatus) update(ref string, delta int) {
ps.l.Lock()
status, ok := ps.statuses[ref]
if ok {
if delta > 0 {
status.Offset += int64(delta)
} else if delta < 0 {
status.Offset = 0
}
ps.statuses[ref] = status
}
ps.l.Unlock()
}
func (ps *pushStatus) add(ref string, d ocispec.Descriptor) {
status := content.Status{
Ref: ref,
Offset: 0,
Total: d.Size,
StartedAt: time.Now(),
}
ps.l.Lock()
_, ok := ps.statuses[ref]
_, complete := ps.complete[d.Digest]
if !ok && !complete {
ps.statuses[ref] = status
}
ps.l.Unlock()
}
func (ps *pushStatus) markComplete(ref string, d ocispec.Descriptor) {
ps.l.Lock()
_, ok := ps.statuses[ref]
if ok {
delete(ps.statuses, ref)
}
ps.complete[d.Digest] = struct{}{}
ps.l.Unlock()
}
func (ps *pushStatus) Status(name string) (content.Status, bool) {
ps.l.Lock()
status, ok := ps.statuses[name]
ps.l.Unlock()
return status, ok
}
func (ps *pushStatus) Check(ctx context.Context, dgst digest.Digest) (bool, error) {
ps.l.Lock()
_, ok := ps.complete[dgst]
ps.l.Unlock()
return ok, nil
}
func (p *progressPusher) Active(ctx context.Context, _ ...string) (ActiveJobs, error) {
return p.status, nil
}
func (p *progressPusher) Check(ctx context.Context, dgst digest.Digest) (bool, error) {
return p.status.Check(ctx, dgst)
}
type progressWriter struct {
content.Writer
ref string
desc ocispec.Descriptor
status *pushStatus
progress *ProgressTracker
}
func (pw *progressWriter) Write(p []byte) (n int, err error) {
n, err = pw.Writer.Write(p)
if err != nil {
// TODO: Handle reset error to reset progress
return
}
pw.status.update(pw.ref, n)
return
}
func (pw *progressWriter) Commit(ctx context.Context, size int64, expected digest.Digest, opts ...content.Opt) error {
err := pw.Writer.Commit(ctx, size, expected, opts...)
if err != nil {
if errdefs.IsAlreadyExists(err) {
pw.progress.MarkExists(pw.desc)
}
// TODO: Handle reset error to reset progress
}
pw.status.markComplete(pw.ref, pw.desc)
return err
}

View File

@@ -0,0 +1,39 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"github.com/containerd/containerd/v2/core/transfer"
)
func (ts *localTransferService) tag(ctx context.Context, ig transfer.ImageGetter, is transfer.ImageStorer, tops *transfer.Config) error {
ctx, done, err := ts.withLease(ctx)
if err != nil {
return err
}
defer done(ctx)
img, err := ig.Get(ctx, ts.images)
if err != nil {
return err
}
_, err = is.Store(ctx, img.Target, ts.images)
return err
}

View File

@@ -0,0 +1,187 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package local
import (
"context"
"fmt"
"io"
"time"
"github.com/containerd/typeurl/v2"
"golang.org/x/sync/semaphore"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
"github.com/containerd/containerd/v2/core/leases"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/unpack"
"github.com/containerd/containerd/v2/internal/kmutex"
"github.com/containerd/containerd/v2/pkg/imageverifier"
"github.com/containerd/errdefs"
)
type localTransferService struct {
leases leases.Manager
content content.Store
images images.Store
verifiers map[string]imageverifier.ImageVerifier
// limiter for upload
limiterU *semaphore.Weighted
// limiter for download operation
limiterD *semaphore.Weighted
config TransferConfig
}
func NewTransferService(lm leases.Manager, cs content.Store, is images.Store, vfs map[string]imageverifier.ImageVerifier, tc *TransferConfig) transfer.Transferrer {
ts := &localTransferService{
leases: lm,
content: cs,
images: is,
verifiers: vfs,
config: *tc,
}
if tc.MaxConcurrentUploadedLayers > 0 {
ts.limiterU = semaphore.NewWeighted(int64(tc.MaxConcurrentUploadedLayers))
}
if tc.MaxConcurrentDownloads > 0 {
ts.limiterD = semaphore.NewWeighted(int64(tc.MaxConcurrentDownloads))
}
return ts
}
func (ts *localTransferService) Transfer(ctx context.Context, src interface{}, dest interface{}, opts ...transfer.Opt) error {
topts := &transfer.Config{}
for _, opt := range opts {
opt(topts)
}
// Figure out matrix of whether source destination combination is supported
switch s := src.(type) {
case transfer.ImageFetcher:
switch d := dest.(type) {
case transfer.ImageStorer:
return ts.pull(ctx, s, d, topts)
}
case transfer.ImageGetter:
switch d := dest.(type) {
case transfer.ImagePusher:
return ts.push(ctx, s, d, topts)
case transfer.ImageExporter:
return ts.exportStream(ctx, s, d, topts)
case transfer.ImageStorer:
return ts.tag(ctx, s, d, topts)
}
case transfer.ImageImporter:
switch d := dest.(type) {
case transfer.ImageExportStreamer:
return ts.echo(ctx, s, d, topts)
case transfer.ImageStorer:
// TODO: verify imports with ImageVerifiers?
return ts.importStream(ctx, s, d, topts)
}
}
return fmt.Errorf("unable to transfer from %s to %s: %w", name(src), name(dest), errdefs.ErrNotImplemented)
}
func name(t interface{}) string {
switch s := t.(type) {
case fmt.Stringer:
return s.String()
case typeurl.Any:
return s.GetTypeUrl()
default:
return fmt.Sprintf("%T", t)
}
}
// echo is mostly used for testing, it implements an import->export which is
// a no-op which only roundtrips the bytes.
func (ts *localTransferService) echo(ctx context.Context, i transfer.ImageImporter, e transfer.ImageExportStreamer, tops *transfer.Config) error {
iis, ok := i.(transfer.ImageImportStreamer)
if !ok {
return fmt.Errorf("echo requires access to raw stream: %w", errdefs.ErrNotImplemented)
}
r, _, err := iis.ImportStream(ctx)
if err != nil {
return err
}
wc, _, err := e.ExportStream(ctx)
if err != nil {
return err
}
// TODO: Use fixed buffer? Send write progress?
_, err = io.Copy(wc, r)
if werr := wc.Close(); werr != nil && err == nil {
err = werr
}
return err
}
// WithLease attaches a lease on the context
func (ts *localTransferService) withLease(ctx context.Context, opts ...leases.Opt) (context.Context, func(context.Context) error, error) {
nop := func(context.Context) error { return nil }
_, ok := leases.FromContext(ctx)
if ok {
return ctx, nop, nil
}
ls := ts.leases
if len(opts) == 0 {
// Use default lease configuration if no options provided
opts = []leases.Opt{
leases.WithRandomID(),
leases.WithExpiration(24 * time.Hour),
}
}
l, err := ls.Create(ctx, opts...)
if err != nil {
return ctx, nop, err
}
ctx = leases.WithLease(ctx, l.ID)
return ctx, func(ctx context.Context) error {
return ls.Delete(ctx, l)
}, nil
}
type TransferConfig struct {
// MaxConcurrentDownloads is the max concurrent content downloads for pull.
MaxConcurrentDownloads int
// MaxConcurrentUploadedLayers is the max concurrent uploads for push
MaxConcurrentUploadedLayers int
// DuplicationSuppressor is used to make sure that there is only one
// in-flight fetch request or unpack handler for a given descriptor's
// digest or chain ID.
DuplicationSuppressor kmutex.KeyedLocker
// BaseHandlers are a set of handlers which get are called on dispatch.
// These handlers always get called before any operation specific
// handlers.
BaseHandlers []images.Handler
// UnpackPlatforms are used to specify supported combination of platforms and snapshotters
UnpackPlatforms []unpack.Platform
// RegistryConfigPath is a path to the root directory containing registry-specific configurations
RegistryConfigPath string
}

View File

@@ -0,0 +1,63 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package plugins
import (
"fmt"
"reflect"
"sync"
"github.com/containerd/errdefs"
"github.com/containerd/typeurl/v2"
)
var register = struct {
sync.RWMutex
r map[string]reflect.Type
}{}
func Register(apiObject, transferObject interface{}) {
url, err := typeurl.TypeURL(apiObject)
if err != nil {
panic(err)
}
// Lock
register.Lock()
defer register.Unlock()
if register.r == nil {
register.r = map[string]reflect.Type{}
}
if _, ok := register.r[url]; ok {
panic(fmt.Sprintf("url already registered: %v", url))
}
t := reflect.TypeOf(transferObject)
if t.Kind() == reflect.Ptr {
t = t.Elem()
}
register.r[url] = t
}
func ResolveType(any typeurl.Any) (interface{}, error) {
register.RLock()
defer register.RUnlock()
if register.r != nil {
if t, ok := register.r[any.GetTypeUrl()]; ok {
return reflect.New(t).Interface(), nil
}
}
return nil, fmt.Errorf("%v not registered: %w", any.GetTypeUrl(), errdefs.ErrNotFound)
}

View File

@@ -0,0 +1,130 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package proxy
import (
"context"
"errors"
"io"
"google.golang.org/protobuf/types/known/anypb"
transferapi "github.com/containerd/containerd/v2/api/services/transfer/v1"
transfertypes "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/containerd/v2/core/transfer"
tstreaming "github.com/containerd/containerd/v2/core/transfer/streaming"
"github.com/containerd/containerd/v2/pkg/oci"
"github.com/containerd/log"
"github.com/containerd/typeurl/v2"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
type proxyTransferrer struct {
client transferapi.TransferClient
streamCreator streaming.StreamCreator
}
// NewTransferrer returns a new transferrer which communicates over a GRPC
// connection using the containerd transfer API
func NewTransferrer(client transferapi.TransferClient, sc streaming.StreamCreator) transfer.Transferrer {
return &proxyTransferrer{
client: client,
streamCreator: sc,
}
}
func (p *proxyTransferrer) Transfer(ctx context.Context, src interface{}, dst interface{}, opts ...transfer.Opt) error {
o := &transfer.Config{}
for _, opt := range opts {
opt(o)
}
apiOpts := &transferapi.TransferOptions{}
if o.Progress != nil {
sid := tstreaming.GenerateID("progress")
stream, err := p.streamCreator.Create(ctx, sid)
if err != nil {
return err
}
apiOpts.ProgressStream = sid
go func() {
for {
a, err := stream.Recv()
if err != nil {
if !errors.Is(err, io.EOF) {
log.G(ctx).WithError(err).Error("progress stream failed to recv")
}
return
}
i, err := typeurl.UnmarshalAny(a)
if err != nil {
log.G(ctx).WithError(err).Warnf("failed to unmarshal progress object: %v", a.GetTypeUrl())
}
switch v := i.(type) {
case *transfertypes.Progress:
var descp *ocispec.Descriptor
if v.Desc != nil {
desc := oci.DescriptorFromProto(v.Desc)
descp = &desc
}
o.Progress(transfer.Progress{
Event: v.Event,
Name: v.Name,
Parents: v.Parents,
Progress: v.Progress,
Total: v.Total,
Desc: descp,
})
default:
log.G(ctx).Warnf("unhandled progress object %T: %v", i, a.GetTypeUrl())
}
}
}()
}
asrc, err := p.marshalAny(ctx, src)
if err != nil {
return err
}
adst, err := p.marshalAny(ctx, dst)
if err != nil {
return err
}
req := &transferapi.TransferRequest{
Source: &anypb.Any{
TypeUrl: asrc.GetTypeUrl(),
Value: asrc.GetValue(),
},
Destination: &anypb.Any{
TypeUrl: adst.GetTypeUrl(),
Value: adst.GetValue(),
},
Options: apiOpts,
}
_, err = p.client.Transfer(ctx, req)
return err
}
func (p *proxyTransferrer) marshalAny(ctx context.Context, i interface{}) (typeurl.Any, error) {
switch m := i.(type) {
case streamMarshaler:
return m.MarshalAny(ctx, p.streamCreator)
}
return typeurl.MarshalAny(i)
}
type streamMarshaler interface {
MarshalAny(context.Context, streaming.StreamCreator) (typeurl.Any, error)
}

View File

@@ -0,0 +1,293 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package registry
import (
"context"
"errors"
"fmt"
"io"
"net/http"
"strings"
"sync"
transfertypes "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/remotes"
"github.com/containerd/containerd/v2/core/remotes/docker"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/containerd/v2/core/transfer"
"github.com/containerd/containerd/v2/core/transfer/plugins"
tstreaming "github.com/containerd/containerd/v2/core/transfer/streaming"
"github.com/containerd/log"
"github.com/containerd/typeurl/v2"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
func init() {
// TODO: Move this to separate package?
plugins.Register(&transfertypes.OCIRegistry{}, &OCIRegistry{})
}
// NewOCIRegistry initializes with hosts, authorizer callback, and headers
func NewOCIRegistry(ref string, headers http.Header, creds CredentialHelper) *OCIRegistry {
// Create an authorizer
var aopts []docker.AuthorizerOpt
if creds != nil {
// TODO: Support bearer
aopts = append(aopts, docker.WithAuthCreds(func(host string) (string, string, error) {
c, err := creds.GetCredentials(context.Background(), ref, host)
if err != nil {
return "", "", err
}
return c.Username, c.Secret, nil
}))
}
ropts := []docker.RegistryOpt{
docker.WithAuthorizer(docker.NewDockerAuthorizer(aopts...)),
}
// TODO: Apply local configuration, maybe dynamically create resolver when requested
resolver := docker.NewResolver(docker.ResolverOptions{
Hosts: docker.ConfigureDefaultRegistries(ropts...),
Headers: headers,
})
return &OCIRegistry{
reference: ref,
headers: headers,
creds: creds,
resolver: resolver,
}
}
// From stream
type CredentialHelper interface {
GetCredentials(ctx context.Context, ref, host string) (Credentials, error)
}
type Credentials struct {
Host string
Username string
Secret string
Header string
}
// OCI
type OCIRegistry struct {
reference string
headers http.Header
creds CredentialHelper
resolver remotes.Resolver
// This could be an interface which returns resolver?
// Resolver could also be a plug-able interface, to call out to a program to fetch?
}
func (r *OCIRegistry) String() string {
return fmt.Sprintf("OCI Registry (%s)", r.reference)
}
func (r *OCIRegistry) Image() string {
return r.reference
}
func (r *OCIRegistry) Resolve(ctx context.Context) (name string, desc ocispec.Descriptor, err error) {
return r.resolver.Resolve(ctx, r.reference)
}
func (r *OCIRegistry) Fetcher(ctx context.Context, ref string) (transfer.Fetcher, error) {
return r.resolver.Fetcher(ctx, ref)
}
func (r *OCIRegistry) Pusher(ctx context.Context, desc ocispec.Descriptor) (transfer.Pusher, error) {
var ref = r.reference
// Annotate ref with digest to push only push tag for single digest
if !strings.Contains(ref, "@") {
ref = ref + "@" + desc.Digest.String()
}
return r.resolver.Pusher(ctx, ref)
}
func (r *OCIRegistry) MarshalAny(ctx context.Context, sm streaming.StreamCreator) (typeurl.Any, error) {
res := &transfertypes.RegistryResolver{}
if r.headers != nil {
res.Headers = map[string]string{}
for k := range r.headers {
res.Headers[k] = r.headers.Get(k)
}
}
if r.creds != nil {
sid := tstreaming.GenerateID("creds")
stream, err := sm.Create(ctx, sid)
if err != nil {
return nil, err
}
go func() {
// Check for context cancellation as well
for {
select {
case <-ctx.Done():
return
default:
}
req, err := stream.Recv()
if err != nil {
// If not EOF, log error
return
}
var s transfertypes.AuthRequest
if err := typeurl.UnmarshalTo(req, &s); err != nil {
log.G(ctx).WithError(err).Error("failed to unmarshal credential request")
continue
}
creds, err := r.creds.GetCredentials(ctx, s.Reference, s.Host)
if err != nil {
log.G(ctx).WithError(err).Error("failed to get credentials")
continue
}
var resp transfertypes.AuthResponse
if creds.Header != "" {
resp.AuthType = transfertypes.AuthType_HEADER
resp.Secret = creds.Header
} else if creds.Username != "" {
resp.AuthType = transfertypes.AuthType_CREDENTIALS
resp.Username = creds.Username
resp.Secret = creds.Secret
} else {
resp.AuthType = transfertypes.AuthType_REFRESH
resp.Secret = creds.Secret
}
a, err := typeurl.MarshalAny(&resp)
if err != nil {
log.G(ctx).WithError(err).Error("failed to marshal credential response")
continue
}
if err := stream.Send(a); err != nil {
if !errors.Is(err, io.EOF) {
log.G(ctx).WithError(err).Error("unexpected send failure")
}
return
}
}
}()
res.AuthStream = sid
}
s := &transfertypes.OCIRegistry{
Reference: r.reference,
Resolver: res,
}
return typeurl.MarshalAny(s)
}
func (r *OCIRegistry) UnmarshalAny(ctx context.Context, sm streaming.StreamGetter, a typeurl.Any) error {
var (
s transfertypes.OCIRegistry
ropts []docker.RegistryOpt
aopts []docker.AuthorizerOpt
)
if err := typeurl.UnmarshalTo(a, &s); err != nil {
return err
}
if s.Resolver != nil {
if sid := s.Resolver.AuthStream; sid != "" {
stream, err := sm.Get(ctx, sid)
if err != nil {
log.G(ctx).WithError(err).WithField("stream", sid).Debug("failed to get auth stream")
return err
}
r.creds = &credCallback{
stream: stream,
}
aopts = append(aopts, docker.WithAuthCreds(func(host string) (string, string, error) {
c, err := r.creds.GetCredentials(context.Background(), s.Reference, host)
if err != nil {
return "", "", err
}
return c.Username, c.Secret, nil
}))
}
r.headers = http.Header{}
for k, v := range s.Resolver.Headers {
r.headers.Add(k, v)
}
}
authorizer := docker.NewDockerAuthorizer(aopts...)
ropts = append(ropts, docker.WithAuthorizer(authorizer))
r.reference = s.Reference
r.resolver = docker.NewResolver(docker.ResolverOptions{
Hosts: docker.ConfigureDefaultRegistries(ropts...),
Headers: r.headers,
})
return nil
}
type credCallback struct {
sync.Mutex
stream streaming.Stream
}
func (cc *credCallback) GetCredentials(ctx context.Context, ref, host string) (Credentials, error) {
cc.Lock()
defer cc.Unlock()
ar := &transfertypes.AuthRequest{
Host: host,
Reference: ref,
}
anyType, err := typeurl.MarshalAny(ar)
if err != nil {
return Credentials{}, err
}
if err := cc.stream.Send(anyType); err != nil {
return Credentials{}, err
}
resp, err := cc.stream.Recv()
if err != nil {
return Credentials{}, err
}
var s transfertypes.AuthResponse
if err := typeurl.UnmarshalTo(resp, &s); err != nil {
return Credentials{}, err
}
creds := Credentials{
Host: host,
}
switch s.AuthType {
case transfertypes.AuthType_CREDENTIALS:
creds.Username = s.Username
creds.Secret = s.Secret
case transfertypes.AuthType_REFRESH:
creds.Secret = s.Secret
case transfertypes.AuthType_HEADER:
creds.Header = s.Secret
}
return creds, nil
}

View File

@@ -0,0 +1,210 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package streaming
import (
"context"
"crypto/rand"
"encoding/base64"
"errors"
"fmt"
"io"
"sync"
"time"
transferapi "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/log"
"github.com/containerd/typeurl/v2"
)
const maxRead = 32 * 1024
const windowSize = 2 * maxRead
var bufPool = &sync.Pool{
New: func() interface{} {
buffer := make([]byte, maxRead)
return &buffer
},
}
func SendStream(ctx context.Context, r io.Reader, stream streaming.Stream) {
window := make(chan int32)
go func() {
defer close(window)
for {
select {
case <-ctx.Done():
return
default:
}
anyType, err := stream.Recv()
if err != nil {
if !errors.Is(err, io.EOF) && !errors.Is(err, context.Canceled) {
log.G(ctx).WithError(err).Error("send stream ended without EOF")
}
return
}
i, err := typeurl.UnmarshalAny(anyType)
if err != nil {
log.G(ctx).WithError(err).Error("failed to unmarshal stream object")
continue
}
switch v := i.(type) {
case *transferapi.WindowUpdate:
select {
case <-ctx.Done():
return
case window <- v.Update:
}
default:
log.G(ctx).Errorf("unexpected stream object of type %T", i)
}
}
}()
go func() {
defer stream.Close()
buf := bufPool.Get().(*[]byte)
defer bufPool.Put(buf)
var remaining int32
for {
if remaining > 0 {
// Don't wait for window update since there are remaining
select {
case <-ctx.Done():
// TODO: Send error message on stream before close to allow remote side to return error
return
case update := <-window:
remaining += update
default:
}
} else {
// Block until window updated
select {
case <-ctx.Done():
// TODO: Send error message on stream before close to allow remote side to return error
return
case update := <-window:
remaining = update
}
}
var max int32 = maxRead
if max > remaining {
max = remaining
}
b := (*buf)[:max]
n, err := r.Read(b)
if err != nil {
if !errors.Is(err, io.EOF) {
log.G(ctx).WithError(err).Errorf("failed to read stream source")
// TODO: Send error message on stream before close to allow remote side to return error
}
return
}
remaining = remaining - int32(n)
data := &transferapi.Data{
Data: b[:n],
}
anyType, err := typeurl.MarshalAny(data)
if err != nil {
log.G(ctx).WithError(err).Errorf("failed to marshal data for send")
// TODO: Send error message on stream before close to allow remote side to return error
return
}
if err := stream.Send(anyType); err != nil {
log.G(ctx).WithError(err).Errorf("send failed")
return
}
}
}()
}
func ReceiveStream(ctx context.Context, stream streaming.Stream) io.Reader {
r, w := io.Pipe()
go func() {
defer stream.Close()
var window int32
for {
var werr error
if window < windowSize {
update := &transferapi.WindowUpdate{
Update: windowSize,
}
anyType, err := typeurl.MarshalAny(update)
if err != nil {
w.CloseWithError(fmt.Errorf("failed to marshal window update: %w", err))
return
}
// check window update error after recv, stream may be complete
if werr = stream.Send(anyType); werr == nil {
window += windowSize
} else if errors.Is(werr, io.EOF) {
// TODO: Why does send return EOF here
werr = nil
}
}
anyType, err := stream.Recv()
if err != nil {
if errors.Is(err, io.EOF) || errors.Is(err, context.Canceled) {
err = nil
} else {
err = fmt.Errorf("received failed: %w", err)
}
w.CloseWithError(err)
return
} else if werr != nil {
// Try receive before erroring out
w.CloseWithError(fmt.Errorf("failed to send window update: %w", werr))
return
}
i, err := typeurl.UnmarshalAny(anyType)
if err != nil {
w.CloseWithError(fmt.Errorf("failed to unmarshal received object: %w", err))
return
}
switch v := i.(type) {
case *transferapi.Data:
n, err := w.Write(v.Data)
if err != nil {
w.CloseWithError(fmt.Errorf("failed to unmarshal received object: %w", err))
// Close will error out sender
return
}
window = window - int32(n)
// TODO: Handle error case
default:
log.G(ctx).Warnf("Ignoring unknown stream object of type %T", i)
continue
}
}
}()
return r
}
func GenerateID(prefix string) string {
t := time.Now()
var b [3]byte
rand.Read(b[:])
return fmt.Sprintf("%s-%d-%s", prefix, t.Nanosecond(), base64.URLEncoding.EncodeToString(b[:]))
}

View File

@@ -0,0 +1,165 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package streaming
import (
"bytes"
"context"
"io"
"testing"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/typeurl/v2"
)
func FuzzSendAndReceive(f *testing.F) {
f.Add([]byte{})
f.Add([]byte{0})
f.Add(bytes.Repeat([]byte{0}, windowSize+1))
f.Add([]byte("hello"))
f.Add(bytes.Repeat([]byte("hello"), windowSize+1))
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
f.Fuzz(func(t *testing.T, expected []byte) {
runSendAndReceiveFuzz(ctx, t, expected)
runSendAndReceiveChainFuzz(ctx, t, expected)
runWriterFuzz(ctx, t, expected)
})
}
func runSendAndReceiveFuzz(ctx context.Context, t *testing.T, expected []byte) {
rs, ws := pipeStream()
r, w := io.Pipe()
SendStream(ctx, r, ws)
or := ReceiveStream(ctx, rs)
go func() {
io.Copy(w, bytes.NewBuffer(expected))
w.Close()
}()
actual, err := io.ReadAll(or)
if err != nil {
t.Fatal(err)
}
if !bytes.Equal(expected, actual) {
t.Fatalf("received bytes are not equal\n\tactual: %v\n\texpected:%v", actual, expected)
}
}
func runSendAndReceiveChainFuzz(ctx context.Context, t *testing.T, expected []byte) {
r, w := io.Pipe()
or := chainStreams(ctx, chainStreams(ctx, chainStreams(ctx, r)))
go func() {
io.Copy(w, bytes.NewBuffer(expected))
w.Close()
}()
actual, err := io.ReadAll(or)
if err != nil {
t.Fatal(err)
}
if !bytes.Equal(expected, actual) {
t.Fatalf("received bytes are not equal\n\tactual: %v\n\texpected:%v", actual, expected)
}
}
func runWriterFuzz(ctx context.Context, t *testing.T, expected []byte) {
rs, ws := pipeStream()
wc := WriteByteStream(ctx, ws)
or := ReceiveStream(ctx, rs)
go func() {
io.Copy(wc, bytes.NewBuffer(expected))
wc.Close()
}()
actual, err := io.ReadAll(or)
if err != nil {
t.Fatal(err)
}
if !bytes.Equal(expected, actual) {
t.Fatalf("received bytes are not equal\n\tactual: %v\n\texpected:%v", actual, expected)
}
}
func chainStreams(ctx context.Context, r io.Reader) io.Reader {
rs, ws := pipeStream()
SendStream(ctx, r, ws)
return ReceiveStream(ctx, rs)
}
func pipeStream() (streaming.Stream, streaming.Stream) {
r := make(chan typeurl.Any)
rc := make(chan struct{})
w := make(chan typeurl.Any)
wc := make(chan struct{})
rs := &testStream{
send: w,
recv: r,
closer: wc,
remote: rc,
}
ws := &testStream{
send: r,
recv: w,
closer: rc,
remote: wc,
}
return rs, ws
}
type testStream struct {
send chan<- typeurl.Any
recv <-chan typeurl.Any
closer chan struct{}
remote <-chan struct{}
}
func (ts *testStream) Send(a typeurl.Any) error {
select {
case <-ts.remote:
return io.ErrClosedPipe
case ts.send <- a:
}
return nil
}
func (ts *testStream) Recv() (typeurl.Any, error) {
select {
case <-ts.remote:
return nil, io.EOF
case a := <-ts.recv:
return a, nil
}
}
func (ts *testStream) Close() error {
select {
case <-ts.closer:
return nil
default:
}
close(ts.closer)
return nil
}

View File

@@ -0,0 +1,130 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package streaming
import (
"context"
"errors"
"io"
"sync/atomic"
transferapi "github.com/containerd/containerd/v2/api/types/transfer"
"github.com/containerd/containerd/v2/core/streaming"
"github.com/containerd/log"
"github.com/containerd/typeurl/v2"
)
func WriteByteStream(ctx context.Context, stream streaming.Stream) io.WriteCloser {
wbs := &writeByteStream{
ctx: ctx,
stream: stream,
updated: make(chan struct{}, 1),
}
go func() {
for {
select {
case <-ctx.Done():
return
default:
}
anyType, err := stream.Recv()
if err != nil {
if !errors.Is(err, io.EOF) && !errors.Is(err, context.Canceled) {
log.G(ctx).WithError(err).Error("send byte stream ended without EOF")
}
return
}
i, err := typeurl.UnmarshalAny(anyType)
if err != nil {
log.G(ctx).WithError(err).Error("failed to unmarshal stream object")
continue
}
switch v := i.(type) {
case *transferapi.WindowUpdate:
atomic.AddInt32(&wbs.remaining, v.Update)
select {
case <-ctx.Done():
return
case wbs.updated <- struct{}{}:
default:
// Don't block if no writes are waiting
}
default:
log.G(ctx).Errorf("unexpected stream object of type %T", i)
}
}
}()
return wbs
}
type writeByteStream struct {
ctx context.Context
stream streaming.Stream
remaining int32
updated chan struct{}
}
func (wbs *writeByteStream) Write(p []byte) (n int, err error) {
for len(p) > 0 {
remaining := atomic.LoadInt32(&wbs.remaining)
if remaining == 0 {
// Don't wait for window update since there are remaining
select {
case <-wbs.ctx.Done():
// TODO: Send error message on stream before close to allow remote side to return error
err = io.ErrShortWrite
return
case <-wbs.updated:
continue
}
}
var max int32 = maxRead
if max > int32(len(p)) {
max = int32(len(p))
}
if max > remaining {
max = remaining
}
// TODO: continue
// remaining = remaining - int32(n)
data := &transferapi.Data{
Data: p[:max],
}
var anyType typeurl.Any
anyType, err = typeurl.MarshalAny(data)
if err != nil {
log.G(wbs.ctx).WithError(err).Errorf("failed to marshal data for send")
// TODO: Send error message on stream before close to allow remote side to return error
return
}
if err = wbs.stream.Send(anyType); err != nil {
log.G(wbs.ctx).WithError(err).Errorf("send failed")
return
}
n += int(max)
p = p[max:]
atomic.AddInt32(&wbs.remaining, -1*max)
}
return
}
func (wbs *writeByteStream) Close() error {
return wbs.stream.Close()
}

136
core/transfer/transfer.go Normal file
View File

@@ -0,0 +1,136 @@
/*
Copyright The containerd Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package transfer
import (
"context"
"io"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/containerd/containerd/v2/core/content"
"github.com/containerd/containerd/v2/core/images"
)
type Transferrer interface {
Transfer(ctx context.Context, source interface{}, destination interface{}, opts ...Opt) error
}
type ImageResolver interface {
Resolve(ctx context.Context) (name string, desc ocispec.Descriptor, err error)
}
type ImageFetcher interface {
ImageResolver
Fetcher(ctx context.Context, ref string) (Fetcher, error)
}
type ImagePusher interface {
Pusher(context.Context, ocispec.Descriptor) (Pusher, error)
}
type Fetcher interface {
Fetch(context.Context, ocispec.Descriptor) (io.ReadCloser, error)
}
type Pusher interface {
Push(context.Context, ocispec.Descriptor) (content.Writer, error)
}
// ImageFilterer is used to filter out child objects of an image
type ImageFilterer interface {
ImageFilter(images.HandlerFunc, content.Store) images.HandlerFunc
}
// ImageStorer is a type which is capable of storing images for
// the provided descriptor. The descriptor may be any type of manifest
// including an index with multiple image references.
type ImageStorer interface {
Store(context.Context, ocispec.Descriptor, images.Store) ([]images.Image, error)
}
// ImageGetter is type which returns an image from an image store
type ImageGetter interface {
Get(context.Context, images.Store) (images.Image, error)
}
// ImageLookup is a type which returns images from an image store
// based on names or prefixes
type ImageLookup interface {
Lookup(context.Context, images.Store) ([]images.Image, error)
}
// ImageExporter exports images to a writer
type ImageExporter interface {
Export(context.Context, content.Store, []images.Image) error
}
// ImageImporter imports an image into a content store
type ImageImporter interface {
Import(context.Context, content.Store) (ocispec.Descriptor, error)
}
// ImageImportStreamer returns an import streamer based on OCI or
// Docker image tar archives. The stream should be a raw tar stream
// and without compression.
type ImageImportStreamer interface {
ImportStream(context.Context) (io.Reader, string, error)
}
type ImageExportStreamer interface {
ExportStream(context.Context) (io.WriteCloser, string, error)
}
type ImageUnpacker interface {
UnpackPlatforms() []UnpackConfiguration
}
// UnpackConfiguration specifies the platform and snapshotter to use for resolving
// the unpack Platform, if snapshotter is not specified the platform default will
// be used.
type UnpackConfiguration struct {
Platform ocispec.Platform
Snapshotter string
}
type ProgressFunc func(Progress)
type Config struct {
Progress ProgressFunc
}
type Opt func(*Config)
func WithProgress(f ProgressFunc) Opt {
return func(opts *Config) {
opts.Progress = f
}
}
// Progress is used to represent a particular progress event or incremental
// update for the provided named object. The parents represent the names of
// the objects which initiated the progress for the provided named object.
// The name and what object it represents is determined by the implementation.
type Progress struct {
Event string
Name string
Parents []string
Progress int64
Total int64
Desc *ocispec.Descriptor // since containerd v2.0
}