Bläddra i källkod

Merge pull request #41601 from tiborvass/bk_vendor

vendor buildkit 6861f17f15364de0fe1fd1e6e8da07598a485123
Tibor Vass 4 år sedan
förälder
incheckning
cf0ce96eb1
100 ändrade filer med 2863 tillägg och 1419 borttagningar
  1. 41 157
      builder/builder-next/adapters/containerimage/pull.go
  2. 2 2
      builder/builder-next/controller.go
  3. 3 2
      builder/builder-next/executor_unix.go
  4. 1 2
      builder/builder-next/executor_windows.go
  5. 1 1
      builder/builder-next/exporter/writer.go
  6. 17 10
      builder/builder-next/worker/worker.go
  7. 4 2
      builder/dockerfile/builder.go
  8. 13 6
      builder/dockerfile/dispatchers.go
  9. 3 3
      builder/dockerfile/dispatchers_test.go
  10. 17 0
      builder/dockerignore/deprecated.go
  11. 0 69
      builder/dockerignore/dockerignore_test.go
  12. 1 1
      builder/remotecontext/detect.go
  13. 0 24
      integration-cli/docker_cli_build_test.go
  14. 8 7
      vendor.conf
  15. 15 4
      vendor/github.com/containerd/containerd/README.md
  16. 206 0
      vendor/github.com/containerd/containerd/remotes/docker/auth/fetch.go
  17. 23 18
      vendor/github.com/containerd/containerd/remotes/docker/auth/parse.go
  18. 68 217
      vendor/github.com/containerd/containerd/remotes/docker/authorizer.go
  19. 10 6
      vendor/github.com/containerd/containerd/remotes/docker/pusher.go
  20. 0 4
      vendor/github.com/containerd/containerd/remotes/docker/resolver.go
  21. 46 0
      vendor/github.com/containerd/containerd/remotes/errors/errors.go
  22. 14 17
      vendor/github.com/containerd/containerd/runtime/v1/shim/client/client.go
  23. 2 1
      vendor/github.com/containerd/containerd/vendor.conf
  24. 1 1
      vendor/github.com/containerd/containerd/version/version.go
  25. 5 4
      vendor/github.com/containerd/go-runc/go.mod
  26. 23 3
      vendor/github.com/containerd/go-runc/io_unix.go
  27. 60 34
      vendor/github.com/containerd/go-runc/runc.go
  28. 38 0
      vendor/github.com/containerd/go-runc/runc_unix.go
  29. 31 0
      vendor/github.com/containerd/go-runc/runc_windows.go
  30. 1 0
      vendor/github.com/mitchellh/hashstructure/go.mod
  31. 15 5
      vendor/github.com/moby/buildkit/README.md
  32. 1 1
      vendor/github.com/moby/buildkit/api/services/control/generate.go
  33. 1 1
      vendor/github.com/moby/buildkit/api/types/generate.go
  34. 236 0
      vendor/github.com/moby/buildkit/cache/blobs.go
  35. 39 35
      vendor/github.com/moby/buildkit/cache/contenthash/checksum.go
  36. 5 3
      vendor/github.com/moby/buildkit/cache/contenthash/filehash.go
  37. 0 4
      vendor/github.com/moby/buildkit/cache/contenthash/filehash_unix.go
  38. 0 10
      vendor/github.com/moby/buildkit/cache/contenthash/filehash_windows.go
  39. 18 5
      vendor/github.com/moby/buildkit/cache/contenthash/tarsum.go
  40. 128 30
      vendor/github.com/moby/buildkit/cache/manager.go
  41. 61 0
      vendor/github.com/moby/buildkit/cache/metadata.go
  42. 16 0
      vendor/github.com/moby/buildkit/cache/metadata/metadata.go
  43. 35 0
      vendor/github.com/moby/buildkit/cache/opts.go
  44. 228 103
      vendor/github.com/moby/buildkit/cache/refs.go
  45. 204 0
      vendor/github.com/moby/buildkit/cache/remote.go
  46. 13 10
      vendor/github.com/moby/buildkit/cache/remotecache/export.go
  47. 12 2
      vendor/github.com/moby/buildkit/cache/remotecache/local/local.go
  48. 14 4
      vendor/github.com/moby/buildkit/cache/remotecache/registry/registry.go
  49. 2 1
      vendor/github.com/moby/buildkit/cache/remotecache/v1/cachestorage.go
  50. 39 2
      vendor/github.com/moby/buildkit/cache/remotecache/v1/chains.go
  51. 5 6
      vendor/github.com/moby/buildkit/cache/remotecache/v1/utils.go
  52. 7 13
      vendor/github.com/moby/buildkit/cache/util/fsutil.go
  53. 43 3
      vendor/github.com/moby/buildkit/client/build.go
  54. 21 14
      vendor/github.com/moby/buildkit/client/client.go
  55. 4 3
      vendor/github.com/moby/buildkit/client/client_unix.go
  56. 5 4
      vendor/github.com/moby/buildkit/client/client_windows.go
  57. 44 22
      vendor/github.com/moby/buildkit/client/llb/definition.go
  58. 24 12
      vendor/github.com/moby/buildkit/client/llb/exec.go
  59. 11 11
      vendor/github.com/moby/buildkit/client/llb/fileop.go
  60. 1 1
      vendor/github.com/moby/buildkit/client/llb/imagemetaresolver/resolver.go
  61. 21 1
      vendor/github.com/moby/buildkit/client/llb/meta.go
  62. 10 3
      vendor/github.com/moby/buildkit/client/llb/source.go
  63. 8 0
      vendor/github.com/moby/buildkit/client/llb/state.go
  64. 1 1
      vendor/github.com/moby/buildkit/client/ociindex/ociindex.go
  65. 10 1
      vendor/github.com/moby/buildkit/cmd/buildkitd/config/config.go
  66. 51 35
      vendor/github.com/moby/buildkit/control/control.go
  67. 24 0
      vendor/github.com/moby/buildkit/control/gateway/gateway.go
  68. 14 3
      vendor/github.com/moby/buildkit/executor/executor.go
  69. 27 19
      vendor/github.com/moby/buildkit/executor/oci/hosts.go
  70. 59 74
      vendor/github.com/moby/buildkit/executor/oci/mounts.go
  71. 220 1
      vendor/github.com/moby/buildkit/executor/oci/spec.go
  72. 33 229
      vendor/github.com/moby/buildkit/executor/oci/spec_unix.go
  73. 42 0
      vendor/github.com/moby/buildkit/executor/oci/spec_windows.go
  74. 1 1
      vendor/github.com/moby/buildkit/executor/oci/user.go
  75. 37 26
      vendor/github.com/moby/buildkit/executor/runcexecutor/executor.go
  76. 36 0
      vendor/github.com/moby/buildkit/executor/runcexecutor/executor_common.go
  77. 160 0
      vendor/github.com/moby/buildkit/executor/runcexecutor/executor_linux.go
  78. 49 0
      vendor/github.com/moby/buildkit/executor/stubs.go
  79. 6 1
      vendor/github.com/moby/buildkit/exporter/containerimage/exptypes/types.go
  80. 2 2
      vendor/github.com/moby/buildkit/exporter/local/export.go
  81. 2 2
      vendor/github.com/moby/buildkit/exporter/tar/export.go
  82. 27 5
      vendor/github.com/moby/buildkit/frontend/dockerfile/builder/build.go
  83. 34 0
      vendor/github.com/moby/buildkit/frontend/dockerfile/builder/caps.go
  84. 39 0
      vendor/github.com/moby/buildkit/frontend/dockerfile/builder/subrequests.go
  85. 38 24
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert.go
  86. 0 16
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_norunmount.go
  87. 0 13
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_nosecrets.go
  88. 0 13
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_nossh.go
  89. 0 2
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_runmount.go
  90. 0 2
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_secrets.go
  91. 0 2
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_ssh.go
  92. 8 0
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell.go
  93. 0 7
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell_unix.go
  94. 0 7
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell_windows.go
  95. 1 1
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/image.go
  96. 1 1
      vendor/github.com/moby/buildkit/frontend/dockerfile/dockerignore/dockerignore.go
  97. 16 11
      vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands.go
  98. 0 7
      vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_nosecrets.go
  99. 0 7
      vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_nossh.go
  100. 0 2
      vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_runmount.go

+ 41 - 157
builder/builder-next/adapters/containerimage/pull.go

