Browse Source

vendor buildkit to 8f4dff0d16ea91cb43315d5f5aa4b27f4fe4e1f2

Signed-off-by: Tibor Vass <tibor@docker.com>
(cherry picked from commit e161a8d1e9e84f50410860d903205e6d2dc86110)
Signed-off-by: Tibor Vass <tibor@docker.com>
Tibor Vass 6 years ago
parent
commit
67541d5841

+ 1 - 1
vendor.conf

@@ -26,7 +26,7 @@ github.com/imdario/mergo v0.3.6
 golang.org/x/sync 1d60e4601c6fd243af51cc01ddf169918a5407ca
 
 # buildkit
-github.com/moby/buildkit 39404586a50d1b9d0fb1c578cf0f4de7bdb7afe5
+github.com/moby/buildkit 8f4dff0d16ea91cb43315d5f5aa4b27f4fe4e1f2
 github.com/tonistiigi/fsutil b19464cd1b6a00773b4f2eb7acf9c30426f9df42
 github.com/grpc-ecosystem/grpc-opentracing 8e809c8a86450a29b90dcc9efbf062d0fe6d9746
 github.com/opentracing/opentracing-go 1361b9cd60be79c4c3a7fa9841b3c132e40066a7

+ 102 - 16
vendor/github.com/moby/buildkit/cache/contenthash/checksum.go

