Add unpack interface to be used by client

Move client unpacker to pkg/unpack

Signed-off-by: Derek McGowan <derek@mcg.dev>
This commit is contained in:
Derek McGowan 2022-03-25 18:08:34 -07:00
parent 030c1ac1ca
commit 8017daa12d
No known key found for this signature in database
GPG Key ID: F58C5D0A4405ACDB
2 changed files with 245 additions and 142 deletions

View File

@ -14,7 +14,7 @@
limitations under the License. limitations under the License.
*/ */
package containerd package unpack
import ( import (
"context" "context"
@ -28,6 +28,7 @@ import (
"time" "time"
"github.com/containerd/containerd/content" "github.com/containerd/containerd/content"
"github.com/containerd/containerd/diff"
"github.com/containerd/containerd/errdefs" "github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/images" "github.com/containerd/containerd/images"
"github.com/containerd/containerd/log" "github.com/containerd/containerd/log"
@ -47,48 +48,179 @@ const (
labelSnapshotRef = "containerd.io/snapshot.ref" labelSnapshotRef = "containerd.io/snapshot.ref"
) )
type unpacker struct { // Result returns information about the unpacks which were completed.
updateCh chan ocispec.Descriptor type Result struct {
snapshotter string Unpacks int
config UnpackConfig
c *Client
limiter *semaphore.Weighted
} }
func (c *Client) newUnpacker(ctx context.Context, rCtx *RemoteContext) (*unpacker, error) { type unpackerConfig struct {
snapshotter, err := c.resolveSnapshotterName(ctx, rCtx.Snapshotter) platforms []*Platform
if err != nil {
return nil, err content content.Store
limiter *semaphore.Weighted
duplicationSuppressor kmutex.KeyedLocker
}
// Platform represents a platform-specific unpack configuration which includes
// the platform matcher as well as snapshotter and applier.
type Platform struct {
Platform platforms.Matcher
SnapshotterKey string
Snapshotter snapshots.Snapshotter
SnapshotOpts []snapshots.Opt
Applier diff.Applier
ApplyOpts []diff.ApplyOpt
}
type UnpackerOpt func(*unpackerConfig) error
func WithUnpackPlatform(u Platform) UnpackerOpt {
return UnpackerOpt(func(c *unpackerConfig) error {
if u.Platform == nil {
u.Platform = platforms.All
}
if u.Snapshotter == nil {
return fmt.Errorf("snapshotter must be provided to unpack")
}
if u.SnapshotterKey == "" {
if s, ok := u.Snapshotter.(fmt.Stringer); ok {
u.SnapshotterKey = s.String()
} else {
u.SnapshotterKey = "unknown"
}
}
if u.Applier == nil {
return fmt.Errorf("applier must be provided to unpack")
}
c.platforms = append(c.platforms, &u)
return nil
})
}
func WithLimiter(l *semaphore.Weighted) UnpackerOpt {
return UnpackerOpt(func(c *unpackerConfig) error {
c.limiter = l
return nil
})
}
func WithDuplicationSuppressor(d kmutex.KeyedLocker) UnpackerOpt {
return UnpackerOpt(func(c *unpackerConfig) error {
c.duplicationSuppressor = d
return nil
})
}
// Unpacker unpacks images by hooking into the image handler process.
// Unpacks happen in the backgrounds and waited on to complete.
type Unpacker struct {
unpackerConfig
unpacks int32
ctx context.Context
eg *errgroup.Group
}
// NewUnpacker creates a new instance of the unpacker which can be used to wrap an
// image handler and unpack in parallel to handling. The unpacker will handle
// calling the block handlers when they are needed by the unpack process.
func NewUnpacker(ctx context.Context, cs content.Store, opts ...UnpackerOpt) (*Unpacker, error) {
eg, ctx := errgroup.WithContext(ctx)
u := &Unpacker{
unpackerConfig: unpackerConfig{
content: cs,
duplicationSuppressor: kmutex.NewNoop(),
},
ctx: ctx,
eg: eg,
} }
var config = UnpackConfig{ for _, opt := range opts {
DuplicationSuppressor: kmutex.NewNoop(), if err := opt(&u.unpackerConfig); err != nil {
}
for _, o := range rCtx.UnpackOpts {
if err := o(ctx, &config); err != nil {
return nil, err return nil, err
} }
} }
var limiter *semaphore.Weighted return u, nil
if rCtx.MaxConcurrentDownloads > 0 { }
limiter = semaphore.NewWeighted(int64(rCtx.MaxConcurrentDownloads))
// Unpack wraps an image handler to filter out blob handling and scheduling them
// during the unpack process. When an image config is encountered, the unpack
// process will be started in a goroutine.
func (u *Unpacker) Unpack(h images.Handler) images.Handler {
var (
lock sync.Mutex
layers = map[digest.Digest][]ocispec.Descriptor{}
)
return images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
unlock, err := u.lockBlobDescriptor(ctx, desc)
if err != nil {
return nil, err
}
children, err := h.Handle(ctx, desc)
unlock()
if err != nil {
return children, err
}
switch desc.MediaType {
case images.MediaTypeDockerSchema2Manifest, ocispec.MediaTypeImageManifest:
var nonLayers []ocispec.Descriptor
var manifestLayers []ocispec.Descriptor
// Split layers from non-layers, layers will be handled after
// the config
for _, child := range children {
if images.IsLayerType(child.MediaType) {
manifestLayers = append(manifestLayers, child)
} else {
nonLayers = append(nonLayers, child)
}
}
lock.Lock()
for _, nl := range nonLayers {
layers[nl.Digest] = manifestLayers
}
lock.Unlock()
children = nonLayers
case images.MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig:
lock.Lock()
l := layers[desc.Digest]
lock.Unlock()
if len(l) > 0 {
u.eg.Go(func() error {
return u.unpack(h, desc, l)
})
}
}
return children, nil
})
}
// Wait waits for any ongoing unpack processes to complete then will return
// the result.
func (u *Unpacker) Wait() (Result, error) {
if err := u.eg.Wait(); err != nil {
return Result{}, err
} }
return &unpacker{ return Result{
updateCh: make(chan ocispec.Descriptor, 128), Unpacks: int(u.unpacks),
snapshotter: snapshotter,
config: config,
c: c,
limiter: limiter,
}, nil }, nil
} }
func (u *unpacker) unpack( func (u *Unpacker) unpack(
ctx context.Context,
rCtx *RemoteContext,
h images.Handler, h images.Handler,
config ocispec.Descriptor, config ocispec.Descriptor,
layers []ocispec.Descriptor, layers []ocispec.Descriptor,
) error { ) error {
p, err := content.ReadBlob(ctx, u.c.ContentStore(), config) ctx := u.ctx
p, err := content.ReadBlob(ctx, u.content, config)
if err != nil { if err != nil {
return err return err
} }
@ -102,21 +234,27 @@ func (u *unpacker) unpack(
return fmt.Errorf("number of layers and diffIDs don't match: %d != %d", len(layers), len(diffIDs)) return fmt.Errorf("number of layers and diffIDs don't match: %d != %d", len(layers), len(diffIDs))
} }
if u.config.CheckPlatformSupported { // TODO: Support multiple unpacks rather than just first match
imgPlatform := platforms.Normalize(ocispec.Platform{OS: i.OS, Architecture: i.Architecture}) var unpack *Platform
snapshotterPlatformMatcher, err := u.c.GetSnapshotterSupportedPlatforms(ctx, u.snapshotter)
if err != nil { imgPlatform := platforms.Normalize(ocispec.Platform{OS: i.OS, Architecture: i.Architecture})
return fmt.Errorf("failed to find supported platforms for snapshotter %s: %w", u.snapshotter, err) for _, up := range u.platforms {
} if up.Platform.Match(imgPlatform) {
if !snapshotterPlatformMatcher.Match(imgPlatform) { unpack = up
return fmt.Errorf("snapshotter %s does not support platform %s for image %s", u.snapshotter, imgPlatform, config.Digest) break
} }
} }
if unpack == nil {
return fmt.Errorf("unpacker does not support platform %s for image %s", imgPlatform, config.Digest)
}
atomic.AddInt32(&u.unpacks, 1)
var ( var (
sn = u.c.SnapshotService(u.snapshotter) sn = unpack.Snapshotter
a = u.c.DiffService() a = unpack.Applier
cs = u.c.ContentStore() cs = u.content
chain []digest.Digest chain []digest.Digest
@ -135,7 +273,7 @@ func (u *unpacker) unpack(
chain = append(chain, diffIDs[i]) chain = append(chain, diffIDs[i])
chainID := identity.ChainID(chain).String() chainID := identity.ChainID(chain).String()
unlock, err := u.lockSnChainID(ctx, chainID) unlock, err := u.lockSnChainID(ctx, chainID, unpack.SnapshotterKey)
if err != nil { if err != nil {
return err return err
} }
@ -158,7 +296,7 @@ func (u *unpacker) unpack(
var ( var (
key string key string
mounts []mount.Mount mounts []mount.Mount
opts = append(rCtx.SnapshotterOpts, snapshots.WithLabels(labels)) opts = append(unpack.SnapshotOpts, snapshots.WithLabels(labels))
) )
for try := 1; try <= 3; try++ { for try := 1; try <= 3; try++ {
@ -214,15 +352,17 @@ func (u *unpacker) unpack(
select { select {
case <-ctx.Done(): case <-ctx.Done():
abort()
return ctx.Err() return ctx.Err()
case err := <-fetchErr: case err := <-fetchErr:
if err != nil { if err != nil {
abort()
return err return err
} }
case <-fetchC[i-fetchOffset]: case <-fetchC[i-fetchOffset]:
} }
diff, err := a.Apply(ctx, desc, mounts, u.config.ApplyOpts...) diff, err := a.Apply(ctx, desc, mounts, unpack.ApplyOpts...)
if err != nil { if err != nil {
abort() abort()
return fmt.Errorf("failed to extract layer %s: %w", diffIDs[i], err) return fmt.Errorf("failed to extract layer %s: %w", diffIDs[i], err)
@ -264,10 +404,10 @@ func (u *unpacker) unpack(
cinfo := content.Info{ cinfo := content.Info{
Digest: config.Digest, Digest: config.Digest,
Labels: map[string]string{ Labels: map[string]string{
fmt.Sprintf("containerd.io/gc.ref.snapshot.%s", u.snapshotter): chainID, fmt.Sprintf("containerd.io/gc.ref.snapshot.%s", unpack.SnapshotterKey): chainID,
}, },
} }
_, err = cs.Update(ctx, cinfo, fmt.Sprintf("labels.containerd.io/gc.ref.snapshot.%s", u.snapshotter)) _, err = cs.Update(ctx, cinfo, fmt.Sprintf("labels.containerd.io/gc.ref.snapshot.%s", unpack.SnapshotterKey))
if err != nil { if err != nil {
return err return err
} }
@ -279,7 +419,7 @@ func (u *unpacker) unpack(
return nil return nil
} }
func (u *unpacker) fetch(ctx context.Context, h images.Handler, layers []ocispec.Descriptor, done []chan struct{}) error { func (u *Unpacker) fetch(ctx context.Context, h images.Handler, layers []ocispec.Descriptor, done []chan struct{}) error {
eg, ctx2 := errgroup.WithContext(ctx) eg, ctx2 := errgroup.WithContext(ctx)
for i, desc := range layers { for i, desc := range layers {
desc := desc desc := desc
@ -313,108 +453,47 @@ func (u *unpacker) fetch(ctx context.Context, h images.Handler, layers []ocispec
return eg.Wait() return eg.Wait()
} }
func (u *unpacker) handlerWrapper( func (u *Unpacker) acquire(ctx context.Context) error {
uctx context.Context,
rCtx *RemoteContext,
unpacks *int32,
) (func(images.Handler) images.Handler, *errgroup.Group) {
eg, uctx := errgroup.WithContext(uctx)
return func(f images.Handler) images.Handler {
var (
lock sync.Mutex
layers = map[digest.Digest][]ocispec.Descriptor{}
)
return images.HandlerFunc(func(ctx context.Context, desc ocispec.Descriptor) ([]ocispec.Descriptor, error) {
unlock, err := u.lockBlobDescriptor(ctx, desc)
if err != nil {
return nil, err
}
children, err := f.Handle(ctx, desc)
unlock()
if err != nil {
return children, err
}
switch desc.MediaType {
case images.MediaTypeDockerSchema2Manifest, ocispec.MediaTypeImageManifest:
var nonLayers []ocispec.Descriptor
var manifestLayers []ocispec.Descriptor
// Split layers from non-layers, layers will be handled after
// the config
for _, child := range children {
if images.IsLayerType(child.MediaType) {
manifestLayers = append(manifestLayers, child)
} else {
nonLayers = append(nonLayers, child)
}
}
lock.Lock()
for _, nl := range nonLayers {
layers[nl.Digest] = manifestLayers
}
lock.Unlock()
children = nonLayers
case images.MediaTypeDockerSchema2Config, ocispec.MediaTypeImageConfig:
lock.Lock()
l := layers[desc.Digest]
lock.Unlock()
if len(l) > 0 {
atomic.AddInt32(unpacks, 1)
eg.Go(func() error {
return u.unpack(uctx, rCtx, f, desc, l)
})
}
}
return children, nil
})
}, eg
}
func (u *unpacker) acquire(ctx context.Context) error {
if u.limiter == nil { if u.limiter == nil {
return nil return nil
} }
return u.limiter.Acquire(ctx, 1) return u.limiter.Acquire(ctx, 1)
} }
func (u *unpacker) release() { func (u *Unpacker) release() {
if u.limiter == nil { if u.limiter == nil {
return return
} }
u.limiter.Release(1) u.limiter.Release(1)
} }
func (u *unpacker) lockSnChainID(ctx context.Context, chainID string) (func(), error) { func (u *Unpacker) lockSnChainID(ctx context.Context, chainID, snapshotter string) (func(), error) {
key := u.makeChainIDKeyWithSnapshotter(chainID) key := u.makeChainIDKeyWithSnapshotter(chainID, snapshotter)
if err := u.config.DuplicationSuppressor.Lock(ctx, key); err != nil { if err := u.duplicationSuppressor.Lock(ctx, key); err != nil {
return nil, err return nil, err
} }
return func() { return func() {
u.config.DuplicationSuppressor.Unlock(key) u.duplicationSuppressor.Unlock(key)
}, nil }, nil
} }
func (u *unpacker) lockBlobDescriptor(ctx context.Context, desc ocispec.Descriptor) (func(), error) { func (u *Unpacker) lockBlobDescriptor(ctx context.Context, desc ocispec.Descriptor) (func(), error) {
key := u.makeBlobDescriptorKey(desc) key := u.makeBlobDescriptorKey(desc)
if err := u.config.DuplicationSuppressor.Lock(ctx, key); err != nil { if err := u.duplicationSuppressor.Lock(ctx, key); err != nil {
return nil, err return nil, err
} }
return func() { return func() {
u.config.DuplicationSuppressor.Unlock(key) u.duplicationSuppressor.Unlock(key)
}, nil }, nil
} }
func (u *unpacker) makeChainIDKeyWithSnapshotter(chainID string) string { func (u *Unpacker) makeChainIDKeyWithSnapshotter(chainID, snapshotter string) string {
return fmt.Sprintf("sn://%s/%v", u.snapshotter, chainID) return fmt.Sprintf("sn://%s/%v", snapshotter, chainID)
} }
func (u *unpacker) makeBlobDescriptorKey(desc ocispec.Descriptor) string { func (u *Unpacker) makeBlobDescriptorKey(desc ocispec.Descriptor) string {
return fmt.Sprintf("blob://%v", desc.Digest) return fmt.Sprintf("blob://%v", desc.Digest)
} }

70
pull.go
View File

@ -23,12 +23,12 @@ import (
"github.com/containerd/containerd/errdefs" "github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/images" "github.com/containerd/containerd/images"
"github.com/containerd/containerd/pkg/unpack"
"github.com/containerd/containerd/platforms" "github.com/containerd/containerd/platforms"
"github.com/containerd/containerd/remotes" "github.com/containerd/containerd/remotes"
"github.com/containerd/containerd/remotes/docker" "github.com/containerd/containerd/remotes/docker"
"github.com/containerd/containerd/remotes/docker/schema1" "github.com/containerd/containerd/remotes/docker/schema1"
ocispec "github.com/opencontainers/image-spec/specs-go/v1" ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"golang.org/x/sync/errgroup"
"golang.org/x/sync/semaphore" "golang.org/x/sync/semaphore"
) )
@ -63,19 +63,46 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (_ Ima
} }
defer done(ctx) defer done(ctx)
var unpacks int32 var unpacker *unpack.Unpacker
var unpackEg *errgroup.Group
var unpackWrapper func(f images.Handler) images.Handler
if pullCtx.Unpack { if pullCtx.Unpack {
// unpacker only supports schema 2 image, for schema 1 this is noop. snapshotterName, err := c.resolveSnapshotterName(ctx, pullCtx.Snapshotter)
u, err := c.newUnpacker(ctx, pullCtx)
if err != nil { if err != nil {
return nil, fmt.Errorf("create unpacker: %w", err) return nil, fmt.Errorf("unable to resolve snapshotter: %w", err)
}
var uconfig UnpackConfig
for _, opt := range pullCtx.UnpackOpts {
if err := opt(ctx, &uconfig); err != nil {
return nil, err
}
}
var platformMatcher platforms.Matcher
if !uconfig.CheckPlatformSupported {
platformMatcher = platforms.All
}
// Check client Unpack config
platform := unpack.Platform{
Platform: platformMatcher,
SnapshotterKey: snapshotterName,
Snapshotter: c.SnapshotService(snapshotterName),
SnapshotOpts: append(pullCtx.SnapshotterOpts, uconfig.SnapshotOpts...),
Applier: c.DiffService(),
ApplyOpts: uconfig.ApplyOpts,
}
uopts := []unpack.UnpackerOpt{unpack.WithUnpackPlatform(platform)}
if pullCtx.MaxConcurrentUploadedLayers > 0 {
uopts = append(uopts, unpack.WithLimiter(semaphore.NewWeighted(int64(pullCtx.MaxConcurrentDownloads))))
}
if uconfig.DuplicationSuppressor != nil {
uopts = append(uopts, unpack.WithDuplicationSuppressor(uconfig.DuplicationSuppressor))
}
unpacker, err = unpack.NewUnpacker(ctx, c.ContentStore(), uopts...)
if err != nil {
return nil, fmt.Errorf("unable to initialize unpacker: %w", err)
} }
unpackWrapper, unpackEg = u.handlerWrapper(ctx, pullCtx, &unpacks)
defer func() { defer func() {
if err := unpackEg.Wait(); err != nil { if _, err := unpacker.Wait(); err != nil {
if retErr == nil { if retErr == nil {
retErr = fmt.Errorf("unpack: %w", err) retErr = fmt.Errorf("unpack: %w", err)
} }
@ -84,9 +111,9 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (_ Ima
wrapper := pullCtx.HandlerWrapper wrapper := pullCtx.HandlerWrapper
pullCtx.HandlerWrapper = func(h images.Handler) images.Handler { pullCtx.HandlerWrapper = func(h images.Handler) images.Handler {
if wrapper == nil { if wrapper == nil {
return unpackWrapper(h) return unpacker.Unpack(h)
} }
return unpackWrapper(wrapper(h)) return unpacker.Unpack(wrapper(h))
} }
} }
@ -98,11 +125,10 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (_ Ima
// NOTE(fuweid): unpacker defers blobs download. before create image // NOTE(fuweid): unpacker defers blobs download. before create image
// record in ImageService, should wait for unpacking(including blobs // record in ImageService, should wait for unpacking(including blobs
// download). // download).
if pullCtx.Unpack { var ur unpack.Result
if unpackEg != nil { if unpacker != nil {
if err := unpackEg.Wait(); err != nil { if ur, err = unpacker.Wait(); err != nil {
return nil, err return nil, err
}
} }
} }
@ -113,13 +139,11 @@ func (c *Client) Pull(ctx context.Context, ref string, opts ...RemoteOpt) (_ Ima
i := NewImageWithPlatform(c, img, pullCtx.PlatformMatcher) i := NewImageWithPlatform(c, img, pullCtx.PlatformMatcher)
if pullCtx.Unpack { if unpacker != nil && ur.Unpacks == 0 {
if unpacks == 0 { // Unpack was tried previously but nothing was unpacked
// Try to unpack is none is done previously. // This is at least required for schema 1 image.
// This is at least required for schema 1 image. if err := i.Unpack(ctx, pullCtx.Snapshotter, pullCtx.UnpackOpts...); err != nil {
if err := i.Unpack(ctx, pullCtx.Snapshotter, pullCtx.UnpackOpts...); err != nil { return nil, fmt.Errorf("failed to unpack image on snapshotter %s: %w", pullCtx.Snapshotter, err)
return nil, fmt.Errorf("failed to unpack image on snapshotter %s: %w", pullCtx.Snapshotter, err)
}
} }
} }