userns support for sources and executor

Signed-off-by: Tonis Tiigi <tonistiigi@gmail.com>
docker-19.03
Tonis Tiigi 2019-03-20 14:54:20 -07:00
parent 6921dbe0f6
commit 9f53ea3d78
12 changed files with 146 additions and 24 deletions

View File

@ -10,6 +10,7 @@ import (
"path/filepath"
"sync"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/locker"
iradix "github.com/hashicorp/go-immutable-radix"
"github.com/hashicorp/golang-lru/simplelru"
@ -51,8 +52,8 @@ func ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, path string,
return getDefaultManager().ChecksumWildcard(ctx, ref, path, followLinks)
}
func GetCacheContext(ctx context.Context, md *metadata.StorageItem) (CacheContext, error) {
return getDefaultManager().GetCacheContext(ctx, md)
func GetCacheContext(ctx context.Context, md *metadata.StorageItem, idmap *idtools.IdentityMapping) (CacheContext, error) {
return getDefaultManager().GetCacheContext(ctx, md, idmap)
}
func SetCacheContext(ctx context.Context, md *metadata.StorageItem, cc CacheContext) error {
@ -81,7 +82,7 @@ type cacheManager struct {
}
func (cm *cacheManager) Checksum(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool) (digest.Digest, error) {
cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()))
cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()), ref.IdentityMapping())
if err != nil {
return "", nil
}
@ -89,14 +90,14 @@ func (cm *cacheManager) Checksum(ctx context.Context, ref cache.ImmutableRef, p
}
func (cm *cacheManager) ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool) (digest.Digest, error) {
cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()))
cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()), ref.IdentityMapping())
if err != nil {
return "", nil
}
return cc.ChecksumWildcard(ctx, ref, p, followLinks)
}
func (cm *cacheManager) GetCacheContext(ctx context.Context, md *metadata.StorageItem) (CacheContext, error) {
func (cm *cacheManager) GetCacheContext(ctx context.Context, md *metadata.StorageItem, idmap *idtools.IdentityMapping) (CacheContext, error) {
cm.locker.Lock(md.ID())
cm.lruMu.Lock()
v, ok := cm.lru.Get(md.ID())
@ -106,7 +107,7 @@ func (cm *cacheManager) GetCacheContext(ctx context.Context, md *metadata.Storag
v.(*cacheContext).linkMap = map[string][][]byte{}
return v.(*cacheContext), nil
}
cc, err := newCacheContext(md)
cc, err := newCacheContext(md, idmap)
if err != nil {
cm.locker.Unlock(md.ID())
return nil, err
@ -152,6 +153,7 @@ type cacheContext struct {
node *iradix.Node
dirtyMap map[string]struct{}
linkMap map[string][][]byte
idmap *idtools.IdentityMapping
}
type mount struct {
@ -191,12 +193,13 @@ func (m *mount) clean() error {
return nil
}
func newCacheContext(md *metadata.StorageItem) (*cacheContext, error) {
func newCacheContext(md *metadata.StorageItem, idmap *idtools.IdentityMapping) (*cacheContext, error) {
cc := &cacheContext{
md: md,
tree: iradix.New(),
dirtyMap: map[string]struct{}{},
linkMap: map[string][][]byte{},
idmap: idmap,
}
if err := cc.load(); err != nil {
return nil, err

6
cache/refs.go vendored
View File

@ -5,6 +5,7 @@ import (
"sync"
"github.com/containerd/containerd/mount"
"github.com/docker/docker/pkg/idtools"
"github.com/moby/buildkit/cache/metadata"
"github.com/moby/buildkit/identity"
"github.com/moby/buildkit/snapshot"
@ -20,6 +21,7 @@ type Ref interface {
Release(context.Context) error
Size(ctx context.Context) (int64, error)
Metadata() *metadata.StorageItem
IdentityMapping() *idtools.IdentityMapping
}
type ImmutableRef interface {
@ -83,6 +85,10 @@ func (cr *cacheRecord) isDead() bool {
return cr.dead || (cr.equalImmutable != nil && cr.equalImmutable.dead) || (cr.equalMutable != nil && cr.equalMutable.dead)
}
func (cr *cacheRecord) IdentityMapping() *idtools.IdentityMapping {
return cr.cm.IdentityMapping()
}
func (cr *cacheRecord) Size(ctx context.Context) (int64, error) {
// this expects that usage() is implemented lazily
s, err := cr.sizeG.Do(ctx, cr.ID(), func(ctx context.Context) (interface{}, error) {

View File

@ -117,7 +117,7 @@ func (w containerdExecutor) Exec(ctx context.Context, meta executor.Meta, root c
opts = append(opts, containerdoci.WithCgroup(cgroupsPath))
}
processMode := oci.ProcessSandbox // FIXME(AkihiroSuda)
spec, cleanup, err := oci.GenerateSpec(ctx, meta, mounts, id, resolvConf, hostsFile, namespace, processMode, opts...)
spec, cleanup, err := oci.GenerateSpec(ctx, meta, mounts, id, resolvConf, hostsFile, namespace, processMode, nil, opts...)
if err != nil {
return err
}

View File

@ -13,6 +13,7 @@ import (
"github.com/containerd/containerd/namespaces"
"github.com/containerd/containerd/oci"
"github.com/containerd/continuity/fs"
"github.com/docker/docker/pkg/idtools"
"github.com/mitchellh/hashstructure"
"github.com/moby/buildkit/executor"
"github.com/moby/buildkit/snapshot"
@ -40,7 +41,7 @@ const (
// GenerateSpec generates spec using containerd functionality.
// opts are ignored for s.Process, s.Hostname, and s.Mounts .
func GenerateSpec(ctx context.Context, meta executor.Meta, mounts []executor.Mount, id, resolvConf, hostsFile string, namespace network.Namespace, processMode ProcessMode, opts ...oci.SpecOpts) (*specs.Spec, func(), error) {
func GenerateSpec(ctx context.Context, meta executor.Meta, mounts []executor.Mount, id, resolvConf, hostsFile string, namespace network.Namespace, processMode ProcessMode, idmap *idtools.IdentityMapping, opts ...oci.SpecOpts) (*specs.Spec, func(), error) {
c := &containers.Container{
ID: id,
}
@ -102,7 +103,14 @@ func GenerateSpec(ctx context.Context, meta executor.Meta, mounts []executor.Mou
}
}
}
// TODO: User
if idmap != nil {
s.Linux.Namespaces = append(s.Linux.Namespaces, specs.LinuxNamespace{
Type: specs.UserNamespace,
})
s.Linux.UIDMappings = specMapping(idmap.UIDs())
s.Linux.GIDMappings = specMapping(idmap.GIDs())
}
sm := &submounts{}
@ -227,3 +235,15 @@ func sub(m mount.Mount, subPath string) (mount.Mount, error) {
m.Source = src
return m, nil
}
func specMapping(s []idtools.IDMap) []specs.LinuxIDMapping {
var ids []specs.LinuxIDMapping
for _, item := range s {
ids = append(ids, specs.LinuxIDMapping{
HostID: uint32(item.HostID),
ContainerID: uint32(item.ContainerID),
Size: uint32(item.Size),
})
}
return ids
}

View File

@ -17,6 +17,7 @@ import (
containerdoci "github.com/containerd/containerd/oci"
"github.com/containerd/continuity/fs"
runc "github.com/containerd/go-runc"
"github.com/docker/docker/pkg/idtools"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/executor"
"github.com/moby/buildkit/executor/oci"
@ -38,7 +39,8 @@ type Opt struct {
// DefaultCgroupParent is the cgroup-parent name for executor
DefaultCgroupParent string
// ProcessMode
ProcessMode oci.ProcessMode
ProcessMode oci.ProcessMode
IdentityMapping *idtools.IdentityMapping
}
var defaultCommandCandidates = []string{"buildkit-runc", "runc"}
@ -51,6 +53,7 @@ type runcExecutor struct {
rootless bool
networkProviders map[pb.NetMode]network.Provider
processMode oci.ProcessMode
idmap *idtools.IdentityMapping
}
func New(opt Opt, networkProviders map[pb.NetMode]network.Provider) (executor.Executor, error) {
@ -107,6 +110,7 @@ func New(opt Opt, networkProviders map[pb.NetMode]network.Provider) (executor.Ex
rootless: opt.Rootless,
networkProviders: networkProviders,
processMode: opt.ProcessMode,
idmap: opt.IdentityMapping,
}
return w, nil
}
@ -157,8 +161,14 @@ func (w *runcExecutor) Exec(ctx context.Context, meta executor.Meta, root cache.
return err
}
defer os.RemoveAll(bundle)
identity := idtools.Identity{}
if w.idmap != nil {
identity = w.idmap.RootPair()
}
rootFSPath := filepath.Join(bundle, "rootfs")
if err := os.Mkdir(rootFSPath, 0700); err != nil {
if err := idtools.MkdirAllAndChown(rootFSPath, 0700, identity); err != nil {
return err
}
if err := mount.All(rootMount, rootFSPath); err != nil {
@ -193,7 +203,7 @@ func (w *runcExecutor) Exec(ctx context.Context, meta executor.Meta, root cache.
}
opts = append(opts, containerdoci.WithCgroup(cgroupsPath))
}
spec, cleanup, err := oci.GenerateSpec(ctx, meta, mounts, id, resolvConf, hostsFile, namespace, w.processMode, opts...)
spec, cleanup, err := oci.GenerateSpec(ctx, meta, mounts, id, resolvConf, hostsFile, namespace, w.processMode, w.idmap, opts...)
if err != nil {
return err
}
@ -208,7 +218,7 @@ func (w *runcExecutor) Exec(ctx context.Context, meta executor.Meta, root cache.
if err != nil {
return errors.Wrapf(err, "working dir %s points to invalid target", newp)
}
if err := os.MkdirAll(newp, 0755); err != nil {
if err := idtools.MkdirAllAndChown(newp, 0755, identity); err != nil {
return errors.Wrapf(err, "failed to create working directory %s", newp)
}

View File

@ -7,6 +7,7 @@ import (
"strings"
"time"
"github.com/docker/docker/pkg/idtools"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/exporter"
"github.com/moby/buildkit/session"
@ -68,6 +69,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source)
return func() error {
var src string
var err error
var idmap *idtools.IdentityMapping
if ref == nil {
src, err = ioutil.TempDir("", "buildkit")
if err != nil {
@ -86,10 +88,30 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source)
if err != nil {
return err
}
idmap = mount.IdentityMapping()
defer lm.Unmount()
}
fs := fsutil.NewFS(src, nil)
walkOpt := &fsutil.WalkOpt{}
if idmap != nil {
walkOpt.Map = func(p string, st *fstypes.Stat) bool {
id, err := idmap.ToHost(idtools.Identity{
UID: int(st.Uid),
GID: int(st.Gid),
})
if err != nil {
return false
}
st.Uid = uint32(id.UID)
st.Gid = uint32(id.GID)
return true
}
}
fs := fsutil.NewFS(src, walkOpt)
lbl := "copying files"
if isMap {
lbl += " " + k

View File

@ -57,7 +57,7 @@ func (wc *streamWriterCloser) Close() error {
return nil
}
func recvDiffCopy(ds grpc.Stream, dest string, cu CacheUpdater, progress progressCb) error {
func recvDiffCopy(ds grpc.Stream, dest string, cu CacheUpdater, progress progressCb, filter func(string, *fstypes.Stat) bool) error {
st := time.Now()
defer func() {
logrus.Debugf("diffcopy took: %v", time.Since(st))
@ -73,6 +73,7 @@ func recvDiffCopy(ds grpc.Stream, dest string, cu CacheUpdater, progress progres
NotifyHashed: cf,
ContentHasher: ch,
ProgressCb: progress,
Filter: fsutil.FilterFunc(filter),
})
}

View File

@ -129,7 +129,7 @@ type progressCb func(int, bool)
type protocol struct {
name string
sendFn func(stream grpc.Stream, fs fsutil.FS, progress progressCb) error
recvFn func(stream grpc.Stream, destDir string, cu CacheUpdater, progress progressCb) error
recvFn func(stream grpc.Stream, destDir string, cu CacheUpdater, progress progressCb, mapFunc func(string, *fstypes.Stat) bool) error
}
func isProtoSupported(p string) bool {
@ -158,6 +158,7 @@ type FSSendRequestOpt struct {
DestDir string
CacheUpdater CacheUpdater
ProgressCb func(int, bool)
Filter func(string, *fstypes.Stat) bool
}
// CacheUpdater is an object capable of sending notifications for the cache hash changes
@ -225,7 +226,7 @@ func FSSync(ctx context.Context, c session.Caller, opt FSSendRequestOpt) error {
panic(fmt.Sprintf("invalid protocol: %q", pr.name))
}
return pr.recvFn(stream, opt.DestDir, opt.CacheUpdater, opt.ProgressCb)
return pr.recvFn(stream, opt.DestDir, opt.CacheUpdater, opt.ProgressCb, opt.Filter)
}
// NewFSSyncTargetDir allows writing into a directory

View File

@ -335,6 +335,16 @@ func (gs *gitSourceHandler) Snapshot(ctx context.Context) (out cache.ImmutableRe
return nil, errors.Wrapf(err, "failed to update submodules for %s", gs.src.Remote)
}
if idmap := mount.IdentityMapping(); idmap != nil {
u := idmap.RootPair()
err := filepath.Walk(gitDir, func(p string, f os.FileInfo, err error) error {
return os.Lchown(p, u.UID, u.GID)
})
if err != nil {
return nil, errors.Wrap(err, "failed to remap git checkout")
}
}
lm.Unmount()
lm = nil

View File

@ -15,6 +15,7 @@ import (
"strings"
"time"
"github.com/docker/docker/pkg/idtools"
"github.com/docker/docker/pkg/locker"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/cache/metadata"
@ -278,8 +279,21 @@ func (hs *httpSourceHandler) save(ctx context.Context, resp *http.Response) (ref
}
f = nil
if hs.src.UID != 0 || hs.src.GID != 0 {
if err := os.Chown(fp, hs.src.UID, hs.src.GID); err != nil {
uid := hs.src.UID
gid := hs.src.GID
if idmap := mount.IdentityMapping(); idmap != nil {
var err error
uid, gid, err = idmap.ToContainer(idtools.Identity{
UID: int(uid),
GID: int(gid),
})
if err != nil {
return nil, "", err
}
}
if gid != 0 || uid != 0 {
if err := os.Chown(fp, uid, gid); err != nil {
return nil, "", err
}
}

View File

@ -4,8 +4,10 @@ import (
"context"
"encoding/json"
"fmt"
"hash"
"time"
"github.com/docker/docker/pkg/idtools"
"github.com/moby/buildkit/cache"
"github.com/moby/buildkit/cache/contenthash"
"github.com/moby/buildkit/cache/metadata"
@ -19,6 +21,7 @@ import (
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/tonistiigi/fsutil"
fstypes "github.com/tonistiigi/fsutil/types"
bolt "go.etcd.io/bbolt"
"golang.org/x/time/rate"
"google.golang.org/grpc/codes"
@ -153,7 +156,7 @@ func (ls *localSourceHandler) Snapshot(ctx context.Context) (out cache.Immutable
}
}()
cc, err := contenthash.GetCacheContext(ctx, mutable.Metadata())
cc, err := contenthash.GetCacheContext(ctx, mutable.Metadata(), mount.IdentityMapping())
if err != nil {
return nil, err
}
@ -165,10 +168,25 @@ func (ls *localSourceHandler) Snapshot(ctx context.Context) (out cache.Immutable
FollowPaths: ls.src.FollowPaths,
OverrideExcludes: false,
DestDir: dest,
CacheUpdater: &cacheUpdater{cc},
CacheUpdater: &cacheUpdater{cc, mount.IdentityMapping()},
ProgressCb: newProgressHandler(ctx, "transferring "+ls.src.Name+":"),
}
if idmap := mount.IdentityMapping(); idmap != nil {
opt.Filter = func(_ string, stat *fstypes.Stat) bool {
uid, gid, err := idmap.ToContainer(idtools.Identity{
UID: int(stat.Uid),
GID: int(stat.Gid),
})
if err != nil {
return false
}
stat.Uid = uint32(uid)
stat.Gid = uint32(gid)
return true
}
}
if err := filesync.FSSync(ctx, caller, opt); err != nil {
if status.Code(err) == codes.NotFound {
return nil, errors.Errorf("local source %s not enabled from the client", ls.src.Name)
@ -245,11 +263,27 @@ func newProgressHandler(ctx context.Context, id string) func(int, bool) {
type cacheUpdater struct {
contenthash.CacheContext
idmap *idtools.IdentityMapping
}
func (cu *cacheUpdater) MarkSupported(bool) {
}
func (cu *cacheUpdater) ContentHasher() fsutil.ContentHasher {
if cu.idmap != nil {
return func(stat *fstypes.Stat) (hash.Hash, error) {
s := *stat
id, err := cu.idmap.ToHost(idtools.Identity{
UID: int(stat.Uid),
GID: int(stat.Gid),
})
if err != nil {
return nil, err
}
s.Uid = uint32(id.UID)
s.Gid = uint32(id.GID)
return contenthash.NewFromStat(&s)
}
}
return contenthash.NewFromStat
}

View File

@ -48,8 +48,9 @@ func NewWorkerOpt(root string, snFactory SnapshotterFactory, rootless bool, proc
// Root directory
Root: filepath.Join(root, "executor"),
// without root privileges
Rootless: rootless,
ProcessMode: processMode,
Rootless: rootless,
ProcessMode: processMode,
IdentityMapping: idmap,
}, network.Default())
if err != nil {
return opt, err