@@ -9,7 +9,6 @@ import (
 	"path"
 	"runtime"
 	"sync"
-	"sync/atomic"
 	"time"
 
 	"github.com/containerd/containerd/content"
@@ -31,6 +30,7 @@ import (
 	"github.com/moby/buildkit/cache"
 	"github.com/moby/buildkit/client/llb"
 	"github.com/moby/buildkit/session"
+	"github.com/moby/buildkit/solver"
 	"github.com/moby/buildkit/source"
 	"github.com/moby/buildkit/util/flightcontrol"
 	"github.com/moby/buildkit/util/imageutil"
@@ -59,18 +59,12 @@ type SourceOpt struct {
 // Source is the source implementation for accessing container images
 type Source struct {
 	SourceOpt
-	g             flightcontrol.Group
-	resolverCache *resolverCache
+	g flightcontrol.Group
 }
 
 // NewSource creates a new image source
 func NewSource(opt SourceOpt) (*Source, error) {
-	is := &Source{
-		SourceOpt:     opt,
-		resolverCache: newResolverCache(),
-	}
-
-	return is, nil
+	return &Source{SourceOpt: opt}, nil
 }
 
 // ID returns image scheme identifier
@@ -78,16 +72,6 @@ func (is *Source) ID() string {
 	return source.DockerImageScheme
 }
 
-func (is *Source) getResolver(hosts docker.RegistryHosts, ref string, sm *session.Manager, g session.Group) remotes.Resolver {
-	if res := is.resolverCache.Get(ref, g); res != nil {
-		return res
-	}
-	auth := resolver.NewSessionAuthenticator(sm, g)
-	r := resolver.New(hosts, auth)
-	r = is.resolverCache.Add(ref, auth, r, g)
-	return r
-}
-
 func (is *Source) resolveLocal(refStr string) (*image.Image, error) {
 	ref, err := distreference.ParseNormalizedNamed(refStr)
 	if err != nil {
@@ -109,8 +93,15 @@ func (is *Source) resolveRemote(ctx context.Context, ref string, platform *ocisp
 		dgst digest.Digest
 		dt   []byte
 	}
-	res, err := is.g.Do(ctx, ref, func(ctx context.Context) (interface{}, error) {
-		dgst, dt, err := imageutil.Config(ctx, ref, is.getResolver(is.RegistryHosts, ref, sm, g), is.ContentStore, nil, platform)
+	p := platforms.DefaultSpec()
+	if platform != nil {
+		p = *platform
+	}
+	// key is used to synchronize resolutions that can happen in parallel when doing multi-stage.
+	key := "getconfig::" + ref + "::" + platforms.Format(p)
+	res, err := is.g.Do(ctx, key, func(ctx context.Context) (interface{}, error) {
+		res := resolver.DefaultPool.GetResolver(is.RegistryHosts, ref, "pull", sm, g)
+		dgst, dt, err := imageutil.Config(ctx, ref, res, is.ContentStore, nil, platform)
 		if err != nil {
 			return nil, err
 		}
@@ -168,7 +159,7 @@ func (is *Source) ResolveImageConfig(ctx context.Context, ref string, opt llb.Re
 }
 
 // Resolve returns access to pulling for an identifier
-func (is *Source) Resolve(ctx context.Context, id source.Identifier, sm *session.Manager) (source.SourceInstance, error) {
+func (is *Source) Resolve(ctx context.Context, id source.Identifier, sm *session.Manager, vtx solver.Vertex) (source.SourceInstance, error) {
 	imageIdentifier, ok := id.(*source.ImageIdentifier)
 	if !ok {
 		return nil, errors.Errorf("invalid image identifier %v", id)
@@ -191,29 +182,20 @@ func (is *Source) Resolve(ctx context.Context, id source.Identifier, sm *session
 
 type puller struct {
 	is               *Source
-	resolveOnce      sync.Once
 	resolveLocalOnce sync.Once
 	src              *source.ImageIdentifier
 	desc             ocispec.Descriptor
 	ref              string
-	resolveErr       error
-	resolverInstance remotes.Resolver
-	resolverOnce     sync.Once
 	config           []byte
 	platform         ocispec.Platform
 	sm               *session.Manager
 }
 
 func (p *puller) resolver(g session.Group) remotes.Resolver {
-	p.resolverOnce.Do(func() {
-		if p.resolverInstance == nil {
-			p.resolverInstance = p.is.getResolver(p.is.RegistryHosts, p.src.Reference.String(), p.sm, g)
-		}
-	})
-	return p.resolverInstance
+	return resolver.DefaultPool.GetResolver(p.is.RegistryHosts, p.src.Reference.String(), "pull", p.sm, g)
 }
 
-func (p *puller) mainManifestKey(dgst digest.Digest, platform ocispec.Platform) (digest.Digest, error) {
+func (p *puller) mainManifestKey(platform ocispec.Platform) (digest.Digest, error) {
 	dt, err := json.Marshal(struct {
 		Digest  digest.Digest
 		OS      string
@@ -271,22 +253,23 @@ func (p *puller) resolveLocal() {
 }
 
 func (p *puller) resolve(ctx context.Context, g session.Group) error {
-	p.resolveOnce.Do(func() {
+	// key is used to synchronize resolutions that can happen in parallel when doing multi-stage.
+	key := "resolve::" + p.ref + "::" + platforms.Format(p.platform)
+	_, err := p.is.g.Do(ctx, key, func(ctx context.Context) (_ interface{}, err error) {
 		resolveProgressDone := oneOffProgress(ctx, "resolve "+p.src.Reference.String())
+		defer func() {
+			resolveProgressDone(err)
+		}()
 
 		ref, err := distreference.ParseNormalizedNamed(p.src.Reference.String())
 		if err != nil {
-			p.resolveErr = err
-			_ = resolveProgressDone(err)
-			return
+			return nil, err
 		}
 
 		if p.desc.Digest == "" && p.config == nil {
 			origRef, desc, err := p.resolver(g).Resolve(ctx, ref.String())
 			if err != nil {
-				p.resolveErr = err
-				_ = resolveProgressDone(err)
-				return
+				return nil, err
 			}
 
 			p.desc = desc
@@ -301,65 +284,61 @@ func (p *puller) resolve(ctx context.Context, g session.Group) error {
 		if p.config == nil && p.desc.MediaType != images.MediaTypeDockerSchema1Manifest {
 			ref, err := distreference.WithDigest(ref, p.desc.Digest)
 			if err != nil {
-				p.resolveErr = err
-				_ = resolveProgressDone(err)
-				return
+				return nil, err
 			}
 			_, dt, err := p.is.ResolveImageConfig(ctx, ref.String(), llb.ResolveImageConfigOpt{Platform: &p.platform, ResolveMode: resolveModeToString(p.src.ResolveMode)}, p.sm, g)
 			if err != nil {
-				p.resolveErr = err
-				_ = resolveProgressDone(err)
-				return
+				return nil, err
 			}
 
 			p.config = dt
 		}
-		_ = resolveProgressDone(nil)
+		return nil, nil
 	})
-	return p.resolveErr
+	return err
 }
 
-func (p *puller) CacheKey(ctx context.Context, g session.Group, index int) (string, bool, error) {
+func (p *puller) CacheKey(ctx context.Context, g session.Group, index int) (string, solver.CacheOpts, bool, error) {
 	p.resolveLocal()
 
 	if p.desc.Digest != "" && index == 0 {
-		dgst, err := p.mainManifestKey(p.desc.Digest, p.platform)
+		dgst, err := p.mainManifestKey(p.platform)
 		if err != nil {
-			return "", false, err
+			return "", nil, false, err
 		}
-		return dgst.String(), false, nil
+		return dgst.String(), nil, false, nil
 	}
 
 	if p.config != nil {
 		k := cacheKeyFromConfig(p.config).String()
 		if k == "" {
-			return digest.FromBytes(p.config).String(), true, nil
+			return digest.FromBytes(p.config).String(), nil, true, nil
 		}
-		return k, true, nil
+		return k, nil, true, nil
 	}
 
 	if err := p.resolve(ctx, g); err != nil {
-		return "", false, err
+		return "", nil, false, err
 	}
 
 	if p.desc.Digest != "" && index == 0 {
-		dgst, err := p.mainManifestKey(p.desc.Digest, p.platform)
+		dgst, err := p.mainManifestKey(p.platform)
 		if err != nil {
-			return "", false, err
+			return "", nil, false, err
 		}
-		return dgst.String(), false, nil
+		return dgst.String(), nil, false, nil
 	}
 
 	k := cacheKeyFromConfig(p.config).String()
 	if k == "" {
-		dgst, err := p.mainManifestKey(p.desc.Digest, p.platform)
+		dgst, err := p.mainManifestKey(p.platform)
 		if err != nil {
-			return "", false, err
+			return "", nil, false, err
 		}
-		return dgst.String(), true, nil
+		return dgst.String(), nil, true, nil
 	}
 
-	return k, true, nil
+	return k, nil, true, nil
 }
 
 func (p *puller) getRef(ctx context.Context, diffIDs []layer.DiffID, opts ...cache.RefOption) (cache.ImmutableRef, error) {
@@ -426,10 +405,6 @@ func (p *puller) Snapshot(ctx context.Context, g session.Group) (cache.Immutable
 	}
 
 	platform := platforms.Only(p.platform)
-	// workaround for GCR bug that requires a request to manifest endpoint for authentication to work.
-	// if current resolver has not used manifests do a dummy request.
-	// in most cases resolver should be cached and extra request is not needed.
-	ensureManifestRequested(ctx, p.resolver(g), p.ref)
 
 	var (
 		schema1Converter *schema1.Converter
@@ -845,97 +820,6 @@ func resolveModeToString(rm source.ResolveMode) string {
 	return ""
 }
 
-type resolverCache struct {
-	mu sync.Mutex
-	m  map[string]cachedResolver
-}
-
-type cachedResolver struct {
-	counter int64 // needs to be 64bit aligned for 32bit systems
-	timeout time.Time
-	remotes.Resolver
-	auth *resolver.SessionAuthenticator
-}
-
-func (cr *cachedResolver) Resolve(ctx context.Context, ref string) (name string, desc ocispec.Descriptor, err error) {
-	atomic.AddInt64(&cr.counter, 1)
-	return cr.Resolver.Resolve(ctx, ref)
-}
-
-func (r *resolverCache) Add(ref string, auth *resolver.SessionAuthenticator, resolver remotes.Resolver, g session.Group) *cachedResolver {
-	r.mu.Lock()
-	defer r.mu.Unlock()
-
-	ref = r.repo(ref)
-
-	cr, ok := r.m[ref]
-	cr.timeout = time.Now().Add(time.Minute)
-	if ok {
-		cr.auth.AddSession(g)
-		return &cr
-	}
-
-	cr.Resolver = resolver
-	cr.auth = auth
-	r.m[ref] = cr
-	return &cr
-}
-
-func (r *resolverCache) repo(refStr string) string {
-	ref, err := distreference.ParseNormalizedNamed(refStr)
-	if err != nil {
-		return refStr
-	}
-	return ref.Name()
-}
-
-func (r *resolverCache) Get(ref string, g session.Group) *cachedResolver {
-	r.mu.Lock()
-	defer r.mu.Unlock()
-
-	ref = r.repo(ref)
-
-	cr, ok := r.m[ref]
-	if ok {
-		cr.auth.AddSession(g)
-		return &cr
-	}
-	return nil
-}
-
-func (r *resolverCache) clean(now time.Time) {
-	r.mu.Lock()
-	for k, cr := range r.m {
-		if now.After(cr.timeout) {
-			delete(r.m, k)
-		}
-	}
-	r.mu.Unlock()
-}
-
-func newResolverCache() *resolverCache {
-	rc := &resolverCache{
-		m: map[string]cachedResolver{},
-	}
-	t := time.NewTicker(time.Minute)
-	go func() {
-		for {
-			rc.clean(<-t.C)
-		}
-	}()
-	return rc
-}
-
-func ensureManifestRequested(ctx context.Context, res remotes.Resolver, ref string) {
-	cr, ok := res.(*cachedResolver)
-	if !ok {
-		return
-	}
-	if atomic.LoadInt64(&cr.counter) == 0 {
-		res.Resolve(ctx, ref)
-	}
-}
-
 func platformMatches(img *image.Image, p *ocispec.Platform) bool {
 	if img.Architecture != p.Architecture {
 		return false

+ 2 - 2
builder/builder-next/controller.go

@@ -34,7 +34,7 @@ import (
 	"github.com/moby/buildkit/frontend/gateway/forwarder"
 	containerdsnapshot "github.com/moby/buildkit/snapshot/containerd"
 	"github.com/moby/buildkit/solver/bboltcachestorage"
-	"github.com/moby/buildkit/util/binfmt_misc"
+	"github.com/moby/buildkit/util/archutil"
 	"github.com/moby/buildkit/util/entitlements"
 	"github.com/moby/buildkit/util/leaseutil"
 	"github.com/moby/buildkit/worker"
@@ -166,7 +166,7 @@ func newController(rt http.RoundTripper, opt Opt) (*control.Controller, error) {
 		return nil, errors.Errorf("snapshotter doesn't support differ")
 	}
 
-	p, err := parsePlatforms(binfmt_misc.SupportedPlatforms(true))
+	p, err := parsePlatforms(archutil.SupportedPlatforms(true))
 	if err != nil {
 		return nil, err
 	}

+ 3 - 2
builder/builder-next/executor_unix.go

@@ -95,11 +95,11 @@ func (iface *lnInterface) init(c libnetwork.NetworkController, n libnetwork.Netw
 	iface.ep = ep
 }
 
-func (iface *lnInterface) Set(s *specs.Spec) {
+func (iface *lnInterface) Set(s *specs.Spec) error {
 	<-iface.ready
 	if iface.err != nil {
 		logrus.WithError(iface.err).Error("failed to set networking spec")
-		return
+		return iface.err
 	}
 	shortNetCtlrID := stringid.TruncateID(iface.provider.NetworkController.ID())
 	// attach netns to bridge within the container namespace, using reexec in a prestart hook
@@ -109,6 +109,7 @@ func (iface *lnInterface) Set(s *specs.Spec) {
 			Args: []string{"libnetwork-setkey", "-exec-root=" + iface.provider.Config().Daemon.ExecRoot, iface.sbx.ContainerID(), shortNetCtlrID},
 		}},
 	}
+	return nil
 }
 
 func (iface *lnInterface) Close() error {

+ 1 - 2
builder/builder-next/executor_windows.go

@@ -7,7 +7,6 @@ import (
 	"github.com/docker/docker/daemon/config"
 	"github.com/docker/docker/pkg/idtools"
 	"github.com/docker/libnetwork"
-	"github.com/moby/buildkit/cache"
 	"github.com/moby/buildkit/executor"
 	"github.com/moby/buildkit/executor/oci"
 )
@@ -19,7 +18,7 @@ func newExecutor(_, _ string, _ libnetwork.NetworkController, _ *oci.DNSConfig,
 type winExecutor struct {
 }
 
-func (w *winExecutor) Run(ctx context.Context, id string, root cache.Mountable, mounts []executor.Mount, process executor.ProcessInfo, started chan<- struct{}) (err error) {
+func (w *winExecutor) Run(ctx context.Context, id string, root executor.Mount, mounts []executor.Mount, process executor.ProcessInfo, started chan<- struct{}) (err error) {
 	return errors.New("buildkit executor not implemented for windows")
 }
 

+ 1 - 1
builder/builder-next/exporter/writer.go

@@ -26,7 +26,7 @@ func emptyImageConfig() ([]byte, error) {
 	}
 	img.RootFS.Type = "layers"
 	img.Config.WorkingDir = "/"
-	img.Config.Env = []string{"PATH=" + system.DefaultPathEnv}
+	img.Config.Env = []string{"PATH=" + system.DefaultPathEnvUnix}
 	dt, err := json.Marshal(img)
 	return dt, errors.Wrap(err, "failed to create empty image config")
 }

+ 17 - 10
builder/builder-next/worker/worker.go

@@ -33,13 +33,15 @@ import (
 	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/snapshot"
 	"github.com/moby/buildkit/solver"
+	"github.com/moby/buildkit/solver/llbsolver/mounts"
 	"github.com/moby/buildkit/solver/llbsolver/ops"
 	"github.com/moby/buildkit/solver/pb"
 	"github.com/moby/buildkit/source"
 	"github.com/moby/buildkit/source/git"
 	"github.com/moby/buildkit/source/http"
 	"github.com/moby/buildkit/source/local"
-	"github.com/moby/buildkit/util/binfmt_misc"
+	"github.com/moby/buildkit/util/archutil"
+	"github.com/moby/buildkit/util/compression"
 	"github.com/moby/buildkit/util/contentutil"
 	"github.com/moby/buildkit/util/progress"
 	digest "github.com/opencontainers/go-digest"
@@ -147,7 +149,7 @@ func (w *Worker) Platforms(noCache bool) []ocispec.Platform {
 		for _, p := range w.Opt.Platforms {
 			pm[platforms.Format(p)] = struct{}{}
 		}
-		for _, p := range binfmt_misc.SupportedPlatforms(noCache) {
+		for _, p := range archutil.SupportedPlatforms(noCache) {
 			if _, ok := pm[p]; !ok {
 				pp, _ := platforms.Parse(p)
 				w.Opt.Platforms = append(w.Opt.Platforms, pp)
@@ -170,13 +172,18 @@ func (w *Worker) ContentStore() content.Store {
 	return w.Opt.ContentStore
 }
 
+// MetadataStore returns the metadata store
+func (w *Worker) MetadataStore() *metadata.Store {
+	return w.Opt.MetadataStore
+}
+
 // LoadRef loads a reference by ID
-func (w *Worker) LoadRef(id string, hidden bool) (cache.ImmutableRef, error) {
+func (w *Worker) LoadRef(ctx context.Context, id string, hidden bool) (cache.ImmutableRef, error) {
 	var opts []cache.RefOption
 	if hidden {
 		opts = append(opts, cache.NoUpdateLastUsed)
 	}
-	return w.CacheManager().Get(context.TODO(), id, opts...)
+	return w.CacheManager().Get(ctx, id, opts...)
 }
 
 // ResolveOp converts a LLB vertex into a LLB operation
@@ -186,9 +193,9 @@ func (w *Worker) ResolveOp(v solver.Vertex, s frontend.FrontendLLBBridge, sm *se
 		case *pb.Op_Source:
 			return ops.NewSourceOp(v, op, baseOp.Platform, w.SourceManager, sm, w)
 		case *pb.Op_Exec:
-			return ops.NewExecOp(v, op, baseOp.Platform, w.CacheManager(), sm, w.MetadataStore, w.Executor(), w)
+			return ops.NewExecOp(v, op, baseOp.Platform, w.CacheManager(), sm, w.Opt.MetadataStore, w.Executor(), w)
 		case *pb.Op_File:
-			return ops.NewFileOp(v, op, w.CacheManager(), w.MetadataStore, w)
+			return ops.NewFileOp(v, op, w.CacheManager(), w.Opt.MetadataStore, w)
 		case *pb.Op_Build:
 			return ops.NewBuildOp(v, op, s, w)
 		}
@@ -230,7 +237,7 @@ func (w *Worker) Exporter(name string, sm *session.Manager) (exporter.Exporter,
 }
 
 // GetRemote returns a remote snapshot reference for a local one
-func (w *Worker) GetRemote(ctx context.Context, ref cache.ImmutableRef, createIfNeeded bool) (*solver.Remote, error) {
+func (w *Worker) GetRemote(ctx context.Context, ref cache.ImmutableRef, createIfNeeded bool, _ compression.Type, _ session.Group) (*solver.Remote, error) {
 	var diffIDs []layer.DiffID
 	var err error
 	if !createIfNeeded {
@@ -265,13 +272,13 @@ func (w *Worker) GetRemote(ctx context.Context, ref cache.ImmutableRef, createIf
 
 // PruneCacheMounts removes the current cache snapshots for specified IDs
 func (w *Worker) PruneCacheMounts(ctx context.Context, ids []string) error {
-	mu := ops.CacheMountsLocker()
+	mu := mounts.CacheMountsLocker()
 	mu.Lock()
 	defer mu.Unlock()
 
 	for _, id := range ids {
 		id = "cache-dir:" + id
-		sis, err := w.MetadataStore.Search(id)
+		sis, err := w.Opt.MetadataStore.Search(id)
 		if err != nil {
 			return err
 		}
@@ -300,7 +307,7 @@ func (w *Worker) PruneCacheMounts(ctx context.Context, ids []string) error {
 		}
 	}
 
-	ops.ClearActiveCacheMounts()
+	mounts.ClearActiveCacheMounts()
 	return nil
 }
 

+ 4 - 2
builder/dockerfile/builder.go

@@ -235,8 +235,10 @@ func processMetaArg(meta instructions.ArgCommand, shlex *shell.Lex, args *BuildA
 	}); err != nil {
 		return err
 	}
-	args.AddArg(meta.Key, meta.Value)
-	args.AddMetaArg(meta.Key, meta.Value)
+	for _, arg := range meta.Args {
+		args.AddArg(arg.Key, arg.Value)
+		args.AddMetaArg(arg.Key, arg.Value)
+	}
 	return nil
 }
 

+ 13 - 6
builder/dockerfile/dispatchers.go

@@ -587,14 +587,21 @@ func dispatchStopSignal(d dispatchRequest, c *instructions.StopSignalCommand) er
 // to builder using the --build-arg flag for expansion/substitution or passing to 'run'.
 // Dockerfile author may optionally set a default value of this variable.
 func dispatchArg(d dispatchRequest, c *instructions.ArgCommand) error {
-
-	commitStr := "ARG " + c.Key
-	if c.Value != nil {
-		commitStr += "=" + *c.Value
+	var commitStr strings.Builder
+	commitStr.WriteString("ARG ")
+	for i, arg := range c.Args {
+		if i > 0 {
+			commitStr.WriteString(" ")
+		}
+		commitStr.WriteString(arg.Key)
+		if arg.Value != nil {
+			commitStr.WriteString("=")
+			commitStr.WriteString(*arg.Value)
+		}
+		d.state.buildArgs.AddArg(arg.Key, arg.Value)
 	}
 
-	d.state.buildArgs.AddArg(c.Key, c.Value)
-	return d.builder.commit(d.state, commitStr)
+	return d.builder.commit(d.state, commitStr.String())
 }
 
 // SHELL powershell -command

+ 3 - 3
builder/dockerfile/dispatchers_test.go

@@ -139,10 +139,10 @@ func TestFromWithArg(t *testing.T) {
 	args := NewBuildArgs(make(map[string]*string))
 
 	val := "sometag"
-	metaArg := instructions.ArgCommand{KeyValuePairOptional: instructions.KeyValuePairOptional{
+	metaArg := instructions.ArgCommand{Args: []instructions.KeyValuePairOptional{{
 		Key:   "THETAG",
 		Value: &val,
-	}}
+	}}}
 	cmd := &instructions.Stage{
 		BaseName: "alpine:${THETAG}",
 	}
@@ -395,7 +395,7 @@ func TestArg(t *testing.T) {
 
 	argName := "foo"
 	argVal := "bar"
-	cmd := &instructions.ArgCommand{KeyValuePairOptional: instructions.KeyValuePairOptional{Key: argName, Value: &argVal}}
+	cmd := &instructions.ArgCommand{Args: []instructions.KeyValuePairOptional{{Key: argName, Value: &argVal}}}
 	err := dispatch(sb, cmd)
 	assert.NilError(t, err)
 

+ 17 - 0
builder/dockerignore/deprecated.go

@@ -0,0 +1,17 @@
+// Package dockerignore is deprecated. Use github.com/moby/buildkit/frontend/dockerfile/dockerignore instead.
+package dockerignore
+
+import (
+	"io"
+
+	"github.com/moby/buildkit/frontend/dockerfile/dockerignore"
+)
+
+// ReadAll reads a .dockerignore file and returns the list of file patterns
+// to ignore. Note this will trim whitespace from each line as well
+// as use GO's "clean" func to get the shortest/cleanest path for each.
+//
+// Deprecated: use github.com/moby/buildkit/frontend/dockerfile/dockerignore.ReadAll instead.
+func ReadAll(reader io.Reader) ([]string, error) {
+	return dockerignore.ReadAll(reader)
+}

+ 0 - 69
builder/dockerignore/dockerignore_test.go

@@ -1,69 +0,0 @@
-package dockerignore // import "github.com/docker/docker/builder/dockerignore"
-
-import (
-	"fmt"
-	"io/ioutil"
-	"os"
-	"path/filepath"
-	"testing"
-)
-
-func TestReadAll(t *testing.T) {
-	tmpDir, err := ioutil.TempDir("", "dockerignore-test")
-	if err != nil {
-		t.Fatal(err)
-	}
-	defer os.RemoveAll(tmpDir)
-
-	di, err := ReadAll(nil)
-	if err != nil {
-		t.Fatalf("Expected not to have error, got %v", err)
-	}
-
-	if diLen := len(di); diLen != 0 {
-		t.Fatalf("Expected to have zero dockerignore entry, got %d", diLen)
-	}
-
-	diName := filepath.Join(tmpDir, ".dockerignore")
-	content := fmt.Sprintf("test1\n/test2\n/a/file/here\n\nlastfile\n# this is a comment\n! /inverted/abs/path\n!\n! \n")
-	err = ioutil.WriteFile(diName, []byte(content), 0777)
-	if err != nil {
-		t.Fatal(err)
-	}
-
-	diFd, err := os.Open(diName)
-	if err != nil {
-		t.Fatal(err)
-	}
-	defer diFd.Close()
-
-	di, err = ReadAll(diFd)
-	if err != nil {
-		t.Fatal(err)
-	}
-
-	if len(di) != 7 {
-		t.Fatalf("Expected 7 entries, got %v", len(di))
-	}
-	if di[0] != "test1" {
-		t.Fatal("First element is not test1")
-	}
-	if di[1] != "test2" { // according to https://docs.docker.com/engine/reference/builder/#dockerignore-file, /foo/bar should be treated as foo/bar
-		t.Fatal("Second element is not test2")
-	}
-	if di[2] != "a/file/here" { // according to https://docs.docker.com/engine/reference/builder/#dockerignore-file, /foo/bar should be treated as foo/bar
-		t.Fatal("Third element is not a/file/here")
-	}
-	if di[3] != "lastfile" {
-		t.Fatal("Fourth element is not lastfile")
-	}
-	if di[4] != "!inverted/abs/path" {
-		t.Fatal("Fifth element is not !inverted/abs/path")
-	}
-	if di[5] != "!" {
-		t.Fatalf("Sixth element is not !, but %s", di[5])
-	}
-	if di[6] != "!" {
-		t.Fatalf("Seventh element is not !, but %s", di[6])
-	}
-}

+ 1 - 1
builder/remotecontext/detect.go

@@ -11,10 +11,10 @@ import (
 	"github.com/containerd/continuity/driver"
 	"github.com/docker/docker/api/types/backend"
 	"github.com/docker/docker/builder"
-	"github.com/docker/docker/builder/dockerignore"
 	"github.com/docker/docker/errdefs"
 	"github.com/docker/docker/pkg/fileutils"
 	"github.com/docker/docker/pkg/urlutil"
+	"github.com/moby/buildkit/frontend/dockerfile/dockerignore"
 	"github.com/moby/buildkit/frontend/dockerfile/parser"
 	"github.com/pkg/errors"
 	"github.com/sirupsen/logrus"

+ 0 - 24
integration-cli/docker_cli_build_test.go

@@ -5608,30 +5608,6 @@ func (s *DockerSuite) TestBuildWithExtraHostInvalidFormat(c *testing.T) {
 
 }
 
-func (s *DockerSuite) TestBuildContChar(c *testing.T) {
-	name := "testbuildcontchar"
-
-	buildImage(name, build.WithDockerfile(`FROM busybox\`)).Assert(c, icmd.Expected{
-		Out: "Step 1/1 : FROM busybox",
-	})
-
-	result := buildImage(name, build.WithDockerfile(`FROM busybox
-		 RUN echo hi \`))
-	result.Assert(c, icmd.Success)
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 1/2 : FROM busybox"))
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 2/2 : RUN echo hi\n"))
-	result = buildImage(name, build.WithDockerfile(`FROM busybox
-		 RUN echo hi \\`))
-	result.Assert(c, icmd.Success)
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 1/2 : FROM busybox"))
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 2/2 : RUN echo hi \\\n"))
-	result = buildImage(name, build.WithDockerfile(`FROM busybox
-		 RUN echo hi \\\`))
-	result.Assert(c, icmd.Success)
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 1/2 : FROM busybox"))
-	assert.Assert(c, strings.Contains(result.Combined(), "Step 2/2 : RUN echo hi \\\\\n"))
-}
-
 func (s *DockerSuite) TestBuildMultiStageCopyFromSyntax(c *testing.T) {
 	dockerfile := `
 		FROM busybox AS first

+ 8 - 7
vendor.conf

@@ -33,13 +33,14 @@ github.com/imdario/mergo                            1afb36080aec31e0d1528973ebe6
 golang.org/x/sync                                   cd5d95a43a6e21273425c7ae415d3df9ea832eeb
 
 # buildkit
-github.com/moby/buildkit                            4d1f260e8490ec438ab66e08bb105577aca0ce06
-github.com/tonistiigi/fsutil                        ae3a8d753069d0f76fbee396457e8b6cfd7cb8c3
+github.com/moby/buildkit                            6861f17f15364de0fe1fd1e6e8da07598a485123
+github.com/tonistiigi/fsutil                        c3ed55f3b48161fd3dc42c17ba09e12ac52d57dc
+github.com/tonistiigi/units                         6950e57a87eaf136bbe44ef2ec8e75b9e3569de2
 github.com/grpc-ecosystem/grpc-opentracing          8e809c8a86450a29b90dcc9efbf062d0fe6d9746
-github.com/opentracing/opentracing-go               1361b9cd60be79c4c3a7fa9841b3c132e40066a7
+github.com/opentracing/opentracing-go               d34af3eaa63c4d08ab54863a4bdd0daa45212e12 # v1.2.0
 github.com/google/shlex                             e7afc7fbc51079733e9468cdfd1efcd7d196cd1d
-github.com/opentracing-contrib/go-stdlib            b1a47cfbdd7543e70e9ef3e73d0802ad306cc1cc
-github.com/mitchellh/hashstructure                  2bca23e0e452137f789efbc8610126fd8b94f73b
+github.com/opentracing-contrib/go-stdlib            8a6ff1ad1691a29e4f7b5d46604f97634997c8c4 # v1.0.0
+github.com/mitchellh/hashstructure                  a38c50148365edc8df43c1580c48fb2b3a1e9cd7 # v1.0.0
 github.com/gofrs/flock                              6caa7350c26b838538005fae7dbee4e69d9398db # v0.7.3
 github.com/grpc-ecosystem/go-grpc-middleware        3c51f7f332123e8be5a157c0802a228ac85bf9db # v1.2.0
 
@@ -129,12 +130,12 @@ github.com/googleapis/gax-go                        bd5b16380fd03dc758d11cef74ba
 google.golang.org/genproto                          3f1135a288c9a07e340ae8ba4cc6c7065a3160e8
 
 # containerd
-github.com/containerd/containerd                    c623d1b36f09f8ef6536a057bd658b3aa8632828 # v1.4.1
+github.com/containerd/containerd                    d4e78200d6da62480c85bf6f26b7221ea938f396
 github.com/containerd/fifo                          f15a3290365b9d2627d189e619ab4008e0069caf
 github.com/containerd/continuity                    efbc4488d8fe1bdc16bde3b2d2990d9b3a899165
 github.com/containerd/cgroups                       318312a373405e5e91134d8063d04d59768a1bff
 github.com/containerd/console                       5d7e1412f07b502a01029ea20e20e0d2be31fa7c # v1.0.1
-github.com/containerd/go-runc                       7016d3ce2328dd2cb1192b2076ebd565c4e8df0c
+github.com/containerd/go-runc                       16b287bc67d069a60fa48db15f330b790b74365b
 github.com/containerd/typeurl                       cd3ce7159eae562a4f60ceff37dada11a939d247 # v1.0.1
 github.com/containerd/ttrpc                         72bb1b21c5b0a4a107f59dd85f6ab58e564b68d6 # v1.0.1
 github.com/gogo/googleapis                          01e0f9cca9b92166042241267ee2a5cdf5cff46c # v1.3.2

+ 15 - 4
vendor/github.com/containerd/containerd/README.md

@@ -10,10 +10,24 @@
 
 containerd is an industry-standard container runtime with an emphasis on simplicity, robustness and portability. It is available as a daemon for Linux and Windows, which can manage the complete container lifecycle of its host system: image transfer and storage, container execution and supervision, low-level storage and network attachments, etc.
 
+containerd is a member of CNCF with ['graduated'](https://landscape.cncf.io/selected=containerd) status.
+
 containerd is designed to be embedded into a larger system, rather than being used directly by developers or end-users.
 
 ![architecture](design/architecture.png)
 
+## Now Recruiting
+
+We are a large inclusive OSS project that is welcoming help of any kind shape or form:
+* Documentation help is needed to make the product easier to consume and extend.
+* We need OSS community outreach / organizing help to get the word out; manage
+and create messaging and educational content; and to help with social media, community forums/groups, and google groups.
+* We are actively inviting new [security advisors](https://github.com/containerd/project/blob/master/GOVERNANCE.md#security-advisors) to join the team.
+* New sub-projects are being created, core and non-core that could use additional development help.
+* Each of the [containerd projects](https://github.com/containerd) has a list of issues currently being worked on or that need help resolving.
+  - If the issue has not already been assigned to someone, or has not made recent progress and you are interested, please inquire.
+  - If you are interested in starting with a smaller / beginner level issue, look for issues with an `exp/beginner` tag, for example [containerd/containerd beginner issues.](https://github.com/containerd/containerd/issues?q=is%3Aissue+is%3Aopen+label%3Aexp%2Fbeginner)
+
 ## Getting Started
 
 See our documentation on [containerd.io](https://containerd.io):
@@ -250,10 +264,7 @@ loaded for the user's shell environment.
 For async communication and long running discussions please use issues and pull requests on the github repo.
 This will be the best place to discuss design and implementation.
 
-For sync communication we have a community slack with a #containerd channel that everyone is welcome to join and chat about development.
-
-**Slack:** Catch us in the #containerd and #containerd-dev channels on dockercommunity.slack.com.
-[Click here for an invite to docker community slack.](https://dockr.ly/slack)
+For sync communication catch us in the `#containerd` and `#containerd-dev` slack channels on Cloud Native Computing Foundation's (CNCF) slack - `cloud-native.slack.com`. Everyone is welcome to join and chat. [Get Invite to CNCF slack.](https://slack.cncf.io)
 
 ### Security audit
 

+ 206 - 0
vendor/github.com/containerd/containerd/remotes/docker/auth/fetch.go

@@ -0,0 +1,206 @@
+/*
+   Copyright The containerd Authors.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package auth
+
+import (
+	"context"
+	"encoding/json"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+
+	"github.com/containerd/containerd/log"
+	remoteserrors "github.com/containerd/containerd/remotes/errors"
+	"github.com/pkg/errors"
+	"golang.org/x/net/context/ctxhttp"
+)
+
+var (
+	// ErrNoToken is returned if a request is successful but the body does not
+	// contain an authorization token.
+	ErrNoToken = errors.New("authorization server did not include a token in the response")
+)
+
+// GenerateTokenOptions generates options for fetching a token based on a challenge
+func GenerateTokenOptions(ctx context.Context, host, username, secret string, c Challenge) (TokenOptions, error) {
+	realm, ok := c.Parameters["realm"]
+	if !ok {
+		return TokenOptions{}, errors.New("no realm specified for token auth challenge")
+	}
+
+	realmURL, err := url.Parse(realm)
+	if err != nil {
+		return TokenOptions{}, errors.Wrap(err, "invalid token auth challenge realm")
+	}
+
+	to := TokenOptions{
+		Realm:    realmURL.String(),
+		Service:  c.Parameters["service"],
+		Username: username,
+		Secret:   secret,
+	}
+
+	scope, ok := c.Parameters["scope"]
+	if ok {
+		to.Scopes = append(to.Scopes, scope)
+	} else {
+		log.G(ctx).WithField("host", host).Debug("no scope specified for token auth challenge")
+	}
+
+	return to, nil
+}
+
+// TokenOptions are optios for requesting a token
+type TokenOptions struct {
+	Realm    string
+	Service  string
+	Scopes   []string
+	Username string
+	Secret   string
+}
+
+// OAuthTokenResponse is response from fetching token with a OAuth POST request
+type OAuthTokenResponse struct {
+	AccessToken  string    `json:"access_token"`
+	RefreshToken string    `json:"refresh_token"`
+	ExpiresIn    int       `json:"expires_in"`
+	IssuedAt     time.Time `json:"issued_at"`
+	Scope        string    `json:"scope"`
+}
+
+// FetchTokenWithOAuth fetches a token using a POST request
+func FetchTokenWithOAuth(ctx context.Context, client *http.Client, headers http.Header, clientID string, to TokenOptions) (*OAuthTokenResponse, error) {
+	form := url.Values{}
+	if len(to.Scopes) > 0 {
+		form.Set("scope", strings.Join(to.Scopes, " "))
+	}
+	form.Set("service", to.Service)
+	form.Set("client_id", clientID)
+
+	if to.Username == "" {
+		form.Set("grant_type", "refresh_token")
+		form.Set("refresh_token", to.Secret)
+	} else {
+		form.Set("grant_type", "password")
+		form.Set("username", to.Username)
+		form.Set("password", to.Secret)
+	}
+
+	req, err := http.NewRequest("POST", to.Realm, strings.NewReader(form.Encode()))
+	if err != nil {
+		return nil, err
+	}
+	req.Header.Set("Content-Type", "application/x-www-form-urlencoded; charset=utf-8")
+	if headers != nil {
+		for k, v := range headers {
+			req.Header[k] = append(req.Header[k], v...)
+		}
+	}
+
+	resp, err := ctxhttp.Do(ctx, client, req)
+	if err != nil {
+		return nil, err
+	}
+	defer resp.Body.Close()
+
+	if resp.StatusCode < 200 || resp.StatusCode >= 400 {
+		return nil, errors.WithStack(remoteserrors.NewUnexpectedStatusErr(resp))
+	}
+
+	decoder := json.NewDecoder(resp.Body)
+
+	var tr OAuthTokenResponse
+	if err = decoder.Decode(&tr); err != nil {
+		return nil, errors.Wrap(err, "unable to decode token response")
+	}
+
+	if tr.AccessToken == "" {
+		return nil, errors.WithStack(ErrNoToken)
+	}
+
+	return &tr, nil
+}
+
+// FetchTokenResponse is response from fetching token with GET request
+type FetchTokenResponse struct {
+	Token        string    `json:"token"`
+	AccessToken  string    `json:"access_token"`
+	ExpiresIn    int       `json:"expires_in"`
+	IssuedAt     time.Time `json:"issued_at"`
+	RefreshToken string    `json:"refresh_token"`
+}
+
+// FetchToken fetches a token using a GET request
+func FetchToken(ctx context.Context, client *http.Client, headers http.Header, to TokenOptions) (*FetchTokenResponse, error) {
+	req, err := http.NewRequest("GET", to.Realm, nil)
+	if err != nil {
+		return nil, err
+	}
+
+	if headers != nil {
+		for k, v := range headers {
+			req.Header[k] = append(req.Header[k], v...)
+		}
+	}
+
+	reqParams := req.URL.Query()
+
+	if to.Service != "" {
+		reqParams.Add("service", to.Service)
+	}
+
+	for _, scope := range to.Scopes {
+		reqParams.Add("scope", scope)
+	}
+
+	if to.Secret != "" {
+		req.SetBasicAuth(to.Username, to.Secret)
+	}
+
+	req.URL.RawQuery = reqParams.Encode()
+
+	resp, err := ctxhttp.Do(ctx, client, req)
+	if err != nil {
+		return nil, err
+	}
+	defer resp.Body.Close()
+
+	if resp.StatusCode < 200 || resp.StatusCode >= 400 {
+		return nil, errors.WithStack(remoteserrors.NewUnexpectedStatusErr(resp))
+	}
+
+	decoder := json.NewDecoder(resp.Body)
+
+	var tr FetchTokenResponse
+	if err = decoder.Decode(&tr); err != nil {
+		return nil, errors.Wrap(err, "unable to decode token response")
+	}
+
+	// `access_token` is equivalent to `token` and if both are specified
+	// the choice is undefined.  Canonicalize `access_token` by sticking
+	// things in `token`.
+	if tr.AccessToken != "" {
+		tr.Token = tr.AccessToken
+	}
+
+	if tr.Token == "" {
+		return nil, errors.WithStack(ErrNoToken)
+	}
+
+	return &tr, nil
+}

+ 23 - 18
vendor/github.com/containerd/containerd/remotes/docker/auth.go → vendor/github.com/containerd/containerd/remotes/docker/auth/parse.go

@@ -14,7 +14,7 @@
    limitations under the License.
 */
 
-package docker
+package auth
 
 import (
 	"net/http"
@@ -22,31 +22,35 @@ import (
 	"strings"
 )
 
-type authenticationScheme byte
+// AuthenticationScheme defines scheme of the authentication method
+type AuthenticationScheme byte
 
 const (
-	basicAuth  authenticationScheme = 1 << iota // Defined in RFC 7617
-	digestAuth                                  // Defined in RFC 7616
-	bearerAuth                                  // Defined in RFC 6750
+	// BasicAuth is scheme for Basic HTTP Authentication RFC 7617
+	BasicAuth AuthenticationScheme = 1 << iota
+	// DigestAuth is scheme for HTTP Digest Access Authentication RFC 7616
+	DigestAuth
+	// BearerAuth is scheme for OAuth 2.0 Bearer Tokens RFC 6750
+	BearerAuth
 )
 
-// challenge carries information from a WWW-Authenticate response header.
+// Challenge carries information from a WWW-Authenticate response header.
 // See RFC 2617.
-type challenge struct {
+type Challenge struct {
 	// scheme is the auth-scheme according to RFC 2617
-	scheme authenticationScheme
+	Scheme AuthenticationScheme
 
 	// parameters are the auth-params according to RFC 2617
-	parameters map[string]string
+	Parameters map[string]string
 }
 
-type byScheme []challenge
+type byScheme []Challenge
 
 func (bs byScheme) Len() int      { return len(bs) }
 func (bs byScheme) Swap(i, j int) { bs[i], bs[j] = bs[j], bs[i] }
 
 // Sort in priority order: token > digest > basic
-func (bs byScheme) Less(i, j int) bool { return bs[i].scheme > bs[j].scheme }
+func (bs byScheme) Less(i, j int) bool { return bs[i].Scheme > bs[j].Scheme }
 
 // Octet types from RFC 2616.
 type octetType byte
@@ -90,22 +94,23 @@ func init() {
 	}
 }
 
-func parseAuthHeader(header http.Header) []challenge {
-	challenges := []challenge{}
+// ParseAuthHeader parses challenges from WWW-Authenticate header
+func ParseAuthHeader(header http.Header) []Challenge {
+	challenges := []Challenge{}
 	for _, h := range header[http.CanonicalHeaderKey("WWW-Authenticate")] {
 		v, p := parseValueAndParams(h)
-		var s authenticationScheme
+		var s AuthenticationScheme
 		switch v {
 		case "basic":
-			s = basicAuth
+			s = BasicAuth
 		case "digest":
-			s = digestAuth
+			s = DigestAuth
 		case "bearer":
-			s = bearerAuth
+			s = BearerAuth
 		default:
 			continue
 		}
-		challenges = append(challenges, challenge{scheme: s, parameters: p})
+		challenges = append(challenges, Challenge{Scheme: s, Parameters: p})
 	}
 	sort.Stable(byScheme(challenges))
 	return challenges

+ 68 - 217
vendor/github.com/containerd/containerd/remotes/docker/authorizer.go

@@ -19,21 +19,17 @@ package docker
 import (
 	"context"
 	"encoding/base64"
-	"encoding/json"
 	"fmt"
-	"io"
-	"io/ioutil"
 	"net/http"
-	"net/url"
 	"strings"
 	"sync"
-	"time"
 
 	"github.com/containerd/containerd/errdefs"
 	"github.com/containerd/containerd/log"
+	"github.com/containerd/containerd/remotes/docker/auth"
+	remoteerrors "github.com/containerd/containerd/remotes/errors"
 	"github.com/pkg/errors"
 	"github.com/sirupsen/logrus"
-	"golang.org/x/net/context/ctxhttp"
 )
 
 type dockerAuthorizer struct {
@@ -135,8 +131,8 @@ func (a *dockerAuthorizer) AddResponses(ctx context.Context, responses []*http.R
 
 	a.mu.Lock()
 	defer a.mu.Unlock()
-	for _, c := range parseAuthHeader(last.Header) {
-		if c.scheme == bearerAuth {
+	for _, c := range auth.ParseAuthHeader(last.Header) {
+		if c.Scheme == auth.BearerAuth {
 			if err := invalidAuthorization(c, responses); err != nil {
 				delete(a.handlers, host)
 				return err
@@ -152,26 +148,35 @@ func (a *dockerAuthorizer) AddResponses(ctx context.Context, responses []*http.R
 				return nil
 			}
 
-			common, err := a.generateTokenOptions(ctx, host, c)
+			var username, secret string
+			if a.credentials != nil {
+				var err error
+				username, secret, err = a.credentials(host)
+				if err != nil {
+					return err
+				}
+			}
+
+			common, err := auth.GenerateTokenOptions(ctx, host, username, secret, c)
 			if err != nil {
 				return err
 			}
 
-			a.handlers[host] = newAuthHandler(a.client, a.header, c.scheme, common)
+			a.handlers[host] = newAuthHandler(a.client, a.header, c.Scheme, common)
 			return nil
-		} else if c.scheme == basicAuth && a.credentials != nil {
+		} else if c.Scheme == auth.BasicAuth && a.credentials != nil {
 			username, secret, err := a.credentials(host)
 			if err != nil {
 				return err
 			}
 
 			if username != "" && secret != "" {
-				common := tokenOptions{
-					username: username,
-					secret:   secret,
+				common := auth.TokenOptions{
+					Username: username,
+					Secret:   secret,
 				}
 
-				a.handlers[host] = newAuthHandler(a.client, a.header, c.scheme, common)
+				a.handlers[host] = newAuthHandler(a.client, a.header, c.Scheme, common)
 				return nil
 			}
 		}
@@ -179,38 +184,6 @@ func (a *dockerAuthorizer) AddResponses(ctx context.Context, responses []*http.R
 	return errors.Wrap(errdefs.ErrNotImplemented, "failed to find supported auth scheme")
 }
 
-func (a *dockerAuthorizer) generateTokenOptions(ctx context.Context, host string, c challenge) (tokenOptions, error) {
-	realm, ok := c.parameters["realm"]
-	if !ok {
-		return tokenOptions{}, errors.New("no realm specified for token auth challenge")
-	}
-
-	realmURL, err := url.Parse(realm)
-	if err != nil {
-		return tokenOptions{}, errors.Wrap(err, "invalid token auth challenge realm")
-	}
-
-	to := tokenOptions{
-		realm:   realmURL.String(),
-		service: c.parameters["service"],
-	}
-
-	scope, ok := c.parameters["scope"]
-	if ok {
-		to.scopes = append(to.scopes, scope)
-	} else {
-		log.G(ctx).WithField("host", host).Debug("no scope specified for token auth challenge")
-	}
-
-	if a.credentials != nil {
-		to.username, to.secret, err = a.credentials(host)
-		if err != nil {
-			return tokenOptions{}, err
-		}
-	}
-	return to, nil
-}
-
 // authResult is used to control limit rate.
 type authResult struct {
 	sync.WaitGroup
@@ -227,17 +200,17 @@ type authHandler struct {
 	client *http.Client
 
 	// only support basic and bearer schemes
-	scheme authenticationScheme
+	scheme auth.AuthenticationScheme
 
 	// common contains common challenge answer
-	common tokenOptions
+	common auth.TokenOptions
 
 	// scopedTokens caches token indexed by scopes, which used in
 	// bearer auth case
 	scopedTokens map[string]*authResult
 }
 
-func newAuthHandler(client *http.Client, hdr http.Header, scheme authenticationScheme, opts tokenOptions) *authHandler {
+func newAuthHandler(client *http.Client, hdr http.Header, scheme auth.AuthenticationScheme, opts auth.TokenOptions) *authHandler {
 	return &authHandler{
 		header:       hdr,
 		client:       client,
@@ -249,17 +222,17 @@ func newAuthHandler(client *http.Client, hdr http.Header, scheme authenticationS
 
 func (ah *authHandler) authorize(ctx context.Context) (string, error) {
 	switch ah.scheme {
-	case basicAuth:
+	case auth.BasicAuth:
 		return ah.doBasicAuth(ctx)
-	case bearerAuth:
+	case auth.BearerAuth:
 		return ah.doBearerAuth(ctx)
 	default:
-		return "", errors.Wrap(errdefs.ErrNotImplemented, "failed to find supported auth scheme")
+		return "", errors.Wrapf(errdefs.ErrNotImplemented, "failed to find supported auth scheme: %s", string(ah.scheme))
 	}
 }
 
 func (ah *authHandler) doBasicAuth(ctx context.Context) (string, error) {
-	username, secret := ah.common.username, ah.common.secret
+	username, secret := ah.common.Username, ah.common.Secret
 
 	if username == "" || secret == "" {
 		return "", fmt.Errorf("failed to handle basic auth because missing username or secret")
@@ -269,14 +242,14 @@ func (ah *authHandler) doBasicAuth(ctx context.Context) (string, error) {
 	return fmt.Sprintf("Basic %s", auth), nil
 }
 
-func (ah *authHandler) doBearerAuth(ctx context.Context) (string, error) {
+func (ah *authHandler) doBearerAuth(ctx context.Context) (token string, err error) {
 	// copy common tokenOptions
 	to := ah.common
 
-	to.scopes = GetTokenScopes(ctx, to.scopes)
+	to.Scopes = GetTokenScopes(ctx, to.Scopes)
 
 	// Docs: https://docs.docker.com/registry/spec/auth/scope
-	scoped := strings.Join(to.scopes, " ")
+	scoped := strings.Join(to.Scopes, " ")
 
 	ah.Lock()
 	if r, exist := ah.scopedTokens[scoped]; exist {
@@ -291,174 +264,52 @@ func (ah *authHandler) doBearerAuth(ctx context.Context) (string, error) {
 	ah.scopedTokens[scoped] = r
 	ah.Unlock()
 
+	defer func() {
+		token = fmt.Sprintf("Bearer %s", token)
+		r.token, r.err = token, err
+		r.Done()
+	}()
+
 	// fetch token for the resource scope
-	var (
-		token string
-		err   error
-	)
-	if to.secret != "" {
+	if to.Secret != "" {
+		defer func() {
+			err = errors.Wrap(err, "failed to fetch oauth token")
+		}()
 		// credential information is provided, use oauth POST endpoint
-		token, err = ah.fetchTokenWithOAuth(ctx, to)
-		err = errors.Wrap(err, "failed to fetch oauth token")
-	} else {
-		// do request anonymously
-		token, err = ah.fetchToken(ctx, to)
-		err = errors.Wrap(err, "failed to fetch anonymous token")
-	}
-	token = fmt.Sprintf("Bearer %s", token)
-
-	r.token, r.err = token, err
-	r.Done()
-	return r.token, r.err
-}
-
-type tokenOptions struct {
-	realm    string
-	service  string
-	scopes   []string
-	username string
-	secret   string
-}
-
-type postTokenResponse struct {
-	AccessToken  string    `json:"access_token"`
-	RefreshToken string    `json:"refresh_token"`
-	ExpiresIn    int       `json:"expires_in"`
-	IssuedAt     time.Time `json:"issued_at"`
-	Scope        string    `json:"scope"`
-}
-
-func (ah *authHandler) fetchTokenWithOAuth(ctx context.Context, to tokenOptions) (string, error) {
-	form := url.Values{}
-	if len(to.scopes) > 0 {
-		form.Set("scope", strings.Join(to.scopes, " "))
-	}
-	form.Set("service", to.service)
-	// TODO: Allow setting client_id
-	form.Set("client_id", "containerd-client")
-
-	if to.username == "" {
-		form.Set("grant_type", "refresh_token")
-		form.Set("refresh_token", to.secret)
-	} else {
-		form.Set("grant_type", "password")
-		form.Set("username", to.username)
-		form.Set("password", to.secret)
-	}
-
-	req, err := http.NewRequest("POST", to.realm, strings.NewReader(form.Encode()))
-	if err != nil {
-		return "", err
-	}
-	req.Header.Set("Content-Type", "application/x-www-form-urlencoded; charset=utf-8")
-	if ah.header != nil {
-		for k, v := range ah.header {
-			req.Header[k] = append(req.Header[k], v...)
-		}
-	}
-
-	resp, err := ctxhttp.Do(ctx, ah.client, req)
-	if err != nil {
-		return "", err
-	}
-	defer resp.Body.Close()
-
-	// Registries without support for POST may return 404 for POST /v2/token.
-	// As of September 2017, GCR is known to return 404.
-	// As of February 2018, JFrog Artifactory is known to return 401.
-	if (resp.StatusCode == 405 && to.username != "") || resp.StatusCode == 404 || resp.StatusCode == 401 {
-		return ah.fetchToken(ctx, to)
-	} else if resp.StatusCode < 200 || resp.StatusCode >= 400 {
-		b, _ := ioutil.ReadAll(io.LimitReader(resp.Body, 64000)) // 64KB
-		log.G(ctx).WithFields(logrus.Fields{
-			"status": resp.Status,
-			"body":   string(b),
-		}).Debugf("token request failed")
-		// TODO: handle error body and write debug output
-		return "", errors.Errorf("unexpected status: %s", resp.Status)
-	}
-
-	decoder := json.NewDecoder(resp.Body)
-
-	var tr postTokenResponse
-	if err = decoder.Decode(&tr); err != nil {
-		return "", fmt.Errorf("unable to decode token response: %s", err)
-	}
-
-	return tr.AccessToken, nil
-}
-
-type getTokenResponse struct {
-	Token        string    `json:"token"`
-	AccessToken  string    `json:"access_token"`
-	ExpiresIn    int       `json:"expires_in"`
-	IssuedAt     time.Time `json:"issued_at"`
-	RefreshToken string    `json:"refresh_token"`
-}
-
-// fetchToken fetches a token using a GET request
-func (ah *authHandler) fetchToken(ctx context.Context, to tokenOptions) (string, error) {
-	req, err := http.NewRequest("GET", to.realm, nil)
-	if err != nil {
-		return "", err
-	}
-
-	if ah.header != nil {
-		for k, v := range ah.header {
-			req.Header[k] = append(req.Header[k], v...)
+		// TODO: Allow setting client_id
+		resp, err := auth.FetchTokenWithOAuth(ctx, ah.client, ah.header, "containerd-client", to)
+		if err != nil {
+			var errStatus remoteerrors.ErrUnexpectedStatus
+			if errors.As(err, &errStatus) {
+				// Registries without support for POST may return 404 for POST /v2/token.
+				// As of September 2017, GCR is known to return 404.
+				// As of February 2018, JFrog Artifactory is known to return 401.
+				if (errStatus.StatusCode == 405 && to.Username != "") || errStatus.StatusCode == 404 || errStatus.StatusCode == 401 {
+					resp, err := auth.FetchToken(ctx, ah.client, ah.header, to)
+					if err != nil {
+						return "", err
+					}
+					return resp.Token, nil
+				}
+				log.G(ctx).WithFields(logrus.Fields{
+					"status": errStatus.Status,
+					"body":   string(errStatus.Body),
+				}).Debugf("token request failed")
+			}
+			return "", err
 		}
+		return resp.AccessToken, nil
 	}
-
-	reqParams := req.URL.Query()
-
-	if to.service != "" {
-		reqParams.Add("service", to.service)
-	}
-
-	for _, scope := range to.scopes {
-		reqParams.Add("scope", scope)
-	}
-
-	if to.secret != "" {
-		req.SetBasicAuth(to.username, to.secret)
-	}
-
-	req.URL.RawQuery = reqParams.Encode()
-
-	resp, err := ctxhttp.Do(ctx, ah.client, req)
+	// do request anonymously
+	resp, err := auth.FetchToken(ctx, ah.client, ah.header, to)
 	if err != nil {
-		return "", err
-	}
-	defer resp.Body.Close()
-
-	if resp.StatusCode < 200 || resp.StatusCode >= 400 {
-		// TODO: handle error body and write debug output
-		return "", errors.Errorf("unexpected status: %s", resp.Status)
+		return "", errors.Wrap(err, "failed to fetch anonymous token")
 	}
-
-	decoder := json.NewDecoder(resp.Body)
-
-	var tr getTokenResponse
-	if err = decoder.Decode(&tr); err != nil {
-		return "", fmt.Errorf("unable to decode token response: %s", err)
-	}
-
-	// `access_token` is equivalent to `token` and if both are specified
-	// the choice is undefined.  Canonicalize `access_token` by sticking
-	// things in `token`.
-	if tr.AccessToken != "" {
-		tr.Token = tr.AccessToken
-	}
-
-	if tr.Token == "" {
-		return "", ErrNoToken
-	}
-
-	return tr.Token, nil
+	return resp.Token, nil
 }
 
-func invalidAuthorization(c challenge, responses []*http.Response) error {
-	errStr := c.parameters["error"]
+func invalidAuthorization(c auth.Challenge, responses []*http.Response) error {
+	errStr := c.Parameters["error"]
 	if errStr == "" {
 		return nil
 	}

+ 10 - 6
vendor/github.com/containerd/containerd/remotes/docker/pusher.go

@@ -30,6 +30,7 @@ import (
 	"github.com/containerd/containerd/images"
 	"github.com/containerd/containerd/log"
 	"github.com/containerd/containerd/remotes"
+	remoteserrors "github.com/containerd/containerd/remotes/errors"
 	digest "github.com/opencontainers/go-digest"
 	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
 	"github.com/pkg/errors"
@@ -112,8 +113,9 @@ func (p dockerPusher) Push(ctx context.Context, desc ocispec.Descriptor) (conten
 				return nil, errors.Wrapf(errdefs.ErrAlreadyExists, "content %v on remote", desc.Digest)
 			}
 		} else if resp.StatusCode != http.StatusNotFound {
-			// TODO: log error
-			return nil, errors.Errorf("unexpected response: %s", resp.Status)
+			err := remoteserrors.NewUnexpectedStatusErr(resp)
+			log.G(ctx).WithField("resp", resp).WithField("body", string(err.(remoteserrors.ErrUnexpectedStatus).Body)).Debug("unexpected response")
+			return nil, err
 		}
 	}
 
@@ -166,8 +168,9 @@ func (p dockerPusher) Push(ctx context.Context, desc ocispec.Descriptor) (conten
 			})
 			return nil, errors.Wrapf(errdefs.ErrAlreadyExists, "content %v on remote", desc.Digest)
 		default:
-			// TODO: log error
-			return nil, errors.Errorf("unexpected response: %s", resp.Status)
+			err := remoteserrors.NewUnexpectedStatusErr(resp)
+			log.G(ctx).WithField("resp", resp).WithField("body", string(err.(remoteserrors.ErrUnexpectedStatus).Body)).Debug("unexpected response")
+			return nil, err
 		}
 
 		var (
@@ -244,8 +247,9 @@ func (p dockerPusher) Push(ctx context.Context, desc ocispec.Descriptor) (conten
 		switch resp.StatusCode {
 		case http.StatusOK, http.StatusCreated, http.StatusNoContent:
 		default:
-			// TODO: log error
-			pr.CloseWithError(errors.Errorf("unexpected response: %s", resp.Status))
+			err := remoteserrors.NewUnexpectedStatusErr(resp)
+			log.G(ctx).WithField("resp", resp).WithField("body", string(err.(remoteserrors.ErrUnexpectedStatus).Body)).Debug("unexpected response")
+			pr.CloseWithError(err)
 		}
 		respC <- resp
 	}()

+ 0 - 4
vendor/github.com/containerd/containerd/remotes/docker/resolver.go

@@ -41,10 +41,6 @@ import (
 )
 
 var (
-	// ErrNoToken is returned if a request is successful but the body does not
-	// contain an authorization token.
-	ErrNoToken = errors.New("authorization server did not include a token in the response")
-
 	// ErrInvalidAuthorization is used when credentials are passed to a server but
 	// those credentials are rejected.
 	ErrInvalidAuthorization = errors.New("authorization failed")

+ 46 - 0
vendor/github.com/containerd/containerd/remotes/errors/errors.go

@@ -0,0 +1,46 @@
+/*
+   Copyright The containerd Authors.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package errors
+
+import (
+	"fmt"
+	"io"
+	"io/ioutil"
+	"net/http"
+)
+
+var _ error = ErrUnexpectedStatus{}
+
+// ErrUnexpectedStatus is returned if a registry API request returned with unexpected HTTP status
+type ErrUnexpectedStatus struct {
+	Status     string
+	StatusCode int
+	Body       []byte
+}
+
+func (e ErrUnexpectedStatus) Error() string {
+	return fmt.Sprintf("unexpected status: %s", e.Status)
+}
+
+// NewUnexpectedStatusErr creates an ErrUnexpectedStatus from HTTP response
+func NewUnexpectedStatusErr(resp *http.Response) error {
+	var b []byte
+	if resp.Body != nil {
+		b, _ = ioutil.ReadAll(io.LimitReader(resp.Body, 64000)) // 64KB
+	}
+	return ErrUnexpectedStatus{Status: resp.Status, StatusCode: resp.StatusCode, Body: b}
+}

+ 14 - 17
vendor/github.com/containerd/containerd/runtime/v1/shim/client/client.go

@@ -22,7 +22,6 @@ import (
 	"context"
 	"fmt"
 	"io"
-	"io/ioutil"
 	"net"
 	"os"
 	"os/exec"
@@ -68,24 +67,22 @@ func WithStart(binary, address, daemonAddress, cgroup string, debug bool, exitHa
 		}
 		defer f.Close()
 
-		stdoutCopy := ioutil.Discard
-		stderrCopy := ioutil.Discard
-		stdoutLog, err := v1.OpenShimStdoutLog(ctx, config.WorkDir)
-		if err != nil {
-			return nil, nil, errors.Wrapf(err, "failed to create stdout log")
-		}
-
-		stderrLog, err := v1.OpenShimStderrLog(ctx, config.WorkDir)
-		if err != nil {
-			return nil, nil, errors.Wrapf(err, "failed to create stderr log")
-		}
+		var stdoutLog io.ReadWriteCloser
+		var stderrLog io.ReadWriteCloser
 		if debug {
-			stdoutCopy = os.Stdout
-			stderrCopy = os.Stderr
-		}
+			stdoutLog, err = v1.OpenShimStdoutLog(ctx, config.WorkDir)
+			if err != nil {
+				return nil, nil, errors.Wrapf(err, "failed to create stdout log")
+			}
+
+			stderrLog, err = v1.OpenShimStderrLog(ctx, config.WorkDir)
+			if err != nil {
+				return nil, nil, errors.Wrapf(err, "failed to create stderr log")
+			}
 
-		go io.Copy(stdoutCopy, stdoutLog)
-		go io.Copy(stderrCopy, stderrLog)
+			go io.Copy(os.Stdout, stdoutLog)
+			go io.Copy(os.Stderr, stderrLog)
+		}
 
 		cmd, err := newCommand(binary, daemonAddress, debug, config, f, stdoutLog, stderrLog)
 		if err != nil {

+ 2 - 1
vendor/github.com/containerd/containerd/vendor.conf

@@ -7,6 +7,7 @@ github.com/containerd/console                       v1.0.0
 github.com/containerd/continuity                    efbc4488d8fe1bdc16bde3b2d2990d9b3a899165
 github.com/containerd/fifo                          f15a3290365b9d2627d189e619ab4008e0069caf
 github.com/containerd/go-runc                       7016d3ce2328dd2cb1192b2076ebd565c4e8df0c
+github.com/containerd/nri                           0afc7f031eaf9c7d9c1a381b7ab5462e89c998fc
 github.com/containerd/ttrpc                         v1.0.1
 github.com/containerd/typeurl                       v1.0.1
 github.com/coreos/go-systemd/v22                    v22.1.0
@@ -57,7 +58,7 @@ gotest.tools/v3                                     v3.0.2
 github.com/cilium/ebpf                              1c8d4c9ef7759622653a1d319284a44652333b28
 
 # cri dependencies
-github.com/containerd/cri                           4e6644c8cf7fb825f62e0007421b7d83dfeab5a1 # master
+github.com/containerd/cri                           35e623e6bf7512e8c82b8ac6052cb1d720189f28 # master
 github.com/davecgh/go-spew                          v1.1.1
 github.com/docker/docker                            4634ce647cf2ce2c6031129ccd109e557244986f
 github.com/docker/spdystream                        449fdfce4d962303d702fec724ef0ad181c92528

+ 1 - 1
vendor/github.com/containerd/containerd/version/version.go

@@ -23,7 +23,7 @@ var (
 	Package = "github.com/containerd/containerd"
 
 	// Version holds the complete version number. Filled in at linking time.
-	Version = "1.4.1+unknown"
+	Version = "1.4.0+unknown"
 
 	// Revision is filled with the VCS (e.g. git) revision being used to build
 	// the program at linking time.

+ 5 - 4
vendor/github.com/containerd/go-runc/go.mod

@@ -3,8 +3,9 @@ module github.com/containerd/go-runc
 go 1.13
 
 require (
-	github.com/containerd/console v0.0.0-20191206165004-02ecf6a7291e
-	github.com/opencontainers/runtime-spec v1.0.1
-	github.com/pkg/errors v0.8.1
-	golang.org/x/sys v0.0.0-20191210023423-ac6580df4449
+	github.com/containerd/console v1.0.1
+	github.com/opencontainers/runtime-spec v1.0.2
+	github.com/pkg/errors v0.9.1
+	github.com/sirupsen/logrus v1.7.0
+	golang.org/x/sys v0.0.0-20200916030750-2334cc1a136f
 )

+ 23 - 3
vendor/github.com/containerd/go-runc/io_unix.go

@@ -20,7 +20,9 @@ package runc
 
 import (
 	"github.com/pkg/errors"
+	"github.com/sirupsen/logrus"
 	"golang.org/x/sys/unix"
+	"runtime"
 )
 
 // NewPipeIO creates pipe pairs to be used with runc
@@ -47,7 +49,13 @@ func NewPipeIO(uid, gid int, opts ...IOOpt) (i IO, err error) {
 		}
 		pipes = append(pipes, stdin)
 		if err = unix.Fchown(int(stdin.r.Fd()), uid, gid); err != nil {
-			return nil, errors.Wrap(err, "failed to chown stdin")
+			// TODO: revert with proper darwin solution, skipping for now
+			// as darwin chown is returning EINVAL on anonymous pipe
+			if runtime.GOOS == "darwin" {
+				logrus.WithError(err).Debug("failed to chown stdin, ignored")
+			} else {
+				return nil, errors.Wrap(err, "failed to chown stdin")
+			}
 		}
 	}
 	if option.OpenStdout {
@@ -56,7 +64,13 @@ func NewPipeIO(uid, gid int, opts ...IOOpt) (i IO, err error) {
 		}
 		pipes = append(pipes, stdout)
 		if err = unix.Fchown(int(stdout.w.Fd()), uid, gid); err != nil {
-			return nil, errors.Wrap(err, "failed to chown stdout")
+			// TODO: revert with proper darwin solution, skipping for now
+			// as darwin chown is returning EINVAL on anonymous pipe
+			if runtime.GOOS == "darwin" {
+				logrus.WithError(err).Debug("failed to chown stdout, ignored")
+			} else {
+				return nil, errors.Wrap(err, "failed to chown stdout")
+			}
 		}
 	}
 	if option.OpenStderr {
@@ -65,7 +79,13 @@ func NewPipeIO(uid, gid int, opts ...IOOpt) (i IO, err error) {
 		}
 		pipes = append(pipes, stderr)
 		if err = unix.Fchown(int(stderr.w.Fd()), uid, gid); err != nil {
-			return nil, errors.Wrap(err, "failed to chown stderr")
+			// TODO: revert with proper darwin solution, skipping for now
+			// as darwin chown is returning EINVAL on anonymous pipe
+			if runtime.GOOS == "darwin" {
+				logrus.WithError(err).Debug("failed to chown stderr, ignored")
+			} else {
+				return nil, errors.Wrap(err, "failed to chown stderr")
+			}
 		}
 	}
 	return &pipeIO{

+ 60 - 34
vendor/github.com/containerd/go-runc/runc.go

@@ -29,7 +29,6 @@ import (
 	"path/filepath"
 	"strconv"
 	"strings"
-	"syscall"
 	"time"
 
 	specs "github.com/opencontainers/runtime-spec/specs-go"
@@ -55,24 +54,9 @@ const (
 	DefaultCommand = "runc"
 )
 
-// Runc is the client to the runc cli
-type Runc struct {
-	//If command is empty, DefaultCommand is used
-	Command       string
-	Root          string
-	Debug         bool
-	Log           string
-	LogFormat     Format
-	PdeathSignal  syscall.Signal
-	Setpgid       bool
-	Criu          string
-	SystemdCgroup bool
-	Rootless      *bool // nil stands for "auto"
-}
-
 // List returns all containers created inside the provided runc root directory
 func (r *Runc) List(context context.Context) ([]*Container, error) {
-	data, err := cmdOutput(r.command(context, "list", "--format=json"), false)
+	data, err := cmdOutput(r.command(context, "list", "--format=json"), false, nil)
 	defer putBuf(data)
 	if err != nil {
 		return nil, err
@@ -86,7 +70,7 @@ func (r *Runc) List(context context.Context) ([]*Container, error) {
 
 // State returns the state for the container provided by id
 func (r *Runc) State(context context.Context, id string) (*Container, error) {
-	data, err := cmdOutput(r.command(context, "state", id), true)
+	data, err := cmdOutput(r.command(context, "state", id), true, nil)
 	defer putBuf(data)
 	if err != nil {
 		return nil, fmt.Errorf("%s: %s", err, data.String())
@@ -111,6 +95,7 @@ type CreateOpts struct {
 	NoPivot       bool
 	NoNewKeyring  bool
 	ExtraFiles    []*os.File
+	Started       chan<- int
 }
 
 func (o *CreateOpts) args() (out []string, err error) {
@@ -156,7 +141,7 @@ func (r *Runc) Create(context context.Context, id, bundle string, opts *CreateOp
 	cmd.ExtraFiles = opts.ExtraFiles
 
 	if cmd.Stdout == nil && cmd.Stderr == nil {
-		data, err := cmdOutput(cmd, true)
+		data, err := cmdOutput(cmd, true, nil)
 		defer putBuf(data)
 		if err != nil {
 			return fmt.Errorf("%s: %s", err, data.String())
@@ -176,7 +161,7 @@ func (r *Runc) Create(context context.Context, id, bundle string, opts *CreateOp
 	}
 	status, err := Monitor.Wait(cmd, ec)
 	if err == nil && status != 0 {
-		err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+		err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 	}
 	return err
 }
@@ -191,6 +176,7 @@ type ExecOpts struct {
 	PidFile       string
 	ConsoleSocket ConsoleSocket
 	Detach        bool
+	Started       chan<- int
 }
 
 func (o *ExecOpts) args() (out []string, err error) {
@@ -210,9 +196,12 @@ func (o *ExecOpts) args() (out []string, err error) {
 	return out, nil
 }
 
-// Exec executres and additional process inside the container based on a full
+// Exec executes an additional process inside the container based on a full
 // OCI Process specification
 func (r *Runc) Exec(context context.Context, id string, spec specs.Process, opts *ExecOpts) error {
+	if opts.Started != nil {
+		defer close(opts.Started)
+	}
 	f, err := ioutil.TempFile(os.Getenv("XDG_RUNTIME_DIR"), "runc-process")
 	if err != nil {
 		return err
@@ -236,10 +225,10 @@ func (r *Runc) Exec(context context.Context, id string, spec specs.Process, opts
 		opts.Set(cmd)
 	}
 	if cmd.Stdout == nil && cmd.Stderr == nil {
-		data, err := cmdOutput(cmd, true)
+		data, err := cmdOutput(cmd, true, opts.Started)
 		defer putBuf(data)
 		if err != nil {
-			return fmt.Errorf("%s: %s", err, data.String())
+			return fmt.Errorf("%w: %s", err, data.String())
 		}
 		return nil
 	}
@@ -247,6 +236,9 @@ func (r *Runc) Exec(context context.Context, id string, spec specs.Process, opts
 	if err != nil {
 		return err
 	}
+	if opts.Started != nil {
+		opts.Started <- cmd.Process.Pid
+	}
 	if opts != nil && opts.IO != nil {
 		if c, ok := opts.IO.(StartCloser); ok {
 			if err := c.CloseAfterStart(); err != nil {
@@ -256,7 +248,7 @@ func (r *Runc) Exec(context context.Context, id string, spec specs.Process, opts
 	}
 	status, err := Monitor.Wait(cmd, ec)
 	if err == nil && status != 0 {
-		err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+		err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 	}
 	return err
 }
@@ -264,6 +256,9 @@ func (r *Runc) Exec(context context.Context, id string, spec specs.Process, opts
 // Run runs the create, start, delete lifecycle of the container
 // and returns its exit status after it has exited
 func (r *Runc) Run(context context.Context, id, bundle string, opts *CreateOpts) (int, error) {
+	if opts.Started != nil {
+		defer close(opts.Started)
+	}
 	args := []string{"run", "--bundle", bundle}
 	if opts != nil {
 		oargs, err := opts.args()
@@ -280,9 +275,12 @@ func (r *Runc) Run(context context.Context, id, bundle string, opts *CreateOpts)
 	if err != nil {
 		return -1, err
 	}
+	if opts.Started != nil {
+		opts.Started <- cmd.Process.Pid
+	}
 	status, err := Monitor.Wait(cmd, ec)
 	if err == nil && status != 0 {
-		err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+		err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 	}
 	return status, err
 }
@@ -403,7 +401,7 @@ func (r *Runc) Resume(context context.Context, id string) error {
 
 // Ps lists all the processes inside the container returning their pids
 func (r *Runc) Ps(context context.Context, id string) ([]int, error) {
-	data, err := cmdOutput(r.command(context, "ps", "--format", "json", id), true)
+	data, err := cmdOutput(r.command(context, "ps", "--format", "json", id), true, nil)
 	defer putBuf(data)
 	if err != nil {
 		return nil, fmt.Errorf("%s: %s", err, data.String())
@@ -417,7 +415,7 @@ func (r *Runc) Ps(context context.Context, id string) ([]int, error) {
 
 // Top lists all the processes inside the container returning the full ps data
 func (r *Runc) Top(context context.Context, id string, psOptions string) (*TopResults, error) {
-	data, err := cmdOutput(r.command(context, "ps", "--format", "table", id, psOptions), true)
+	data, err := cmdOutput(r.command(context, "ps", "--format", "table", id, psOptions), true, nil)
 	defer putBuf(data)
 	if err != nil {
 		return nil, fmt.Errorf("%s: %s", err, data.String())
@@ -452,6 +450,10 @@ type CheckpointOpts struct {
 	// EmptyNamespaces creates a namespace for the container but does not save its properties
 	// Provide the namespaces you wish to be checkpointed without their settings on restore
 	EmptyNamespaces []string
+	// LazyPages uses userfaultfd to lazily restore memory pages
+	LazyPages bool
+	// StatusFile is the file criu writes \0 to once lazy-pages is ready
+	StatusFile *os.File
 }
 
 type CgroupMode string
@@ -493,6 +495,9 @@ func (o *CheckpointOpts) args() (out []string) {
 	for _, ns := range o.EmptyNamespaces {
 		out = append(out, "--empty-ns", ns)
 	}
+	if o.LazyPages {
+		out = append(out, "--lazy-pages")
+	}
 	return out
 }
 
@@ -511,13 +516,23 @@ func PreDump(args []string) []string {
 // Checkpoint allows you to checkpoint a container using criu
 func (r *Runc) Checkpoint(context context.Context, id string, opts *CheckpointOpts, actions ...CheckpointAction) error {
 	args := []string{"checkpoint"}
+	extraFiles := []*os.File{}
 	if opts != nil {
 		args = append(args, opts.args()...)
+		if opts.StatusFile != nil {
+			// pass the status file to the child process
+			extraFiles = []*os.File{opts.StatusFile}
+			// set status-fd to 3 as this will be the file descriptor
+			// of the first file passed with cmd.ExtraFiles
+			args = append(args, "--status-fd", "3")
+		}
 	}
 	for _, a := range actions {
 		args = a(args)
 	}
-	return r.runOrError(r.command(context, append(args, id)...))
+	cmd := r.command(context, append(args, id)...)
+	cmd.ExtraFiles = extraFiles
+	return r.runOrError(cmd)
 }
 
 type RestoreOpts struct {
@@ -583,7 +598,7 @@ func (r *Runc) Restore(context context.Context, id, bundle string, opts *Restore
 	}
 	status, err := Monitor.Wait(cmd, ec)
 	if err == nil && status != 0 {
-		err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+		err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 	}
 	return status, err
 }
@@ -612,7 +627,7 @@ type Version struct {
 
 // Version returns the runc and runtime-spec versions
 func (r *Runc) Version(context context.Context) (Version, error) {
-	data, err := cmdOutput(r.command(context, "--version"), false)
+	data, err := cmdOutput(r.command(context, "--version"), false, nil)
 	defer putBuf(data)
 	if err != nil {
 		return Version{}, err
@@ -680,11 +695,11 @@ func (r *Runc) runOrError(cmd *exec.Cmd) error {
 		}
 		status, err := Monitor.Wait(cmd, ec)
 		if err == nil && status != 0 {
-			err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+			err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 		}
 		return err
 	}
-	data, err := cmdOutput(cmd, true)
+	data, err := cmdOutput(cmd, true, nil)
 	defer putBuf(data)
 	if err != nil {
 		return fmt.Errorf("%s: %s", err, data.String())
@@ -694,7 +709,7 @@ func (r *Runc) runOrError(cmd *exec.Cmd) error {
 
 // callers of cmdOutput are expected to call putBuf on the returned Buffer
 // to ensure it is released back to the shared pool after use.
-func cmdOutput(cmd *exec.Cmd, combined bool) (*bytes.Buffer, error) {
+func cmdOutput(cmd *exec.Cmd, combined bool, started chan<- int) (*bytes.Buffer, error) {
 	b := getBuf()
 
 	cmd.Stdout = b
@@ -705,11 +720,22 @@ func cmdOutput(cmd *exec.Cmd, combined bool) (*bytes.Buffer, error) {
 	if err != nil {
 		return nil, err
 	}
+	if started != nil {
+		started <- cmd.Process.Pid
+	}
 
 	status, err := Monitor.Wait(cmd, ec)
 	if err == nil && status != 0 {
-		err = fmt.Errorf("%s did not terminate successfully", cmd.Args[0])
+		err = fmt.Errorf("%s did not terminate successfully: %w", cmd.Args[0], &ExitError{status})
 	}
 
 	return b, err
 }
+
+type ExitError struct {
+	Status int
+}
+
+func (e *ExitError) Error() string {
+	return fmt.Sprintf("exit status %d", e.Status)
+}

+ 38 - 0
vendor/github.com/containerd/go-runc/runc_unix.go

@@ -0,0 +1,38 @@
+//+build !windows
+
+/*
+   Copyright The containerd Authors.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package runc
+
+import (
+	"golang.org/x/sys/unix"
+)
+
+// Runc is the client to the runc cli
+type Runc struct {
+	//If command is empty, DefaultCommand is used
+	Command       string
+	Root          string
+	Debug         bool
+	Log           string
+	LogFormat     Format
+	PdeathSignal  unix.Signal
+	Setpgid       bool
+	Criu          string
+	SystemdCgroup bool
+	Rootless      *bool // nil stands for "auto"
+}

+ 31 - 0
vendor/github.com/containerd/go-runc/runc_windows.go

@@ -0,0 +1,31 @@
+/*
+   Copyright The containerd Authors.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package runc
+
+// Runc is the client to the runc cli
+type Runc struct {
+	//If command is empty, DefaultCommand is used
+	Command       string
+	Root          string
+	Debug         bool
+	Log           string
+	LogFormat     Format
+	Setpgid       bool
+	Criu          string
+	SystemdCgroup bool
+	Rootless      *bool // nil stands for "auto"
+}

+ 1 - 0
vendor/github.com/mitchellh/hashstructure/go.mod

@@ -0,0 +1 @@
+module github.com/mitchellh/hashstructure

+ 15 - 5
vendor/github.com/moby/buildkit/README.md

@@ -62,6 +62,7 @@ You don't need to read this document unless you want to use the full-featured st
 - [Expose BuildKit as a TCP service](#expose-buildkit-as-a-tcp-service)
   - [Load balancing](#load-balancing)
 - [Containerizing BuildKit](#containerizing-buildkit)
+  - [Podman](#podman)
   - [Kubernetes](#kubernetes)
   - [Daemonless](#daemonless)
 - [Opentracing support](#opentracing-support)
@@ -127,11 +128,6 @@ We are open to adding more backends.
 The buildkitd daemon listens gRPC API on `/run/buildkit/buildkitd.sock` by default, but you can also use TCP sockets.
 See [Expose BuildKit as a TCP service](#expose-buildkit-as-a-tcp-service).
 
-:information_source: Notice to Fedora 31 users:
-
-* As runc still does not work on cgroup v2 environment like Fedora 31, you need to substitute runc with crun. Run `buildkitd` with `--oci-worker-binary=crun`.
-* If you want to use runc, you need to configure the system to use cgroup v1. Run `sudo grubby --update-kernel=ALL --args="systemd.unified_cgroup_hierarchy=0"` and reboot.
-
 ### Exploring LLB
 
 BuildKit builds are based on a binary intermediate format called LLB that is used for defining the dependency graph for processes running part of your build. tl;dr: LLB is to Dockerfile what LLVM IR is to C.
@@ -150,6 +146,9 @@ Currently, the following high-level languages has been implemented for LLB:
 -   [Mockerfile](https://matt-rickard.com/building-a-new-dockerfile-frontend/)
 -   [Gockerfile](https://github.com/po3rin/gockerfile)
 -   [bldr (Pkgfile)](https://github.com/talos-systems/bldr/)
+-   [HLB](https://github.com/openllb/hlb)
+-   [Earthfile (Earthly)](https://github.com/earthly/earthly)
+-   [Cargo Wharf (Rust)](https://github.com/denzp/cargo-wharf)
 -   (open a PR to add your own language)
 
 ### Exploring Dockerfiles
@@ -353,6 +352,7 @@ The directory layout conforms to OCI Image Spec v1.0.
 -   `mode=max`: export all the layers of all intermediate steps. Not supported for `inline` cache exporter.
 -   `ref=docker.io/user/image:tag`: reference for `registry` cache exporter
 -   `dest=path/to/output-dir`: directory for `local` cache exporter
+-   `oci-mediatypes=true|false`: whether to use OCI mediatypes in exported manifests for `local` and `registry` exporter. Since BuildKit `v0.8` defaults to true.
 
 #### `--import-cache` options
 -   `type`: `registry` or `local`. Use `registry` to import `inline` cache.
@@ -418,6 +418,16 @@ export BUILDKIT_HOST=docker-container://buildkitd
 buildctl build --help
 ```
 
+### Podman
+To connect to a BuildKit daemon running in a Podman container, use `podman-container://` instead of `docker-container://` .
+
+```bash
+podman run -d --name buildkitd --privileged moby/buildkit:latest
+buildctl --addr=podman-container://buildkitd build --frontend dockerfile.v0 --local context=. --local dockerfile=. --output type=oci | podman load foo
+```
+
+`sudo` is not required.
+
 ### Kubernetes
 
 For Kubernetes deployments, see [`examples/kubernetes`](./examples/kubernetes).

+ 1 - 1
vendor/github.com/moby/buildkit/api/services/control/generate.go

@@ -1,3 +1,3 @@
-package moby_buildkit_v1
+package moby_buildkit_v1 //nolint:golint
 
 //go:generate protoc -I=. -I=../../../vendor/ -I=../../../../../../ --gogo_out=plugins=grpc:. control.proto

+ 1 - 1
vendor/github.com/moby/buildkit/api/types/generate.go

@@ -1,3 +1,3 @@
-package moby_buildkit_v1_types
+package moby_buildkit_v1_types //nolint:golint
 
 //go:generate protoc -I=. -I=../../vendor/ -I=../../../../../ --gogo_out=plugins=grpc:. worker.proto

+ 236 - 0
vendor/github.com/moby/buildkit/cache/blobs.go

@@ -0,0 +1,236 @@
+package cache
+
+import (
+	"context"
+
+	"github.com/containerd/containerd/diff"
+	"github.com/containerd/containerd/leases"
+	"github.com/containerd/containerd/mount"
+	"github.com/moby/buildkit/session"
+	"github.com/moby/buildkit/util/compression"
+	"github.com/moby/buildkit/util/flightcontrol"
+	"github.com/moby/buildkit/util/winlayers"
+	digest "github.com/opencontainers/go-digest"
+	imagespecidentity "github.com/opencontainers/image-spec/identity"
+	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
+	"github.com/pkg/errors"
+	"golang.org/x/sync/errgroup"
+)
+
+var g flightcontrol.Group
+
+const containerdUncompressed = "containerd.io/uncompressed"
+
+type CompareWithParent interface {
+	CompareWithParent(ctx context.Context, ref string, opts ...diff.Opt) (ocispec.Descriptor, error)
+}
+
+var ErrNoBlobs = errors.Errorf("no blobs for snapshot")
+
+// computeBlobChain ensures every ref in a parent chain has an associated blob in the content store. If
+// a blob is missing and createIfNeeded is true, then the blob will be created, otherwise ErrNoBlobs will
+// be returned. Caller must hold a lease when calling this function.
+func (sr *immutableRef) computeBlobChain(ctx context.Context, createIfNeeded bool, compressionType compression.Type, s session.Group) error {
+	if _, ok := leases.FromContext(ctx); !ok {
+		return errors.Errorf("missing lease requirement for computeBlobChain")
+	}
+
+	if err := sr.Finalize(ctx, true); err != nil {
+		return err
+	}
+
+	if isTypeWindows(sr) {
+		ctx = winlayers.UseWindowsLayerMode(ctx)
+	}
+
+	return computeBlobChain(ctx, sr, createIfNeeded, compressionType, s)
+}
+
+func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool, compressionType compression.Type, s session.Group) error {
+	baseCtx := ctx
+	eg, ctx := errgroup.WithContext(ctx)
+	var currentDescr ocispec.Descriptor
+	if sr.parent != nil {
+		eg.Go(func() error {
+			return computeBlobChain(ctx, sr.parent, createIfNeeded, compressionType, s)
+		})
+	}
+	eg.Go(func() error {
+		dp, err := g.Do(ctx, sr.ID(), func(ctx context.Context) (interface{}, error) {
+			refInfo := sr.Info()
+			if refInfo.Blob != "" {
+				return nil, nil
+			} else if !createIfNeeded {
+				return nil, errors.WithStack(ErrNoBlobs)
+			}
+
+			var mediaType string
+			switch compressionType {
+			case compression.Uncompressed:
+				mediaType = ocispec.MediaTypeImageLayer
+			case compression.Gzip:
+				mediaType = ocispec.MediaTypeImageLayerGzip
+			default:
+				return nil, errors.Errorf("unknown layer compression type: %q", compressionType)
+			}
+
+			var descr ocispec.Descriptor
+			var err error
+
+			if pc, ok := sr.cm.Differ.(CompareWithParent); ok {
+				descr, err = pc.CompareWithParent(ctx, sr.ID(), diff.WithMediaType(mediaType))
+				if err != nil {
+					return nil, err
+				}
+			}
+			if descr.Digest == "" {
+				// reference needs to be committed
+				var lower []mount.Mount
+				if sr.parent != nil {
+					m, err := sr.parent.Mount(ctx, true, s)
+					if err != nil {
+						return nil, err
+					}
+					var release func() error
+					lower, release, err = m.Mount()
+					if err != nil {
+						return nil, err
+					}
+					if release != nil {
+						defer release()
+					}
+				}
+				m, err := sr.Mount(ctx, true, s)
+				if err != nil {
+					return nil, err
+				}
+				upper, release, err := m.Mount()
+				if err != nil {
+					return nil, err
+				}
+				if release != nil {
+					defer release()
+				}
+				descr, err = sr.cm.Differ.Compare(ctx, lower, upper,
+					diff.WithMediaType(mediaType),
+					diff.WithReference(sr.ID()),
+				)
+				if err != nil {
+					return nil, err
+				}
+			}
+
+			if descr.Annotations == nil {
+				descr.Annotations = map[string]string{}
+			}
+
+			info, err := sr.cm.ContentStore.Info(ctx, descr.Digest)
+			if err != nil {
+				return nil, err
+			}
+
+			if diffID, ok := info.Labels[containerdUncompressed]; ok {
+				descr.Annotations[containerdUncompressed] = diffID
+			} else if compressionType == compression.Uncompressed {
+				descr.Annotations[containerdUncompressed] = descr.Digest.String()
+			} else {
+				return nil, errors.Errorf("unknown layer compression type")
+			}
+
+			return descr, nil
+
+		})
+		if err != nil {
+			return err
+		}
+
+		if dp != nil {
+			currentDescr = dp.(ocispec.Descriptor)
+		}
+		return nil
+	})
+	err := eg.Wait()
+	if err != nil {
+		return err
+	}
+	if currentDescr.Digest != "" {
+		if err := sr.setBlob(baseCtx, currentDescr); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// setBlob associates a blob with the cache record.
+// A lease must be held for the blob when calling this function
+// Caller should call Info() for knowing what current values are actually set
+func (sr *immutableRef) setBlob(ctx context.Context, desc ocispec.Descriptor) error {
+	if _, ok := leases.FromContext(ctx); !ok {
+		return errors.Errorf("missing lease requirement for setBlob")
+	}
+
+	diffID, err := diffIDFromDescriptor(desc)
+	if err != nil {
+		return err
+	}
+	if _, err := sr.cm.ContentStore.Info(ctx, desc.Digest); err != nil {
+		return err
+	}
+
+	sr.mu.Lock()
+	defer sr.mu.Unlock()
+
+	if getChainID(sr.md) != "" {
+		return nil
+	}
+
+	if err := sr.finalize(ctx, true); err != nil {
+		return err
+	}
+
+	p := sr.parent
+	var parentChainID digest.Digest
+	var parentBlobChainID digest.Digest
+	if p != nil {
+		pInfo := p.Info()
+		if pInfo.ChainID == "" || pInfo.BlobChainID == "" {
+			return errors.Errorf("failed to set blob for reference with non-addressable parent")
+		}
+		parentChainID = pInfo.ChainID
+		parentBlobChainID = pInfo.BlobChainID
+	}
+
+	if err := sr.cm.LeaseManager.AddResource(ctx, leases.Lease{ID: sr.ID()}, leases.Resource{
+		ID:   desc.Digest.String(),
+		Type: "content",
+	}); err != nil {
+		return err
+	}
+
+	queueDiffID(sr.md, diffID.String())
+	queueBlob(sr.md, desc.Digest.String())
+	chainID := diffID
+	blobChainID := imagespecidentity.ChainID([]digest.Digest{desc.Digest, diffID})
+	if parentChainID != "" {
+		chainID = imagespecidentity.ChainID([]digest.Digest{parentChainID, chainID})
+		blobChainID = imagespecidentity.ChainID([]digest.Digest{parentBlobChainID, blobChainID})
+	}
+	queueChainID(sr.md, chainID.String())
+	queueBlobChainID(sr.md, blobChainID.String())
+	queueMediaType(sr.md, desc.MediaType)
+	queueBlobSize(sr.md, desc.Size)
+	if err := sr.md.Commit(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func isTypeWindows(sr *immutableRef) bool {
+	if GetLayerType(sr) == "windows" {
+		return true
+	}
+	if parent := sr.parent; parent != nil {
+		return isTypeWindows(parent)
+	}
+	return false
+}

+ 39 - 35
vendor/github.com/moby/buildkit/cache/contenthash/checksum.go

@@ -11,12 +11,13 @@ import (
 	"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"
 	"github.com/moby/buildkit/cache"
 	"github.com/moby/buildkit/cache/metadata"
+	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/snapshot"
+	"github.com/moby/locker"
 	digest "github.com/opencontainers/go-digest"
 	"github.com/pkg/errors"
 	"github.com/tonistiigi/fsutil"
@@ -44,12 +45,12 @@ func getDefaultManager() *cacheManager {
 // header, "/dir" is for contents. For the root node "" (empty string) is the
 // key for root, "/" for the root header
 
-func Checksum(ctx context.Context, ref cache.ImmutableRef, path string, followLinks bool) (digest.Digest, error) {
-	return getDefaultManager().Checksum(ctx, ref, path, followLinks)
+func Checksum(ctx context.Context, ref cache.ImmutableRef, path string, followLinks bool, s session.Group) (digest.Digest, error) {
+	return getDefaultManager().Checksum(ctx, ref, path, followLinks, s)
 }
 
-func ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, path string, followLinks bool) (digest.Digest, error) {
-	return getDefaultManager().ChecksumWildcard(ctx, ref, path, followLinks)
+func ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, path string, followLinks bool, s session.Group) (digest.Digest, error) {
+	return getDefaultManager().ChecksumWildcard(ctx, ref, path, followLinks, s)
 }
 
 func GetCacheContext(ctx context.Context, md *metadata.StorageItem, idmap *idtools.IdentityMapping) (CacheContext, error) {
@@ -65,8 +66,8 @@ func ClearCacheContext(md *metadata.StorageItem) {
 }
 
 type CacheContext interface {
-	Checksum(ctx context.Context, ref cache.Mountable, p string, followLinks bool) (digest.Digest, error)
-	ChecksumWildcard(ctx context.Context, ref cache.Mountable, p string, followLinks bool) (digest.Digest, error)
+	Checksum(ctx context.Context, ref cache.Mountable, p string, followLinks bool, s session.Group) (digest.Digest, error)
+	ChecksumWildcard(ctx context.Context, ref cache.Mountable, p string, followLinks bool, s session.Group) (digest.Digest, error)
 	HandleChange(kind fsutil.ChangeKind, p string, fi os.FileInfo, err error) error
 }
 
@@ -85,20 +86,20 @@ type cacheManager struct {
 	lruMu  sync.Mutex
 }
 
-func (cm *cacheManager) Checksum(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool) (digest.Digest, error) {
+func (cm *cacheManager) Checksum(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool, s session.Group) (digest.Digest, error) {
 	cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()), ref.IdentityMapping())
 	if err != nil {
 		return "", nil
 	}
-	return cc.Checksum(ctx, ref, p, followLinks)
+	return cc.Checksum(ctx, ref, p, followLinks, s)
 }
 
-func (cm *cacheManager) ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool) (digest.Digest, error) {
+func (cm *cacheManager) ChecksumWildcard(ctx context.Context, ref cache.ImmutableRef, p string, followLinks bool, s session.Group) (digest.Digest, error) {
 	cc, err := cm.GetCacheContext(ctx, ensureOriginMetadata(ref.Metadata()), ref.IdentityMapping())
 	if err != nil {
 		return "", nil
 	}
-	return cc.ChecksumWildcard(ctx, ref, p, followLinks)
+	return cc.ChecksumWildcard(ctx, ref, p, followLinks, s)
 }
 
 func (cm *cacheManager) GetCacheContext(ctx context.Context, md *metadata.StorageItem, idmap *idtools.IdentityMapping) (CacheContext, error) {
@@ -170,13 +171,14 @@ type mount struct {
 	mountable cache.Mountable
 	mountPath string
 	unmount   func() error
+	session   session.Group
 }
 
 func (m *mount) mount(ctx context.Context) (string, error) {
 	if m.mountPath != "" {
 		return m.mountPath, nil
 	}
-	mounts, err := m.mountable.Mount(ctx, true)
+	mounts, err := m.mountable.Mount(ctx, true, m.session)
 	if err != nil {
 		return "", err
 	}
@@ -380,13 +382,13 @@ func (cc *cacheContext) HandleChange(kind fsutil.ChangeKind, p string, fi os.Fil
 	return nil
 }
 
-func (cc *cacheContext) ChecksumWildcard(ctx context.Context, mountable cache.Mountable, p string, followLinks bool) (digest.Digest, error) {
-	m := &mount{mountable: mountable}
+func (cc *cacheContext) ChecksumWildcard(ctx context.Context, mountable cache.Mountable, p string, followLinks bool, s session.Group) (digest.Digest, error) {
+	m := &mount{mountable: mountable, session: s}
 	defer m.clean()
 
 	wildcards, err := cc.wildcards(ctx, m, p)
 	if err != nil {
-		return "", nil
+		return "", err
 	}
 
 	if followLinks {
@@ -413,13 +415,12 @@ func (cc *cacheContext) ChecksumWildcard(ctx context.Context, mountable cache.Mo
 			digester.Hash().Write([]byte(w.Record.Digest))
 		}
 		return digester.Digest(), nil
-	} else {
-		return wildcards[0].Record.Digest, nil
 	}
+	return wildcards[0].Record.Digest, nil
 }
 
-func (cc *cacheContext) Checksum(ctx context.Context, mountable cache.Mountable, p string, followLinks bool) (digest.Digest, error) {
-	m := &mount{mountable: mountable}
+func (cc *cacheContext) Checksum(ctx context.Context, mountable cache.Mountable, p string, followLinks bool, s session.Group) (digest.Digest, error) {
+	m := &mount{mountable: mountable, session: s}
 	defer m.clean()
 
 	return cc.checksumFollow(ctx, m, p, followLinks)
@@ -688,24 +689,24 @@ func (cc *cacheContext) needsScanFollow(root *iradix.Node, p string, linksWalked
 	if p == "/" {
 		p = ""
 	}
-	if v, ok := root.Get(convertPathToKey([]byte(p))); !ok {
+	v, ok := root.Get(convertPathToKey([]byte(p)))
+	if !ok {
 		if p == "" {
 			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)
+	}
+	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 false, nil
 }
@@ -875,12 +876,15 @@ func ensureOriginMetadata(md *metadata.StorageItem) *metadata.StorageItem {
 }
 
 var pool32K = sync.Pool{
-	New: func() interface{} { return make([]byte, 32*1024) }, // 32K
+	New: func() interface{} {
+		buf := make([]byte, 32*1024) // 32K
+		return &buf
+	},
 }
 
 func poolsCopy(dst io.Writer, src io.Reader) (written int64, err error) {
-	buf := pool32K.Get().([]byte)
-	written, err = io.CopyBuffer(dst, src, buf)
+	buf := pool32K.Get().(*[]byte)
+	written, err = io.CopyBuffer(dst, src, *buf)
 	pool32K.Put(buf)
 	return
 }

+ 5 - 3
vendor/github.com/moby/buildkit/cache/contenthash/filehash.go

@@ -40,20 +40,22 @@ func NewFileHash(path string, fi os.FileInfo) (hash.Hash, error) {
 }
 
 func NewFromStat(stat *fstypes.Stat) (hash.Hash, error) {
+	// Clear the socket bit since archive/tar.FileInfoHeader does not handle it
+	stat.Mode &^= uint32(os.ModeSocket)
+
 	fi := &statInfo{stat}
 	hdr, err := tar.FileInfoHeader(fi, stat.Linkname)
 	if err != nil {
 		return nil, err
 	}
 	hdr.Name = "" // note: empty name is different from current has in docker build. Name is added on recursive directory scan instead
-	hdr.Mode = int64(chmodWindowsTarEntry(os.FileMode(hdr.Mode)))
 	hdr.Devmajor = stat.Devmajor
 	hdr.Devminor = stat.Devminor
 
 	if len(stat.Xattrs) > 0 {
-		hdr.Xattrs = make(map[string]string, len(stat.Xattrs))
+		hdr.PAXRecords = make(map[string]string, len(stat.Xattrs))
 		for k, v := range stat.Xattrs {
-			hdr.Xattrs[k] = string(v)
+			hdr.PAXRecords["SCHILY.xattr."+k] = string(v)
 		}
 	}
 	// fmt.Printf("hdr: %#v\n", hdr)

+ 0 - 4
vendor/github.com/moby/buildkit/cache/contenthash/filehash_unix.go

@@ -12,10 +12,6 @@ import (
 	"golang.org/x/sys/unix"
 )
 
-func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
-	return perm
-}
-
 func setUnixOpt(path string, fi os.FileInfo, stat *fstypes.Stat) error {
 	s := fi.Sys().(*syscall.Stat_t)
 

+ 0 - 10
vendor/github.com/moby/buildkit/cache/contenthash/filehash_windows.go

@@ -8,16 +8,6 @@ import (
 	fstypes "github.com/tonistiigi/fsutil/types"
 )
 
-// chmodWindowsTarEntry is used to adjust the file permissions used in tar
-// header based on the platform the archival is done.
-func chmodWindowsTarEntry(perm os.FileMode) os.FileMode {
-	perm &= 0755
-	// Add the x bit: make everything +x from windows
-	perm |= 0111
-
-	return perm
-}
-
 func setUnixOpt(path string, fi os.FileInfo, stat *fstypes.Stat) error {
 	return nil
 }

+ 18 - 5
vendor/github.com/moby/buildkit/cache/contenthash/tarsum.go

@@ -36,11 +36,24 @@ func v0TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
 }
 
 func v1TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
+	pax := h.PAXRecords
+	if len(h.Xattrs) > 0 { //nolint deprecated
+		if pax == nil {
+			pax = map[string]string{}
+			for k, v := range h.Xattrs { //nolint deprecated
+				pax["SCHILY.xattr."+k] = v
+			}
+		}
+	}
+
 	// Get extended attributes.
-	xAttrKeys := make([]string, len(h.Xattrs))
-	for k := range h.Xattrs {
-		if k == "security.capability" || !strings.HasPrefix(k, "security.") && !strings.HasPrefix(k, "system.") {
-			xAttrKeys = append(xAttrKeys, k)
+	xAttrKeys := make([]string, len(h.PAXRecords))
+	for k := range pax {
+		if strings.HasPrefix(k, "SCHILY.xattr.") {
+			k = strings.TrimPrefix(k, "SCHILY.xattr.")
+			if k == "security.capability" || !strings.HasPrefix(k, "security.") && !strings.HasPrefix(k, "system.") {
+				xAttrKeys = append(xAttrKeys, k)
+			}
 		}
 	}
 	sort.Strings(xAttrKeys)
@@ -56,7 +69,7 @@ func v1TarHeaderSelect(h *tar.Header) (orderedHeaders [][2]string) {
 
 	// Finally, append the sorted xattrs.
 	for _, k := range xAttrKeys {
-		orderedHeaders = append(orderedHeaders, [2]string{k, h.Xattrs[k]})
+		orderedHeaders = append(orderedHeaders, [2]string{k, h.PAXRecords["SCHILY.xattr."+k]})
 	}
 
 	return

+ 128 - 30
vendor/github.com/moby/buildkit/cache/manager.go

@@ -8,6 +8,7 @@ import (
 
 	"github.com/containerd/containerd/content"
 	"github.com/containerd/containerd/diff"
+	"github.com/containerd/containerd/errdefs"
 	"github.com/containerd/containerd/filters"
 	"github.com/containerd/containerd/gc"
 	"github.com/containerd/containerd/leases"
@@ -15,7 +16,9 @@ import (
 	"github.com/moby/buildkit/cache/metadata"
 	"github.com/moby/buildkit/client"
 	"github.com/moby/buildkit/identity"
+	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/snapshot"
+	"github.com/moby/buildkit/util/flightcontrol"
 	digest "github.com/opencontainers/go-digest"
 	imagespecidentity "github.com/opencontainers/image-spec/identity"
 	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
@@ -38,14 +41,15 @@ type ManagerOpt struct {
 	PruneRefChecker ExternalRefCheckerFunc
 	GarbageCollect  func(ctx context.Context) (gc.Stats, error)
 	Applier         diff.Applier
+	Differ          diff.Comparer
 }
 
 type Accessor interface {
 	GetByBlob(ctx context.Context, desc ocispec.Descriptor, parent ImmutableRef, opts ...RefOption) (ImmutableRef, error)
 	Get(ctx context.Context, id string, opts ...RefOption) (ImmutableRef, error)
 
-	New(ctx context.Context, parent ImmutableRef, opts ...RefOption) (MutableRef, error)
-	GetMutable(ctx context.Context, id string) (MutableRef, error) // Rebase?
+	New(ctx context.Context, parent ImmutableRef, s session.Group, opts ...RefOption) (MutableRef, error)
+	GetMutable(ctx context.Context, id string, opts ...RefOption) (MutableRef, error) // Rebase?
 	IdentityMapping() *idtools.IdentityMapping
 	Metadata(string) *metadata.StorageItem
 }
@@ -74,6 +78,7 @@ type cacheManager struct {
 	md *metadata.Store
 
 	muPrune sync.Mutex // make sure parallel prune is not allowed so there will not be inconsistent results
+	unlazyG flightcontrol.Group
 }
 
 func NewManager(opt ManagerOpt) (Manager, error) {
@@ -92,7 +97,7 @@ func NewManager(opt ManagerOpt) (Manager, error) {
 	return cm, nil
 }
 
-func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor, parent ImmutableRef, opts ...RefOption) (ir ImmutableRef, err error) {
+func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor, parent ImmutableRef, opts ...RefOption) (ir ImmutableRef, rerr error) {
 	diffID, err := diffIDFromDescriptor(desc)
 	if err != nil {
 		return nil, err
@@ -100,9 +105,12 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 	chainID := diffID
 	blobChainID := imagespecidentity.ChainID([]digest.Digest{desc.Digest, diffID})
 
-	if desc.Digest != "" {
-		if _, err := cm.ContentStore.Info(ctx, desc.Digest); err != nil {
-			return nil, errors.Wrapf(err, "failed to get blob %s", desc.Digest)
+	descHandlers := descHandlersOf(opts...)
+	if desc.Digest != "" && (descHandlers == nil || descHandlers[desc.Digest] == nil) {
+		if _, err := cm.ContentStore.Info(ctx, desc.Digest); errors.Is(err, errdefs.ErrNotFound) {
+			return nil, NeedsRemoteProvidersError([]digest.Digest{desc.Digest})
+		} else if err != nil {
+			return nil, err
 		}
 	}
 
@@ -115,7 +123,8 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 		}
 		chainID = imagespecidentity.ChainID([]digest.Digest{pInfo.ChainID, chainID})
 		blobChainID = imagespecidentity.ChainID([]digest.Digest{pInfo.BlobChainID, blobChainID})
-		p2, err := cm.Get(ctx, parent.ID(), NoUpdateLastUsed)
+
+		p2, err := cm.Get(ctx, parent.ID(), NoUpdateLastUsed, descHandlers)
 		if err != nil {
 			return nil, err
 		}
@@ -128,7 +137,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 
 	releaseParent := false
 	defer func() {
-		if releaseParent || err != nil && p != nil {
+		if releaseParent || rerr != nil && p != nil {
 			p.Release(context.TODO())
 		}
 	}()
@@ -141,14 +150,17 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 		return nil, err
 	}
 
-	for _, si := range sis {
-		ref, err := cm.get(ctx, si.ID(), opts...)
+	if len(sis) > 0 {
+		ref, err := cm.get(ctx, sis[0].ID(), opts...)
 		if err != nil && !IsNotFound(err) {
-			return nil, errors.Wrapf(err, "failed to get record %s by blobchainid", si.ID())
+			return nil, errors.Wrapf(err, "failed to get record %s by blobchainid", sis[0].ID())
 		}
 		if p != nil {
 			releaseParent = true
 		}
+		if err := setImageRefMetadata(ref, opts...); err != nil {
+			return nil, errors.Wrapf(err, "failed to append image ref metadata to ref %s", ref.ID())
+		}
 		return ref, nil
 	}
 
@@ -158,13 +170,12 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 	}
 
 	var link ImmutableRef
-	for _, si := range sis {
-		ref, err := cm.get(ctx, si.ID(), opts...)
+	if len(sis) > 0 {
+		ref, err := cm.get(ctx, sis[0].ID(), opts...)
 		if err != nil && !IsNotFound(err) {
-			return nil, errors.Wrapf(err, "failed to get record %s by chainid", si.ID())
+			return nil, errors.Wrapf(err, "failed to get record %s by chainid", sis[0].ID())
 		}
 		link = ref
-		break
 	}
 
 	id := identity.NewID()
@@ -188,7 +199,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 	}
 
 	defer func() {
-		if err != nil {
+		if rerr != nil {
 			if err := cm.ManagerOpt.LeaseManager.Delete(context.TODO(), leases.Lease{
 				ID: l.ID,
 			}); err != nil {
@@ -227,6 +238,10 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 		return nil, err
 	}
 
+	if err := setImageRefMetadata(rec, opts...); err != nil {
+		return nil, errors.Wrapf(err, "failed to append image ref metadata to ref %s", rec.ID())
+	}
+
 	queueDiffID(rec.md, diffID.String())
 	queueBlob(rec.md, desc.Digest.String())
 	queueChainID(rec.md, chainID.String())
@@ -234,6 +249,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 	queueSnapshotID(rec.md, snapshotID)
 	queueBlobOnly(rec.md, blobOnly)
 	queueMediaType(rec.md, desc.MediaType)
+	queueBlobSize(rec.md, desc.Size)
 	queueCommitted(rec.md)
 
 	if err := rec.md.Commit(); err != nil {
@@ -242,7 +258,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispec.Descriptor,
 
 	cm.records[id] = rec
 
-	return rec.ref(true), nil
+	return rec.ref(true, descHandlers), nil
 }
 
 // init loads all snapshots from metadata state and tries to load the records
@@ -308,25 +324,52 @@ func (cm *cacheManager) get(ctx context.Context, id string, opts ...RefOption) (
 		}
 	}
 
+	descHandlers := descHandlersOf(opts...)
+
 	if rec.mutable {
 		if len(rec.refs) != 0 {
 			return nil, errors.Wrapf(ErrLocked, "%s is locked", id)
 		}
 		if rec.equalImmutable != nil {
-			return rec.equalImmutable.ref(triggerUpdate), nil
+			return rec.equalImmutable.ref(triggerUpdate, descHandlers), nil
 		}
-		return rec.mref(triggerUpdate).commit(ctx)
+		return rec.mref(triggerUpdate, descHandlers).commit(ctx)
 	}
 
-	return rec.ref(triggerUpdate), nil
+	return rec.ref(triggerUpdate, descHandlers), nil
 }
 
 // getRecord returns record for id. Requires manager lock.
 func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOption) (cr *cacheRecord, retErr error) {
+	checkLazyProviders := func(rec *cacheRecord) error {
+		missing := NeedsRemoteProvidersError(nil)
+		dhs := descHandlersOf(opts...)
+		for {
+			blob := digest.Digest(getBlob(rec.md))
+			if isLazy, err := rec.isLazy(ctx); err != nil {
+				return err
+			} else if isLazy && dhs[blob] == nil {
+				missing = append(missing, blob)
+			}
+
+			if rec.parent == nil {
+				break
+			}
+			rec = rec.parent.cacheRecord
+		}
+		if len(missing) > 0 {
+			return missing
+		}
+		return nil
+	}
+
 	if rec, ok := cm.records[id]; ok {
 		if rec.isDead() {
 			return nil, errors.Wrapf(errNotFound, "failed to get dead record %s", id)
 		}
+		if err := checkLazyProviders(rec); err != nil {
+			return nil, err
+		}
 		return rec, nil
 	}
 
@@ -343,11 +386,17 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt
 			}
 			return nil, err
 		}
+
+		// parent refs are possibly lazy so keep it hold the description handlers.
+		var dhs DescHandlers
+		if mutable.parent != nil {
+			dhs = mutable.parent.descHandlers
+		}
 		rec := &cacheRecord{
 			mu:           &sync.Mutex{},
 			cm:           cm,
 			refs:         make(map[ref]struct{}),
-			parent:       mutable.parentRef(false),
+			parent:       mutable.parentRef(false, dhs),
 			md:           md,
 			equalMutable: &mutableRef{cacheRecord: mutable},
 		}
@@ -393,25 +442,39 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt
 		return nil, err
 	}
 
+	if err := setImageRefMetadata(rec, opts...); err != nil {
+		return nil, errors.Wrapf(err, "failed to append image ref metadata to ref %s", rec.ID())
+	}
+
 	cm.records[id] = rec
+	if err := checkLazyProviders(rec); err != nil {
+		return nil, err
+	}
 	return rec, nil
 }
 
-func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, opts ...RefOption) (mr MutableRef, err error) {
+func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Group, opts ...RefOption) (mr MutableRef, err error) {
 	id := identity.NewID()
 
 	var parent *immutableRef
 	var parentID string
 	var parentSnapshotID string
 	if s != nil {
-		p, err := cm.Get(ctx, s.ID(), NoUpdateLastUsed)
-		if err != nil {
+		if _, ok := s.(*immutableRef); ok {
+			parent = s.Clone().(*immutableRef)
+		} else {
+			p, err := cm.Get(ctx, s.ID(), append(opts, NoUpdateLastUsed)...)
+			if err != nil {
+				return nil, err
+			}
+			parent = p.(*immutableRef)
+		}
+		if err := parent.Finalize(ctx, true); err != nil {
 			return nil, err
 		}
-		if err := p.Finalize(ctx, true); err != nil {
+		if err := parent.Extract(ctx, sess); err != nil {
 			return nil, err
 		}
-		parent = p.(*immutableRef)
 		parentSnapshotID = getSnapshotID(parent.md)
 		parentID = parent.ID()
 	}
@@ -469,18 +532,28 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, opts ...RefOpti
 		return nil, err
 	}
 
+	if err := setImageRefMetadata(rec, opts...); err != nil {
+		return nil, errors.Wrapf(err, "failed to append image ref metadata to ref %s", rec.ID())
+	}
+
 	cm.mu.Lock()
 	defer cm.mu.Unlock()
 
 	cm.records[id] = rec // TODO: save to db
 
-	return rec.mref(true), nil
+	// parent refs are possibly lazy so keep it hold the description handlers.
+	var dhs DescHandlers
+	if parent != nil {
+		dhs = parent.descHandlers
+	}
+	return rec.mref(true, dhs), nil
 }
-func (cm *cacheManager) GetMutable(ctx context.Context, id string) (MutableRef, error) {
+
+func (cm *cacheManager) GetMutable(ctx context.Context, id string, opts ...RefOption) (MutableRef, error) {
 	cm.mu.Lock()
 	defer cm.mu.Unlock()
 
-	rec, err := cm.getRecord(ctx, id)
+	rec, err := cm.getRecord(ctx, id, opts...)
 	if err != nil {
 		return nil, err
 	}
@@ -506,7 +579,7 @@ func (cm *cacheManager) GetMutable(ctx context.Context, id string) (MutableRef,
 		rec.equalImmutable = nil
 	}
 
-	return rec.mref(true), nil
+	return rec.mref(true, descHandlersOf(opts...)), nil
 }
 
 func (cm *cacheManager) Prune(ctx context.Context, ch chan client.UsageInfo, opts ...client.PruneInfo) error {
@@ -957,6 +1030,31 @@ func WithCreationTime(tm time.Time) RefOption {
 	}
 }
 
+// Need a separate type for imageRef because it needs to be called outside
+// initializeMetadata while still being a RefOption, so wrapping it in a
+// different type ensures initializeMetadata won't catch it too and duplicate
+// setting the metadata.
+type imageRefOption func(m withMetadata) error
+
+// WithImageRef appends the given imageRef to the cache ref's metadata
+func WithImageRef(imageRef string) RefOption {
+	return imageRefOption(func(m withMetadata) error {
+		return appendImageRef(m.Metadata(), imageRef)
+	})
+}
+
+func setImageRefMetadata(m withMetadata, opts ...RefOption) error {
+	md := m.Metadata()
+	for _, opt := range opts {
+		if fn, ok := opt.(imageRefOption); ok {
+			if err := fn(m); err != nil {
+				return err
+			}
+		}
+	}
+	return md.Commit()
+}
+
 func initializeMetadata(m withMetadata, parent string, opts ...RefOption) error {
 	md := m.Metadata()
 	if tm := GetCreatedAt(md); !tm.IsZero() {

+ 61 - 0
vendor/github.com/moby/buildkit/cache/metadata.go

@@ -28,6 +28,10 @@ const keyBlob = "cache.blob"
 const keySnapshot = "cache.snapshot"
 const keyBlobOnly = "cache.blobonly"
 const keyMediaType = "cache.mediatype"
+const keyImageRefs = "cache.imageRefs"
+
+// BlobSize is the packed blob size as specified in the oci descriptor
+const keyBlobSize = "cache.blobsize"
 
 const keyDeleted = "cache.deleted"
 
@@ -307,6 +311,63 @@ func getSize(si *metadata.StorageItem) int64 {
 	return size
 }
 
+func appendImageRef(si *metadata.StorageItem, s string) error {
+	return si.GetAndSetValue(keyImageRefs, func(v *metadata.Value) (*metadata.Value, error) {
+		var imageRefs []string
+		if v != nil {
+			if err := v.Unmarshal(&imageRefs); err != nil {
+				return nil, err
+			}
+		}
+		for _, existing := range imageRefs {
+			if existing == s {
+				return nil, metadata.ErrSkipSetValue
+			}
+		}
+		imageRefs = append(imageRefs, s)
+		v, err := metadata.NewValue(imageRefs)
+		if err != nil {
+			return nil, errors.Wrap(err, "failed to create imageRefs value")
+		}
+		return v, nil
+	})
+}
+
+func getImageRefs(si *metadata.StorageItem) []string {
+	v := si.Get(keyImageRefs)
+	if v == nil {
+		return nil
+	}
+	var refs []string
+	if err := v.Unmarshal(&refs); err != nil {
+		return nil
+	}
+	return refs
+}
+
+func queueBlobSize(si *metadata.StorageItem, s int64) error {
+	v, err := metadata.NewValue(s)
+	if err != nil {
+		return errors.Wrap(err, "failed to create blobsize value")
+	}
+	si.Queue(func(b *bolt.Bucket) error {
+		return si.SetValue(b, keyBlobSize, v)
+	})
+	return nil
+}
+
+func getBlobSize(si *metadata.StorageItem) int64 {
+	v := si.Get(keyBlobSize)
+	if v == nil {
+		return sizeUnknown
+	}
+	var size int64
+	if err := v.Unmarshal(&size); err != nil {
+		return sizeUnknown
+	}
+	return size
+}
+
 func getEqualMutable(si *metadata.StorageItem) string {
 	v := si.Get(keyEqualMutable)
 	if v == nil {

+ 16 - 0
vendor/github.com/moby/buildkit/cache/metadata/metadata.go

@@ -372,6 +372,22 @@ func (s *StorageItem) SetValue(b *bolt.Bucket, key string, v *Value) error {
 	return nil
 }
 
+var ErrSkipSetValue = errors.New("skip setting metadata value")
+
+func (s *StorageItem) GetAndSetValue(key string, fn func(*Value) (*Value, error)) error {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.Update(func(b *bolt.Bucket) error {
+		v, err := fn(s.values[key])
+		if errors.Is(err, ErrSkipSetValue) {
+			return nil
+		} else if err != nil {
+			return err
+		}
+		return s.SetValue(b, key, v)
+	})
+}
+
 type Value struct {
 	Value json.RawMessage `json:"value,omitempty"`
 	Index string          `json:"index,omitempty"`

+ 35 - 0
vendor/github.com/moby/buildkit/cache/opts.go

@@ -0,0 +1,35 @@
+package cache
+
+import (
+	"fmt"
+
+	"github.com/containerd/containerd/content"
+	"github.com/moby/buildkit/session"
+	"github.com/moby/buildkit/util/progress"
+	digest "github.com/opencontainers/go-digest"
+)
+
+type DescHandler struct {
+	Provider       func(session.Group) content.Provider
+	Progress       progress.Controller
+	SnapshotLabels map[string]string
+}
+
+type DescHandlers map[digest.Digest]*DescHandler
+
+func descHandlersOf(opts ...RefOption) DescHandlers {
+	for _, opt := range opts {
+		if opt, ok := opt.(DescHandlers); ok {
+			return opt
+		}
+	}
+	return nil
+}
+
+type DescHandlerKey digest.Digest
+
+type NeedsRemoteProvidersError []digest.Digest
+
+func (m NeedsRemoteProvidersError) Error() string {
+	return fmt.Sprintf("missing descriptor handlers for lazy blobs %+v", []digest.Digest(m))
+}

+ 228 - 103
vendor/github.com/moby/buildkit/cache/refs.go

@@ -14,14 +14,18 @@ import (
 	"github.com/docker/docker/pkg/idtools"
 	"github.com/moby/buildkit/cache/metadata"
 	"github.com/moby/buildkit/identity"
+	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/snapshot"
+	"github.com/moby/buildkit/solver"
+	"github.com/moby/buildkit/util/compression"
 	"github.com/moby/buildkit/util/flightcontrol"
 	"github.com/moby/buildkit/util/leaseutil"
+	"github.com/moby/buildkit/util/winlayers"
 	digest "github.com/opencontainers/go-digest"
-	imagespecidentity "github.com/opencontainers/image-spec/identity"
 	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
 	"github.com/pkg/errors"
 	"github.com/sirupsen/logrus"
+	"golang.org/x/sync/errgroup"
 )
 
 // Ref is a reference to cacheable objects.
@@ -41,8 +45,8 @@ type ImmutableRef interface {
 	Clone() ImmutableRef
 
 	Info() RefInfo
-	SetBlob(ctx context.Context, desc ocispec.Descriptor) error
-	Extract(ctx context.Context) error // +progress
+	Extract(ctx context.Context, s session.Group) error // +progress
+	GetRemote(ctx context.Context, createIfNeeded bool, compressionType compression.Type, s session.Group) (*solver.Remote, error)
 }
 
 type RefInfo struct {
@@ -61,7 +65,7 @@ type MutableRef interface {
 }
 
 type Mountable interface {
-	Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error)
+	Mount(ctx context.Context, readonly bool, s session.Group) (snapshot.Mountable, error)
 }
 
 type ref interface {
@@ -93,15 +97,23 @@ type cacheRecord struct {
 }
 
 // hold ref lock before calling
-func (cr *cacheRecord) ref(triggerLastUsed bool) *immutableRef {
-	ref := &immutableRef{cacheRecord: cr, triggerLastUsed: triggerLastUsed}
+func (cr *cacheRecord) ref(triggerLastUsed bool, descHandlers DescHandlers) *immutableRef {
+	ref := &immutableRef{
+		cacheRecord:     cr,
+		triggerLastUsed: triggerLastUsed,
+		descHandlers:    descHandlers,
+	}
 	cr.refs[ref] = struct{}{}
 	return ref
 }
 
 // hold ref lock before calling
-func (cr *cacheRecord) mref(triggerLastUsed bool) *mutableRef {
-	ref := &mutableRef{cacheRecord: cr, triggerLastUsed: triggerLastUsed}
+func (cr *cacheRecord) mref(triggerLastUsed bool, descHandlers DescHandlers) *mutableRef {
+	ref := &mutableRef{
+		cacheRecord:     cr,
+		triggerLastUsed: triggerLastUsed,
+		descHandlers:    descHandlers,
+	}
 	cr.refs[ref] = struct{}{}
 	return ref
 }
@@ -131,6 +143,22 @@ func (cr *cacheRecord) isDead() bool {
 	return cr.dead || (cr.equalImmutable != nil && cr.equalImmutable.dead) || (cr.equalMutable != nil && cr.equalMutable.dead)
 }
 
+func (cr *cacheRecord) isLazy(ctx context.Context) (bool, error) {
+	if !getBlobOnly(cr.md) {
+		return false, nil
+	}
+	dgst := getBlob(cr.md)
+	// special case for moby where there is no compressed blob (empty digest)
+	if dgst == "" {
+		return false, nil
+	}
+	_, err := cr.cm.ContentStore.Info(ctx, digest.Digest(dgst))
+	if errors.Is(err, errdefs.ErrNotFound) {
+		return true, nil
+	}
+	return false, err
+}
+
 func (cr *cacheRecord) IdentityMapping() *idtools.IdentityMapping {
 	return cr.cm.IdentityMapping()
 }
@@ -186,27 +214,18 @@ func (cr *cacheRecord) Size(ctx context.Context) (int64, error) {
 	return s.(int64), nil
 }
 
-func (cr *cacheRecord) Parent() ImmutableRef {
-	if p := cr.parentRef(true); p != nil { // avoid returning typed nil pointer
-		return p
-	}
-	return nil
-}
-
-func (cr *cacheRecord) parentRef(hidden bool) *immutableRef {
+func (cr *cacheRecord) parentRef(hidden bool, descHandlers DescHandlers) *immutableRef {
 	p := cr.parent
 	if p == nil {
 		return nil
 	}
 	p.mu.Lock()
 	defer p.mu.Unlock()
-	return p.ref(hidden)
+	return p.ref(hidden, descHandlers)
 }
 
-func (cr *cacheRecord) Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error) {
-	cr.mu.Lock()
-	defer cr.mu.Unlock()
-
+// must be called holding cacheRecord mu
+func (cr *cacheRecord) mount(ctx context.Context, readonly bool) (snapshot.Mountable, error) {
 	if cr.mutable {
 		m, err := cr.cm.Snapshotter.Mounts(ctx, getSnapshotID(cr.md))
 		if err != nil {
@@ -282,20 +301,29 @@ func (cr *cacheRecord) ID() string {
 type immutableRef struct {
 	*cacheRecord
 	triggerLastUsed bool
+	descHandlers    DescHandlers
 }
 
 type mutableRef struct {
 	*cacheRecord
 	triggerLastUsed bool
+	descHandlers    DescHandlers
 }
 
 func (sr *immutableRef) Clone() ImmutableRef {
 	sr.mu.Lock()
-	ref := sr.ref(false)
+	ref := sr.ref(false, sr.descHandlers)
 	sr.mu.Unlock()
 	return ref
 }
 
+func (sr *immutableRef) Parent() ImmutableRef {
+	if p := sr.parentRef(true, sr.descHandlers); p != nil { // avoid returning typed nil pointer
+		return p
+	}
+	return nil
+}
+
 func (sr *immutableRef) Info() RefInfo {
 	return RefInfo{
 		ChainID:     digest.Digest(getChainID(sr.md)),
@@ -308,25 +336,181 @@ func (sr *immutableRef) Info() RefInfo {
 	}
 }
 
-func (sr *immutableRef) Extract(ctx context.Context) error {
-	_, err := sr.sizeG.Do(ctx, sr.ID()+"-extract", func(ctx context.Context) (interface{}, error) {
+func (sr *immutableRef) ociDesc() (ocispec.Descriptor, error) {
+	desc := ocispec.Descriptor{
+		Digest:      digest.Digest(getBlob(sr.md)),
+		Size:        getBlobSize(sr.md),
+		MediaType:   getMediaType(sr.md),
+		Annotations: make(map[string]string),
+	}
+
+	diffID := getDiffID(sr.md)
+	if diffID != "" {
+		desc.Annotations["containerd.io/uncompressed"] = diffID
+	}
+
+	createdAt := GetCreatedAt(sr.md)
+	if !createdAt.IsZero() {
+		createdAt, err := createdAt.MarshalText()
+		if err != nil {
+			return ocispec.Descriptor{}, err
+		}
+		desc.Annotations["buildkit/createdat"] = string(createdAt)
+	}
+
+	return desc, nil
+}
+
+// order is from parent->child, sr will be at end of slice
+func (sr *immutableRef) parentRefChain() []*immutableRef {
+	var count int
+	for ref := sr; ref != nil; ref = ref.parent {
+		count++
+	}
+	refs := make([]*immutableRef, count)
+	for i, ref := count-1, sr; ref != nil; i, ref = i-1, ref.parent {
+		refs[i] = ref
+	}
+	return refs
+}
+
+func (sr *immutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (snapshot.Mountable, error) {
+	if err := sr.Extract(ctx, s); err != nil {
+		return nil, err
+	}
+
+	sr.mu.Lock()
+	defer sr.mu.Unlock()
+	return sr.mount(ctx, readonly)
+}
+
+func (sr *immutableRef) Extract(ctx context.Context, s session.Group) (rerr error) {
+	if !getBlobOnly(sr.md) {
+		return
+	}
+
+	ctx, done, err := leaseutil.WithLease(ctx, sr.cm.LeaseManager, leaseutil.MakeTemporary)
+	if err != nil {
+		return err
+	}
+	defer done(ctx)
+
+	if GetLayerType(sr) == "windows" {
+		ctx = winlayers.UseWindowsLayerMode(ctx)
+	}
+
+	if _, err := sr.prepareRemoteSnapshots(ctx, sr.descHandlers); err != nil {
+		return err
+	}
+
+	return sr.extract(ctx, sr.descHandlers, s)
+}
+
+func (sr *immutableRef) prepareRemoteSnapshots(ctx context.Context, dhs DescHandlers) (bool, error) {
+	ok, err := sr.sizeG.Do(ctx, sr.ID()+"-prepare-remote-snapshot", func(ctx context.Context) (_ interface{}, rerr error) {
 		snapshotID := getSnapshotID(sr.md)
 		if _, err := sr.cm.Snapshotter.Stat(ctx, snapshotID); err == nil {
-			queueBlobOnly(sr.md, false)
-			return nil, sr.md.Commit()
+			return true, nil
+		}
+		desc, err := sr.ociDesc()
+		if err != nil {
+			return false, err
+		}
+		dh := dhs[desc.Digest]
+		if dh == nil {
+			return false, nil
 		}
 
 		parentID := ""
 		if sr.parent != nil {
-			if err := sr.parent.Extract(ctx); err != nil {
-				return nil, err
+			if ok, err := sr.parent.prepareRemoteSnapshots(ctx, dhs); !ok {
+				return false, err
 			}
 			parentID = getSnapshotID(sr.parent.md)
 		}
-		info := sr.Info()
-		key := fmt.Sprintf("extract-%s %s", identity.NewID(), info.ChainID)
 
-		err := sr.cm.Snapshotter.Prepare(ctx, key, parentID)
+		// Hint labels to the snapshotter
+		labels := dh.SnapshotLabels
+		if labels == nil {
+			labels = make(map[string]string)
+		}
+		labels["containerd.io/snapshot.ref"] = snapshotID
+		opt := snapshots.WithLabels(labels)
+
+		// Try to preapre the remote snapshot
+		key := fmt.Sprintf("tmp-%s %s", identity.NewID(), sr.Info().ChainID)
+		if err = sr.cm.Snapshotter.Prepare(ctx, key, parentID, opt); err != nil {
+			if errdefs.IsAlreadyExists(err) {
+				// Check if the targeting snapshot ID has been prepared as a remote
+				// snapshot in the snapshotter.
+				if _, err := sr.cm.Snapshotter.Stat(ctx, snapshotID); err == nil {
+					// We can use this remote snapshot without unlazying.
+					// Try the next layer as well.
+					return true, nil
+				}
+			}
+		}
+
+		// This layer cannot be prepared without unlazying.
+		return false, nil
+	})
+	return ok.(bool), err
+}
+
+func (sr *immutableRef) extract(ctx context.Context, dhs DescHandlers, s session.Group) error {
+	_, err := sr.sizeG.Do(ctx, sr.ID()+"-extract", func(ctx context.Context) (_ interface{}, rerr error) {
+		snapshotID := getSnapshotID(sr.md)
+		if _, err := sr.cm.Snapshotter.Stat(ctx, snapshotID); err == nil {
+			return nil, nil
+		}
+
+		if sr.cm.Applier == nil {
+			return nil, errors.New("extract requires an applier")
+		}
+
+		eg, egctx := errgroup.WithContext(ctx)
+
+		parentID := ""
+		if sr.parent != nil {
+			eg.Go(func() error {
+				if err := sr.parent.extract(egctx, dhs, s); err != nil {
+					return err
+				}
+				parentID = getSnapshotID(sr.parent.md)
+				return nil
+			})
+		}
+
+		desc, err := sr.ociDesc()
+		if err != nil {
+			return nil, err
+		}
+		dh := dhs[desc.Digest]
+
+		eg.Go(func() error {
+			// unlazies if needed, otherwise a no-op
+			return lazyRefProvider{
+				ref:     sr,
+				desc:    desc,
+				dh:      dh,
+				session: s,
+			}.Unlazy(egctx)
+		})
+
+		if err := eg.Wait(); err != nil {
+			return nil, err
+		}
+
+		if dh != nil && dh.Progress != nil {
+			_, stopProgress := dh.Progress.Start(ctx)
+			defer stopProgress(rerr)
+			statusDone := dh.Progress.Status("extracting "+desc.Digest.String(), "extracting")
+			defer statusDone()
+		}
+
+		key := fmt.Sprintf("extract-%s %s", identity.NewID(), sr.Info().ChainID)
+
+		err = sr.cm.Snapshotter.Prepare(ctx, key, parentID)
 		if err != nil {
 			return nil, err
 		}
@@ -339,10 +523,7 @@ func (sr *immutableRef) Extract(ctx context.Context) error {
 		if err != nil {
 			return nil, err
 		}
-		_, err = sr.cm.Applier.Apply(ctx, ocispec.Descriptor{
-			Digest:    info.Blob,
-			MediaType: info.MediaType,
-		}, mounts)
+		_, err = sr.cm.Applier.Apply(ctx, desc, mounts)
 		if err != nil {
 			unmount()
 			return nil, err
@@ -357,6 +538,7 @@ func (sr *immutableRef) Extract(ctx context.Context) error {
 			}
 		}
 		queueBlobOnly(sr.md, false)
+		setSize(sr.md, sizeUnknown)
 		if err := sr.md.Commit(); err != nil {
 			return nil, err
 		}
@@ -365,65 +547,6 @@ func (sr *immutableRef) Extract(ctx context.Context) error {
 	return err
 }
 
-// SetBlob associates a blob with the cache record.
-// A lease must be held for the blob when calling this function
-// Caller should call Info() for knowing what current values are actually set
-func (sr *immutableRef) SetBlob(ctx context.Context, desc ocispec.Descriptor) error {
-	diffID, err := diffIDFromDescriptor(desc)
-	if err != nil {
-		return err
-	}
-	if _, err := sr.cm.ContentStore.Info(ctx, desc.Digest); err != nil {
-		return err
-	}
-
-	sr.mu.Lock()
-	defer sr.mu.Unlock()
-
-	if getChainID(sr.md) != "" {
-		return nil
-	}
-
-	if err := sr.finalize(ctx, true); err != nil {
-		return err
-	}
-
-	p := sr.parent
-	var parentChainID digest.Digest
-	var parentBlobChainID digest.Digest
-	if p != nil {
-		pInfo := p.Info()
-		if pInfo.ChainID == "" || pInfo.BlobChainID == "" {
-			return errors.Errorf("failed to set blob for reference with non-addressable parent")
-		}
-		parentChainID = pInfo.ChainID
-		parentBlobChainID = pInfo.BlobChainID
-	}
-
-	if err := sr.cm.LeaseManager.AddResource(ctx, leases.Lease{ID: sr.ID()}, leases.Resource{
-		ID:   desc.Digest.String(),
-		Type: "content",
-	}); err != nil {
-		return err
-	}
-
-	queueDiffID(sr.md, diffID.String())
-	queueBlob(sr.md, desc.Digest.String())
-	chainID := diffID
-	blobChainID := imagespecidentity.ChainID([]digest.Digest{desc.Digest, diffID})
-	if parentChainID != "" {
-		chainID = imagespecidentity.ChainID([]digest.Digest{parentChainID, chainID})
-		blobChainID = imagespecidentity.ChainID([]digest.Digest{parentBlobChainID, blobChainID})
-	}
-	queueChainID(sr.md, chainID.String())
-	queueBlobChainID(sr.md, blobChainID.String())
-	queueMediaType(sr.md, desc.MediaType)
-	if err := sr.md.Commit(); err != nil {
-		return err
-	}
-	return nil
-}
-
 func (sr *immutableRef) Release(ctx context.Context) error {
 	sr.cm.mu.Lock()
 	defer sr.cm.mu.Unlock()
@@ -555,7 +678,7 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) {
 	rec := &cacheRecord{
 		mu:           sr.mu,
 		cm:           sr.cm,
-		parent:       sr.parentRef(false),
+		parent:       sr.parentRef(false, sr.descHandlers),
 		equalMutable: sr,
 		refs:         make(map[ref]struct{}),
 		md:           md,
@@ -588,13 +711,16 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) {
 		return nil, err
 	}
 
-	ref := rec.ref(true)
+	ref := rec.ref(true, sr.descHandlers)
 	sr.equalImmutable = ref
 	return ref, nil
 }
 
-func (sr *mutableRef) updatesLastUsed() bool {
-	return sr.triggerLastUsed
+func (sr *mutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (snapshot.Mountable, error) {
+	sr.mu.Lock()
+	defer sr.mu.Unlock()
+
+	return sr.mount(ctx, readonly)
 }
 
 func (sr *mutableRef) Commit(ctx context.Context) (ImmutableRef, error) {
@@ -633,11 +759,10 @@ func (sr *mutableRef) release(ctx context.Context) error {
 			}
 		}
 		return sr.remove(ctx, true)
-	} else {
-		if sr.updateLastUsed() {
-			updateLastUsed(sr.md)
-			sr.triggerLastUsed = false
-		}
+	}
+	if sr.updateLastUsed() {
+		updateLastUsed(sr.md)
+		sr.triggerLastUsed = false
 	}
 	return nil
 }

+ 204 - 0
vendor/github.com/moby/buildkit/cache/remote.go

@@ -0,0 +1,204 @@
+package cache
+
+import (
+	"context"
+	"fmt"
+	"net/url"
+	"strings"
+
+	"github.com/containerd/containerd/content"
+	"github.com/containerd/containerd/errdefs"
+	"github.com/containerd/containerd/reference"
+	"github.com/moby/buildkit/session"
+	"github.com/moby/buildkit/solver"
+	"github.com/moby/buildkit/util/compression"
+	"github.com/moby/buildkit/util/contentutil"
+	"github.com/moby/buildkit/util/leaseutil"
+	"github.com/moby/buildkit/util/pull/pullprogress"
+	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
+	"github.com/pkg/errors"
+	"golang.org/x/sync/errgroup"
+)
+
+type Unlazier interface {
+	Unlazy(ctx context.Context) error
+}
+
+// GetRemote gets a *solver.Remote from content store for this ref (potentially pulling lazily).
+// Note: Use WorkerRef.GetRemote instead as moby integration requires custom GetRemote implementation.
+func (sr *immutableRef) GetRemote(ctx context.Context, createIfNeeded bool, compressionType compression.Type, s session.Group) (*solver.Remote, error) {
+	ctx, done, err := leaseutil.WithLease(ctx, sr.cm.LeaseManager, leaseutil.MakeTemporary)
+	if err != nil {
+		return nil, err
+	}
+	defer done(ctx)
+
+	err = sr.computeBlobChain(ctx, createIfNeeded, compressionType, s)
+	if err != nil {
+		return nil, err
+	}
+
+	mprovider := &lazyMultiProvider{mprovider: contentutil.NewMultiProvider(nil)}
+	remote := &solver.Remote{
+		Provider: mprovider,
+	}
+
+	for _, ref := range sr.parentRefChain() {
+		desc, err := ref.ociDesc()
+		if err != nil {
+			return nil, err
+		}
+
+		// NOTE: The media type might be missing for some migrated ones
+		// from before lease based storage. If so, we should detect
+		// the media type from blob data.
+		//
+		// Discussion: https://github.com/moby/buildkit/pull/1277#discussion_r352795429
+		if desc.MediaType == "" {
+			desc.MediaType, err = compression.DetectLayerMediaType(ctx, sr.cm.ContentStore, desc.Digest, false)
+			if err != nil {
+				return nil, err
+			}
+		}
+
+		// update distribution source annotation for lazy-refs (non-lazy refs
+		// will already have their dsl stored in the content store, which is
+		// used by the push handlers)
+		if isLazy, err := ref.isLazy(ctx); err != nil {
+			return nil, err
+		} else if isLazy {
+			imageRefs := getImageRefs(ref.md)
+			for _, imageRef := range imageRefs {
+				refspec, err := reference.Parse(imageRef)
+				if err != nil {
+					return nil, err
+				}
+
+				u, err := url.Parse("dummy://" + refspec.Locator)
+				if err != nil {
+					return nil, err
+				}
+
+				source, repo := u.Hostname(), strings.TrimPrefix(u.Path, "/")
+				if desc.Annotations == nil {
+					desc.Annotations = make(map[string]string)
+				}
+				dslKey := fmt.Sprintf("%s.%s", "containerd.io/distribution.source", source)
+
+				var existingRepos []string
+				if existings, ok := desc.Annotations[dslKey]; ok {
+					existingRepos = strings.Split(existings, ",")
+				}
+				addNewRepo := true
+				for _, existing := range existingRepos {
+					if existing == repo {
+						addNewRepo = false
+						break
+					}
+				}
+				if addNewRepo {
+					existingRepos = append(existingRepos, repo)
+				}
+				desc.Annotations[dslKey] = strings.Join(existingRepos, ",")
+			}
+		}
+
+		remote.Descriptors = append(remote.Descriptors, desc)
+		mprovider.Add(lazyRefProvider{
+			ref:     ref,
+			desc:    desc,
+			dh:      sr.descHandlers[desc.Digest],
+			session: s,
+		})
+	}
+	return remote, nil
+}
+
+type lazyMultiProvider struct {
+	mprovider *contentutil.MultiProvider
+	plist     []lazyRefProvider
+}
+
+func (mp *lazyMultiProvider) Add(p lazyRefProvider) {
+	mp.mprovider.Add(p.desc.Digest, p)
+	mp.plist = append(mp.plist, p)
+}
+
+func (mp *lazyMultiProvider) ReaderAt(ctx context.Context, desc ocispec.Descriptor) (content.ReaderAt, error) {
+	return mp.mprovider.ReaderAt(ctx, desc)
+}
+
+func (mp *lazyMultiProvider) Unlazy(ctx context.Context) error {
+	eg, egctx := errgroup.WithContext(ctx)
+	for _, p := range mp.plist {
+		p := p
+		eg.Go(func() error {
+			return p.Unlazy(egctx)
+		})
+	}
+	return eg.Wait()
+}
+
+type lazyRefProvider struct {
+	ref     *immutableRef
+	desc    ocispec.Descriptor
+	dh      *DescHandler
+	session session.Group
+}
+
+func (p lazyRefProvider) ReaderAt(ctx context.Context, desc ocispec.Descriptor) (content.ReaderAt, error) {
+	if desc.Digest != p.desc.Digest {
+		return nil, errdefs.ErrNotFound
+	}
+	if err := p.Unlazy(ctx); err != nil {
+		return nil, err
+	}
+	return p.ref.cm.ContentStore.ReaderAt(ctx, desc)
+}
+
+func (p lazyRefProvider) Unlazy(ctx context.Context) error {
+	_, err := p.ref.cm.unlazyG.Do(ctx, string(p.desc.Digest), func(ctx context.Context) (_ interface{}, rerr error) {
+		if isLazy, err := p.ref.isLazy(ctx); err != nil {
+			return nil, err
+		} else if !isLazy {
+			return nil, nil
+		}
+
+		if p.dh == nil {
+			// shouldn't happen, if you have a lazy immutable ref it already should be validated
+			// that descriptor handlers exist for it
+			return nil, errors.New("unexpected nil descriptor handler")
+		}
+
+		if p.dh.Progress != nil {
+			var stopProgress func(error)
+			ctx, stopProgress = p.dh.Progress.Start(ctx)
+			defer stopProgress(rerr)
+		}
+
+		// For now, just pull down the whole content and then return a ReaderAt from the local content
+		// store. If efficient partial reads are desired in the future, something more like a "tee"
+		// that caches remote partial reads to a local store may need to replace this.
+		err := contentutil.Copy(ctx, p.ref.cm.ContentStore, &pullprogress.ProviderWithProgress{
+			Provider: p.dh.Provider(p.session),
+			Manager:  p.ref.cm.ContentStore,
+		}, p.desc)
+		if err != nil {
+			return nil, err
+		}
+
+		if imageRefs := getImageRefs(p.ref.md); len(imageRefs) > 0 {
+			// just use the first image ref, it's arbitrary
+			imageRef := imageRefs[0]
+			if GetDescription(p.ref.md) == "" {
+				queueDescription(p.ref.md, "pulled from "+imageRef)
+				err := p.ref.md.Commit()
+				if err != nil {
+					return nil, err
+				}
+			}
+		}
+		return nil, err
+	})
+	return err
+}

+ 13 - 10
vendor/github.com/moby/buildkit/cache/remotecache/export.go

@@ -12,6 +12,7 @@ import (
 	v1 "github.com/moby/buildkit/cache/remotecache/v1"
 	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/solver"
+	"github.com/moby/buildkit/util/compression"
 	"github.com/moby/buildkit/util/contentutil"
 	"github.com/moby/buildkit/util/progress"
 	digest "github.com/opencontainers/go-digest"
@@ -55,20 +56,17 @@ type contentCacheExporter struct {
 	solver.CacheExporterTarget
 	chains   *v1.CacheChains
 	ingester content.Ingester
+	oci      bool
 }
 
-func NewExporter(ingester content.Ingester) Exporter {
+func NewExporter(ingester content.Ingester, oci bool) Exporter {
 	cc := v1.NewCacheChains()
-	return &contentCacheExporter{CacheExporterTarget: cc, chains: cc, ingester: ingester}
+	return &contentCacheExporter{CacheExporterTarget: cc, chains: cc, ingester: ingester, oci: oci}
 }
 
 func (ce *contentCacheExporter) Finalize(ctx context.Context) (map[string]string, error) {
-	return export(ctx, ce.ingester, ce.chains)
-}
-
-func export(ctx context.Context, ingester content.Ingester, cc *v1.CacheChains) (map[string]string, error) {
 	res := make(map[string]string)
-	config, descs, err := cc.Marshal()
+	config, descs, err := ce.chains.Marshal()
 	if err != nil {
 		return nil, err
 	}
@@ -86,6 +84,9 @@ func export(ctx context.Context, ingester content.Ingester, cc *v1.CacheChains)
 	var mfst manifestList
 	mfst.SchemaVersion = 2
 	mfst.MediaType = images.MediaTypeDockerSchema2ManifestList
+	if ce.oci {
+		mfst.MediaType = ocispec.MediaTypeImageIndex
+	}
 
 	for _, l := range config.Layers {
 		dgstPair, ok := descs[l.Blob]
@@ -93,13 +94,15 @@ func export(ctx context.Context, ingester content.Ingester, cc *v1.CacheChains)
 			return nil, errors.Errorf("missing blob %s", l.Blob)
 		}
 		layerDone := oneOffProgress(ctx, fmt.Sprintf("writing layer %s", l.Blob))
-		if err := contentutil.Copy(ctx, ingester, dgstPair.Provider, dgstPair.Descriptor); err != nil {
+		if err := contentutil.Copy(ctx, ce.ingester, dgstPair.Provider, dgstPair.Descriptor); err != nil {
 			return nil, layerDone(errors.Wrap(err, "error writing layer blob"))
 		}
 		layerDone(nil)
 		mfst.Manifests = append(mfst.Manifests, dgstPair.Descriptor)
 	}
 
+	mfst.Manifests = compression.ConvertAllLayerMediaTypes(ce.oci, mfst.Manifests...)
+
 	dt, err := json.Marshal(config)
 	if err != nil {
 		return nil, err
@@ -111,7 +114,7 @@ func export(ctx context.Context, ingester content.Ingester, cc *v1.CacheChains)
 		MediaType: v1.CacheConfigMediaTypeV0,
 	}
 	configDone := oneOffProgress(ctx, fmt.Sprintf("writing config %s", dgst))
-	if err := content.WriteBlob(ctx, ingester, dgst.String(), bytes.NewReader(dt), desc); err != nil {
+	if err := content.WriteBlob(ctx, ce.ingester, dgst.String(), bytes.NewReader(dt), desc); err != nil {
 		return nil, configDone(errors.Wrap(err, "error writing config blob"))
 	}
 	configDone(nil)
@@ -130,7 +133,7 @@ func export(ctx context.Context, ingester content.Ingester, cc *v1.CacheChains)
 		MediaType: mfst.MediaType,
 	}
 	mfstDone := oneOffProgress(ctx, fmt.Sprintf("writing manifest %s", dgst))
-	if err := content.WriteBlob(ctx, ingester, dgst.String(), bytes.NewReader(dt), desc); err != nil {
+	if err := content.WriteBlob(ctx, ce.ingester, dgst.String(), bytes.NewReader(dt), desc); err != nil {
 		return nil, mfstDone(errors.Wrap(err, "error writing manifest blob"))
 	}
 	descJSON, err := json.Marshal(desc)

+ 12 - 2
vendor/github.com/moby/buildkit/cache/remotecache/local/local.go

@@ -2,6 +2,7 @@ package local
 
 import (
 	"context"
+	"strconv"
 	"time"
 
 	"github.com/containerd/containerd/content"
@@ -17,6 +18,7 @@ const (
 	attrDigest           = "digest"
 	attrSrc              = "src"
 	attrDest             = "dest"
+	attrOCIMediatypes    = "oci-mediatypes"
 	contentStoreIDPrefix = "local:"
 )
 
@@ -27,12 +29,20 @@ func ResolveCacheExporterFunc(sm *session.Manager) remotecache.ResolveCacheExpor
 		if store == "" {
 			return nil, errors.New("local cache exporter requires dest")
 		}
+		ociMediatypes := true
+		if v, ok := attrs[attrOCIMediatypes]; ok {
+			b, err := strconv.ParseBool(v)
+			if err != nil {
+				return nil, errors.Wrapf(err, "failed to parse %s", attrOCIMediatypes)
+			}
+			ociMediatypes = b
+		}
 		csID := contentStoreIDPrefix + store
 		cs, err := getContentStore(ctx, sm, g, csID)
 		if err != nil {
 			return nil, err
 		}
-		return remotecache.NewExporter(cs), nil
+		return remotecache.NewExporter(cs, ociMediatypes), nil
 	}
 }
 
@@ -76,7 +86,7 @@ func getContentStore(ctx context.Context, sm *session.Manager, g session.Group,
 	timeoutCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
 	defer cancel()
 
-	caller, err := sm.Get(timeoutCtx, sessionID)
+	caller, err := sm.Get(timeoutCtx, sessionID, false)
 	if err != nil {
 		return nil, err
 	}

+ 14 - 4
vendor/github.com/moby/buildkit/cache/remotecache/registry/registry.go

@@ -2,6 +2,7 @@ package registry
 
 import (
 	"context"
+	"strconv"
 
 	"github.com/containerd/containerd/content"
 	"github.com/containerd/containerd/remotes/docker"
@@ -28,7 +29,8 @@ func canonicalizeRef(rawRef string) (string, error) {
 }
 
 const (
-	attrRef = "ref"
+	attrRef           = "ref"
+	attrOCIMediatypes = "oci-mediatypes"
 )
 
 func ResolveCacheExporterFunc(sm *session.Manager, hosts docker.RegistryHosts) remotecache.ResolveCacheExporterFunc {
@@ -37,12 +39,20 @@ func ResolveCacheExporterFunc(sm *session.Manager, hosts docker.RegistryHosts) r
 		if err != nil {
 			return nil, err
 		}
-		remote := resolver.New(hosts, resolver.NewSessionAuthenticator(sm, g))
+		ociMediatypes := true
+		if v, ok := attrs[attrOCIMediatypes]; ok {
+			b, err := strconv.ParseBool(v)
+			if err != nil {
+				return nil, errors.Wrapf(err, "failed to parse %s", attrOCIMediatypes)
+			}
+			ociMediatypes = b
+		}
+		remote := resolver.DefaultPool.GetResolver(hosts, ref, "push", sm, g)
 		pusher, err := remote.Pusher(ctx, ref)
 		if err != nil {
 			return nil, err
 		}
-		return remotecache.NewExporter(contentutil.FromPusher(pusher)), nil
+		return remotecache.NewExporter(contentutil.FromPusher(pusher), ociMediatypes), nil
 	}
 }
 
@@ -52,7 +62,7 @@ func ResolveCacheImporterFunc(sm *session.Manager, cs content.Store, hosts docke
 		if err != nil {
 			return nil, specs.Descriptor{}, err
 		}
-		remote := resolver.New(hosts, resolver.NewSessionAuthenticator(sm, g))
+		remote := resolver.DefaultPool.GetResolver(hosts, ref, "pull", sm, g)
 		xref, desc, err := remote.Resolve(ctx, ref)
 		if err != nil {
 			return nil, specs.Descriptor{}, err

+ 2 - 1
vendor/github.com/moby/buildkit/cache/remotecache/v1/cachestorage.go

@@ -5,6 +5,7 @@ import (
 	"time"
 
 	"github.com/moby/buildkit/identity"
+	"github.com/moby/buildkit/session"
 	"github.com/moby/buildkit/solver"
 	"github.com/moby/buildkit/worker"
 	digest "github.com/opencontainers/go-digest"
@@ -260,7 +261,7 @@ func (cs *cacheResultStorage) Load(ctx context.Context, res solver.CacheResult)
 	return worker.NewWorkerRefResult(ref, cs.w), nil
 }
 
-func (cs *cacheResultStorage) LoadRemote(ctx context.Context, res solver.CacheResult) (*solver.Remote, error) {
+func (cs *cacheResultStorage) LoadRemote(ctx context.Context, res solver.CacheResult, _ session.Group) (*solver.Remote, error) {
 	if r := cs.byResultID(res.ID); r != nil && r.result != nil {
 		return r.result, nil
 	}

+ 39 - 2
vendor/github.com/moby/buildkit/cache/remotecache/v1/chains.go

@@ -23,7 +23,7 @@ func (c *CacheChains) Add(dgst digest.Digest) solver.CacheExporterRecord {
 	if strings.HasPrefix(dgst.String(), "random:") {
 		return &nopRecord{}
 	}
-	it := &item{c: c, dgst: dgst}
+	it := &item{c: c, dgst: dgst, backlinks: map[*item]struct{}{}}
 	c.items = append(c.items, it)
 	return it
 }
@@ -44,6 +44,17 @@ func (c *CacheChains) normalize() error {
 		byKey: map[digest.Digest]*item{},
 	}
 
+	validated := make([]*item, 0, len(c.items))
+	for _, it := range c.items {
+		it.validate()
+	}
+	for _, it := range c.items {
+		if !it.invalid {
+			validated = append(validated, it)
+		}
+	}
+	c.items = validated
+
 	for _, it := range c.items {
 		_, err := normalizeItem(it, st)
 		if err != nil {
@@ -99,7 +110,9 @@ type item struct {
 	result     *solver.Remote
 	resultTime time.Time
 
-	links []map[link]struct{}
+	links     []map[link]struct{}
+	backlinks map[*item]struct{}
+	invalid   bool
 }
 
 type link struct {
@@ -126,6 +139,30 @@ func (c *item) LinkFrom(rec solver.CacheExporterRecord, index int, selector stri
 	}
 
 	c.links[index][link{src: src, selector: selector}] = struct{}{}
+	src.backlinks[c] = struct{}{}
+}
+
+func (c *item) validate() {
+	for _, m := range c.links {
+		if len(m) == 0 {
+			c.invalid = true
+			for bl := range c.backlinks {
+				changed := false
+				for _, m := range bl.links {
+					for l := range m {
+						if l.src == c {
+							delete(m, l)
+							changed = true
+						}
+					}
+				}
+				if changed {
+					bl.validate()
+				}
+			}
+			return
+		}
+	}
 }
 
 func (c *item) walkAllResults(fn func(i *item) error, visited map[*item]struct{}) error {

+ 5 - 6
vendor/github.com/moby/buildkit/cache/remotecache/v1/utils.go

@@ -4,10 +4,9 @@ import (
 	"fmt"
 	"sort"
 
-	"github.com/containerd/containerd/content"
+	"github.com/moby/buildkit/exporter/containerimage/exptypes"
 	"github.com/moby/buildkit/solver"
 	digest "github.com/opencontainers/go-digest"
-	ocispec "github.com/opencontainers/image-spec/specs-go/v1"
 	"github.com/pkg/errors"
 )
 
@@ -230,10 +229,6 @@ func marshalRemote(r *solver.Remote, state *marshalState) string {
 	if len(r.Descriptors) == 0 {
 		return ""
 	}
-	type Remote struct {
-		Descriptors []ocispec.Descriptor
-		Provider    content.Provider
-	}
 	var parentID string
 	if len(r.Descriptors) > 1 {
 		r2 := &solver.Remote{
@@ -244,6 +239,10 @@ func marshalRemote(r *solver.Remote, state *marshalState) string {
 	}
 	desc := r.Descriptors[len(r.Descriptors)-1]
 
+	if desc.Digest == exptypes.EmptyGZLayer {
+		return parentID
+	}
+
 	state.descriptors[desc.Digest] = DescriptorProviderPair{
 		Descriptor: desc,
 		Provider:   r.Provider,

+ 7 - 13
vendor/github.com/moby/buildkit/cache/util/fsutil.go

@@ -8,7 +8,6 @@ import (
 	"path/filepath"
 
 	"github.com/containerd/continuity/fs"
-	"github.com/moby/buildkit/cache"
 	"github.com/moby/buildkit/snapshot"
 	"github.com/pkg/errors"
 	"github.com/tonistiigi/fsutil"
@@ -25,12 +24,7 @@ type FileRange struct {
 	Length int
 }
 
-func withMount(ctx context.Context, ref cache.ImmutableRef, cb func(string) error) error {
-	mount, err := ref.Mount(ctx, true)
-	if err != nil {
-		return err
-	}
-
+func withMount(ctx context.Context, mount snapshot.Mountable, cb func(string) error) error {
 	lm := snapshot.LocalMounter(mount)
 
 	root, err := lm.Mount()
@@ -55,10 +49,10 @@ func withMount(ctx context.Context, ref cache.ImmutableRef, cb func(string) erro
 	return nil
 }
 
-func ReadFile(ctx context.Context, ref cache.ImmutableRef, req ReadRequest) ([]byte, error) {
+func ReadFile(ctx context.Context, mount snapshot.Mountable, req ReadRequest) ([]byte, error) {
 	var dt []byte
 
-	err := withMount(ctx, ref, func(root string) error {
+	err := withMount(ctx, mount, func(root string) error {
 		fp, err := fs.RootPath(root, req.Filename)
 		if err != nil {
 			return errors.WithStack(err)
@@ -90,7 +84,7 @@ type ReadDirRequest struct {
 	IncludePattern string
 }
 
-func ReadDir(ctx context.Context, ref cache.ImmutableRef, req ReadDirRequest) ([]*fstypes.Stat, error) {
+func ReadDir(ctx context.Context, mount snapshot.Mountable, req ReadDirRequest) ([]*fstypes.Stat, error) {
 	var (
 		rd []*fstypes.Stat
 		wo fsutil.WalkOpt
@@ -98,7 +92,7 @@ func ReadDir(ctx context.Context, ref cache.ImmutableRef, req ReadDirRequest) ([
 	if req.IncludePattern != "" {
 		wo.IncludePatterns = append(wo.IncludePatterns, req.IncludePattern)
 	}
-	err := withMount(ctx, ref, func(root string) error {
+	err := withMount(ctx, mount, func(root string) error {
 		fp, err := fs.RootPath(root, req.Path)
 		if err != nil {
 			return errors.WithStack(err)
@@ -123,9 +117,9 @@ func ReadDir(ctx context.Context, ref cache.ImmutableRef, req ReadDirRequest) ([
 	return rd, err
 }
 
-func StatFile(ctx context.Context, ref cache.ImmutableRef, path string) (*fstypes.Stat, error) {
+func StatFile(ctx context.Context, mount snapshot.Mountable, path string) (*fstypes.Stat, error) {
 	var st *fstypes.Stat
-	err := withMount(ctx, ref, func(root string) error {
+	err := withMount(ctx, mount, func(root string) error {
 		fp, err := fs.RootPath(root, path)
 		if err != nil {
 			return errors.WithStack(err)

+ 43 - 3
vendor/github.com/moby/buildkit/client/build.go

@@ -45,11 +45,14 @@ func (c *Client) Build(ctx context.Context, opt SolveOpt, product string, buildF
 	}
 
 	cb := func(ref string, s *session.Session) error {
-		g, err := grpcclient.New(ctx, feOpts, s.ID(), product, c.gatewayClientForBuild(ref), gworkers)
+		gwClient := c.gatewayClientForBuild(ref)
+		g, err := grpcclient.New(ctx, feOpts, s.ID(), product, gwClient, gworkers)
 		if err != nil {
 			return err
 		}
 
+		gwClient.caps = g.BuildOpts().Caps
+
 		if err := g.Run(ctx, buildFunc); err != nil {
 			return errors.Wrap(err, "failed to run Build function")
 		}
@@ -59,14 +62,18 @@ func (c *Client) Build(ctx context.Context, opt SolveOpt, product string, buildF
 	return c.solve(ctx, nil, cb, opt, statusChan)
 }
 
-func (c *Client) gatewayClientForBuild(buildid string) gatewayapi.LLBBridgeClient {
+func (c *Client) gatewayClientForBuild(buildid string) *gatewayClientForBuild {
 	g := gatewayapi.NewLLBBridgeClient(c.conn)
-	return &gatewayClientForBuild{g, buildid}
+	return &gatewayClientForBuild{
+		gateway: g,
+		buildID: buildid,
+	}
 }
 
 type gatewayClientForBuild struct {
 	gateway gatewayapi.LLBBridgeClient
 	buildID string
+	caps    apicaps.CapSet
 }
 
 func (g *gatewayClientForBuild) ResolveImageConfig(ctx context.Context, in *gatewayapi.ResolveImageConfigRequest, opts ...grpc.CallOption) (*gatewayapi.ResolveImageConfigResponse, error) {
@@ -85,11 +92,17 @@ func (g *gatewayClientForBuild) ReadFile(ctx context.Context, in *gatewayapi.Rea
 }
 
 func (g *gatewayClientForBuild) ReadDir(ctx context.Context, in *gatewayapi.ReadDirRequest, opts ...grpc.CallOption) (*gatewayapi.ReadDirResponse, error) {
+	if err := g.caps.Supports(gatewayapi.CapReadDir); err != nil {
+		return nil, err
+	}
 	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
 	return g.gateway.ReadDir(ctx, in, opts...)
 }
 
 func (g *gatewayClientForBuild) StatFile(ctx context.Context, in *gatewayapi.StatFileRequest, opts ...grpc.CallOption) (*gatewayapi.StatFileResponse, error) {
+	if err := g.caps.Supports(gatewayapi.CapStatFile); err != nil {
+		return nil, err
+	}
 	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
 	return g.gateway.StatFile(ctx, in, opts...)
 }
@@ -105,6 +118,33 @@ func (g *gatewayClientForBuild) Return(ctx context.Context, in *gatewayapi.Retur
 }
 
 func (g *gatewayClientForBuild) Inputs(ctx context.Context, in *gatewayapi.InputsRequest, opts ...grpc.CallOption) (*gatewayapi.InputsResponse, error) {
+	if err := g.caps.Supports(gatewayapi.CapFrontendInputs); err != nil {
+		return nil, err
+	}
 	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
 	return g.gateway.Inputs(ctx, in, opts...)
 }
+
+func (g *gatewayClientForBuild) NewContainer(ctx context.Context, in *gatewayapi.NewContainerRequest, opts ...grpc.CallOption) (*gatewayapi.NewContainerResponse, error) {
+	if err := g.caps.Supports(gatewayapi.CapGatewayExec); err != nil {
+		return nil, err
+	}
+	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
+	return g.gateway.NewContainer(ctx, in, opts...)
+}
+
+func (g *gatewayClientForBuild) ReleaseContainer(ctx context.Context, in *gatewayapi.ReleaseContainerRequest, opts ...grpc.CallOption) (*gatewayapi.ReleaseContainerResponse, error) {
+	if err := g.caps.Supports(gatewayapi.CapGatewayExec); err != nil {
+		return nil, err
+	}
+	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
+	return g.gateway.ReleaseContainer(ctx, in, opts...)
+}
+
+func (g *gatewayClientForBuild) ExecProcess(ctx context.Context, opts ...grpc.CallOption) (gatewayapi.LLBBridge_ExecProcessClient, error) {
+	if err := g.caps.Supports(gatewayapi.CapGatewayExec); err != nil {
+		return nil, err
+	}
+	ctx = buildid.AppendToOutgoingContext(ctx, g.buildID)
+	return g.gateway.ExecProcess(ctx, opts...)
+}

+ 21 - 14
vendor/github.com/moby/buildkit/client/client.go

@@ -6,8 +6,9 @@ import (
 	"crypto/x509"
 	"io/ioutil"
 	"net"
-	"time"
+	"net/url"
 
+	"github.com/containerd/containerd/defaults"
 	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
 	"github.com/grpc-ecosystem/grpc-opentracing/go/otgrpc"
 	controlapi "github.com/moby/buildkit/api/services/control"
@@ -30,7 +31,10 @@ type ClientOpt interface{}
 
 // New returns a new buildkit client. Address can be empty for the system-default address.
 func New(ctx context.Context, address string, opts ...ClientOpt) (*Client, error) {
-	gopts := []grpc.DialOption{}
+	gopts := []grpc.DialOption{
+		grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(defaults.DefaultMaxRecvMsgSize)),
+		grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(defaults.DefaultMaxSendMsgSize)),
+	}
 	needDialer := true
 	needWithInsecure := true
 
@@ -54,7 +58,7 @@ func New(ctx context.Context, address string, opts ...ClientOpt) (*Client, error
 			stream = append(stream, otgrpc.OpenTracingStreamClientInterceptor(wt.tracer))
 		}
 		if wd, ok := o.(*withDialer); ok {
-			gopts = append(gopts, grpc.WithDialer(wd.dialer))
+			gopts = append(gopts, grpc.WithContextDialer(wd.dialer))
 			needDialer = false
 		}
 	}
@@ -63,9 +67,7 @@ func New(ctx context.Context, address string, opts ...ClientOpt) (*Client, error
 		if err != nil {
 			return nil, err
 		}
-		// TODO(AkihiroSuda): use WithContextDialer (requires grpc 1.19)
-		// https://github.com/grpc/grpc-go/commit/40cb5618f475e7b9d61aa7920ae4b04ef9bbaf89
-		gopts = append(gopts, grpc.WithDialer(dialFn))
+		gopts = append(gopts, grpc.WithContextDialer(dialFn))
 	}
 	if needWithInsecure {
 		gopts = append(gopts, grpc.WithInsecure())
@@ -74,6 +76,15 @@ func New(ctx context.Context, address string, opts ...ClientOpt) (*Client, error
 		address = appdefaults.Address
 	}
 
+	// grpc-go uses a slightly different naming scheme: https://github.com/grpc/grpc/blob/master/doc/naming.md
+	// This will end up setting rfc non-complient :authority header to address string (e.g. tcp://127.0.0.1:1234).
+	// So, here sets right authority header via WithAuthority DialOption.
+	addressURL, err := url.Parse(address)
+	if err != nil {
+		return nil, err
+	}
+	gopts = append(gopts, grpc.WithAuthority(addressURL.Host))
+
 	unary = append(unary, grpcerrors.UnaryClientInterceptor)
 	stream = append(stream, grpcerrors.StreamClientInterceptor)
 
@@ -118,10 +129,10 @@ func WithFailFast() ClientOpt {
 }
 
 type withDialer struct {
-	dialer func(string, time.Duration) (net.Conn, error)
+	dialer func(context.Context, string) (net.Conn, error)
 }
 
-func WithDialer(df func(string, time.Duration) (net.Conn, error)) ClientOpt {
+func WithContextDialer(df func(context.Context, string) (net.Conn, error)) ClientOpt {
 	return &withDialer{dialer: df}
 }
 
@@ -179,17 +190,13 @@ type withTracer struct {
 	tracer opentracing.Tracer
 }
 
-func resolveDialer(address string) (func(string, time.Duration) (net.Conn, error), error) {
+func resolveDialer(address string) (func(context.Context, string) (net.Conn, error), error) {
 	ch, err := connhelper.GetConnectionHelper(address)
 	if err != nil {
 		return nil, err
 	}
 	if ch != nil {
-		f := func(a string, _ time.Duration) (net.Conn, error) {
-			ctx := context.Background()
-			return ch.ContextDialer(ctx, a)
-		}
-		return f, nil
+		return ch.ContextDialer, nil
 	}
 	// basic dialer
 	return dialer, nil

+ 4 - 3
vendor/github.com/moby/buildkit/client/client_unix.go

@@ -3,17 +3,18 @@
 package client
 
 import (
+	"context"
 	"net"
 	"strings"
-	"time"
 
 	"github.com/pkg/errors"
 )
 
-func dialer(address string, timeout time.Duration) (net.Conn, error) {
+func dialer(ctx context.Context, address string) (net.Conn, error) {
 	addrParts := strings.SplitN(address, "://", 2)
 	if len(addrParts) != 2 {
 		return nil, errors.Errorf("invalid address %s", address)
 	}
-	return net.DialTimeout(addrParts[0], addrParts[1], timeout)
+	var d net.Dialer
+	return d.DialContext(ctx, addrParts[0], addrParts[1])
 }

+ 5 - 4
vendor/github.com/moby/buildkit/client/client_windows.go

@@ -1,15 +1,15 @@
 package client
 
 import (
+	"context"
 	"net"
 	"strings"
-	"time"
 
 	winio "github.com/Microsoft/go-winio"
 	"github.com/pkg/errors"
 )
 
-func dialer(address string, timeout time.Duration) (net.Conn, error) {
+func dialer(ctx context.Context, address string) (net.Conn, error) {
 	addrParts := strings.SplitN(address, "://", 2)
 	if len(addrParts) != 2 {
 		return nil, errors.Errorf("invalid address %s", address)
@@ -17,8 +17,9 @@ func dialer(address string, timeout time.Duration) (net.Conn, error) {
 	switch addrParts[0] {
 	case "npipe":
 		address = strings.Replace(addrParts[1], "/", "\\", -1)
-		return winio.DialPipe(address, &timeout)
+		return winio.DialPipeContext(ctx, address)
 	default:
-		return net.DialTimeout(addrParts[0], addrParts[1], timeout)
+		var d net.Dialer
+		return d.DialContext(ctx, addrParts[0], addrParts[1])
 	}
 }

+ 44 - 22
vendor/github.com/moby/buildkit/client/llb/definition.go

@@ -16,14 +16,15 @@ import (
 // LLB state can be reconstructed from the definition.
 type DefinitionOp struct {
 	MarshalCache
-	mu        sync.Mutex
-	ops       map[digest.Digest]*pb.Op
-	defs      map[digest.Digest][]byte
-	metas     map[digest.Digest]pb.OpMetadata
-	sources   map[digest.Digest][]*SourceLocation
-	platforms map[digest.Digest]*specs.Platform
-	dgst      digest.Digest
-	index     pb.OutputIndex
+	mu         sync.Mutex
+	ops        map[digest.Digest]*pb.Op
+	defs       map[digest.Digest][]byte
+	metas      map[digest.Digest]pb.OpMetadata
+	sources    map[digest.Digest][]*SourceLocation
+	platforms  map[digest.Digest]*specs.Platform
+	dgst       digest.Digest
+	index      pb.OutputIndex
+	inputCache map[digest.Digest][]*DefinitionOp
 }
 
 // NewDefinitionOp returns a new operation from a marshalled definition.
@@ -89,13 +90,14 @@ func NewDefinitionOp(def *pb.Definition) (*DefinitionOp, error) {
 	}
 
 	return &DefinitionOp{
-		ops:       ops,
-		defs:      defs,
-		metas:     def.Metadata,
-		sources:   srcs,
-		platforms: platforms,
-		dgst:      dgst,
-		index:     index,
+		ops:        ops,
+		defs:       defs,
+		metas:      def.Metadata,
+		sources:    srcs,
+		platforms:  platforms,
+		dgst:       dgst,
+		index:      index,
+		inputCache: make(map[digest.Digest][]*DefinitionOp),
 	}, nil
 }
 
@@ -188,14 +190,34 @@ func (d *DefinitionOp) Inputs() []Output {
 	d.mu.Unlock()
 
 	for _, input := range op.Inputs {
-		vtx := &DefinitionOp{
-			ops:       d.ops,
-			defs:      d.defs,
-			metas:     d.metas,
-			platforms: d.platforms,
-			dgst:      input.Digest,
-			index:     input.Index,
+		var vtx *DefinitionOp
+		d.mu.Lock()
+		if existingIndexes, ok := d.inputCache[input.Digest]; ok {
+			if int(input.Index) < len(existingIndexes) && existingIndexes[input.Index] != nil {
+				vtx = existingIndexes[input.Index]
+			}
 		}
+		if vtx == nil {
+			vtx = &DefinitionOp{
+				ops:        d.ops,
+				defs:       d.defs,
+				metas:      d.metas,
+				platforms:  d.platforms,
+				dgst:       input.Digest,
+				index:      input.Index,
+				inputCache: d.inputCache,
+			}
+			existingIndexes := d.inputCache[input.Digest]
+			indexDiff := int(input.Index) - len(existingIndexes)
+			if indexDiff >= 0 {
+				// make room in the slice for the new index being set
+				existingIndexes = append(existingIndexes, make([]*DefinitionOp, indexDiff+1)...)
+			}
+			existingIndexes[input.Index] = vtx
+			d.inputCache[input.Digest] = existingIndexes
+		}
+		d.mu.Unlock()
+
 		inputs = append(inputs, &output{vertex: vtx, platform: platform, getIndex: func() (pb.OutputIndex, error) {
 			return pb.OutputIndex(vtx.index), nil
 		}})

+ 24 - 12
vendor/github.com/moby/buildkit/client/llb/exec.go

@@ -2,7 +2,7 @@ package llb
 
 import (
 	"context"
-	_ "crypto/sha256"
+	_ "crypto/sha256" // for opencontainers/go-digest
 	"fmt"
 	"net"
 	"sort"
@@ -153,7 +153,13 @@ func (e *ExecOp) Marshal(ctx context.Context, c *Constraints) (digest.Digest, []
 	}
 	if c.Caps != nil {
 		if err := c.Caps.Supports(pb.CapExecMetaSetsDefaultPath); err != nil {
-			env = env.SetDefault("PATH", system.DefaultPathEnv)
+			os := "linux"
+			if c.Platform != nil {
+				os = c.Platform.OS
+			} else if e.constraints.Platform != nil {
+				os = e.constraints.Platform.OS
+			}
+			env = env.SetDefault("PATH", system.DefaultPathEnv(os))
 		} else {
 			addCap(&e.constraints, pb.CapExecMetaSetsDefaultPath)
 		}
@@ -174,11 +180,17 @@ func (e *ExecOp) Marshal(ctx context.Context, c *Constraints) (digest.Digest, []
 		return "", nil, nil, nil, err
 	}
 
+	hostname, err := getHostname(e.base)(ctx)
+	if err != nil {
+		return "", nil, nil, nil, err
+	}
+
 	meta := &pb.Meta{
-		Args: args,
-		Env:  env.ToArray(),
-		Cwd:  cwd,
-		User: user,
+		Args:     args,
+		Env:      env.ToArray(),
+		Cwd:      cwd,
+		User:     user,
+		Hostname: hostname,
 	}
 	extraHosts, err := getExtraHosts(e.base)(ctx)
 	if err != nil {
@@ -217,9 +229,9 @@ func (e *ExecOp) Marshal(ctx context.Context, c *Constraints) (digest.Digest, []
 
 	if p := e.proxyEnv; p != nil {
 		peo.Meta.ProxyEnv = &pb.ProxyEnv{
-			HttpProxy:  p.HttpProxy,
-			HttpsProxy: p.HttpsProxy,
-			FtpProxy:   p.FtpProxy,
+			HttpProxy:  p.HTTPProxy,
+			HttpsProxy: p.HTTPSProxy,
+			FtpProxy:   p.FTPProxy,
 			NoProxy:    p.NoProxy,
 		}
 		addCap(&e.constraints, pb.CapExecMetaProxy)
@@ -629,9 +641,9 @@ type MountInfo struct {
 }
 
 type ProxyEnv struct {
-	HttpProxy  string
-	HttpsProxy string
-	FtpProxy   string
+	HTTPProxy  string
+	HTTPSProxy string
+	FTPProxy   string
 	NoProxy    string
 }
 

+ 11 - 11
vendor/github.com/moby/buildkit/client/llb/fileop.go

@@ -2,7 +2,7 @@ package llb
 
 import (
 	"context"
-	_ "crypto/sha256"
+	_ "crypto/sha256" // for opencontainers/go-digest
 	"os"
 	"path"
 	"strconv"
@@ -252,13 +252,13 @@ func (co ChownOpt) SetCopyOption(mi *CopyInfo) {
 	mi.ChownOpt = &co
 }
 
-func (cp *ChownOpt) marshal(base pb.InputIndex) *pb.ChownOpt {
-	if cp == nil {
+func (co *ChownOpt) marshal(base pb.InputIndex) *pb.ChownOpt {
+	if co == nil {
 		return nil
 	}
 	return &pb.ChownOpt{
-		User:  cp.User.marshal(base),
-		Group: cp.Group.marshal(base),
+		User:  co.User.marshal(base),
+		Group: co.Group.marshal(base),
 	}
 }
 
@@ -476,17 +476,17 @@ func (a *fileActionCopy) toProtoAction(ctx context.Context, parent string, base
 	}, nil
 }
 
-func (c *fileActionCopy) sourcePath(ctx context.Context) (string, error) {
-	p := path.Clean(c.src)
+func (a *fileActionCopy) sourcePath(ctx context.Context) (string, error) {
+	p := path.Clean(a.src)
 	if !path.IsAbs(p) {
-		if c.state != nil {
-			dir, err := c.state.GetDir(ctx)
+		if a.state != nil {
+			dir, err := a.state.GetDir(ctx)
 			if err != nil {
 				return "", err
 			}
 			p = path.Join("/", dir, p)
-		} else if c.fas != nil {
-			dir, err := c.fas.state.GetDir(ctx)
+		} else if a.fas != nil {
+			dir, err := a.fas.state.GetDir(ctx)
 			if err != nil {
 				return "", err
 			}

+ 1 - 1
vendor/github.com/moby/buildkit/client/llb/imagemetaresolver/resolver.go

@@ -8,10 +8,10 @@ import (
 	"github.com/containerd/containerd/platforms"
 	"github.com/containerd/containerd/remotes"
 	"github.com/containerd/containerd/remotes/docker"
-	"github.com/docker/docker/pkg/locker"
 	"github.com/moby/buildkit/client/llb"
 	"github.com/moby/buildkit/util/contentutil"
 	"github.com/moby/buildkit/util/imageutil"
+	"github.com/moby/locker"
 	digest "github.com/opencontainers/go-digest"
 	specs "github.com/opencontainers/image-spec/specs-go/v1"
 )

+ 21 - 1
vendor/github.com/moby/buildkit/client/llb/meta.go

@@ -19,6 +19,7 @@ var (
 	keyDir       = contextKeyT("llb.exec.dir")
 	keyEnv       = contextKeyT("llb.exec.env")
 	keyUser      = contextKeyT("llb.exec.user")
+	keyHostname  = contextKeyT("llb.exec.hostname")
 	keyExtraHost = contextKeyT("llb.exec.extrahost")
 	keyPlatform  = contextKeyT("llb.platform")
 	keyNetwork   = contextKeyT("llb.network")
@@ -143,6 +144,25 @@ func getUser(s State) func(context.Context) (string, error) {
 	}
 }
 
+func Hostname(str string) StateOption {
+	return func(s State) State {
+		return s.WithValue(keyHostname, str)
+	}
+}
+
+func getHostname(s State) func(context.Context) (string, error) {
+	return func(ctx context.Context) (string, error) {
+		v, err := s.getValue(keyHostname)(ctx)
+		if err != nil {
+			return "", err
+		}
+		if v != nil {
+			return v.(string), nil
+		}
+		return "", nil
+	}
+}
+
 func args(args ...string) StateOption {
 	return func(s State) State {
 		return s.WithValue(keyArgs, args)
@@ -155,7 +175,7 @@ func shlexf(str string, replace bool, v ...interface{}) StateOption {
 	}
 	return func(s State) State {
 		arg, err := shlex.Split(str)
-		if err != nil {
+		if err != nil { //nolint
 			// TODO: handle error
 		}
 		return args(arg...)(s)

+ 10 - 3
vendor/github.com/moby/buildkit/client/llb/source.go

@@ -2,7 +2,7 @@ package llb
 
 import (
 	"context"
-	_ "crypto/sha256"
+	_ "crypto/sha256" // for opencontainers/go-digest
 	"encoding/json"
 	"os"
 	"strconv"
@@ -233,11 +233,15 @@ func Git(remote, ref string, opts ...GitOption) State {
 	}
 	if gi.AuthTokenSecret != "" {
 		attrs[pb.AttrAuthTokenSecret] = gi.AuthTokenSecret
-		addCap(&gi.Constraints, pb.CapSourceGitHttpAuth)
+		if gi.addAuthCap {
+			addCap(&gi.Constraints, pb.CapSourceGitHTTPAuth)
+		}
 	}
 	if gi.AuthHeaderSecret != "" {
 		attrs[pb.AttrAuthHeaderSecret] = gi.AuthHeaderSecret
-		addCap(&gi.Constraints, pb.CapSourceGitHttpAuth)
+		if gi.addAuthCap {
+			addCap(&gi.Constraints, pb.CapSourceGitHTTPAuth)
+		}
 	}
 
 	addCap(&gi.Constraints, pb.CapSourceGit)
@@ -260,6 +264,7 @@ type GitInfo struct {
 	KeepGitDir       bool
 	AuthTokenSecret  string
 	AuthHeaderSecret string
+	addAuthCap       bool
 }
 
 func KeepGitDir() GitOption {
@@ -271,12 +276,14 @@ func KeepGitDir() GitOption {
 func AuthTokenSecret(v string) GitOption {
 	return gitOptionFunc(func(gi *GitInfo) {
 		gi.AuthTokenSecret = v
+		gi.addAuthCap = true
 	})
 }
 
 func AuthHeaderSecret(v string) GitOption {
 	return gitOptionFunc(func(gi *GitInfo) {
 		gi.AuthHeaderSecret = v
+		gi.addAuthCap = true
 	})
 }
 

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

@@ -320,6 +320,14 @@ func (s State) User(v string) State {
 	return User(v)(s)
 }
 
+func (s State) Hostname(v string) State {
+	return Hostname(v)(s)
+}
+
+func (s State) GetHostname(ctx context.Context) (string, error) {
+	return getHostname(s)(ctx)
+}
+
 func (s State) Platform(p specs.Platform) State {
 	return platform(p)(s)
 }

+ 1 - 1
vendor/github.com/moby/buildkit/client/ociindex/ociindex.go

@@ -6,7 +6,7 @@ import (
 	"os"
 
 	"github.com/gofrs/flock"
-	"github.com/opencontainers/image-spec/specs-go/v1"
+	v1 "github.com/opencontainers/image-spec/specs-go/v1"
 	"github.com/pkg/errors"
 )
 

+ 10 - 1
vendor/github.com/moby/buildkit/cmd/buildkitd/config/config.go

@@ -1,5 +1,7 @@
 package config
 
+import "github.com/BurntSushi/toml"
+
 // Config provides containerd configuration data for the server
 type Config struct {
 	Debug bool `toml:"debug"`
@@ -78,7 +80,13 @@ type OCIConfig struct {
 	// incomplete and the intention is to make it default without config.
 	UserRemapUnsupported string `toml:"userRemapUnsupported"`
 	// For use in storing the OCI worker binary name that will replace buildkit-runc
-	Binary string `toml:"binary"`
+	Binary               string `toml:"binary"`
+	ProxySnapshotterPath string `toml:"proxySnapshotterPath"`
+
+	// StargzSnapshotterConfig is configuration for stargz snapshotter.
+	// Decoding this is delayed in order to remove the dependency from this
+	// config pkg to stargz snapshotter's config pkg.
+	StargzSnapshotterConfig toml.Primitive `toml:"stargzSnapshotter"`
 }
 
 type ContainerdConfig struct {
@@ -89,6 +97,7 @@ type ContainerdConfig struct {
 	Namespace string            `toml:"namespace"`
 	GCConfig
 	NetworkConfig
+	Snapshotter string `toml:"snapshotter"`
 }
 
 type GCPolicy struct {

+ 51 - 35
vendor/github.com/moby/buildkit/control/control.go

@@ -63,7 +63,7 @@ func NewController(opt Opt) (*Controller, error) {
 		cache:            cache,
 		gatewayForwarder: gatewayForwarder,
 	}
-	c.throttledGC = throttle.ThrottleAfter(time.Minute, c.gc)
+	c.throttledGC = throttle.After(time.Minute, c.gc)
 
 	defer func() {
 		time.AfterFunc(time.Second, c.throttledGC)
@@ -305,40 +305,56 @@ func (c *Controller) Status(req *controlapi.StatusRequest, stream controlapi.Con
 			if !ok {
 				return nil
 			}
-			sr := controlapi.StatusResponse{}
-			for _, v := range ss.Vertexes {
-				sr.Vertexes = append(sr.Vertexes, &controlapi.Vertex{
-					Digest:    v.Digest,
-					Inputs:    v.Inputs,
-					Name:      v.Name,
-					Started:   v.Started,
-					Completed: v.Completed,
-					Error:     v.Error,
-					Cached:    v.Cached,
-				})
-			}
-			for _, v := range ss.Statuses {
-				sr.Statuses = append(sr.Statuses, &controlapi.VertexStatus{
-					ID:        v.ID,
-					Vertex:    v.Vertex,
-					Name:      v.Name,
-					Current:   v.Current,
-					Total:     v.Total,
-					Timestamp: v.Timestamp,
-					Started:   v.Started,
-					Completed: v.Completed,
-				})
-			}
-			for _, v := range ss.Logs {
-				sr.Logs = append(sr.Logs, &controlapi.VertexLog{
-					Vertex:    v.Vertex,
-					Stream:    int64(v.Stream),
-					Msg:       v.Data,
-					Timestamp: v.Timestamp,
-				})
-			}
-			if err := stream.SendMsg(&sr); err != nil {
-				return err
+			logSize := 0
+			retry := false
+			for {
+				sr := controlapi.StatusResponse{}
+				for _, v := range ss.Vertexes {
+					sr.Vertexes = append(sr.Vertexes, &controlapi.Vertex{
+						Digest:    v.Digest,
+						Inputs:    v.Inputs,
+						Name:      v.Name,
+						Started:   v.Started,
+						Completed: v.Completed,
+						Error:     v.Error,
+						Cached:    v.Cached,
+					})
+				}
+				for _, v := range ss.Statuses {
+					sr.Statuses = append(sr.Statuses, &controlapi.VertexStatus{
+						ID:        v.ID,
+						Vertex:    v.Vertex,
+						Name:      v.Name,
+						Current:   v.Current,
+						Total:     v.Total,
+						Timestamp: v.Timestamp,
+						Started:   v.Started,
+						Completed: v.Completed,
+					})
+				}
+				for i, v := range ss.Logs {
+					sr.Logs = append(sr.Logs, &controlapi.VertexLog{
+						Vertex:    v.Vertex,
+						Stream:    int64(v.Stream),
+						Msg:       v.Data,
+						Timestamp: v.Timestamp,
+					})
+					logSize += len(v.Data)
+					// avoid logs growing big and split apart if they do
+					if logSize > 1024*1024 {
+						ss.Vertexes = nil
+						ss.Statuses = nil
+						ss.Logs = ss.Logs[i+1:]
+						retry = true
+						break
+					}
+				}
+				if err := stream.SendMsg(&sr); err != nil {
+					return err
+				}
+				if !retry {
+					break
+				}
 			}
 		}
 	})

+ 24 - 0
vendor/github.com/moby/buildkit/control/gateway/gateway.go

@@ -152,3 +152,27 @@ func (gwf *GatewayForwarder) StatFile(ctx context.Context, req *gwapi.StatFileRe
 	}
 	return fwd.StatFile(ctx, req)
 }
+
+func (gwf *GatewayForwarder) NewContainer(ctx context.Context, req *gwapi.NewContainerRequest) (*gwapi.NewContainerResponse, error) {
+	fwd, err := gwf.lookupForwarder(ctx)
+	if err != nil {
+		return nil, errors.Wrap(err, "forwarding NewContainer")
+	}
+	return fwd.NewContainer(ctx, req)
+}
+
+func (gwf *GatewayForwarder) ReleaseContainer(ctx context.Context, req *gwapi.ReleaseContainerRequest) (*gwapi.ReleaseContainerResponse, error) {
+	fwd, err := gwf.lookupForwarder(ctx)
+	if err != nil {
+		return nil, errors.Wrap(err, "forwarding ReleaseContainer")
+	}
+	return fwd.ReleaseContainer(ctx, req)
+}
+
+func (gwf *GatewayForwarder) ExecProcess(srv gwapi.LLBBridge_ExecProcessServer) error {
+	fwd, err := gwf.lookupForwarder(srv.Context())
+	if err != nil {
+		return errors.Wrap(err, "forwarding ExecProcess")
+	}
+	return fwd.ExecProcess(srv)
+}

+ 14 - 3
vendor/github.com/moby/buildkit/executor/executor.go

@@ -5,7 +5,7 @@ import (
 	"io"
 	"net"
 
-	"github.com/moby/buildkit/cache"
+	"github.com/moby/buildkit/snapshot"
 	"github.com/moby/buildkit/solver/pb"
 )
 
@@ -14,6 +14,7 @@ type Meta struct {
 	Env            []string
 	User           string
 	Cwd            string
+	Hostname       string
 	Tty            bool
 	ReadonlyRootFS bool
 	ExtraHosts     []HostIP
@@ -21,24 +22,34 @@ type Meta struct {
 	SecurityMode   pb.SecurityMode
 }
 
+type Mountable interface {
+	Mount(ctx context.Context, readonly bool) (snapshot.Mountable, error)
+}
+
 type Mount struct {
-	Src      cache.Mountable
+	Src      Mountable
 	Selector string
 	Dest     string
 	Readonly bool
 }
 
+type WinSize struct {
+	Rows uint32
+	Cols uint32
+}
+
 type ProcessInfo struct {
 	Meta           Meta
 	Stdin          io.ReadCloser
 	Stdout, Stderr io.WriteCloser
+	Resize         <-chan WinSize
 }
 
 type Executor interface {
 	// Run will start a container for the given process with rootfs, mounts.
 	// `id` is an optional name for the container so it can be referenced later via Exec.
 	// `started` is an optional channel that will be closed when the container setup completes and has started running.
-	Run(ctx context.Context, id string, rootfs cache.Mountable, mounts []Mount, process ProcessInfo, started chan<- struct{}) error
+	Run(ctx context.Context, id string, rootfs Mount, mounts []Mount, process ProcessInfo, started chan<- struct{}) error
 	// Exec will start a process in container matching `id`. An error will be returned
 	// if the container failed to start (via Run) or has exited before Exec is called.
 	Exec(ctx context.Context, id string, process ProcessInfo) error

+ 27 - 19
vendor/github.com/moby/buildkit/executor/oci/hosts.go

@@ -14,28 +14,26 @@ import (
 	"github.com/pkg/errors"
 )
 
-const hostsContent = `
-127.0.0.1	localhost buildkitsandbox
-::1	localhost ip6-localhost ip6-loopback
-`
+const defaultHostname = "buildkitsandbox"
 
-func GetHostsFile(ctx context.Context, stateDir string, extraHosts []executor.HostIP, idmap *idtools.IdentityMapping) (string, func(), error) {
-	if len(extraHosts) == 0 {
-		_, err := g.Do(ctx, stateDir, func(ctx context.Context) (interface{}, error) {
-			_, _, err := makeHostsFile(stateDir, nil, idmap)
-			return nil, err
-		})
-		if err != nil {
-			return "", nil, err
-		}
-		return filepath.Join(stateDir, "hosts"), func() {}, nil
+func GetHostsFile(ctx context.Context, stateDir string, extraHosts []executor.HostIP, idmap *idtools.IdentityMapping, hostname string) (string, func(), error) {
+	if len(extraHosts) != 0 || hostname != defaultHostname {
+		return makeHostsFile(stateDir, extraHosts, idmap, hostname)
+	}
+
+	_, err := g.Do(ctx, stateDir, func(ctx context.Context) (interface{}, error) {
+		_, _, err := makeHostsFile(stateDir, nil, idmap, hostname)
+		return nil, err
+	})
+	if err != nil {
+		return "", nil, err
 	}
-	return makeHostsFile(stateDir, extraHosts, idmap)
+	return filepath.Join(stateDir, "hosts"), func() {}, nil
 }
 
-func makeHostsFile(stateDir string, extraHosts []executor.HostIP, idmap *idtools.IdentityMapping) (string, func(), error) {
+func makeHostsFile(stateDir string, extraHosts []executor.HostIP, idmap *idtools.IdentityMapping, hostname string) (string, func(), error) {
 	p := filepath.Join(stateDir, "hosts")
-	if len(extraHosts) != 0 {
+	if len(extraHosts) != 0 || hostname != defaultHostname {
 		p += "." + identity.NewID()
 	}
 	_, err := os.Stat(p)
@@ -47,8 +45,7 @@ func makeHostsFile(stateDir string, extraHosts []executor.HostIP, idmap *idtools
 	}
 
 	b := &bytes.Buffer{}
-
-	if _, err := b.Write([]byte(hostsContent)); err != nil {
+	if _, err := b.Write([]byte(initHostsFile(hostname))); err != nil {
 		return "", nil, err
 	}
 
@@ -77,3 +74,14 @@ func makeHostsFile(stateDir string, extraHosts []executor.HostIP, idmap *idtools
 		os.RemoveAll(p)
 	}, nil
 }
+
+func initHostsFile(hostname string) string {
+	var hosts string
+	if hostname != "" {
+		hosts = fmt.Sprintf("127.0.0.1	localhost %s", hostname)
+	} else {
+		hosts = fmt.Sprintf("127.0.0.1	localhost %s", defaultHostname)
+	}
+	hosts = fmt.Sprintf("%s\n::1	localhost ip6-localhost ip6-loopback\n", hosts)
+	return hosts
+}

+ 59 - 74
vendor/github.com/moby/buildkit/executor/oci/mounts.go

@@ -5,82 +5,57 @@ import (
 	"path/filepath"
 	"strings"
 
+	"github.com/containerd/containerd/containers"
+	"github.com/containerd/containerd/oci"
 	specs "github.com/opencontainers/runtime-spec/specs-go"
-	"github.com/pkg/errors"
 )
 
-// MountOpts sets oci spec specific info for mount points
-type MountOpts func([]specs.Mount) ([]specs.Mount, error)
-
-//GetMounts returns default required for buildkit
-// https://github.com/moby/buildkit/issues/429
-func GetMounts(ctx context.Context, mountOpts ...MountOpts) ([]specs.Mount, error) {
-	mounts := []specs.Mount{
-		{
-			Destination: "/proc",
-			Type:        "proc",
-			Source:      "proc",
-		},
-		{
-			Destination: "/dev",
-			Type:        "tmpfs",
-			Source:      "tmpfs",
-			Options:     []string{"nosuid", "strictatime", "mode=755", "size=65536k"},
-		},
-		{
-			Destination: "/dev/pts",
-			Type:        "devpts",
-			Source:      "devpts",
-			Options:     []string{"nosuid", "noexec", "newinstance", "ptmxmode=0666", "mode=0620", "gid=5"},
-		},
-		{
-			Destination: "/dev/shm",
-			Type:        "tmpfs",
-			Source:      "shm",
-			Options:     []string{"nosuid", "noexec", "nodev", "mode=1777", "size=65536k"},
-		},
-		{
-			Destination: "/dev/mqueue",
-			Type:        "mqueue",
-			Source:      "mqueue",
-			Options:     []string{"nosuid", "noexec", "nodev"},
-		},
-		{
-			Destination: "/sys",
-			Type:        "sysfs",
-			Source:      "sysfs",
-			Options:     []string{"nosuid", "noexec", "nodev", "ro"},
-		},
-	}
-	var err error
-	for _, o := range mountOpts {
-		mounts, err = o(mounts)
-		if err != nil {
-			return nil, err
+func withRemovedMount(destination string) oci.SpecOpts {
+	return func(_ context.Context, _ oci.Client, _ *containers.Container, s *specs.Spec) error {
+		newMounts := []specs.Mount{}
+		for _, o := range s.Mounts {
+			if o.Destination != destination {
+				newMounts = append(newMounts, o)
+			}
 		}
+		s.Mounts = newMounts
+
+		return nil
 	}
-	return mounts, nil
 }
 
-func withROBind(src, dest string) func(m []specs.Mount) ([]specs.Mount, error) {
-	return func(m []specs.Mount) ([]specs.Mount, error) {
-		m = append(m, specs.Mount{
+func withROBind(src, dest string) oci.SpecOpts {
+	return func(_ context.Context, _ oci.Client, _ *containers.Container, s *specs.Spec) error {
+		s.Mounts = append(s.Mounts, specs.Mount{
 			Destination: dest,
 			Type:        "bind",
 			Source:      src,
 			Options:     []string{"nosuid", "noexec", "nodev", "rbind", "ro"},
 		})
-		return m, nil
+		return nil
+	}
+}
+
+func withCGroup() oci.SpecOpts {
+	return func(_ context.Context, _ oci.Client, _ *containers.Container, s *specs.Spec) error {
+		s.Mounts = append(s.Mounts, specs.Mount{
+			Destination: "/sys/fs/cgroup",
+			Type:        "cgroup",
+			Source:      "cgroup",
+			Options:     []string{"ro", "nosuid", "noexec", "nodev"},
+		})
+		return nil
 	}
+
 }
 
 func hasPrefix(p, prefixDir string) bool {
 	prefixDir = filepath.Clean(prefixDir)
-	if prefixDir == "/" {
+	if filepath.Base(prefixDir) == string(filepath.Separator) {
 		return true
 	}
 	p = filepath.Clean(p)
-	return p == prefixDir || strings.HasPrefix(p, prefixDir+"/")
+	return p == prefixDir || strings.HasPrefix(p, prefixDir+string(filepath.Separator))
 }
 
 func removeMountsWithPrefix(mounts []specs.Mount, prefixDir string) []specs.Mount {
@@ -93,25 +68,35 @@ func removeMountsWithPrefix(mounts []specs.Mount, prefixDir string) []specs.Moun
 	return ret
 }
 
-func withProcessMode(processMode ProcessMode) func([]specs.Mount) ([]specs.Mount, error) {
-	return func(m []specs.Mount) ([]specs.Mount, error) {
-		switch processMode {
-		case ProcessSandbox:
-			// keep the default
-		case NoProcessSandbox:
-			m = removeMountsWithPrefix(m, "/proc")
-			procMount := specs.Mount{
-				Destination: "/proc",
-				Type:        "bind",
-				Source:      "/proc",
-				// NOTE: "rbind"+"ro" does not make /proc read-only recursively.
-				// So we keep maskedPath and readonlyPaths (although not mandatory for rootless mode)
-				Options: []string{"rbind"},
+func withBoundProc() oci.SpecOpts {
+	return func(_ context.Context, _ oci.Client, _ *containers.Container, s *specs.Spec) error {
+		s.Mounts = removeMountsWithPrefix(s.Mounts, "/proc")
+		procMount := specs.Mount{
+			Destination: "/proc",
+			Type:        "bind",
+			Source:      "/proc",
+			// NOTE: "rbind"+"ro" does not make /proc read-only recursively.
+			// So we keep maskedPath and readonlyPaths (although not mandatory for rootless mode)
+			Options: []string{"rbind"},
+		}
+		s.Mounts = append([]specs.Mount{procMount}, s.Mounts...)
+
+		var maskedPaths []string
+		for _, s := range s.Linux.MaskedPaths {
+			if !hasPrefix(s, "/proc") {
+				maskedPaths = append(maskedPaths, s)
 			}
-			m = append([]specs.Mount{procMount}, m...)
-		default:
-			return nil, errors.Errorf("unknown process mode: %v", processMode)
 		}
-		return m, nil
+		s.Linux.MaskedPaths = maskedPaths
+
+		var readonlyPaths []string
+		for _, s := range s.Linux.ReadonlyPaths {
+			if !hasPrefix(s, "/proc") {
+				readonlyPaths = append(readonlyPaths, s)
+			}
+		}
+		s.Linux.ReadonlyPaths = readonlyPaths
+
+		return nil
 	}
 }

+ 220 - 1
vendor/github.com/moby/buildkit/executor/oci/spec.go

@@ -1,6 +1,25 @@
 package oci
 
-// ProcMode configures PID namespaces
+import (
+	"context"
+	"path"
+	"sync"
+
+	"github.com/containerd/containerd/containers"
+	"github.com/containerd/containerd/mount"
+	"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"
+	"github.com/moby/buildkit/util/network"
+	specs "github.com/opencontainers/runtime-spec/specs-go"
+	"github.com/pkg/errors"
+)
+
+// ProcessMode configures PID namespaces
 type ProcessMode int
 
 const (
@@ -11,3 +30,203 @@ const (
 	// NoProcessSandbox should be enabled only when the BuildKit is running in a container as an unprivileged user.
 	NoProcessSandbox
 )
+
+// Ideally we don't have to import whole containerd just for the default spec
+
+// 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, idmap *idtools.IdentityMapping, opts ...oci.SpecOpts) (*specs.Spec, func(), error) {
+	c := &containers.Container{
+		ID: id,
+	}
+
+	// containerd/oci.GenerateSpec requires a namespace, which
+	// will be used to namespace specs.Linux.CgroupsPath if generated
+	if _, ok := namespaces.Namespace(ctx); !ok {
+		ctx = namespaces.WithNamespace(ctx, "buildkit")
+	}
+
+	if mountOpts, err := generateMountOpts(resolvConf, hostsFile); err == nil {
+		opts = append(opts, mountOpts...)
+	} else {
+		return nil, nil, err
+	}
+
+	if securityOpts, err := generateSecurityOpts(meta.SecurityMode); err == nil {
+		opts = append(opts, securityOpts...)
+	} else {
+		return nil, nil, err
+	}
+
+	if processModeOpts, err := generateProcessModeOpts(processMode); err == nil {
+		opts = append(opts, processModeOpts...)
+	} else {
+		return nil, nil, err
+	}
+
+	if idmapOpts, err := generateIDmapOpts(idmap); err == nil {
+		opts = append(opts, idmapOpts...)
+	} else {
+		return nil, nil, err
+	}
+
+	hostname := defaultHostname
+	if meta.Hostname != "" {
+		hostname = meta.Hostname
+	}
+
+	opts = append(opts,
+		oci.WithProcessArgs(meta.Args...),
+		oci.WithEnv(meta.Env),
+		oci.WithProcessCwd(meta.Cwd),
+		oci.WithNewPrivileges,
+		oci.WithHostname(hostname),
+	)
+
+	s, err := oci.GenerateSpec(ctx, nil, c, opts...)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	// set the networking information on the spec
+	if err := namespace.Set(s); err != nil {
+		return nil, nil, err
+	}
+
+	s.Process.Rlimits = nil // reset open files limit
+
+	sm := &submounts{}
+
+	var releasers []func() error
+	releaseAll := func() {
+		sm.cleanup()
+		for _, f := range releasers {
+			f()
+		}
+	}
+
+	for _, m := range mounts {
+		if m.Src == nil {
+			return nil, nil, errors.Errorf("mount %s has no source", m.Dest)
+		}
+		mountable, err := m.Src.Mount(ctx, m.Readonly)
+		if err != nil {
+			releaseAll()
+			return nil, nil, errors.Wrapf(err, "failed to mount %s", m.Dest)
+		}
+		mounts, release, err := mountable.Mount()
+		if err != nil {
+			releaseAll()
+			return nil, nil, errors.WithStack(err)
+		}
+		releasers = append(releasers, release)
+		for _, mount := range mounts {
+			mount, err = sm.subMount(mount, m.Selector)
+			if err != nil {
+				releaseAll()
+				return nil, nil, err
+			}
+			s.Mounts = append(s.Mounts, specs.Mount{
+				Destination: m.Dest,
+				Type:        mount.Type,
+				Source:      mount.Source,
+				Options:     mount.Options,
+			})
+		}
+	}
+
+	return s, releaseAll, nil
+}
+
+type mountRef struct {
+	mount   mount.Mount
+	unmount func() error
+}
+
+type submounts struct {
+	m map[uint64]mountRef
+}
+
+func (s *submounts) subMount(m mount.Mount, subPath string) (mount.Mount, error) {
+	if path.Join("/", subPath) == "/" {
+		return m, nil
+	}
+	if s.m == nil {
+		s.m = map[uint64]mountRef{}
+	}
+	h, err := hashstructure.Hash(m, nil)
+	if err != nil {
+		return mount.Mount{}, nil
+	}
+	if mr, ok := s.m[h]; ok {
+		sm, err := sub(mr.mount, subPath)
+		if err != nil {
+			return mount.Mount{}, nil
+		}
+		return sm, nil
+	}
+
+	lm := snapshot.LocalMounterWithMounts([]mount.Mount{m})
+
+	mp, err := lm.Mount()
+	if err != nil {
+		return mount.Mount{}, err
+	}
+
+	opts := []string{"rbind"}
+	for _, opt := range m.Options {
+		if opt == "ro" {
+			opts = append(opts, opt)
+		}
+	}
+
+	s.m[h] = mountRef{
+		mount: mount.Mount{
+			Source:  mp,
+			Type:    "bind",
+			Options: opts,
+		},
+		unmount: lm.Unmount,
+	}
+
+	sm, err := sub(s.m[h].mount, subPath)
+	if err != nil {
+		return mount.Mount{}, err
+	}
+	return sm, nil
+}
+
+func (s *submounts) cleanup() {
+	var wg sync.WaitGroup
+	wg.Add(len(s.m))
+	for _, m := range s.m {
+		func(m mountRef) {
+			go func() {
+				m.unmount()
+				wg.Done()
+			}()
+		}(m)
+	}
+	wg.Wait()
+}
+
+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
+}
+
+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
+}

+ 33 - 229
vendor/github.com/moby/buildkit/executor/oci/spec_unix.go

@@ -3,252 +3,56 @@
 package oci
 
 import (
-	"context"
-	"path"
-	"sync"
-
-	"github.com/containerd/containerd/containers"
 	"github.com/containerd/containerd/contrib/seccomp"
-	"github.com/containerd/containerd/mount"
-	"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"
 	"github.com/moby/buildkit/solver/pb"
 	"github.com/moby/buildkit/util/entitlements/security"
-	"github.com/moby/buildkit/util/network"
 	"github.com/moby/buildkit/util/system"
 	specs "github.com/opencontainers/runtime-spec/specs-go"
-	"github.com/pkg/errors"
 )
 
-// Ideally we don't have to import whole containerd just for the default spec
-
-// 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, idmap *idtools.IdentityMapping, opts ...oci.SpecOpts) (*specs.Spec, func(), error) {
-	c := &containers.Container{
-		ID: id,
-	}
-	_, ok := namespaces.Namespace(ctx)
-	if !ok {
-		ctx = namespaces.WithNamespace(ctx, "buildkit")
-	}
-	if meta.SecurityMode == pb.SecurityMode_INSECURE {
-		opts = append(opts, security.WithInsecureSpec())
-	} else if system.SeccompSupported() && meta.SecurityMode == pb.SecurityMode_SANDBOX {
-		opts = append(opts, seccomp.WithDefaultProfile())
-	}
-
-	switch processMode {
-	case NoProcessSandbox:
-		// Mount for /proc is replaced in GetMounts()
-		opts = append(opts,
-			oci.WithHostNamespace(specs.PIDNamespace))
-		// TODO(AkihiroSuda): Configure seccomp to disable ptrace (and prctl?) explicitly
-	}
-
-	// Note that containerd.GenerateSpec is namespaced so as to make
-	// specs.Linux.CgroupsPath namespaced
-	s, err := oci.GenerateSpec(ctx, nil, c, opts...)
-	if err != nil {
-		return nil, nil, err
-	}
-	// set the networking information on the spec
-	namespace.Set(s)
-
-	s.Process.Args = meta.Args
-	s.Process.Env = meta.Env
-	s.Process.Cwd = meta.Cwd
-	s.Process.Rlimits = nil           // reset open files limit
-	s.Process.NoNewPrivileges = false // reset nonewprivileges
-	s.Hostname = "buildkitsandbox"
-
-	s.Mounts, err = GetMounts(ctx,
-		withProcessMode(processMode),
+func generateMountOpts(resolvConf, hostsFile string) ([]oci.SpecOpts, error) {
+	return []oci.SpecOpts{
+		// https://github.com/moby/buildkit/issues/429
+		withRemovedMount("/run"),
 		withROBind(resolvConf, "/etc/resolv.conf"),
 		withROBind(hostsFile, "/etc/hosts"),
-	)
-	if err != nil {
-		return nil, nil, err
-	}
-
-	s.Mounts = append(s.Mounts, specs.Mount{
-		Destination: "/sys/fs/cgroup",
-		Type:        "cgroup",
-		Source:      "cgroup",
-		Options:     []string{"ro", "nosuid", "noexec", "nodev"},
-	})
-
-	if processMode == NoProcessSandbox {
-		var maskedPaths []string
-		for _, s := range s.Linux.MaskedPaths {
-			if !hasPrefix(s, "/proc") {
-				maskedPaths = append(maskedPaths, s)
-			}
-		}
-		s.Linux.MaskedPaths = maskedPaths
-		var readonlyPaths []string
-		for _, s := range s.Linux.ReadonlyPaths {
-			if !hasPrefix(s, "/proc") {
-				readonlyPaths = append(readonlyPaths, s)
-			}
-		}
-		s.Linux.ReadonlyPaths = readonlyPaths
-	}
-
-	if meta.SecurityMode == pb.SecurityMode_INSECURE {
-		if err = oci.WithWriteableCgroupfs(ctx, nil, c, s); err != nil {
-			return nil, nil, err
-		}
-		if err = oci.WithWriteableSysfs(ctx, nil, c, s); err != nil {
-			return nil, nil, err
-		}
-	}
-
-	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{}
-
-	var releasers []func() error
-	releaseAll := func() {
-		sm.cleanup()
-		for _, f := range releasers {
-			f()
-		}
-	}
-
-	for _, m := range mounts {
-		if m.Src == nil {
-			return nil, nil, errors.Errorf("mount %s has no source", m.Dest)
-		}
-		mountable, err := m.Src.Mount(ctx, m.Readonly)
-		if err != nil {
-			releaseAll()
-			return nil, nil, errors.Wrapf(err, "failed to mount %s", m.Dest)
-		}
-		mounts, release, err := mountable.Mount()
-		if err != nil {
-			releaseAll()
-			return nil, nil, errors.WithStack(err)
-		}
-		releasers = append(releasers, release)
-		for _, mount := range mounts {
-			mount, err = sm.subMount(mount, m.Selector)
-			if err != nil {
-				releaseAll()
-				return nil, nil, err
-			}
-			s.Mounts = append(s.Mounts, specs.Mount{
-				Destination: m.Dest,
-				Type:        mount.Type,
-				Source:      mount.Source,
-				Options:     mount.Options,
-			})
-		}
-	}
-
-	return s, releaseAll, nil
-}
-
-type mountRef struct {
-	mount   mount.Mount
-	unmount func() error
+		withCGroup(),
+	}, nil
 }
 
-type submounts struct {
-	m map[uint64]mountRef
+// generateSecurityOpts may affect mounts, so must be called after generateMountOpts
+func generateSecurityOpts(mode pb.SecurityMode) ([]oci.SpecOpts, error) {
+	if mode == pb.SecurityMode_INSECURE {
+		return []oci.SpecOpts{
+			security.WithInsecureSpec(),
+			oci.WithWriteableCgroupfs,
+			oci.WithWriteableSysfs,
+		}, nil
+	} else if system.SeccompSupported() && mode == pb.SecurityMode_SANDBOX {
+		return []oci.SpecOpts{seccomp.WithDefaultProfile()}, nil
+	}
+	return nil, nil
 }
 
-func (s *submounts) subMount(m mount.Mount, subPath string) (mount.Mount, error) {
-	if path.Join("/", subPath) == "/" {
-		return m, nil
-	}
-	if s.m == nil {
-		s.m = map[uint64]mountRef{}
-	}
-	h, err := hashstructure.Hash(m, nil)
-	if err != nil {
-		return mount.Mount{}, nil
-	}
-	if mr, ok := s.m[h]; ok {
-		sm, err := sub(mr.mount, subPath)
-		if err != nil {
-			return mount.Mount{}, nil
-		}
-		return sm, nil
-	}
-
-	lm := snapshot.LocalMounterWithMounts([]mount.Mount{m})
-
-	mp, err := lm.Mount()
-	if err != nil {
-		return mount.Mount{}, err
-	}
-
-	opts := []string{"rbind"}
-	for _, opt := range m.Options {
-		if opt == "ro" {
-			opts = append(opts, opt)
-		}
-	}
-
-	s.m[h] = mountRef{
-		mount: mount.Mount{
-			Source:  mp,
-			Type:    "bind",
-			Options: opts,
-		},
-		unmount: lm.Unmount,
-	}
-
-	sm, err := sub(s.m[h].mount, subPath)
-	if err != nil {
-		return mount.Mount{}, err
-	}
-	return sm, nil
-}
-
-func (s *submounts) cleanup() {
-	var wg sync.WaitGroup
-	wg.Add(len(s.m))
-	for _, m := range s.m {
-		func(m mountRef) {
-			go func() {
-				m.unmount()
-				wg.Done()
-			}()
-		}(m)
-	}
-	wg.Wait()
-}
-
-func sub(m mount.Mount, subPath string) (mount.Mount, error) {
-	src, err := fs.RootPath(m.Source, subPath)
-	if err != nil {
-		return mount.Mount{}, err
+// generateProcessModeOpts may affect mounts, so must be called after generateMountOpts
+func generateProcessModeOpts(mode ProcessMode) ([]oci.SpecOpts, error) {
+	if mode == NoProcessSandbox {
+		return []oci.SpecOpts{
+			oci.WithHostNamespace(specs.PIDNamespace),
+			withBoundProc(),
+		}, nil
+		// TODO(AkihiroSuda): Configure seccomp to disable ptrace (and prctl?) explicitly
 	}
-	m.Source = src
-	return m, nil
+	return nil, 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),
-		})
+func generateIDmapOpts(idmap *idtools.IdentityMapping) ([]oci.SpecOpts, error) {
+	if idmap == nil {
+		return nil, nil
 	}
-	return ids
+	return []oci.SpecOpts{
+		oci.WithUserNamespace(specMapping(idmap.UIDs()), specMapping(idmap.GIDs())),
+	}, nil
 }

+ 42 - 0
vendor/github.com/moby/buildkit/executor/oci/spec_windows.go

@@ -0,0 +1,42 @@
+// +build windows
+
+package oci
+
+import (
+	"github.com/containerd/containerd/contrib/seccomp"
+	"github.com/containerd/containerd/oci"
+	"github.com/docker/docker/pkg/idtools"
+	"github.com/moby/buildkit/solver/pb"
+	"github.com/moby/buildkit/util/system"
+	"github.com/pkg/errors"
+)
+
+func generateMountOpts(resolvConf, hostsFile string) ([]oci.SpecOpts, error) {
+	return nil, nil
+}
+
+// generateSecurityOpts may affect mounts, so must be called after generateMountOpts
+func generateSecurityOpts(mode pb.SecurityMode) ([]oci.SpecOpts, error) {
+	if mode == pb.SecurityMode_INSECURE {
+		return nil, errors.New("no support for running in insecure mode on Windows")
+	} else if system.SeccompSupported() && mode == pb.SecurityMode_SANDBOX {
+		// TODO: Can LCOW support seccomp? Does that even make sense?
+		return []oci.SpecOpts{seccomp.WithDefaultProfile()}, nil
+	}
+	return nil, nil
+}
+
+// generateProcessModeOpts may affect mounts, so must be called after generateMountOpts
+func generateProcessModeOpts(mode ProcessMode) ([]oci.SpecOpts, error) {
+	if mode == NoProcessSandbox {
+		return nil, errors.New("no support for NoProcessSandbox on Windows")
+	}
+	return nil, nil
+}
+
+func generateIDmapOpts(idmap *idtools.IdentityMapping) ([]oci.SpecOpts, error) {
+	if idmap == nil {
+		return nil, nil
+	}
+	return nil, errors.New("no support for IdentityMapping on Windows")
+}

+ 1 - 1
vendor/github.com/moby/buildkit/executor/oci/user.go

@@ -14,7 +14,7 @@ import (
 	"github.com/pkg/errors"
 )
 
-func GetUser(ctx context.Context, root, username string) (uint32, uint32, []uint32, error) {
+func GetUser(root, username string) (uint32, uint32, []uint32, error) {
 	// fast path from uid/gid
 	if uid, gid, err := ParseUIDGID(username); err == nil {
 		return uid, gid, nil, nil

+ 37 - 26
vendor/github.com/moby/buildkit/executor/runcexecutor/executor.go

@@ -12,15 +12,16 @@ import (
 	"syscall"
 	"time"
 
+	"github.com/containerd/containerd"
 	"github.com/containerd/containerd/mount"
 	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"
 	"github.com/moby/buildkit/identity"
+	"github.com/moby/buildkit/solver/errdefs"
 	"github.com/moby/buildkit/solver/pb"
 	"github.com/moby/buildkit/util/network"
 	rootlessspecconv "github.com/moby/buildkit/util/rootless/specconv"
@@ -52,7 +53,6 @@ var defaultCommandCandidates = []string{"buildkit-runc", "runc"}
 type runcExecutor struct {
 	runc             *runc.Runc
 	root             string
-	cmd              string
 	cgroupParent     string
 	rootless         bool
 	networkProviders map[pb.NetMode]network.Provider
@@ -103,15 +103,16 @@ func New(opt Opt, networkProviders map[pb.NetMode]network.Provider) (executor.Ex
 	os.RemoveAll(filepath.Join(root, "resolv.conf"))
 
 	runtime := &runc.Runc{
-		Command:      cmd,
-		Log:          filepath.Join(root, "runc-log.json"),
-		LogFormat:    runc.JSON,
-		PdeathSignal: syscall.SIGKILL, // this can still leak the process
-		Setpgid:      true,
+		Command:   cmd,
+		Log:       filepath.Join(root, "runc-log.json"),
+		LogFormat: runc.JSON,
+		Setpgid:   true,
 		// we don't execute runc with --rootless=(true|false) explicitly,
 		// so as to support non-runc runtimes
 	}
 
+	updateRuncFieldsForHostOS(runtime)
+
 	w := &runcExecutor{
 		runc:             runtime,
 		root:             root,
@@ -128,7 +129,7 @@ func New(opt Opt, networkProviders map[pb.NetMode]network.Provider) (executor.Ex
 	return w, nil
 }
 
-func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable, mounts []executor.Mount, process executor.ProcessInfo, started chan<- struct{}) (err error) {
+func (w *runcExecutor) Run(ctx context.Context, id string, root executor.Mount, mounts []executor.Mount, process executor.ProcessInfo, started chan<- struct{}) (err error) {
 	meta := process.Meta
 
 	startedOnce := sync.Once{}
@@ -168,7 +169,7 @@ func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable,
 		return err
 	}
 
-	hostsFile, clean, err := oci.GetHostsFile(ctx, w.root, meta.ExtraHosts, w.idmap)
+	hostsFile, clean, err := oci.GetHostsFile(ctx, w.root, meta.ExtraHosts, w.idmap, meta.Hostname)
 	if err != nil {
 		return err
 	}
@@ -176,7 +177,7 @@ func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable,
 		defer clean()
 	}
 
-	mountable, err := root.Mount(ctx, false)
+	mountable, err := root.Src.Mount(ctx, false)
 	if err != nil {
 		return err
 	}
@@ -213,7 +214,9 @@ func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable,
 	}
 	defer mount.Unmount(rootFSPath, 0)
 
-	uid, gid, sgids, err := oci.GetUser(ctx, rootFSPath, meta.User)
+	defer executor.MountStubsCleaner(rootFSPath, mounts)()
+
+	uid, gid, sgids, err := oci.GetUser(rootFSPath, meta.User)
 	if err != nil {
 		return err
 	}
@@ -258,7 +261,7 @@ func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable,
 	defer cleanup()
 
 	spec.Root.Path = rootFSPath
-	if _, ok := root.(cache.ImmutableRef); ok { // TODO: pass in with mount, not ref type
+	if root.Readonly {
 		spec.Root.Readonly = true
 	}
 
@@ -323,28 +326,37 @@ func (w *runcExecutor) Run(ctx context.Context, id string, root cache.Mountable,
 			close(started)
 		})
 	}
-	status, err := w.runc.Run(runCtx, id, bundle, &runc.CreateOpts{
-		IO:      &forwardIO{stdin: process.Stdin, stdout: process.Stdout, stderr: process.Stderr},
-		NoPivot: w.noPivot,
-	})
+
+	err = w.run(runCtx, id, bundle, process)
 	close(ended)
+	return exitError(ctx, err)
+}
 
-	if status != 0 || err != nil {
-		if err == nil {
-			err = errors.Errorf("exit code: %d", status)
+func exitError(ctx context.Context, err error) error {
+	if err != nil {
+		exitErr := &errdefs.ExitError{
+			ExitCode: containerd.UnknownExitStatus,
+			Err:      err,
+		}
+		var runcExitError *runc.ExitError
+		if errors.As(err, &runcExitError) {
+			exitErr = &errdefs.ExitError{
+				ExitCode: uint32(runcExitError.Status),
+			}
 		}
 		select {
 		case <-ctx.Done():
-			return errors.Wrapf(ctx.Err(), err.Error())
+			exitErr.Err = errors.Wrapf(ctx.Err(), exitErr.Error())
+			return exitErr
 		default:
-			return stack.Enable(err)
+			return stack.Enable(exitErr)
 		}
 	}
 
 	return nil
 }
 
-func (w *runcExecutor) Exec(ctx context.Context, id string, process executor.ProcessInfo) error {
+func (w *runcExecutor) Exec(ctx context.Context, id string, process executor.ProcessInfo) (err error) {
 	// first verify the container is running, if we get an error assume the container
 	// is in the process of being created and check again every 100ms or until
 	// context is canceled.
@@ -386,7 +398,7 @@ func (w *runcExecutor) Exec(ctx context.Context, id string, process executor.Pro
 	}
 
 	if process.Meta.User != "" {
-		uid, gid, sgids, err := oci.GetUser(ctx, state.Rootfs, process.Meta.User)
+		uid, gid, sgids, err := oci.GetUser(state.Rootfs, process.Meta.User)
 		if err != nil {
 			return err
 		}
@@ -407,9 +419,8 @@ func (w *runcExecutor) Exec(ctx context.Context, id string, process executor.Pro
 		spec.Process.Env = process.Meta.Env
 	}
 
-	return w.runc.Exec(ctx, id, *spec.Process, &runc.ExecOpts{
-		IO: &forwardIO{stdin: process.Stdin, stdout: process.Stdout, stderr: process.Stderr},
-	})
+	err = w.exec(ctx, id, state.Bundle, spec.Process, process)
+	return exitError(ctx, err)
 }
 
 type forwardIO struct {

+ 36 - 0
vendor/github.com/moby/buildkit/executor/runcexecutor/executor_common.go

@@ -0,0 +1,36 @@
+// +build !linux
+
+package runcexecutor
+
+import (
+	"context"
+
+	runc "github.com/containerd/go-runc"
+	"github.com/moby/buildkit/executor"
+	"github.com/opencontainers/runtime-spec/specs-go"
+	"github.com/pkg/errors"
+)
+
+var unsupportedConsoleError = errors.New("tty for runc is only supported on linux")
+
+func updateRuncFieldsForHostOS(runtime *runc.Runc) {}
+
+func (w *runcExecutor) run(ctx context.Context, id, bundle string, process executor.ProcessInfo) error {
+	if process.Meta.Tty {
+		return unsupportedConsoleError
+	}
+	_, err := w.runc.Run(ctx, id, bundle, &runc.CreateOpts{
+		IO:      &forwardIO{stdin: process.Stdin, stdout: process.Stdout, stderr: process.Stderr},
+		NoPivot: w.noPivot,
+	})
+	return err
+}
+
+func (w *runcExecutor) exec(ctx context.Context, id, bundle string, specsProcess *specs.Process, process executor.ProcessInfo) error {
+	if process.Meta.Tty {
+		return unsupportedConsoleError
+	}
+	return w.runc.Exec(ctx, id, *specsProcess, &runc.ExecOpts{
+		IO: &forwardIO{stdin: process.Stdin, stdout: process.Stdout, stderr: process.Stderr},
+	})
+}

+ 160 - 0
vendor/github.com/moby/buildkit/executor/runcexecutor/executor_linux.go

@@ -0,0 +1,160 @@
+package runcexecutor
+
+import (
+	"context"
+	"io"
+	"os"
+	"syscall"
+	"time"
+
+	"github.com/containerd/console"
+	runc "github.com/containerd/go-runc"
+	"github.com/docker/docker/pkg/signal"
+	"github.com/moby/buildkit/executor"
+	"github.com/opencontainers/runtime-spec/specs-go"
+	"github.com/pkg/errors"
+	"github.com/sirupsen/logrus"
+	"golang.org/x/sync/errgroup"
+)
+
+func updateRuncFieldsForHostOS(runtime *runc.Runc) {
+	// PdeathSignal only supported on unix platforms
+	runtime.PdeathSignal = syscall.SIGKILL // this can still leak the process
+}
+
+func (w *runcExecutor) run(ctx context.Context, id, bundle string, process executor.ProcessInfo) error {
+	return w.callWithIO(ctx, id, bundle, process, func(ctx context.Context, started chan<- int, io runc.IO) error {
+		_, err := w.runc.Run(ctx, id, bundle, &runc.CreateOpts{
+			NoPivot: w.noPivot,
+			Started: started,
+			IO:      io,
+		})
+		return err
+	})
+}
+
+func (w *runcExecutor) exec(ctx context.Context, id, bundle string, specsProcess *specs.Process, process executor.ProcessInfo) error {
+	return w.callWithIO(ctx, id, bundle, process, func(ctx context.Context, started chan<- int, io runc.IO) error {
+		return w.runc.Exec(ctx, id, *specsProcess, &runc.ExecOpts{
+			Started: started,
+			IO:      io,
+		})
+	})
+}
+
+type runcCall func(ctx context.Context, started chan<- int, io runc.IO) error
+
+func (w *runcExecutor) callWithIO(ctx context.Context, id, bundle string, process executor.ProcessInfo, call runcCall) error {
+	ctx, cancel := context.WithCancel(ctx)
+	defer cancel()
+
+	if !process.Meta.Tty {
+		return call(ctx, nil, &forwardIO{stdin: process.Stdin, stdout: process.Stdout, stderr: process.Stderr})
+	}
+
+	ptm, ptsName, err := console.NewPty()
+	if err != nil {
+		return err
+	}
+
+	pts, err := os.OpenFile(ptsName, os.O_RDWR|syscall.O_NOCTTY, 0)
+	if err != nil {
+		ptm.Close()
+		return err
+	}
+
+	eg, ctx := errgroup.WithContext(ctx)
+
+	defer func() {
+		if process.Stdin != nil {
+			process.Stdin.Close()
+		}
+		pts.Close()
+		ptm.Close()
+		cancel() // this will shutdown resize loop
+		err := eg.Wait()
+		if err != nil {
+			logrus.Warningf("error while shutting down tty io: %s", err)
+		}
+	}()
+
+	if process.Stdin != nil {
+		eg.Go(func() error {
+			_, err := io.Copy(ptm, process.Stdin)
+			// stdin might be a pipe, so this is like EOF
+			if errors.Is(err, io.ErrClosedPipe) {
+				return nil
+			}
+			return err
+		})
+	}
+
+	if process.Stdout != nil {
+		eg.Go(func() error {
+			_, err := io.Copy(process.Stdout, ptm)
+			// ignore `read /dev/ptmx: input/output error` when ptm is closed
+			var ptmClosedError *os.PathError
+			if errors.As(err, &ptmClosedError) {
+				if ptmClosedError.Op == "read" &&
+					ptmClosedError.Path == "/dev/ptmx" &&
+					ptmClosedError.Err == syscall.EIO {
+					return nil
+				}
+			}
+			return err
+		})
+	}
+
+	started := make(chan int, 1)
+
+	eg.Go(func() error {
+		startedCtx, timeout := context.WithTimeout(ctx, 10*time.Second)
+		defer timeout()
+		var runcProcess *os.Process
+		select {
+		case <-startedCtx.Done():
+			return errors.New("runc started message never received")
+		case pid, ok := <-started:
+			if !ok {
+				return errors.New("runc process failed to send pid")
+			}
+			runcProcess, err = os.FindProcess(pid)
+			if err != nil {
+				return errors.Wrapf(err, "unable to find runc process for pid %d", pid)
+			}
+			defer runcProcess.Release()
+		}
+
+		for {
+			select {
+			case <-ctx.Done():
+				return nil
+			case resize := <-process.Resize:
+				err = ptm.Resize(console.WinSize{
+					Height: uint16(resize.Rows),
+					Width:  uint16(resize.Cols),
+				})
+				if err != nil {
+					logrus.Errorf("failed to resize ptm: %s", err)
+				}
+				err = runcProcess.Signal(signal.SIGWINCH)
+				if err != nil {
+					logrus.Errorf("failed to send SIGWINCH to process: %s", err)
+				}
+			}
+		}
+	})
+
+	runcIO := &forwardIO{}
+	if process.Stdin != nil {
+		runcIO.stdin = pts
+	}
+	if process.Stdout != nil {
+		runcIO.stdout = pts
+	}
+	if process.Stderr != nil {
+		runcIO.stderr = pts
+	}
+
+	return call(ctx, started, runcIO)
+}

+ 49 - 0
vendor/github.com/moby/buildkit/executor/stubs.go

@@ -0,0 +1,49 @@
+package executor
+
+import (
+	"errors"
+	"os"
+	"path/filepath"
+	"syscall"
+
+	"github.com/containerd/continuity/fs"
+)
+
+func MountStubsCleaner(dir string, mounts []Mount) func() {
+	names := []string{"/etc/resolv.conf", "/etc/hosts"}
+
+	for _, m := range mounts {
+		names = append(names, m.Dest)
+	}
+
+	paths := make([]string, 0, len(names))
+
+	for _, p := range names {
+		p = filepath.Join("/", p)
+		if p == "/" {
+			continue
+		}
+		realPath, err := fs.RootPath(dir, p)
+		if err != nil {
+			continue
+		}
+
+		_, err = os.Lstat(realPath)
+		if errors.Is(err, os.ErrNotExist) || errors.Is(err, syscall.ENOTDIR) {
+			paths = append(paths, realPath)
+		}
+	}
+
+	return func() {
+		for _, p := range paths {
+			st, err := os.Lstat(p)
+			if err != nil {
+				continue
+			}
+			if st.Size() != 0 {
+				continue
+			}
+			os.Remove(p)
+		}
+	}
+}

+ 6 - 1
vendor/github.com/moby/buildkit/exporter/containerimage/exptypes/types.go

@@ -1,11 +1,16 @@
 package exptypes
 
-import specs "github.com/opencontainers/image-spec/specs-go/v1"
+import (
+	"github.com/opencontainers/go-digest"
+	specs "github.com/opencontainers/image-spec/specs-go/v1"
+)
 
 const ExporterImageConfigKey = "containerimage.config"
 const ExporterInlineCache = "containerimage.inlinecache"
 const ExporterPlatformsKey = "refs.platforms"
 
+const EmptyGZLayer = digest.Digest("sha256:4f4fb700ef54461cfa02571ae0db9a0dc1e0cdb5577484a6d75e68dc38e8acc1")
+
 type Platforms struct {
 	Platforms []Platform
 }

+ 2 - 2
vendor/github.com/moby/buildkit/exporter/local/export.go

@@ -51,7 +51,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source,
 	timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
 	defer cancel()
 
-	caller, err := e.opt.SessionManager.Get(timeoutCtx, sessionID)
+	caller, err := e.opt.SessionManager.Get(timeoutCtx, sessionID, false)
 	if err != nil {
 		return nil, err
 	}
@@ -70,7 +70,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source,
 				}
 				defer os.RemoveAll(src)
 			} else {
-				mount, err := ref.Mount(ctx, true)
+				mount, err := ref.Mount(ctx, true, session.NewGroup(sessionID))
 				if err != nil {
 					return err
 				}

+ 2 - 2
vendor/github.com/moby/buildkit/exporter/tar/export.go

@@ -65,7 +65,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source,
 			}
 			defers = append(defers, func() { os.RemoveAll(src) })
 		} else {
-			mount, err := ref.Mount(ctx, true)
+			mount, err := ref.Mount(ctx, true, session.NewGroup(sessionID))
 			if err != nil {
 				return nil, err
 			}
@@ -135,7 +135,7 @@ func (e *localExporterInstance) Export(ctx context.Context, inp exporter.Source,
 	timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
 	defer cancel()
 
-	caller, err := e.opt.SessionManager.Get(timeoutCtx, sessionID)
+	caller, err := e.opt.SessionManager.Get(timeoutCtx, sessionID, false)
 	if err != nil {
 		return nil, err
 	}

+ 27 - 5
vendor/github.com/moby/buildkit/frontend/dockerfile/builder/build.go

@@ -14,11 +14,11 @@ import (
 	"strings"
 
 	"github.com/containerd/containerd/platforms"
-	"github.com/docker/docker/builder/dockerignore"
 	controlapi "github.com/moby/buildkit/api/services/control"
 	"github.com/moby/buildkit/client/llb"
 	"github.com/moby/buildkit/exporter/containerimage/exptypes"
 	"github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb"
+	"github.com/moby/buildkit/frontend/dockerfile/dockerignore"
 	"github.com/moby/buildkit/frontend/dockerfile/parser"
 	"github.com/moby/buildkit/frontend/gateway/client"
 	gwpb "github.com/moby/buildkit/frontend/gateway/pb"
@@ -53,16 +53,23 @@ const (
 	keyNameDockerfile          = "dockerfilekey"
 	keyContextSubDir           = "contextsubdir"
 	keyContextKeepGitDir       = "build-arg:BUILDKIT_CONTEXT_KEEP_GIT_DIR"
+	keySyntax                  = "build-arg:BUILDKIT_SYNTAX"
+	keyHostname                = "hostname"
 )
 
 var httpPrefix = regexp.MustCompile(`^https?://`)
-var gitUrlPathWithFragmentSuffix = regexp.MustCompile(`\.git(?:#.+)?$`)
+var gitURLPathWithFragmentSuffix = regexp.MustCompile(`\.git(?:#.+)?$`)
 
 func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 	opts := c.BuildOpts().Opts
 	caps := c.BuildOpts().LLBCaps
 	gwcaps := c.BuildOpts().Caps
 
+	allowForward, capsError := validateCaps(opts["frontend.caps"])
+	if !allowForward && capsError != nil {
+		return nil, capsError
+	}
+
 	marshalOpts := []llb.ConstraintsOpt{llb.WithCaps(caps)}
 
 	localNameContext := DefaultLocalNameContext
@@ -317,8 +324,14 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 	}
 
 	if _, ok := opts["cmdline"]; !ok {
-		ref, cmdline, loc, ok := dockerfile2llb.DetectSyntax(bytes.NewBuffer(dtDockerfile))
-		if ok {
+		if cmdline, ok := opts[keySyntax]; ok {
+			p := strings.SplitN(strings.TrimSpace(cmdline), " ", 2)
+			res, err := forwardGateway(ctx, c, p[0], cmdline)
+			if err != nil && len(errdefs.Sources(err)) == 0 {
+				return nil, errors.Wrapf(err, "failed with %s = %s", keySyntax, cmdline)
+			}
+			return res, err
+		} else if ref, cmdline, loc, ok := dockerfile2llb.DetectSyntax(bytes.NewBuffer(dtDockerfile)); ok {
 			res, err := forwardGateway(ctx, c, ref, cmdline)
 			if err != nil && len(errdefs.Sources(err)) == 0 {
 				return nil, wrapSource(err, sourceMap, loc)
@@ -327,6 +340,14 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 		}
 	}
 
+	if capsError != nil {
+		return nil, capsError
+	}
+
+	if res, ok, err := checkSubRequest(ctx, opts); ok {
+		return res, err
+	}
+
 	exportMap := len(targetPlatforms) > 1
 
 	if v := opts[keyMultiPlatform]; v != "" {
@@ -375,6 +396,7 @@ func Build(ctx context.Context, c client.Client) (*client.Result, error) {
 					OverrideCopyImage: opts[keyOverrideCopyImage],
 					LLBCaps:           &caps,
 					SourceMap:         sourceMap,
+					Hostname:          opts[keyHostname],
 				})
 
 				if err != nil {
@@ -512,7 +534,7 @@ func filter(opt map[string]string, key string) map[string]string {
 
 func detectGitContext(ref, gitContext string) (*llb.State, bool) {
 	found := false
-	if httpPrefix.MatchString(ref) && gitUrlPathWithFragmentSuffix.MatchString(ref) {
+	if httpPrefix.MatchString(ref) && gitURLPathWithFragmentSuffix.MatchString(ref) {
 		found = true
 	}
 

+ 34 - 0
vendor/github.com/moby/buildkit/frontend/dockerfile/builder/caps.go

@@ -0,0 +1,34 @@
+package builder
+
+import (
+	"strings"
+
+	"github.com/moby/buildkit/solver/errdefs"
+	"github.com/moby/buildkit/util/grpcerrors"
+	"github.com/moby/buildkit/util/stack"
+	"google.golang.org/grpc/codes"
+)
+
+var enabledCaps = map[string]struct{}{
+	"moby.buildkit.frontend.inputs":      {},
+	"moby.buildkit.frontend.subrequests": {},
+}
+
+func validateCaps(req string) (forward bool, err error) {
+	if req == "" {
+		return
+	}
+	caps := strings.Split(req, ",")
+	for _, c := range caps {
+		parts := strings.SplitN(c, "+", 2)
+		if _, ok := enabledCaps[parts[0]]; !ok {
+			err = stack.Enable(grpcerrors.WrapCode(errdefs.NewUnsupportedFrontendCapError(parts[0]), codes.Unimplemented))
+			if strings.Contains(c, "+forward") {
+				forward = true
+			} else {
+				return false, err
+			}
+		}
+	}
+	return
+}

+ 39 - 0
vendor/github.com/moby/buildkit/frontend/dockerfile/builder/subrequests.go

@@ -0,0 +1,39 @@
+package builder
+
+import (
+	"context"
+	"encoding/json"
+
+	"github.com/moby/buildkit/frontend/gateway/client"
+	"github.com/moby/buildkit/frontend/subrequests"
+	"github.com/moby/buildkit/solver/errdefs"
+)
+
+func checkSubRequest(ctx context.Context, opts map[string]string) (*client.Result, bool, error) {
+	req, ok := opts["requestid"]
+	if !ok {
+		return nil, false, nil
+	}
+	switch req {
+	case subrequests.RequestSubrequestsDescribe:
+		res, err := describe()
+		return res, true, err
+	default:
+		return nil, true, errdefs.NewUnsupportedSubrequestError(req)
+	}
+}
+
+func describe() (*client.Result, error) {
+	all := []subrequests.Request{
+		subrequests.SubrequestsDescribeDefinition,
+	}
+	dt, err := json.MarshalIndent(all, "  ", "")
+	if err != nil {
+		return nil, err
+	}
+	res := client.NewResult()
+	res.Metadata = map[string][]byte{
+		"result.json": dt,
+	}
+	return res, nil
+}

+ 38 - 24
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert.go

@@ -62,6 +62,7 @@ type ConvertOpt struct {
 	LLBCaps           *apicaps.CapSet
 	ContextLocalName  string
 	SourceMap         *llb.SourceMap
+	Hostname          string
 }
 
 func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State, *Image, error) {
@@ -94,11 +95,13 @@ func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State,
 
 	shlex := shell.NewLex(dockerfile.EscapeToken)
 
-	for _, metaArg := range metaArgs {
-		if metaArg.Value != nil {
-			*metaArg.Value, _ = shlex.ProcessWordWithMap(*metaArg.Value, metaArgsToMap(optMetaArgs))
+	for _, cmd := range metaArgs {
+		for _, metaArg := range cmd.Args {
+			if metaArg.Value != nil {
+				*metaArg.Value, _ = shlex.ProcessWordWithMap(*metaArg.Value, metaArgsToMap(optMetaArgs))
+			}
+			optMetaArgs = append(optMetaArgs, setKVValue(metaArg, opt.BuildArgs))
 		}
-		optMetaArgs = append(optMetaArgs, setKVValue(metaArg.KeyValuePairOptional, opt.BuildArgs))
 	}
 
 	metaResolver := opt.MetaResolver
@@ -314,7 +317,11 @@ func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State,
 
 		// make sure that PATH is always set
 		if _, ok := shell.BuildEnvs(d.image.Config.Env)["PATH"]; !ok {
-			d.image.Config.Env = append(d.image.Config.Env, "PATH="+system.DefaultPathEnv)
+			var os string
+			if d.platform != nil {
+				os = d.platform.OS
+			}
+			d.image.Config.Env = append(d.image.Config.Env, "PATH="+system.DefaultPathEnv(os))
 		}
 
 		// initialize base metadata from image conf
@@ -322,6 +329,9 @@ func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State,
 			k, v := parseKeyValue(env)
 			d.state = d.state.AddEnv(k, v)
 		}
+		if opt.Hostname != "" {
+			d.state = d.state.Hostname(opt.Hostname)
+		}
 		if d.image.Config.WorkingDir != "" {
 			if err = dispatchWorkdir(d, &instructions.WorkdirCommand{Path: d.image.Config.WorkingDir}, false, nil); err != nil {
 				return nil, nil, parser.WithLocation(err, d.stage.Location)
@@ -1072,26 +1082,30 @@ func dispatchShell(d *dispatchState, c *instructions.ShellCommand) error {
 }
 
 func dispatchArg(d *dispatchState, c *instructions.ArgCommand, metaArgs []instructions.KeyValuePairOptional, buildArgValues map[string]string) error {
-	commitStr := "ARG " + c.Key
-	buildArg := setKVValue(c.KeyValuePairOptional, buildArgValues)
+	commitStrs := make([]string, 0, len(c.Args))
+	for _, arg := range c.Args {
+		buildArg := setKVValue(arg, buildArgValues)
 
-	if c.Value != nil {
-		commitStr += "=" + *c.Value
-	}
-	if buildArg.Value == nil {
-		for _, ma := range metaArgs {
-			if ma.Key == buildArg.Key {
-				buildArg.Value = ma.Value
+		commitStr := arg.Key
+		if arg.Value != nil {
+			commitStr += "=" + *arg.Value
+		}
+		commitStrs = append(commitStrs, commitStr)
+		if buildArg.Value == nil {
+			for _, ma := range metaArgs {
+				if ma.Key == buildArg.Key {
+					buildArg.Value = ma.Value
+				}
 			}
 		}
-	}
 
-	if buildArg.Value != nil {
-		d.state = d.state.AddEnv(buildArg.Key, *buildArg.Value)
-	}
+		if buildArg.Value != nil {
+			d.state = d.state.AddEnv(buildArg.Key, *buildArg.Value)
+		}
 
-	d.buildArgs = append(d.buildArgs, buildArg)
-	return commitToHistory(&d.image, commitStr, false, nil)
+		d.buildArgs = append(d.buildArgs, buildArg)
+	}
+	return commitToHistory(&d.image, "ARG "+strings.Join(commitStrs, " "), false, nil)
 }
 
 func pathRelativeToWorkingDir(s llb.State, p string) (string, error) {
@@ -1308,15 +1322,15 @@ func proxyEnvFromBuildArgs(args map[string]string) *llb.ProxyEnv {
 	isNil := true
 	for k, v := range args {
 		if strings.EqualFold(k, "http_proxy") {
-			pe.HttpProxy = v
+			pe.HTTPProxy = v
 			isNil = false
 		}
 		if strings.EqualFold(k, "https_proxy") {
-			pe.HttpsProxy = v
+			pe.HTTPSProxy = v
 			isNil = false
 		}
 		if strings.EqualFold(k, "ftp_proxy") {
-			pe.FtpProxy = v
+			pe.FTPProxy = v
 			isNil = false
 		}
 		if strings.EqualFold(k, "no_proxy") {
@@ -1339,7 +1353,7 @@ func withShell(img Image, args []string) []string {
 	if len(img.Config.Shell) > 0 {
 		shell = append([]string{}, img.Config.Shell...)
 	} else {
-		shell = defaultShell()
+		shell = defaultShell(img.OS)
 	}
 	return append(shell, strings.Join(args, " "))
 }

+ 0 - 16
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_norunmount.go

@@ -1,16 +0,0 @@
-// +build !dfrunmount
-
-package dockerfile2llb
-
-import (
-	"github.com/moby/buildkit/client/llb"
-	"github.com/moby/buildkit/frontend/dockerfile/instructions"
-)
-
-func detectRunMount(cmd *command, allDispatchStates *dispatchStates) bool {
-	return false
-}
-
-func dispatchRunMounts(d *dispatchState, c *instructions.RunCommand, sources []*dispatchState, opt dispatchOpt) ([]llb.RunOption, error) {
-	return nil, nil
-}

+ 0 - 13
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_nosecrets.go

@@ -1,13 +0,0 @@
-// +build dfrunmount,!dfsecrets
-
-package dockerfile2llb
-
-import (
-	"github.com/moby/buildkit/client/llb"
-	"github.com/moby/buildkit/frontend/dockerfile/instructions"
-	"github.com/pkg/errors"
-)
-
-func dispatchSecret(m *instructions.Mount) (llb.RunOption, error) {
-	return nil, errors.Errorf("secret mounts not allowed")
-}

+ 0 - 13
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_nossh.go

@@ -1,13 +0,0 @@
-// +build dfrunmount,!dfssh
-
-package dockerfile2llb
-
-import (
-	"github.com/moby/buildkit/client/llb"
-	"github.com/moby/buildkit/frontend/dockerfile/instructions"
-	"github.com/pkg/errors"
-)
-
-func dispatchSSH(m *instructions.Mount) (llb.RunOption, error) {
-	return nil, errors.Errorf("ssh mounts not allowed")
-}

+ 0 - 2
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_runmount.go

@@ -1,5 +1,3 @@
-// +build dfrunmount
-
 package dockerfile2llb
 
 import (

+ 0 - 2
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_secrets.go

@@ -1,5 +1,3 @@
-// +build dfsecrets
-
 package dockerfile2llb
 
 import (

+ 0 - 2
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/convert_ssh.go

@@ -1,5 +1,3 @@
-// +build dfssh
-
 package dockerfile2llb
 
 import (

+ 8 - 0
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell.go

@@ -0,0 +1,8 @@
+package dockerfile2llb
+
+func defaultShell(os string) []string {
+	if os == "windows" {
+		return []string{"cmd", "/S", "/C"}
+	}
+	return []string{"/bin/sh", "-c"}
+}

+ 0 - 7
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell_unix.go

@@ -1,7 +0,0 @@
-// +build !windows
-
-package dockerfile2llb
-
-func defaultShell() []string {
-	return []string{"/bin/sh", "-c"}
-}

+ 0 - 7
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/defaultshell_windows.go

@@ -1,7 +0,0 @@
-// +build windows
-
-package dockerfile2llb
-
-func defaultShell() []string {
-	return []string{"cmd", "/S", "/C"}
-}

+ 1 - 1
vendor/github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb/image.go

@@ -74,6 +74,6 @@ func emptyImage(platform specs.Platform) Image {
 	}
 	img.RootFS.Type = "layers"
 	img.Config.WorkingDir = "/"
-	img.Config.Env = []string{"PATH=" + system.DefaultPathEnv}
+	img.Config.Env = []string{"PATH=" + system.DefaultPathEnv(platform.OS)}
 	return img
 }

+ 1 - 1
builder/dockerignore/dockerignore.go → vendor/github.com/moby/buildkit/frontend/dockerfile/dockerignore/dockerignore.go

@@ -1,4 +1,4 @@
-package dockerignore // import "github.com/docker/docker/builder/dockerignore"
+package dockerignore
 
 import (
 	"bufio"

+ 16 - 11
vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands.go

@@ -21,8 +21,9 @@ func (kvp *KeyValuePair) String() string {
 
 // KeyValuePairOptional is the same as KeyValuePair but Value is optional
 type KeyValuePairOptional struct {
-	Key   string
-	Value *string
+	Key     string
+	Value   *string
+	Comment string
 }
 
 func (kvpo *KeyValuePairOptional) ValueString() string {
@@ -380,22 +381,25 @@ func (c *StopSignalCommand) CheckPlatform(platform string) error {
 // Dockerfile author may optionally set a default value of this variable.
 type ArgCommand struct {
 	withNameAndCode
-	KeyValuePairOptional
+	Args []KeyValuePairOptional
 }
 
 // Expand variables
 func (c *ArgCommand) Expand(expander SingleWordExpander) error {
-	p, err := expander(c.Key)
-	if err != nil {
-		return err
-	}
-	c.Key = p
-	if c.Value != nil {
-		p, err = expander(*c.Value)
+	for i, v := range c.Args {
+		p, err := expander(v.Key)
 		if err != nil {
 			return err
 		}
-		c.Value = &p
+		v.Key = p
+		if v.Value != nil {
+			p, err = expander(*v.Value)
+			if err != nil {
+				return err
+			}
+			v.Value = &p
+		}
+		c.Args[i] = v
 	}
 	return nil
 }
@@ -416,6 +420,7 @@ type Stage struct {
 	SourceCode string
 	Platform   string
 	Location   []parser.Range
+	Comment    string
 }
 
 // AddCommand to the stage

+ 0 - 7
vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_nosecrets.go

@@ -1,7 +0,0 @@
-// +build !dfsecrets
-
-package instructions
-
-func isSecretMountsSupported() bool {
-	return false
-}

+ 0 - 7
vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_nossh.go

@@ -1,7 +0,0 @@
-// +build !dfssh
-
-package instructions
-
-func isSSHMountsSupported() bool {
-	return false
-}

+ 0 - 2
vendor/github.com/moby/buildkit/frontend/dockerfile/instructions/commands_runmount.go

@@ -1,5 +1,3 @@
-// +build dfrunmount
-
 package instructions
 
 import (

Vissa filer visades inte eftersom för många filer har ändrats