瀏覽代碼

Add resolve all images and unit test

Add single resolve function to get a consistent list of images matching
the same digest.

Signed-off-by: Derek McGowan <derek@mcg.dev>
Derek McGowan 1 年之前
父節點
當前提交
529d19bad8

+ 149 - 0
daemon/containerd/image.go

@@ -29,6 +29,8 @@ import (
 
 var truncatedID = regexp.MustCompile(`^(sha256:)?([a-f0-9]{4,64})$`)
 
+var errInconsistentData error = errors.New("consistency error: data changed during operation, retry")
+
 // GetImage returns an image corresponding to the image referred to by refOrID.
 func (i *ImageService) GetImage(ctx context.Context, refOrID string, options imagetype.GetImageOpts) (*image.Image, error) {
 	desc, err := i.resolveImage(ctx, refOrID)
@@ -382,3 +384,150 @@ func (i *ImageService) getImageLabelByDigest(ctx context.Context, target digest.
 
 	return value, nil
 }
+
+func convertError(err error) error {
+	// TODO: Convert containerd error to Docker error
+	return err
+}
+
+// resolveAllReferences resolves the reference name or ID to an image and returns all the images with
+// the same target.
+//
+// Returns:
+//
+// 1: *(github.com/containerd/containerd/images).Image
+//
+//	An image match from the image store with the provided refOrID
+//
+// 2: [](github.com/containerd/containerd/images).Image
+//
+//	List of all images with the same target that matches the refOrID. If the first argument is
+//	non-nil, the image list will all have the same target as the matched image. If the first
+//	argument is nil but the list is non-empty, this value is a list of all the images with a
+//	target that matches the digest provided in the refOrID, but none are an image name match
+//	to refOrID.
+//
+// 3: error
+//
+//	An error looking up refOrID or no images found with matching name or target. Note that the first
+//	argument may be nil with a nil error if the second argument is non-empty.
+func (i *ImageService) resolveAllReferences(ctx context.Context, refOrID string) (*containerdimages.Image, []containerdimages.Image, error) {
+	parsed, err := reference.ParseAnyReference(refOrID)
+	if err != nil {
+		return nil, nil, errdefs.InvalidParameter(err)
+	}
+	var dgst digest.Digest
+	var img *containerdimages.Image
+
+	if truncatedID.MatchString(refOrID) {
+		if d, ok := parsed.(reference.Digested); ok {
+			if cimg, err := i.images.Get(ctx, d.String()); err == nil {
+				img = &cimg
+				dgst = d.Digest()
+				if cimg.Target.Digest != dgst {
+					// Ambiguous image reference, use reference name
+					log.G(ctx).WithField("image", refOrID).WithField("target", cimg.Target.Digest).Warn("digest reference points to image with a different digest")
+					dgst = cimg.Target.Digest
+				}
+			} else if !cerrdefs.IsNotFound(err) {
+				return nil, nil, convertError(err)
+			} else {
+				dgst = d.Digest()
+			}
+		} else {
+			idWithoutAlgo := strings.TrimPrefix(refOrID, "sha256:")
+			name := reference.TagNameOnly(parsed.(reference.Named)).String()
+			filters := []string{
+				fmt.Sprintf("name==%q", name), // Or it could just look like one.
+				"target.digest~=" + strconv.Quote(fmt.Sprintf(`^sha256:%s[0-9a-fA-F]{%d}$`, regexp.QuoteMeta(idWithoutAlgo), 64-len(idWithoutAlgo))),
+			}
+			imgs, err := i.images.List(ctx, filters...)
+			if err != nil {
+				return nil, nil, convertError(err)
+			}
+
+			if len(imgs) == 0 {
+				return nil, nil, images.ErrImageDoesNotExist{Ref: parsed}
+			}
+
+			for _, limg := range imgs {
+				if limg.Name == name {
+					copyImg := limg
+					img = &copyImg
+				}
+				if dgst != "" {
+					if limg.Target.Digest != dgst {
+						return nil, nil, errdefs.NotFound(errors.New("ambiguous reference"))
+					}
+				} else {
+					dgst = limg.Target.Digest
+				}
+			}
+
+			// Return immediately if target digest matches already included
+			if img == nil || len(imgs) > 1 {
+				return img, imgs, nil
+			}
+		}
+	} else {
+		named, ok := parsed.(reference.Named)
+		if !ok {
+			return nil, nil, errdefs.InvalidParameter(errors.New("invalid name reference"))
+		}
+
+		digested, ok := parsed.(reference.Digested)
+		if ok {
+			dgst = digested.Digest()
+		}
+
+		name := reference.TagNameOnly(named).String()
+
+		cimg, err := i.images.Get(ctx, name)
+		if err != nil {
+			if !cerrdefs.IsNotFound(err) {
+				return nil, nil, convertError(err)
+			}
+			return nil, nil, images.ErrImageDoesNotExist{Ref: parsed}
+		} else {
+			img = &cimg
+			if dgst != "" && img.Target.Digest != dgst {
+				// Ambiguous image reference, use reference name
+				log.G(ctx).WithField("image", name).WithField("target", cimg.Target.Digest).Warn("digest reference points to image with a different digest")
+			}
+			dgst = img.Target.Digest
+		}
+
+	}
+
+	// Lookup up all associated images and check for consistency with first reference
+	// Ideally operations dependent on multiple values will rely on the garbage collector,
+	// this logic will just check for consistency and throw an error
+	imgs, err := i.images.List(ctx, "target.digest=="+dgst.String())
+	if err != nil {
+		return nil, nil, errors.Wrap(err, "failed to lookup digest")
+	}
+	if len(imgs) == 0 {
+		if img == nil {
+			return nil, nil, images.ErrImageDoesNotExist{Ref: parsed}
+		}
+		err = errInconsistentData
+	} else if img != nil {
+		// Check to ensure the original img is in the list still
+		err = errInconsistentData
+		for _, rimg := range imgs {
+			if rimg.Name == img.Name {
+				err = nil
+				break
+			}
+		}
+	}
+	if errors.Is(err, errInconsistentData) {
+		if retries, ok := ctx.Value(errInconsistentData).(int); !ok || retries < 3 {
+			log.G(ctx).WithFields(log.Fields{"retry": retries, "ref": refOrID}).Info("image changed during lookup, retrying")
+			return i.resolveAllReferences(context.WithValue(ctx, errInconsistentData, retries+1), refOrID)
+		}
+		return nil, nil, err
+	}
+
+	return img, imgs, nil
+}

+ 287 - 0
daemon/containerd/image_test.go

@@ -0,0 +1,287 @@
+package containerd
+
+import (
+	"context"
+	"io"
+	"math/rand"
+	"path/filepath"
+	"testing"
+
+	"github.com/containerd/containerd/images"
+	"github.com/containerd/containerd/metadata"
+	"github.com/containerd/containerd/namespaces"
+	"github.com/containerd/log/logtest"
+	"github.com/distribution/reference"
+	dockerimages "github.com/docker/docker/daemon/images"
+	"github.com/opencontainers/go-digest"
+	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
+
+	"go.etcd.io/bbolt"
+
+	"gotest.tools/v3/assert"
+	is "gotest.tools/v3/assert/cmp"
+)
+
+func TestLookup(t *testing.T) {
+	ctx := namespaces.WithNamespace(context.TODO(), "testing")
+	ctx = logtest.WithT(ctx, t)
+	mdb := newTestDB(ctx, t)
+	service := &ImageService{
+		images: metadata.NewImageStore(mdb),
+	}
+
+	ubuntuLatest := images.Image{
+		Name:   "docker.io/library/ubuntu:latest",
+		Target: desc(10),
+	}
+	ubuntuLatestWithDigest := images.Image{
+		Name:   "docker.io/library/ubuntu:latest@" + digestFor(10).String(),
+		Target: desc(10),
+	}
+	ubuntuLatestWithOldDigest := images.Image{
+		Name:   "docker.io/library/ubuntu:latest@" + digestFor(11).String(),
+		Target: desc(11),
+	}
+	ambiguousShortName := images.Image{
+		Name:   "docker.io/library/abcdef:latest",
+		Target: desc(12),
+	}
+	ambiguousShortNameWithDigest := images.Image{
+		Name:   "docker.io/library/abcdef:latest@" + digestFor(12).String(),
+		Target: desc(12),
+	}
+	shortNameIsHashAlgorithm := images.Image{
+		Name:   "docker.io/library/sha256:defcab",
+		Target: desc(13),
+	}
+
+	testImages := []images.Image{
+		ubuntuLatest,
+		ubuntuLatestWithDigest,
+		ubuntuLatestWithOldDigest,
+		ambiguousShortName,
+		ambiguousShortNameWithDigest,
+		shortNameIsHashAlgorithm,
+		{
+			Name:   "docker.io/test/volatile:retried",
+			Target: desc(14),
+		},
+		{
+			Name:   "docker.io/test/volatile:inconsistent",
+			Target: desc(15),
+		},
+	}
+	for _, img := range testImages {
+		if _, err := service.images.Create(ctx, img); err != nil {
+			t.Fatalf("failed to create image %q: %v", img.Name, err)
+		}
+	}
+
+	for _, tc := range []struct {
+		lookup string
+		img    *images.Image
+		all    []images.Image
+		err    error
+	}{
+		{
+			// Get ubuntu images with default "latest" tag
+			lookup: "ubuntu",
+			img:    &ubuntuLatest,
+			all:    []images.Image{ubuntuLatest, ubuntuLatestWithDigest},
+		},
+		{
+			// Get all images by image id
+			lookup: ubuntuLatest.Target.Digest.String(),
+			img:    nil,
+			all:    []images.Image{ubuntuLatest, ubuntuLatestWithDigest},
+		},
+		{
+			// Fail to lookup reference with no tag, reference has both tag and digest
+			lookup: "ubuntu@" + ubuntuLatestWithOldDigest.Target.Digest.String(),
+			err:    dockerimages.ErrImageDoesNotExist{Ref: nameDigest("ubuntu", ubuntuLatestWithOldDigest.Target.Digest)},
+		},
+		{
+			// Get all image with both tag and digest
+			lookup: "ubuntu:latest@" + ubuntuLatestWithOldDigest.Target.Digest.String(),
+			img:    &ubuntuLatestWithOldDigest,
+			all:    []images.Image{ubuntuLatestWithOldDigest},
+		},
+		{
+			// Get abcdef image which also matches short image id
+			lookup: "abcdef",
+			img:    &ambiguousShortName,
+			all:    []images.Image{ambiguousShortName, ambiguousShortNameWithDigest},
+		},
+		{
+			// Fail to lookup image named "sha256" with tag that doesn't exist
+			lookup: "sha256:abcdef",
+			err:    dockerimages.ErrImageDoesNotExist{Ref: nameTag("sha256", "abcdef")},
+		},
+		{
+			// Lookup with shortened image id
+			lookup: ambiguousShortName.Target.Digest.Encoded()[:8],
+			img:    nil,
+			all:    []images.Image{ambiguousShortName, ambiguousShortNameWithDigest},
+		},
+		{
+			// Lookup an actual image named "sha256" in the default namespace
+			lookup: "sha256:defcab",
+			img:    &shortNameIsHashAlgorithm,
+			all:    []images.Image{shortNameIsHashAlgorithm},
+		},
+	} {
+		tc := tc
+		t.Run(tc.lookup, func(t *testing.T) {
+			t.Parallel()
+			img, all, err := service.resolveAllReferences(ctx, tc.lookup)
+			if tc.err == nil {
+				assert.NilError(t, err)
+			} else {
+				assert.Error(t, err, tc.err.Error())
+			}
+			if tc.img == nil {
+				assert.Assert(t, is.Nil(img))
+			} else {
+				assert.Assert(t, img != nil)
+				assert.Check(t, is.Equal(img.Name, tc.img.Name))
+				assert.Check(t, is.Equal(img.Target.Digest, tc.img.Target.Digest))
+			}
+
+			assert.Assert(t, is.Len(tc.all, len(all)))
+
+			// Order should match
+			for i := range all {
+				assert.Check(t, is.Equal(all[i].Name, tc.all[i].Name), "image[%d]", i)
+				assert.Check(t, is.Equal(all[i].Target.Digest, tc.all[i].Target.Digest), "image[%d]", i)
+			}
+		})
+	}
+
+	t.Run("fail-inconsistency", func(t *testing.T) {
+		service := &ImageService{
+			images: &mutateOnGetImageStore{
+				Store: service.images,
+				getMutations: []images.Image{
+					{
+						Name:   "docker.io/test/volatile:inconsistent",
+						Target: desc(18),
+					},
+					{
+						Name:   "docker.io/test/volatile:inconsistent",
+						Target: desc(19),
+					},
+					{
+						Name:   "docker.io/test/volatile:inconsistent",
+						Target: desc(20),
+					},
+					{
+						Name:   "docker.io/test/volatile:inconsistent",
+						Target: desc(21),
+					},
+					{
+						Name:   "docker.io/test/volatile:inconsistent",
+						Target: desc(22),
+					},
+				},
+				t: t,
+			},
+		}
+
+		_, _, err := service.resolveAllReferences(ctx, "test/volatile:inconsistent")
+		assert.ErrorIs(t, err, errInconsistentData)
+	})
+
+	t.Run("retry-inconsistency", func(t *testing.T) {
+		service := &ImageService{
+			images: &mutateOnGetImageStore{
+				Store: service.images,
+				getMutations: []images.Image{
+					{
+						Name:   "docker.io/test/volatile:retried",
+						Target: desc(16),
+					},
+					{
+						Name:   "docker.io/test/volatile:retried",
+						Target: desc(17),
+					},
+				},
+				t: t,
+			},
+		}
+
+		img, all, err := service.resolveAllReferences(ctx, "test/volatile:retried")
+		assert.NilError(t, err)
+
+		assert.Assert(t, img != nil)
+		assert.Check(t, is.Equal(img.Name, "docker.io/test/volatile:retried"))
+		assert.Check(t, is.Equal(img.Target.Digest, digestFor(17)))
+		assert.Assert(t, is.Len(all, 1))
+		assert.Check(t, is.Equal(all[0].Name, "docker.io/test/volatile:retried"))
+		assert.Check(t, is.Equal(all[0].Target.Digest, digestFor(17)))
+	})
+}
+
+type mutateOnGetImageStore struct {
+	images.Store
+	getMutations []images.Image
+	t            *testing.T
+}
+
+func (m *mutateOnGetImageStore) Get(ctx context.Context, name string) (images.Image, error) {
+	img, err := m.Store.Get(ctx, name)
+	if len(m.getMutations) > 0 {
+		m.Store.Update(ctx, m.getMutations[0])
+		m.getMutations = m.getMutations[1:]
+		m.t.Logf("Get %s", name)
+	}
+	return img, err
+}
+
+func nameDigest(name string, dgst digest.Digest) reference.Reference {
+	named, _ := reference.WithName(name)
+	digested, _ := reference.WithDigest(named, dgst)
+	return digested
+}
+
+func nameTag(name, tag string) reference.Reference {
+	named, _ := reference.WithName(name)
+	tagged, _ := reference.WithTag(named, tag)
+	return tagged
+}
+
+func desc(size int64) ocispec.Descriptor {
+	return ocispec.Descriptor{
+		Digest:    digestFor(size),
+		Size:      size,
+		MediaType: ocispec.MediaTypeImageIndex,
+	}
+
+}
+
+func digestFor(i int64) digest.Digest {
+	r := rand.New(rand.NewSource(i))
+	dgstr := digest.SHA256.Digester()
+	_, err := io.Copy(dgstr.Hash(), io.LimitReader(r, i))
+	if err != nil {
+		panic(err)
+	}
+	return dgstr.Digest()
+}
+
+func newTestDB(ctx context.Context, t *testing.T) *metadata.DB {
+	t.Helper()
+
+	p := filepath.Join(t.TempDir(), "metadata")
+	bdb, err := bbolt.Open(p, 0600, &bbolt.Options{})
+	if err != nil {
+		t.Fatal(err)
+	}
+	t.Cleanup(func() { bdb.Close() })
+
+	mdb := metadata.NewDB(bdb, nil, nil)
+	if err := mdb.Init(ctx); err != nil {
+		t.Fatal(err)
+	}
+
+	return mdb
+}

+ 6 - 3
daemon/containerd/service.go

@@ -7,6 +7,7 @@ import (
 
 	"github.com/containerd/containerd"
 	cerrdefs "github.com/containerd/containerd/errdefs"
+	"github.com/containerd/containerd/images"
 	"github.com/containerd/containerd/plugin"
 	"github.com/containerd/containerd/remotes/docker"
 	"github.com/containerd/containerd/snapshots"
@@ -14,7 +15,7 @@ import (
 	"github.com/distribution/reference"
 	"github.com/docker/docker/container"
 	daemonevents "github.com/docker/docker/daemon/events"
-	"github.com/docker/docker/daemon/images"
+	daemonimages "github.com/docker/docker/daemon/images"
 	"github.com/docker/docker/daemon/snapshotter"
 	"github.com/docker/docker/errdefs"
 	"github.com/docker/docker/image"
@@ -27,6 +28,7 @@ import (
 // ImageService implements daemon.ImageService
 type ImageService struct {
 	client          *containerd.Client
+	images          images.Store
 	containers      container.Store
 	snapshotter     string
 	registryHosts   docker.RegistryHosts
@@ -59,6 +61,7 @@ type ImageServiceConfig struct {
 func NewService(config ImageServiceConfig) *ImageService {
 	return &ImageService{
 		client:          config.Client,
+		images:          config.Client.ImageService(),
 		containers:      config.Containers,
 		snapshotter:     config.Snapshotter,
 		registryHosts:   config.RegistryHosts,
@@ -70,8 +73,8 @@ func NewService(config ImageServiceConfig) *ImageService {
 }
 
 // DistributionServices return services controlling daemon image storage.
-func (i *ImageService) DistributionServices() images.DistributionServices {
-	return images.DistributionServices{}
+func (i *ImageService) DistributionServices() daemonimages.DistributionServices {
+	return daemonimages.DistributionServices{}
 }
 
 // CountImages returns the number of images stored by ImageService

+ 56 - 0
vendor/github.com/containerd/log/logtest/context.go

@@ -0,0 +1,56 @@
+/*
+   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 logtest
+
+import (
+	"context"
+	"fmt"
+	"io"
+	"path/filepath"
+	"runtime"
+	"testing"
+
+	"github.com/containerd/log"
+	"github.com/sirupsen/logrus"
+)
+
+// WithT adds a logging hook for the given test
+// Changes debug level to debug, clears output, and
+// outputs all log messages as test logs.
+func WithT(ctx context.Context, t testing.TB) context.Context {
+	// Create a new logger to avoid adding hooks from multiple tests
+	l := logrus.New()
+
+	// Increase debug level for tests
+	l.SetLevel(logrus.DebugLevel)
+	l.SetOutput(io.Discard)
+	l.SetReportCaller(true)
+
+	// Add testing hook
+	l.AddHook(&testHook{
+		t: t,
+		fmt: &logrus.TextFormatter{
+			DisableColors:   true,
+			TimestampFormat: log.RFC3339NanoFixed,
+			CallerPrettyfier: func(frame *runtime.Frame) (string, string) {
+				return filepath.Base(frame.Function), fmt.Sprintf("%s:%d", frame.File, frame.Line)
+			},
+		},
+	})
+
+	return log.WithLogger(ctx, logrus.NewEntry(l).WithField("testcase", t.Name()))
+}

+ 50 - 0
vendor/github.com/containerd/log/logtest/log_hook.go

@@ -0,0 +1,50 @@
+/*
+   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 logtest
+
+import (
+	"bytes"
+	"sync"
+	"testing"
+
+	"github.com/sirupsen/logrus"
+)
+
+type testHook struct {
+	t   testing.TB
+	fmt logrus.Formatter
+	mu  sync.Mutex
+}
+
+func (*testHook) Levels() []logrus.Level {
+	return logrus.AllLevels
+}
+
+func (h *testHook) Fire(e *logrus.Entry) error {
+	s, err := h.fmt.Format(e)
+	if err != nil {
+		return err
+	}
+
+	// Because the logger could be called from multiple goroutines,
+	// but t.Log() is not designed for.
+	h.mu.Lock()
+	defer h.mu.Unlock()
+	h.t.Log(string(bytes.TrimRight(s, "\n")))
+
+	return nil
+}

+ 1 - 0
vendor/modules.txt

@@ -359,6 +359,7 @@ github.com/containerd/go-runc
 # github.com/containerd/log v0.1.0
 ## explicit; go 1.20
 github.com/containerd/log
+github.com/containerd/log/logtest
 # github.com/containerd/nydus-snapshotter v0.8.2
 ## explicit; go 1.19
 github.com/containerd/nydus-snapshotter/pkg/converter