2018-04-13 21:03:55 +00:00
|
|
|
package ops
|
|
|
|
|
|
|
|
import (
|
|
|
|
"bytes"
|
|
|
|
"context"
|
|
|
|
"encoding/json"
|
|
|
|
"fmt"
|
2018-07-19 21:44:18 +00:00
|
|
|
"io/ioutil"
|
2018-08-01 21:15:43 +00:00
|
|
|
"net"
|
2018-05-25 18:15:32 +00:00
|
|
|
"os"
|
2018-04-13 21:03:55 +00:00
|
|
|
"path"
|
2018-07-19 21:44:18 +00:00
|
|
|
"path/filepath"
|
2018-04-13 21:03:55 +00:00
|
|
|
"runtime"
|
|
|
|
"sort"
|
|
|
|
"strings"
|
2018-06-08 22:57:17 +00:00
|
|
|
"sync"
|
2018-06-18 20:57:36 +00:00
|
|
|
"time"
|
2018-04-13 21:03:55 +00:00
|
|
|
|
2018-06-08 05:28:52 +00:00
|
|
|
"github.com/containerd/containerd/mount"
|
2018-06-18 20:57:36 +00:00
|
|
|
"github.com/docker/docker/pkg/locker"
|
2018-04-13 21:03:55 +00:00
|
|
|
"github.com/moby/buildkit/cache"
|
2018-05-15 19:09:09 +00:00
|
|
|
"github.com/moby/buildkit/cache/metadata"
|
2018-07-26 19:07:52 +00:00
|
|
|
"github.com/moby/buildkit/client"
|
2018-04-13 21:03:55 +00:00
|
|
|
"github.com/moby/buildkit/executor"
|
2018-07-19 21:44:18 +00:00
|
|
|
"github.com/moby/buildkit/identity"
|
|
|
|
"github.com/moby/buildkit/session"
|
|
|
|
"github.com/moby/buildkit/session/secrets"
|
2018-09-04 23:33:07 +00:00
|
|
|
"github.com/moby/buildkit/session/sshforward"
|
2018-06-08 05:28:52 +00:00
|
|
|
"github.com/moby/buildkit/snapshot"
|
2018-05-11 05:58:41 +00:00
|
|
|
"github.com/moby/buildkit/solver"
|
|
|
|
"github.com/moby/buildkit/solver/llbsolver"
|
2018-04-13 21:03:55 +00:00
|
|
|
"github.com/moby/buildkit/solver/pb"
|
|
|
|
"github.com/moby/buildkit/util/progress/logs"
|
2018-09-20 12:57:28 +00:00
|
|
|
utilsystem "github.com/moby/buildkit/util/system"
|
2018-04-13 21:03:55 +00:00
|
|
|
"github.com/moby/buildkit/worker"
|
|
|
|
digest "github.com/opencontainers/go-digest"
|
2018-07-20 04:55:57 +00:00
|
|
|
"github.com/opencontainers/runc/libcontainer/system"
|
2018-04-13 21:03:55 +00:00
|
|
|
"github.com/pkg/errors"
|
2018-05-15 19:09:09 +00:00
|
|
|
"github.com/sirupsen/logrus"
|
2018-09-18 18:18:08 +00:00
|
|
|
bolt "go.etcd.io/bbolt"
|
2018-09-04 23:33:07 +00:00
|
|
|
"google.golang.org/grpc/codes"
|
|
|
|
"google.golang.org/grpc/status"
|
2018-04-13 21:03:55 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
const execCacheType = "buildkit.exec.v0"
|
|
|
|
|
|
|
|
type execOp struct {
|
|
|
|
op *pb.ExecOp
|
|
|
|
cm cache.Manager
|
2018-07-19 21:44:18 +00:00
|
|
|
sm *session.Manager
|
2018-05-15 19:09:09 +00:00
|
|
|
md *metadata.Store
|
2018-04-13 21:03:55 +00:00
|
|
|
exec executor.Executor
|
|
|
|
w worker.Worker
|
|
|
|
numInputs int
|
2018-06-18 20:57:36 +00:00
|
|
|
|
|
|
|
cacheMounts map[string]*cacheRefShare
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
2018-07-19 21:44:18 +00:00
|
|
|
func NewExecOp(v solver.Vertex, op *pb.Op_Exec, cm cache.Manager, sm *session.Manager, md *metadata.Store, exec executor.Executor, w worker.Worker) (solver.Op, error) {
|
2018-04-13 21:03:55 +00:00
|
|
|
return &execOp{
|
2018-06-18 20:57:36 +00:00
|
|
|
op: op.Exec,
|
|
|
|
cm: cm,
|
2018-07-19 21:44:18 +00:00
|
|
|
sm: sm,
|
2018-06-18 20:57:36 +00:00
|
|
|
md: md,
|
|
|
|
exec: exec,
|
|
|
|
numInputs: len(v.Inputs()),
|
|
|
|
w: w,
|
|
|
|
cacheMounts: map[string]*cacheRefShare{},
|
2018-04-13 21:03:55 +00:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func cloneExecOp(old *pb.ExecOp) pb.ExecOp {
|
|
|
|
n := *old
|
2018-05-20 21:48:35 +00:00
|
|
|
meta := *n.Meta
|
2018-08-01 21:15:43 +00:00
|
|
|
meta.ExtraHosts = nil
|
|
|
|
for i := range n.Meta.ExtraHosts {
|
|
|
|
h := *n.Meta.ExtraHosts[i]
|
|
|
|
meta.ExtraHosts = append(meta.ExtraHosts, &h)
|
|
|
|
}
|
2018-05-20 21:48:35 +00:00
|
|
|
n.Meta = &meta
|
2018-04-13 21:03:55 +00:00
|
|
|
n.Mounts = nil
|
|
|
|
for i := range n.Mounts {
|
|
|
|
m := *n.Mounts[i]
|
|
|
|
n.Mounts = append(n.Mounts, &m)
|
|
|
|
}
|
|
|
|
return n
|
|
|
|
}
|
|
|
|
|
2018-04-24 21:00:58 +00:00
|
|
|
func (e *execOp) CacheMap(ctx context.Context, index int) (*solver.CacheMap, bool, error) {
|
2018-04-13 21:03:55 +00:00
|
|
|
op := cloneExecOp(e.op)
|
2018-08-01 21:15:43 +00:00
|
|
|
for i := range op.Meta.ExtraHosts {
|
|
|
|
h := op.Meta.ExtraHosts[i]
|
|
|
|
h.IP = ""
|
|
|
|
op.Meta.ExtraHosts[i] = h
|
|
|
|
}
|
2018-04-13 21:03:55 +00:00
|
|
|
for i := range op.Mounts {
|
|
|
|
op.Mounts[i].Selector = ""
|
|
|
|
}
|
2018-05-20 21:48:35 +00:00
|
|
|
op.Meta.ProxyEnv = nil
|
2018-04-13 21:03:55 +00:00
|
|
|
|
|
|
|
dt, err := json.Marshal(struct {
|
|
|
|
Type string
|
|
|
|
Exec *pb.ExecOp
|
|
|
|
OS string
|
|
|
|
Arch string
|
|
|
|
}{
|
|
|
|
Type: execCacheType,
|
|
|
|
Exec: &op,
|
|
|
|
OS: runtime.GOOS,
|
|
|
|
Arch: runtime.GOARCH,
|
|
|
|
})
|
|
|
|
if err != nil {
|
2018-04-24 21:00:58 +00:00
|
|
|
return nil, false, err
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
cm := &solver.CacheMap{
|
|
|
|
Digest: digest.FromBytes(dt),
|
|
|
|
Deps: make([]struct {
|
|
|
|
Selector digest.Digest
|
|
|
|
ComputeDigestFunc solver.ResultBasedCacheFunc
|
|
|
|
}, e.numInputs),
|
|
|
|
}
|
|
|
|
|
|
|
|
deps, err := e.getMountDeps()
|
|
|
|
if err != nil {
|
2018-04-24 21:00:58 +00:00
|
|
|
return nil, false, err
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for i, dep := range deps {
|
|
|
|
if len(dep.Selectors) != 0 {
|
|
|
|
dgsts := make([][]byte, 0, len(dep.Selectors))
|
|
|
|
for _, p := range dep.Selectors {
|
|
|
|
dgsts = append(dgsts, []byte(p))
|
|
|
|
}
|
|
|
|
cm.Deps[i].Selector = digest.FromBytes(bytes.Join(dgsts, []byte{0}))
|
|
|
|
}
|
|
|
|
if !dep.NoContentBasedHash {
|
|
|
|
cm.Deps[i].ComputeDigestFunc = llbsolver.NewContentHashFunc(dedupePaths(dep.Selectors))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-04-24 21:00:58 +00:00
|
|
|
return cm, true, nil
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func dedupePaths(inp []string) []string {
|
|
|
|
old := make(map[string]struct{}, len(inp))
|
|
|
|
for _, p := range inp {
|
|
|
|
old[p] = struct{}{}
|
|
|
|
}
|
|
|
|
paths := make([]string, 0, len(old))
|
|
|
|
for p1 := range old {
|
|
|
|
var skip bool
|
|
|
|
for p2 := range old {
|
|
|
|
if p1 != p2 && strings.HasPrefix(p1, p2) {
|
|
|
|
skip = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if !skip {
|
|
|
|
paths = append(paths, p1)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
sort.Slice(paths, func(i, j int) bool {
|
|
|
|
return paths[i] < paths[j]
|
|
|
|
})
|
|
|
|
return paths
|
|
|
|
}
|
|
|
|
|
|
|
|
type dep struct {
|
|
|
|
Selectors []string
|
|
|
|
NoContentBasedHash bool
|
|
|
|
}
|
|
|
|
|
|
|
|
func (e *execOp) getMountDeps() ([]dep, error) {
|
|
|
|
deps := make([]dep, e.numInputs)
|
|
|
|
for _, m := range e.op.Mounts {
|
|
|
|
if m.Input == pb.Empty {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if int(m.Input) >= len(deps) {
|
|
|
|
return nil, errors.Errorf("invalid mountinput %v", m)
|
|
|
|
}
|
|
|
|
|
|
|
|
sel := m.Selector
|
|
|
|
if sel != "" {
|
|
|
|
sel = path.Join("/", sel)
|
|
|
|
deps[m.Input].Selectors = append(deps[m.Input].Selectors, sel)
|
|
|
|
}
|
|
|
|
|
|
|
|
if !m.Readonly || m.Dest == pb.RootMount { // exclude read-only rootfs
|
|
|
|
deps[m.Input].NoContentBasedHash = true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return deps, nil
|
|
|
|
}
|
|
|
|
|
2018-06-18 20:57:36 +00:00
|
|
|
func (e *execOp) getRefCacheDir(ctx context.Context, ref cache.ImmutableRef, id string, m *pb.Mount, sharing pb.CacheSharingOpt) (mref cache.MutableRef, err error) {
|
2018-05-15 19:09:09 +00:00
|
|
|
|
|
|
|
key := "cache-dir:" + id
|
|
|
|
if ref != nil {
|
|
|
|
key += ":" + ref.ID()
|
|
|
|
}
|
2018-06-08 22:57:17 +00:00
|
|
|
|
2018-06-18 20:57:36 +00:00
|
|
|
if ref, ok := e.cacheMounts[key]; ok {
|
|
|
|
return ref.clone(), nil
|
|
|
|
}
|
|
|
|
defer func() {
|
|
|
|
if err == nil {
|
|
|
|
share := &cacheRefShare{MutableRef: mref, refs: map[*cacheRef]struct{}{}}
|
|
|
|
e.cacheMounts[key] = share
|
|
|
|
mref = share.clone()
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
switch sharing {
|
|
|
|
case pb.CacheSharingOpt_SHARED:
|
|
|
|
return sharedCacheRefs.get(key, func() (cache.MutableRef, error) {
|
|
|
|
return e.getRefCacheDirNoCache(ctx, key, ref, id, m, false)
|
|
|
|
})
|
|
|
|
case pb.CacheSharingOpt_PRIVATE:
|
|
|
|
return e.getRefCacheDirNoCache(ctx, key, ref, id, m, false)
|
|
|
|
case pb.CacheSharingOpt_LOCKED:
|
|
|
|
return e.getRefCacheDirNoCache(ctx, key, ref, id, m, true)
|
|
|
|
default:
|
|
|
|
return nil, errors.Errorf("invalid cache sharing option: %s", sharing.String())
|
|
|
|
}
|
|
|
|
|
2018-06-08 22:57:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-18 20:57:36 +00:00
|
|
|
func (e *execOp) getRefCacheDirNoCache(ctx context.Context, key string, ref cache.ImmutableRef, id string, m *pb.Mount, block bool) (cache.MutableRef, error) {
|
2018-06-08 22:57:17 +00:00
|
|
|
makeMutable := func(cache.ImmutableRef) (cache.MutableRef, error) {
|
|
|
|
desc := fmt.Sprintf("cached mount %s from exec %s", m.Dest, strings.Join(e.op.Meta.Args, " "))
|
2018-07-26 19:07:52 +00:00
|
|
|
return e.cm.New(ctx, ref, cache.WithRecordType(client.UsageRecordTypeCacheMount), cache.WithDescription(desc), cache.CachePolicyRetain)
|
2018-06-08 22:57:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-18 20:57:36 +00:00
|
|
|
cacheRefsLocker.Lock(key)
|
|
|
|
defer cacheRefsLocker.Unlock(key)
|
|
|
|
for {
|
|
|
|
sis, err := e.md.Search(key)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
locked := false
|
|
|
|
for _, si := range sis {
|
|
|
|
if mRef, err := e.cm.GetMutable(ctx, si.ID()); err == nil {
|
|
|
|
logrus.Debugf("reusing ref for cache dir: %s", mRef.ID())
|
|
|
|
return mRef, nil
|
|
|
|
} else if errors.Cause(err) == cache.ErrLocked {
|
|
|
|
locked = true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if block && locked {
|
|
|
|
cacheRefsLocker.Unlock(key)
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
cacheRefsLocker.Lock(key)
|
|
|
|
return nil, ctx.Err()
|
|
|
|
case <-time.After(100 * time.Millisecond):
|
|
|
|
cacheRefsLocker.Lock(key)
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
break
|
2018-05-15 19:09:09 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
mRef, err := makeMutable(ref)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
si, _ := e.md.Get(mRef.ID())
|
|
|
|
v, err := metadata.NewValue(key)
|
|
|
|
if err != nil {
|
|
|
|
mRef.Release(context.TODO())
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
v.Index = key
|
|
|
|
if err := si.Update(func(b *bolt.Bucket) error {
|
|
|
|
return si.SetValue(b, key, v)
|
|
|
|
}); err != nil {
|
|
|
|
mRef.Release(context.TODO())
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return mRef, nil
|
|
|
|
}
|
|
|
|
|
2018-09-04 23:33:07 +00:00
|
|
|
func (e *execOp) getSSHMountable(ctx context.Context, m *pb.Mount) (cache.Mountable, error) {
|
|
|
|
sessionID := session.FromContext(ctx)
|
|
|
|
if sessionID == "" {
|
|
|
|
return nil, errors.New("could not access local files without session")
|
|
|
|
}
|
|
|
|
|
|
|
|
timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
|
|
|
caller, err := e.sm.Get(timeoutCtx, sessionID)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := sshforward.CheckSSHID(ctx, caller, m.SSHOpt.ID); err != nil {
|
|
|
|
if m.SSHOpt.Optional {
|
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
if st, ok := status.FromError(err); ok && st.Code() == codes.Unimplemented {
|
|
|
|
return nil, errors.Errorf("no ssh forwarded from the client")
|
|
|
|
}
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return &sshMount{mount: m, caller: caller}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type sshMount struct {
|
|
|
|
mount *pb.Mount
|
|
|
|
caller session.Caller
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *sshMount) Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error) {
|
|
|
|
return &sshMountInstance{sm: sm}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type sshMountInstance struct {
|
|
|
|
sm *sshMount
|
|
|
|
cleanup func() error
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *sshMountInstance) Mount() ([]mount.Mount, error) {
|
|
|
|
ctx, cancel := context.WithCancel(context.TODO())
|
|
|
|
|
|
|
|
sock, cleanup, err := sshforward.MountSSHSocket(ctx, sm.sm.caller, sshforward.SocketOpt{
|
|
|
|
ID: sm.sm.mount.SSHOpt.ID,
|
|
|
|
UID: int(sm.sm.mount.SSHOpt.Uid),
|
|
|
|
GID: int(sm.sm.mount.SSHOpt.Gid),
|
|
|
|
Mode: int(sm.sm.mount.SSHOpt.Mode),
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
cancel()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
sm.cleanup = func() error {
|
|
|
|
var err error
|
|
|
|
if cleanup != nil {
|
|
|
|
err = cleanup()
|
|
|
|
}
|
|
|
|
cancel()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
return []mount.Mount{{
|
|
|
|
Type: "bind",
|
|
|
|
Source: sock,
|
|
|
|
Options: []string{"rbind"},
|
|
|
|
}}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *sshMountInstance) Release() error {
|
|
|
|
if sm.cleanup != nil {
|
|
|
|
if err := sm.cleanup(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-07-19 21:44:18 +00:00
|
|
|
func (e *execOp) getSecretMountable(ctx context.Context, m *pb.Mount) (cache.Mountable, error) {
|
|
|
|
if m.SecretOpt == nil {
|
|
|
|
return nil, errors.Errorf("invalid sercet mount options")
|
|
|
|
}
|
|
|
|
sopt := *m.SecretOpt
|
|
|
|
|
|
|
|
id := sopt.ID
|
|
|
|
if id == "" {
|
|
|
|
return nil, errors.Errorf("secret ID missing from mount options")
|
|
|
|
}
|
|
|
|
|
|
|
|
sessionID := session.FromContext(ctx)
|
|
|
|
if sessionID == "" {
|
|
|
|
return nil, errors.New("could not access local files without session")
|
|
|
|
}
|
|
|
|
|
|
|
|
timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
|
|
|
caller, err := e.sm.Get(timeoutCtx, sessionID)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
dt, err := secrets.GetSecret(ctx, caller, id)
|
|
|
|
if err != nil {
|
|
|
|
if errors.Cause(err) == secrets.ErrNotFound && m.SecretOpt.Optional {
|
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return &secretMount{mount: m, data: dt}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type secretMount struct {
|
|
|
|
mount *pb.Mount
|
|
|
|
data []byte
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *secretMount) Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error) {
|
|
|
|
return &secretMountInstance{sm: sm}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type secretMountInstance struct {
|
|
|
|
sm *secretMount
|
|
|
|
root string
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *secretMountInstance) Mount() ([]mount.Mount, error) {
|
|
|
|
dir, err := ioutil.TempDir("", "buildkit-secrets")
|
|
|
|
if err != nil {
|
|
|
|
return nil, errors.Wrap(err, "failed to create temp dir")
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := os.Chmod(dir, 0711); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-07-20 04:55:57 +00:00
|
|
|
tmpMount := mount.Mount{
|
2018-07-19 21:44:18 +00:00
|
|
|
Type: "tmpfs",
|
|
|
|
Source: "tmpfs",
|
2018-07-20 04:55:57 +00:00
|
|
|
Options: []string{"nodev", "nosuid", "noexec", fmt.Sprintf("uid=%d,gid=%d", os.Geteuid(), os.Getegid())},
|
|
|
|
}
|
|
|
|
|
|
|
|
if system.RunningInUserNS() {
|
|
|
|
tmpMount.Options = nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := mount.All([]mount.Mount{tmpMount}, dir); err != nil {
|
2018-07-19 21:44:18 +00:00
|
|
|
return nil, errors.Wrap(err, "unable to setup secret mount")
|
|
|
|
}
|
|
|
|
sm.root = dir
|
|
|
|
|
|
|
|
randID := identity.NewID()
|
|
|
|
fp := filepath.Join(dir, randID)
|
|
|
|
if err := ioutil.WriteFile(fp, sm.sm.data, 0600); err != nil {
|
|
|
|
sm.Release()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := os.Chown(fp, int(sm.sm.mount.SecretOpt.Uid), int(sm.sm.mount.SecretOpt.Gid)); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := os.Chmod(fp, os.FileMode(sm.sm.mount.SecretOpt.Mode)); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return []mount.Mount{{
|
|
|
|
Type: "bind",
|
|
|
|
Source: fp,
|
|
|
|
Options: []string{"ro", "rbind"},
|
|
|
|
}}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (sm *secretMountInstance) Release() error {
|
|
|
|
if sm.root != "" {
|
|
|
|
if err := mount.Unmount(sm.root, 0); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return os.RemoveAll(sm.root)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-09-20 12:57:28 +00:00
|
|
|
func addDefaultEnvvar(env []string, k, v string) []string {
|
|
|
|
for _, e := range env {
|
|
|
|
if strings.HasPrefix(e, k+"=") {
|
|
|
|
return env
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return append(env, k+"="+v)
|
|
|
|
}
|
|
|
|
|
2018-04-13 21:03:55 +00:00
|
|
|
func (e *execOp) Exec(ctx context.Context, inputs []solver.Result) ([]solver.Result, error) {
|
|
|
|
var mounts []executor.Mount
|
|
|
|
var root cache.Mountable
|
|
|
|
var readonlyRootFS bool
|
|
|
|
|
|
|
|
var outputs []cache.Ref
|
|
|
|
|
|
|
|
defer func() {
|
|
|
|
for _, o := range outputs {
|
|
|
|
if o != nil {
|
|
|
|
go o.Release(context.TODO())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
// loop over all mounts, fill in mounts, root and outputs
|
2018-04-13 21:03:55 +00:00
|
|
|
for _, m := range e.op.Mounts {
|
|
|
|
var mountable cache.Mountable
|
|
|
|
var ref cache.ImmutableRef
|
2018-05-15 17:37:56 +00:00
|
|
|
|
2018-05-15 19:09:09 +00:00
|
|
|
if m.Dest == pb.RootMount && m.MountType != pb.MountType_BIND {
|
|
|
|
return nil, errors.Errorf("invalid mount type %s for %s", m.MountType.String(), m.Dest)
|
|
|
|
}
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
// if mount is based on input validate and load it
|
2018-04-13 21:03:55 +00:00
|
|
|
if m.Input != pb.Empty {
|
|
|
|
if int(m.Input) > len(inputs) {
|
|
|
|
return nil, errors.Errorf("missing input %d", m.Input)
|
|
|
|
}
|
|
|
|
inp := inputs[int(m.Input)]
|
|
|
|
workerRef, ok := inp.Sys().(*worker.WorkerRef)
|
|
|
|
if !ok {
|
|
|
|
return nil, errors.Errorf("invalid reference for exec %T", inp.Sys())
|
|
|
|
}
|
|
|
|
ref = workerRef.ImmutableRef
|
|
|
|
mountable = ref
|
|
|
|
}
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
makeMutable := func(cache.ImmutableRef) (cache.MutableRef, error) {
|
2018-04-13 21:03:55 +00:00
|
|
|
desc := fmt.Sprintf("mount %s from exec %s", m.Dest, strings.Join(e.op.Meta.Args, " "))
|
2018-05-10 04:58:29 +00:00
|
|
|
return e.cm.New(ctx, ref, cache.WithDescription(desc))
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
2018-05-15 19:09:09 +00:00
|
|
|
switch m.MountType {
|
|
|
|
case pb.MountType_BIND:
|
|
|
|
// if mount creates an output
|
|
|
|
if m.Output != pb.SkipOutput {
|
|
|
|
// it it is readonly and not root then output is the input
|
|
|
|
if m.Readonly && ref != nil && m.Dest != pb.RootMount {
|
|
|
|
outputs = append(outputs, ref.Clone())
|
|
|
|
} else {
|
|
|
|
// otherwise output and mount is the mutable child
|
|
|
|
active, err := makeMutable(ref)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
outputs = append(outputs, active)
|
|
|
|
mountable = active
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
2018-06-07 20:54:04 +00:00
|
|
|
} else if ref == nil {
|
|
|
|
// this case is empty readonly scratch without output that is not really useful for anything but don't error
|
|
|
|
active, err := makeMutable(ref)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
defer active.Release(context.TODO())
|
|
|
|
mountable = active
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
2018-06-07 20:54:04 +00:00
|
|
|
|
2018-05-15 19:09:09 +00:00
|
|
|
case pb.MountType_CACHE:
|
|
|
|
if m.CacheOpt == nil {
|
|
|
|
return nil, errors.Errorf("missing cache mount options")
|
|
|
|
}
|
2018-06-18 20:57:36 +00:00
|
|
|
mRef, err := e.getRefCacheDir(ctx, ref, m.CacheOpt.ID, m, m.CacheOpt.Sharing)
|
2018-05-15 19:09:09 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
mountable = mRef
|
|
|
|
defer func() {
|
|
|
|
go mRef.Release(context.TODO())
|
|
|
|
}()
|
|
|
|
if m.Output != pb.SkipOutput && ref != nil {
|
|
|
|
outputs = append(outputs, ref.Clone())
|
|
|
|
}
|
2018-06-08 05:28:52 +00:00
|
|
|
|
|
|
|
case pb.MountType_TMPFS:
|
|
|
|
mountable = newTmpfs()
|
|
|
|
|
2018-07-19 21:44:18 +00:00
|
|
|
case pb.MountType_SECRET:
|
|
|
|
secretMount, err := e.getSecretMountable(ctx, m)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if secretMount == nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
mountable = secretMount
|
2018-09-04 23:33:07 +00:00
|
|
|
|
|
|
|
case pb.MountType_SSH:
|
|
|
|
sshMount, err := e.getSSHMountable(ctx, m)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if sshMount == nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
mountable = sshMount
|
|
|
|
|
2018-05-15 19:09:09 +00:00
|
|
|
default:
|
|
|
|
return nil, errors.Errorf("mount type %s not implemented", m.MountType)
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
// validate that there is a mount
|
2018-04-13 21:03:55 +00:00
|
|
|
if mountable == nil {
|
|
|
|
return nil, errors.Errorf("mount %s has no input", m.Dest)
|
|
|
|
}
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
// if dest is root we need mutable ref even if there is no output
|
2018-04-13 21:03:55 +00:00
|
|
|
if m.Dest == pb.RootMount {
|
|
|
|
root = mountable
|
|
|
|
readonlyRootFS = m.Readonly
|
|
|
|
if m.Output == pb.SkipOutput && readonlyRootFS {
|
2018-05-15 17:37:56 +00:00
|
|
|
active, err := makeMutable(ref)
|
2018-04-13 21:03:55 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
defer func() {
|
|
|
|
go active.Release(context.TODO())
|
|
|
|
}()
|
|
|
|
root = active
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
mounts = append(mounts, executor.Mount{Src: mountable, Dest: m.Dest, Readonly: m.Readonly, Selector: m.Selector})
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-05-15 17:37:56 +00:00
|
|
|
// sort mounts so parents are mounted first
|
2018-04-13 21:03:55 +00:00
|
|
|
sort.Slice(mounts, func(i, j int) bool {
|
|
|
|
return mounts[i].Dest < mounts[j].Dest
|
|
|
|
})
|
|
|
|
|
2018-08-01 21:15:43 +00:00
|
|
|
extraHosts, err := parseExtraHosts(e.op.Meta.ExtraHosts)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-04-13 21:03:55 +00:00
|
|
|
meta := executor.Meta{
|
|
|
|
Args: e.op.Meta.Args,
|
|
|
|
Env: e.op.Meta.Env,
|
|
|
|
Cwd: e.op.Meta.Cwd,
|
|
|
|
User: e.op.Meta.User,
|
|
|
|
ReadonlyRootFS: readonlyRootFS,
|
2018-08-01 21:15:43 +00:00
|
|
|
ExtraHosts: extraHosts,
|
2018-08-04 19:42:01 +00:00
|
|
|
NetMode: e.op.Network,
|
2018-04-13 21:03:55 +00:00
|
|
|
}
|
|
|
|
|
2018-05-20 21:48:35 +00:00
|
|
|
if e.op.Meta.ProxyEnv != nil {
|
|
|
|
meta.Env = append(meta.Env, proxyEnvList(e.op.Meta.ProxyEnv)...)
|
|
|
|
}
|
2018-09-20 12:57:28 +00:00
|
|
|
meta.Env = addDefaultEnvvar(meta.Env, "PATH", utilsystem.DefaultPathEnv)
|
2018-05-20 21:48:35 +00:00
|
|
|
|
2018-05-25 18:15:32 +00:00
|
|
|
stdout, stderr := logs.NewLogStreams(ctx, os.Getenv("BUILDKIT_DEBUG_EXEC_OUTPUT") == "1")
|
2018-04-13 21:03:55 +00:00
|
|
|
defer stdout.Close()
|
|
|
|
defer stderr.Close()
|
|
|
|
|
|
|
|
if err := e.exec.Exec(ctx, meta, root, mounts, nil, stdout, stderr); err != nil {
|
|
|
|
return nil, errors.Wrapf(err, "executor failed running %v", meta.Args)
|
|
|
|
}
|
|
|
|
|
|
|
|
refs := []solver.Result{}
|
|
|
|
for i, out := range outputs {
|
|
|
|
if mutable, ok := out.(cache.MutableRef); ok {
|
|
|
|
ref, err := mutable.Commit(ctx)
|
|
|
|
if err != nil {
|
|
|
|
return nil, errors.Wrapf(err, "error committing %s", mutable.ID())
|
|
|
|
}
|
|
|
|
refs = append(refs, worker.NewWorkerRefResult(ref, e.w))
|
|
|
|
} else {
|
|
|
|
refs = append(refs, worker.NewWorkerRefResult(out.(cache.ImmutableRef), e.w))
|
|
|
|
}
|
|
|
|
outputs[i] = nil
|
|
|
|
}
|
|
|
|
return refs, nil
|
|
|
|
}
|
2018-05-20 21:48:35 +00:00
|
|
|
|
|
|
|
func proxyEnvList(p *pb.ProxyEnv) []string {
|
|
|
|
out := []string{}
|
|
|
|
if v := p.HttpProxy; v != "" {
|
|
|
|
out = append(out, "HTTP_PROXY="+v, "http_proxy="+v)
|
|
|
|
}
|
|
|
|
if v := p.HttpsProxy; v != "" {
|
|
|
|
out = append(out, "HTTPS_PROXY="+v, "https_proxy="+v)
|
|
|
|
}
|
|
|
|
if v := p.FtpProxy; v != "" {
|
|
|
|
out = append(out, "FTP_PROXY="+v, "ftp_proxy="+v)
|
|
|
|
}
|
|
|
|
if v := p.NoProxy; v != "" {
|
|
|
|
out = append(out, "NO_PROXY="+v, "no_proxy="+v)
|
|
|
|
}
|
|
|
|
return out
|
|
|
|
}
|
2018-06-08 05:28:52 +00:00
|
|
|
|
|
|
|
func newTmpfs() cache.Mountable {
|
|
|
|
return &tmpfs{}
|
|
|
|
}
|
|
|
|
|
|
|
|
type tmpfs struct {
|
|
|
|
}
|
|
|
|
|
|
|
|
func (f *tmpfs) Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error) {
|
|
|
|
return &tmpfsMount{readonly: readonly}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type tmpfsMount struct {
|
|
|
|
readonly bool
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *tmpfsMount) Mount() ([]mount.Mount, error) {
|
|
|
|
opt := []string{"nosuid"}
|
|
|
|
if m.readonly {
|
|
|
|
opt = append(opt, "ro")
|
|
|
|
}
|
|
|
|
return []mount.Mount{{
|
|
|
|
Type: "tmpfs",
|
2018-06-08 20:42:38 +00:00
|
|
|
Source: "tmpfs",
|
2018-06-08 05:28:52 +00:00
|
|
|
Options: opt,
|
|
|
|
}}, nil
|
|
|
|
}
|
|
|
|
func (m *tmpfsMount) Release() error {
|
|
|
|
return nil
|
|
|
|
}
|
2018-06-08 22:57:17 +00:00
|
|
|
|
2018-06-18 20:57:36 +00:00
|
|
|
var cacheRefsLocker = locker.New()
|
2018-06-08 22:57:17 +00:00
|
|
|
var sharedCacheRefs = &cacheRefs{}
|
|
|
|
|
|
|
|
type cacheRefs struct {
|
|
|
|
mu sync.Mutex
|
|
|
|
shares map[string]*cacheRefShare
|
|
|
|
}
|
|
|
|
|
|
|
|
func (r *cacheRefs) get(key string, fn func() (cache.MutableRef, error)) (cache.MutableRef, error) {
|
|
|
|
r.mu.Lock()
|
|
|
|
defer r.mu.Unlock()
|
|
|
|
|
|
|
|
if r.shares == nil {
|
|
|
|
r.shares = map[string]*cacheRefShare{}
|
|
|
|
}
|
|
|
|
|
|
|
|
share, ok := r.shares[key]
|
|
|
|
if ok {
|
|
|
|
return share.clone(), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
mref, err := fn()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
share = &cacheRefShare{MutableRef: mref, main: r, key: key, refs: map[*cacheRef]struct{}{}}
|
|
|
|
r.shares[key] = share
|
|
|
|
|
|
|
|
return share.clone(), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type cacheRefShare struct {
|
|
|
|
cache.MutableRef
|
|
|
|
mu sync.Mutex
|
|
|
|
refs map[*cacheRef]struct{}
|
|
|
|
main *cacheRefs
|
|
|
|
key string
|
|
|
|
}
|
|
|
|
|
|
|
|
func (r *cacheRefShare) clone() cache.MutableRef {
|
|
|
|
cacheRef := &cacheRef{cacheRefShare: r}
|
|
|
|
r.mu.Lock()
|
|
|
|
r.refs[cacheRef] = struct{}{}
|
|
|
|
r.mu.Unlock()
|
|
|
|
return cacheRef
|
|
|
|
}
|
|
|
|
|
|
|
|
func (r *cacheRefShare) release(ctx context.Context) error {
|
2018-06-18 20:57:36 +00:00
|
|
|
if r.main != nil {
|
|
|
|
r.main.mu.Lock()
|
|
|
|
defer r.main.mu.Unlock()
|
|
|
|
delete(r.main.shares, r.key)
|
|
|
|
}
|
2018-06-08 22:57:17 +00:00
|
|
|
return r.MutableRef.Release(ctx)
|
|
|
|
}
|
|
|
|
|
|
|
|
type cacheRef struct {
|
|
|
|
*cacheRefShare
|
|
|
|
}
|
|
|
|
|
|
|
|
func (r *cacheRef) Release(ctx context.Context) error {
|
|
|
|
r.mu.Lock()
|
|
|
|
defer r.mu.Unlock()
|
|
|
|
delete(r.refs, r)
|
|
|
|
if len(r.refs) == 0 {
|
|
|
|
return r.release(ctx)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2018-08-01 21:15:43 +00:00
|
|
|
|
|
|
|
func parseExtraHosts(ips []*pb.HostIP) ([]executor.HostIP, error) {
|
|
|
|
out := make([]executor.HostIP, len(ips))
|
|
|
|
for i, hip := range ips {
|
|
|
|
ip := net.ParseIP(hip.IP)
|
|
|
|
if ip == nil {
|
|
|
|
return nil, errors.Errorf("failed to parse IP %s", hip.IP)
|
|
|
|
}
|
|
|
|
out[i] = executor.HostIP{
|
|
|
|
IP: ip,
|
|
|
|
Host: hip.Host,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return out, nil
|
|
|
|
}
|