Skip to content

Commit

Permalink
Add support for idmapped layers
Browse files Browse the repository at this point in the history
Signed-off-by: David Son <[email protected]>
  • Loading branch information
sondavidb committed Oct 29, 2024
1 parent 1b5483e commit fe36d4d
Show file tree
Hide file tree
Showing 17 changed files with 1,588 additions and 51 deletions.
5 changes: 3 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -186,9 +186,10 @@ ctr-with-idmapping: $(OUTDIR)/ctr-with-idmapping

$(OUTDIR)/ctr-with-idmapping:
# Use a custom fork for testing ID-mapping as containerd doesn't fully support this yet.
git clone https://github.com/sondavidb/containerd.git tempfolder
git clone https://github.com/containerd/containerd.git tempfolder
cd tempfolder && \
git checkout multi-uid-gid && \
git checkout v1.7.22 && \
git apply $(SOCI_SNAPSHOTTER_PROJECT_ROOT)/integration/config/ctr.patch && \
make bin/ctr && \
cp bin/ctr $(OUTDIR)/ctr-with-idmapping && \
cd ../
Expand Down
1 change: 1 addition & 0 deletions config/config.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ filesystem_cache_type=""
resolve_result_entry=0
debug=false
allow_no_verification=true
allow_idmap=true
# disable_verification=false
# Causes TestRunWithDefaultConfig to break, but
# fine to use in /etc/soci-snapshotter-grpc-config.toml
Expand Down
1 change: 1 addition & 0 deletions config/fs.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ type FSConfig struct {
NoPrometheus bool `toml:"no_prometheus"`
MountTimeoutSec int64 `toml:"mount_timeout_sec"`
FuseMetricsEmitWaitDurationSec int64 `toml:"fuse_metrics_emit_wait_duration_sec"`
AllowIDMap bool `toml:"allow_idmap" default:"true"`

RetryableHTTPClientConfig `toml:"http"`
BlobConfig `toml:"blob"`
Expand Down
98 changes: 81 additions & 17 deletions fs/fs.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,9 +45,12 @@ package fs
import (
"context"
"fmt"
"io"
golog "log"
"net/http"
"os"
"os/exec"
"path/filepath"
"sync"
"syscall"
"time"
Expand All @@ -59,6 +62,7 @@ import (
layermetrics "github.com/awslabs/soci-snapshotter/fs/metrics/layer"
"github.com/awslabs/soci-snapshotter/fs/remote"
"github.com/awslabs/soci-snapshotter/fs/source"
"github.com/awslabs/soci-snapshotter/idtools"
"github.com/awslabs/soci-snapshotter/metadata"
"github.com/awslabs/soci-snapshotter/snapshot"
"github.com/awslabs/soci-snapshotter/soci"
Expand All @@ -67,6 +71,7 @@ import (
ctdsnapshotters "github.com/containerd/containerd/pkg/snapshotters"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
"github.com/containerd/errdefs"
"github.com/containerd/log"
metrics "github.com/docker/go-metrics"
fusefs "github.com/hanwen/go-fuse/v2/fs"
Expand Down Expand Up @@ -455,6 +460,58 @@ func (fs *filesystem) getSociContext(ctx context.Context, imageRef, indexDigest,
return c, err
}

func getIDMappedMountpoint(mountpoint, activeLayerKey string) string {
d := filepath.Dir(mountpoint)
return filepath.Join(fmt.Sprintf("%s_%s", d, activeLayerKey), "fs")
}

func (fs *filesystem) IDMapMount(ctx context.Context, mountpoint, activeLayerKey string, idmapper idtools.IDMap) (string, error) {
newMountpoint := getIDMappedMountpoint(mountpoint, activeLayerKey)
logger := log.G(ctx).WithField("mountpoint", newMountpoint)

logger.Debug("creating remote id-mapped mount")
if err := os.Mkdir(filepath.Dir(newMountpoint), 0700); err != nil {
return "", err
}
if err := os.Mkdir(newMountpoint, 0755); err != nil {
return "", err
}

fs.layerMu.Lock()
l := fs.layer[mountpoint]
if l == nil {
fs.layerMu.Unlock()
logger.Error("failed to create remote id-mapped mount")
return "", errdefs.ErrNotFound
}
fs.layer[newMountpoint] = l
fs.layerMu.Unlock()
node, err := l.RootNode(0, idmapper)
if err != nil {
return "", err
}

fuseLogger := log.L.
WithField("mountpoint", mountpoint).
WriterLevel(logrus.TraceLevel)

return newMountpoint, fs.setupFuseServer(ctx, newMountpoint, node, l, fuseLogger, nil)
}

func (fs *filesystem) IDMapMountLocal(ctx context.Context, mountpoint, activeLayerKey string, idmapper idtools.IDMap) (string, error) {
newMountpoint := getIDMappedMountpoint(mountpoint, activeLayerKey)
logger := log.G(ctx).WithField("mountpoint", newMountpoint)

logger.Debug("creating local id-mapped mount")
if err := idtools.RemapDir(ctx, mountpoint, newMountpoint, idmapper); err != nil {
logger.WithError(err).Error("failed to create local mount")
return "", err
}

logger.Debug("successfully created local mountpoint")
return newMountpoint, nil
}

func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[string]string) (retErr error) {
// Setting the start time to measure the Mount operation duration.
start := time.Now()
Expand Down Expand Up @@ -560,7 +617,7 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
}
}()

node, err := l.RootNode(0)
node, err := l.RootNode(0, idtools.IDMap{})
if err != nil {
log.G(ctx).WithError(err).Warnf("Failed to get root node")
retErr = fmt.Errorf("failed to get root node: %w", err)
Expand All @@ -577,6 +634,16 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
fs.layerMu.Unlock()
fs.metricsController.Add(mountpoint, l)

// Pass in a logger to go-fuse with the layer digest
// The go-fuse logs are useful for tracing exactly what's happening at the fuse level.
fuseLogger := log.L.
WithField("layerDigest", labels[ctdsnapshotters.TargetLayerDigestLabel]).
WriterLevel(logrus.TraceLevel)

return fs.setupFuseServer(ctx, mountpoint, node, l, fuseLogger, c)
}

func (fs *filesystem) setupFuseServer(ctx context.Context, mountpoint string, node fusefs.InodeEmbedder, l layer.Layer, logger *io.PipeWriter, c *sociContext) error {
// mount the node to the specified mountpoint
// TODO: bind mount the state directory as a read-only fs on snapshotter's side
rawFS := fusefs.NewNodeFS(node, &fusefs.Options{
Expand All @@ -585,40 +652,37 @@ func (fs *filesystem) Mount(ctx context.Context, mountpoint string, labels map[s
NegativeTimeout: &fs.negativeTimeout,
NullPermissions: true,
})
// Pass in a logger to go-fuse with the layer digest
// The go-fuse logs are useful for tracing exactly what's happening at the fuse level.
logger := log.L.
WithField("layerDigest", labels[ctdsnapshotters.TargetLayerDigestLabel]).
WriterLevel(logrus.TraceLevel)
mountOpts := &fuse.MountOptions{
AllowOther: true, // allow users other than root&mounter to access fs
FsName: "soci", // name this filesystem as "soci"
Debug: fs.debug,
Logger: golog.New(logger, "", 0),
DisableXAttrs: l.DisableXAttrs(),
Options: []string{"default_permissions", "ro"},
}
if _, err := exec.LookPath(fusermountBin); err == nil {
mountOpts.Options = []string{"suid"} // option for fusermount; allow setuid inside container
} else {
log.G(ctx).WithError(err).Infof("%s not installed; trying direct mount", fusermountBin)
log.G(ctx).WithField("binary", fusermountBin).WithError(err).Info("fuse binary not installed; trying direct mount")
mountOpts.DirectMount = true
}
server, err := fuse.NewServer(rawFS, mountpoint, mountOpts)
if err != nil {
log.G(ctx).WithError(err).Debug("failed to make filesystem server")
retErr = err
return
log.G(ctx).WithError(err).Error("failed to make filesystem server")
return err
}

go server.Serve()

// Send a signal to the background fetcher that a new image is being mounted
// and to pause all background fetches.
c.bgFetchPauseOnce.Do(func() {
if fs.bgFetcher != nil {
fs.bgFetcher.Pause()
}
})
if c != nil {
// Send a signal to the background fetcher that a new image is being mounted
// and to pause all background fetches.
c.bgFetchPauseOnce.Do(func() {
if fs.bgFetcher != nil {
fs.bgFetcher.Pause()
}
})
}

return server.WaitMount()
}
Expand Down
11 changes: 7 additions & 4 deletions fs/fs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/awslabs/soci-snapshotter/fs/layer"
"github.com/awslabs/soci-snapshotter/fs/remote"
"github.com/awslabs/soci-snapshotter/fs/source"
"github.com/awslabs/soci-snapshotter/idtools"
"github.com/containerd/containerd/reference"
"github.com/containerd/containerd/remotes/docker"
fusefs "github.com/hanwen/go-fuse/v2/fs"
Expand Down Expand Up @@ -83,10 +84,12 @@ func (l *breakableLayer) Info() layer.Info {
Size: 1,
}
}
func (l *breakableLayer) DisableXAttrs() bool { return false }
func (l *breakableLayer) RootNode(uint32) (fusefs.InodeEmbedder, error) { return nil, nil }
func (l *breakableLayer) Verify(tocDigest digest.Digest) error { return nil }
func (l *breakableLayer) SkipVerify() {}
func (l *breakableLayer) DisableXAttrs() bool { return false }
func (l *breakableLayer) RootNode(uint32, idtools.IDMap) (fusefs.InodeEmbedder, error) {
return nil, nil
}
func (l *breakableLayer) Verify(tocDigest digest.Digest) error { return nil }
func (l *breakableLayer) SkipVerify() {}
func (l *breakableLayer) ReadAt([]byte, int64, ...remote.Option) (int, error) {
return 0, fmt.Errorf("fail")
}
Expand Down
7 changes: 4 additions & 3 deletions fs/layer/layer.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ import (
"github.com/awslabs/soci-snapshotter/fs/remote"

spanmanager "github.com/awslabs/soci-snapshotter/fs/span-manager"
"github.com/awslabs/soci-snapshotter/idtools"
"github.com/awslabs/soci-snapshotter/metadata"
"github.com/awslabs/soci-snapshotter/soci"
"github.com/awslabs/soci-snapshotter/util/lrucache"
Expand Down Expand Up @@ -86,7 +87,7 @@ type Layer interface {
Info() Info

// RootNode returns the root node of this layer.
RootNode(baseInode uint32) (fusefs.InodeEmbedder, error)
RootNode(baseInode uint32, idMapper idtools.IDMap) (fusefs.InodeEmbedder, error)

// Check checks if the layer is still connectable.
Check() error
Expand Down Expand Up @@ -456,11 +457,11 @@ func (l *layerRef) Done() {
l.done()
}

func (l *layer) RootNode(baseInode uint32) (fusefs.InodeEmbedder, error) {
func (l *layer) RootNode(baseInode uint32, idMapper idtools.IDMap) (fusefs.InodeEmbedder, error) {
if l.isClosed() {
return nil, fmt.Errorf("layer is already closed")
}
return newNode(l.desc.Digest, l.r, l.blob, baseInode, l.resolver.overlayOpaqueType, l.resolver.config.LogFuseOperations, l.fuseOperationCounter)
return newNode(l.desc.Digest, l.r, l.blob, baseInode, l.resolver.overlayOpaqueType, l.resolver.config.LogFuseOperations, l.fuseOperationCounter, idMapper)
}

func (l *layer) ReadAt(p []byte, offset int64, opts ...remote.Option) (int, error) {
Expand Down
40 changes: 23 additions & 17 deletions fs/layer/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ import (
commonmetrics "github.com/awslabs/soci-snapshotter/fs/metrics/common"
"github.com/awslabs/soci-snapshotter/fs/reader"
"github.com/awslabs/soci-snapshotter/fs/remote"
"github.com/awslabs/soci-snapshotter/idtools"
"github.com/awslabs/soci-snapshotter/metadata"
"github.com/containerd/log"
fusefs "github.com/hanwen/go-fuse/v2/fs"
Expand Down Expand Up @@ -189,7 +190,7 @@ func (f *FuseOperationCounter) Run(ctx context.Context) {

// logFSOperations may cause sensitive information to be emitted to logs
// e.g. filenames and paths within an image
func newNode(layerDgst digest.Digest, r reader.Reader, blob remote.Blob, baseInode uint32, opaque OverlayOpaqueType, logFSOperations bool, opCounter *FuseOperationCounter) (fusefs.InodeEmbedder, error) {
func newNode(layerDgst digest.Digest, r reader.Reader, blob remote.Blob, baseInode uint32, opaque OverlayOpaqueType, logFSOperations bool, opCounter *FuseOperationCounter, idMapper idtools.IDMap) (fusefs.InodeEmbedder, error) {
rootID := r.Metadata().RootID()
rootAttr, err := r.Metadata().GetAttr(rootID)
if err != nil {
Expand All @@ -210,9 +211,10 @@ func newNode(layerDgst digest.Digest, r reader.Reader, blob remote.Blob, baseIno
}
ffs.s = ffs.newState(layerDgst, blob)
return &node{
id: rootID,
attr: rootAttr,
fs: ffs,
id: rootID,
attr: rootAttr,
fs: ffs,
idMapper: idMapper,
}, nil
}

Expand Down Expand Up @@ -272,9 +274,10 @@ func (fs *fs) inodeOfID(id uint32) (uint64, error) {
// node is a filesystem inode abstraction.
type node struct {
fusefs.Inode
fs *fs
id uint32
attr metadata.Attr
fs *fs
id uint32
attr metadata.Attr
idMapper idtools.IDMap

ents []fuse.DirEntry
entsCached bool
Expand Down Expand Up @@ -407,14 +410,14 @@ func (n *node) Lookup(ctx context.Context, name string, out *fuse.EntryOut) (*fu
n.fs.reportFailure(fuseOpLookup, fmt.Errorf("%s: %v", fuseOpLookup, err))
return nil, syscall.EIO
}
entryToAttr(ino, tn.attr, &out.Attr)
n.entryToAttr(ino, tn.attr, &out.Attr)
case *whiteout:
ino, err := n.fs.inodeOfID(tn.id)
if err != nil {
n.fs.reportFailure(fuseOpLookup, fmt.Errorf("%s: %v", fuseOpLookup, err))
return nil, syscall.EIO
}
entryToAttr(ino, tn.attr, &out.Attr)
n.entryToAttr(ino, tn.attr, &out.Attr)
default:
n.fs.reportFailure(fuseOpLookup, fmt.Errorf("%s: unknown node type detected", fuseOpLookup))
return nil, syscall.EIO
Expand Down Expand Up @@ -463,10 +466,11 @@ func (n *node) Lookup(ctx context.Context, name string, out *fuse.EntryOut) (*fu
return nil, syscall.EIO
}
return n.NewInode(ctx, &node{
id: id,
fs: n.fs,
attr: ce,
}, entryToAttr(ino, ce, &out.Attr)), 0
id: id,
fs: n.fs,
attr: ce,
idMapper: n.idMapper,
}, n.entryToAttr(ino, ce, &out.Attr)), 0
}

var _ = (fusefs.NodeOpener)((*node)(nil))
Expand Down Expand Up @@ -495,7 +499,7 @@ func (n *node) Getattr(ctx context.Context, f fusefs.FileHandle, out *fuse.AttrO
n.fs.reportFailure(fuseOpGetattr, fmt.Errorf("%s: %v", fuseOpGetattr, err))
return syscall.EIO
}
entryToAttr(ino, n.attr, &out.Attr)
n.entryToAttr(ino, n.attr, &out.Attr)
return 0
}

Expand Down Expand Up @@ -594,7 +598,7 @@ func (f *file) Getattr(ctx context.Context, out *fuse.AttrOut) syscall.Errno {
f.n.fs.reportFailure(fuseOpFileGetattr, fmt.Errorf("%s: %v", fuseOpFileGetattr, err))
return syscall.EIO
}
entryToAttr(ino, f.n.attr, &out.Attr)
f.n.entryToAttr(ino, f.n.attr, &out.Attr)
return 0
}

Expand Down Expand Up @@ -797,7 +801,7 @@ func (sf *statFile) updateStatUnlocked() ([]byte, error) {
}

// entryToAttr converts metadata.Attr to go-fuse's Attr.
func entryToAttr(ino uint64, e metadata.Attr, out *fuse.Attr) fusefs.StableAttr {
func (n *node) entryToAttr(ino uint64, e metadata.Attr, out *fuse.Attr) fusefs.StableAttr {
out.Ino = ino
out.Size = uint64(e.Size)
if e.Mode&os.ModeSymlink != 0 {
Expand All @@ -808,7 +812,9 @@ func entryToAttr(ino uint64, e metadata.Attr, out *fuse.Attr) fusefs.StableAttr
mtime := e.ModTime
out.SetTimes(nil, &mtime, nil)
out.Mode = fileModeToSystemMode(e.Mode)
out.Owner = fuse.Owner{Uid: uint32(e.UID), Gid: uint32(e.GID)}
// Potentially dangerous casting int -> uint32? But probably fine.
mappedID, _ := n.idMapper.ToHost(idtools.User{Uid: uint32(e.UID), Gid: uint32(e.GID)})
out.Owner = fuse.Owner{Uid: mappedID.Uid, Gid: mappedID.Gid}
out.Rdev = uint32(unix.Mkdev(uint32(e.DevMajor), uint32(e.DevMinor)))
out.Nlink = uint32(e.NumLink)
if out.Nlink == 0 {
Expand Down
3 changes: 2 additions & 1 deletion fs/layer/node_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,8 @@ func TestEntryToAttr(t *testing.T) {
tc := tc
t.Run(tc.name, func(t *testing.T) {
var actual fuse.Attr
entryToAttr(0, tc.attr, &actual)
var n node
n.entryToAttr(0, tc.attr, &actual)
tc.expected.Mtime = actual.Mtime
if actual != tc.expected {
t.Fatalf("unexpected fuse attr. actual %v expected %v", actual, tc.expected)
Expand Down
3 changes: 2 additions & 1 deletion fs/layer/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ import (
"github.com/awslabs/soci-snapshotter/fs/reader"
"github.com/awslabs/soci-snapshotter/fs/remote"
spanmanager "github.com/awslabs/soci-snapshotter/fs/span-manager"
"github.com/awslabs/soci-snapshotter/idtools"
"github.com/awslabs/soci-snapshotter/metadata"
"github.com/awslabs/soci-snapshotter/util/testutil"
"github.com/awslabs/soci-snapshotter/ztoc"
Expand Down Expand Up @@ -360,7 +361,7 @@ func hasSize(name string, size int) check {
}

func getRootNode(t *testing.T, r reader.Reader, opaque OverlayOpaqueType) *node {
rootNode, err := newNode(testStateLayerDigest, &testReader{r}, &testBlobState{10, 5}, 100, opaque, false, nil)
rootNode, err := newNode(testStateLayerDigest, &testReader{r}, &testBlobState{10, 5}, 100, opaque, false, nil, idtools.IDMap{})
if err != nil {
t.Fatalf("failed to get root node: %v", err)
}
Expand Down
Loading

0 comments on commit fe36d4d

Please sign in to comment.