@@ -10,7 +10,6 @@ import (
 	"path/filepath"
 	"sync"
 
-	"github.com/containerd/continuity/fs"
 	"github.com/docker/docker/pkg/locker"
 	iradix "github.com/hashicorp/go-immutable-radix"
 	"github.com/hashicorp/golang-lru/simplelru"
@@ -400,7 +399,11 @@ func (cc *cacheContext) commitActiveTransaction() {
 
 func (cc *cacheContext) lazyChecksum(ctx context.Context, m *mount, p string) (*CacheRecord, error) {
 	root := cc.tree.Root()
-	if cc.needsScan(root, p) {
+	scan, err := cc.needsScan(root, p)
+	if err != nil {
+		return nil, err
+	}
+	if scan {
 		if err := cc.scanPath(ctx, m, p); err != nil {
 			return nil, err
 		}
@@ -418,13 +421,13 @@ func (cc *cacheContext) lazyChecksum(ctx context.Context, m *mount, p string) (*
 }
 
 func (cc *cacheContext) checksum(ctx context.Context, root *iradix.Node, txn *iradix.Txn, m *mount, k []byte) (*CacheRecord, bool, error) {
-	v, ok := root.Get(k)
-
-	if !ok {
+	k, cr, err := getFollowLinks(root, k)
+	if err != nil {
+		return nil, false, err
+	}
+	if cr == nil {
 		return nil, false, errors.Wrapf(errNotFound, "%s not found", convertKeyToPath(k))
 	}
-	cr := v.(*CacheRecord)
-
 	if cr.Digest != "" {
 		return cr, false, nil
 	}
@@ -491,17 +494,37 @@ func (cc *cacheContext) checksum(ctx context.Context, root *iradix.Node, txn *ir
 	return cr2, true, nil
 }
 
-func (cc *cacheContext) needsScan(root *iradix.Node, p string) bool {
+// needsScan returns false if path is in the tree or a parent path is in tree
+// and subpath is missing
+func (cc *cacheContext) needsScan(root *iradix.Node, p string) (bool, error) {
+	var linksWalked int
+	return cc.needsScanFollow(root, p, &linksWalked)
+}
+
+func (cc *cacheContext) needsScanFollow(root *iradix.Node, p string, linksWalked *int) (bool, error) {
 	if p == "/" {
 		p = ""
 	}
-	if _, ok := root.Get(convertPathToKey([]byte(p))); !ok {
+	if v, ok := root.Get(convertPathToKey([]byte(p))); !ok {
 		if p == "" {
-			return true
+			return true, nil
+		}
+		return cc.needsScanFollow(root, path.Clean(path.Dir(p)), linksWalked)
+	} else {
+		cr := v.(*CacheRecord)
+		if cr.Type == CacheRecordTypeSymlink {
+			if *linksWalked > 255 {
+				return false, errTooManyLinks
+			}
+			*linksWalked++
+			link := path.Clean(cr.Linkname)
+			if !path.IsAbs(cr.Linkname) {
+				link = path.Join("/", path.Dir(p), link)
+			}
+			return cc.needsScanFollow(root, link, linksWalked)
 		}
-		return cc.needsScan(root, path.Clean(path.Dir(p)))
 	}
-	return false
+	return false, nil
 }
 
 func (cc *cacheContext) scanPath(ctx context.Context, m *mount, p string) (retErr error) {
@@ -513,14 +536,23 @@ func (cc *cacheContext) scanPath(ctx context.Context, m *mount, p string) (retEr
 		return err
 	}
 
-	parentPath, err := fs.RootPath(mp, filepath.FromSlash(d))
+	n := cc.tree.Root()
+	txn := cc.tree.Txn()
+
+	parentPath, err := rootPath(mp, filepath.FromSlash(d), func(p, link string) error {
+		cr := &CacheRecord{
+			Type:     CacheRecordTypeSymlink,
+			Linkname: filepath.ToSlash(link),
+		}
+		k := []byte(filepath.Join("/", filepath.ToSlash(p)))
+		k = convertPathToKey(k)
+		txn.Insert(k, cr)
+		return nil
+	})
 	if err != nil {
 		return err
 	}
 
-	n := cc.tree.Root()
-	txn := cc.tree.Txn()
-
 	err = filepath.Walk(parentPath, func(path string, fi os.FileInfo, err error) error {
 		if err != nil {
 			return errors.Wrapf(err, "failed to walk %s", path)
@@ -566,6 +598,45 @@ func (cc *cacheContext) scanPath(ctx context.Context, m *mount, p string) (retEr
 	return nil
 }
 
+func getFollowLinks(root *iradix.Node, k []byte) ([]byte, *CacheRecord, error) {
+	var linksWalked int
+	return getFollowLinksWalk(root, k, &linksWalked)
+}
+
+func getFollowLinksWalk(root *iradix.Node, k []byte, linksWalked *int) ([]byte, *CacheRecord, error) {
+	v, ok := root.Get(k)
+	if ok {
+		return k, v.(*CacheRecord), nil
+	}
+	if len(k) == 0 {
+		return nil, nil, nil
+	}
+
+	dir, file := splitKey(k)
+
+	_, parent, err := getFollowLinksWalk(root, dir, linksWalked)
+	if err != nil {
+		return nil, nil, err
+	}
+	if parent != nil && parent.Type == CacheRecordTypeSymlink {
+		*linksWalked++
+		if *linksWalked > 255 {
+			return nil, nil, errors.Errorf("too many links")
+		}
+		dirPath := path.Clean(string(convertKeyToPath(dir)))
+		if dirPath == "." || dirPath == "/" {
+			dirPath = ""
+		}
+		link := parent.Linkname
+		if !path.IsAbs(link) {
+			link = path.Join("/", path.Join(path.Dir(dirPath), link))
+		}
+		return getFollowLinksWalk(root, append(convertPathToKey([]byte(link)), file...), linksWalked)
+	}
+
+	return nil, nil, nil
+}
+
 func prepareDigest(fp, p string, fi os.FileInfo) (digest.Digest, error) {
 	h, err := NewFileHash(fp, fi)
 	if err != nil {
@@ -632,3 +703,18 @@ func convertPathToKey(p []byte) []byte {
 func convertKeyToPath(p []byte) []byte {
 	return bytes.Replace([]byte(p), []byte{0}, []byte("/"), -1)
 }
+
+func splitKey(k []byte) ([]byte, []byte) {
+	foundBytes := false
+	i := len(k) - 1
+	for {
+		if i <= 0 || foundBytes && k[i] == 0 {
+			break
+		}
+		if k[i] != 0 {
+			foundBytes = true
+		}
+		i--
+	}
+	return append([]byte{}, k[:i]...), k[i:]
+}

+ 107 - 0
vendor/github.com/moby/buildkit/cache/contenthash/path.go

@@ -0,0 +1,107 @@
+package contenthash
+
+import (
+	"errors"
+	"os"
+	"path/filepath"
+)
+
+var (
+	errTooManyLinks = errors.New("too many links")
+)
+
+type onSymlinkFunc func(string, string) error
+
+// rootPath joins a path with a root, evaluating and bounding any
+// symlink to the root directory.
+// This is containerd/continuity/fs RootPath implementation with a callback on
+// resolving the symlink.
+func rootPath(root, path string, cb onSymlinkFunc) (string, error) {
+	if path == "" {
+		return root, nil
+	}
+	var linksWalked int // to protect against cycles
+	for {
+		i := linksWalked
+		newpath, err := walkLinks(root, path, &linksWalked, cb)
+		if err != nil {
+			return "", err
+		}
+		path = newpath
+		if i == linksWalked {
+			newpath = filepath.Join("/", newpath)
+			if path == newpath {
+				return filepath.Join(root, newpath), nil
+			}
+			path = newpath
+		}
+	}
+}
+
+func walkLink(root, path string, linksWalked *int, cb onSymlinkFunc) (newpath string, islink bool, err error) {
+	if *linksWalked > 255 {
+		return "", false, errTooManyLinks
+	}
+
+	path = filepath.Join("/", path)
+	if path == "/" {
+		return path, false, nil
+	}
+	realPath := filepath.Join(root, path)
+
+	fi, err := os.Lstat(realPath)
+	if err != nil {
+		// If path does not yet exist, treat as non-symlink
+		if os.IsNotExist(err) {
+			return path, false, nil
+		}
+		return "", false, err
+	}
+	if fi.Mode()&os.ModeSymlink == 0 {
+		return path, false, nil
+	}
+	newpath, err = os.Readlink(realPath)
+	if err != nil {
+		return "", false, err
+	}
+	if cb != nil {
+		if err := cb(path, newpath); err != nil {
+			return "", false, err
+		}
+	}
+	*linksWalked++
+	return newpath, true, nil
+}
+
+func walkLinks(root, path string, linksWalked *int, cb onSymlinkFunc) (string, error) {
+	switch dir, file := filepath.Split(path); {
+	case dir == "":
+		newpath, _, err := walkLink(root, file, linksWalked, cb)
+		return newpath, err
+	case file == "":
+		if os.IsPathSeparator(dir[len(dir)-1]) {
+			if dir == "/" {
+				return dir, nil
+			}
+			return walkLinks(root, dir[:len(dir)-1], linksWalked, cb)
+		}
+		newpath, _, err := walkLink(root, dir, linksWalked, cb)
+		return newpath, err
+	default:
+		newdir, err := walkLinks(root, dir, linksWalked, cb)
+		if err != nil {
+			return "", err
+		}
+		newpath, islink, err := walkLink(root, filepath.Join(newdir, file), linksWalked, cb)
+		if err != nil {
+			return "", err
+		}
+		if !islink {
+			return newpath, nil
+		}
+		if filepath.IsAbs(newpath) {
+			return newpath, nil
+		}
+		return filepath.Join(newdir, newpath), nil
+	}
+}

+ 8 - 0
vendor/github.com/moby/buildkit/client/llb/state.go

@@ -411,6 +411,13 @@ func WithoutDefaultExportCache() ConstraintsOpt {
 	})
 }
 
+// WithCaps exposes supported LLB caps to the marshaler
+func WithCaps(caps apicaps.CapSet) ConstraintsOpt {
+	return constraintsOptFunc(func(c *Constraints) {
+		c.Caps = &caps
+	})
+}
+
 type constraintsWrapper struct {
 	Constraints
 }
@@ -424,6 +431,7 @@ type Constraints struct {
 	WorkerConstraints []string
 	Metadata          pb.OpMetadata
 	LocalUniqueID     string
+	Caps              *apicaps.CapSet
 }
 
 func Platform(p specs.Platform) ConstraintsOpt {

+ 18 - 5
vendor/github.com/moby/buildkit/executor/oci/spec_unix.go

@@ -11,6 +11,7 @@ import (
 	"github.com/containerd/containerd/mount"
 	"github.com/containerd/containerd/namespaces"
 	"github.com/containerd/containerd/oci"
+	"github.com/containerd/continuity/fs"
 	"github.com/mitchellh/hashstructure"
 	"github.com/moby/buildkit/executor"
 	"github.com/moby/buildkit/snapshot"
@@ -114,7 +115,11 @@ func (s *submounts) subMount(m mount.Mount, subPath string) (mount.Mount, error)
 		return mount.Mount{}, nil
 	}
 	if mr, ok := s.m[h]; ok {
-		return sub(mr.mount, subPath), nil
+		sm, err := sub(mr.mount, subPath)
+		if err != nil {
+			return mount.Mount{}, nil
+		}
+		return sm, nil
 	}
 
 	lm := snapshot.LocalMounterWithMounts([]mount.Mount{m})
@@ -140,7 +145,11 @@ func (s *submounts) subMount(m mount.Mount, subPath string) (mount.Mount, error)
 		unmount: lm.Unmount,
 	}
 
-	return sub(s.m[h].mount, subPath), nil
+	sm, err := sub(s.m[h].mount, subPath)
+	if err != nil {
+		return mount.Mount{}, err
+	}
+	return sm, nil
 }
 
 func (s *submounts) cleanup() {
@@ -157,7 +166,11 @@ func (s *submounts) cleanup() {
 	wg.Wait()
 }
 
-func sub(m mount.Mount, subPath string) mount.Mount {
-	m.Source = path.Join(m.Source, subPath)
-	return m
+func sub(m mount.Mount, subPath string) (mount.Mount, error) {
+	src, err := fs.RootPath(m.Source, subPath)
+	if err != nil {
+		return mount.Mount{}, err
+	}
+	m.Source = src
+	return m, nil
 }

+ 7 - 1
vendor/github.com/moby/buildkit/executor/runcexecutor/executor.go

@@ -263,7 +263,13 @@ func (w *runcExecutor) Exec(ctx context.Context, meta executor.Meta, root cache.
 	}
 
 	if status != 0 {
-		return errors.Errorf("exit code: %d", status)
+		err := errors.Errorf("exit code: %d", status)
+		select {
+		case <-ctx.Done():
+			return errors.Wrapf(ctx.Err(), err.Error())
+		default:
+			return err
+		}
 	}
 
 	return nil

+ 7 - 3
vendor/github.com/moby/buildkit/frontend/dockerfile/builder/build.go

@@ -48,6 +48,9 @@ var gitUrlPathWithFragmentSuffix = regexp.MustCompile("\\.git(?:#.+)?$")
 
 func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 	opts := c.BuildOpts().Opts
+	caps := c.BuildOpts().LLBCaps
+
+	marshalOpts := []llb.ConstraintsOpt{llb.WithCaps(caps)}
 
 	defaultBuildPlatform := platforms.DefaultSpec()
 	if workers := c.BuildOpts().Workers; len(workers) > 0 && len(workers[0].Platforms) > 0 {
@@ -111,7 +114,7 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 		buildContext = &src
 	} else if httpPrefix.MatchString(opts[LocalNameContext]) {
 		httpContext := llb.HTTP(opts[LocalNameContext], llb.Filename("context"), dockerfile2llb.WithInternalName("load remote build context"))
-		def, err := httpContext.Marshal()
+		def, err := httpContext.Marshal(marshalOpts...)
 		if err != nil {
 			return nil, errors.Wrapf(err, "failed to marshal httpcontext")
 		}
@@ -154,7 +157,7 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 		}
 	}
 
-	def, err := src.Marshal()
+	def, err := src.Marshal(marshalOpts...)
 	if err != nil {
 		return nil, errors.Wrapf(err, "failed to marshal local source")
 	}
@@ -195,7 +198,7 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 				)
 				dockerignoreState = &st
 			}
-			def, err := dockerignoreState.Marshal()
+			def, err := dockerignoreState.Marshal(marshalOpts...)
 			if err != nil {
 				return err
 			}
@@ -272,6 +275,7 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 					ExtraHosts:        extraHosts,
 					ForceNetMode:      defaultNetMode,
 					OverrideCopyImage: opts[keyOverrideCopyImage],
+					LLBCaps:           &caps,
 				})
 
 				if err != nil {

+ 10 - 2
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert.go

@@ -23,6 +23,7 @@ import (
 	"github.com/moby/buildkit/frontend/dockerfile/shell"
 	gw "github.com/moby/buildkit/frontend/gateway/client"
 	"github.com/moby/buildkit/solver/pb"
+	"github.com/moby/buildkit/util/apicaps"
 	"github.com/moby/buildkit/util/system"
 	specs "github.com/opencontainers/image-spec/specs-go/v1"
 	"github.com/pkg/errors"
@@ -34,7 +35,7 @@ const (
 	localNameContext = "context"
 	historyComment   = "buildkit.dockerfile.v0"
 
-	DefaultCopyImage = "tonistiigi/copy:v0.1.4@sha256:d9d49bedbbe2b27df88115e6aff7b9cd11ed2fbd8d9013f02d3da735c08c92e5"
+	DefaultCopyImage = "tonistiigi/copy:v0.1.5@sha256:eab89b76ffbb3c807663a67a41e8be31b8a0e362d7fb074a55bddace563a28bb"
 )
 
 type ConvertOpt struct {
@@ -57,6 +58,7 @@ type ConvertOpt struct {
 	ExtraHosts        []llb.HostIP
 	ForceNetMode      pb.NetMode
 	OverrideCopyImage string
+	LLBCaps           *apicaps.CapSet
 }
 
 func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State, *Image, error) {
@@ -368,7 +370,13 @@ func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State,
 	}
 	buildContext.Output = bc.Output()
 
-	st := target.state.SetMarshalDefaults(llb.Platform(platformOpt.targetPlatform))
+	defaults := []llb.ConstraintsOpt{
+		llb.Platform(platformOpt.targetPlatform),
+	}
+	if opt.LLBCaps != nil {
+		defaults = append(defaults, llb.WithCaps(*opt.LLBCaps))
+	}
+	st := target.state.SetMarshalDefaults(defaults...)
 
 	if !platformOpt.implicitTarget {
 		target.image.OS = platformOpt.targetPlatform.OS

+ 3 - 0
vendor/github.com/moby/buildkit/frontend/gateway/client/client.go

@@ -4,6 +4,7 @@ import (
 	"context"
 
 	"github.com/moby/buildkit/solver/pb"
+	"github.com/moby/buildkit/util/apicaps"
 	digest "github.com/opencontainers/go-digest"
 	specs "github.com/opencontainers/image-spec/specs-go/v1"
 )
@@ -49,6 +50,8 @@ type BuildOpts struct {
 	SessionID string
 	Workers   []WorkerInfo
 	Product   string
+	LLBCaps   apicaps.CapSet
+	Caps      apicaps.CapSet
 }
 
 type ResolveImageConfigOpt struct {

+ 4 - 0
vendor/github.com/moby/buildkit/frontend/gateway/forwarder/forward.go

@@ -8,8 +8,10 @@ import (
 	clienttypes "github.com/moby/buildkit/client"
 	"github.com/moby/buildkit/frontend"
 	"github.com/moby/buildkit/frontend/gateway/client"
+	gwpb "github.com/moby/buildkit/frontend/gateway/pb"
 	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/solver"
+	opspb "github.com/moby/buildkit/solver/pb"
 	"github.com/moby/buildkit/util/apicaps"
 	"github.com/moby/buildkit/worker"
 	"github.com/pkg/errors"
@@ -79,6 +81,8 @@ func (c *bridgeClient) BuildOpts() client.BuildOpts {
 		SessionID: c.sid,
 		Workers:   workers,
 		Product:   apicaps.ExportedProduct,
+		Caps:      gwpb.Caps.CapSet(gwpb.Caps.All()),
+		LLBCaps:   opspb.Caps.CapSet(opspb.Caps.All()),
 	}
 }
 

+ 2 - 12
vendor/github.com/moby/buildkit/frontend/gateway/grpcclient/client.go

@@ -329,21 +329,11 @@ func (c *grpcClient) BuildOpts() client.BuildOpts {
 		SessionID: c.sessionID,
 		Workers:   c.workers,
 		Product:   c.product,
+		LLBCaps:   c.llbCaps,
+		Caps:      c.caps,
 	}
 }
 
-func (c *grpcClient) Opts() map[string]string {
-	return c.opts
-}
-
-func (c *grpcClient) SessionID() string {
-	return c.sessionID
-}
-
-func (c *grpcClient) WorkerInfos() []client.WorkerInfo {
-	return c.workers
-}
-
 type reference struct {
 	id string
 	c  *grpcClient

+ 1 - 1
vendor/github.com/moby/buildkit/solver/edge.go

@@ -271,7 +271,7 @@ func (e *edge) currentIndexKey() *CacheKey {
 func (e *edge) skipPhase2SlowCache(dep *dep) bool {
 	isPhase1 := false
 	for _, dep := range e.deps {
-		if !dep.slowCacheComplete && e.slowCacheFunc(dep) != nil && len(dep.keyMap) == 0 {
+		if (!dep.slowCacheComplete && e.slowCacheFunc(dep) != nil || dep.state < edgeStatusCacheSlow) && len(dep.keyMap) == 0 {
 			isPhase1 = true
 			break
 		}

+ 13 - 15
vendor/github.com/moby/buildkit/solver/jobs.go

@@ -3,6 +3,7 @@ package solver
 import (
 	"context"
 	"fmt"
+	"strings"
 	"sync"
 	"time"
 
@@ -560,15 +561,14 @@ func (s *sharedOp) CalcSlowCache(ctx context.Context, index Index, f ResultBased
 		key, err := f(ctx, res)
 		complete := true
 		if err != nil {
-			canceled := false
 			select {
 			case <-ctx.Done():
-				canceled = true
+				if strings.Contains(err.Error(), context.Canceled.Error()) {
+					complete = false
+					err = errors.Wrap(ctx.Err(), err.Error())
+				}
 			default:
 			}
-			if canceled && errors.Cause(err) == context.Canceled {
-				complete = false
-			}
 		}
 		s.slowMu.Lock()
 		defer s.slowMu.Unlock()
@@ -615,15 +615,14 @@ func (s *sharedOp) CacheMap(ctx context.Context, index int) (*cacheMapResp, erro
 		res, done, err := op.CacheMap(ctx, len(s.cacheRes))
 		complete := true
 		if err != nil {
-			canceled := false
 			select {
 			case <-ctx.Done():
-				canceled = true
+				if strings.Contains(err.Error(), context.Canceled.Error()) {
+					complete = false
+					err = errors.Wrap(ctx.Err(), err.Error())
+				}
 			default:
 			}
-			if canceled && errors.Cause(err) == context.Canceled {
-				complete = false
-			}
 		}
 		if complete {
 			if err == nil {
@@ -669,15 +668,14 @@ func (s *sharedOp) Exec(ctx context.Context, inputs []Result) (outputs []Result,
 		res, err := op.Exec(ctx, inputs)
 		complete := true
 		if err != nil {
-			canceled := false
 			select {
 			case <-ctx.Done():
-				canceled = true
+				if strings.Contains(err.Error(), context.Canceled.Error()) {
+					complete = false
+					err = errors.Wrap(ctx.Err(), err.Error())
+				}
 			default:
 			}
-			if canceled && errors.Cause(err) == context.Canceled {
-				complete = false
-			}
 		}
 		if complete {
 			if res != nil {