Update overlay snapshot driver to use metastore
Update tests to use boltdb. Update test suite to pass context. Signed-off-by: Derek McGowan <derek@mcgstyle.net> (github: dmcgowan)
This commit is contained in:
		| @@ -21,7 +21,7 @@ const ( | ||||
|  | ||||
| func TestBtrfs(t *testing.T) { | ||||
| 	testutil.RequiresRoot(t) | ||||
| 	testsuite.SnapshotterSuite(t, "Btrfs", func(root string) (snapshot.Snapshotter, func(), error) { | ||||
| 	testsuite.SnapshotterSuite(t, "Btrfs", func(ctx context.Context, root string) (snapshot.Snapshotter, func(), error) { | ||||
| 		device := setupBtrfsLoopbackDevice(t, root) | ||||
| 		snapshotter, err := NewSnapshotter(device.deviceName, root) | ||||
| 		if err != nil { | ||||
|   | ||||
| @@ -7,12 +7,13 @@ import ( | ||||
| 	"os" | ||||
| 	"path/filepath" | ||||
| 	"strings" | ||||
| 	"sync" | ||||
|  | ||||
| 	"github.com/docker/containerd" | ||||
| 	"github.com/docker/containerd/log" | ||||
| 	"github.com/docker/containerd/plugin" | ||||
| 	"github.com/docker/containerd/snapshot" | ||||
| 	digest "github.com/opencontainers/go-digest" | ||||
| 	"github.com/docker/containerd/snapshot/storage" | ||||
| 	"github.com/docker/containerd/snapshot/storage/boltdb" | ||||
| 	"github.com/pkg/errors" | ||||
| ) | ||||
|  | ||||
| @@ -20,32 +21,40 @@ func init() { | ||||
| 	plugin.Register("snapshot-overlay", &plugin.Registration{ | ||||
| 		Type: plugin.SnapshotPlugin, | ||||
| 		Init: func(ic *plugin.InitContext) (interface{}, error) { | ||||
| 			return NewSnapshotter(filepath.Join(ic.Root, "snapshot", "overlay")) | ||||
| 			root := filepath.Join(ic.Root, "snapshot", "overlay") | ||||
| 			ms, err := boltdb.NewMetaStore(ic.Context, filepath.Join(root, "metadata.db")) | ||||
| 			if err != nil { | ||||
| 				return nil, err | ||||
| 			} | ||||
| 			return NewSnapshotter(root, ms) | ||||
| 		}, | ||||
| 	}) | ||||
| } | ||||
|  | ||||
| type Snapshotter struct { | ||||
| 	root  string | ||||
| 	links *cache | ||||
| 	root string | ||||
| 	ms   storage.MetaStore | ||||
| } | ||||
|  | ||||
| func NewSnapshotter(root string) (snapshot.Snapshotter, error) { | ||||
| type activeSnapshot struct { | ||||
| 	id       string | ||||
| 	name     string | ||||
| 	parentID interface{} | ||||
| 	readonly bool | ||||
| } | ||||
|  | ||||
| func NewSnapshotter(root string, ms storage.MetaStore) (snapshot.Snapshotter, error) { | ||||
| 	if err := os.MkdirAll(root, 0700); err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	for _, p := range []string{ | ||||
| 		"committed", // committed snapshots | ||||
| 		"active",    // active snapshots | ||||
| 		"index",     // snapshots by hashed name | ||||
| 	} { | ||||
| 		if err := os.MkdirAll(filepath.Join(root, p), 0700); err != nil { | ||||
| 			return nil, err | ||||
| 		} | ||||
|  | ||||
| 	if err := os.MkdirAll(filepath.Join(root, "snapshots"), 0700); err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
|  | ||||
| 	return &Snapshotter{ | ||||
| 		root:  root, | ||||
| 		links: newCache(), | ||||
| 		root: root, | ||||
| 		ms:   ms, | ||||
| 	}, nil | ||||
| } | ||||
|  | ||||
| @@ -55,86 +64,20 @@ func NewSnapshotter(root string) (snapshot.Snapshotter, error) { | ||||
| // Should be used for parent resolution, existence checks and to discern | ||||
| // the kind of snapshot. | ||||
| func (o *Snapshotter) Stat(ctx context.Context, key string) (snapshot.Info, error) { | ||||
| 	path, err := o.links.get(filepath.Join(o.root, "index", hash(key))) | ||||
| 	if err != nil { | ||||
| 		if !os.IsNotExist(err) { | ||||
| 			return snapshot.Info{}, err | ||||
| 		} | ||||
|  | ||||
| 		return snapshot.Info{}, errors.Errorf("snapshot %v not found", key) | ||||
| 	} | ||||
|  | ||||
| 	// TODO(stevvooe): We don't confirm the name to avoid the lookup cost. | ||||
| 	return o.stat(path) | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) stat(path string) (snapshot.Info, error) { | ||||
| 	ppath, err := o.links.get(filepath.Join(path, "parent")) | ||||
| 	if err != nil { | ||||
| 		if !os.IsNotExist(err) { | ||||
| 			return snapshot.Info{}, err | ||||
| 		} | ||||
|  | ||||
| 		// no parent | ||||
| 	} | ||||
|  | ||||
| 	kp, err := ioutil.ReadFile(filepath.Join(path, "name")) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, false) | ||||
| 	if err != nil { | ||||
| 		return snapshot.Info{}, err | ||||
| 	} | ||||
|  | ||||
| 	var parent string | ||||
| 	if ppath != "" { | ||||
| 		p, err := ioutil.ReadFile(filepath.Join(ppath, "name")) | ||||
| 		if err != nil { | ||||
| 			return snapshot.Info{}, err | ||||
| 		} | ||||
| 		parent = string(p) | ||||
| 	} | ||||
|  | ||||
| 	ro := true | ||||
| 	kind := snapshot.KindCommitted | ||||
| 	if strings.HasPrefix(path, filepath.Join(o.root, "active")) { | ||||
| 		// TODO(stevvooe): Maybe there is a better way? | ||||
| 		kind = snapshot.KindActive | ||||
|  | ||||
| 		// TODO(stevvooe): We haven't introduced this to overlay yet. | ||||
| 		// We'll add it when we add tests for it. | ||||
| 		ro = false | ||||
| 	} | ||||
|  | ||||
| 	return snapshot.Info{ | ||||
| 		Name:     string(kp), | ||||
| 		Parent:   parent, | ||||
| 		Kind:     kind, | ||||
| 		Readonly: ro, | ||||
| 	}, nil | ||||
| 	defer t.Rollback() | ||||
| 	return o.ms.Stat(ctx, key) | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) Prepare(ctx context.Context, key, parent string) ([]containerd.Mount, error) { | ||||
| 	active, err := o.newActiveDir(key, false) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	if parent != "" { | ||||
| 		if err := active.setParent(parent); err != nil { | ||||
| 			return nil, err | ||||
| 		} | ||||
| 	} | ||||
| 	return active.mounts(o.links) | ||||
| 	return o.createActive(ctx, key, parent, false) | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) View(ctx context.Context, key, parent string) ([]containerd.Mount, error) { | ||||
| 	active, err := o.newActiveDir(key, true) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	if parent != "" { | ||||
| 		if err := active.setParent(parent); err != nil { | ||||
| 			return nil, err | ||||
| 		} | ||||
| 	} | ||||
| 	return active.mounts(o.links) | ||||
| 	return o.createActive(ctx, key, parent, true) | ||||
| } | ||||
|  | ||||
| // Mounts returns the mounts for the transaction identified by key. Can be | ||||
| @@ -142,263 +85,208 @@ func (o *Snapshotter) View(ctx context.Context, key, parent string) ([]container | ||||
| // | ||||
| // This can be used to recover mounts after calling View or Prepare. | ||||
| func (o *Snapshotter) Mounts(ctx context.Context, key string) ([]containerd.Mount, error) { | ||||
| 	active := o.getActive(key) | ||||
| 	return active.mounts(o.links) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, false) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	active, err := o.ms.GetActive(ctx, key) | ||||
| 	t.Rollback() | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "failed to get active mount") | ||||
| 	} | ||||
| 	return o.mounts(active), nil | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) Commit(ctx context.Context, name, key string) error { | ||||
| 	active := o.getActive(key) | ||||
| 	return active.commit(name, o.links) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, true) | ||||
| 	if err != nil { | ||||
| 		return err | ||||
| 	} | ||||
| 	if _, err := o.ms.Commit(ctx, key, name); err != nil { | ||||
| 		if rerr := t.Rollback(); rerr != nil { | ||||
| 			log.G(ctx).WithError(rerr).Warn("Failure rolling back transaction") | ||||
| 		} | ||||
| 		return errors.Wrap(err, "failed to commit snapshot") | ||||
| 	} | ||||
| 	return t.Commit() | ||||
| } | ||||
|  | ||||
| // Remove abandons the transaction identified by key. All resources | ||||
| // associated with the key will be removed. | ||||
| func (o *Snapshotter) Remove(ctx context.Context, key string) error { | ||||
| 	panic("not implemented") | ||||
| func (o *Snapshotter) Remove(ctx context.Context, key string) (err error) { | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, true) | ||||
| 	if err != nil { | ||||
| 		return err | ||||
| 	} | ||||
| 	defer func() { | ||||
| 		if err != nil && t != nil { | ||||
| 			if rerr := t.Rollback(); rerr != nil { | ||||
| 				log.G(ctx).WithError(rerr).Warn("Failure rolling back transaction") | ||||
| 			} | ||||
| 		} | ||||
| 	}() | ||||
|  | ||||
| 	id, err := o.ms.Remove(ctx, key) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "failed to remove") | ||||
| 	} | ||||
|  | ||||
| 	path := filepath.Join(o.root, "snapshots", id) | ||||
| 	renamed := filepath.Join(o.root, "snapshots", "rm-"+id) | ||||
| 	if err := os.Rename(path, renamed); err != nil { | ||||
| 		return errors.Wrap(err, "failed to rename") | ||||
| 	} | ||||
|  | ||||
| 	err = t.Commit() | ||||
| 	t = nil | ||||
| 	if err != nil { | ||||
| 		if err1 := os.Rename(renamed, path); err1 != nil { | ||||
| 			// May cause inconsistent data on disk | ||||
| 			log.G(ctx).WithError(err1).WithField("path", renamed).Errorf("Failed to rename after failed commit") | ||||
| 		} | ||||
| 		return errors.Wrap(err, "failed to commit") | ||||
| 	} | ||||
| 	if err := os.RemoveAll(renamed); err != nil { | ||||
| 		// Must be cleaned up, any "rm-*" could be removed if no active transactions | ||||
| 		log.G(ctx).WithError(err).WithField("path", renamed).Warnf("Failed to remove root filesystem") | ||||
| 	} | ||||
|  | ||||
| 	return nil | ||||
| } | ||||
|  | ||||
| // Walk the committed snapshots. | ||||
| func (o *Snapshotter) Walk(ctx context.Context, fn func(context.Context, snapshot.Info) error) error { | ||||
| 	root := filepath.Join(o.root, "index") | ||||
| 	return filepath.Walk(root, func(path string, fi os.FileInfo, err error) error { | ||||
| 		if err != nil { | ||||
| 			return err | ||||
| 		} | ||||
|  | ||||
| 		if path == root { | ||||
| 			return nil | ||||
| 		} | ||||
|  | ||||
| 		if fi.Mode()&os.ModeSymlink == 0 { | ||||
| 			// only follow links | ||||
| 			return filepath.SkipDir | ||||
| 		} | ||||
|  | ||||
| 		target, err := o.links.get(path) | ||||
| 		if err != nil { | ||||
| 			if !os.IsNotExist(err) { | ||||
| 				return err | ||||
| 			} | ||||
| 		} | ||||
|  | ||||
| 		si, err := o.stat(target) | ||||
| 		if err != nil { | ||||
| 			return err | ||||
| 		} | ||||
|  | ||||
| 		if err := fn(ctx, si); err != nil { | ||||
| 			return err | ||||
| 		} | ||||
|  | ||||
| 		return nil | ||||
| 	}) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, false) | ||||
| 	if err != nil { | ||||
| 		return err | ||||
| 	} | ||||
| 	defer t.Rollback() | ||||
| 	return o.ms.Walk(ctx, fn) | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) newActiveDir(key string, readonly bool) (*activeDir, error) { | ||||
| func (o *Snapshotter) createActive(ctx context.Context, key, parent string, readonly bool) ([]containerd.Mount, error) { | ||||
| 	var ( | ||||
| 		path      = filepath.Join(o.root, "active", hash(key)) | ||||
| 		name      = filepath.Join(path, "name") | ||||
| 		indexlink = filepath.Join(o.root, "index", hash(key)) | ||||
| 		path        string | ||||
| 		snapshotDir = filepath.Join(o.root, "snapshots") | ||||
| 	) | ||||
| 	a := &activeDir{ | ||||
| 		path:         path, | ||||
| 		committedDir: filepath.Join(o.root, "committed"), | ||||
| 		indexlink:    indexlink, | ||||
|  | ||||
| 	td, err := ioutil.TempDir(snapshotDir, "new-") | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "failed to create temp dir") | ||||
| 	} | ||||
| 	defer func() { | ||||
| 		if err != nil { | ||||
| 			if td != "" { | ||||
| 				if err1 := os.RemoveAll(td); err1 != nil { | ||||
| 					err = errors.Wrapf(err, "remove failed: %v", err1) | ||||
| 				} | ||||
| 			} | ||||
| 			if path != "" { | ||||
| 				if err1 := os.RemoveAll(path); err1 != nil { | ||||
| 					err = errors.Wrapf(err, "failed to remove path: %v", err1) | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 	}() | ||||
|  | ||||
| 	if err = os.MkdirAll(filepath.Join(td, "fs"), 0700); err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	if !readonly { | ||||
| 		for _, p := range []string{ | ||||
| 			"work", | ||||
| 			"fs", | ||||
| 		} { | ||||
| 			if err := os.MkdirAll(filepath.Join(path, p), 0700); err != nil { | ||||
| 				a.delete() | ||||
| 				return nil, err | ||||
| 			} | ||||
| 		} | ||||
| 	} else { | ||||
| 		if err := os.MkdirAll(filepath.Join(path, "fs"), 0700); err != nil { | ||||
| 			a.delete() | ||||
| 		if err = os.MkdirAll(filepath.Join(td, "work"), 0700); err != nil { | ||||
| 			return nil, err | ||||
| 		} | ||||
| 	} | ||||
|  | ||||
| 	if err := ioutil.WriteFile(name, []byte(key), 0644); err != nil { | ||||
| 		a.delete() | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, true) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
|  | ||||
| 	// link from namespace | ||||
| 	if err := os.Symlink(path, indexlink); err != nil { | ||||
| 		a.delete() | ||||
| 		return nil, err | ||||
| 	} | ||||
|  | ||||
| 	return a, nil | ||||
| } | ||||
|  | ||||
| func (o *Snapshotter) getActive(key string) *activeDir { | ||||
| 	return &activeDir{ | ||||
| 		path:         filepath.Join(o.root, "active", hash(key)), | ||||
| 		committedDir: filepath.Join(o.root, "committed"), | ||||
| 		indexlink:    filepath.Join(o.root, "index", hash(key)), | ||||
| 	} | ||||
| } | ||||
|  | ||||
| func hash(k string) string { | ||||
| 	return digest.FromString(k).Hex() | ||||
| } | ||||
|  | ||||
| type activeDir struct { | ||||
| 	committedDir string | ||||
| 	path         string | ||||
| 	indexlink    string | ||||
| } | ||||
|  | ||||
| func (a *activeDir) delete() error { | ||||
| 	return os.RemoveAll(a.path) | ||||
| } | ||||
|  | ||||
| func (a *activeDir) setParent(name string) error { | ||||
| 	return os.Symlink(filepath.Join(a.committedDir, hash(name)), filepath.Join(a.path, "parent")) | ||||
| } | ||||
|  | ||||
| func (a *activeDir) commit(name string, c *cache) error { | ||||
| 	if _, err := os.Stat(filepath.Join(a.path, "fs")); err != nil { | ||||
| 		if os.IsNotExist(err) { | ||||
| 			return errors.New("cannot commit view") | ||||
| 	active, err := o.ms.CreateActive(ctx, key, parent, readonly) | ||||
| 	if err != nil { | ||||
| 		if rerr := t.Rollback(); rerr != nil { | ||||
| 			// TODO: log rollback error | ||||
| 		} | ||||
| 		return err | ||||
| 		return nil, errors.Wrap(err, "failed to create active") | ||||
| 	} | ||||
|  | ||||
| 	// TODO(stevvooe): This doesn't quite meet the current model. The new model | ||||
| 	// is to copy all of this out and let the transaction continue. We don't | ||||
| 	// really have tests for it yet, but this will be the spot to fix it. | ||||
| 	// | ||||
| 	// Nothing should be removed until remove is called on the active | ||||
| 	// transaction. | ||||
| 	if err := os.RemoveAll(filepath.Join(a.path, "work")); err != nil { | ||||
| 		return err | ||||
| 	path = filepath.Join(snapshotDir, active.ID) | ||||
| 	if err := os.Rename(td, path); err != nil { | ||||
| 		if rerr := t.Rollback(); rerr != nil { | ||||
| 			// TODO: log rollback error | ||||
| 		} | ||||
| 		return nil, errors.Wrap(err, "failed to rename") | ||||
| 	} | ||||
| 	td = "" | ||||
|  | ||||
| 	if err := t.Commit(); err != nil { | ||||
| 		return nil, errors.Wrap(err, "commit failed") | ||||
| 	} | ||||
|  | ||||
| 	if err := ioutil.WriteFile(filepath.Join(a.path, "name"), []byte(name), 0644); err != nil { | ||||
| 		return err | ||||
| 	} | ||||
|  | ||||
| 	c.invalidate(a.path) // clears parent cache, since we end up moving. | ||||
| 	c.invalidate(filepath.Join(a.path, "parent")) | ||||
| 	c.invalidate(a.indexlink) | ||||
|  | ||||
| 	committed := filepath.Join(a.committedDir, hash(name)) | ||||
| 	if err := os.Rename(a.path, committed); err != nil { | ||||
| 		return err | ||||
| 	} | ||||
|  | ||||
| 	if err := os.Remove(a.indexlink); err != nil { | ||||
| 		return err | ||||
| 	} | ||||
|  | ||||
| 	indexlink := filepath.Join(filepath.Dir(a.indexlink), hash(name)) | ||||
| 	return os.Symlink(committed, indexlink) | ||||
| 	return o.mounts(active), nil | ||||
| } | ||||
|  | ||||
| func (a *activeDir) mounts(c *cache) ([]containerd.Mount, error) { | ||||
| 	var ( | ||||
| 		parents []string | ||||
| 		err     error | ||||
| 		current = a.path | ||||
| 	) | ||||
| 	for { | ||||
| 		if current, err = c.get(filepath.Join(current, "parent")); err != nil { | ||||
| 			if os.IsNotExist(err) { | ||||
| 				break | ||||
| 			} | ||||
|  | ||||
| 			return nil, err | ||||
| 		} | ||||
| 		parents = append(parents, filepath.Join(current, "fs")) | ||||
| 	} | ||||
| 	if len(parents) == 0 { | ||||
| func (o *Snapshotter) mounts(active storage.Active) []containerd.Mount { | ||||
| 	if len(active.ParentIDs) == 0 { | ||||
| 		// if we only have one layer/no parents then just return a bind mount as overlay | ||||
| 		// will not work | ||||
| 		roFlag := "rw" | ||||
| 		if _, err := os.Stat(filepath.Join(a.path, "work")); err != nil { | ||||
| 			if !os.IsNotExist(err) { | ||||
| 				return nil, err | ||||
| 			} | ||||
| 		if active.Readonly { | ||||
| 			roFlag = "ro" | ||||
| 		} | ||||
|  | ||||
| 		return []containerd.Mount{ | ||||
| 			{ | ||||
| 				Source: filepath.Join(a.path, "fs"), | ||||
| 				Source: o.upperPath(active.ID), | ||||
| 				Type:   "bind", | ||||
| 				Options: []string{ | ||||
| 					roFlag, | ||||
| 					"rbind", | ||||
| 				}, | ||||
| 			}, | ||||
| 		}, nil | ||||
| 		} | ||||
| 	} | ||||
| 	var options []string | ||||
|  | ||||
| 	if _, err := os.Stat(filepath.Join(a.path, "work")); err == nil { | ||||
| 	if !active.Readonly { | ||||
| 		options = append(options, | ||||
| 			fmt.Sprintf("workdir=%s", filepath.Join(a.path, "work")), | ||||
| 			fmt.Sprintf("upperdir=%s", filepath.Join(a.path, "fs")), | ||||
| 			fmt.Sprintf("workdir=%s", o.workPath(active.ID)), | ||||
| 			fmt.Sprintf("upperdir=%s", o.upperPath(active.ID)), | ||||
| 		) | ||||
| 	} else if !os.IsNotExist(err) { | ||||
| 		return nil, err | ||||
| 	} else if len(parents) == 1 { | ||||
| 	} else if len(active.ParentIDs) == 1 { | ||||
| 		return []containerd.Mount{ | ||||
| 			{ | ||||
| 				Source: parents[0], | ||||
| 				Source: o.upperPath(active.ParentIDs[0]), | ||||
| 				Type:   "bind", | ||||
| 				Options: []string{ | ||||
| 					"ro", | ||||
| 					"rbind", | ||||
| 				}, | ||||
| 			}, | ||||
| 		}, nil | ||||
| 		} | ||||
| 	} | ||||
|  | ||||
| 	options = append(options, fmt.Sprintf("lowerdir=%s", strings.Join(parents, ":"))) | ||||
| 	parentPaths := make([]string, len(active.ParentIDs)) | ||||
| 	for i := range active.ParentIDs { | ||||
| 		parentPaths[i] = o.upperPath(active.ParentIDs[i]) | ||||
| 	} | ||||
|  | ||||
| 	options = append(options, fmt.Sprintf("lowerdir=%s", strings.Join(parentPaths, ":"))) | ||||
| 	return []containerd.Mount{ | ||||
| 		{ | ||||
| 			Type:    "overlay", | ||||
| 			Source:  "overlay", | ||||
| 			Options: options, | ||||
| 		}, | ||||
| 	}, nil | ||||
| } | ||||
|  | ||||
| func newCache() *cache { | ||||
| 	return &cache{ | ||||
| 		links: make(map[string]string), | ||||
| 	} | ||||
|  | ||||
| } | ||||
|  | ||||
| type cache struct { | ||||
| 	mu    sync.Mutex | ||||
| 	links map[string]string | ||||
| func (o *Snapshotter) upperPath(id string) string { | ||||
| 	return filepath.Join(o.root, "snapshots", id, "fs") | ||||
| } | ||||
|  | ||||
| func (c *cache) get(path string) (string, error) { | ||||
| 	c.mu.Lock() | ||||
| 	defer c.mu.Unlock() | ||||
| 	target, ok := c.links[path] | ||||
| 	if !ok { | ||||
| 		link, err := os.Readlink(path) | ||||
| 		if err != nil { | ||||
| 			return "", err | ||||
| 		} | ||||
| 		c.links[path], target = link, link | ||||
| 	} | ||||
| 	return target, nil | ||||
| } | ||||
|  | ||||
| func (c *cache) invalidate(path string) { | ||||
| 	c.mu.Lock() | ||||
| 	defer c.mu.Unlock() | ||||
|  | ||||
| 	delete(c.links, path) | ||||
| func (o *Snapshotter) workPath(id string) string { | ||||
| 	return filepath.Join(o.root, "snapshots", id, "work") | ||||
| } | ||||
|   | ||||
| @@ -11,20 +11,27 @@ import ( | ||||
|  | ||||
| 	"github.com/docker/containerd" | ||||
| 	"github.com/docker/containerd/snapshot" | ||||
| 	"github.com/docker/containerd/snapshot/storage/boltdb" | ||||
| 	"github.com/docker/containerd/snapshot/testsuite" | ||||
| 	"github.com/docker/containerd/testutil" | ||||
| ) | ||||
|  | ||||
| func boltSnapshotter(ctx context.Context, root string) (snapshot.Snapshotter, func(), error) { | ||||
| 	store, err := boltdb.NewMetaStore(ctx, filepath.Join(root, "metadata.db")) | ||||
| 	if err != nil { | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
| 	snapshotter, err := NewSnapshotter(root, store) | ||||
| 	if err != nil { | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
|  | ||||
| 	return snapshotter, func() {}, nil | ||||
| } | ||||
|  | ||||
| func TestOverlay(t *testing.T) { | ||||
| 	testutil.RequiresRoot(t) | ||||
| 	testsuite.SnapshotterSuite(t, "Overlay", func(root string) (snapshot.Snapshotter, func(), error) { | ||||
| 		snapshotter, err := NewSnapshotter(root) | ||||
| 		if err != nil { | ||||
| 			t.Fatal(err) | ||||
| 		} | ||||
|  | ||||
| 		return snapshotter, func() {}, nil | ||||
| 	}) | ||||
| 	testsuite.SnapshotterSuite(t, "Overlay", boltSnapshotter) | ||||
| } | ||||
|  | ||||
| func TestOverlayMounts(t *testing.T) { | ||||
| @@ -34,7 +41,7 @@ func TestOverlayMounts(t *testing.T) { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	defer os.RemoveAll(root) | ||||
| 	o, err := NewSnapshotter(root) | ||||
| 	o, _, err := boltSnapshotter(ctx, root) | ||||
| 	if err != nil { | ||||
| 		t.Error(err) | ||||
| 		return | ||||
| @@ -51,7 +58,7 @@ func TestOverlayMounts(t *testing.T) { | ||||
| 	if m.Type != "bind" { | ||||
| 		t.Errorf("mount type should be bind but received %q", m.Type) | ||||
| 	} | ||||
| 	expected := filepath.Join(root, "active", hash("/tmp/test"), "fs") | ||||
| 	expected := filepath.Join(root, "snapshots", "1", "fs") | ||||
| 	if m.Source != expected { | ||||
| 		t.Errorf("expected source %q but received %q", expected, m.Source) | ||||
| 	} | ||||
| @@ -70,7 +77,7 @@ func TestOverlayCommit(t *testing.T) { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	defer os.RemoveAll(root) | ||||
| 	o, err := NewSnapshotter(root) | ||||
| 	o, _, err := boltSnapshotter(ctx, root) | ||||
| 	if err != nil { | ||||
| 		t.Error(err) | ||||
| 		return | ||||
| @@ -99,7 +106,7 @@ func TestOverlayOverlayMount(t *testing.T) { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	defer os.RemoveAll(root) | ||||
| 	o, err := NewSnapshotter(root) | ||||
| 	o, _, err := boltSnapshotter(ctx, root) | ||||
| 	if err != nil { | ||||
| 		t.Error(err) | ||||
| 		return | ||||
| @@ -129,11 +136,10 @@ func TestOverlayOverlayMount(t *testing.T) { | ||||
| 		t.Errorf("expected source %q but received %q", "overlay", m.Source) | ||||
| 	} | ||||
| 	var ( | ||||
| 		ah    = hash("/tmp/layer2") | ||||
| 		sh    = hash("base") | ||||
| 		work  = "workdir=" + filepath.Join(root, "active", ah, "work") | ||||
| 		upper = "upperdir=" + filepath.Join(root, "active", ah, "fs") | ||||
| 		lower = "lowerdir=" + filepath.Join(root, "committed", sh, "fs") | ||||
| 		bp    = getBasePath(ctx, o, root, "/tmp/layer2") | ||||
| 		work  = "workdir=" + filepath.Join(bp, "work") | ||||
| 		upper = "upperdir=" + filepath.Join(bp, "fs") | ||||
| 		lower = "lowerdir=" + getParents(ctx, o, root, "/tmp/layer2")[0] | ||||
| 	) | ||||
| 	for i, v := range []string{ | ||||
| 		work, | ||||
| @@ -146,6 +152,40 @@ func TestOverlayOverlayMount(t *testing.T) { | ||||
| 	} | ||||
| } | ||||
|  | ||||
| func getBasePath(ctx context.Context, sn snapshot.Snapshotter, root, key string) string { | ||||
| 	o := sn.(*Snapshotter) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, false) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 	defer t.Rollback() | ||||
|  | ||||
| 	active, err := o.ms.GetActive(ctx, key) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
|  | ||||
| 	return filepath.Join(root, "snapshots", active.ID) | ||||
| } | ||||
|  | ||||
| func getParents(ctx context.Context, sn snapshot.Snapshotter, root, key string) []string { | ||||
| 	o := sn.(*Snapshotter) | ||||
| 	ctx, t, err := o.ms.TransactionContext(ctx, false) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 	defer t.Rollback() | ||||
| 	active, err := o.ms.GetActive(ctx, key) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 	parents := make([]string, len(active.ParentIDs)) | ||||
| 	for i := range active.ParentIDs { | ||||
| 		parents[i] = filepath.Join(root, "snapshots", active.ParentIDs[i], "fs") | ||||
| 	} | ||||
| 	return parents | ||||
| } | ||||
|  | ||||
| func TestOverlayOverlayRead(t *testing.T) { | ||||
| 	testutil.RequiresRoot(t) | ||||
| 	ctx := context.TODO() | ||||
| @@ -154,7 +194,7 @@ func TestOverlayOverlayRead(t *testing.T) { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	defer os.RemoveAll(root) | ||||
| 	o, err := NewSnapshotter(root) | ||||
| 	o, _, err := boltSnapshotter(ctx, root) | ||||
| 	if err != nil { | ||||
| 		t.Error(err) | ||||
| 		return | ||||
| @@ -206,7 +246,7 @@ func TestOverlayView(t *testing.T) { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	defer os.RemoveAll(root) | ||||
| 	o, err := NewSnapshotter(root) | ||||
| 	o, _, err := boltSnapshotter(ctx, root) | ||||
| 	if err != nil { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| @@ -228,7 +268,7 @@ func TestOverlayView(t *testing.T) { | ||||
| 	if err != nil { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	if err := ioutil.WriteFile(filepath.Join(root, "active", hash(key), "fs", "foo"), []byte("hi, again"), 0660); err != nil { | ||||
| 	if err := ioutil.WriteFile(filepath.Join(getParents(ctx, o, root, "/tmp/top")[0], "foo"), []byte("hi, again"), 0660); err != nil { | ||||
| 		t.Fatal(err) | ||||
| 	} | ||||
| 	if err := o.Commit(ctx, "top", key); err != nil { | ||||
| @@ -246,7 +286,7 @@ func TestOverlayView(t *testing.T) { | ||||
| 	if m.Type != "bind" { | ||||
| 		t.Errorf("mount type should be bind but received %q", m.Type) | ||||
| 	} | ||||
| 	expected := filepath.Join(root, "committed", hash("base"), "fs") | ||||
| 	expected := getParents(ctx, o, root, "/tmp/view1")[0] | ||||
| 	if m.Source != expected { | ||||
| 		t.Errorf("expected source %q but received %q", expected, m.Source) | ||||
| 	} | ||||
| @@ -274,7 +314,8 @@ func TestOverlayView(t *testing.T) { | ||||
| 	if len(m.Options) != 1 { | ||||
| 		t.Errorf("expected 1 mount option but got %d", len(m.Options)) | ||||
| 	} | ||||
| 	expected = fmt.Sprintf("lowerdir=%s:%s", filepath.Join(root, "committed", hash("top"), "fs"), filepath.Join(root, "committed", hash("base"), "fs")) | ||||
| 	lowers := getParents(ctx, o, root, "/tmp/view2") | ||||
| 	expected = fmt.Sprintf("lowerdir=%s:%s", lowers[0], lowers[1]) | ||||
| 	if m.Options[0] != expected { | ||||
| 		t.Errorf("expected option %q but received %q", expected, m.Options[0]) | ||||
| 	} | ||||
|   | ||||
| @@ -17,7 +17,7 @@ import ( | ||||
| ) | ||||
|  | ||||
| // SnapshotterSuite runs a test suite on the snapshotter given a factory function. | ||||
| func SnapshotterSuite(t *testing.T, name string, snapshotterFn func(root string) (snapshot.Snapshotter, func(), error)) { | ||||
| func SnapshotterSuite(t *testing.T, name string, snapshotterFn func(ctx context.Context, root string) (snapshot.Snapshotter, func(), error)) { | ||||
| 	t.Run("Basic", makeTest(t, name, snapshotterFn, checkSnapshotterBasic)) | ||||
| 	t.Run("StatActive", makeTest(t, name, snapshotterFn, checkSnapshotterStatActive)) | ||||
| 	t.Run("StatComitted", makeTest(t, name, snapshotterFn, checkSnapshotterStatCommitted)) | ||||
| @@ -25,8 +25,9 @@ func SnapshotterSuite(t *testing.T, name string, snapshotterFn func(root string) | ||||
| 	t.Run("PreareViewFailingtest", makeTest(t, name, snapshotterFn, checkSnapshotterPrepareView)) | ||||
| } | ||||
|  | ||||
| func makeTest(t *testing.T, name string, snapshotterFn func(root string) (snapshot.Snapshotter, func(), error), fn func(t *testing.T, snapshotter snapshot.Snapshotter, work string)) func(t *testing.T) { | ||||
| func makeTest(t *testing.T, name string, snapshotterFn func(ctx context.Context, root string) (snapshot.Snapshotter, func(), error), fn func(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string)) func(t *testing.T) { | ||||
| 	return func(t *testing.T) { | ||||
| 		ctx := context.Background() | ||||
| 		oldumask := syscall.Umask(0) | ||||
| 		defer syscall.Umask(oldumask) | ||||
| 		// Make two directories: a snapshotter root and a play area for the tests: | ||||
| @@ -46,7 +47,7 @@ func makeTest(t *testing.T, name string, snapshotterFn func(root string) (snapsh | ||||
| 			t.Fatal(err) | ||||
| 		} | ||||
|  | ||||
| 		snapshotter, cleanup, err := snapshotterFn(root) | ||||
| 		snapshotter, cleanup, err := snapshotterFn(ctx, root) | ||||
| 		if err != nil { | ||||
| 			t.Fatal(err) | ||||
| 		} | ||||
| @@ -58,14 +59,12 @@ func makeTest(t *testing.T, name string, snapshotterFn func(root string) (snapsh | ||||
| 		} | ||||
|  | ||||
| 		defer testutil.DumpDir(t, tmpDir) | ||||
| 		fn(t, snapshotter, work) | ||||
| 		fn(ctx, t, snapshotter, work) | ||||
| 	} | ||||
| } | ||||
|  | ||||
| // checkSnapshotterBasic tests the basic workflow of a snapshot snapshotter. | ||||
| func checkSnapshotterBasic(t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	ctx := context.TODO() | ||||
|  | ||||
| func checkSnapshotterBasic(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	initialApplier := fstest.Apply( | ||||
| 		fstest.CreateFile("/foo", []byte("foo\n"), 0777), | ||||
| 		fstest.CreateDir("/a", 0755), | ||||
| @@ -189,11 +188,15 @@ func checkSnapshotterBasic(t *testing.T, snapshotter snapshot.Snapshotter, work | ||||
| 		fstest.Apply(initialApplier, diffApplier)); err != nil { | ||||
| 		t.Fatalf("failure reason: %+v", err) | ||||
| 	} | ||||
|  | ||||
| 	// TODO: check after remove implemented | ||||
| 	//assert.Error(t, snapshotter.Remove(ctx, committed)) | ||||
| 	//assert.NoError(t, snapshotter.Remove(ctx, nextCommitted)) | ||||
| 	//assert.NoError(t, snapshotter.Remove(ctx, committed)) | ||||
| } | ||||
|  | ||||
| // Create a New Layer on top of base layer with Prepare, Stat on new layer, should return Active layer. | ||||
| func checkSnapshotterStatActive(t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	ctx := context.TODO() | ||||
| func checkSnapshotterStatActive(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	preparing := filepath.Join(work, "preparing") | ||||
| 	if err := os.MkdirAll(preparing, 0777); err != nil { | ||||
| 		t.Fatal(err) | ||||
| @@ -227,8 +230,7 @@ func checkSnapshotterStatActive(t *testing.T, snapshotter snapshot.Snapshotter, | ||||
| } | ||||
|  | ||||
| // Commit a New Layer on top of base layer with Prepare & Commit , Stat on new layer, should return Committed layer. | ||||
| func checkSnapshotterStatCommitted(t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	ctx := context.TODO() | ||||
| func checkSnapshotterStatCommitted(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	preparing := filepath.Join(work, "preparing") | ||||
| 	if err := os.MkdirAll(preparing, 0777); err != nil { | ||||
| 		t.Fatal(err) | ||||
| @@ -289,8 +291,7 @@ func snapshotterPrepareMount(ctx context.Context, snapshotter snapshot.Snapshott | ||||
| } | ||||
|  | ||||
| // Given A <- B <- C, B is the parent of C and A is a transitive parent of C (in this case, a "grandparent") | ||||
| func checkSnapshotterTransitivity(t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	ctx := context.TODO() | ||||
| func checkSnapshotterTransitivity(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	preparing, err := snapshotterPrepareMount(ctx, snapshotter, "preparing", "", work) | ||||
| 	if err != nil { | ||||
| 		t.Fatal(err) | ||||
| @@ -344,9 +345,7 @@ func checkSnapshotterTransitivity(t *testing.T, snapshotter snapshot.Snapshotter | ||||
| } | ||||
|  | ||||
| // Creating two layers with Prepare or View with same key must fail. | ||||
| func checkSnapshotterPrepareView(t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	ctx := context.TODO() | ||||
|  | ||||
| func checkSnapshotterPrepareView(ctx context.Context, t *testing.T, snapshotter snapshot.Snapshotter, work string) { | ||||
| 	preparing, err := snapshotterPrepareMount(ctx, snapshotter, "preparing", "", work) | ||||
| 	if err != nil { | ||||
| 		t.Fatal(err) | ||||
|   | ||||
		Reference in New Issue
	
	Block a user
	 Derek McGowan
					Derek McGowan