diff --git a/snapshot/btrfs/btrfs.go b/snapshot/btrfs/btrfs.go index bee93ac..9c9ae75 100644 --- a/snapshot/btrfs/btrfs.go +++ b/snapshot/btrfs/btrfs.go @@ -2,16 +2,16 @@ package btrfs import ( "context" - "crypto/sha256" "fmt" - "io/ioutil" "os" "path/filepath" - "strings" "github.com/docker/containerd" + "github.com/docker/containerd/log" "github.com/docker/containerd/plugin" "github.com/docker/containerd/snapshot" + "github.com/docker/containerd/snapshot/storage" + "github.com/docker/containerd/snapshot/storage/boltdb" "github.com/pkg/errors" "github.com/stevvooe/go-btrfs" ) @@ -25,11 +25,19 @@ func init() { Type: plugin.SnapshotPlugin, Config: &btrfsConfig{}, Init: func(ic *plugin.InitContext) (interface{}, error) { + root := filepath.Join(ic.Root, "snapshot", "btrfs") conf := ic.Config.(*btrfsConfig) if conf.Device == "" { + // TODO: check device for root return nil, errors.Errorf("btrfs requires \"device\" configuration") } - return NewSnapshotter(conf.Device, filepath.Join(ic.Root, "snapshot", "btrfs")) + + ms, err := boltdb.NewMetaStore(ic.Context, filepath.Join(root, "metadata.db")) + if err != nil { + return nil, err + } + + return NewSnapshotter(conf.Device, root, ms) }, }) } @@ -37,30 +45,29 @@ func init() { type Snapshotter struct { device string // maybe we can resolve it with path? root string // root provides paths for internal storage. + ms storage.MetaStore } -func NewSnapshotter(device, root string) (snapshot.Snapshotter, error) { +func NewSnapshotter(device, root string, ms storage.MetaStore) (snapshot.Snapshotter, error) { var ( active = filepath.Join(root, "active") snapshots = filepath.Join(root, "snapshots") - parents = filepath.Join(root, "parents") - index = filepath.Join(root, "index") - names = filepath.Join(root, "names") ) for _, path := range []string{ active, snapshots, - parents, - index, - names, } { if err := os.MkdirAll(path, 0755); err != nil { return nil, err } } - return &Snapshotter{device: device, root: root}, nil + return &Snapshotter{ + device: device, + root: root, + ms: ms, + }, nil } // Stat returns the info for an active or committed snapshot by name or @@ -69,123 +76,68 @@ func NewSnapshotter(device, root string) (snapshot.Snapshotter, error) { // Should be used for parent resolution, existence checks and to discern // the kind of snapshot. func (b *Snapshotter) Stat(ctx context.Context, key string) (snapshot.Info, error) { - // resolve the snapshot out of the index. - target, err := os.Readlink(filepath.Join(b.root, "index", hash(key))) + ctx, t, err := b.ms.TransactionContext(ctx, false) if err != nil { - if !os.IsNotExist(err) { - return snapshot.Info{}, err - } - - return snapshot.Info{}, errors.Errorf("snapshot %v not found", key) + return snapshot.Info{}, err } - - return b.stat(target) + defer t.Rollback() + return b.ms.Stat(ctx, key) } -func (b *Snapshotter) stat(target string) (snapshot.Info, error) { - var ( - parents = filepath.Join(b.root, "parents") - names = filepath.Join(b.root, "names") - namep = filepath.Join(names, filepath.Base(target)) - parentlink = filepath.Join(parents, filepath.Base(target)) - ) - - // grab information about the subvolume - info, err := btrfs.SubvolInfo(target) +// Walk the committed snapshots. +func (b *Snapshotter) Walk(ctx context.Context, fn func(context.Context, snapshot.Info) error) error { + ctx, t, err := b.ms.TransactionContext(ctx, false) if err != nil { - return snapshot.Info{}, err + return err } - - // read the name out of the names! - nameraw, err := ioutil.ReadFile(namep) - if err != nil { - return snapshot.Info{}, err - } - - // resolve the parents path. - parentp, err := os.Readlink(parentlink) - if err != nil { - if !os.IsNotExist(err) { - return snapshot.Info{}, err - } - - // no parent! - } - - var parent string - if parentp != "" { - // okay, grab the basename of the parent and look up its name! - parentnamep := filepath.Join(names, filepath.Base(parentp)) - - p, err := ioutil.ReadFile(parentnamep) - if err != nil { - return snapshot.Info{}, err - } - - parent = string(p) - } - - kind := snapshot.KindCommitted - if strings.HasPrefix(target, filepath.Join(b.root, "active")) { - kind = snapshot.KindActive - } - - return snapshot.Info{ - Name: string(nameraw), - Parent: parent, - Readonly: info.Readonly, - Kind: kind, - }, nil - + defer t.Rollback() + return b.ms.Walk(ctx, fn) } func (b *Snapshotter) Prepare(ctx context.Context, key, parent string) ([]containerd.Mount, error) { - return b.makeActive(key, parent, false) + return b.makeActive(ctx, key, parent, false) } func (b *Snapshotter) View(ctx context.Context, key, parent string) ([]containerd.Mount, error) { - return b.makeActive(key, parent, true) + return b.makeActive(ctx, key, parent, true) } -func (b *Snapshotter) makeActive(key, parent string, readonly bool) ([]containerd.Mount, error) { - var ( - active = filepath.Join(b.root, "active") - snapshots = filepath.Join(b.root, "snapshots") - parents = filepath.Join(b.root, "parents") - index = filepath.Join(b.root, "index") - names = filepath.Join(b.root, "names") - keyh = hash(key) - parenth = hash(parent) - target = filepath.Join(active, keyh) - namep = filepath.Join(names, keyh) - indexlink = filepath.Join(index, keyh) - parentlink = filepath.Join(parents, keyh) - parentp = filepath.Join(snapshots, parenth) // parent must be restricted to snaps - ) - - if parent == "" { - // create new subvolume - // btrfs subvolume create /dir - if err := btrfs.SubvolCreate(target); err != nil { - return nil, err - } - } else { - // btrfs subvolume snapshot /parent /subvol - if err := btrfs.SubvolSnapshot(target, parentp, readonly); err != nil { - return nil, err - } - - if err := os.Symlink(parentp, parentlink); err != nil { - return nil, err - } +func (b *Snapshotter) makeActive(ctx context.Context, key, parent string, readonly bool) ([]containerd.Mount, error) { + ctx, t, err := b.ms.TransactionContext(ctx, true) + if err != nil { + return nil, err } + defer func() { + if err != nil && t != nil { + if rerr := t.Rollback(); rerr != nil { + log.G(ctx).WithError(rerr).Warn("Failure rolling back transaction") + } + } + }() - // write in the name - if err := ioutil.WriteFile(namep, []byte(key), 0644); err != nil { + a, err := b.ms.CreateActive(ctx, key, parent, readonly) + if err != nil { return nil, err } - if err := os.Symlink(target, indexlink); err != nil { + target := filepath.Join(b.root, "active", a.ID) + + if len(a.ParentIDs) == 0 { + // create new subvolume + // btrfs subvolume create /dir + if err = btrfs.SubvolCreate(target); err != nil { + return nil, err + } + } else { + parentp := filepath.Join(b.root, "snapshots", a.ParentIDs[0]) + // btrfs subvolume snapshot /parent /subvol + if err = btrfs.SubvolSnapshot(target, parentp, a.Readonly); err != nil { + return nil, err + } + } + err = t.Commit() + t = nil + if err != nil { return nil, err } @@ -218,82 +170,42 @@ func (b *Snapshotter) mounts(dir string) ([]containerd.Mount, error) { }, nil } -func (b *Snapshotter) Commit(ctx context.Context, name, key string) error { - var ( - active = filepath.Join(b.root, "active") - snapshots = filepath.Join(b.root, "snapshots") - index = filepath.Join(b.root, "index") - parents = filepath.Join(b.root, "parents") - names = filepath.Join(b.root, "names") - keyh = hash(key) - nameh = hash(name) - dir = filepath.Join(active, keyh) - target = filepath.Join(snapshots, nameh) - keynamep = filepath.Join(names, keyh) - namep = filepath.Join(names, nameh) - keyparentlink = filepath.Join(parents, keyh) - parentlink = filepath.Join(parents, nameh) - keyindexlink = filepath.Join(index, keyh) - indexlink = filepath.Join(index, nameh) - ) - - info, err := btrfs.SubvolInfo(dir) +func (b *Snapshotter) Commit(ctx context.Context, name, key string) (err error) { + ctx, t, err := b.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") + } + } + }() - if info.Readonly { - return fmt.Errorf("may not commit view snapshot %q", dir) - } - - // look up the parent information to make sure we have the right parent link. - parentp, err := os.Readlink(keyparentlink) + id, err := b.ms.Commit(ctx, key, name) if err != nil { - if !os.IsNotExist(err) { - return err + return errors.Wrap(err, "failed to commit") + } + + source := filepath.Join(b.root, "active", id) + target := filepath.Join(b.root, "snapshots", id) + + if err := btrfs.SubvolSnapshot(target, source, true); err != nil { + return err + } + + err = t.Commit() + t = nil + if err != nil { + if derr := btrfs.SubvolDelete(target); derr != nil { + log.G(ctx).WithError(derr).Error("Failed to clean up new snapshot: %v", target) } - - // we have no parent! - } - - if err := os.MkdirAll(snapshots, 0755); err != nil { return err } - if err := btrfs.SubvolSnapshot(target, dir, true); err != nil { - return err - } - - // remove the key name path as we no longer need it. - if err := os.Remove(keynamep); err != nil { - return err - } - - if err := os.Remove(keyindexlink); err != nil { - return err - } - - if err := ioutil.WriteFile(namep, []byte(name), 0755); err != nil { - return err - } - - if parentp != "" { - // move over the parent link into the commit name. - if err := os.Rename(keyparentlink, parentlink); err != nil { - return err - } - - // TODO(stevvooe): For this to not break horribly, we should really - // start taking a full lock. We are going to move all this metadata - // into common storage, so let's not fret over it for now. - } - - if err := os.Symlink(target, indexlink); err != nil { - return err - } - - if err := btrfs.SubvolDelete(dir); err != nil { - return errors.Wrapf(err, "delete subvol failed on %v", dir) + if derr := btrfs.SubvolDelete(source); derr != nil { + log.G(ctx).WithError(derr).Warn("Failed to clean up active snapshot: %v", source) } return nil @@ -304,55 +216,87 @@ func (b *Snapshotter) Commit(ctx context.Context, name, key string) error { // // This can be used to recover mounts after calling View or Prepare. func (b *Snapshotter) Mounts(ctx context.Context, key string) ([]containerd.Mount, error) { - dir := filepath.Join(b.root, "active", hash(key)) + ctx, t, err := b.ms.TransactionContext(ctx, false) + if err != nil { + return nil, err + } + a, err := b.ms.GetActive(ctx, key) + t.Rollback() + if err != nil { + return nil, errors.Wrap(err, "failed to get active snapshot") + } + dir := filepath.Join(b.root, "active", a.ID) return b.mounts(dir) } // Remove abandons the transaction identified by key. All resources // associated with the key will be removed. -func (b *Snapshotter) Remove(ctx context.Context, key string) error { - panic("not implemented") -} +func (b *Snapshotter) Remove(ctx context.Context, key string) (err error) { + var ( + source, removed string + readonly bool + ) -// Walk the committed snapshots. -func (b *Snapshotter) Walk(ctx context.Context, fn func(context.Context, snapshot.Info) error) error { - // TODO(stevvooe): Copy-pasted almost verbatim from overlay. Really need to - // unify the metadata for snapshot implementations. - root := filepath.Join(b.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 := os.Readlink(path) - if err != nil { - if !os.IsNotExist(err) { - return err + ctx, t, err := b.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") } } - si, err := b.stat(target) + if removed != "" { + if derr := btrfs.SubvolDelete(removed); derr != nil { + log.G(ctx).WithError(derr).Warn("Failed to clean up removed snapshot: %v", removed) + } + } + }() + + id, k, err := b.ms.Remove(ctx, key) + if err != nil { + return errors.Wrap(err, "failed to remove snapshot") + } + + if k == snapshot.KindActive { + source = filepath.Join(b.root, "active", id) + + info, err := btrfs.SubvolInfo(source) if err != nil { + source = "" return err } - if err := fn(ctx, si); err != nil { - return err + readonly = info.Readonly + removed = filepath.Join(b.root, "active", "rm-"+id) + } else { + source = filepath.Join(b.root, "snapshots", id) + removed = filepath.Join(b.root, "snapshots", "rm-"+id) + readonly = true + } + + if err := btrfs.SubvolSnapshot(removed, source, readonly); err != nil { + removed = "" + return err + } + + if err := btrfs.SubvolDelete(source); err != nil { + return errors.Wrapf(err, "failed to remove snapshot %v", source) + } + + err = t.Commit() + t = nil + if err != nil { + // Attempt to restore source + if err1 := btrfs.SubvolSnapshot(source, removed, readonly); err1 != nil { + // Keep removed to allow for manual restore + removed = "" + log.G(ctx).WithError(err1).Error("Failed to restore source snapshot %v from %v", source, removed) } + return err + } - return nil - }) -} - -func hash(k string) string { - return fmt.Sprintf("%x", sha256.Sum224([]byte(k))) + return nil } diff --git a/snapshot/btrfs/btrfs_test.go b/snapshot/btrfs/btrfs_test.go index 436802b..9b90b7d 100644 --- a/snapshot/btrfs/btrfs_test.go +++ b/snapshot/btrfs/btrfs_test.go @@ -11,6 +11,7 @@ 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" ) @@ -19,11 +20,14 @@ const ( mib = 1024 * 1024 ) -func TestBtrfs(t *testing.T) { - testutil.RequiresRoot(t) - testsuite.SnapshotterSuite(t, "Btrfs", func(ctx context.Context, root string) (snapshot.Snapshotter, func(), error) { +func boltSnapshotter(t *testing.T) func(context.Context, string) (snapshot.Snapshotter, func(), error) { + return func(ctx context.Context, root string) (snapshot.Snapshotter, func(), error) { device := setupBtrfsLoopbackDevice(t, root) - snapshotter, err := NewSnapshotter(device.deviceName, root) + store, err := boltdb.NewMetaStore(ctx, filepath.Join(root, "metadata.db")) + if err != nil { + return nil, nil, err + } + snapshotter, err := NewSnapshotter(device.deviceName, root, store) if err != nil { t.Fatal(err) } @@ -31,34 +35,39 @@ func TestBtrfs(t *testing.T) { return snapshotter, func() { device.remove(t) }, nil - }) + } +} + +func TestBtrfs(t *testing.T) { + testutil.RequiresRoot(t) + testsuite.SnapshotterSuite(t, "Btrfs", boltSnapshotter(t)) } func TestBtrfsMounts(t *testing.T) { testutil.RequiresRoot(t) - ctx := context.TODO() + ctx := context.Background() // create temporary directory for mount point mountPoint, err := ioutil.TempDir("", "containerd-btrfs-test") if err != nil { t.Fatal("could not create mount point for btrfs test", err) } + defer os.RemoveAll(mountPoint) t.Log("temporary mount point created", mountPoint) - device := setupBtrfsLoopbackDevice(t, mountPoint) - defer device.remove(t) - - root, err := ioutil.TempDir(device.mountPoint, "TestBtrfsPrepare-") + root, err := ioutil.TempDir(mountPoint, "TestBtrfsPrepare-") if err != nil { t.Fatal(err) } defer os.RemoveAll(root) - target := filepath.Join(root, "test") - b, err := NewSnapshotter(device.deviceName, root) + b, c, err := boltSnapshotter(t)(ctx, root) if err != nil { t.Fatal(err) } + defer c() + + target := filepath.Join(root, "test") mounts, err := b.Prepare(ctx, target, "") if err != nil { t.Fatal(err) diff --git a/snapshot/overlay/overlay.go b/snapshot/overlay/overlay.go index 96459c7..37c54fe 100644 --- a/snapshot/overlay/overlay.go +++ b/snapshot/overlay/overlay.go @@ -126,7 +126,7 @@ func (o *Snapshotter) Remove(ctx context.Context, key string) (err error) { } }() - id, err := o.ms.Remove(ctx, key) + id, _, err := o.ms.Remove(ctx, key) if err != nil { return errors.Wrap(err, "failed to remove") } @@ -206,7 +206,7 @@ func (o *Snapshotter) createActive(ctx context.Context, key, parent string, read active, err := o.ms.CreateActive(ctx, key, parent, readonly) if err != nil { if rerr := t.Rollback(); rerr != nil { - // TODO: log rollback error + log.G(ctx).WithError(rerr).Warn("Failure rolling back transaction") } return nil, errors.Wrap(err, "failed to create active") } @@ -214,7 +214,7 @@ func (o *Snapshotter) createActive(ctx context.Context, key, parent string, read path = filepath.Join(snapshotDir, active.ID) if err := os.Rename(td, path); err != nil { if rerr := t.Rollback(); rerr != nil { - // TODO: log rollback error + log.G(ctx).WithError(rerr).Warn("Failure rolling back transaction") } return nil, errors.Wrap(err, "failed to rename") } diff --git a/snapshot/storage/boltdb/bolt.go b/snapshot/storage/boltdb/bolt.go index 061aa0d..4ae1891 100644 --- a/snapshot/storage/boltdb/bolt.go +++ b/snapshot/storage/boltdb/bolt.go @@ -273,7 +273,7 @@ func (ms *boltMetastore) parents(bkt *bolt.Bucket, parent *Snapshot) (parents [] return } -func (ms *boltMetastore) Remove(ctx context.Context, key string) (id string, err error) { +func (ms *boltMetastore) Remove(ctx context.Context, key string) (id string, k snapshot.Kind, err error) { err = ms.withBucket(ctx, func(ctx context.Context, bkt *bolt.Bucket) error { var ss Snapshot b := bkt.Get([]byte(key)) @@ -309,12 +309,10 @@ func (ms *boltMetastore) Remove(ctx context.Context, key string) (id string, err } id = fmt.Sprintf("%d", ss.ID) + k = fromProtoActive(ss.Active) return nil }) - if err != nil { - return "", err - } return } diff --git a/snapshot/storage/metastore.go b/snapshot/storage/metastore.go index 821bcdc..f863e7c 100644 --- a/snapshot/storage/metastore.go +++ b/snapshot/storage/metastore.go @@ -36,8 +36,8 @@ type MetaStore interface { // Remove removes a snapshot from the metastore. The provided context // must contain a writable transaction. The string identifier for the - // snapshot is returned. - Remove(ctx context.Context, key string) (string, error) + // snapshot is returned as well as the kind. + Remove(ctx context.Context, key string) (string, snapshot.Kind, error) // Commit renames the active snapshot transaction referenced by `key` // as a committed snapshot referenced by `Name`. The resulting snapshot diff --git a/snapshot/storage/testsuite/bench.go b/snapshot/storage/testsuite/bench.go index 608bef5..6a42961 100644 --- a/snapshot/storage/testsuite/bench.go +++ b/snapshot/storage/testsuite/bench.go @@ -152,7 +152,7 @@ func createActiveBenchmark(ctx context.Context, b *testing.B, ms storage.MetaSto b.Fatal(err) } b.StopTimer() - if _, err := ms.Remove(ctx, "active"); err != nil { + if _, _, err := ms.Remove(ctx, "active"); err != nil { b.Fatal(err) } b.StartTimer() @@ -166,7 +166,7 @@ func removeBenchmark(ctx context.Context, b *testing.B, ms storage.MetaStore) { b.Fatal(err) } b.StartTimer() - if _, err := ms.Remove(ctx, "active"); err != nil { + if _, _, err := ms.Remove(ctx, "active"); err != nil { b.Fatal(err) } } @@ -183,7 +183,7 @@ func commitBenchmark(ctx context.Context, b *testing.B, ms storage.MetaStore) { b.Fatal(err) } b.StopTimer() - if _, err := ms.Remove(ctx, "committed"); err != nil { + if _, _, err := ms.Remove(ctx, "committed"); err != nil { b.Fatal(err) } } diff --git a/snapshot/testsuite/testsuite.go b/snapshot/testsuite/testsuite.go index 329865a..240de78 100644 --- a/snapshot/testsuite/testsuite.go +++ b/snapshot/testsuite/testsuite.go @@ -189,10 +189,10 @@ func checkSnapshotterBasic(ctx context.Context, t *testing.T, snapshotter snapsh 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)) + assert.NoError(t, snapshotter.Remove(ctx, nextnext)) + 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.