Add platform match comparer interface
Adds a new platform interface for matching and comparing platforms. This new interface allows both filtering and ordering of platforms to support running multiple platform and choosing the best platform. Signed-off-by: Derek McGowan <derek@mcgstyle.net>
This commit is contained in:
parent
830363acac
commit
9edcfcc1cb
68
client.go
68
client.go
@ -259,9 +259,10 @@ type RemoteContext struct {
|
||||
// If no resolver is provided, defaults to Docker registry resolver.
|
||||
Resolver remotes.Resolver
|
||||
|
||||
// Platforms defines which platforms to handle when doing the image operation.
|
||||
// If this field is empty, content for all platforms will be pulled.
|
||||
Platforms []string
|
||||
// PlatformMatcher is used to match the platforms for an image
|
||||
// operation and define the preference when a single match is required
|
||||
// from multiple platforms.
|
||||
PlatformMatcher platforms.MatchComparer
|
||||
|
||||
// Unpack is done after an image is pulled to extract into a snapshotter.
|
||||
// If an image is not unpacked on pull, it can be unpacked any time
|
||||
@ -283,6 +284,12 @@ type RemoteContext struct {
|
||||
// manifests. If this option is false then any image which resolves
|
||||
// to schema 1 will return an error since schema 1 is not supported.
|
||||
ConvertSchema1 bool
|
||||
|
||||
// Platforms defines which platforms to handle when doing the image operation.
|
||||
// Platforms is ignored when a PlatformMatcher is set, otherwise the
|
||||
// platforms will be used to create a PlatformMatcher with no ordering
|
||||
// preference.
|
||||
Platforms []string
|
||||
}
|
||||
|
||||
func defaultRemoteContext() *RemoteContext {
|
||||
@ -308,6 +315,23 @@ func (c *Client) Fetch(ctx context.Context, ref string, opts ...RemoteOpt) (imag
|
||||
return images.Image{}, errors.New("unpack on fetch not supported, try pull")
|
||||
}
|
||||
|
||||
if fetchCtx.PlatformMatcher == nil {
|
||||
if len(fetchCtx.Platforms) == 0 {
|
||||
fetchCtx.PlatformMatcher = platforms.All
|
||||
} else {
|
||||
var ps []ocispec.Platform
|
||||
for _, s := range fetchCtx.Platforms {
|
||||
p, err := platforms.Parse(s)
|
||||
if err != nil {
|
||||
return images.Image{}, errors.Wrapf(err, "invalid platform %s", s)
|
||||
}
|
||||
ps = append(ps, p)
|
||||
}
|
||||
|
||||
fetchCtx.PlatformMatcher = platforms.Any(ps...)
|
||||
}
|
||||
}
|
||||
|
||||
ctx, done, err := c.WithLease(ctx)
|
||||
if err != nil {
|
||||
return images.Image{}, err
|
||||
@ -327,10 +351,19 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (Image
|
||||
}
|
||||
}
|
||||
|
||||
if len(pullCtx.Platforms) > 1 {
|
||||
return nil, errors.New("cannot pull multiplatform image locally, try Fetch")
|
||||
} else if len(pullCtx.Platforms) == 0 {
|
||||
pullCtx.Platforms = []string{platforms.Default()}
|
||||
if pullCtx.PlatformMatcher == nil {
|
||||
if len(pullCtx.Platforms) > 1 {
|
||||
return nil, errors.New("cannot pull multiplatform image locally, try Fetch")
|
||||
} else if len(pullCtx.Platforms) == 0 {
|
||||
pullCtx.PlatformMatcher = platforms.Default()
|
||||
} else {
|
||||
p, err := platforms.Parse(pullCtx.Platforms[0])
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "invalid platform %s", pullCtx.Platforms[0])
|
||||
}
|
||||
|
||||
pullCtx.PlatformMatcher = platforms.Only(p)
|
||||
}
|
||||
}
|
||||
|
||||
ctx, done, err := c.WithLease(ctx)
|
||||
@ -344,7 +377,7 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (Image
|
||||
return nil, err
|
||||
}
|
||||
|
||||
i := NewImageWithPlatform(c, img, pullCtx.Platforms[0])
|
||||
i := NewImageWithPlatform(c, img, pullCtx.PlatformMatcher)
|
||||
|
||||
if pullCtx.Unpack {
|
||||
if err := i.Unpack(ctx, pullCtx.Snapshotter); err != nil {
|
||||
@ -380,7 +413,7 @@ func (c *Client) fetch(ctx context.Context, rCtx *RemoteContext, ref string) (im
|
||||
// Set any children labels for that content
|
||||
childrenHandler = images.SetChildrenLabels(store, childrenHandler)
|
||||
// Filter children by platforms
|
||||
childrenHandler = images.FilterPlatforms(childrenHandler, rCtx.Platforms...)
|
||||
childrenHandler = images.FilterPlatforms(childrenHandler, rCtx.PlatformMatcher)
|
||||
|
||||
handler = images.Handlers(append(rCtx.BaseHandlers,
|
||||
remotes.FetchHandler(store, fetcher),
|
||||
@ -437,13 +470,28 @@ func (c *Client) Push(ctx context.Context, ref string, desc ocispec.Descriptor,
|
||||
return err
|
||||
}
|
||||
}
|
||||
if pushCtx.PlatformMatcher == nil {
|
||||
if len(pushCtx.Platforms) > 0 {
|
||||
var ps []ocispec.Platform
|
||||
for _, platform := range pushCtx.Platforms {
|
||||
p, err := platforms.Parse(platform)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "invalid platform %s", platform)
|
||||
}
|
||||
ps = append(ps, p)
|
||||
}
|
||||
pushCtx.PlatformMatcher = platforms.Any(ps...)
|
||||
} else {
|
||||
pushCtx.PlatformMatcher = platforms.All
|
||||
}
|
||||
}
|
||||
|
||||
pusher, err := pushCtx.Resolver.Pusher(ctx, ref)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return remotes.PushContent(ctx, pusher, desc, c.ContentStore(), pushCtx.Platforms, pushCtx.BaseHandlers...)
|
||||
return remotes.PushContent(ctx, pusher, desc, c.ContentStore(), pushCtx.PlatformMatcher, pushCtx.BaseHandlers...)
|
||||
}
|
||||
|
||||
// GetImage returns an existing image
|
||||
|
@ -89,7 +89,7 @@ type RemoteOpt func(*Client, *RemoteContext) error
|
||||
// content for
|
||||
func WithPlatform(platform string) RemoteOpt {
|
||||
if platform == "" {
|
||||
platform = platforms.Default()
|
||||
platform = platforms.DefaultString()
|
||||
}
|
||||
return func(_ *Client, c *RemoteContext) error {
|
||||
for _, p := range c.Platforms {
|
||||
@ -103,6 +103,16 @@ func WithPlatform(platform string) RemoteOpt {
|
||||
}
|
||||
}
|
||||
|
||||
// WithPlatformMatcher specifies the matcher to use for
|
||||
// determining which platforms to pull content for.
|
||||
// This value supersedes anything set with `WithPlatform`.
|
||||
func WithPlatformMatcher(m platforms.MatchComparer) RemoteOpt {
|
||||
return func(_ *Client, c *RemoteContext) error {
|
||||
c.PlatformMatcher = m
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// WithPullUnpack is used to unpack an image after pull. This
|
||||
// uses the snapshotter, content store, and diff service
|
||||
// configured for the client.
|
||||
|
@ -185,7 +185,7 @@ func TestImagePull(t *testing.T) {
|
||||
|
||||
ctx, cancel := testContext()
|
||||
defer cancel()
|
||||
_, err = client.Pull(ctx, testImage, WithPlatform(platforms.Default()))
|
||||
_, err = client.Pull(ctx, testImage, WithPlatformMatcher(platforms.Default()))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -304,7 +304,7 @@ func TestImagePullSchema1(t *testing.T) {
|
||||
ctx, cancel := testContext()
|
||||
defer cancel()
|
||||
schema1TestImage := "gcr.io/google_containers/pause:3.0@sha256:0d093c962a6c2dd8bb8727b661e2b5f13e9df884af9945b4cc7088d9350cd3ee"
|
||||
_, err = client.Pull(ctx, schema1TestImage, WithPlatform(platforms.Default()), WithSchema1Conversion)
|
||||
_, err = client.Pull(ctx, schema1TestImage, WithPlatform(platforms.DefaultString()), WithSchema1Conversion)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -113,7 +113,7 @@ func NewFetchConfig(ctx context.Context, clicontext *cli.Context) (*FetchConfig,
|
||||
if !clicontext.Bool("all-platforms") {
|
||||
p := clicontext.StringSlice("platform")
|
||||
if len(p) == 0 {
|
||||
p = append(p, platforms.Default())
|
||||
p = append(p, platforms.DefaultString())
|
||||
}
|
||||
config.Platforms = p
|
||||
}
|
||||
|
@ -25,6 +25,7 @@ import (
|
||||
"github.com/containerd/containerd/images"
|
||||
"github.com/containerd/containerd/log"
|
||||
"github.com/containerd/containerd/platforms"
|
||||
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/urfave/cli"
|
||||
)
|
||||
@ -86,26 +87,28 @@ command. As part of this process, we do the following:
|
||||
|
||||
// TODO: Show unpack status
|
||||
|
||||
var p []string
|
||||
var p []ocispec.Platform
|
||||
if context.Bool("all-platforms") {
|
||||
all, err := images.Platforms(ctx, client.ContentStore(), img.Target)
|
||||
p, err = images.Platforms(ctx, client.ContentStore(), img.Target)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "unable to resolve image platforms")
|
||||
}
|
||||
p = make([]string, len(all))
|
||||
for i := range all {
|
||||
p[i] = platforms.Format(all[i])
|
||||
}
|
||||
} else {
|
||||
p = context.StringSlice("platform")
|
||||
for _, s := range context.StringSlice("platform") {
|
||||
ps, err := platforms.Parse(s)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "unable to parse platform %s", s)
|
||||
}
|
||||
p = append(p, ps)
|
||||
}
|
||||
}
|
||||
if len(p) == 0 {
|
||||
p = append(p, platforms.Default())
|
||||
p = append(p, platforms.DefaultSpec())
|
||||
}
|
||||
|
||||
for _, platform := range p {
|
||||
fmt.Printf("unpacking %s %s...\n", platform, img.Target.Digest)
|
||||
i := containerd.NewImageWithPlatform(client, img, platform)
|
||||
fmt.Printf("unpacking %s %s...\n", platforms.Format(platform), img.Target.Digest)
|
||||
i := containerd.NewImageWithPlatform(client, img, platforms.Only(platform))
|
||||
err = i.Unpack(ctx, context.String("snapshotter"))
|
||||
if err != nil {
|
||||
return err
|
||||
|
6
image.go
6
image.go
@ -63,7 +63,7 @@ func NewImage(client *Client, i images.Image) Image {
|
||||
}
|
||||
|
||||
// NewImageWithPlatform returns a client image object from the metadata image
|
||||
func NewImageWithPlatform(client *Client, i images.Image, platform string) Image {
|
||||
func NewImageWithPlatform(client *Client, i images.Image, platform platforms.MatchComparer) Image {
|
||||
return &image{
|
||||
client: client,
|
||||
i: i,
|
||||
@ -75,7 +75,7 @@ type image struct {
|
||||
client *Client
|
||||
|
||||
i images.Image
|
||||
platform string
|
||||
platform platforms.MatchComparer
|
||||
}
|
||||
|
||||
func (i *image) Name() string {
|
||||
@ -186,7 +186,7 @@ func (i *image) Unpack(ctx context.Context, snapshotterName string) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *image) getLayers(ctx context.Context, platform string) ([]rootfs.Layer, error) {
|
||||
func (i *image) getLayers(ctx context.Context, platform platforms.MatchComparer) ([]rootfs.Layer, error) {
|
||||
cs := i.client.ContentStore()
|
||||
|
||||
manifest, err := images.Manifest(ctx, cs, i.i.Target, platform)
|
||||
|
@ -182,9 +182,47 @@ func SetChildrenLabels(manager content.Manager, f HandlerFunc) HandlerFunc {
|
||||
}
|
||||
}
|
||||
|
||||
// FilterPlatforms is a handler wrapper which limits the descriptors returned
|
||||
// FilterPlatformList is a handler wrapper which limits the descriptors returned
|
||||
// by a handler to the specified platforms.
|
||||
func FilterPlatforms(f HandlerFunc, platformList ...string) HandlerFunc {
|
||||
func FilterPlatformList(f HandlerFunc, platformList ...string) HandlerFunc {
|
||||
return func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
|
||||
children, err := f(ctx, desc)
|
||||
if err != nil {
|
||||
return children, err
|
||||
}
|
||||
|
||||
if len(platformList) == 0 {
|
||||
return children, nil
|
||||
}
|
||||
|
||||
var m platforms.Matcher
|
||||
|
||||
if len(platformList) > 0 {
|
||||
ps := make([]ocispec.Platform, len(platformList))
|
||||
for i, platform := range platformList {
|
||||
p, err := platforms.Parse(platform)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ps[i] = p
|
||||
}
|
||||
m = platforms.Any(ps...)
|
||||
}
|
||||
|
||||
var descs []ocispec.Descriptor
|
||||
for _, d := range children {
|
||||
if d.Platform == nil || m.Match(*d.Platform) {
|
||||
descs = append(descs, d)
|
||||
}
|
||||
}
|
||||
|
||||
return descs, nil
|
||||
}
|
||||
}
|
||||
|
||||
// FilterPlatforms is a handler wrapper which limits the descriptors returned
|
||||
// based on matching the specified platform matcher.
|
||||
func FilterPlatforms(f HandlerFunc, m platforms.Matcher) HandlerFunc {
|
||||
return func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
|
||||
children, err := f(ctx, desc)
|
||||
if err != nil {
|
||||
@ -193,20 +231,12 @@ func FilterPlatforms(f HandlerFunc, platformList ...string) HandlerFunc {
|
||||
|
||||
var descs []ocispec.Descriptor
|
||||
|
||||
if len(platformList) == 0 {
|
||||
if m == nil {
|
||||
descs = children
|
||||
} else {
|
||||
for _, platform := range platformList {
|
||||
p, err := platforms.Parse(platform)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
matcher := platforms.NewMatcher(p)
|
||||
|
||||
for _, d := range children {
|
||||
if d.Platform == nil || matcher.Match(*d.Platform) {
|
||||
descs = append(descs, d)
|
||||
}
|
||||
for _, d := range children {
|
||||
if d.Platform == nil || m.Match(*d.Platform) {
|
||||
descs = append(descs, d)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ package images
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"sort"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
@ -93,7 +94,7 @@ type Store interface {
|
||||
//
|
||||
// The caller can then use the descriptor to resolve and process the
|
||||
// configuration of the image.
|
||||
func (image *Image) Config(ctx context.Context, provider content.Provider, platform string) (ocispec.Descriptor, error) {
|
||||
func (image *Image) Config(ctx context.Context, provider content.Provider, platform platforms.MatchComparer) (ocispec.Descriptor, error) {
|
||||
return Config(ctx, provider, image.Target, platform)
|
||||
}
|
||||
|
||||
@ -101,7 +102,7 @@ func (image *Image) Config(ctx context.Context, provider content.Provider, platf
|
||||
//
|
||||
// These are used to verify that a set of layers unpacked to the expected
|
||||
// values.
|
||||
func (image *Image) RootFS(ctx context.Context, provider content.Provider, platform string) ([]digest.Digest, error) {
|
||||
func (image *Image) RootFS(ctx context.Context, provider content.Provider, platform platforms.MatchComparer) ([]digest.Digest, error) {
|
||||
desc, err := image.Config(ctx, provider, platform)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -110,7 +111,7 @@ func (image *Image) RootFS(ctx context.Context, provider content.Provider, platf
|
||||
}
|
||||
|
||||
// Size returns the total size of an image's packed resources.
|
||||
func (image *Image) Size(ctx context.Context, provider content.Provider, platform string) (int64, error) {
|
||||
func (image *Image) Size(ctx context.Context, provider content.Provider, platform platforms.MatchComparer) (int64, error) {
|
||||
var size int64
|
||||
return size, Walk(ctx, Handlers(HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
|
||||
if desc.Size < 0 {
|
||||
@ -121,27 +122,22 @@ func (image *Image) Size(ctx context.Context, provider content.Provider, platfor
|
||||
}), FilterPlatforms(ChildrenHandler(provider), platform)), image.Target)
|
||||
}
|
||||
|
||||
type platformManifest struct {
|
||||
p *ocispec.Platform
|
||||
m *ocispec.Manifest
|
||||
}
|
||||
|
||||
// Manifest resolves a manifest from the image for the given platform.
|
||||
//
|
||||
// TODO(stevvooe): This violates the current platform agnostic approach to this
|
||||
// package by returning a specific manifest type. We'll need to refactor this
|
||||
// to return a manifest descriptor or decide that we want to bring the API in
|
||||
// this direction because this abstraction is not needed.`
|
||||
func Manifest(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform string) (ocispec.Manifest, error) {
|
||||
func Manifest(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform platforms.MatchComparer) (ocispec.Manifest, error) {
|
||||
var (
|
||||
matcher platforms.Matcher
|
||||
m *ocispec.Manifest
|
||||
p ocispec.Platform
|
||||
m []platformManifest
|
||||
wasIndex bool
|
||||
)
|
||||
if platform != "" {
|
||||
var err error
|
||||
p, err = platforms.Parse(platform)
|
||||
if err != nil {
|
||||
return ocispec.Manifest{}, err
|
||||
}
|
||||
matcher = platforms.NewMatcher(p)
|
||||
}
|
||||
|
||||
if err := Walk(ctx, HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
|
||||
switch desc.MediaType {
|
||||
@ -156,8 +152,8 @@ func Manifest(ctx context.Context, provider content.Provider, image ocispec.Desc
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if platform != "" {
|
||||
if desc.Platform != nil && !matcher.Match(*desc.Platform) {
|
||||
if platform != nil {
|
||||
if desc.Platform != nil && !platform.Match(*desc.Platform) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
@ -172,14 +168,17 @@ func Manifest(ctx context.Context, provider content.Provider, image ocispec.Desc
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if !matcher.Match(platforms.Normalize(ocispec.Platform{OS: image.OS, Architecture: image.Architecture})) {
|
||||
if !platform.Match(platforms.Normalize(ocispec.Platform{OS: image.OS, Architecture: image.Architecture})) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
m = &manifest
|
||||
m = append(m, platformManifest{
|
||||
p: desc.Platform,
|
||||
m: &manifest,
|
||||
})
|
||||
|
||||
return nil, nil
|
||||
case MediaTypeDockerSchema2ManifestList, ocispec.MediaTypeImageIndex:
|
||||
@ -193,13 +192,13 @@ func Manifest(ctx context.Context, provider content.Provider, image ocispec.Desc
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if platform == "" {
|
||||
if platform == nil {
|
||||
return idx.Manifests, nil
|
||||
}
|
||||
|
||||
var descs []ocispec.Descriptor
|
||||
for _, d := range idx.Manifests {
|
||||
if d.Platform == nil || matcher.Match(*d.Platform) {
|
||||
if d.Platform == nil || platform.Match(*d.Platform) {
|
||||
descs = append(descs, d)
|
||||
}
|
||||
}
|
||||
@ -214,15 +213,25 @@ func Manifest(ctx context.Context, provider content.Provider, image ocispec.Desc
|
||||
return ocispec.Manifest{}, err
|
||||
}
|
||||
|
||||
if m == nil {
|
||||
if len(m) == 0 {
|
||||
err := errors.Wrapf(errdefs.ErrNotFound, "manifest %v", image.Digest)
|
||||
if wasIndex {
|
||||
err = errors.Wrapf(errdefs.ErrNotFound, "no match for current platform %s in manifest %v", platforms.Format(p), image.Digest)
|
||||
err = errors.Wrapf(errdefs.ErrNotFound, "no match for platform in manifest %v", image.Digest)
|
||||
}
|
||||
return ocispec.Manifest{}, err
|
||||
}
|
||||
|
||||
return *m, nil
|
||||
sort.SliceStable(m, func(i, j int) bool {
|
||||
if m[i].p == nil {
|
||||
return false
|
||||
}
|
||||
if m[j].p == nil {
|
||||
return true
|
||||
}
|
||||
return platform.Less(*m[i].p, *m[j].p)
|
||||
})
|
||||
|
||||
return *m[0].m, nil
|
||||
}
|
||||
|
||||
// Config resolves the image configuration descriptor using a content provided
|
||||
@ -230,7 +239,7 @@ func Manifest(ctx context.Context, provider content.Provider, image ocispec.Desc
|
||||
//
|
||||
// The caller can then use the descriptor to resolve and process the
|
||||
// configuration of the image.
|
||||
func Config(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform string) (ocispec.Descriptor, error) {
|
||||
func Config(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform platforms.MatchComparer) (ocispec.Descriptor, error) {
|
||||
manifest, err := Manifest(ctx, provider, image, platform)
|
||||
if err != nil {
|
||||
return ocispec.Descriptor{}, err
|
||||
@ -276,7 +285,7 @@ func Platforms(ctx context.Context, provider content.Provider, image ocispec.Des
|
||||
// in the provider.
|
||||
//
|
||||
// If there is a problem resolving content, an error will be returned.
|
||||
func Check(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform string) (available bool, required, present, missing []ocispec.Descriptor, err error) {
|
||||
func Check(ctx context.Context, provider content.Provider, image ocispec.Descriptor, platform platforms.MatchComparer) (available bool, required, present, missing []ocispec.Descriptor, err error) {
|
||||
mfst, err := Manifest(ctx, provider, image, platform)
|
||||
if err != nil {
|
||||
if errdefs.IsNotFound(err) {
|
||||
|
192
platforms/compare.go
Normal file
192
platforms/compare.go
Normal file
@ -0,0 +1,192 @@
|
||||
/*
|
||||
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 platforms
|
||||
|
||||
import specs "github.com/opencontainers/image-spec/specs-go/v1"
|
||||
|
||||
// MatchComparer is able to match and compare platforms to
|
||||
// filter and sort platforms.
|
||||
type MatchComparer interface {
|
||||
Matcher
|
||||
|
||||
Less(specs.Platform, specs.Platform) bool
|
||||
}
|
||||
|
||||
// Only returns a match comparer for a single platform
|
||||
// using default resolution logic for the platform.
|
||||
//
|
||||
// For ARMv7, will also match ARMv6 and ARMv5
|
||||
// For ARMv6, will also match ARMv5
|
||||
func Only(platform specs.Platform) MatchComparer {
|
||||
platform = Normalize(platform)
|
||||
if platform.Architecture == "arm" {
|
||||
if platform.Variant == "v7" {
|
||||
return orderedPlatformComparer{
|
||||
matchers: []Matcher{
|
||||
&matcher{
|
||||
Platform: platform,
|
||||
},
|
||||
&matcher{
|
||||
Platform: specs.Platform{
|
||||
Architecture: platform.Architecture,
|
||||
OS: platform.OS,
|
||||
OSVersion: platform.OSVersion,
|
||||
OSFeatures: platform.OSFeatures,
|
||||
Variant: "v6",
|
||||
},
|
||||
},
|
||||
&matcher{
|
||||
Platform: specs.Platform{
|
||||
Architecture: platform.Architecture,
|
||||
OS: platform.OS,
|
||||
OSVersion: platform.OSVersion,
|
||||
OSFeatures: platform.OSFeatures,
|
||||
Variant: "v5",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
if platform.Variant == "v6" {
|
||||
return orderedPlatformComparer{
|
||||
matchers: []Matcher{
|
||||
&matcher{
|
||||
Platform: platform,
|
||||
},
|
||||
&matcher{
|
||||
Platform: specs.Platform{
|
||||
Architecture: platform.Architecture,
|
||||
OS: platform.OS,
|
||||
OSVersion: platform.OSVersion,
|
||||
OSFeatures: platform.OSFeatures,
|
||||
Variant: "v5",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return singlePlatformComparer{
|
||||
Matcher: &matcher{
|
||||
Platform: platform,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Ordered returns a platform MatchComparer which matches any of the platforms
|
||||
// but orders them in order they are provided.
|
||||
func Ordered(platforms ...specs.Platform) MatchComparer {
|
||||
matchers := make([]Matcher, len(platforms))
|
||||
for i := range platforms {
|
||||
matchers[i] = NewMatcher(platforms[i])
|
||||
}
|
||||
return orderedPlatformComparer{
|
||||
matchers: matchers,
|
||||
}
|
||||
}
|
||||
|
||||
// Any returns a platform MatchComparer which matches any of the platforms
|
||||
// with no preference for ordering.
|
||||
func Any(platforms ...specs.Platform) MatchComparer {
|
||||
matchers := make([]Matcher, len(platforms))
|
||||
for i := range platforms {
|
||||
matchers[i] = NewMatcher(platforms[i])
|
||||
}
|
||||
return anyPlatformComparer{
|
||||
matchers: matchers,
|
||||
}
|
||||
}
|
||||
|
||||
// All is a platform MatchComparer which matches all platforms
|
||||
// with preference for ordering.
|
||||
var All MatchComparer = allPlatformComparer{}
|
||||
|
||||
type singlePlatformComparer struct {
|
||||
Matcher
|
||||
}
|
||||
|
||||
func (c singlePlatformComparer) Less(p1, p2 specs.Platform) bool {
|
||||
return c.Match(p1) && !c.Match(p2)
|
||||
}
|
||||
|
||||
type orderedPlatformComparer struct {
|
||||
matchers []Matcher
|
||||
}
|
||||
|
||||
func (c orderedPlatformComparer) Match(platform specs.Platform) bool {
|
||||
for _, m := range c.matchers {
|
||||
if m.Match(platform) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (c orderedPlatformComparer) Less(p1 specs.Platform, p2 specs.Platform) bool {
|
||||
for _, m := range c.matchers {
|
||||
p1m := m.Match(p1)
|
||||
p2m := m.Match(p2)
|
||||
if p1m && !p2m {
|
||||
return true
|
||||
}
|
||||
if p1m || p2m {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type anyPlatformComparer struct {
|
||||
matchers []Matcher
|
||||
}
|
||||
|
||||
func (c anyPlatformComparer) Match(platform specs.Platform) bool {
|
||||
for _, m := range c.matchers {
|
||||
if m.Match(platform) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (c anyPlatformComparer) Less(p1, p2 specs.Platform) bool {
|
||||
var p1m, p2m bool
|
||||
for _, m := range c.matchers {
|
||||
if !p1m && m.Match(p1) {
|
||||
p1m = true
|
||||
}
|
||||
if !p2m && m.Match(p2) {
|
||||
p2m = true
|
||||
}
|
||||
if p1m && p2m {
|
||||
return false
|
||||
}
|
||||
}
|
||||
// If one matches, and the other does, sort match first
|
||||
return p1m && !p2m
|
||||
}
|
||||
|
||||
type allPlatformComparer struct{}
|
||||
|
||||
func (allPlatformComparer) Match(specs.Platform) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (allPlatformComparer) Less(specs.Platform, specs.Platform) bool {
|
||||
return false
|
||||
}
|
@ -22,8 +22,13 @@ import (
|
||||
specs "github.com/opencontainers/image-spec/specs-go/v1"
|
||||
)
|
||||
|
||||
// Default returns the default specifier for the platform.
|
||||
func Default() string {
|
||||
// Default returns the default matcher for the platform.
|
||||
func Default() MatchComparer {
|
||||
return Only(DefaultSpec())
|
||||
}
|
||||
|
||||
// DefaultString returns the default string specifier for the platform.
|
||||
func DefaultString() string {
|
||||
return Format(DefaultSpec())
|
||||
}
|
||||
|
||||
|
@ -35,7 +35,7 @@ func TestDefault(t *testing.T) {
|
||||
t.Fatalf("default platform not as expected: %#v != %#v", p, expected)
|
||||
}
|
||||
|
||||
s := Default()
|
||||
s := DefaultString()
|
||||
if s != Format(p) {
|
||||
t.Fatalf("default specifier should match formatted default spec: %v != %v", s, p)
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ import (
|
||||
"github.com/containerd/containerd/errdefs"
|
||||
"github.com/containerd/containerd/images"
|
||||
"github.com/containerd/containerd/log"
|
||||
"github.com/containerd/containerd/platforms"
|
||||
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
@ -155,7 +156,7 @@ func push(ctx context.Context, provider content.Provider, pusher Pusher, desc oc
|
||||
//
|
||||
// Base handlers can be provided which will be called before any push specific
|
||||
// handlers.
|
||||
func PushContent(ctx context.Context, pusher Pusher, desc ocispec.Descriptor, provider content.Provider, platforms []string, baseHandlers ...images.Handler) error {
|
||||
func PushContent(ctx context.Context, pusher Pusher, desc ocispec.Descriptor, provider content.Provider, platform platforms.MatchComparer, baseHandlers ...images.Handler) error {
|
||||
var m sync.Mutex
|
||||
manifestStack := []ocispec.Descriptor{}
|
||||
|
||||
@ -175,7 +176,7 @@ func PushContent(ctx context.Context, pusher Pusher, desc ocispec.Descriptor, pr
|
||||
pushHandler := PushHandler(pusher, provider)
|
||||
|
||||
handlers := append(baseHandlers,
|
||||
images.FilterPlatforms(images.ChildrenHandler(provider), platforms...),
|
||||
images.FilterPlatforms(images.ChildrenHandler(provider), platform),
|
||||
filterHandler,
|
||||
pushHandler,
|
||||
)
|
||||
|
Loading…
Reference in New Issue
Block a user