From 2fcce87cf43a0ec1fa40f542aa80412e4bda1344 Mon Sep 17 00:00:00 2001 From: Erik Sipsma Date: Fri, 8 Oct 2021 21:23:53 +0000 Subject: [PATCH 1/4] Include DescHandler CacheOpts during export. Before this, descriptor handlers were not included with calls to the exporter, which then sometimes called LoadRef and failed to get a ref because it was lazy. This change results in the DescHandlers of the already loaded refs to get plugged into context so they can be re-used by the exporter if it needs to load the ref again. Signed-off-by: Erik Sipsma --- cache/refs.go | 9 +++++++++ solver/cacheopts.go | 6 +++++- solver/llbsolver/solver.go | 22 ++++++++++++++++++++++ worker/base/worker.go | 11 ++++++++--- 4 files changed, 44 insertions(+), 4 deletions(-) diff --git a/cache/refs.go b/cache/refs.go index 33134ab07e97..3d7dbb4eee17 100644 --- a/cache/refs.go +++ b/cache/refs.go @@ -34,6 +34,7 @@ type Ref interface { RefMetadata Release(context.Context) error IdentityMapping() *idtools.IdentityMapping + DescHandler(digest.Digest) *DescHandler } type ImmutableRef interface { @@ -313,12 +314,20 @@ type immutableRef struct { descHandlers DescHandlers } +func (sr *immutableRef) DescHandler(dgst digest.Digest) *DescHandler { + return sr.descHandlers[dgst] +} + type mutableRef struct { *cacheRecord triggerLastUsed bool descHandlers DescHandlers } +func (sr *mutableRef) DescHandler(dgst digest.Digest) *DescHandler { + return sr.descHandlers[dgst] +} + func (sr *immutableRef) Clone() ImmutableRef { sr.mu.Lock() ref := sr.ref(false, sr.descHandlers) diff --git a/solver/cacheopts.go b/solver/cacheopts.go index 58094695b9b6..c997c3493fcd 100644 --- a/solver/cacheopts.go +++ b/solver/cacheopts.go @@ -21,8 +21,12 @@ func CacheOptGetterOf(ctx context.Context) func(keys ...interface{}) map[interfa return nil } +func WithCacheOptGetter(ctx context.Context, getter func(keys ...interface{}) map[interface{}]interface{}) context.Context { + return context.WithValue(ctx, cacheOptGetterKey{}, getter) +} + func withAncestorCacheOpts(ctx context.Context, start *state) context.Context { - return context.WithValue(ctx, cacheOptGetterKey{}, func(keys ...interface{}) map[interface{}]interface{} { + return WithCacheOptGetter(ctx, func(keys ...interface{}) map[interface{}]interface{} { keySet := make(map[interface{}]struct{}) for _, k := range keys { keySet[k] = struct{}{} diff --git a/solver/llbsolver/solver.go b/solver/llbsolver/solver.go index 6a2a4e002b51..35422b4b18ed 100644 --- a/solver/llbsolver/solver.go +++ b/solver/llbsolver/solver.go @@ -245,6 +245,13 @@ func (s *Solver) Solve(ctx context.Context, id string, sessionID string, req fro if err != nil { return err } + + workerRef, ok := r.Sys().(*worker.WorkerRef) + if !ok { + return errors.Errorf("invalid reference: %T", r.Sys()) + } + ctx = withDescHandlerCacheOpts(ctx, workerRef.ImmutableRef) + // all keys have same export chain so exporting others is not needed _, err = r.CacheKeys()[0].Exporter.ExportTo(ctx, e, solver.CacheExportOpt{ ResolveRemotes: workerRefResolver(solver.CompressionOpt{ @@ -308,6 +315,7 @@ func inlineCache(ctx context.Context, e remotecache.Exporter, res solver.CachedR digests = append(digests, desc.Digest) } + ctx = withDescHandlerCacheOpts(ctx, workerRef.ImmutableRef) if _, err := res.CacheKeys()[0].Exporter.ExportTo(ctx, e, solver.CacheExportOpt{ ResolveRemotes: workerRefResolver(compressionopt, true, g), // load as many compression blobs as possible Mode: solver.CacheExportModeMin, @@ -322,6 +330,20 @@ func inlineCache(ctx context.Context, e remotecache.Exporter, res solver.CachedR return nil, nil } +func withDescHandlerCacheOpts(ctx context.Context, ref cache.ImmutableRef) context.Context { + return solver.WithCacheOptGetter(ctx, func(keys ...interface{}) map[interface{}]interface{} { + vals := make(map[interface{}]interface{}) + for _, k := range keys { + if key, ok := k.(cache.DescHandlerKey); ok { + if handler := ref.DescHandler(digest.Digest(key)); handler != nil { + vals[k] = handler + } + } + } + return vals + }) +} + func (s *Solver) Status(ctx context.Context, id string, statusChan chan *client.SolveStatus) error { j, err := s.solver.Get(id) if err != nil { diff --git a/worker/base/worker.go b/worker/base/worker.go index 8aa5e3066565..f3370df63916 100644 --- a/worker/base/worker.go +++ b/worker/base/worker.go @@ -232,15 +232,20 @@ func (w *Worker) LoadRef(ctx context.Context, id string, hidden bool) (cache.Imm } descHandlers := cache.DescHandlers(make(map[digest.Digest]*cache.DescHandler)) for k, v := range optGetter(keys...) { - if v, ok := v.(*cache.DescHandler); ok { - descHandlers[k.(digest.Digest)] = v + if key, ok := k.(cache.DescHandlerKey); ok { + if handler, ok := v.(*cache.DescHandler); ok { + descHandlers[digest.Digest(key)] = handler + } } } opts = append(opts, descHandlers) ref, err = w.CacheMgr.Get(ctx, id, opts...) } } - return ref, err + if err != nil { + return nil, errors.Wrap(err, "failed to load ref") + } + return ref, nil } func (w *Worker) Executor() executor.Executor { From 03ed0548ef763331a4a69b8a29c7395fc63b999c Mon Sep 17 00:00:00 2001 From: Erik Sipsma Date: Thu, 28 Oct 2021 12:38:08 -0700 Subject: [PATCH 2/4] cache: Replace Parent method with LayerChain. The Parent method will no longer make sense with forthcoming Merge and Diff support as refs will become capable of having multiple parents. It was also only ever used externally to get the full chain of refs for each layer in the ref's chain. The newly added LayerChain method replaces Parents with a method that just returns a slice of refs for each layer in the ref's chain. This will work more seamlessly with Merge and Diff (in which case it returns the "flattened" ancestors of the ref) in addition to being a bit easier to use for the exiting cases anyways. Signed-off-by: Erik Sipsma --- cache/refs.go | 35 +++++++++++++++++++++------- exporter/containerimage/writer.go | 38 +++++++++++++++++-------------- source/containerimage/pull.go | 24 ++++++------------- 3 files changed, 55 insertions(+), 42 deletions(-) diff --git a/cache/refs.go b/cache/refs.go index 3d7dbb4eee17..7b466702fb42 100644 --- a/cache/refs.go +++ b/cache/refs.go @@ -13,6 +13,7 @@ import ( "github.com/containerd/containerd/mount" "github.com/containerd/containerd/snapshots" "github.com/docker/docker/pkg/idtools" + "github.com/hashicorp/go-multierror" "github.com/moby/buildkit/identity" "github.com/moby/buildkit/session" "github.com/moby/buildkit/snapshot" @@ -39,7 +40,6 @@ type Ref interface { type ImmutableRef interface { Ref - Parent() ImmutableRef Clone() ImmutableRef // Finalize commits the snapshot to the driver if it's not already. // This means the snapshot can no longer be mounted as mutable. @@ -47,6 +47,7 @@ type ImmutableRef interface { Extract(ctx context.Context, s session.Group) error // +progress GetRemotes(ctx context.Context, createIfNeeded bool, compressionopt solver.CompressionOpt, all bool, s session.Group) ([]*solver.Remote, error) + LayerChain() RefList } type MutableRef interface { @@ -314,6 +315,31 @@ type immutableRef struct { descHandlers DescHandlers } +type RefList []ImmutableRef + +func (l RefList) Release(ctx context.Context) (rerr error) { + for i, r := range l { + if r == nil { + continue + } + if err := r.Release(ctx); err != nil { + rerr = multierror.Append(rerr, err).ErrorOrNil() + } else { + l[i] = nil + } + } + return rerr +} + +func (sr *immutableRef) LayerChain() RefList { + chain := sr.parentRefChain() + l := RefList(make([]ImmutableRef, len(chain))) + for i, p := range chain { + l[i] = p.Clone() + } + return l +} + func (sr *immutableRef) DescHandler(dgst digest.Digest) *DescHandler { return sr.descHandlers[dgst] } @@ -335,13 +361,6 @@ func (sr *immutableRef) Clone() ImmutableRef { 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) ociDesc(ctx context.Context, dhs DescHandlers) (ocispecs.Descriptor, error) { desc := ocispecs.Descriptor{ Digest: sr.getBlob(), diff --git a/exporter/containerimage/writer.go b/exporter/containerimage/writer.go index 54749d0612e8..8c8faa760e36 100644 --- a/exporter/containerimage/writer.go +++ b/exporter/containerimage/writer.go @@ -521,27 +521,31 @@ type refMetadata struct { } func getRefMetadata(ref cache.ImmutableRef, limit int) []refMetadata { - if limit <= 0 { - return nil - } - now := time.Now() - meta := refMetadata{ - description: "created by buildkit", // shouldn't be shown but don't fail build - createdAt: &now, - } if ref == nil { - return append(getRefMetadata(nil, limit-1), meta) + return make([]refMetadata, limit) } - if descr := ref.GetDescription(); descr != "" { - meta.description = descr + + layerChain := ref.LayerChain() + defer layerChain.Release(context.TODO()) + + if limit < len(layerChain) { + layerChain = layerChain[len(layerChain)-limit:] } - createdAt := ref.GetCreatedAt() - meta.createdAt = &createdAt - p := ref.Parent() - if p != nil { - defer p.Release(context.TODO()) + + metas := make([]refMetadata, len(layerChain)) + for i, layer := range layerChain { + meta := &metas[i] + + if description := layer.GetDescription(); description != "" { + meta.description = description + } else { + meta.description = "created by buildkit" // shouldn't be shown but don't fail build + } + + createdAt := layer.GetCreatedAt() + meta.createdAt = &createdAt } - return append(getRefMetadata(p, limit-1), meta) + return metas } func oneOffProgress(ctx context.Context, id string) func(err error) error { diff --git a/source/containerimage/pull.go b/source/containerimage/pull.go index 902f8f547ea1..1162f1c338de 100644 --- a/source/containerimage/pull.go +++ b/source/containerimage/pull.go @@ -33,7 +33,7 @@ import ( "github.com/moby/buildkit/util/progress/controller" "github.com/moby/buildkit/util/pull" "github.com/moby/buildkit/util/resolver" - digest "github.com/opencontainers/go-digest" + "github.com/opencontainers/go-digest" "github.com/opencontainers/image-spec/identity" ocispecs "github.com/opencontainers/image-spec/specs-go/v1" "github.com/pkg/errors" @@ -304,6 +304,7 @@ func (p *puller) Snapshot(ctx context.Context, g session.Group) (ir cache.Immuta }() var parent cache.ImmutableRef + setWindowsLayerType := p.Platform.OS == "windows" && runtime.GOOS != "windows" for _, layerDesc := range p.manifest.Descriptors { parent = current current, err = p.CacheAccessor.GetByBlob(ctx, layerDesc, parent, @@ -314,6 +315,11 @@ func (p *puller) Snapshot(ctx context.Context, g session.Group) (ir cache.Immuta if err != nil { return nil, err } + if setWindowsLayerType { + if err := current.SetLayerType("windows"); err != nil { + return nil, err + } + } } for _, desc := range p.manifest.Nonlayers { @@ -340,12 +346,6 @@ func (p *puller) Snapshot(ctx context.Context, g session.Group) (ir cache.Immuta } } - if current != nil && p.Platform.OS == "windows" && runtime.GOOS != "windows" { - if err := markRefLayerTypeWindows(current); err != nil { - return nil, err - } - } - if p.id.RecordType != "" && current.GetRecordType() == "" { if err := current.SetRecordType(p.id.RecordType); err != nil { return nil, err @@ -355,16 +355,6 @@ func (p *puller) Snapshot(ctx context.Context, g session.Group) (ir cache.Immuta return current, nil } -func markRefLayerTypeWindows(ref cache.ImmutableRef) error { - if parent := ref.Parent(); parent != nil { - defer parent.Release(context.TODO()) - if err := markRefLayerTypeWindows(parent); err != nil { - return err - } - } - return ref.SetLayerType("windows") -} - // cacheKeyFromConfig returns a stable digest from image config. If image config // is a known oci image we will use chainID of layers. func cacheKeyFromConfig(dt []byte) digest.Digest { From 9321ec2f82306a66dffe3a4c9a06da3563125d83 Mon Sep 17 00:00:00 2001 From: Erik Sipsma Date: Thu, 28 Oct 2021 12:59:26 -0700 Subject: [PATCH 3/4] Refactor cache record mount cache. This is mostly just preparation for merge-op. The existing Extract method is updated to be usable for unlazying any type of refs rather than just lazy blobs. The way views are created is simplified and centralized in one location. Signed-off-by: Erik Sipsma --- cache/contenthash/checksum_test.go | 3 +- cache/manager.go | 25 +- cache/refs.go | 365 +++++++++++++++----------- snapshot/snapshotter.go | 25 -- snapshot/staticmountable.go | 36 +++ solver/llbsolver/mounts/mount_test.go | 4 +- source/git/gitsource_test.go | 15 +- source/http/httpsource_test.go | 5 +- worker/runc/runc.go | 2 +- worker/runc/runc_test.go | 20 +- 10 files changed, 283 insertions(+), 217 deletions(-) create mode 100644 snapshot/staticmountable.go diff --git a/cache/contenthash/checksum_test.go b/cache/contenthash/checksum_test.go index 4d01f1e0278e..8d8efdf34774 100644 --- a/cache/contenthash/checksum_test.go +++ b/cache/contenthash/checksum_test.go @@ -1208,11 +1208,12 @@ func setupCacheManager(t *testing.T, tmpdir string, snapshotterName string, snap md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) require.NoError(t, err) + lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter(snapshotterName, containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter(snapshotterName)), nil), MetadataStore: md, - LeaseManager: leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit"), + LeaseManager: lm, ContentStore: mdb.ContentStore(), GarbageCollect: mdb.GarbageCollect, }) diff --git a/cache/manager.go b/cache/manager.go index 8ddcfe8dfbc0..8399262afb3c 100644 --- a/cache/manager.go +++ b/cache/manager.go @@ -216,10 +216,10 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, } }() - if err := cm.ManagerOpt.LeaseManager.AddResource(ctx, l, leases.Resource{ + if err := cm.LeaseManager.AddResource(ctx, l, leases.Resource{ ID: snapshotID, - Type: "snapshots/" + cm.ManagerOpt.Snapshotter.Name(), - }); err != nil { + Type: "snapshots/" + cm.Snapshotter.Name(), + }); err != nil && !errdefs.IsAlreadyExists(err) { return nil, errors.Wrapf(err, "failed to add snapshot %s to lease", id) } @@ -504,24 +504,25 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr } }() - if err := cm.ManagerOpt.LeaseManager.AddResource(ctx, l, leases.Resource{ - ID: id, - Type: "snapshots/" + cm.ManagerOpt.Snapshotter.Name(), - }); err != nil { - return nil, errors.Wrapf(err, "failed to add snapshot %s to lease", id) + snapshotID := id + if err := cm.LeaseManager.AddResource(ctx, leases.Lease{ID: id}, leases.Resource{ + ID: snapshotID, + Type: "snapshots/" + cm.Snapshotter.Name(), + }); err != nil && !errdefs.IsAlreadyExists(err) { + return nil, errors.Wrapf(err, "failed to add snapshot %s to lease", snapshotID) } if cm.Snapshotter.Name() == "stargz" && parent != nil { if rerr := parent.withRemoteSnapshotLabelsStargzMode(ctx, sess, func() { - err = cm.Snapshotter.Prepare(ctx, id, parentSnapshotID) + err = cm.Snapshotter.Prepare(ctx, snapshotID, parentSnapshotID) }); rerr != nil { return nil, rerr } } else { - err = cm.Snapshotter.Prepare(ctx, id, parentSnapshotID) + err = cm.Snapshotter.Prepare(ctx, snapshotID, parentSnapshotID) } if err != nil { - return nil, errors.Wrapf(err, "failed to prepare %s", id) + return nil, errors.Wrapf(err, "failed to prepare %s", parentSnapshotID) } cm.mu.Lock() @@ -538,7 +539,7 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr cacheMetadata: md, } - opts = append(opts, withSnapshotID(id)) + opts = append(opts, withSnapshotID(snapshotID)) if err := initializeMetadata(rec.cacheMetadata, parentID, opts...); err != nil { return nil, err } diff --git a/cache/refs.go b/cache/refs.go index 7b466702fb42..cc595cd55993 100644 --- a/cache/refs.go +++ b/cache/refs.go @@ -75,8 +75,7 @@ type cacheRecord struct { // dead means record is marked as deleted dead bool - view string - viewMount snapshot.Mountable + mountCache snapshot.Mountable sizeG flightcontrol.Group @@ -164,6 +163,14 @@ func (cr *cacheRecord) IdentityMapping() *idtools.IdentityMapping { return cr.cm.IdentityMapping() } +func (cr *cacheRecord) viewLeaseID() string { + return cr.ID() + "-view" +} + +func (cr *cacheRecord) viewSnapshotID() string { + return cr.getSnapshotID() + "-view" +} + func (cr *cacheRecord) size(ctx context.Context) (int64, error) { // this expects that usage() is implemented lazily s, err := cr.sizeG.Do(ctx, cr.ID(), func(ctx context.Context) (interface{}, error) { @@ -239,52 +246,59 @@ func (cr *cacheRecord) parentRef(hidden bool, descHandlers DescHandlers) *immuta return p.ref(hidden, descHandlers) } -// 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, cr.getSnapshotID()) - if err != nil { - return nil, errors.Wrapf(err, "failed to mount %s", cr.ID()) - } - if readonly { - m = setReadonly(m) - } - return m, nil - } - - if cr.equalMutable != nil && readonly { - m, err := cr.cm.Snapshotter.Mounts(ctx, cr.equalMutable.getSnapshotID()) - if err != nil { - return nil, errors.Wrapf(err, "failed to mount %s", cr.equalMutable.ID()) - } - return setReadonly(m), nil +// caller must hold cr.mu +func (cr *cacheRecord) mount(ctx context.Context, s session.Group) (_ snapshot.Mountable, rerr error) { + if cr.mountCache != nil { + return cr.mountCache, nil } - if err := cr.finalize(ctx); err != nil { - return nil, err - } - if cr.viewMount == nil { // TODO: handle this better - view := identity.NewID() - l, err := cr.cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { - l.ID = view + var mountSnapshotID string + if cr.mutable { + mountSnapshotID = cr.getSnapshotID() + } else if cr.equalMutable != nil { + mountSnapshotID = cr.equalMutable.getSnapshotID() + } else { + mountSnapshotID = cr.viewSnapshotID() + if _, err := cr.cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { + l.ID = cr.viewLeaseID() l.Labels = map[string]string{ "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), } return nil - }, leaseutil.MakeTemporary) - if err != nil { + }, leaseutil.MakeTemporary); err != nil && !errdefs.IsAlreadyExists(err) { return nil, err } - ctx = leases.WithLease(ctx, l.ID) - m, err := cr.cm.Snapshotter.View(ctx, view, cr.getSnapshotID()) - if err != nil { - cr.cm.LeaseManager.Delete(context.TODO(), leases.Lease{ID: l.ID}) - return nil, errors.Wrapf(err, "failed to mount %s", cr.ID()) + defer func() { + if rerr != nil { + cr.cm.LeaseManager.Delete(context.TODO(), leases.Lease{ID: cr.viewLeaseID()}) + } + }() + if err := cr.cm.LeaseManager.AddResource(ctx, leases.Lease{ID: cr.viewLeaseID()}, leases.Resource{ + ID: mountSnapshotID, + Type: "snapshots/" + cr.cm.Snapshotter.Name(), + }); err != nil && !errdefs.IsAlreadyExists(err) { + return nil, err + } + // Return the mount direct from View rather than setting it using the Mounts call below. + // The two are equivalent for containerd snapshotters but the moby snapshotter requires + // the use of the mountable returned by View in this case. + mnts, err := cr.cm.Snapshotter.View(ctx, mountSnapshotID, cr.getSnapshotID()) + if err != nil && !errdefs.IsAlreadyExists(err) { + return nil, err } - cr.view = view - cr.viewMount = m + cr.mountCache = mnts } - return cr.viewMount, nil + + if cr.mountCache != nil { + return cr.mountCache, nil + } + + mnts, err := cr.cm.Snapshotter.Mounts(ctx, mountSnapshotID) + if err != nil { + return nil, err + } + cr.mountCache = mnts + return cr.mountCache, nil } // call when holding the manager lock @@ -299,8 +313,10 @@ func (cr *cacheRecord) remove(ctx context.Context, removeSnapshot bool) error { } } if removeSnapshot { - if err := cr.cm.LeaseManager.Delete(context.TODO(), leases.Lease{ID: cr.ID()}); err != nil { - return errors.Wrapf(err, "failed to remove %s", cr.ID()) + if err := cr.cm.LeaseManager.Delete(ctx, leases.Lease{ + ID: cr.ID(), + }); err != nil && !errdefs.IsNotFound(err) { + return errors.Wrapf(err, "failed to delete lease for %s", cr.ID()) } } if err := cr.cm.MetadataStore.Clear(cr.ID()); err != nil { @@ -362,6 +378,11 @@ func (sr *immutableRef) Clone() ImmutableRef { } func (sr *immutableRef) ociDesc(ctx context.Context, dhs DescHandlers) (ocispecs.Descriptor, error) { + dgst := sr.getBlob() + if dgst == "" { + return ocispecs.Descriptor{}, errors.Errorf("no blob set for cache record %s", sr.ID()) + } + desc := ocispecs.Descriptor{ Digest: sr.getBlob(), Size: sr.getBlobSize(), @@ -446,7 +467,7 @@ func getCompressionVariantBlob(ctx context.Context, cs content.Store, dgst diges func (sr *immutableRef) addCompressionBlob(ctx context.Context, desc ocispecs.Descriptor, compressionType compression.Type) error { cs := sr.cm.ContentStore - if err := sr.cm.ManagerOpt.LeaseManager.AddResource(ctx, leases.Lease{ID: sr.ID()}, leases.Resource{ + if err := sr.cm.LeaseManager.AddResource(ctx, leases.Lease{ID: sr.ID()}, leases.Resource{ ID: desc.Digest.String(), Type: "content", }); err != nil { @@ -544,7 +565,13 @@ func (sr *immutableRef) parentRefChain() []*immutableRef { return refs } -func (sr *immutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (snapshot.Mountable, error) { +func (sr *immutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (_ snapshot.Mountable, rerr error) { + if sr.equalMutable != nil && !readonly { + if err := sr.Finalize(ctx); err != nil { + return nil, err + } + } + if err := sr.Extract(ctx, s); err != nil { return nil, err } @@ -552,35 +579,33 @@ func (sr *immutableRef) Mount(ctx context.Context, readonly bool, s session.Grou sr.mu.Lock() defer sr.mu.Unlock() + if sr.mountCache != nil { + return sr.mountCache, nil + } + + var mnt snapshot.Mountable if sr.cm.Snapshotter.Name() == "stargz" { - var ( - m snapshot.Mountable - rerr error - ) if err := sr.withRemoteSnapshotLabelsStargzMode(ctx, s, func() { - m, rerr = sr.mount(ctx, readonly) + mnt, rerr = sr.mount(ctx, s) }); err != nil { return nil, err } - return m, rerr + } else { + mnt, rerr = sr.mount(ctx, s) + } + if rerr != nil { + return nil, rerr } - return sr.mount(ctx, readonly) + if readonly { + mnt = setReadonly(mnt) + } + return mnt, nil } func (sr *immutableRef) Extract(ctx context.Context, s session.Group) (rerr error) { if !sr.getBlobOnly() { - return - } - - ctx, done, err := leaseutil.WithLease(ctx, sr.cm.LeaseManager, leaseutil.MakeTemporary) - if err != nil { - return err - } - defer done(ctx) - - if sr.GetLayerType() == "windows" { - ctx = winlayers.UseWindowsLayerMode(ctx) + return nil } if sr.cm.Snapshotter.Name() == "stargz" { @@ -588,14 +613,14 @@ func (sr *immutableRef) Extract(ctx context.Context, s session.Group) (rerr erro if rerr = sr.prepareRemoteSnapshotsStargzMode(ctx, s); rerr != nil { return } - rerr = sr.extract(ctx, sr.descHandlers, s) + rerr = sr.unlazy(ctx, sr.descHandlers, s) }); err != nil { return err } return rerr } - return sr.extract(ctx, sr.descHandlers, s) + return sr.unlazy(ctx, sr.descHandlers, s) } func (sr *immutableRef) withRemoteSnapshotLabelsStargzMode(ctx context.Context, s session.Group, f func()) error { @@ -726,94 +751,115 @@ func makeTmpLabelsStargzMode(labels map[string]string, s session.Group) (fields return } -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 := sr.getSnapshotID() - if _, err := sr.cm.Snapshotter.Stat(ctx, snapshotID); err == nil { +func (sr *immutableRef) unlazy(ctx context.Context, dhs DescHandlers, s session.Group) error { + _, err := sr.sizeG.Do(ctx, sr.ID()+"-unlazy", func(ctx context.Context) (_ interface{}, rerr error) { + if _, err := sr.cm.Snapshotter.Stat(ctx, sr.getSnapshotID()); err == nil { return nil, nil } - if sr.cm.Applier == nil { - return nil, errors.New("extract requires an applier") - } + return nil, sr.unlazyLayer(ctx, dhs, s) + }) + return err +} - eg, egctx := errgroup.WithContext(ctx) +// should be called within sizeG.Do call for this ref's ID +func (sr *immutableRef) unlazyLayer(ctx context.Context, dhs DescHandlers, s session.Group) (rerr error) { + if !sr.getBlobOnly() { + return nil + } - parentID := "" - if sr.parent != nil { - eg.Go(func() error { - if err := sr.parent.extract(egctx, dhs, s); err != nil { - return err - } - parentID = sr.parent.getSnapshotID() - return nil - }) - } + if sr.cm.Applier == nil { + return errors.New("unlazy requires an applier") + } - desc, err := sr.ociDesc(ctx, dhs) + if _, ok := leases.FromContext(ctx); !ok { + leaseCtx, done, err := leaseutil.WithLease(ctx, sr.cm.LeaseManager, leaseutil.MakeTemporary) if err != nil { - return nil, err + return err } - dh := dhs[desc.Digest] + defer done(leaseCtx) + ctx = leaseCtx + } + + if sr.GetLayerType() == "windows" { + ctx = winlayers.UseWindowsLayerMode(ctx) + } + eg, egctx := errgroup.WithContext(ctx) + + parentID := "" + if sr.parent != nil { 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 := sr.parent.unlazy(egctx, dhs, s); err != nil { + return err + } + parentID = sr.parent.getSnapshotID() + return nil }) + } - if err := eg.Wait(); err != nil { - return nil, err - } + desc, err := sr.ociDesc(ctx, dhs) + if err != nil { + return err + } + dh := dhs[desc.Digest] - 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() - } + eg.Go(func() error { + // unlazies if needed, otherwise a no-op + return lazyRefProvider{ + ref: sr, + desc: desc, + dh: dh, + session: s, + }.Unlazy(egctx) + }) - key := fmt.Sprintf("extract-%s %s", identity.NewID(), sr.getChainID()) + if err := eg.Wait(); err != nil { + return err + } - err = sr.cm.Snapshotter.Prepare(ctx, key, parentID) - if 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() + } - mountable, err := sr.cm.Snapshotter.Mounts(ctx, key) - if err != nil { - return nil, err - } - mounts, unmount, err := mountable.Mount() - if err != nil { - return nil, err - } - _, err = sr.cm.Applier.Apply(ctx, desc, mounts) - if err != nil { - unmount() - return nil, err - } + key := fmt.Sprintf("extract-%s %s", identity.NewID(), sr.getChainID()) - if err := unmount(); err != nil { - return nil, err - } - if err := sr.cm.Snapshotter.Commit(ctx, sr.getSnapshotID(), key); err != nil { - if !errors.Is(err, errdefs.ErrAlreadyExists) { - return nil, err - } - } - sr.queueBlobOnly(false) - sr.queueSize(sizeUnknown) - if err := sr.commitMetadata(); err != nil { - return nil, err + err = sr.cm.Snapshotter.Prepare(ctx, key, parentID) + if err != nil { + return err + } + + mountable, err := sr.cm.Snapshotter.Mounts(ctx, key) + if err != nil { + return err + } + mounts, unmount, err := mountable.Mount() + if err != nil { + return err + } + _, err = sr.cm.Applier.Apply(ctx, desc, mounts) + if err != nil { + unmount() + return err + } + + if err := unmount(); err != nil { + return err + } + if err := sr.cm.Snapshotter.Commit(ctx, sr.getSnapshotID(), key); err != nil { + if !errors.Is(err, errdefs.ErrAlreadyExists) { + return err } - return nil, nil - }) - return err + } + sr.queueBlobOnly(false) + sr.queueSize(sizeUnknown) + if err := sr.commitMetadata(); err != nil { + return err + } + return nil } func (sr *immutableRef) Release(ctx context.Context) error { @@ -853,16 +899,13 @@ func (sr *immutableRef) release(ctx context.Context) error { } if len(sr.refs) == 0 { - if sr.viewMount != nil { // TODO: release viewMount earlier if possible - if err := sr.cm.LeaseManager.Delete(ctx, leases.Lease{ID: sr.view}); err != nil { - return errors.Wrapf(err, "failed to remove view lease %s", sr.view) - } - sr.view = "" - sr.viewMount = nil - } - if sr.equalMutable != nil { sr.equalMutable.release(ctx) + } else { + if err := sr.cm.LeaseManager.Delete(ctx, leases.Lease{ID: sr.viewLeaseID()}); err != nil && !errdefs.IsNotFound(err) { + return err + } + sr.mountCache = nil } } @@ -895,19 +938,20 @@ func (cr *cacheRecord) finalize(ctx context.Context) error { } } - if err := cr.cm.ManagerOpt.LeaseManager.AddResource(ctx, leases.Lease{ID: cr.ID()}, leases.Resource{ - ID: cr.ID(), - Type: "snapshots/" + cr.cm.ManagerOpt.Snapshotter.Name(), + if err := cr.cm.LeaseManager.AddResource(ctx, leases.Lease{ID: cr.ID()}, leases.Resource{ + ID: cr.getSnapshotID(), + Type: "snapshots/" + cr.cm.Snapshotter.Name(), }); err != nil { cr.cm.LeaseManager.Delete(context.TODO(), leases.Lease{ID: cr.ID()}) - return errors.Wrapf(err, "failed to add snapshot %s to lease", cr.ID()) + return errors.Wrapf(err, "failed to add snapshot %s to lease", cr.getSnapshotID()) } - err = cr.cm.Snapshotter.Commit(ctx, cr.ID(), mutable.ID()) - if err != nil { + if err := cr.cm.Snapshotter.Commit(ctx, cr.getSnapshotID(), mutable.getSnapshotID()); err != nil { cr.cm.LeaseManager.Delete(context.TODO(), leases.Lease{ID: cr.ID()}) - return errors.Wrapf(err, "failed to commit %s", mutable.ID()) + return errors.Wrapf(err, "failed to commit %s to %s", mutable.getSnapshotID(), cr.getSnapshotID()) } + cr.mountCache = nil + mutable.dead = true go func() { cr.cm.mu.Lock() @@ -919,7 +963,6 @@ func (cr *cacheRecord) finalize(ctx context.Context) error { cr.equalMutable = nil cr.clearEqualMutable() - cr.queueSnapshotID(cr.ID()) return cr.commitMetadata() } @@ -965,7 +1008,7 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) { md.queueCommitted(true) md.queueSize(sizeUnknown) - md.queueSnapshotID(sr.getSnapshotID()) + md.queueSnapshotID(id) md.setEqualMutable(sr.ID()) if err := md.commitMetadata(); err != nil { return nil, err @@ -976,24 +1019,32 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) { return ref, nil } -func (sr *mutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (snapshot.Mountable, error) { +func (sr *mutableRef) Mount(ctx context.Context, readonly bool, s session.Group) (_ snapshot.Mountable, rerr error) { sr.mu.Lock() defer sr.mu.Unlock() + if sr.mountCache != nil { + return sr.mountCache, nil + } + + var mnt snapshot.Mountable if sr.cm.Snapshotter.Name() == "stargz" && sr.parent != nil { - var ( - m snapshot.Mountable - rerr error - ) if err := sr.parent.withRemoteSnapshotLabelsStargzMode(ctx, s, func() { - m, rerr = sr.mount(ctx, readonly) + mnt, rerr = sr.mount(ctx, s) }); err != nil { return nil, err } - return m, rerr + } else { + mnt, rerr = sr.mount(ctx, s) + } + if rerr != nil { + return nil, rerr } - return sr.mount(ctx, readonly) + if readonly { + mnt = setReadonly(mnt) + } + return mnt, nil } func (sr *mutableRef) Commit(ctx context.Context) (ImmutableRef, error) { diff --git a/snapshot/snapshotter.go b/snapshot/snapshotter.go index a4bf1c4f7751..99745098af37 100644 --- a/snapshot/snapshotter.go +++ b/snapshot/snapshotter.go @@ -2,9 +2,7 @@ package snapshot import ( "context" - "os" "sync" - "sync/atomic" "github.com/containerd/containerd/mount" "github.com/containerd/containerd/snapshots" @@ -70,29 +68,6 @@ func (s *fromContainerd) IdentityMapping() *idtools.IdentityMapping { return s.idmap } -type staticMountable struct { - count int32 - id string - mounts []mount.Mount - idmap *idtools.IdentityMapping -} - -func (cm *staticMountable) Mount() ([]mount.Mount, func() error, error) { - atomic.AddInt32(&cm.count, 1) - return cm.mounts, func() error { - if atomic.AddInt32(&cm.count, -1) < 0 { - if v := os.Getenv("BUILDKIT_DEBUG_PANIC_ON_ERROR"); v == "1" { - panic("release of released mount " + cm.id) - } - } - return nil - }, nil -} - -func (cm *staticMountable) IdentityMapping() *idtools.IdentityMapping { - return cm.idmap -} - // NewContainerdSnapshotter converts snapshotter to containerd snapshotter func NewContainerdSnapshotter(s Snapshotter) (snapshots.Snapshotter, func() error) { cs := &containerdSnapshotter{Snapshotter: s} diff --git a/snapshot/staticmountable.go b/snapshot/staticmountable.go new file mode 100644 index 000000000000..dd9443006fae --- /dev/null +++ b/snapshot/staticmountable.go @@ -0,0 +1,36 @@ +package snapshot + +import ( + "os" + "sync/atomic" + + "github.com/containerd/containerd/mount" + "github.com/docker/docker/pkg/idtools" +) + +type staticMountable struct { + count int32 + id string + mounts []mount.Mount + idmap *idtools.IdentityMapping +} + +func (cm *staticMountable) Mount() ([]mount.Mount, func() error, error) { + // return a copy to prevent changes to mount.Mounts in the slice from affecting cm + mounts := make([]mount.Mount, len(cm.mounts)) + copy(mounts, cm.mounts) + + atomic.AddInt32(&cm.count, 1) + return cm.mounts, func() error { + if atomic.AddInt32(&cm.count, -1) < 0 { + if v := os.Getenv("BUILDKIT_DEBUG_PANIC_ON_ERROR"); v == "1" { + panic("release of released mount " + cm.id) + } + } + return nil + }, nil +} + +func (cm *staticMountable) IdentityMapping() *idtools.IdentityMapping { + return cm.idmap +} diff --git a/solver/llbsolver/mounts/mount_test.go b/solver/llbsolver/mounts/mount_test.go index 2d78ce458b44..b018a3ab0b98 100644 --- a/solver/llbsolver/mounts/mount_test.go +++ b/solver/llbsolver/mounts/mount_test.go @@ -108,7 +108,7 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() return nil, nil, err } - lm := ctdmetadata.NewLeaseManager(mdb) + lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), ns) md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) if err != nil { @@ -119,7 +119,7 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() Snapshotter: snapshot.FromContainerdSnapshotter(opt.snapshotterName, containerdsnapshot.NSSnapshotter(ns, mdb.Snapshotter(opt.snapshotterName)), nil), MetadataStore: md, ContentStore: mdb.ContentStore(), - LeaseManager: leaseutil.WithNamespace(lm, ns), + LeaseManager: lm, GarbageCollect: mdb.GarbageCollect, Applier: apply.NewFileSystemApplier(mdb.ContentStore()), }) diff --git a/source/git/gitsource_test.go b/source/git/gitsource_test.go index d2d597d3561c..bf3be616c1dc 100644 --- a/source/git/gitsource_test.go +++ b/source/git/gitsource_test.go @@ -83,7 +83,7 @@ func testRepeatedFetch(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref1.Release(context.TODO()) - mount, err := ref1.Mount(ctx, false, nil) + mount, err := ref1.Mount(ctx, true, nil) require.NoError(t, err) lm := snapshot.LocalMounter(mount) @@ -136,7 +136,7 @@ func testRepeatedFetch(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref3.Release(context.TODO()) - mount, err = ref3.Mount(ctx, false, nil) + mount, err = ref3.Mount(ctx, true, nil) require.NoError(t, err) lm = snapshot.LocalMounter(mount) @@ -213,7 +213,7 @@ func testFetchBySHA(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref1.Release(context.TODO()) - mount, err := ref1.Mount(ctx, false, nil) + mount, err := ref1.Mount(ctx, true, nil) require.NoError(t, err) lm := snapshot.LocalMounter(mount) @@ -306,7 +306,7 @@ func testMultipleRepos(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref1.Release(context.TODO()) - mount, err := ref1.Mount(ctx, false, nil) + mount, err := ref1.Mount(ctx, true, nil) require.NoError(t, err) lm := snapshot.LocalMounter(mount) @@ -318,7 +318,7 @@ func testMultipleRepos(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref2.Release(context.TODO()) - mount, err = ref2.Mount(ctx, false, nil) + mount, err = ref2.Mount(ctx, true, nil) require.NoError(t, err) lm = snapshot.LocalMounter(mount) @@ -420,7 +420,7 @@ func testSubdir(t *testing.T, keepGitDir bool) { require.NoError(t, err) defer ref1.Release(context.TODO()) - mount, err := ref1.Mount(ctx, false, nil) + mount, err := ref1.Mount(ctx, true, nil) require.NoError(t, err) lm := snapshot.LocalMounter(mount) @@ -455,11 +455,12 @@ func setupGitSource(t *testing.T, tmpdir string) source.Source { md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) require.NoError(t, err) + lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter("native", containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter("native")), nil), MetadataStore: md, - LeaseManager: leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit"), + LeaseManager: lm, ContentStore: mdb.ContentStore(), GarbageCollect: mdb.GarbageCollect, }) diff --git a/source/http/httpsource_test.go b/source/http/httpsource_test.go index 127793042c58..c3c5bd069288 100644 --- a/source/http/httpsource_test.go +++ b/source/http/httpsource_test.go @@ -313,7 +313,7 @@ func TestHTTPChecksum(t *testing.T) { } func readFile(ctx context.Context, ref cache.ImmutableRef, fp string) ([]byte, error) { - mount, err := ref.Mount(ctx, false, nil) + mount, err := ref.Mount(ctx, true, nil) if err != nil { return nil, err } @@ -358,11 +358,12 @@ func newHTTPSource(tmpdir string) (source.Source, error) { if err != nil { return nil, err } + lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter("native", containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter("native")), nil), MetadataStore: md, - LeaseManager: leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit"), + LeaseManager: lm, ContentStore: mdb.ContentStore(), GarbageCollect: mdb.GarbageCollect, }) diff --git a/worker/runc/runc.go b/worker/runc/runc.go index 8611ed26c180..99f26818c4ca 100644 --- a/worker/runc/runc.go +++ b/worker/runc/runc.go @@ -101,8 +101,8 @@ func NewWorkerOpt(root string, snFactory SnapshotterFactory, rootless bool, proc for k, v := range labels { xlabels[k] = v } - snap := containerdsnapshot.NewSnapshotter(snFactory.Name, mdb.Snapshotter(snFactory.Name), "buildkit", idmap) lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") + snap := containerdsnapshot.NewSnapshotter(snFactory.Name, mdb.Snapshotter(snFactory.Name), "buildkit", idmap) if err := cache.MigrateV2( context.TODO(), diff --git a/worker/runc/runc_test.go b/worker/runc/runc_test.go index a10556e80f0f..06a0b83d7cec 100644 --- a/worker/runc/runc_test.go +++ b/worker/runc/runc_test.go @@ -73,7 +73,7 @@ func TestRuncWorker(t *testing.T) { require.NoError(t, err) snap := tests.NewBusyboxSourceSnapshot(ctx, t, w, sm) - mounts, err := snap.Mount(ctx, false, nil) + mounts, err := snap.Mount(ctx, true, nil) require.NoError(t, err) lm := snapshot.LocalMounter(mounts) @@ -98,7 +98,7 @@ func TestRuncWorker(t *testing.T) { require.NoError(t, err) // for _, d := range du { - // fmt.Printf("du: %+v\n", d) + // t.Logf("du: %+v\n", d) // } for _, d := range du { @@ -111,16 +111,16 @@ func TestRuncWorker(t *testing.T) { } stderr := bytes.NewBuffer(nil) - err = w.WorkerOpt.Executor.Run(ctx, "", execMount(snap), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) + err = w.WorkerOpt.Executor.Run(ctx, "", execMount(snap, true), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) require.Error(t, err) // Read-only root // typical error is like `mkdir /.../rootfs/proc: read-only file system`. // make sure the error is caused before running `echo foo > /bar`. require.Contains(t, stderr.String(), "read-only file system") - root, err := w.CacheMgr.New(ctx, snap, nil) + root, err := w.CacheMgr.New(ctx, snap, nil, cache.CachePolicyRetain) require.NoError(t, err) - err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) + err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root, false), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) require.NoError(t, err) meta = executor.Meta{ @@ -128,13 +128,13 @@ func TestRuncWorker(t *testing.T) { Cwd: "/", } - err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) + err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root, false), nil, executor.ProcessInfo{Meta: meta, Stderr: &nopCloser{stderr}}, nil) require.NoError(t, err) rf, err := root.Commit(ctx) require.NoError(t, err) - mounts, err = rf.Mount(ctx, false, nil) + mounts, err = rf.Mount(ctx, true, nil) require.NoError(t, err) lm = snapshot.LocalMounter(mounts) @@ -207,7 +207,7 @@ func TestRuncWorkerNoProcessSandbox(t *testing.T) { } stdout := bytes.NewBuffer(nil) stderr := bytes.NewBuffer(nil) - err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root), nil, executor.ProcessInfo{Meta: meta, Stdout: &nopCloser{stdout}, Stderr: &nopCloser{stderr}}, nil) + err = w.WorkerOpt.Executor.Run(ctx, "", execMount(root, false), nil, executor.ProcessInfo{Meta: meta, Stdout: &nopCloser{stdout}, Stderr: &nopCloser{stderr}}, nil) require.NoError(t, err, fmt.Sprintf("stdout=%q, stderr=%q", stdout.String(), stderr.String())) require.Equal(t, string(selfCmdline), stdout.String()) } @@ -244,8 +244,8 @@ func (n *nopCloser) Close() error { return nil } -func execMount(m cache.Mountable) executor.Mount { - return executor.Mount{Src: &mountable{m: m}} +func execMount(m cache.Mountable, readonly bool) executor.Mount { + return executor.Mount{Src: &mountable{m: m}, Readonly: readonly} } type mountable struct { From d73e62f878746bb7d8f73fdd0d7bd3043c8b5f5f Mon Sep 17 00:00:00 2001 From: Erik Sipsma Date: Tue, 3 Aug 2021 01:57:39 +0000 Subject: [PATCH 4/4] Add initial MergeOp implementation. This consists of just the base MergeOp with support for merging LLB results that include deletions using hardlinks as the efficient path and copies as fallback. Signed-off-by: Erik Sipsma --- api/services/control/control.pb.go | 237 +++++---- api/services/control/control.proto | 3 +- cache/blobs.go | 306 ++++++----- cache/contenthash/checksum_test.go | 10 +- cache/manager.go | 332 +++++++++--- cache/manager_test.go | 134 ++++- cache/metadata.go | 9 + cache/refs.go | 277 +++++++--- cache/remote.go | 3 +- client/client_test.go | 393 ++++++++++++++- client/diskusage.go | 4 +- client/llb/merge.go | 96 ++++ client/llb/merge_test.go | 33 ++ client/prune.go | 2 +- cmd/buildctl/debug/dumpllb.go | 2 + cmd/buildctl/diskusage.go | 5 +- control/control.go | 4 +- go.mod | 2 +- hack/test | 2 +- snapshot/diffapply_unix.go | 414 +++++++++++++++ snapshot/diffapply_windows.go | 24 + snapshot/localmounter.go | 32 ++ snapshot/localmounter_unix.go | 2 +- snapshot/merge.go | 219 ++++++++ snapshot/snapshotter.go | 11 + snapshot/snapshotter_test.go | 629 +++++++++++++++++++++++ solver/llbsolver/mounts/mount_test.go | 10 +- solver/llbsolver/ops/merge.go | 95 ++++ solver/llbsolver/result.go | 2 +- solver/llbsolver/vertex.go | 37 +- solver/pb/caps.go | 7 + solver/pb/ops.pb.go | 700 ++++++++++++++++++++------ solver/pb/ops.proto | 17 +- source/containerimage/pull.go | 2 +- source/git/gitsource_test.go | 10 +- source/http/httpsource_test.go | 10 +- source/local/local.go | 2 + util/overlay/overlay_linux.go | 7 +- worker/base/worker.go | 2 + 39 files changed, 3518 insertions(+), 568 deletions(-) create mode 100644 client/llb/merge.go create mode 100644 client/llb/merge_test.go create mode 100644 snapshot/diffapply_unix.go create mode 100644 snapshot/diffapply_windows.go create mode 100644 snapshot/merge.go create mode 100644 snapshot/snapshotter_test.go create mode 100644 solver/llbsolver/ops/merge.go diff --git a/api/services/control/control.pb.go b/api/services/control/control.pb.go index d4f829672448..62830fc490db 100644 --- a/api/services/control/control.pb.go +++ b/api/services/control/control.pb.go @@ -205,13 +205,14 @@ type UsageRecord struct { Mutable bool `protobuf:"varint,2,opt,name=Mutable,proto3" json:"Mutable,omitempty"` InUse bool `protobuf:"varint,3,opt,name=InUse,proto3" json:"InUse,omitempty"` Size_ int64 `protobuf:"varint,4,opt,name=Size,proto3" json:"Size,omitempty"` - Parent string `protobuf:"bytes,5,opt,name=Parent,proto3" json:"Parent,omitempty"` + Parent string `protobuf:"bytes,5,opt,name=Parent,proto3" json:"Parent,omitempty"` // Deprecated: Do not use. CreatedAt time.Time `protobuf:"bytes,6,opt,name=CreatedAt,proto3,stdtime" json:"CreatedAt"` LastUsedAt *time.Time `protobuf:"bytes,7,opt,name=LastUsedAt,proto3,stdtime" json:"LastUsedAt,omitempty"` UsageCount int64 `protobuf:"varint,8,opt,name=UsageCount,proto3" json:"UsageCount,omitempty"` Description string `protobuf:"bytes,9,opt,name=Description,proto3" json:"Description,omitempty"` RecordType string `protobuf:"bytes,10,opt,name=RecordType,proto3" json:"RecordType,omitempty"` Shared bool `protobuf:"varint,11,opt,name=Shared,proto3" json:"Shared,omitempty"` + Parents []string `protobuf:"bytes,12,rep,name=Parents,proto3" json:"Parents,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -278,6 +279,7 @@ func (m *UsageRecord) GetSize_() int64 { return 0 } +// Deprecated: Do not use. func (m *UsageRecord) GetParent() string { if m != nil { return m.Parent @@ -327,6 +329,13 @@ func (m *UsageRecord) GetShared() bool { return false } +func (m *UsageRecord) GetParents() []string { + if m != nil { + return m.Parents + } + return nil +} + type SolveRequest struct { Ref string `protobuf:"bytes,1,opt,name=Ref,proto3" json:"Ref,omitempty"` Definition *pb.Definition `protobuf:"bytes,2,opt,name=Definition,proto3" json:"Definition,omitempty"` @@ -1155,95 +1164,96 @@ func init() { func init() { proto.RegisterFile("control.proto", fileDescriptor_0c5120591600887d) } var fileDescriptor_0c5120591600887d = []byte{ - // 1397 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4d, 0x6f, 0x1b, 0xc5, - 0x1b, 0xef, 0xda, 0xf1, 0xdb, 0x63, 0x27, 0x4a, 0xa7, 0xfd, 0x57, 0xab, 0xfd, 0x8b, 0xc4, 0x6c, - 0x8b, 0x64, 0x55, 0xed, 0x3a, 0x35, 0x14, 0x95, 0x08, 0x50, 0xeb, 0xb8, 0xa8, 0xa9, 0x1a, 0x51, - 0x36, 0x2d, 0x95, 0x7a, 0x40, 0x5a, 0xdb, 0x13, 0x77, 0x95, 0xf5, 0xce, 0x32, 0x33, 0x1b, 0x6a, - 0x3e, 0x00, 0x67, 0xbe, 0x03, 0x07, 0x4e, 0x9c, 0x38, 0xf0, 0x09, 0x90, 0x7a, 0xe4, 0xdc, 0x43, - 0x40, 0xb9, 0xc3, 0x9d, 0x1b, 0x9a, 0x97, 0x75, 0xc6, 0xb1, 0x9d, 0xc4, 0xe9, 0xc9, 0xf3, 0x8c, - 0x9f, 0xdf, 0x6f, 0x9f, 0xd7, 0x99, 0x79, 0x60, 0xb9, 0x47, 0x62, 0x4e, 0x49, 0xe4, 0x25, 0x94, - 0x70, 0x82, 0x56, 0x87, 0xa4, 0x3b, 0xf2, 0xba, 0x69, 0x18, 0xf5, 0xf7, 0x43, 0xee, 0x1d, 0xdc, - 0x71, 0x6e, 0x0f, 0x42, 0xfe, 0x2a, 0xed, 0x7a, 0x3d, 0x32, 0x6c, 0x0e, 0xc8, 0x80, 0x34, 0xa5, - 0x62, 0x37, 0xdd, 0x93, 0x92, 0x14, 0xe4, 0x4a, 0x11, 0x38, 0xeb, 0x03, 0x42, 0x06, 0x11, 0x3e, - 0xd6, 0xe2, 0xe1, 0x10, 0x33, 0x1e, 0x0c, 0x13, 0xad, 0x70, 0xcb, 0xe0, 0x13, 0x1f, 0x6b, 0x66, - 0x1f, 0x6b, 0x32, 0x12, 0x1d, 0x60, 0xda, 0x4c, 0xba, 0x4d, 0x92, 0x30, 0xad, 0xdd, 0x9c, 0xab, - 0x1d, 0x24, 0x61, 0x93, 0x8f, 0x12, 0xcc, 0x9a, 0xdf, 0x11, 0xba, 0x8f, 0xa9, 0x02, 0xb8, 0x3f, - 0x58, 0x50, 0x7b, 0x4a, 0xd3, 0x18, 0xfb, 0xf8, 0xdb, 0x14, 0x33, 0x8e, 0xae, 0x41, 0x71, 0x2f, - 0x8c, 0x38, 0xa6, 0xb6, 0x55, 0xcf, 0x37, 0x2a, 0xbe, 0x96, 0xd0, 0x2a, 0xe4, 0x83, 0x28, 0xb2, - 0x73, 0x75, 0xab, 0x51, 0xf6, 0xc5, 0x12, 0x35, 0xa0, 0xb6, 0x8f, 0x71, 0xd2, 0x49, 0x69, 0xc0, - 0x43, 0x12, 0xdb, 0xf9, 0xba, 0xd5, 0xc8, 0xb7, 0x97, 0xde, 0x1c, 0xae, 0x5b, 0xfe, 0xc4, 0x3f, - 0xc8, 0x85, 0x8a, 0x90, 0xdb, 0x23, 0x8e, 0x99, 0xbd, 0x64, 0xa8, 0x1d, 0x6f, 0xbb, 0x37, 0x61, - 0xb5, 0x13, 0xb2, 0xfd, 0xe7, 0x2c, 0x18, 0x9c, 0x65, 0x8b, 0xfb, 0x18, 0x2e, 0x1b, 0xba, 0x2c, - 0x21, 0x31, 0xc3, 0xe8, 0x2e, 0x14, 0x29, 0xee, 0x11, 0xda, 0x97, 0xca, 0xd5, 0xd6, 0x7b, 0xde, - 0xc9, 0xdc, 0x78, 0x1a, 0x20, 0x94, 0x7c, 0xad, 0xec, 0xfe, 0x9b, 0x83, 0xaa, 0xb1, 0x8f, 0x56, - 0x20, 0xb7, 0xdd, 0xb1, 0xad, 0xba, 0xd5, 0xa8, 0xf8, 0xb9, 0xed, 0x0e, 0xb2, 0xa1, 0xb4, 0x93, - 0xf2, 0xa0, 0x1b, 0x61, 0xed, 0x7b, 0x26, 0xa2, 0xab, 0x50, 0xd8, 0x8e, 0x9f, 0x33, 0x2c, 0x1d, - 0x2f, 0xfb, 0x4a, 0x40, 0x08, 0x96, 0x76, 0xc3, 0xef, 0xb1, 0x72, 0xd3, 0x97, 0x6b, 0xe1, 0xc7, - 0xd3, 0x80, 0xe2, 0x98, 0xdb, 0x05, 0xc9, 0xab, 0x25, 0xd4, 0x86, 0xca, 0x16, 0xc5, 0x01, 0xc7, - 0xfd, 0x07, 0xdc, 0x2e, 0xd6, 0xad, 0x46, 0xb5, 0xe5, 0x78, 0xaa, 0x20, 0xbc, 0xac, 0x20, 0xbc, - 0x67, 0x59, 0x41, 0xb4, 0xcb, 0x6f, 0x0e, 0xd7, 0x2f, 0xfd, 0xf8, 0xa7, 0x88, 0xdb, 0x18, 0x86, - 0xee, 0x03, 0x3c, 0x09, 0x18, 0x7f, 0xce, 0x24, 0x49, 0xe9, 0x4c, 0x92, 0x25, 0x49, 0x60, 0x60, - 0xd0, 0x1a, 0x80, 0x0c, 0xc0, 0x16, 0x49, 0x63, 0x6e, 0x97, 0xa5, 0xdd, 0xc6, 0x0e, 0xaa, 0x43, - 0xb5, 0x83, 0x59, 0x8f, 0x86, 0x89, 0x4c, 0x73, 0x45, 0xba, 0x60, 0x6e, 0x09, 0x06, 0x15, 0xbd, - 0x67, 0xa3, 0x04, 0xdb, 0x20, 0x15, 0x8c, 0x1d, 0xe1, 0xff, 0xee, 0xab, 0x80, 0xe2, 0xbe, 0x5d, - 0x95, 0xa1, 0xd2, 0x92, 0xfb, 0x53, 0x11, 0x6a, 0xbb, 0xa2, 0x8a, 0xb3, 0x84, 0xaf, 0x42, 0xde, - 0xc7, 0x7b, 0x3a, 0xfa, 0x62, 0x89, 0x3c, 0x80, 0x0e, 0xde, 0x0b, 0xe3, 0x50, 0x7e, 0x3b, 0x27, - 0xdd, 0x5b, 0xf1, 0x92, 0xae, 0x77, 0xbc, 0xeb, 0x1b, 0x1a, 0xc8, 0x81, 0xf2, 0xc3, 0xd7, 0x09, - 0xa1, 0xa2, 0x68, 0xf2, 0x92, 0x66, 0x2c, 0xa3, 0x17, 0xb0, 0x9c, 0xad, 0x1f, 0x70, 0x4e, 0x45, - 0x29, 0x8a, 0x42, 0xb9, 0x33, 0x5d, 0x28, 0xa6, 0x51, 0xde, 0x04, 0xe6, 0x61, 0xcc, 0xe9, 0xc8, - 0x9f, 0xe4, 0x11, 0x35, 0xb2, 0x8b, 0x19, 0x13, 0x16, 0xaa, 0x04, 0x67, 0xa2, 0x30, 0xe7, 0x0b, - 0x4a, 0x62, 0x8e, 0xe3, 0xbe, 0x4c, 0x70, 0xc5, 0x1f, 0xcb, 0xc2, 0x9c, 0x6c, 0xad, 0xcc, 0x29, - 0x9d, 0xcb, 0x9c, 0x09, 0x8c, 0x36, 0x67, 0x62, 0x0f, 0x6d, 0x42, 0x61, 0x2b, 0xe8, 0xbd, 0xc2, - 0x32, 0x97, 0xd5, 0xd6, 0xda, 0x34, 0xa1, 0xfc, 0xfb, 0x4b, 0x99, 0x3c, 0x26, 0x5b, 0xf1, 0x92, - 0xaf, 0x20, 0xe8, 0x1b, 0xa8, 0x3d, 0x8c, 0x79, 0xc8, 0x23, 0x3c, 0xc4, 0x31, 0x67, 0x76, 0x45, - 0x34, 0x5e, 0x7b, 0xf3, 0xed, 0xe1, 0xfa, 0xc7, 0x73, 0x8f, 0x96, 0x94, 0x87, 0x51, 0x13, 0x1b, - 0x28, 0xcf, 0xa0, 0xf0, 0x27, 0xf8, 0xd0, 0x4b, 0x58, 0xc9, 0x8c, 0xdd, 0x8e, 0x93, 0x94, 0x33, - 0x1b, 0xa4, 0xd7, 0xad, 0x73, 0x7a, 0xad, 0x40, 0xca, 0xed, 0x13, 0x4c, 0xce, 0x7d, 0x40, 0xd3, - 0xb9, 0x12, 0x35, 0xb5, 0x8f, 0x47, 0x59, 0x4d, 0xed, 0xe3, 0x91, 0x68, 0xdc, 0x83, 0x20, 0x4a, - 0x55, 0x43, 0x57, 0x7c, 0x25, 0x6c, 0xe6, 0xee, 0x59, 0x82, 0x61, 0x3a, 0xbc, 0x0b, 0x31, 0x7c, - 0x05, 0x57, 0x66, 0x98, 0x3a, 0x83, 0xe2, 0x86, 0x49, 0x31, 0x5d, 0xd3, 0xc7, 0x94, 0xee, 0x2f, - 0x79, 0xa8, 0x99, 0x09, 0x43, 0x1b, 0x70, 0x45, 0xf9, 0xe9, 0xe3, 0xbd, 0x0e, 0x4e, 0x28, 0xee, - 0x89, 0xb3, 0x40, 0x93, 0xcf, 0xfa, 0x0b, 0xb5, 0xe0, 0xea, 0xf6, 0x50, 0x6f, 0x33, 0x03, 0x92, - 0x93, 0xc7, 0xea, 0xcc, 0xff, 0x10, 0x81, 0xff, 0x29, 0x2a, 0x19, 0x09, 0x03, 0x94, 0x97, 0x09, - 0xfb, 0xe4, 0xf4, 0xaa, 0xf2, 0x66, 0x62, 0x55, 0xde, 0x66, 0xf3, 0xa2, 0xcf, 0xa0, 0xa4, 0xfe, - 0xc8, 0x1a, 0xf3, 0xfa, 0xe9, 0x9f, 0x50, 0x64, 0x19, 0x46, 0xc0, 0x95, 0x1f, 0xcc, 0x2e, 0x2c, - 0x00, 0xd7, 0x18, 0xe7, 0x11, 0x38, 0xf3, 0x4d, 0x5e, 0xa4, 0x04, 0xdc, 0x9f, 0x2d, 0xb8, 0x3c, - 0xf5, 0x21, 0x71, 0x2f, 0xc8, 0xd3, 0x51, 0x51, 0xc8, 0x35, 0xea, 0x40, 0x41, 0x75, 0x7e, 0x4e, - 0x1a, 0xec, 0x9d, 0xc3, 0x60, 0xcf, 0x68, 0x7b, 0x05, 0x76, 0xee, 0x01, 0x5c, 0xac, 0x58, 0xdd, - 0xdf, 0x2c, 0x58, 0xd6, 0x5d, 0xa6, 0x2f, 0xd1, 0x00, 0x56, 0xb3, 0x16, 0xca, 0xf6, 0xf4, 0x75, - 0x7a, 0x77, 0x6e, 0x83, 0x2a, 0x35, 0xef, 0x24, 0x4e, 0xd9, 0x38, 0x45, 0xe7, 0x6c, 0x65, 0x75, - 0x75, 0x42, 0x75, 0x21, 0xcb, 0xdf, 0x87, 0xe5, 0x5d, 0x1e, 0xf0, 0x94, 0xcd, 0xbd, 0x39, 0xdc, - 0x5f, 0x2d, 0x58, 0xc9, 0x74, 0xb4, 0x77, 0x1f, 0x41, 0xf9, 0x00, 0x53, 0x8e, 0x5f, 0x63, 0xa6, - 0xbd, 0xb2, 0xa7, 0xbd, 0xfa, 0x5a, 0x6a, 0xf8, 0x63, 0x4d, 0xb4, 0x09, 0x65, 0x26, 0x79, 0x70, - 0x96, 0xa8, 0xb5, 0x79, 0x28, 0xfd, 0xbd, 0xb1, 0x3e, 0x6a, 0xc2, 0x52, 0x44, 0x06, 0x4c, 0xf7, - 0xcc, 0xff, 0xe7, 0xe1, 0x9e, 0x90, 0x81, 0x2f, 0x15, 0xdd, 0xc3, 0x1c, 0x14, 0xd5, 0x1e, 0x7a, - 0x0c, 0xc5, 0x7e, 0x38, 0xc0, 0x8c, 0x2b, 0xaf, 0xda, 0x2d, 0x71, 0x4e, 0xbf, 0x3d, 0x5c, 0xbf, - 0x69, 0x1c, 0xc4, 0x24, 0xc1, 0xb1, 0x78, 0x91, 0x06, 0x61, 0x8c, 0x29, 0x6b, 0x0e, 0xc8, 0x6d, - 0x05, 0xf1, 0x3a, 0xf2, 0xc7, 0xd7, 0x0c, 0x82, 0x2b, 0x54, 0xc7, 0xad, 0x6c, 0xf9, 0x8b, 0x71, - 0x29, 0x06, 0x51, 0xc9, 0x71, 0x30, 0xc4, 0xfa, 0x7a, 0x95, 0x6b, 0x71, 0xc3, 0xf7, 0x44, 0xa9, - 0xf6, 0xe5, 0xbb, 0xa7, 0xec, 0x6b, 0x09, 0x6d, 0x42, 0x89, 0xf1, 0x80, 0x8a, 0x63, 0xa3, 0x70, - 0xce, 0xa7, 0x49, 0x06, 0x40, 0x9f, 0x43, 0xa5, 0x47, 0x86, 0x49, 0x84, 0x05, 0xba, 0x78, 0x4e, - 0xf4, 0x31, 0x44, 0x54, 0x0f, 0xa6, 0x94, 0x50, 0xf9, 0x28, 0xaa, 0xf8, 0x4a, 0x70, 0xff, 0xc9, - 0x41, 0xcd, 0x4c, 0xd6, 0xd4, 0x83, 0xef, 0x31, 0x14, 0x55, 0xea, 0x55, 0xd5, 0x5d, 0x2c, 0x54, - 0x8a, 0x61, 0x66, 0xa8, 0x6c, 0x28, 0xf5, 0x52, 0x2a, 0x5f, 0x83, 0xea, 0x8d, 0x98, 0x89, 0xc2, - 0x60, 0x4e, 0x78, 0x10, 0xc9, 0x50, 0xe5, 0x7d, 0x25, 0x88, 0x47, 0xe2, 0x78, 0x26, 0x58, 0xec, - 0x91, 0x38, 0x86, 0x99, 0x69, 0x28, 0xbd, 0x53, 0x1a, 0xca, 0x0b, 0xa7, 0xc1, 0xfd, 0xdd, 0x82, - 0xca, 0xb8, 0xca, 0x8d, 0xe8, 0x5a, 0xef, 0x1c, 0xdd, 0x89, 0xc8, 0xe4, 0x2e, 0x16, 0x99, 0x6b, - 0x50, 0x64, 0x9c, 0xe2, 0x60, 0xa8, 0xc6, 0x17, 0x5f, 0x4b, 0xe2, 0x3c, 0x19, 0xb2, 0x81, 0xcc, - 0x50, 0xcd, 0x17, 0x4b, 0xd7, 0x85, 0x9a, 0x9c, 0x54, 0x76, 0x30, 0x13, 0x6f, 0x63, 0x91, 0xdb, - 0x7e, 0xc0, 0x03, 0xe9, 0x47, 0xcd, 0x97, 0x6b, 0xf7, 0x16, 0xa0, 0x27, 0x21, 0xe3, 0x2f, 0xe4, - 0x84, 0xc5, 0xce, 0x1a, 0x63, 0x76, 0xe1, 0xca, 0x84, 0xb6, 0x3e, 0xa5, 0x3e, 0x3d, 0x31, 0xc8, - 0xdc, 0x98, 0x3e, 0x35, 0xe4, 0x20, 0xe7, 0x29, 0xe0, 0xe4, 0x3c, 0xd3, 0xfa, 0x3b, 0x0f, 0xa5, - 0x2d, 0x35, 0xa3, 0xa2, 0x67, 0x50, 0x19, 0xcf, 0x49, 0xc8, 0x9d, 0xa6, 0x39, 0x39, 0x70, 0x39, - 0xd7, 0x4f, 0xd5, 0xd1, 0xf6, 0x3d, 0x82, 0x82, 0x9c, 0x18, 0xd1, 0x8c, 0x63, 0xd0, 0x1c, 0x25, - 0x9d, 0xd3, 0x27, 0xb0, 0x0d, 0x4b, 0x30, 0xc9, 0x3b, 0x64, 0x16, 0x93, 0xf9, 0xfa, 0x73, 0xd6, - 0xcf, 0xb8, 0x7c, 0xd0, 0x0e, 0x14, 0x75, 0x3b, 0xcf, 0x52, 0x35, 0x6f, 0x0a, 0xa7, 0x3e, 0x5f, - 0x41, 0x91, 0x6d, 0x58, 0x68, 0x67, 0xfc, 0xa0, 0x9f, 0x65, 0x9a, 0x59, 0x06, 0xce, 0x19, 0xff, - 0x37, 0xac, 0x0d, 0x0b, 0xbd, 0x84, 0xaa, 0x91, 0x68, 0x34, 0x23, 0xa1, 0xd3, 0x55, 0xe3, 0x7c, - 0x70, 0x86, 0x96, 0x32, 0xb6, 0x5d, 0x7b, 0x73, 0xb4, 0x66, 0xfd, 0x71, 0xb4, 0x66, 0xfd, 0x75, - 0xb4, 0x66, 0x75, 0x8b, 0xb2, 0xee, 0x3f, 0xfc, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xfb, 0x65, 0x7c, - 0xd6, 0xa7, 0x10, 0x00, 0x00, + // 1413 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xcd, 0x6e, 0x1b, 0xb7, + 0x16, 0xce, 0x48, 0xd6, 0xdf, 0x91, 0x6c, 0x38, 0x4c, 0x6e, 0x30, 0x98, 0x8b, 0x6b, 0xeb, 0x4e, + 0x72, 0x01, 0x21, 0x48, 0x46, 0x8e, 0x6e, 0x53, 0xa4, 0x46, 0x5b, 0x24, 0xb2, 0x52, 0xc4, 0x41, + 0x8c, 0xa6, 0x74, 0xd2, 0x00, 0x59, 0x14, 0x18, 0x49, 0xb4, 0x32, 0xf0, 0x68, 0x38, 0x25, 0x29, + 0x37, 0xea, 0x03, 0x74, 0xdd, 0x5d, 0x1f, 0xa0, 0x8b, 0xae, 0xba, 0xea, 0xa2, 0x4f, 0x50, 0x20, + 0xcb, 0xae, 0xb3, 0x70, 0x8b, 0xec, 0xdb, 0x67, 0x28, 0xf8, 0x33, 0x32, 0x65, 0x49, 0xfe, 0xcb, + 0x4a, 0x3c, 0x9c, 0xf3, 0x7d, 0x3a, 0xbf, 0x24, 0x0f, 0x2c, 0xf7, 0x68, 0x22, 0x18, 0x8d, 0x83, + 0x94, 0x51, 0x41, 0xd1, 0xea, 0x90, 0x76, 0xc7, 0x41, 0x77, 0x14, 0xc5, 0xfd, 0xfd, 0x48, 0x04, + 0x07, 0x77, 0xbc, 0xdb, 0x83, 0x48, 0xbc, 0x1a, 0x75, 0x83, 0x1e, 0x1d, 0x36, 0x07, 0x74, 0x40, + 0x9b, 0x4a, 0xb1, 0x3b, 0xda, 0x53, 0x92, 0x12, 0xd4, 0x4a, 0x13, 0x78, 0xeb, 0x03, 0x4a, 0x07, + 0x31, 0x39, 0xd2, 0x12, 0xd1, 0x90, 0x70, 0x11, 0x0e, 0x53, 0xa3, 0x70, 0xcb, 0xe2, 0x93, 0x7f, + 0xd6, 0xcc, 0xfe, 0xac, 0xc9, 0x69, 0x7c, 0x40, 0x58, 0x33, 0xed, 0x36, 0x69, 0xca, 0x8d, 0x76, + 0x73, 0xa1, 0x76, 0x98, 0x46, 0x4d, 0x31, 0x4e, 0x09, 0x6f, 0x7e, 0x43, 0xd9, 0x3e, 0x61, 0x1a, + 0xe0, 0x7f, 0xe7, 0x40, 0xed, 0x29, 0x1b, 0x25, 0x04, 0x93, 0xaf, 0x47, 0x84, 0x0b, 0x74, 0x0d, + 0x8a, 0x7b, 0x51, 0x2c, 0x08, 0x73, 0x9d, 0x7a, 0xbe, 0x51, 0xc1, 0x46, 0x42, 0xab, 0x90, 0x0f, + 0xe3, 0xd8, 0xcd, 0xd5, 0x9d, 0x46, 0x19, 0xcb, 0x25, 0x6a, 0x40, 0x6d, 0x9f, 0x90, 0xb4, 0x33, + 0x62, 0xa1, 0x88, 0x68, 0xe2, 0xe6, 0xeb, 0x4e, 0x23, 0xdf, 0x5e, 0x7a, 0x73, 0xb8, 0xee, 0xe0, + 0xa9, 0x2f, 0xc8, 0x87, 0x8a, 0x94, 0xdb, 0x63, 0x41, 0xb8, 0xbb, 0x64, 0xa9, 0x1d, 0x6d, 0xfb, + 0x37, 0x61, 0xb5, 0x13, 0xf1, 0xfd, 0xe7, 0x3c, 0x1c, 0x9c, 0x66, 0x8b, 0xff, 0x18, 0x2e, 0x5b, + 0xba, 0x3c, 0xa5, 0x09, 0x27, 0xe8, 0x2e, 0x14, 0x19, 0xe9, 0x51, 0xd6, 0x57, 0xca, 0xd5, 0xd6, + 0x7f, 0x82, 0xe3, 0xb9, 0x09, 0x0c, 0x40, 0x2a, 0x61, 0xa3, 0xec, 0xff, 0x90, 0x87, 0xaa, 0xb5, + 0x8f, 0x56, 0x20, 0xb7, 0xdd, 0x71, 0x9d, 0xba, 0xd3, 0xa8, 0xe0, 0xdc, 0x76, 0x07, 0xb9, 0x50, + 0xda, 0x19, 0x89, 0xb0, 0x1b, 0x13, 0xe3, 0x7b, 0x26, 0xa2, 0xab, 0x50, 0xd8, 0x4e, 0x9e, 0x73, + 0xa2, 0x1c, 0x2f, 0x63, 0x2d, 0x20, 0x04, 0x4b, 0xbb, 0xd1, 0xb7, 0x44, 0xbb, 0x89, 0xd5, 0x1a, + 0x79, 0x50, 0x7c, 0x1a, 0x32, 0x92, 0x08, 0xb7, 0x20, 0x79, 0xdb, 0x39, 0xd7, 0xc1, 0x66, 0x07, + 0xb5, 0xa1, 0xb2, 0xc5, 0x48, 0x28, 0x48, 0xff, 0x81, 0x70, 0x8b, 0x75, 0xa7, 0x51, 0x6d, 0x79, + 0x81, 0x2e, 0x8a, 0x20, 0x2b, 0x8a, 0xe0, 0x59, 0x56, 0x14, 0xed, 0xf2, 0x9b, 0xc3, 0xf5, 0x4b, + 0xdf, 0xff, 0x21, 0x63, 0x37, 0x81, 0xa1, 0xfb, 0x00, 0x4f, 0x42, 0x2e, 0x9e, 0x73, 0x45, 0x52, + 0x3a, 0x95, 0x64, 0x49, 0x11, 0x58, 0x18, 0xb4, 0x06, 0xa0, 0x82, 0xb0, 0x45, 0x47, 0x89, 0x70, + 0xcb, 0xca, 0x76, 0x6b, 0x07, 0xd5, 0xa1, 0xda, 0x21, 0xbc, 0xc7, 0xa2, 0x54, 0xa5, 0xba, 0xa2, + 0xc2, 0x63, 0x6f, 0x49, 0x06, 0x1d, 0xc1, 0x67, 0xe3, 0x94, 0xb8, 0xa0, 0x14, 0xac, 0x1d, 0x99, + 0xcb, 0xdd, 0x57, 0x21, 0x23, 0x7d, 0xb7, 0xaa, 0xc2, 0x65, 0x24, 0x19, 0x5f, 0x1d, 0x09, 0xee, + 0xd6, 0x54, 0x92, 0x33, 0xd1, 0xff, 0xb1, 0x08, 0xb5, 0x5d, 0x59, 0xe3, 0x59, 0x39, 0xac, 0x42, + 0x1e, 0x93, 0x3d, 0x93, 0x1b, 0xb9, 0x44, 0x01, 0x40, 0x87, 0xec, 0x45, 0x49, 0xa4, 0xac, 0xca, + 0x29, 0xc7, 0x57, 0x82, 0xb4, 0x1b, 0x1c, 0xed, 0x62, 0x4b, 0x03, 0x79, 0x50, 0x7e, 0xf8, 0x3a, + 0xa5, 0x4c, 0x96, 0x54, 0x5e, 0xd1, 0x4c, 0x64, 0xf4, 0x02, 0x96, 0xb3, 0xf5, 0x03, 0x21, 0x98, + 0x2c, 0x54, 0x59, 0x46, 0x77, 0x66, 0xcb, 0xc8, 0x36, 0x2a, 0x98, 0xc2, 0x3c, 0x4c, 0x04, 0x1b, + 0xe3, 0x69, 0x1e, 0xe9, 0xe1, 0x2e, 0xe1, 0x5c, 0x5a, 0xa8, 0xd2, 0x8f, 0x33, 0x51, 0x9a, 0xf3, + 0x19, 0xa3, 0x89, 0x20, 0x49, 0x5f, 0xa5, 0xbe, 0x82, 0x27, 0xb2, 0x34, 0x27, 0x5b, 0x6b, 0x73, + 0x4a, 0x67, 0x32, 0x67, 0x0a, 0x63, 0xcc, 0x99, 0xda, 0x43, 0x9b, 0x50, 0xd8, 0x0a, 0x7b, 0xaf, + 0x88, 0xca, 0x72, 0xb5, 0xb5, 0x36, 0x4b, 0xa8, 0x3e, 0x7f, 0xae, 0xd2, 0xca, 0x55, 0xa3, 0x5e, + 0xc2, 0x1a, 0x82, 0xbe, 0x82, 0xda, 0xc3, 0x44, 0x44, 0x22, 0x26, 0x43, 0x95, 0xb1, 0x8a, 0xcc, + 0x58, 0x7b, 0xf3, 0xed, 0xe1, 0xfa, 0x87, 0x0b, 0x0f, 0x9e, 0x91, 0x88, 0xe2, 0x26, 0xb1, 0x50, + 0x81, 0x45, 0x81, 0xa7, 0xf8, 0xd0, 0x4b, 0x58, 0xc9, 0x8c, 0xdd, 0x4e, 0xd2, 0x91, 0xe0, 0x2e, + 0x28, 0xaf, 0x5b, 0x67, 0xf4, 0x5a, 0x83, 0xb4, 0xdb, 0xc7, 0x98, 0xbc, 0xfb, 0x80, 0x66, 0x73, + 0x25, 0x6b, 0x6a, 0x9f, 0x8c, 0xb3, 0x9a, 0xda, 0x27, 0x63, 0xd9, 0xd6, 0x07, 0x61, 0x3c, 0xd2, + 0xed, 0x5e, 0xc1, 0x5a, 0xd8, 0xcc, 0xdd, 0x73, 0x24, 0xc3, 0x6c, 0x78, 0xcf, 0xc5, 0xf0, 0x05, + 0x5c, 0x99, 0x63, 0xea, 0x1c, 0x8a, 0x1b, 0x36, 0xc5, 0x6c, 0x4d, 0x1f, 0x51, 0xfa, 0x3f, 0xe7, + 0xa1, 0x66, 0x27, 0x0c, 0x6d, 0xc0, 0x15, 0xed, 0x27, 0x26, 0x7b, 0x1d, 0x92, 0x32, 0xd2, 0x93, + 0xa7, 0x84, 0x21, 0x9f, 0xf7, 0x09, 0xb5, 0xe0, 0xea, 0xf6, 0xd0, 0x6c, 0x73, 0x0b, 0x92, 0x53, + 0xfd, 0x38, 0xf7, 0x1b, 0xa2, 0xf0, 0x2f, 0x4d, 0xa5, 0x22, 0x61, 0x81, 0xf2, 0x2a, 0x61, 0x1f, + 0x9d, 0x5c, 0x55, 0xc1, 0x5c, 0xac, 0xce, 0xdb, 0x7c, 0x5e, 0xf4, 0x09, 0x94, 0xf4, 0x87, 0xac, + 0x31, 0xaf, 0x9f, 0xfc, 0x17, 0x9a, 0x2c, 0xc3, 0x48, 0xb8, 0xf6, 0x83, 0xbb, 0x85, 0x73, 0xc0, + 0x0d, 0xc6, 0x7b, 0x04, 0xde, 0x62, 0x93, 0xcf, 0x53, 0x02, 0xfe, 0x4f, 0x0e, 0x5c, 0x9e, 0xf9, + 0x23, 0x79, 0x6b, 0xa8, 0x73, 0x53, 0x53, 0xa8, 0x35, 0xea, 0x40, 0x41, 0x77, 0x7e, 0x4e, 0x19, + 0x1c, 0x9c, 0xc1, 0xe0, 0xc0, 0x6a, 0x7b, 0x0d, 0xf6, 0xee, 0x01, 0x5c, 0xac, 0x58, 0xfd, 0x5f, + 0x1d, 0x58, 0x36, 0x5d, 0x66, 0xae, 0xd8, 0x10, 0x56, 0xb3, 0x16, 0xca, 0xf6, 0xcc, 0x65, 0x7b, + 0x77, 0x61, 0x83, 0x6a, 0xb5, 0xe0, 0x38, 0x4e, 0xdb, 0x38, 0x43, 0xe7, 0x6d, 0x65, 0x75, 0x75, + 0x4c, 0xf5, 0x5c, 0x96, 0xff, 0x17, 0x96, 0x77, 0x45, 0x28, 0x46, 0x7c, 0xe1, 0xcd, 0xe1, 0xff, + 0xe2, 0xc0, 0x4a, 0xa6, 0x63, 0xbc, 0xfb, 0x00, 0xca, 0x07, 0x84, 0x09, 0xf2, 0x9a, 0x70, 0xe3, + 0x95, 0x3b, 0xeb, 0xd5, 0x97, 0x4a, 0x03, 0x4f, 0x34, 0xd1, 0x26, 0x94, 0xb9, 0xe2, 0x21, 0x59, + 0xa2, 0xd6, 0x16, 0xa1, 0xcc, 0xff, 0x4d, 0xf4, 0x51, 0x13, 0x96, 0x62, 0x3a, 0xe0, 0xa6, 0x67, + 0xfe, 0xbd, 0x08, 0xf7, 0x84, 0x0e, 0xb0, 0x52, 0xf4, 0x0f, 0x73, 0x50, 0xd4, 0x7b, 0xe8, 0x31, + 0x14, 0xfb, 0xd1, 0x80, 0x70, 0xa1, 0xbd, 0x6a, 0xb7, 0xe4, 0x39, 0xfd, 0xf6, 0x70, 0xfd, 0xa6, + 0x75, 0x10, 0xd3, 0x94, 0x24, 0xf2, 0xbd, 0x1a, 0x46, 0x09, 0x61, 0xbc, 0x39, 0xa0, 0xb7, 0x35, + 0x24, 0xe8, 0xa8, 0x1f, 0x6c, 0x18, 0x24, 0x57, 0xa4, 0x8f, 0x5b, 0xd5, 0xf2, 0x17, 0xe3, 0xd2, + 0x0c, 0xb2, 0x92, 0x93, 0x70, 0x48, 0xcc, 0xf5, 0xaa, 0xd6, 0xf2, 0xee, 0xef, 0xc9, 0x52, 0xed, + 0xab, 0x57, 0x51, 0x19, 0x1b, 0x09, 0x6d, 0x42, 0x89, 0x8b, 0x90, 0xc9, 0x63, 0xa3, 0x70, 0xc6, + 0x47, 0x4b, 0x06, 0x40, 0x9f, 0x42, 0xa5, 0x47, 0x87, 0x69, 0x4c, 0x24, 0xba, 0x78, 0x46, 0xf4, + 0x11, 0x44, 0x56, 0x0f, 0x61, 0x8c, 0x32, 0xf5, 0x5c, 0xaa, 0x60, 0x2d, 0xf8, 0x7f, 0xe7, 0xa0, + 0x66, 0x27, 0x6b, 0xe6, 0x39, 0xf8, 0x18, 0x8a, 0x3a, 0xf5, 0xba, 0xea, 0x2e, 0x16, 0x2a, 0xcd, + 0x30, 0x37, 0x54, 0x2e, 0x94, 0x7a, 0x23, 0xa6, 0xde, 0x8a, 0xfa, 0x05, 0x99, 0x89, 0xd2, 0x60, + 0x41, 0x45, 0x18, 0xab, 0x50, 0xe5, 0xb1, 0x16, 0xe4, 0xf3, 0x71, 0x32, 0x31, 0x9c, 0xef, 0xf9, + 0x38, 0x81, 0xd9, 0x69, 0x28, 0xbd, 0x57, 0x1a, 0xca, 0xe7, 0x4e, 0x83, 0xff, 0x9b, 0x03, 0x95, + 0x49, 0x95, 0x5b, 0xd1, 0x75, 0xde, 0x3b, 0xba, 0x53, 0x91, 0xc9, 0x5d, 0x2c, 0x32, 0xd7, 0xa0, + 0xc8, 0x05, 0x23, 0xe1, 0x50, 0x0f, 0x37, 0xd8, 0x48, 0xf2, 0x3c, 0x19, 0xf2, 0x81, 0xca, 0x50, + 0x0d, 0xcb, 0xa5, 0xef, 0x43, 0x4d, 0xcd, 0x31, 0x3b, 0x84, 0xcb, 0x57, 0xb3, 0xcc, 0x6d, 0x3f, + 0x14, 0xa1, 0xf2, 0xa3, 0x86, 0xd5, 0xda, 0xbf, 0x05, 0xe8, 0x49, 0xc4, 0xc5, 0x0b, 0x35, 0x7f, + 0xf1, 0xd3, 0x86, 0x9c, 0x5d, 0xb8, 0x32, 0xa5, 0x6d, 0x4e, 0xa9, 0x8f, 0x8f, 0x8d, 0x39, 0x37, + 0x66, 0x4f, 0x0d, 0x35, 0xe6, 0x05, 0x1a, 0x38, 0x3d, 0xed, 0xb4, 0xfe, 0xca, 0x43, 0x69, 0x4b, + 0x4f, 0xb0, 0xe8, 0x19, 0x54, 0x26, 0x53, 0x14, 0xf2, 0x67, 0x69, 0x8e, 0x8f, 0x63, 0xde, 0xf5, + 0x13, 0x75, 0x8c, 0x7d, 0x8f, 0xa0, 0xa0, 0xe6, 0x49, 0x34, 0xe7, 0x18, 0xb4, 0x07, 0x4d, 0xef, + 0xe4, 0xf9, 0x6c, 0xc3, 0x91, 0x4c, 0xea, 0x0e, 0x99, 0xc7, 0x64, 0xbf, 0xfe, 0xbc, 0xf5, 0x53, + 0x2e, 0x1f, 0xb4, 0x03, 0x45, 0xd3, 0xce, 0xf3, 0x54, 0xed, 0x9b, 0xc2, 0xab, 0x2f, 0x56, 0xd0, + 0x64, 0x1b, 0x0e, 0xda, 0x99, 0x3c, 0xe8, 0xe7, 0x99, 0x66, 0x97, 0x81, 0x77, 0xca, 0xf7, 0x86, + 0xb3, 0xe1, 0xa0, 0x97, 0x50, 0xb5, 0x12, 0x8d, 0xe6, 0x24, 0x74, 0xb6, 0x6a, 0xbc, 0xff, 0x9d, + 0xa2, 0xa5, 0x8d, 0x6d, 0xd7, 0xde, 0xbc, 0x5b, 0x73, 0x7e, 0x7f, 0xb7, 0xe6, 0xfc, 0xf9, 0x6e, + 0xcd, 0xe9, 0x16, 0x55, 0xdd, 0xff, 0xff, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x5d, 0x51, 0xa3, + 0x94, 0xc5, 0x10, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1750,6 +1760,15 @@ func (m *UsageRecord) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.Parents) > 0 { + for iNdEx := len(m.Parents) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Parents[iNdEx]) + copy(dAtA[i:], m.Parents[iNdEx]) + i = encodeVarintControl(dAtA, i, uint64(len(m.Parents[iNdEx]))) + i-- + dAtA[i] = 0x62 + } + } if m.Shared { i-- if m.Shared { @@ -2736,6 +2755,12 @@ func (m *UsageRecord) Size() (n int) { if m.Shared { n += 2 } + if len(m.Parents) > 0 { + for _, s := range m.Parents { + l = len(s) + n += 1 + l + sovControl(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -3740,6 +3765,38 @@ func (m *UsageRecord) Unmarshal(dAtA []byte) error { } } m.Shared = bool(v != 0) + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Parents", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowControl + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthControl + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthControl + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Parents = append(m.Parents, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipControl(dAtA[iNdEx:]) diff --git a/api/services/control/control.proto b/api/services/control/control.proto index a36ca02f5fc0..8951d1b94a98 100644 --- a/api/services/control/control.proto +++ b/api/services/control/control.proto @@ -41,13 +41,14 @@ message UsageRecord { bool Mutable = 2; bool InUse = 3; int64 Size = 4; - string Parent = 5; + string Parent = 5 [deprecated=true]; google.protobuf.Timestamp CreatedAt = 6 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false]; google.protobuf.Timestamp LastUsedAt = 7 [(gogoproto.stdtime) = true]; int64 UsageCount = 8; string Description = 9; string RecordType = 10; bool Shared = 11; + repeated string Parents = 12; } message SolveRequest { diff --git a/cache/blobs.go b/cache/blobs.go index 7583caaab038..08c3b94246be 100644 --- a/cache/blobs.go +++ b/cache/blobs.go @@ -54,170 +54,177 @@ type compressor func(dest io.Writer, requiredMediaType string) (io.WriteCloser, func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool, compressionType compression.Type, forceCompression bool, s session.Group) error { eg, ctx := errgroup.WithContext(ctx) - if sr.parent != nil { + switch sr.kind() { + case Merge: + for _, parent := range sr.mergeParents { + parent := parent + eg.Go(func() error { + return computeBlobChain(ctx, parent, createIfNeeded, compressionType, forceCompression, s) + }) + } + case Layer: eg.Go(func() error { - return computeBlobChain(ctx, sr.parent, createIfNeeded, compressionType, forceCompression, s) + return computeBlobChain(ctx, sr.layerParent, createIfNeeded, compressionType, forceCompression, s) }) - } - - eg.Go(func() error { - _, err := g.Do(ctx, fmt.Sprintf("%s-%t", sr.ID(), createIfNeeded), func(ctx context.Context) (interface{}, error) { - if sr.getBlob() != "" { - return nil, nil - } - if !createIfNeeded { - return nil, errors.WithStack(ErrNoBlobs) - } + fallthrough + case BaseLayer: + eg.Go(func() error { + _, err := g.Do(ctx, fmt.Sprintf("%s-%t", sr.ID(), createIfNeeded), func(ctx context.Context) (interface{}, error) { + if sr.getBlob() != "" { + return nil, nil + } + if !createIfNeeded { + return nil, errors.WithStack(ErrNoBlobs) + } - var mediaType string - var compressorFunc compressor - var finalize func(context.Context, content.Store) (map[string]string, error) - switch compressionType { - case compression.Uncompressed: - mediaType = ocispecs.MediaTypeImageLayer - case compression.Gzip: - mediaType = ocispecs.MediaTypeImageLayerGzip - case compression.EStargz: - compressorFunc, finalize = compressEStargz() - mediaType = ocispecs.MediaTypeImageLayerGzip - case compression.Zstd: - compressorFunc = zstdWriter - mediaType = ocispecs.MediaTypeImageLayer + "+zstd" - default: - return nil, errors.Errorf("unknown layer compression type: %q", compressionType) - } + var mediaType string + var compressorFunc compressor + var finalize func(context.Context, content.Store) (map[string]string, error) + switch compressionType { + case compression.Uncompressed: + mediaType = ocispecs.MediaTypeImageLayer + case compression.Gzip: + mediaType = ocispecs.MediaTypeImageLayerGzip + case compression.EStargz: + compressorFunc, finalize = compressEStargz() + mediaType = ocispecs.MediaTypeImageLayerGzip + case compression.Zstd: + compressorFunc = zstdWriter + mediaType = ocispecs.MediaTypeImageLayer + "+zstd" + default: + return nil, errors.Errorf("unknown layer compression type: %q", compressionType) + } - var lower []mount.Mount - if sr.parent != nil { - m, err := sr.parent.Mount(ctx, true, s) + var lower []mount.Mount + if sr.layerParent != nil { + m, err := sr.layerParent.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 } - var release func() error - lower, release, err = m.Mount() + upper, 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() - } - var desc ocispecs.Descriptor - - // Determine differ and error/log handling according to the platform, envvar and the snapshotter. - var enableOverlay, fallback, logWarnOnErr bool - if forceOvlStr := os.Getenv("BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF"); forceOvlStr != "" { - enableOverlay, err = strconv.ParseBool(forceOvlStr) - if err != nil { - return nil, errors.Wrapf(err, "invalid boolean in BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF") - } - fallback = false // prohibit fallback on debug - } else if !isTypeWindows(sr) { - enableOverlay, fallback = true, true - switch sr.cm.ManagerOpt.Snapshotter.Name() { - case "overlayfs", "stargz": - // overlayfs-based snapshotters should support overlay diff. so print warn log on failure. - logWarnOnErr = true - case "fuse-overlayfs": - // not supported with fuse-overlayfs snapshotter which doesn't provide overlayfs mounts. - // TODO: add support for fuse-overlayfs - enableOverlay = false + var desc ocispecs.Descriptor + + // Determine differ and error/log handling according to the platform, envvar and the snapshotter. + var enableOverlay, fallback, logWarnOnErr bool + if forceOvlStr := os.Getenv("BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF"); forceOvlStr != "" { + enableOverlay, err = strconv.ParseBool(forceOvlStr) + if err != nil { + return nil, errors.Wrapf(err, "invalid boolean in BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF") + } + fallback = false // prohibit fallback on debug + } else if !isTypeWindows(sr) { + enableOverlay, fallback = true, true + switch sr.cm.Snapshotter.Name() { + case "overlayfs", "stargz": + // overlayfs-based snapshotters should support overlay diff. so print warn log on failure. + logWarnOnErr = true + case "fuse-overlayfs": + // not supported with fuse-overlayfs snapshotter which doesn't provide overlayfs mounts. + // TODO: add support for fuse-overlayfs + enableOverlay = false + } } - } - if enableOverlay { - computed, ok, err := sr.tryComputeOverlayBlob(ctx, lower, upper, mediaType, sr.ID(), compressorFunc) - if !ok || err != nil { - if !fallback { - if !ok { - return nil, errors.Errorf("overlay mounts not detected (lower=%+v,upper=%+v)", lower, upper) + if enableOverlay { + computed, ok, err := sr.tryComputeOverlayBlob(ctx, lower, upper, mediaType, sr.ID(), compressorFunc) + if !ok || err != nil { + if !fallback { + if !ok { + return nil, errors.Errorf("overlay mounts not detected (lower=%+v,upper=%+v)", lower, upper) + } + if err != nil { + return nil, errors.Wrapf(err, "failed to compute overlay diff") + } } - if err != nil { - return nil, errors.Wrapf(err, "failed to compute overlay diff") + if logWarnOnErr { + logrus.Warnf("failed to compute blob by overlay differ (ok=%v): %v", ok, err) } } - if logWarnOnErr { - logrus.Warnf("failed to compute blob by overlay differ (ok=%v): %v", ok, err) + if ok { + desc = computed } } - if ok { - desc = computed + + if desc.Digest == "" { + desc, err = sr.cm.Differ.Compare(ctx, lower, upper, + diff.WithMediaType(mediaType), + diff.WithReference(sr.ID()), + diff.WithCompressor(compressorFunc), + ) + if err != nil { + return nil, err + } } - } - if desc.Digest == "" { - desc, err = sr.cm.Differ.Compare(ctx, lower, upper, - diff.WithMediaType(mediaType), - diff.WithReference(sr.ID()), - diff.WithCompressor(compressorFunc), - ) + if desc.Annotations == nil { + desc.Annotations = map[string]string{} + } + if finalize != nil { + a, err := finalize(ctx, sr.cm.ContentStore) + if err != nil { + return nil, errors.Wrapf(err, "failed to finalize compression") + } + for k, v := range a { + desc.Annotations[k] = v + } + } + info, err := sr.cm.ContentStore.Info(ctx, desc.Digest) if err != nil { return nil, err } - } - if desc.Annotations == nil { - desc.Annotations = map[string]string{} - } - if finalize != nil { - a, err := finalize(ctx, sr.cm.ContentStore) - if err != nil { - return nil, errors.Wrapf(err, "failed to finalize compression") + if diffID, ok := info.Labels[containerdUncompressed]; ok { + desc.Annotations[containerdUncompressed] = diffID + } else if mediaType == ocispecs.MediaTypeImageLayer { + desc.Annotations[containerdUncompressed] = desc.Digest.String() + } else { + return nil, errors.Errorf("unknown layer compression type") } - for k, v := range a { - desc.Annotations[k] = v - } - } - info, err := sr.cm.ContentStore.Info(ctx, desc.Digest) + if err := sr.setBlob(ctx, compressionType, desc); err != nil { + return nil, err + } + return nil, nil + }) if err != nil { - return nil, err + return err } - if diffID, ok := info.Labels[containerdUncompressed]; ok { - desc.Annotations[containerdUncompressed] = diffID - } else if mediaType == ocispecs.MediaTypeImageLayer { - desc.Annotations[containerdUncompressed] = desc.Digest.String() - } else { - return nil, errors.Errorf("unknown layer compression type") - } - - if err := sr.setBlob(ctx, compressionType, desc); err != nil { - return nil, err + if forceCompression { + if err := ensureCompression(ctx, sr, compressionType, s); err != nil { + return errors.Wrapf(err, "failed to ensure compression type of %q", compressionType) + } } - - return nil, nil + return nil }) - if err != nil { - return err - } - if forceCompression { - if err := ensureCompression(ctx, sr, compressionType, s); err != nil { - return errors.Wrapf(err, "failed to ensure compression type of %q", compressionType) - } - } - return nil - }) + } if err := eg.Wait(); err != nil { return err } - return sr.setChains(ctx) + return sr.computeChainMetadata(ctx) } // 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, compressionType compression.Type, desc ocispecs.Descriptor) error { if _, ok := leases.FromContext(ctx); !ok { return errors.Errorf("missing lease requirement for setBlob") @@ -267,9 +274,9 @@ func (sr *immutableRef) setBlob(ctx context.Context, compressionType compression return nil } -func (sr *immutableRef) setChains(ctx context.Context) error { +func (sr *immutableRef) computeChainMetadata(ctx context.Context) error { if _, ok := leases.FromContext(ctx); !ok { - return errors.Errorf("missing lease requirement for setChains") + return errors.Errorf("missing lease requirement for computeChainMetadata") } sr.mu.Lock() @@ -281,13 +288,37 @@ func (sr *immutableRef) setChains(ctx context.Context) error { var chainIDs []digest.Digest var blobChainIDs []digest.Digest - if sr.parent != nil { - chainIDs = append(chainIDs, digest.Digest(sr.parent.getChainID())) - blobChainIDs = append(blobChainIDs, digest.Digest(sr.parent.getBlobChainID())) + + // Blobs should be set the actual layers in the ref's chain, no + // any merge refs. + layerChain := sr.layerChain() + var layerParent *cacheRecord + switch sr.kind() { + case Merge: + layerParent = layerChain[len(layerChain)-1].cacheRecord + case Layer: + // skip the last layer in the chain, which is this ref itself + layerParent = layerChain[len(layerChain)-2].cacheRecord + } + if layerParent != nil { + if parentChainID := layerParent.getChainID(); parentChainID != "" { + chainIDs = append(chainIDs, parentChainID) + } else { + return errors.Errorf("failed to set chain for reference with non-addressable parent") + } + if parentBlobChainID := layerParent.getBlobChainID(); parentBlobChainID != "" { + blobChainIDs = append(blobChainIDs, parentBlobChainID) + } else { + return errors.Errorf("failed to set blobchain for reference with non-addressable parent") + } + } + + switch sr.kind() { + case Layer, BaseLayer: + diffID := digest.Digest(sr.getDiffID()) + chainIDs = append(chainIDs, diffID) + blobChainIDs = append(blobChainIDs, imagespecidentity.ChainID([]digest.Digest{digest.Digest(sr.getBlob()), diffID})) } - diffID := digest.Digest(sr.getDiffID()) - chainIDs = append(chainIDs, diffID) - blobChainIDs = append(blobChainIDs, imagespecidentity.ChainID([]digest.Digest{digest.Digest(sr.getBlob()), diffID})) chainID := imagespecidentity.ChainID(chainIDs) blobChainID := imagespecidentity.ChainID(blobChainIDs) @@ -304,8 +335,15 @@ func isTypeWindows(sr *immutableRef) bool { if sr.GetLayerType() == "windows" { return true } - if parent := sr.parent; parent != nil { - return isTypeWindows(parent) + switch sr.kind() { + case Merge: + for _, p := range sr.mergeParents { + if isTypeWindows(p) { + return true + } + } + case Layer: + return isTypeWindows(sr.layerParent) } return false } diff --git a/cache/contenthash/checksum_test.go b/cache/contenthash/checksum_test.go index 8d8efdf34774..4ca11b0ee708 100644 --- a/cache/contenthash/checksum_test.go +++ b/cache/contenthash/checksum_test.go @@ -13,6 +13,8 @@ import ( "time" "github.com/containerd/containerd/content/local" + "github.com/containerd/containerd/diff/apply" + "github.com/containerd/containerd/diff/walking" ctdmetadata "github.com/containerd/containerd/metadata" "github.com/containerd/containerd/snapshots" "github.com/containerd/containerd/snapshots/native" @@ -22,6 +24,7 @@ import ( "github.com/moby/buildkit/snapshot" containerdsnapshot "github.com/moby/buildkit/snapshot/containerd" "github.com/moby/buildkit/util/leaseutil" + "github.com/moby/buildkit/util/winlayers" digest "github.com/opencontainers/go-digest" "github.com/pkg/errors" "github.com/stretchr/testify/require" @@ -1209,12 +1212,17 @@ func setupCacheManager(t *testing.T, tmpdir string, snapshotterName string, snap md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) require.NoError(t, err) lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") + c := mdb.ContentStore() + applier := winlayers.NewFileSystemApplierWithWindows(c, apply.NewFileSystemApplier(c)) + differ := winlayers.NewWalkingDiffWithWindows(c, walking.NewWalkingDiff(c)) cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter(snapshotterName, containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter(snapshotterName)), nil), MetadataStore: md, LeaseManager: lm, - ContentStore: mdb.ContentStore(), + ContentStore: c, + Applier: applier, + Differ: differ, GarbageCollect: mdb.GarbageCollect, }) require.NoError(t, err) diff --git a/cache/manager.go b/cache/manager.go index 8399262afb3c..75e4af32fab3 100644 --- a/cache/manager.go +++ b/cache/manager.go @@ -3,6 +3,7 @@ package cache import ( "context" "sort" + "strings" "sync" "time" @@ -18,6 +19,7 @@ import ( "github.com/moby/buildkit/identity" "github.com/moby/buildkit/session" "github.com/moby/buildkit/snapshot" + "github.com/moby/buildkit/util/bklog" "github.com/moby/buildkit/util/flightcontrol" digest "github.com/opencontainers/go-digest" imagespecidentity "github.com/opencontainers/image-spec/identity" @@ -53,6 +55,7 @@ type Accessor interface { 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 + Merge(ctx context.Context, parents []ImmutableRef, opts ...RefOption) (ImmutableRef, error) } type Controller interface { @@ -73,9 +76,16 @@ type ExternalRefChecker interface { } type cacheManager struct { - records map[string]*cacheRecord - mu sync.Mutex - ManagerOpt + records map[string]*cacheRecord + mu sync.Mutex + Snapshotter snapshot.MergeSnapshotter + ContentStore content.Store + LeaseManager leases.Manager + PruneRefChecker ExternalRefCheckerFunc + GarbageCollect func(ctx context.Context) (gc.Stats, error) + Applier diff.Applier + Differ diff.Comparer + MetadataStore *metadata.Store muPrune sync.Mutex // make sure parallel prune is not allowed so there will not be inconsistent results unlazyG flightcontrol.Group @@ -83,8 +93,15 @@ type cacheManager struct { func NewManager(opt ManagerOpt) (Manager, error) { cm := &cacheManager{ - ManagerOpt: opt, - records: make(map[string]*cacheRecord), + Snapshotter: snapshot.NewMergeSnapshotter(context.TODO(), opt.Snapshotter, opt.LeaseManager), + ContentStore: opt.ContentStore, + LeaseManager: opt.LeaseManager, + PruneRefChecker: opt.PruneRefChecker, + GarbageCollect: opt.GarbageCollect, + Applier: opt.Applier, + Differ: opt.Differ, + MetadataStore: opt.MetadataStore, + records: make(map[string]*cacheRecord), } if err := cm.init(context.TODO()); err != nil { @@ -114,7 +131,6 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, } var p *immutableRef - var parentID string if parent != nil { p2, err := cm.Get(ctx, parent.ID(), NoUpdateLastUsed, descHandlers) if err != nil { @@ -122,18 +138,17 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, } p = p2.(*immutableRef) + if err := p.Finalize(ctx); err != nil { + p.Release(context.TODO()) + return nil, err + } + if p.getChainID() == "" || p.getBlobChainID() == "" { p.Release(context.TODO()) return nil, errors.Errorf("failed to get ref by blob on non-addressable parent") } chainID = imagespecidentity.ChainID([]digest.Digest{p.getChainID(), chainID}) blobChainID = imagespecidentity.ChainID([]digest.Digest{p.getBlobChainID(), blobChainID}) - - if err := p.Finalize(ctx); err != nil { - p.Release(context.TODO()) - return nil, err - } - parentID = p.ID() } releaseParent := false @@ -195,7 +210,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, go link.Release(context.TODO()) } - l, err := cm.ManagerOpt.LeaseManager.Create(ctx, func(l *leases.Lease) error { + l, err := cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { l.ID = id l.Labels = map[string]string{ "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), @@ -208,7 +223,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, defer func() { if rerr != nil { - if err := cm.ManagerOpt.LeaseManager.Delete(context.TODO(), leases.Lease{ + if err := cm.LeaseManager.Delete(context.TODO(), leases.Lease{ ID: l.ID, }); err != nil { logrus.Errorf("failed to remove lease: %+v", err) @@ -224,7 +239,7 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, } if desc.Digest != "" { - if err := cm.ManagerOpt.LeaseManager.AddResource(ctx, leases.Lease{ID: id}, leases.Resource{ + if err := cm.LeaseManager.AddResource(ctx, leases.Lease{ID: id}, leases.Resource{ ID: desc.Digest.String(), Type: "content", }); err != nil { @@ -238,11 +253,11 @@ func (cm *cacheManager) GetByBlob(ctx context.Context, desc ocispecs.Descriptor, mu: &sync.Mutex{}, cm: cm, refs: make(map[ref]struct{}), - parent: p, + parentRefs: parentRefs{layerParent: p}, cacheMetadata: md, } - if err := initializeMetadata(rec.cacheMetadata, parentID, opts...); err != nil { + if err := initializeMetadata(rec.cacheMetadata, rec.parentRefs, opts...); err != nil { return nil, err } @@ -342,18 +357,16 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt checkLazyProviders := func(rec *cacheRecord) error { missing := NeedsRemoteProvidersError(nil) dhs := descHandlersOf(opts...) - for { - blob := rec.getBlob() - if isLazy, err := rec.isLazy(ctx); err != nil { + if err := rec.walkUniqueAncestors(func(cr *cacheRecord) error { + blob := cr.getBlob() + if isLazy, err := cr.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 + return nil + }); err != nil { + return err } if len(missing) > 0 { return missing @@ -375,6 +388,17 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt if !ok { return nil, errors.Wrap(errNotFound, id) } + + parents, err := cm.parentsOf(ctx, md, opts...) + if err != nil { + return nil, errors.Wrapf(err, "failed to get parents") + } + defer func() { + if retErr != nil { + parents.release(context.TODO()) + } + }() + if mutableID := md.getEqualMutable(); mutableID != "" { mutable, err := cm.getRecord(ctx, mutableID) if err != nil { @@ -385,16 +409,11 @@ 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, dhs), + parentRefs: parents, cacheMetadata: md, equalMutable: &mutableRef{cacheRecord: mutable}, } @@ -403,28 +422,12 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt return rec, nil } - var parent *immutableRef - if parentID := md.getParent(); parentID != "" { - var err error - parent, err = cm.get(ctx, parentID, append(opts, NoUpdateLastUsed)...) - if err != nil { - return nil, err - } - defer func() { - if retErr != nil { - parent.mu.Lock() - parent.release(context.TODO()) - parent.mu.Unlock() - } - }() - } - rec := &cacheRecord{ mu: &sync.Mutex{}, mutable: !md.getCommitted(), cm: cm, refs: make(map[ref]struct{}), - parent: parent, + parentRefs: parents, cacheMetadata: md, } @@ -436,7 +439,7 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt return nil, errors.Wrapf(errNotFound, "failed to get deleted record %s", id) } - if err := initializeMetadata(rec.cacheMetadata, md.getParent(), opts...); err != nil { + if err := initializeMetadata(rec.cacheMetadata, rec.parentRefs, opts...); err != nil { return nil, err } @@ -451,11 +454,29 @@ func (cm *cacheManager) getRecord(ctx context.Context, id string, opts ...RefOpt return rec, nil } +func (cm *cacheManager) parentsOf(ctx context.Context, md *cacheMetadata, opts ...RefOption) (ps parentRefs, rerr error) { + if parentID := md.getParent(); parentID != "" { + p, err := cm.get(ctx, parentID, append(opts, NoUpdateLastUsed)) + if err != nil { + return ps, err + } + ps.layerParent = p + return ps, nil + } + for _, parentID := range md.getMergeParents() { + p, err := cm.get(ctx, parentID, append(opts, NoUpdateLastUsed)) + if err != nil { + return ps, err + } + ps.mergeParents = append(ps.mergeParents, p) + } + return ps, nil +} + 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 { if _, ok := s.(*immutableRef); ok { @@ -474,7 +495,6 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr return nil, err } parentSnapshotID = parent.getSnapshotID() - parentID = parent.ID() } defer func() { @@ -483,7 +503,7 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr } }() - l, err := cm.ManagerOpt.LeaseManager.Create(ctx, func(l *leases.Lease) error { + l, err := cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { l.ID = id l.Labels = map[string]string{ "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), @@ -496,7 +516,7 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr defer func() { if err != nil { - if err := cm.ManagerOpt.LeaseManager.Delete(context.TODO(), leases.Lease{ + if err := cm.LeaseManager.Delete(context.TODO(), leases.Lease{ ID: l.ID, }); err != nil { logrus.Errorf("failed to remove lease: %+v", err) @@ -505,7 +525,7 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr }() snapshotID := id - if err := cm.LeaseManager.AddResource(ctx, leases.Lease{ID: id}, leases.Resource{ + if err := cm.LeaseManager.AddResource(ctx, l, leases.Resource{ ID: snapshotID, Type: "snapshots/" + cm.Snapshotter.Name(), }); err != nil && !errdefs.IsAlreadyExists(err) { @@ -522,7 +542,7 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr err = cm.Snapshotter.Prepare(ctx, snapshotID, parentSnapshotID) } if err != nil { - return nil, errors.Wrapf(err, "failed to prepare %s", parentSnapshotID) + return nil, errors.Wrapf(err, "failed to prepare %v as %s", parentSnapshotID, snapshotID) } cm.mu.Lock() @@ -535,12 +555,12 @@ func (cm *cacheManager) New(ctx context.Context, s ImmutableRef, sess session.Gr mutable: true, cm: cm, refs: make(map[ref]struct{}), - parent: parent, + parentRefs: parentRefs{layerParent: parent}, cacheMetadata: md, } opts = append(opts, withSnapshotID(snapshotID)) - if err := initializeMetadata(rec.cacheMetadata, parentID, opts...); err != nil { + if err := initializeMetadata(rec.cacheMetadata, rec.parentRefs, opts...); err != nil { return nil, err } @@ -591,6 +611,128 @@ func (cm *cacheManager) GetMutable(ctx context.Context, id string, opts ...RefOp return rec.mref(true, descHandlersOf(opts...)), nil } +func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef, opts ...RefOption) (ir ImmutableRef, rerr error) { + // TODO:(sipsma) optimize merge further by + // * Removing repeated occurrences of input layers (only leaving the uppermost) + // * Reusing existing merges that are equivalent to this one + // * Reusing existing merges that can be used as a base for this one + // * Calculating diffs only once (across both merges and during computeBlobChain). Save diff metadata so it can be reapplied. + // These optimizations may make sense here in cache, in the snapshotter or both. + // Be sure that any optimizations handle existing pre-optimization refs correctly. + + id := identity.NewID() + + parents := parentRefs{mergeParents: make([]*immutableRef, 0, len(inputParents))} + dhs := make(map[digest.Digest]*DescHandler) + defer func() { + if rerr != nil { + parents.release(context.TODO()) + } + }() + for _, inputParent := range inputParents { + if inputParent == nil { + continue + } + var parent *immutableRef + if p, ok := inputParent.(*immutableRef); ok { + parent = p + } else { + // inputParent implements ImmutableRef but isn't our internal struct, get an instance of the internal struct + // by calling Get on its ID. + p, err := cm.Get(ctx, inputParent.ID(), append(opts, NoUpdateLastUsed)...) + if err != nil { + return nil, err + } + parent = p.(*immutableRef) + defer parent.Release(context.TODO()) + } + switch parent.kind() { + case Merge: + // if parent is itself a merge, flatten it out by just setting our parents directly to its parents + for _, grandparent := range parent.mergeParents { + parents.mergeParents = append(parents.mergeParents, grandparent.clone()) + } + case Layer, BaseLayer: + parents.mergeParents = append(parents.mergeParents, parent.clone()) + } + for dgst, handler := range parent.descHandlers { + dhs[dgst] = handler + } + } + + if len(parents.mergeParents) == 0 { + // merge of nothing is nothing + return nil, nil + } + if len(parents.mergeParents) == 1 { + // merge of 1 thing is that thing + return parents.mergeParents[0], nil + } + + for _, parent := range parents.mergeParents { + if err := parent.Finalize(ctx); err != nil { + return nil, errors.Wrapf(err, "failed to finalize parent during merge") + } + } + + cm.mu.Lock() + defer cm.mu.Unlock() + + // Build the new ref + md, _ := cm.getMetadata(id) + + rec := &cacheRecord{ + mu: &sync.Mutex{}, + mutable: false, + cm: cm, + cacheMetadata: md, + parentRefs: parents, + refs: make(map[ref]struct{}), + } + + if err := initializeMetadata(rec.cacheMetadata, rec.parentRefs, opts...); err != nil { + return nil, err + } + + snapshotID := id + l, err := cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { + l.ID = id + l.Labels = map[string]string{ + "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), + } + return nil + }) + if err != nil { + return nil, errors.Wrap(err, "failed to create lease") + } + defer func() { + if rerr != nil { + if err := cm.LeaseManager.Delete(context.TODO(), leases.Lease{ + ID: l.ID, + }); err != nil { + bklog.G(ctx).Errorf("failed to remove lease: %+v", err) + } + } + }() + + if err := cm.LeaseManager.AddResource(ctx, leases.Lease{ID: id}, leases.Resource{ + ID: snapshotID, + Type: "snapshots/" + cm.Snapshotter.Name(), + }); err != nil { + return nil, err + } + + rec.queueSnapshotID(snapshotID) + + if err := rec.commitMetadata(); err != nil { + return nil, err + } + + cm.records[id] = rec + + return rec.ref(true, dhs), nil +} + func (cm *cacheManager) Prune(ctx context.Context, ch chan client.UsageInfo, opts ...client.PruneInfo) error { cm.muPrune.Lock() @@ -690,7 +832,7 @@ func (cm *cacheManager) prune(ctx context.Context, ch chan client.UsageInfo, opt shared := false if opt.checkShared != nil { - shared = opt.checkShared.Exists(cr.ID(), cr.parentChain()) + shared = opt.checkShared.Exists(cr.ID(), cr.layerDigestChain()) } if !opt.all { @@ -806,8 +948,14 @@ func (cm *cacheManager) prune(ctx context.Context, ch chan client.UsageInfo, opt UsageCount: usageCount, } - if cr.parent != nil { - c.Parent = cr.parent.ID() + switch cr.kind() { + case Layer: + c.Parents = []string{cr.layerParent.ID()} + case Merge: + c.Parents = make([]string, len(cr.mergeParents)) + for i, p := range cr.mergeParents { + c.Parents[i] = p.ID() + } } if c.Size == sizeUnknown && cr.equalImmutable != nil { c.Size = cr.equalImmutable.getSize() // benefit from DiskUsage calc @@ -851,12 +999,15 @@ func (cm *cacheManager) markShared(m map[string]*cacheUsageInfo) error { return errors.WithStack(err) } - var markAllParentsShared func(string) - markAllParentsShared = func(id string) { - if v, ok := m[id]; ok { - v.shared = true - if v.parent != "" { - markAllParentsShared(v.parent) + var markAllParentsShared func(...string) + markAllParentsShared = func(ids ...string) { + for _, id := range ids { + if id == "" { + continue + } + if v, ok := m[id]; ok { + v.shared = true + markAllParentsShared(v.parents...) } } } @@ -874,7 +1025,7 @@ func (cm *cacheManager) markShared(m map[string]*cacheUsageInfo) error { type cacheUsageInfo struct { refs int - parent string + parents []string size int64 mutable bool createdAt time.Time @@ -917,13 +1068,20 @@ func (cm *cacheManager) DiskUsage(ctx context.Context, opt client.DiskUsageInfo) description: cr.GetDescription(), doubleRef: cr.equalImmutable != nil, recordType: cr.GetRecordType(), - parentChain: cr.parentChain(), + parentChain: cr.layerDigestChain(), } if c.recordType == "" { c.recordType = client.UsageRecordTypeRegular } - if cr.parent != nil { - c.parent = cr.parent.ID() + + switch cr.kind() { + case Layer: + c.parents = []string{cr.layerParent.ID()} + case Merge: + c.parents = make([]string, len(cr.mergeParents)) + for i, p := range cr.mergeParents { + c.parents[i] = p.ID() + } } if cr.mutable && c.refs > 0 { c.size = 0 // size can not be determined because it is changing @@ -940,12 +1098,14 @@ func (cm *cacheManager) DiskUsage(ctx context.Context, opt client.DiskUsageInfo) } for id := range rescan { v := m[id] - if v.refs == 0 && v.parent != "" { - m[v.parent].refs-- - if v.doubleRef { - m[v.parent].refs-- + if v.refs == 0 { + for _, p := range v.parents { + m[p].refs-- + if v.doubleRef { + m[p].refs-- + } + rescan[p] = struct{}{} } - rescan[v.parent] = struct{}{} } delete(rescan, id) } @@ -962,7 +1122,7 @@ func (cm *cacheManager) DiskUsage(ctx context.Context, opt client.DiskUsageInfo) Mutable: cr.mutable, InUse: cr.refs > 0, Size: cr.size, - Parent: cr.parent, + Parents: cr.parents, CreatedAt: cr.createdAt, Description: cr.description, LastUsedAt: cr.lastUsedAt, @@ -1079,13 +1239,23 @@ func withSnapshotID(id string) RefOption { }) } -func initializeMetadata(m *cacheMetadata, parent string, opts ...RefOption) error { +func initializeMetadata(m *cacheMetadata, parents parentRefs, opts ...RefOption) error { if tm := m.GetCreatedAt(); !tm.IsZero() { return nil } - if err := m.queueParent(parent); err != nil { - return err + if parents.layerParent != nil { + if err := m.queueParent(parents.layerParent.ID()); err != nil { + return err + } + } else if len(parents.mergeParents) > 0 { + var ids []string + for _, p := range parents.mergeParents { + ids = append(ids, p.ID()) + } + if err := m.queueMergeParents(ids); err != nil { + return err + } } if err := m.queueCreatedAt(time.Now()); err != nil { @@ -1112,8 +1282,8 @@ func adaptUsageInfo(info *client.UsageInfo) filters.Adaptor { switch fieldpath[0] { case "id": return info.ID, info.ID != "" - case "parent": - return info.Parent, info.Parent != "" + case "parents": + return strings.Join(info.Parents, ";"), len(info.Parents) > 0 case "description": return info.Description, info.Description != "" case "inuse": diff --git a/cache/manager_test.go b/cache/manager_test.go index c4fc244a9a55..4ccb2622696a 100644 --- a/cache/manager_test.go +++ b/cache/manager_test.go @@ -29,6 +29,7 @@ import ( "github.com/containerd/containerd/namespaces" "github.com/containerd/containerd/snapshots" "github.com/containerd/containerd/snapshots/native" + "github.com/containerd/continuity/fs/fstest" "github.com/containerd/stargz-snapshotter/estargz" "github.com/klauspost/compress/zstd" "github.com/moby/buildkit/cache/metadata" @@ -128,8 +129,11 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() return nil, nil, err } - store = containerdsnapshot.NewContentStore(mdb.ContentStore(), ns) + c := mdb.ContentStore() + store = containerdsnapshot.NewContentStore(c, ns) lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), ns) + applier := winlayers.NewFileSystemApplierWithWindows(store, apply.NewFileSystemApplier(store)) + differ := winlayers.NewWalkingDiffWithWindows(store, walking.NewWalkingDiff(store)) md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) if err != nil { @@ -142,8 +146,8 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() ContentStore: store, LeaseManager: lm, GarbageCollect: mdb.GarbageCollect, - Applier: winlayers.NewFileSystemApplierWithWindows(store, apply.NewFileSystemApplier(store)), - Differ: winlayers.NewWalkingDiffWithWindows(store, walking.NewWalkingDiff(store)), + Applier: applier, + Differ: differ, }) if err != nil { return nil, nil, err @@ -529,7 +533,7 @@ func TestExtractOnMutable(t *testing.T) { err = snap.(*immutableRef).setBlob(leaseCtx, compressionType, desc) done(context.TODO()) require.NoError(t, err) - err = snap.(*immutableRef).setChains(leaseCtx) + err = snap.(*immutableRef).computeChainMetadata(leaseCtx) require.NoError(t, err) snap2, err := cm.GetByBlob(ctx, desc2, snap) @@ -652,7 +656,7 @@ func TestSetBlob(t *testing.T) { } err = snap.(*immutableRef).setBlob(ctx, compressionType, desc) require.NoError(t, err) - err = snap.(*immutableRef).setChains(ctx) + err = snap.(*immutableRef).computeChainMetadata(ctx) require.NoError(t, err) snapRef = snap.(*immutableRef) @@ -682,7 +686,7 @@ func TestSetBlob(t *testing.T) { } err = snap2.(*immutableRef).setBlob(ctx, compressionType2, desc2) require.NoError(t, err) - err = snap2.(*immutableRef).setChains(ctx) + err = snap2.(*immutableRef).computeChainMetadata(ctx) require.NoError(t, err) snapRef2 := snap2.(*immutableRef) @@ -1273,7 +1277,7 @@ func TestGetRemotes(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, len(remotes)) remote := remotes[0] - refChain := ir.parentRefChain() + refChain := ir.layerChain() for i, desc := range remote.Descriptors { switch compressionType { case compression.Uncompressed: @@ -1494,6 +1498,122 @@ func checkDescriptor(ctx context.Context, t *testing.T, cs content.Store, desc o } } +func TestMergeOp(t *testing.T) { + if runtime.GOOS == "windows" { + t.Skip("Depends on unimplemented merge-op support on Windows") + } + + // This just tests the basic Merge method and some of the logic with releasing merge refs. + // Tests for the fs merge logic are in client_test and snapshotter_test. + t.Parallel() + + ctx := namespaces.WithNamespace(context.Background(), "buildkit-test") + + tmpdir, err := ioutil.TempDir("", "cachemanager") + require.NoError(t, err) + defer os.RemoveAll(tmpdir) + + snapshotter, err := native.NewSnapshotter(filepath.Join(tmpdir, "snapshots")) + require.NoError(t, err) + + co, cleanup, err := newCacheManager(ctx, cmOpt{ + snapshotter: snapshotter, + snapshotterName: "native", + }) + require.NoError(t, err) + defer cleanup() + cm := co.manager + + emptyMerge, err := cm.Merge(ctx, nil) + require.NoError(t, err) + require.Nil(t, emptyMerge) + + var baseRefs []ImmutableRef + for i := 0; i < 6; i++ { + active, err := cm.New(ctx, nil, nil) + require.NoError(t, err) + m, err := active.Mount(ctx, false, nil) + require.NoError(t, err) + lm := snapshot.LocalMounter(m) + target, err := lm.Mount() + require.NoError(t, err) + err = fstest.Apply( + fstest.CreateFile(strconv.Itoa(i), []byte(strconv.Itoa(i)), 0777), + ).Apply(target) + require.NoError(t, err) + err = lm.Unmount() + require.NoError(t, err) + snap, err := active.Commit(ctx) + require.NoError(t, err) + baseRefs = append(baseRefs, snap) + size, err := snap.(*immutableRef).size(ctx) + require.NoError(t, err) + require.EqualValues(t, 8192, size) + } + + singleMerge, err := cm.Merge(ctx, baseRefs[:1]) + require.NoError(t, err) + m, err := singleMerge.Mount(ctx, true, nil) + require.NoError(t, err) + ms, unmount, err := m.Mount() + require.NoError(t, err) + require.Len(t, ms, 1) + require.Equal(t, ms[0].Type, "bind") + err = fstest.CheckDirectoryEqualWithApplier(ms[0].Source, fstest.Apply( + fstest.CreateFile(strconv.Itoa(0), []byte(strconv.Itoa(0)), 0777), + )) + require.NoError(t, err) + require.NoError(t, unmount()) + require.NoError(t, singleMerge.Release(ctx)) + + err = cm.Prune(ctx, nil, client.PruneInfo{Filter: []string{ + "id==" + singleMerge.ID(), + }}) + require.NoError(t, err) + + merge1, err := cm.Merge(ctx, baseRefs[:3]) + require.NoError(t, err) + _, err = merge1.Mount(ctx, true, nil) + require.NoError(t, err) + size1, err := merge1.(*immutableRef).size(ctx) + require.NoError(t, err) + require.EqualValues(t, 4096, size1) // hardlinking means all but the first snapshot doesn't take up space + checkDiskUsage(ctx, t, cm, 7, 0) + + merge2, err := cm.Merge(ctx, baseRefs[3:]) + require.NoError(t, err) + _, err = merge2.Mount(ctx, true, nil) + require.NoError(t, err) + size2, err := merge2.(*immutableRef).size(ctx) + require.NoError(t, err) + require.EqualValues(t, 4096, size2) + checkDiskUsage(ctx, t, cm, 8, 0) + + for _, ref := range baseRefs { + require.NoError(t, ref.Release(ctx)) + } + checkDiskUsage(ctx, t, cm, 8, 0) + // should still be able to use merges based on released refs + + merge3, err := cm.Merge(ctx, []ImmutableRef{merge1, merge2}) + require.NoError(t, err) + require.NoError(t, merge1.Release(ctx)) + require.NoError(t, merge2.Release(ctx)) + _, err = merge3.Mount(ctx, true, nil) + require.NoError(t, err) + size3, err := merge3.(*immutableRef).size(ctx) + require.NoError(t, err) + require.EqualValues(t, 4096, size3) + require.Len(t, merge3.(*immutableRef).mergeParents, 6) + checkDiskUsage(ctx, t, cm, 7, 2) + + require.NoError(t, merge3.Release(ctx)) + checkDiskUsage(ctx, t, cm, 0, 9) + err = cm.Prune(ctx, nil, client.PruneInfo{All: true}) + require.NoError(t, err) + checkDiskUsage(ctx, t, cm, 0, 0) +} + func checkDiskUsage(ctx context.Context, t *testing.T, cm Manager, inuse, unused int) { du, err := cm.DiskUsage(ctx, client.DiskUsageInfo{}) require.NoError(t, err) diff --git a/cache/metadata.go b/cache/metadata.go index 84b634ee04de..8a8c72488a00 100644 --- a/cache/metadata.go +++ b/cache/metadata.go @@ -24,6 +24,7 @@ const keyLayerType = "cache.layerType" const keyRecordType = "cache.recordType" const keyCommitted = "snapshot.committed" const keyParent = "cache.parent" +const keyMergeParents = "cache.mergeParents" const keyDiffID = "cache.diffID" const keyChainID = "cache.chainID" const keyBlobChainID = "cache.blobChainID" @@ -306,6 +307,14 @@ func (md *cacheMetadata) getParent() string { return md.GetString(keyParent) } +func (md *cacheMetadata) queueMergeParents(parents []string) error { + return md.queueValue(keyMergeParents, parents, "") +} + +func (md *cacheMetadata) getMergeParents() []string { + return md.getStringSlice(keyMergeParents) +} + func (md *cacheMetadata) queueSize(s int64) error { return md.queueValue(keySize, s, "") } diff --git a/cache/refs.go b/cache/refs.go index cc595cd55993..c574b1244847 100644 --- a/cache/refs.go +++ b/cache/refs.go @@ -69,7 +69,7 @@ type cacheRecord struct { mutable bool refs map[ref]struct{} - parent *immutableRef + parentRefs *cacheMetadata // dead means record is marked as deleted @@ -83,7 +83,7 @@ type cacheRecord struct { equalMutable *mutableRef equalImmutable *immutableRef - parentChainCache []digest.Digest + layerDigestChainCache []digest.Digest } // hold ref lock before calling @@ -108,24 +108,66 @@ func (cr *cacheRecord) mref(triggerLastUsed bool, descHandlers DescHandlers) *mu return ref } -func (cr *cacheRecord) parentChain() []digest.Digest { - if cr.parentChainCache != nil { - return cr.parentChainCache +// parentRefs is a disjoint union type that holds either a single layerParent for this record, a list +// of parents if this is a merged record or all nil fields if this record has no parents. At most one +// field should be non-nil at a time. +type parentRefs struct { + layerParent *immutableRef + mergeParents []*immutableRef +} + +// caller must hold cacheManager.mu +func (p parentRefs) release(ctx context.Context) (rerr error) { + if p.layerParent != nil { + p.layerParent.mu.Lock() + defer p.layerParent.mu.Unlock() + rerr = p.layerParent.release(ctx) + } else if len(p.mergeParents) > 0 { + for i, parent := range p.mergeParents { + if parent == nil { + continue + } + parent.mu.Lock() + if err := parent.release(ctx); err != nil { + rerr = multierror.Append(rerr, err).ErrorOrNil() + } else { + p.mergeParents[i] = nil + } + parent.mu.Unlock() + } } - blob := cr.getBlob() - if blob == "" { - return nil + return rerr +} + +func (p parentRefs) clone() parentRefs { + if p.layerParent != nil { + p.layerParent = p.layerParent.clone() + } else if len(p.mergeParents) > 0 { + newParents := make([]*immutableRef, len(p.mergeParents)) + for i, p := range p.mergeParents { + newParents[i] = p.clone() + } + p.mergeParents = newParents } + return p +} + +type refKind int + +const ( + BaseLayer refKind = iota + Layer + Merge +) - var parent []digest.Digest - if cr.parent != nil { - parent = cr.parent.parentChain() +func (cr *cacheRecord) kind() refKind { + if len(cr.mergeParents) > 0 { + return Merge } - pcc := make([]digest.Digest, len(parent)+1) - copy(pcc, parent) - pcc[len(parent)] = digest.Digest(blob) - cr.parentChainCache = pcc - return pcc + if cr.layerParent != nil { + return Layer + } + return BaseLayer } // hold ref lock before calling @@ -133,6 +175,48 @@ func (cr *cacheRecord) isDead() bool { return cr.dead || (cr.equalImmutable != nil && cr.equalImmutable.dead) || (cr.equalMutable != nil && cr.equalMutable.dead) } +var errSkipWalk = errors.New("skip") + +// walkAncestors calls the provided func on cr and each of its ancestors, counting both layer +// and merge parents. It starts at cr and does a depth-first walk to parents. It will visit +// a record and its parents multiple times if encountered more than once. It will only skip +// visiting parents of a record if errSkipWalk is returned. If any other error is returned, +// the walk will stop and return the error to the caller. +func (cr *cacheRecord) walkAncestors(f func(*cacheRecord) error) error { + curs := []*cacheRecord{cr} + for len(curs) > 0 { + cur := curs[len(curs)-1] + curs = curs[:len(curs)-1] + if err := f(cur); err != nil { + if errors.Is(err, errSkipWalk) { + continue + } + return err + } + switch cur.kind() { + case Layer: + curs = append(curs, cur.layerParent.cacheRecord) + case Merge: + for _, p := range cur.mergeParents { + curs = append(curs, p.cacheRecord) + } + } + } + return nil +} + +// walkUniqueAncestors calls walkAncestors but skips a record if it's already been visited. +func (cr *cacheRecord) walkUniqueAncestors(f func(*cacheRecord) error) error { + memo := make(map[*cacheRecord]struct{}) + return cr.walkAncestors(func(cr *cacheRecord) error { + if _, ok := memo[cr]; ok { + return errSkipWalk + } + memo[cr] = struct{}{} + return f(cr) + }) +} + func (cr *cacheRecord) isLazy(ctx context.Context) (bool, error) { if !cr.getBlobOnly() { return false, nil @@ -188,7 +272,7 @@ func (cr *cacheRecord) size(ctx context.Context) (int64, error) { var usage snapshots.Usage if !cr.getBlobOnly() { var err error - usage, err = cr.cm.ManagerOpt.Snapshotter.Usage(ctx, driverID) + usage, err = cr.cm.Snapshotter.Usage(ctx, driverID) if err != nil { cr.mu.Lock() isDead := cr.isDead() @@ -236,16 +320,6 @@ func (cr *cacheRecord) size(ctx context.Context) (int64, error) { return s.(int64), nil } -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, descHandlers) -} - // caller must hold cr.mu func (cr *cacheRecord) mount(ctx context.Context, s session.Group) (_ snapshot.Mountable, rerr error) { if cr.mountCache != nil { @@ -304,14 +378,6 @@ func (cr *cacheRecord) mount(ctx context.Context, s session.Group) (_ snapshot.M // call when holding the manager lock func (cr *cacheRecord) remove(ctx context.Context, removeSnapshot bool) error { delete(cr.cm.records, cr.ID()) - if cr.parent != nil { - cr.parent.mu.Lock() - err := cr.parent.release(ctx) - cr.parent.mu.Unlock() - if err != nil { - return err - } - } if removeSnapshot { if err := cr.cm.LeaseManager.Delete(ctx, leases.Lease{ ID: cr.ID(), @@ -320,7 +386,10 @@ func (cr *cacheRecord) remove(ctx context.Context, removeSnapshot bool) error { } } if err := cr.cm.MetadataStore.Clear(cr.ID()); err != nil { - return err + return errors.Wrapf(err, "failed to delete metadata of %s", cr.ID()) + } + if err := cr.parentRefs.release(ctx); err != nil { + return errors.Wrapf(err, "failed to release parents of %s", cr.ID()) } return nil } @@ -331,6 +400,58 @@ type immutableRef struct { descHandlers DescHandlers } +// Order is from parent->child, sr will be at end of slice. Refs should not +// be released as they are used internally in the underlying cacheRecords. +func (sr *immutableRef) layerChain() []*immutableRef { + var count int + sr.layerWalk(func(*immutableRef) { + count++ + }) + layers := make([]*immutableRef, count) + var index int + sr.layerWalk(func(sr *immutableRef) { + layers[index] = sr + index++ + }) + return layers +} + +// layerWalk visits each ref representing an actual layer in the chain for +// sr (including sr). The layers are visited from lowest->highest as ordered +// in the remote for the ref. +func (sr *immutableRef) layerWalk(f func(*immutableRef)) { + switch sr.kind() { + case Merge: + for _, parent := range sr.mergeParents { + parent.layerWalk(f) + } + case Layer: + sr.layerParent.layerWalk(f) + fallthrough + case BaseLayer: + f(sr) + } +} + +// hold cacheRecord.mu lock before calling +func (cr *cacheRecord) layerDigestChain() []digest.Digest { + if cr.layerDigestChainCache != nil { + return cr.layerDigestChainCache + } + switch cr.kind() { + case Merge: + for _, parent := range cr.mergeParents { + cr.layerDigestChainCache = append(cr.layerDigestChainCache, parent.layerDigestChain()...) + } + case Layer: + cr.layerDigestChainCache = append(cr.layerDigestChainCache, cr.layerParent.layerDigestChain()...) + fallthrough + case BaseLayer: + cr.layerDigestChainCache = append(cr.layerDigestChainCache, cr.getBlob()) + } + return cr.layerDigestChainCache +} + type RefList []ImmutableRef func (l RefList) Release(ctx context.Context) (rerr error) { @@ -348,7 +469,7 @@ func (l RefList) Release(ctx context.Context) (rerr error) { } func (sr *immutableRef) LayerChain() RefList { - chain := sr.parentRefChain() + chain := sr.layerChain() l := RefList(make([]ImmutableRef, len(chain))) for i, p := range chain { l[i] = p.Clone() @@ -370,13 +491,17 @@ func (sr *mutableRef) DescHandler(dgst digest.Digest) *DescHandler { return sr.descHandlers[dgst] } -func (sr *immutableRef) Clone() ImmutableRef { +func (sr *immutableRef) clone() *immutableRef { sr.mu.Lock() ref := sr.ref(false, sr.descHandlers) sr.mu.Unlock() return ref } +func (sr *immutableRef) Clone() ImmutableRef { + return sr.clone() +} + func (sr *immutableRef) ociDesc(ctx context.Context, dhs DescHandlers) (ocispecs.Descriptor, error) { dgst := sr.getBlob() if dgst == "" { @@ -552,19 +677,6 @@ func getBlobDesc(ctx context.Context, cs content.Store, dgst digest.Digest) (oci 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, rerr error) { if sr.equalMutable != nil && !readonly { if err := sr.Finalize(ctx); err != nil { @@ -604,7 +716,7 @@ func (sr *immutableRef) Mount(ctx context.Context, readonly bool, s session.Grou } func (sr *immutableRef) Extract(ctx context.Context, s session.Group) (rerr error) { - if !sr.getBlobOnly() { + if (sr.kind() == Layer || sr.kind() == BaseLayer) && !sr.getBlobOnly() { return nil } @@ -625,7 +737,7 @@ func (sr *immutableRef) Extract(ctx context.Context, s session.Group) (rerr erro func (sr *immutableRef) withRemoteSnapshotLabelsStargzMode(ctx context.Context, s session.Group, f func()) error { dhs := sr.descHandlers - for _, r := range sr.parentRefChain() { + for _, r := range sr.layerChain() { r := r info, err := r.cm.Snapshotter.Stat(ctx, r.getSnapshotID()) if err != nil && !errdefs.IsNotFound(err) { @@ -667,7 +779,7 @@ func (sr *immutableRef) withRemoteSnapshotLabelsStargzMode(ctx context.Context, func (sr *immutableRef) prepareRemoteSnapshotsStargzMode(ctx context.Context, s session.Group) error { _, err := sr.sizeG.Do(ctx, sr.ID()+"-prepare-remote-snapshot", func(ctx context.Context) (_ interface{}, rerr error) { dhs := sr.descHandlers - for _, r := range sr.parentRefChain() { + for _, r := range sr.layerChain() { r := r snapshotID := r.getSnapshotID() if _, err := r.cm.Snapshotter.Stat(ctx, snapshotID); err == nil { @@ -699,8 +811,8 @@ func (sr *immutableRef) prepareRemoteSnapshotsStargzMode(ctx context.Context, s } ) parentID := "" - if r.parent != nil { - parentID = r.parent.getSnapshotID() + if r.layerParent != nil { + parentID = r.layerParent.getSnapshotID() } if err := r.cm.Snapshotter.Prepare(ctx, key, parentID, opts...); err != nil { if errdefs.IsAlreadyExists(err) { @@ -757,11 +869,40 @@ func (sr *immutableRef) unlazy(ctx context.Context, dhs DescHandlers, s session. return nil, nil } - return nil, sr.unlazyLayer(ctx, dhs, s) + switch sr.kind() { + case Merge: + return nil, sr.unlazyMerge(ctx, dhs, s) + case Layer, BaseLayer: + return nil, sr.unlazyLayer(ctx, dhs, s) + } + return nil, nil }) return err } +// should be called within sizeG.Do call for this ref's ID +func (sr *immutableRef) unlazyMerge(ctx context.Context, dhs DescHandlers, s session.Group) error { + eg, egctx := errgroup.WithContext(ctx) + for _, parent := range sr.mergeParents { + parent := parent + eg.Go(func() error { + return parent.unlazy(egctx, dhs, s) + }) + } + if err := eg.Wait(); err != nil { + return err + } + var diffs []snapshot.Diff + sr.layerWalk(func(sr *immutableRef) { + diff := snapshot.Diff{Upper: sr.getSnapshotID()} + if sr.layerParent != nil { + diff.Lower = sr.layerParent.getSnapshotID() + } + diffs = append(diffs, diff) + }) + return sr.cm.Snapshotter.Merge(ctx, sr.getSnapshotID(), diffs) +} + // should be called within sizeG.Do call for this ref's ID func (sr *immutableRef) unlazyLayer(ctx context.Context, dhs DescHandlers, s session.Group) (rerr error) { if !sr.getBlobOnly() { @@ -788,12 +929,12 @@ func (sr *immutableRef) unlazyLayer(ctx context.Context, dhs DescHandlers, s ses eg, egctx := errgroup.WithContext(ctx) parentID := "" - if sr.parent != nil { + if sr.layerParent != nil { eg.Go(func() error { - if err := sr.parent.unlazy(egctx, dhs, s); err != nil { + if err := sr.layerParent.unlazy(egctx, dhs, s); err != nil { return err } - parentID = sr.parent.getSnapshotID() + parentID = sr.layerParent.getSnapshotID() return nil }) } @@ -925,7 +1066,7 @@ func (cr *cacheRecord) finalize(ctx context.Context) error { return nil } - _, err := cr.cm.ManagerOpt.LeaseManager.Create(ctx, func(l *leases.Lease) error { + _, err := cr.cm.LeaseManager.Create(ctx, func(l *leases.Lease) error { l.ID = cr.ID() l.Labels = map[string]string{ "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), @@ -970,7 +1111,7 @@ func (sr *mutableRef) shouldUpdateLastUsed() bool { return sr.triggerLastUsed } -func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) { +func (sr *mutableRef) commit(ctx context.Context) (_ *immutableRef, rerr error) { if !sr.mutable || len(sr.refs) == 0 { return nil, errors.Wrapf(errInvalid, "invalid mutable ref %p", sr) } @@ -980,7 +1121,7 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) { rec := &cacheRecord{ mu: sr.mu, cm: sr.cm, - parent: sr.parentRef(false, sr.descHandlers), + parentRefs: sr.parentRefs.clone(), equalMutable: sr, refs: make(map[ref]struct{}), cacheMetadata: md, @@ -992,11 +1133,7 @@ func (sr *mutableRef) commit(ctx context.Context) (*immutableRef, error) { } } - parentID := "" - if rec.parent != nil { - parentID = rec.parent.ID() - } - if err := initializeMetadata(rec.cacheMetadata, parentID); err != nil { + if err := initializeMetadata(rec.cacheMetadata, rec.parentRefs); err != nil { return nil, err } @@ -1028,8 +1165,8 @@ func (sr *mutableRef) Mount(ctx context.Context, readonly bool, s session.Group) } var mnt snapshot.Mountable - if sr.cm.Snapshotter.Name() == "stargz" && sr.parent != nil { - if err := sr.parent.withRemoteSnapshotLabelsStargzMode(ctx, s, func() { + if sr.cm.Snapshotter.Name() == "stargz" && sr.layerParent != nil { + if err := sr.layerParent.withRemoteSnapshotLabelsStargzMode(ctx, s, func() { mnt, rerr = sr.mount(ctx, s) }); err != nil { return nil, err diff --git a/cache/remote.go b/cache/remote.go index d1cb86f24561..28a061620739 100644 --- a/cache/remote.go +++ b/cache/remote.go @@ -145,7 +145,7 @@ func (sr *immutableRef) getRemote(ctx context.Context, createIfNeeded bool, comp return nil, err } - chain := sr.parentRefChain() + chain := sr.layerChain() mproviderBase := contentutil.NewMultiProvider(nil) mprovider := &lazyMultiProvider{mprovider: mproviderBase} remote := &solver.Remote{ @@ -156,7 +156,6 @@ func (sr *immutableRef) getRemote(ctx context.Context, createIfNeeded bool, comp 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. diff --git a/client/client_test.go b/client/client_test.go index 7c32e20f67a7..a2d2dea72c1f 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -137,6 +137,8 @@ func TestIntegration(t *testing.T) { testLocalSourceDiffer, testBuildExportZstd, testPullZstdImage, + testMergeOp, + testMergeOpCache, }, mirrors) integration.Run(t, []integration.Test{ @@ -3888,6 +3890,389 @@ func testProxyEnv(t *testing.T, sb integration.Sandbox) { require.Equal(t, string(dt), "httpvalue-httpsvalue-noproxyvalue-noproxyvalue-allproxyvalue-allproxyvalue") } +func testMergeOp(t *testing.T, sb integration.Sandbox) { + requiresLinux(t) + + c, err := New(sb.Context(), sb.Address()) + require.NoError(t, err) + defer c.Close() + + requireContents := func(state llb.State, files ...fstest.Applier) { + def, err := state.Marshal(sb.Context()) + require.NoError(t, err) + + destDir, err := ioutil.TempDir("", "buildkit") + require.NoError(t, err) + defer os.RemoveAll(destDir) + + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{ + { + Type: ExporterLocal, + OutputDir: destDir, + }, + }, + }, nil) + require.NoError(t, err) + + require.NoError(t, fstest.CheckDirectoryEqualWithApplier(destDir, fstest.Apply(files...))) + } + + stateA := llb.Scratch(). + File(llb.Mkfile("/foo", 0777, []byte("A"))). + File(llb.Mkfile("/a", 0777, []byte("A"))). + File(llb.Mkdir("/bar", 0700)). + File(llb.Mkfile("/bar/A", 0777, []byte("A"))) + stateB := stateA. + File(llb.Rm("/foo")). + File(llb.Mkfile("/b", 0777, []byte("B"))). + File(llb.Mkfile("/bar/B", 0774, []byte("B"))) + stateC := llb.Scratch(). + File(llb.Mkfile("/foo", 0775, []byte("C"))). + File(llb.Mkfile("/c", 0777, []byte("C"))). + File(llb.Mkdir("/bar", 0777)). + File(llb.Mkfile("/bar/A", 0400, []byte("C"))) + + mergeA := llb.Merge([]llb.State{stateA, stateC}) + requireContents(mergeA, + fstest.CreateFile("foo", []byte("C"), 0775), + fstest.CreateFile("c", []byte("C"), 0777), + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/A", []byte("C"), 0400), + fstest.CreateFile("a", []byte("A"), 0777), + ) + + mergeB := llb.Merge([]llb.State{stateC, stateB}) + requireContents(mergeB, + fstest.CreateFile("a", []byte("A"), 0777), + fstest.CreateFile("b", []byte("B"), 0777), + fstest.CreateFile("c", []byte("C"), 0777), + fstest.CreateDir("bar", 0700), + fstest.CreateFile("bar/A", []byte("A"), 0777), + fstest.CreateFile("bar/B", []byte("B"), 0774), + ) + + stateD := llb.Scratch().File(llb.Mkdir("/qaz", 0755)) + mergeC := llb.Merge([]llb.State{mergeA, mergeB, stateD}) + requireContents(mergeC, + fstest.CreateFile("a", []byte("A"), 0777), + fstest.CreateFile("b", []byte("B"), 0777), + fstest.CreateFile("c", []byte("C"), 0777), + fstest.CreateDir("bar", 0700), + fstest.CreateFile("bar/A", []byte("A"), 0777), + fstest.CreateFile("bar/B", []byte("B"), 0774), + fstest.CreateDir("qaz", 0755), + ) + + runA := llb.Merge([]llb.State{llb.Image("busybox"), mergeC}). + Run(llb.Shlex("sh -c -e -x '" + strings.Join([]string{ + // turn /a file into a dir, mv b and c into it + "rm /a", + "mkdir /a", + "mv /b /c /a/", + // remove+recreate /bar to make it opaque on overlay snapshotters + "rm -rf /bar", + "mkdir -m 0777 /bar", + "echo -n D > /bar/D", + // turn /qaz dir into a file + "rm -rf /qaz", + "touch /qaz", + }, " && ") + "'")).Root(). + File(llb.Rm("/bin")). // get rid of stuff from busybox image that is tedious to assert on + File(llb.Rm("/dev")). + File(llb.Rm("/etc")). + File(llb.Rm("/home")). + File(llb.Rm("/root")). + File(llb.Rm("/tmp")). + File(llb.Rm("/usr")). + File(llb.Rm("/var")). + File(llb.Rm("/proc")). + File(llb.Rm("/sys")) + stateE := llb.Scratch(). + File(llb.Mkfile("/foo", 0777, []byte("E"))). + File(llb.Mkdir("/bar", 0777)). + File(llb.Mkfile("/bar/A", 0777, []byte("A"))). + File(llb.Mkfile("/bar/E", 0777, nil)) + mergeD := llb.Merge([]llb.State{stateE, runA}) + requireContents(mergeD, + fstest.CreateDir("a", 0755), + fstest.CreateFile("a/b", []byte("B"), 0777), + fstest.CreateFile("a/c", []byte("C"), 0777), + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/D", []byte("D"), 0644), + fstest.CreateFile("bar/E", nil, 0777), // exists because opaques dirs are converted to explicit whiteouts + fstest.CreateFile("qaz", nil, 0644), + // /foo from stateE is not here because it is deleted in stateB, which is part of a submerge of mergeD + ) +} + +func testMergeOpCache(t *testing.T, sb integration.Sandbox) { + skipDockerd(t, sb) + requiresLinux(t) + + cdAddress := sb.ContainerdAddress() + if cdAddress == "" { + t.Skip("test requires containerd worker") + } + + client, err := newContainerd(cdAddress) + require.NoError(t, err) + defer client.Close() + + ctx := namespaces.WithNamespace(sb.Context(), "buildkit") + + registry, err := sb.NewRegistry() + if errors.Is(err, integration.ErrorRequirements) { + t.Skip(err.Error()) + } + require.NoError(t, err) + + c, err := New(sb.Context(), sb.Address()) + require.NoError(t, err) + defer c.Close() + + // push the busybox image to the mutable registry + sourceImage := "busybox:latest" + def, err := llb.Image(sourceImage).Marshal(sb.Context()) + require.NoError(t, err) + + busyboxTargetNoTag := registry + "/buildkit/testlazyimage:" + busyboxTarget := busyboxTargetNoTag + "latest" + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{ + { + Type: ExporterImage, + Attrs: map[string]string{ + "name": busyboxTarget, + "push": "true", + }, + }, + }, + }, nil) + require.NoError(t, err) + + imageService := client.ImageService() + contentStore := client.ContentStore() + + busyboxImg, err := imageService.Get(ctx, busyboxTarget) + require.NoError(t, err) + + busyboxManifest, err := images.Manifest(ctx, contentStore, busyboxImg.Target, nil) + require.NoError(t, err) + + for _, layer := range busyboxManifest.Layers { + _, err = contentStore.Info(ctx, layer.Digest) + require.NoError(t, err) + } + + // clear all local state out + err = imageService.Delete(ctx, busyboxImg.Name, images.SynchronousDelete()) + require.NoError(t, err) + checkAllReleasable(t, c, sb, true) + + for _, layer := range busyboxManifest.Layers { + _, err = contentStore.Info(ctx, layer.Digest) + require.ErrorIs(t, err, ctderrdefs.ErrNotFound, "unexpected error %v", err) + } + + // make a new merge that includes the lazy busybox as a base and exports inline cache + input1 := llb.Scratch(). + File(llb.Mkdir("/dir", 0777)). + File(llb.Mkfile("/dir/1", 0777, nil)) + input1Copy := llb.Scratch().File(llb.Copy(input1, "/dir/1", "/foo/1", &llb.CopyInfo{CreateDestPath: true})) + + // put random contents in the file to ensure it's not re-run later + input2 := llb.Image("alpine:latest").Run(llb.Args([]string{"sh", "-c", strings.Join([]string{ + "mkdir /dir", + "cat /dev/urandom | head -c 100 | sha256sum > /dir/2", + }, " && ")})).Root() + input2Copy := llb.Scratch().File(llb.Copy(input2, "/dir/2", "/bar/2", &llb.CopyInfo{CreateDestPath: true})) + + merge := llb.Merge([]llb.State{llb.Image(busyboxTarget), input1Copy, input2Copy}) + + def, err = merge.Marshal(sb.Context()) + require.NoError(t, err) + + target := registry + "/buildkit/testmerge:latest" + cacheTarget := registry + "/buildkit/testmergecache:latest" + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{ + { + Type: ExporterImage, + Attrs: map[string]string{ + "name": target, + "push": "true", + }, + }, + }, + CacheExports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + }, nil) + require.NoError(t, err) + + // verify that the busybox image stayed lazy + for _, layer := range busyboxManifest.Layers { + _, err = contentStore.Info(ctx, layer.Digest) + require.ErrorIs(t, err, ctderrdefs.ErrNotFound, "unexpected error %v", err) + } + + // get the random value at /bar/2 + destDir, err := ioutil.TempDir("", "buildkit") + require.NoError(t, err) + defer os.RemoveAll(destDir) + + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{ + { + Type: ExporterLocal, + OutputDir: destDir, + }, + }, + }, nil) + require.NoError(t, err) + + bar2Contents, err := ioutil.ReadFile(filepath.Join(destDir, "bar", "2")) + require.NoError(t, err) + + // clear all local state out + img, err := imageService.Get(ctx, target) + require.NoError(t, err) + + manifest, err := images.Manifest(ctx, contentStore, img.Target, nil) + require.NoError(t, err) + + err = imageService.Delete(ctx, img.Name, images.SynchronousDelete()) + require.NoError(t, err) + checkAllReleasable(t, c, sb, true) + + for _, layer := range manifest.Layers { + _, err = contentStore.Info(ctx, layer.Digest) + require.ErrorIs(t, err, ctderrdefs.ErrNotFound, "unexpected error %v", err) + } + + // re-run the same build with cache imports and verify everything stays lazy + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{{ + Type: ExporterImage, + Attrs: map[string]string{ + "name": target, + "push": "true", + }, + }}, + CacheImports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + CacheExports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + }, nil) + require.NoError(t, err) + + // verify everything from before stayed lazy except the middle layer for input1Copy + img, err = imageService.Get(ctx, target) + require.NoError(t, err) + + manifest, err = images.Manifest(ctx, contentStore, img.Target, nil) + require.NoError(t, err) + + for i, layer := range manifest.Layers { + _, err = contentStore.Info(ctx, layer.Digest) + require.ErrorIs(t, err, ctderrdefs.ErrNotFound, "unexpected error %v for index %d", err, i) + } + + // re-run the build with a change only to input1 using the remote cache + input1 = llb.Scratch(). + File(llb.Mkdir("/dir", 0777)). + File(llb.Mkfile("/dir/1", 0444, nil)) + input1Copy = llb.Scratch().File(llb.Copy(input1, "/dir/1", "/foo/1", &llb.CopyInfo{CreateDestPath: true})) + + merge = llb.Merge([]llb.State{llb.Image(busyboxTarget), input1Copy, input2Copy}) + + def, err = merge.Marshal(sb.Context()) + require.NoError(t, err) + + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{{ + Type: ExporterImage, + Attrs: map[string]string{ + "name": target, + "push": "true", + }, + }}, + CacheImports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + CacheExports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + }, nil) + require.NoError(t, err) + + // verify everything from before stayed lazy except the middle layer for input1Copy + img, err = imageService.Get(ctx, target) + require.NoError(t, err) + + manifest, err = images.Manifest(ctx, contentStore, img.Target, nil) + require.NoError(t, err) + + for i, layer := range manifest.Layers { + switch i { + case 0, 2: + // bottom and top layer should stay lazy as they didn't change + _, err = contentStore.Info(ctx, layer.Digest) + require.ErrorIs(t, err, ctderrdefs.ErrNotFound, "unexpected error %v for index %d", err, i) + case 1: + // middle layer had to be rebuilt, should exist locally + _, err = contentStore.Info(ctx, layer.Digest) + require.NoError(t, err) + default: + require.Fail(t, "unexpected layer index %d", i) + } + } + + // check the random value at /bar/2 didn't change + destDir, err = ioutil.TempDir("", "buildkit") + require.NoError(t, err) + defer os.RemoveAll(destDir) + + _, err = c.Solve(sb.Context(), def, SolveOpt{ + Exports: []ExportEntry{ + { + Type: ExporterLocal, + OutputDir: destDir, + }, + }, + CacheImports: []CacheOptionsEntry{{ + Type: "registry", + Attrs: map[string]string{ + "ref": cacheTarget, + }, + }}, + }, nil) + require.NoError(t, err) + + newBar2Contents, err := ioutil.ReadFile(filepath.Join(destDir, "bar", "2")) + require.NoError(t, err) + + require.Equalf(t, bar2Contents, newBar2Contents, "bar/2 contents changed") +} + func requiresLinux(t *testing.T) { if runtime.GOOS != "linux" { t.Skipf("unsupported GOOS: %s", runtime.GOOS) @@ -3972,15 +4357,19 @@ loop0: retries = 0 for { count := 0 - err = client.ContentStore().Walk(ctx, func(content.Info) error { + var infos []content.Info + err = client.ContentStore().Walk(ctx, func(info content.Info) error { count++ + infos = append(infos, info) return nil }) require.NoError(t, err) if count == 0 { break } - require.True(t, 20 > retries) + if retries >= 20 { + require.FailNowf(t, "content still exists", "%+v", infos) + } retries++ time.Sleep(500 * time.Millisecond) } diff --git a/client/diskusage.go b/client/diskusage.go index 8034f977c173..2a2373f9d36a 100644 --- a/client/diskusage.go +++ b/client/diskusage.go @@ -18,7 +18,7 @@ type UsageInfo struct { CreatedAt time.Time LastUsedAt *time.Time UsageCount int - Parent string + Parents []string Description string RecordType UsageRecordType Shared bool @@ -44,7 +44,7 @@ func (c *Client) DiskUsage(ctx context.Context, opts ...DiskUsageOption) ([]*Usa Mutable: d.Mutable, InUse: d.InUse, Size: d.Size_, - Parent: d.Parent, + Parents: d.Parents, CreatedAt: d.CreatedAt, Description: d.Description, UsageCount: int(d.UsageCount), diff --git a/client/llb/merge.go b/client/llb/merge.go new file mode 100644 index 000000000000..8177d71d2a8e --- /dev/null +++ b/client/llb/merge.go @@ -0,0 +1,96 @@ +package llb + +import ( + "context" + + "github.com/moby/buildkit/solver/pb" + digest "github.com/opencontainers/go-digest" + "github.com/pkg/errors" +) + +type MergeOp struct { + MarshalCache + inputs []Output + output Output + constraints Constraints +} + +func NewMerge(inputs []State, c Constraints) *MergeOp { + op := &MergeOp{constraints: c} + for _, input := range inputs { + op.inputs = append(op.inputs, input.Output()) + } + op.output = &output{vertex: op} + return op +} + +func (m *MergeOp) Validate(ctx context.Context, constraints *Constraints) error { + if len(m.inputs) < 2 { + return errors.Errorf("merge must have at least 2 inputs") + } + return nil +} + +func (m *MergeOp) Marshal(ctx context.Context, constraints *Constraints) (digest.Digest, []byte, *pb.OpMetadata, []*SourceLocation, error) { + if m.Cached(constraints) { + return m.Load() + } + if err := m.Validate(ctx, constraints); err != nil { + return "", nil, nil, nil, err + } + + pop, md := MarshalConstraints(constraints, &m.constraints) + pop.Platform = nil // merge op is not platform specific + + op := &pb.MergeOp{} + for _, input := range m.inputs { + op.Inputs = append(op.Inputs, &pb.MergeInput{Input: pb.InputIndex(len(pop.Inputs))}) + pbInput, err := input.ToInput(ctx, constraints) + if err != nil { + return "", nil, nil, nil, err + } + pop.Inputs = append(pop.Inputs, pbInput) + } + pop.Op = &pb.Op_Merge{Merge: op} + + dt, err := pop.Marshal() + if err != nil { + return "", nil, nil, nil, err + } + + m.Store(dt, md, m.constraints.SourceLocations, constraints) + return m.Load() +} + +func (m *MergeOp) Output() Output { + return m.output +} + +func (m *MergeOp) Inputs() []Output { + return m.inputs +} + +func Merge(inputs []State, opts ...ConstraintsOpt) State { + // filter out any scratch inputs, which have no effect when merged + var filteredInputs []State + for _, input := range inputs { + if input.Output() != nil { + filteredInputs = append(filteredInputs, input) + } + } + if len(filteredInputs) == 0 { + // a merge of only scratch results in scratch + return Scratch() + } + if len(filteredInputs) == 1 { + // a merge of a single non-empty input results in that non-empty input + return filteredInputs[0] + } + + var c Constraints + for _, o := range opts { + o.SetConstraintsOption(&c) + } + addCap(&c, pb.CapMergeOp) + return NewState(NewMerge(filteredInputs, c).Output()) +} diff --git a/client/llb/merge_test.go b/client/llb/merge_test.go new file mode 100644 index 000000000000..524cec0553e6 --- /dev/null +++ b/client/llb/merge_test.go @@ -0,0 +1,33 @@ +package llb + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestScratchMerge(t *testing.T) { + t.Parallel() + + s := Merge(nil) + require.Nil(t, s.Output()) + + s = Merge([]State{}) + require.Nil(t, s.Output()) + + s = Merge([]State{Scratch()}) + require.Nil(t, s.Output()) + + s = Merge([]State{Scratch(), Scratch()}) + require.Nil(t, s.Output()) + + input := Image("foo") + s = Merge([]State{input}) + require.Equal(t, input.Output(), s.Output()) + + s = Merge([]State{Scratch(), input, Scratch()}) + require.Equal(t, input.Output(), s.Output()) + + s = Merge([]State{Scratch(), input, Image("bar")}) + require.NotEqual(t, input.Output(), s.Output()) +} diff --git a/client/prune.go b/client/prune.go index 27fe5dd8cdbc..024f61de332a 100644 --- a/client/prune.go +++ b/client/prune.go @@ -42,7 +42,7 @@ func (c *Client) Prune(ctx context.Context, ch chan UsageInfo, opts ...PruneOpti Mutable: d.Mutable, InUse: d.InUse, Size: d.Size_, - Parent: d.Parent, + Parents: d.Parents, CreatedAt: d.CreatedAt, Description: d.Description, UsageCount: int(d.UsageCount), diff --git a/cmd/buildctl/debug/dumpllb.go b/cmd/buildctl/debug/dumpllb.go index 6ed33ae0fff5..2daa1510ab79 100644 --- a/cmd/buildctl/debug/dumpllb.go +++ b/cmd/buildctl/debug/dumpllb.go @@ -111,6 +111,8 @@ func attr(dgst digest.Digest, op pb.Op) (string, string) { return strings.Join(op.Exec.Meta.Args, " "), "box" case *pb.Op_Build: return "build", "box3d" + case *pb.Op_Merge: + return "merge", "invtriangle" case *pb.Op_File: names := []string{} diff --git a/cmd/buildctl/diskusage.go b/cmd/buildctl/diskusage.go index 33144fa5dde5..48f64cfa4cf4 100644 --- a/cmd/buildctl/diskusage.go +++ b/cmd/buildctl/diskusage.go @@ -4,6 +4,7 @@ import ( "fmt" "io" "os" + "strings" "text/tabwriter" "github.com/moby/buildkit/client" @@ -61,8 +62,8 @@ func printKV(w io.Writer, k string, v interface{}) { func printVerbose(tw *tabwriter.Writer, du []*client.UsageInfo) { for _, di := range du { printKV(tw, "ID", di.ID) - if di.Parent != "" { - printKV(tw, "Parent", di.Parent) + if len(di.Parents) > 0 { + printKV(tw, "Parents", strings.Join(di.Parents, ";")) } printKV(tw, "Created at", di.CreatedAt) printKV(tw, "Mutable", di.Mutable) diff --git a/control/control.go b/control/control.go index 1eff83433abf..b3790ce4181a 100644 --- a/control/control.go +++ b/control/control.go @@ -109,7 +109,7 @@ func (c *Controller) DiskUsage(ctx context.Context, r *controlapi.DiskUsageReque Mutable: r.Mutable, InUse: r.InUse, Size_: r.Size, - Parent: r.Parent, + Parents: r.Parents, UsageCount: int64(r.UsageCount), Description: r.Description, CreatedAt: r.CreatedAt, @@ -177,7 +177,7 @@ func (c *Controller) Prune(req *controlapi.PruneRequest, stream controlapi.Contr Mutable: r.Mutable, InUse: r.InUse, Size_: r.Size, - Parent: r.Parent, + Parents: r.Parents, UsageCount: int64(r.UsageCount), Description: r.Description, CreatedAt: r.CreatedAt, diff --git a/go.mod b/go.mod index f486c6ca3319..a4d7085db171 100644 --- a/go.mod +++ b/go.mod @@ -31,6 +31,7 @@ require ( github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/hashicorp/go-immutable-radix v1.3.1 + github.com/hashicorp/go-multierror v1.1.1 github.com/hashicorp/golang-lru v0.5.3 github.com/ishidawataru/sctp v0.0.0-20210226210310-f2269e66cdee // indirect github.com/klauspost/compress v1.13.5 @@ -100,7 +101,6 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/hanwen/go-fuse/v2 v2.1.0 // indirect github.com/hashicorp/errwrap v1.0.0 // indirect - github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/moby/sys/mountinfo v0.5.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/hack/test b/hack/test index 91e159d6ca69..d5b253055f94 100755 --- a/hack/test +++ b/hack/test @@ -61,7 +61,7 @@ if ! docker container inspect "$cacheVolume" >/dev/null 2>/dev/null; then fi if [ "$TEST_INTEGRATION" == 1 ]; then - cid=$(docker create --rm -v /tmp $coverageVol --volumes-from=$cacheVolume -e TEST_DOCKERD -e SKIP_INTEGRATION_TESTS -e BUILDKIT_REGISTRY_MIRROR_DIR=/root/.cache/registry --privileged $iid go test $coverageFlags ${TESTFLAGS:--v} ${TESTPKGS:-./...}) + cid=$(docker create --rm -v /tmp $coverageVol --volumes-from=$cacheVolume -e TEST_DOCKERD -e SKIP_INTEGRATION_TESTS ${BUILDKIT_INTEGRATION_SNAPSHOTTER:+"-eBUILDKIT_INTEGRATION_SNAPSHOTTER"} -e BUILDKIT_REGISTRY_MIRROR_DIR=/root/.cache/registry --privileged $iid go test $coverageFlags ${TESTFLAGS:--v} ${TESTPKGS:-./...}) if [ "$TEST_DOCKERD" = "1" ]; then docker cp "$TEST_DOCKERD_BINARY" $cid:/usr/bin/ fi diff --git a/snapshot/diffapply_unix.go b/snapshot/diffapply_unix.go new file mode 100644 index 000000000000..b030e09d6a5e --- /dev/null +++ b/snapshot/diffapply_unix.go @@ -0,0 +1,414 @@ +//go:build !windows +// +build !windows + +package snapshot + +import ( + "context" + gofs "io/fs" + "os" + "path/filepath" + "syscall" + + "github.com/containerd/containerd/leases" + "github.com/containerd/containerd/mount" + "github.com/containerd/containerd/snapshots" + "github.com/containerd/continuity/fs" + "github.com/containerd/continuity/sysx" + "github.com/containerd/stargz-snapshotter/snapshot/overlayutils" + "github.com/moby/buildkit/identity" + "github.com/moby/buildkit/util/bklog" + "github.com/moby/buildkit/util/leaseutil" + "github.com/moby/buildkit/util/overlay" + "github.com/pkg/errors" + "golang.org/x/sys/unix" +) + +// diffApply calculates the diff between two directories and directly applies the changes to a separate mount (without using +// the content store as an intermediary). If useHardlink is set to true, it will hardlink non-directories instead of copying +// them when applying. This obviously requires that each of the mounts provided are for immutable, committed snapshots. +// externalHardlinks tracks any such hardlinks, which is needed for doing correct disk usage calculations elsewhere. +func diffApply(ctx context.Context, lowerMountable, upperMountable, applyMountable Mountable, useHardlink bool, externalHardlinks map[uint64]struct{}, createWhiteoutDeletes bool) error { + if applyMountable == nil { + return errors.New("invalid nil apply mounts") + } + + var lowerMounts []mount.Mount + if lowerMountable != nil { + mounts, unmountLower, err := lowerMountable.Mount() + if err != nil { + return err + } + lowerMounts = mounts + defer unmountLower() + } + + var upperMounts []mount.Mount + if upperMountable != nil { + mounts, unmountUpper, err := upperMountable.Mount() + if err != nil { + return err + } + upperMounts = mounts + defer unmountUpper() + } + + lowerMnter := LocalMounterWithMounts(lowerMounts) + lowerView, err := lowerMnter.Mount() + if err != nil { + return err + } + defer lowerMnter.Unmount() + + upperMnter := LocalMounterWithMounts(upperMounts) + upperView, err := upperMnter.Mount() + if err != nil { + return err + } + defer upperMnter.Unmount() + + applyMounts, unmountApply, err := applyMountable.Mount() + if err != nil { + return err + } + defer unmountApply() + + var applyRoot string + if useHardlink { + applyRoot, err = getRWDir(applyMounts) + if err != nil { + return err + } + } else { + applyMnter := LocalMounterWithMounts(applyMounts) + applyRoot, err = applyMnter.Mount() + if err != nil { + return err + } + defer applyMnter.Unmount() + } + type pathTime struct { + applyPath string + atime unix.Timespec + mtime unix.Timespec + } + + // times holds the paths+times we visited and need to set + var times []pathTime + + visited := make(map[string]struct{}) + inodes := make(map[uint64]string) + + diffCalculator := func(cf fs.ChangeFunc) error { + return fs.Changes(ctx, lowerView, upperView, cf) + } + upperRoot := upperView + + // If using hardlinks, set upperRoot to the underlying filesystem so hardlinks can avoid EXDEV + if useHardlink && len(upperMounts) == 1 { + switch upperMounts[0].Type { + case "bind", "rbind": + upperRoot = upperMounts[0].Source + case "overlay": + if upperdir, err := overlay.GetUpperdir(lowerMounts, upperMounts); err == nil { + upperRoot = upperdir + diffCalculator = func(cf fs.ChangeFunc) error { + return overlay.Changes(ctx, cf, upperRoot, upperView, lowerView) + } + } else { + useHardlink = false + } + } + } + + var changeFunc fs.ChangeFunc + changeFunc = func(kind fs.ChangeKind, changePath string, upperFi os.FileInfo, prevErr error) error { + if prevErr != nil { + return prevErr + } + + applyPath, err := safeJoin(applyRoot, changePath) + if err != nil { + return errors.Wrapf(err, "failed to get apply path for root %q, change %q", applyRoot, changePath) + } + upperPath, err := safeJoin(upperRoot, changePath) + if err != nil { + return errors.Wrapf(err, "failed to get upper path for root %q, change %q", upperPath, changePath) + } + + visited[upperPath] = struct{}{} + + if kind == fs.ChangeKindUnmodified { + return nil + } + + // When using overlay, a delete is represented with a whiteout device, which we actually want to + // hardlink or recreate here. If upperFi is non-nil, that means we have a whiteout device we can + // hardlink in. If upperFi is nil though, we need to first remove what's at the path and then, in + // the overlay case, create a whiteout device to represent the delete. This can happen when the + // overlay differ encounters an opaque dir, in which case it switches to the walking differ in order + // to convert from the opaque representation to the "explicit whiteout" format. + if kind == fs.ChangeKindDelete && upperFi == nil { + if err := os.RemoveAll(applyPath); err != nil { + return errors.Wrapf(err, "failed to remove path %s during apply", applyPath) + } + // Create the whiteout device only if enabled and if we are directly applying to an upperdir (as + // opposed to applying to an overlay mount itself). + if createWhiteoutDeletes && upperRoot != upperView { + if err := unix.Mknod(applyPath, unix.S_IFCHR, int(unix.Mkdev(0, 0))); err != nil { + return errors.Wrap(err, "failed to create whiteout delete") + } + } + return nil + } + + upperType := upperFi.Mode().Type() + if upperType == os.ModeIrregular { + return errors.Errorf("unhandled irregular mode file during merge at path %q", changePath) + } + + upperStat, ok := upperFi.Sys().(*syscall.Stat_t) + if !ok { + return errors.Errorf("unhandled stat type for %+v", upperFi) + } + + // Check to see if the parent dir needs to be created. This is needed when we are visiting a dirent + // that changes but never visit its parent dir because the parent did not change in the diff + changeParent := filepath.Dir(changePath) + if changeParent != "/" { + upperParent := filepath.Dir(upperPath) + if _, ok := visited[upperParent]; !ok { + parentInfo, err := os.Lstat(upperParent) + if err != nil { + return errors.Wrap(err, "failed to stat parent path during apply") + } + if err := changeFunc(fs.ChangeKindAdd, changeParent, parentInfo, nil); err != nil { + return err + } + } + } + + applyFi, err := os.Lstat(applyPath) + if err != nil && !os.IsNotExist(err) { + return errors.Wrapf(err, "failed to stat path during apply") + } + + // if there is an existing file/dir at the applyPath, delete it unless both it and upper are dirs (in which case they get merged) + if applyFi != nil && !(applyFi.IsDir() && upperFi.IsDir()) { + if err := os.RemoveAll(applyPath); err != nil { + return errors.Wrapf(err, "failed to remove path %s during apply", applyPath) + } + applyFi = nil + } + + // hardlink fast-path + if useHardlink { + switch upperType { + case os.ModeDir, os.ModeNamedPipe, os.ModeSocket: + // Directories can't be hard-linked, so they just have to be recreated. + // Named pipes and sockets can be hard-linked but is best to avoid as it could enable IPC by sharing merge inputs. + break + default: + // TODO:(sipsma) consider handling EMLINK by falling back to copy + if err := os.Link(upperPath, applyPath); err != nil { + return errors.Wrapf(err, "failed to hardlink %q to %q during apply", upperPath, applyPath) + } + // mark this inode as one coming from a separate snapshot, needed for disk usage calculations elsewhere + externalHardlinks[upperStat.Ino] = struct{}{} + return nil + } + } + + switch upperType { + case 0: // regular file + if upperStat.Nlink > 1 { + if linkedPath, ok := inodes[upperStat.Ino]; ok { + if err := os.Link(linkedPath, applyPath); err != nil { + return errors.Wrap(err, "failed to create hardlink during apply") + } + return nil // no other metadata updates needed when hardlinking + } + inodes[upperStat.Ino] = applyPath + } + if err := fs.CopyFile(applyPath, upperPath); err != nil { + return errors.Wrapf(err, "failed to copy from %s to %s during apply", upperPath, applyPath) + } + case os.ModeDir: + if applyFi == nil { + // applyPath doesn't exist, make it a dir + if err := os.Mkdir(applyPath, upperFi.Mode()); err != nil { + return errors.Wrap(err, "failed to create applied dir") + } + } + case os.ModeSymlink: + if target, err := os.Readlink(upperPath); err != nil { + return errors.Wrap(err, "failed to read symlink during apply") + } else if err := os.Symlink(target, applyPath); err != nil { + return errors.Wrap(err, "failed to create symlink during apply") + } + case os.ModeCharDevice, os.ModeDevice, os.ModeNamedPipe, os.ModeSocket: + if err := unix.Mknod(applyPath, uint32(upperFi.Mode()), int(upperStat.Rdev)); err != nil { + return errors.Wrap(err, "failed to create device during apply") + } + default: + // should never be here, all types should be handled + return errors.Errorf("unhandled file type %q during merge at path %q", upperType.String(), changePath) + } + + xattrs, err := sysx.LListxattr(upperPath) + if err != nil { + return errors.Wrapf(err, "failed to list xattrs of upper path %s", upperPath) + } + for _, xattr := range xattrs { + if isOpaqueXattr(xattr) { + // Don't recreate opaque xattrs during merge. These should only be set when using overlay snapshotters, + // in which case we are converting from the "opaque whiteout" format to the "explicit whiteout" format during + // the merge (as taken care of by the overlay differ). + continue + } + xattrVal, err := sysx.LGetxattr(upperPath, xattr) + if err != nil { + return errors.Wrapf(err, "failed to get xattr %s of upper path %s", xattr, upperPath) + } + if err := sysx.LSetxattr(applyPath, xattr, xattrVal, 0); err != nil { + // This can often fail, so just log it: https://github.com/moby/buildkit/issues/1189 + bklog.G(ctx).Debugf("failed to set xattr %s of path %s during apply", xattr, applyPath) + } + } + + if err := os.Lchown(applyPath, int(upperStat.Uid), int(upperStat.Gid)); err != nil { + return errors.Wrap(err, "failed to chown applied dir") + } + + if upperType != os.ModeSymlink { + if err := os.Chmod(applyPath, upperFi.Mode()); err != nil { + return errors.Wrap(err, "failed to chmod applied dir") + } + } + + // save the times we should set on this path, to be applied at the end. + times = append(times, pathTime{ + applyPath: applyPath, + atime: unix.Timespec{ + Sec: upperStat.Atim.Sec, + Nsec: upperStat.Atim.Nsec, + }, + mtime: unix.Timespec{ + Sec: upperStat.Mtim.Sec, + Nsec: upperStat.Mtim.Nsec, + }, + }) + return nil + } + + if err := diffCalculator(changeFunc); err != nil { + return err + } + + // Set times now that everything has been modified. + for i := range times { + ts := times[len(times)-1-i] + if err := unix.UtimesNanoAt(unix.AT_FDCWD, ts.applyPath, []unix.Timespec{ts.atime, ts.mtime}, unix.AT_SYMLINK_NOFOLLOW); err != nil { + return errors.Wrapf(err, "failed to update times of path %q", ts.applyPath) + } + } + + return nil +} + +func safeJoin(root, path string) (string, error) { + dir, base := filepath.Split(path) + parent, err := fs.RootPath(root, dir) + if err != nil { + return "", err + } + return filepath.Join(parent, base), nil +} + +// diskUsage calculates the disk space used by the provided mounts, similar to the normal containerd snapshotter disk usage +// calculations but with the extra ability to take into account hardlinks that were created between snapshots, ensuring that +// they don't get double counted. +func diskUsage(ctx context.Context, mountable Mountable, externalHardlinks map[uint64]struct{}) (snapshots.Usage, error) { + mounts, unmount, err := mountable.Mount() + if err != nil { + return snapshots.Usage{}, err + } + defer unmount() + + inodes := make(map[uint64]struct{}) + var usage snapshots.Usage + root, err := getRWDir(mounts) + if err != nil { + return snapshots.Usage{}, err + } + if err := filepath.WalkDir(root, func(path string, dirent gofs.DirEntry, err error) error { + if err != nil { + return err + } + info, err := dirent.Info() + if err != nil { + return err + } + stat := info.Sys().(*syscall.Stat_t) + if _, ok := inodes[stat.Ino]; ok { + return nil + } + inodes[stat.Ino] = struct{}{} + if _, ok := externalHardlinks[stat.Ino]; !ok { + usage.Inodes++ + usage.Size += stat.Blocks * 512 // 512 is always block size, see "man 2 stat" + } + return nil + }); err != nil { + return snapshots.Usage{}, err + } + return usage, nil +} + +func isOpaqueXattr(s string) bool { + for _, k := range []string{"trusted.overlay.opaque", "user.overlay.opaque"} { + if s == k { + return true + } + } + return false +} + +// needsUserXAttr checks whether overlay mounts should be provided the userxattr option. We can't use +// NeedsUserXAttr from the overlayutils package directly because we don't always have direct knowledge +// of the root of the snapshotter state (such as when using a remote snapshotter). Instead, we create +// a temporary new snapshot and test using its root, which works because single layer snapshots will +// use bind-mounts even when created by an overlay based snapshotter. +func needsUserXAttr(ctx context.Context, sn Snapshotter, lm leases.Manager) (bool, error) { + key := identity.NewID() + + ctx, done, err := leaseutil.WithLease(ctx, lm, leaseutil.MakeTemporary) + if err != nil { + return false, errors.Wrap(err, "failed to create lease for checking user xattr") + } + defer done(context.TODO()) + + err = sn.Prepare(ctx, key, "") + if err != nil { + return false, err + } + mntable, err := sn.Mounts(ctx, key) + if err != nil { + return false, err + } + mnts, unmount, err := mntable.Mount() + if err != nil { + return false, err + } + defer unmount() + + var userxattr bool + if err := mount.WithTempMount(ctx, mnts, func(root string) error { + var err error + userxattr, err = overlayutils.NeedsUserXAttr(root) + return err + }); err != nil { + return false, err + } + return userxattr, nil +} diff --git a/snapshot/diffapply_windows.go b/snapshot/diffapply_windows.go new file mode 100644 index 000000000000..64eef8e06660 --- /dev/null +++ b/snapshot/diffapply_windows.go @@ -0,0 +1,24 @@ +//go:build windows +// +build windows + +package snapshot + +import ( + "context" + + "github.com/containerd/containerd/leases" + "github.com/containerd/containerd/snapshots" + "github.com/pkg/errors" +) + +func diffApply(ctx context.Context, lowerMountable, upperMountable, applyMountable Mountable, useHardlink bool, externalHardlinks map[uint64]struct{}, createWhiteoutDeletes bool) error { + return errors.New("diff apply not supported on windows") +} + +func diskUsage(ctx context.Context, mountable Mountable, externalHardlinks map[uint64]struct{}) (snapshots.Usage, error) { + return snapshots.Usage{}, errors.New("disk usage not supported on windows") +} + +func needsUserXAttr(ctx context.Context, sn Snapshotter, lm leases.Manager) (bool, error) { + return false, errors.New("needs userxattr not supported on windows") +} diff --git a/snapshot/localmounter.go b/snapshot/localmounter.go index 9ddb7c1af642..6f210ea09456 100644 --- a/snapshot/localmounter.go +++ b/snapshot/localmounter.go @@ -1,9 +1,11 @@ package snapshot import ( + "strings" "sync" "github.com/containerd/containerd/mount" + "github.com/pkg/errors" ) type Mounter interface { @@ -30,3 +32,33 @@ type localMounter struct { target string release func() error } + +// RWDirMount extracts out just a writable directory from the provided mounts and returns it. +// It's intended to supply the directory to which changes being made to the mount can be +// written directly. A writable directory includes an upperdir if provided an overlay or a rw +// bind mount source. If the mount doesn't have a writable directory, an error is returned. +func getRWDir(mounts []mount.Mount) (string, error) { + if len(mounts) != 1 { + return "", errors.New("cannot extract writable directory from zero or multiple mounts") + } + mnt := mounts[0] + switch mnt.Type { + case "overlay": + for _, opt := range mnt.Options { + if strings.HasPrefix(opt, "upperdir=") { + upperdir := strings.SplitN(opt, "=", 2)[1] + return upperdir, nil + } + } + return "", errors.New("cannot extract writable directory from overlay mount without upperdir") + case "bind", "rbind": + for _, opt := range mnt.Options { + if opt == "ro" { + return "", errors.New("cannot extract writable directory from read-only bind mount") + } + } + return mnt.Source, nil + default: + return "", errors.Errorf("cannot extract writable directory from unhandled mount type %q", mnt.Type) + } +} diff --git a/snapshot/localmounter_unix.go b/snapshot/localmounter_unix.go index 37e18eb07c45..ef73e263fc91 100644 --- a/snapshot/localmounter_unix.go +++ b/snapshot/localmounter_unix.go @@ -16,7 +16,7 @@ func (lm *localMounter) Mount() (string, error) { lm.mu.Lock() defer lm.mu.Unlock() - if lm.mounts == nil { + if lm.mounts == nil && lm.mountable != nil { mounts, release, err := lm.mountable.Mount() if err != nil { return "", err diff --git a/snapshot/merge.go b/snapshot/merge.go new file mode 100644 index 000000000000..1d48c263d6ca --- /dev/null +++ b/snapshot/merge.go @@ -0,0 +1,219 @@ +package snapshot + +import ( + "context" + "strconv" + + "github.com/containerd/containerd/leases" + "github.com/containerd/containerd/pkg/userns" + "github.com/containerd/containerd/snapshots" + "github.com/moby/buildkit/identity" + "github.com/moby/buildkit/util/bklog" + "github.com/moby/buildkit/util/leaseutil" + "github.com/pkg/errors" +) + +// hardlinkMergeSnapshotters are the names of snapshotters that support merges implemented by +// creating "hardlink farms" where non-directory objects are hard-linked into the merged tree +// from their parent snapshots. +var hardlinkMergeSnapshotters = map[string]struct{}{ + "native": {}, + "overlayfs": {}, + "stargz": {}, +} + +// overlayBasedSnapshotters are the names of snapshotter that use overlay mounts, which +// enables optimizations such as skipping the base layer when doing a hardlink merge. +var overlayBasedSnapshotters = map[string]struct{}{ + "overlayfs": {}, + "stargz": {}, +} + +type Diff struct { + Lower string + Upper string +} + +type MergeSnapshotter interface { + Snapshotter + // Merge creates a snapshot whose contents are the provided diffs applied onto one + // another in the provided order, starting from scratch. The diffs are calculated + // the same way that diffs are calculated during exports, which ensures that the + // result of merging these diffs looks the same as exporting the diffs as layer + // blobs and unpacking them as an image. + // + // Each key in the provided diffs is expected to be a committed snapshot. The + // snapshot created by Merge is also committed. + // + // The size of a merged snapshot (as returned by the Usage method) depends on the merge + // implementation. Implementations using hardlinks to create merged views will take up + // less space than those that use copies, for example. + Merge(ctx context.Context, key string, diffs []Diff, opts ...snapshots.Opt) error +} + +type mergeSnapshotter struct { + Snapshotter + lm leases.Manager + + // Whether we should try to implement merges by hardlinking between underlying directories + useHardlinks bool + + // Whether the snapshotter is overlay-based, which enables some required behavior like + // creation of whiteout devices to represent deletes in addition to some optimizations + // like using the first merge input as the parent snapshot. + overlayBased bool +} + +func NewMergeSnapshotter(ctx context.Context, sn Snapshotter, lm leases.Manager) MergeSnapshotter { + name := sn.Name() + _, useHardlinks := hardlinkMergeSnapshotters[name] + _, overlayBased := overlayBasedSnapshotters[name] + + if overlayBased && userns.RunningInUserNS() { + // When using an overlay-based snapshotter, if we are running rootless on a pre-5.11 + // kernel, we will not have userxattr. This results in opaque xattrs not being visible + // to us and thus breaking the overlay-optimized differ. This also means that there are + // cases where in order to apply a deletion, we'd need to create a whiteout device but + // may not have access to one to hardlink, so we just fall back to not using hardlinks + // at all in this case. + userxattr, err := needsUserXAttr(ctx, sn, lm) + if err != nil { + bklog.G(ctx).Debugf("failed to check user xattr: %v", err) + useHardlinks = false + } else { + useHardlinks = userxattr + } + } + + return &mergeSnapshotter{ + Snapshotter: sn, + lm: lm, + useHardlinks: useHardlinks, + overlayBased: overlayBased, + } +} + +func (sn *mergeSnapshotter) Merge(ctx context.Context, key string, diffs []Diff, opts ...snapshots.Opt) error { + var baseKey string + if sn.overlayBased { + // Overlay-based snapshotters can skip the base snapshot of the merge (if one exists) and just use it as the + // parent of the merge snapshot. Other snapshotters will start empty (with baseKey set to ""). + // Find the baseKey by following the chain of diffs for as long as it follows the pattern of the current lower + // being the parent of the current upper and equal to the previous upper. + var baseIndex int + for i, diff := range diffs { + info, err := sn.Stat(ctx, diff.Upper) + if err != nil { + return err + } + if info.Parent != diff.Lower { + break + } + if diff.Lower != baseKey { + break + } + baseKey = diff.Upper + baseIndex = i + 1 + } + diffs = diffs[baseIndex:] + } + + tempLeaseCtx, done, err := leaseutil.WithLease(ctx, sn.lm, leaseutil.MakeTemporary) + if err != nil { + return errors.Wrap(err, "failed to create temporary lease for view mounts during merge") + } + defer done(context.TODO()) + + // Make the snapshot that will be merged into + prepareKey := identity.NewID() + if err := sn.Prepare(tempLeaseCtx, prepareKey, baseKey); err != nil { + return errors.Wrapf(err, "failed to prepare %q", key) + } + applyMounts, err := sn.Mounts(ctx, prepareKey) + if err != nil { + return errors.Wrapf(err, "failed to get mounts of %q", key) + } + + // externalHardlinks keeps track of which inodes have been hard-linked between snapshots (which is + // enabled when sn.useHardlinks is set to true) + externalHardlinks := make(map[uint64]struct{}) + + for _, diff := range diffs { + var lowerMounts Mountable + if diff.Lower != "" { + viewID := identity.NewID() + var err error + lowerMounts, err = sn.View(tempLeaseCtx, viewID, diff.Lower) + if err != nil { + return errors.Wrapf(err, "failed to get mounts of lower %q", diff.Lower) + } + } + + viewID := identity.NewID() + upperMounts, err := sn.View(tempLeaseCtx, viewID, diff.Upper) + if err != nil { + return errors.Wrapf(err, "failed to get mounts of upper %q", diff.Upper) + } + + err = diffApply(tempLeaseCtx, lowerMounts, upperMounts, applyMounts, sn.useHardlinks, externalHardlinks, sn.overlayBased) + if err != nil { + return err + } + } + + // save the correctly calculated usage as a label on the committed key + usage, err := diskUsage(ctx, applyMounts, externalHardlinks) + if err != nil { + return errors.Wrap(err, "failed to get disk usage of diff apply merge") + } + if err := sn.Commit(ctx, key, prepareKey, withMergeUsage(usage)); err != nil { + return errors.Wrapf(err, "failed to commit %q", key) + } + return nil +} + +func (sn *mergeSnapshotter) Usage(ctx context.Context, key string) (snapshots.Usage, error) { + // If key was created by Merge, we may need to use the annotated mergeUsage key as + // the snapshotter's usage method is wrong when hardlinks are used to create the merge. + if info, err := sn.Stat(ctx, key); err != nil { + return snapshots.Usage{}, err + } else if usage, ok, err := mergeUsageOf(info); err != nil { + return snapshots.Usage{}, err + } else if ok { + return usage, nil + } + return sn.Snapshotter.Usage(ctx, key) +} + +// mergeUsage{Size,Inodes}Label hold the correct usage calculations for diffApplyMerges, for which the builtin usage +// is wrong because it can't account for hardlinks made across immutable snapshots +const mergeUsageSizeLabel = "buildkit.mergeUsageSize" +const mergeUsageInodesLabel = "buildkit.mergeUsageInodes" + +func withMergeUsage(usage snapshots.Usage) snapshots.Opt { + return snapshots.WithLabels(map[string]string{ + mergeUsageSizeLabel: strconv.Itoa(int(usage.Size)), + mergeUsageInodesLabel: strconv.Itoa(int(usage.Inodes)), + }) +} + +func mergeUsageOf(info snapshots.Info) (usage snapshots.Usage, ok bool, rerr error) { + if info.Labels == nil { + return snapshots.Usage{}, false, nil + } + if str, ok := info.Labels[mergeUsageSizeLabel]; ok { + i, err := strconv.Atoi(str) + if err != nil { + return snapshots.Usage{}, false, err + } + usage.Size = int64(i) + } + if str, ok := info.Labels[mergeUsageInodesLabel]; ok { + i, err := strconv.Atoi(str) + if err != nil { + return snapshots.Usage{}, false, err + } + usage.Inodes = int64(i) + } + return usage, true, nil +} diff --git a/snapshot/snapshotter.go b/snapshot/snapshotter.go index 99745098af37..17791c3b61ae 100644 --- a/snapshot/snapshotter.go +++ b/snapshot/snapshotter.go @@ -7,6 +7,7 @@ import ( "github.com/containerd/containerd/mount" "github.com/containerd/containerd/snapshots" "github.com/docker/docker/pkg/idtools" + "github.com/pkg/errors" ) type Mountable interface { @@ -53,6 +54,7 @@ func (s *fromContainerd) Mounts(ctx context.Context, key string) (Mountable, err } return &staticMountable{mounts: mounts, idmap: s.idmap, id: key}, nil } + func (s *fromContainerd) Prepare(ctx context.Context, key, parent string, opts ...snapshots.Opt) error { _, err := s.Snapshotter.Prepare(ctx, key, parent, opts...) return err @@ -68,6 +70,15 @@ func (s *fromContainerd) IdentityMapping() *idtools.IdentityMapping { return s.idmap } +func (s *fromContainerd) Commit(ctx context.Context, name, key string, opts ...snapshots.Opt) error { + info, err := s.Stat(ctx, key) + if err != nil { + return errors.Wrap(err, "failed to stat active key during commit") + } + opts = append(opts, snapshots.WithLabels(snapshots.FilterInheritedLabels(info.Labels))) + return s.Snapshotter.Commit(ctx, name, key, opts...) +} + // NewContainerdSnapshotter converts snapshotter to containerd snapshotter func NewContainerdSnapshotter(s Snapshotter) (snapshots.Snapshotter, func() error) { cs := &containerdSnapshotter{Snapshotter: s} diff --git a/snapshot/snapshotter_test.go b/snapshot/snapshotter_test.go new file mode 100644 index 000000000000..b65fdf811297 --- /dev/null +++ b/snapshot/snapshotter_test.go @@ -0,0 +1,629 @@ +//go:build linux +// +build linux + +package snapshot + +import ( + "context" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "strings" + "syscall" + "testing" + "time" + + "github.com/containerd/containerd/content/local" + "github.com/containerd/containerd/leases" + ctdmetadata "github.com/containerd/containerd/metadata" + "github.com/containerd/containerd/mount" + "github.com/containerd/containerd/namespaces" + "github.com/containerd/containerd/snapshots" + "github.com/containerd/containerd/snapshots/native" + "github.com/containerd/containerd/snapshots/overlay" + "github.com/containerd/continuity/fs/fstest" + "github.com/hashicorp/go-multierror" + "github.com/moby/buildkit/identity" + "github.com/moby/buildkit/util/leaseutil" + "github.com/stretchr/testify/require" + bolt "go.etcd.io/bbolt" +) + +func newSnapshotter(ctx context.Context, snapshotterName string) (_ context.Context, _ *mergeSnapshotter, _ func() error, rerr error) { + ns := "buildkit-test" + ctx = namespaces.WithNamespace(ctx, ns) + + defers := make([]func() error, 0) + cleanup := func() error { + var err error + for i := range defers { + err = multierror.Append(err, defers[len(defers)-1-i]()).ErrorOrNil() + } + return err + } + defer func() { + if rerr != nil && cleanup != nil { + cleanup() + } + }() + + tmpdir, err := ioutil.TempDir("", "buildkit-test") + if err != nil { + return nil, nil, nil, err + } + defers = append(defers, func() error { + return os.RemoveAll(tmpdir) + }) + + var ctdSnapshotter snapshots.Snapshotter + var noHardlink bool + switch snapshotterName { + case "native-nohardlink": + noHardlink = true + fallthrough + case "native": + ctdSnapshotter, err = native.NewSnapshotter(filepath.Join(tmpdir, "snapshots")) + if err != nil { + return nil, nil, nil, err + } + case "overlayfs": + ctdSnapshotter, err = overlay.NewSnapshotter(filepath.Join(tmpdir, "snapshots")) + if err != nil { + return nil, nil, nil, err + } + default: + return nil, nil, nil, fmt.Errorf("unhandled snapshotter: %s", snapshotterName) + } + + store, err := local.NewStore(tmpdir) + if err != nil { + return nil, nil, nil, err + } + + db, err := bolt.Open(filepath.Join(tmpdir, "containerdmeta.db"), 0644, nil) + if err != nil { + return nil, nil, nil, err + } + defers = append(defers, func() error { + return db.Close() + }) + + mdb := ctdmetadata.NewDB(db, store, map[string]snapshots.Snapshotter{ + snapshotterName: ctdSnapshotter, + }) + if err := mdb.Init(context.TODO()); err != nil { + return nil, nil, nil, err + } + + lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), ns) + snapshotter := NewMergeSnapshotter(ctx, FromContainerdSnapshotter(snapshotterName, mdb.Snapshotter(snapshotterName), nil), lm).(*mergeSnapshotter) + if noHardlink { + snapshotter.useHardlinks = false + } + + leaseID := identity.NewID() + _, err = lm.Create(ctx, func(l *leases.Lease) error { + l.ID = leaseID + l.Labels = map[string]string{ + "containerd.io/gc.flat": time.Now().UTC().Format(time.RFC3339Nano), + } + return nil + }, leaseutil.MakeTemporary) + if err != nil { + return nil, nil, nil, err + } + ctx = leases.WithLease(ctx, leaseID) + + return ctx, snapshotter, cleanup, nil +} + +func TestMerge(t *testing.T) { + for _, snName := range []string{"overlayfs", "native", "native-nohardlink"} { + snName := snName + t.Run(snName, func(t *testing.T) { + t.Parallel() + if snName == "overlayfs" { + requireRoot(t) + } + + ctx, sn, cleanup, err := newSnapshotter(context.Background(), snName) + require.NoError(t, err) + defer cleanup() + + ts := time.Unix(0, 0) + snapA := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateFile("foo", []byte("A"), 0777), + fstest.Lchtimes("foo", ts, ts.Add(2*time.Second)), + + fstest.CreateFile("a", []byte("A"), 0777), + fstest.Lchtimes("a", ts, ts.Add(4*time.Second)), + + fstest.CreateDir("bar", 0700), + fstest.CreateFile("bar/A", []byte("A"), 0777), + fstest.Lchtimes("bar/A", ts, ts.Add(6*time.Second)), + fstest.Lchtimes("bar", ts, ts.Add(6*time.Second)), + ) + snapB := committedKey(ctx, t, sn, identity.NewID(), snapA.Name, + fstest.Remove("/foo"), + + fstest.CreateFile("b", []byte("B"), 0777), + fstest.Lchtimes("b", ts, ts.Add(4*time.Second)), + + fstest.CreateFile("bar/B", []byte("B"), 0774), + fstest.Lchtimes("bar/B", ts, ts.Add(9*time.Second)), + fstest.Lchtimes("bar", ts, ts.Add(9*time.Second)), + ) + snapC := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateFile("foo", []byte("C"), 0775), + fstest.Lchtimes("foo", ts, ts.Add(4*time.Second)), + + fstest.CreateFile("c", []byte("C"), 0777), + fstest.Lchtimes("c", ts, ts.Add(6*time.Second)), + + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/A", []byte("C"), 0400), + fstest.Lchtimes("bar/A", ts, ts.Add(12*time.Second)), + fstest.Lchtimes("bar", ts, ts.Add(12*time.Second)), + + fstest.Symlink("foo", "symlink"), + fstest.Lchtimes("symlink", ts, ts.Add(3*time.Second)), + fstest.Link("bar/A", "hardlink"), + fstest.Symlink("../..", "dontfollowme"), + fstest.Lchtimes("dontfollowme", ts, ts.Add(2*time.Second)), + ) + + mergeA := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + {"", snapA.Name}, {snapA.Name, snapB.Name}, + {"", snapC.Name}, + }) + requireContents(ctx, t, sn, mergeA.Name, + fstest.CreateFile("a", []byte("A"), 0777), + fstest.CreateFile("b", []byte("B"), 0777), + fstest.CreateFile("c", []byte("C"), 0777), + + fstest.CreateFile("foo", []byte("C"), 0775), + + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/A", []byte("C"), 0400), + fstest.CreateFile("bar/B", []byte("B"), 0774), + + fstest.Symlink("foo", "symlink"), + fstest.Link("bar/A", "hardlink"), + fstest.Symlink("../..", "dontfollowme"), + ) + withMount(ctx, t, sn, mergeA.Name, func(root string) { + requireMtime(t, filepath.Join(root, "a"), ts.Add(4*time.Second)) + requireMtime(t, filepath.Join(root, "b"), ts.Add(4*time.Second)) + requireMtime(t, filepath.Join(root, "c"), ts.Add(6*time.Second)) + requireMtime(t, filepath.Join(root, "foo"), ts.Add(4*time.Second)) + requireMtime(t, filepath.Join(root, "bar"), ts.Add(12*time.Second)) + requireMtime(t, filepath.Join(root, "bar/A"), ts.Add(12*time.Second)) + requireMtime(t, filepath.Join(root, "bar/B"), ts.Add(9*time.Second)) + requireMtime(t, filepath.Join(root, "symlink"), ts.Add(3*time.Second)) + requireMtime(t, filepath.Join(root, "dontfollowme"), ts.Add(2*time.Second)) + }) + + mergeB := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + {"", snapC.Name}, + {"", snapA.Name}, {snapA.Name, snapB.Name}, + }) + requireContents(ctx, t, sn, mergeB.Name, + fstest.CreateFile("a", []byte("A"), 0777), + fstest.CreateFile("b", []byte("B"), 0777), + fstest.CreateFile("c", []byte("C"), 0777), + + fstest.CreateDir("bar", 0700), + fstest.CreateFile("bar/A", []byte("A"), 0777), + fstest.CreateFile("bar/B", []byte("B"), 0774), + + fstest.Symlink("foo", "symlink"), + fstest.CreateFile("hardlink", []byte("C"), 0400), // bar/A was overwritten, not considered hardlink + fstest.Symlink("../..", "dontfollowme"), + ) + withMount(ctx, t, sn, mergeB.Name, func(root string) { + requireMtime(t, filepath.Join(root, "a"), ts.Add(4*time.Second)) + requireMtime(t, filepath.Join(root, "b"), ts.Add(4*time.Second)) + requireMtime(t, filepath.Join(root, "c"), ts.Add(6*time.Second)) + requireMtime(t, filepath.Join(root, "bar"), ts.Add(9*time.Second)) + requireMtime(t, filepath.Join(root, "bar/A"), ts.Add(6*time.Second)) + requireMtime(t, filepath.Join(root, "bar/B"), ts.Add(9*time.Second)) + requireMtime(t, filepath.Join(root, "symlink"), ts.Add(3*time.Second)) + requireMtime(t, filepath.Join(root, "dontfollowme"), ts.Add(2*time.Second)) + }) + + snapD := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateDir("bar", 0750), + fstest.CreateFile("bar/D", []byte("D"), 0444), + fstest.CreateDir("fs", 0770), + fstest.CreateFile("x", []byte("X"), 0400), + fstest.Link("x", "hardlink"), + fstest.Symlink("fs", "symlink"), + fstest.Link("symlink", "hardsymlink"), + ) + + mergeC := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + // mergeA + {"", snapA.Name}, {snapA.Name, snapB.Name}, + {"", snapC.Name}, + // mergeB + {"", snapC.Name}, + {"", snapA.Name}, {snapA.Name, snapB.Name}, + // snapD + {"", snapD.Name}, + }) + requireContents(ctx, t, sn, mergeC.Name, + fstest.CreateFile("a", []byte("A"), 0777), + fstest.CreateFile("b", []byte("B"), 0777), + fstest.CreateFile("c", []byte("C"), 0777), + fstest.CreateDir("bar", 0750), + fstest.CreateFile("bar/A", []byte("A"), 0777), + fstest.CreateFile("bar/B", []byte("B"), 0774), + fstest.CreateFile("bar/D", []byte("D"), 0444), + fstest.CreateDir("fs", 0770), + fstest.CreateFile("x", []byte("X"), 0400), + fstest.Link("x", "hardlink"), + fstest.Symlink("fs", "symlink"), + fstest.Link("symlink", "hardsymlink"), + fstest.Symlink("../..", "dontfollowme"), + ) + + snapE := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateFile("qaz", nil, 0444), + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/B", []byte("B"), 0444), + ) + snapF := committedKey(ctx, t, sn, identity.NewID(), mergeC.Name, + fstest.Remove("a"), + fstest.CreateDir("a", 0770), + fstest.Rename("b", "a/b"), + fstest.Rename("c", "a/c"), + + fstest.RemoveAll("bar"), + fstest.CreateDir("bar", 0777), + fstest.CreateFile("bar/D", []byte("D2"), 0444), + + fstest.RemoveAll("fs"), + fstest.CreateFile("fs", nil, 0764), + + fstest.Remove("x"), + fstest.CreateDir("x", 0555), + + fstest.Remove("hardsymlink"), + fstest.CreateDir("hardsymlink", 0707), + ) + + mergeD := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + {"", snapE.Name}, + // mergeC + {"", snapA.Name}, {snapA.Name, snapB.Name}, + {"", snapC.Name}, + {"", snapC.Name}, + {"", snapA.Name}, {snapA.Name, snapB.Name}, + {"", snapD.Name}, + // snapF + {mergeC.Name, snapF.Name}, + }) + requireContents(ctx, t, sn, mergeD.Name, + fstest.CreateDir("a", 0770), + fstest.CreateFile("a/b", []byte("B"), 0777), + fstest.CreateDir("bar", 0777), + fstest.CreateFile("a/c", []byte("C"), 0777), + fstest.CreateFile("bar/D", []byte("D2"), 0444), + fstest.CreateFile("fs", nil, 0764), + fstest.CreateDir("x", 0555), + fstest.CreateFile("hardlink", []byte("X"), 0400), + fstest.Symlink("fs", "symlink"), + fstest.CreateDir("hardsymlink", 0707), + fstest.Symlink("../..", "dontfollowme"), + fstest.CreateFile("qaz", nil, 0444), + ) + }) + } +} + +func TestHardlinks(t *testing.T) { + for _, snName := range []string{"overlayfs", "native"} { + snName := snName + t.Run(snName, func(t *testing.T) { + t.Parallel() + if snName == "overlayfs" { + requireRoot(t) + } + + ctx, sn, cleanup, err := newSnapshotter(context.Background(), snName) + require.NoError(t, err) + defer cleanup() + + base1Snap := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateFile("1", []byte("1"), 0600), + ) + base2Snap := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateFile("2", []byte("2"), 0600), + ) + + mergeSnap := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + {"", base1Snap.Name}, + {"", base2Snap.Name}, + }) + stat1 := statPath(ctx, t, sn, mergeSnap.Name, "1") + var expected1Links int + switch snName { + case "overlayfs": + // base merge input is used as parent, not merged with hardlinks + expected1Links = 1 + case "native": + expected1Links = 2 + } + require.EqualValues(t, expected1Links, stat1.Nlink) + stat1Ino := stat1.Ino + + stat2 := statPath(ctx, t, sn, mergeSnap.Name, "2") + require.EqualValues(t, 2, stat2.Nlink) + stat2Ino := stat2.Ino + + childSnap := committedKey(ctx, t, sn, identity.NewID(), mergeSnap.Name, + fstest.CreateFile("1", []byte("11"), 0644), + fstest.CreateFile("2", []byte("22"), 0644), + ) + stat1 = statPath(ctx, t, sn, childSnap.Name, "1") + require.EqualValues(t, 1, stat1.Nlink) + require.NotEqualValues(t, stat1Ino, stat1.Ino) + stat2 = statPath(ctx, t, sn, childSnap.Name, "2") + require.EqualValues(t, 1, stat2.Nlink) + require.NotEqualValues(t, stat2Ino, stat2.Ino) + + // verify the original files and the files inthe merge are unchanged + requireContents(ctx, t, sn, base1Snap.Name, + fstest.CreateFile("1", []byte("1"), 0600), + ) + requireContents(ctx, t, sn, base2Snap.Name, + fstest.CreateFile("2", []byte("2"), 0600), + ) + requireContents(ctx, t, sn, mergeSnap.Name, + fstest.CreateFile("1", []byte("1"), 0600), + fstest.CreateFile("2", []byte("2"), 0600), + ) + }) + } +} + +func TestUsage(t *testing.T) { + for _, snName := range []string{"overlayfs", "native", "native-nohardlink"} { + snName := snName + t.Run(snName, func(t *testing.T) { + t.Parallel() + if snName == "overlayfs" { + requireRoot(t) + } + + ctx, sn, cleanup, err := newSnapshotter(context.Background(), snName) + require.NoError(t, err) + defer cleanup() + + const direntByteSize = 4096 + + base1Snap := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateDir("foo", 0777), + fstest.CreateFile("foo/1", []byte("a"), 0777), + ) + require.EqualValues(t, 3, base1Snap.Inodes) + require.EqualValues(t, 3*direntByteSize, base1Snap.Size) + + base2Snap := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateDir("foo", 0777), + fstest.CreateFile("foo/2", []byte("aa"), 0777), + ) + require.EqualValues(t, 3, base2Snap.Inodes) + require.EqualValues(t, 3*direntByteSize, base2Snap.Size) + + base3Snap := committedKey(ctx, t, sn, identity.NewID(), "", + fstest.CreateDir("foo", 0777), + fstest.CreateFile("foo/3", []byte("aaa"), 0777), + fstest.CreateFile("bar", nil, 0777), + ) + require.EqualValues(t, 4, base3Snap.Inodes) + require.EqualValues(t, 3*direntByteSize, base3Snap.Size) + + mergeSnap := mergeKey(ctx, t, sn, identity.NewID(), []Diff{ + {"", base1Snap.Name}, + {"", base2Snap.Name}, + {"", base3Snap.Name}, + }) + switch snName { + case "overlayfs", "native": + // / and /foo were created/copied. Others should be hard-linked + require.EqualValues(t, 2, mergeSnap.Inodes) + require.EqualValues(t, 2*direntByteSize, mergeSnap.Size) + case "native-nohardlink": + require.EqualValues(t, 6, mergeSnap.Inodes) + require.EqualValues(t, 5*direntByteSize, mergeSnap.Size) + } + }) + } +} + +type snapshotInfo struct { + snapshots.Info + snapshots.Usage +} + +func getInfo(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key string) snapshotInfo { + t.Helper() + info, err := sn.Stat(ctx, key) + require.NoError(t, err) + usage, err := sn.Usage(ctx, key) + require.NoError(t, err) + return snapshotInfo{info, usage} +} + +func activeKey( + ctx context.Context, + t *testing.T, + sn *mergeSnapshotter, + key string, + parent string, + files ...fstest.Applier, +) snapshotInfo { + t.Helper() + + err := sn.Prepare(ctx, key, parent) + require.NoError(t, err) + + if len(files) > 0 { + mnts, cleanup := getMounts(ctx, t, sn, key) + defer cleanup() + mnter := LocalMounterWithMounts(mnts) + root, err := mnter.Mount() + require.NoError(t, err) + defer mnter.Unmount() + require.NoError(t, fstest.Apply(files...).Apply(root)) + } + + return getInfo(ctx, t, sn, key) +} + +func commitActiveKey( + ctx context.Context, + t *testing.T, + sn *mergeSnapshotter, + name string, + activeKey string, +) snapshotInfo { + t.Helper() + err := sn.Commit(ctx, name, activeKey) + require.NoError(t, err) + return getInfo(ctx, t, sn, name) +} + +func committedKey( + ctx context.Context, + t *testing.T, + sn *mergeSnapshotter, + key string, + parent string, + files ...fstest.Applier, +) snapshotInfo { + t.Helper() + prepareKey := identity.NewID() + activeKey(ctx, t, sn, prepareKey, parent, files...) + return commitActiveKey(ctx, t, sn, key, prepareKey) +} + +func mergeKey( + ctx context.Context, + t *testing.T, + sn *mergeSnapshotter, + key string, + diffs []Diff, +) snapshotInfo { + t.Helper() + err := sn.Merge(ctx, key, diffs) + require.NoError(t, err) + return getInfo(ctx, t, sn, key) +} + +func getMounts(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key string) ([]mount.Mount, func() error) { + t.Helper() + + var mntable Mountable + var err error + if info := getInfo(ctx, t, sn, key); info.Kind == snapshots.KindCommitted { + mntable, err = sn.View(ctx, identity.NewID(), key) + } else { + mntable, err = sn.Mounts(ctx, key) + } + require.NoError(t, err) + + mnts, cleanup, err := mntable.Mount() + require.NoError(t, err) + return mnts, cleanup +} + +func withMount(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key string, f func(root string)) { + t.Helper() + mounts, cleanup := getMounts(ctx, t, sn, key) + defer cleanup() + mnter := LocalMounterWithMounts(mounts) + root, err := mnter.Mount() + require.NoError(t, err) + defer mnter.Unmount() + f(root) +} + +func requireContents(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key string, files ...fstest.Applier) { + t.Helper() + withMount(ctx, t, sn, key, func(root string) { + require.NoError(t, fstest.CheckDirectoryEqualWithApplier(root, fstest.Apply(files...))) + }) +} + +func trySyscallStat(t *testing.T, path string) *syscall.Stat_t { + t.Helper() + info, err := os.Stat(path) + if err == nil { + return info.Sys().(*syscall.Stat_t) + } + require.ErrorIs(t, err, os.ErrNotExist) + return nil +} + +func requireMtime(t *testing.T, path string, mtime time.Time) { + t.Helper() + info, err := os.Lstat(path) + require.NoError(t, err) + stat := info.Sys().(*syscall.Stat_t) + require.Equal(t, mtime.UnixNano(), stat.Mtim.Nano()) +} + +func tryStatPath(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key, path string) (st *syscall.Stat_t) { + t.Helper() + mounts, cleanup := getMounts(ctx, t, sn, key) + defer cleanup() + require.Len(t, mounts, 1) + mnt := mounts[0] + + if mnt.Type == "overlay" { + var upperdir string + var lowerdirs []string + for _, opt := range mnt.Options { + if strings.HasPrefix(opt, "upperdir=") { + upperdir = strings.SplitN(opt, "upperdir=", 2)[1] + } else if strings.HasPrefix(opt, "lowerdir=") { + lowerdirs = strings.Split(strings.SplitN(opt, "lowerdir=", 2)[1], ":") + } + } + if upperdir != "" { + st = trySyscallStat(t, filepath.Join(upperdir, path)) + if st != nil { + return st + } + } + for _, lowerdir := range lowerdirs { + st = trySyscallStat(t, filepath.Join(lowerdir, path)) + if st != nil { + return st + } + } + return nil + } + + withMount(ctx, t, sn, key, func(root string) { + st = trySyscallStat(t, filepath.Join(root, path)) + }) + return st +} + +func statPath(ctx context.Context, t *testing.T, sn *mergeSnapshotter, key, path string) (st *syscall.Stat_t) { + t.Helper() + st = tryStatPath(ctx, t, sn, key, path) + require.NotNil(t, st) + return st +} + +func requireRoot(t *testing.T) { + t.Helper() + if os.Getuid() != 0 { + t.Skip("test requires root") + } +} diff --git a/solver/llbsolver/mounts/mount_test.go b/solver/llbsolver/mounts/mount_test.go index b018a3ab0b98..4172c8807847 100644 --- a/solver/llbsolver/mounts/mount_test.go +++ b/solver/llbsolver/mounts/mount_test.go @@ -12,6 +12,7 @@ import ( "github.com/containerd/containerd/content" "github.com/containerd/containerd/content/local" "github.com/containerd/containerd/diff/apply" + "github.com/containerd/containerd/diff/walking" "github.com/containerd/containerd/leases" ctdmetadata "github.com/containerd/containerd/metadata" "github.com/containerd/containerd/namespaces" @@ -23,6 +24,7 @@ import ( containerdsnapshot "github.com/moby/buildkit/snapshot/containerd" "github.com/moby/buildkit/solver/pb" "github.com/moby/buildkit/util/leaseutil" + "github.com/moby/buildkit/util/winlayers" "github.com/pkg/errors" "github.com/stretchr/testify/require" bolt "go.etcd.io/bbolt" @@ -109,6 +111,9 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() } lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), ns) + c := mdb.ContentStore() + applier := winlayers.NewFileSystemApplierWithWindows(c, apply.NewFileSystemApplier(c)) + differ := winlayers.NewWalkingDiffWithWindows(c, walking.NewWalkingDiff(c)) md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) if err != nil { @@ -118,10 +123,11 @@ func newCacheManager(ctx context.Context, opt cmOpt) (co *cmOut, cleanup func() cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter(opt.snapshotterName, containerdsnapshot.NSSnapshotter(ns, mdb.Snapshotter(opt.snapshotterName)), nil), MetadataStore: md, - ContentStore: mdb.ContentStore(), + ContentStore: c, + Applier: applier, + Differ: differ, LeaseManager: lm, GarbageCollect: mdb.GarbageCollect, - Applier: apply.NewFileSystemApplier(mdb.ContentStore()), }) if err != nil { return nil, nil, err diff --git a/solver/llbsolver/ops/merge.go b/solver/llbsolver/ops/merge.go new file mode 100644 index 000000000000..24d858cb93f1 --- /dev/null +++ b/solver/llbsolver/ops/merge.go @@ -0,0 +1,95 @@ +package ops + +import ( + "context" + "encoding/json" + "fmt" + "strings" + + "github.com/moby/buildkit/worker" + "github.com/pkg/errors" + + "github.com/moby/buildkit/cache" + "github.com/moby/buildkit/session" + "github.com/moby/buildkit/solver" + "github.com/moby/buildkit/solver/llbsolver" + "github.com/moby/buildkit/solver/pb" + digest "github.com/opencontainers/go-digest" +) + +const mergeCacheType = "buildkit.merge.v0" + +type mergeOp struct { + op *pb.MergeOp + worker worker.Worker +} + +func NewMergeOp(v solver.Vertex, op *pb.Op_Merge, w worker.Worker) (solver.Op, error) { + if err := llbsolver.ValidateOp(&pb.Op{Op: op}); err != nil { + return nil, err + } + return &mergeOp{ + op: op.Merge, + worker: w, + }, nil +} + +func (m *mergeOp) CacheMap(ctx context.Context, group session.Group, index int) (*solver.CacheMap, bool, error) { + dt, err := json.Marshal(struct { + Type string + Merge *pb.MergeOp + }{ + Type: mergeCacheType, + Merge: m.op, + }) + if err != nil { + return nil, false, err + } + + cm := &solver.CacheMap{ + Digest: digest.Digest(dt), + Deps: make([]struct { + Selector digest.Digest + ComputeDigestFunc solver.ResultBasedCacheFunc + PreprocessFunc solver.PreprocessFunc + }, len(m.op.Inputs)), + } + + return cm, true, nil +} + +func (m *mergeOp) Exec(ctx context.Context, g session.Group, inputs []solver.Result) ([]solver.Result, error) { + refs := make([]cache.ImmutableRef, len(inputs)) + ids := make([]string, len(inputs)) + var index int + for _, inp := range inputs { + if inp == nil { + continue + } + wref, ok := inp.Sys().(*worker.WorkerRef) + if !ok { + return nil, errors.Errorf("invalid reference for merge %T", inp.Sys()) + } + refs[index] = wref.ImmutableRef + ids[index] = wref.ImmutableRef.ID() + index++ + } + refs = refs[:index] + ids = ids[:index] + + if len(refs) == 0 { + return nil, nil + } + + mergedRef, err := m.worker.CacheManager().Merge(ctx, refs, + cache.WithDescription(fmt.Sprintf("merge %s", strings.Join(ids, ";")))) + if err != nil { + return nil, err + } + + return []solver.Result{worker.NewWorkerRefResult(mergedRef, m.worker)}, nil +} + +func (m *mergeOp) Acquire(ctx context.Context) (release solver.ReleaseFunc, err error) { + return func() {}, nil +} diff --git a/solver/llbsolver/result.go b/solver/llbsolver/result.go index 321a83f904c1..0ad88c71e54a 100644 --- a/solver/llbsolver/result.go +++ b/solver/llbsolver/result.go @@ -66,7 +66,7 @@ func NewContentHashFunc(selectors []Selector) solver.ResultBasedCacheFunc { s, ) if err != nil { - return err + return errors.Wrapf(err, "failed to calculate checksum of ref %s", ref.ID()) } dgsts[i] = []byte(dgst) return nil diff --git a/solver/llbsolver/vertex.go b/solver/llbsolver/vertex.go index d7c025f098de..58b0948b11e4 100644 --- a/solver/llbsolver/vertex.go +++ b/solver/llbsolver/vertex.go @@ -2,6 +2,7 @@ package llbsolver import ( "fmt" + "strconv" "strings" "github.com/containerd/containerd/platforms" @@ -169,7 +170,11 @@ func newVertex(dgst digest.Digest, op *pb.Op, opMeta *pb.OpMetadata, load func(d } } - vtx := &vertex{sys: op, options: opt, digest: dgst, name: llbOpName(op)} + name, err := llbOpName(op, load) + if err != nil { + return nil, err + } + vtx := &vertex{sys: op, options: opt, digest: dgst, name: name} for _, in := range op.Inputs { sub, err := load(in.Digest) if err != nil { @@ -242,25 +247,35 @@ func loadLLB(def *pb.Definition, fn func(digest.Digest, *pb.Op, func(digest.Dige return solver.Edge{Vertex: v, Index: solver.Index(lastOp.Inputs[0].Index)}, nil } -func llbOpName(op *pb.Op) string { - switch op := op.Op.(type) { +func llbOpName(pbOp *pb.Op, load func(digest.Digest) (solver.Vertex, error)) (string, error) { + switch op := pbOp.Op.(type) { case *pb.Op_Source: if id, err := source.FromLLB(op, nil); err == nil { if id, ok := id.(*source.LocalIdentifier); ok { if len(id.IncludePatterns) == 1 { - return op.Source.Identifier + " (" + id.IncludePatterns[0] + ")" + return op.Source.Identifier + " (" + id.IncludePatterns[0] + ")", nil } } } - return op.Source.Identifier + return op.Source.Identifier, nil case *pb.Op_Exec: - return strings.Join(op.Exec.Meta.Args, " ") + return strings.Join(op.Exec.Meta.Args, " "), nil case *pb.Op_File: - return fileOpName(op.File.Actions) + return fileOpName(op.File.Actions), nil case *pb.Op_Build: - return "build" + return "build", nil + case *pb.Op_Merge: + subnames := make([]string, len(pbOp.Inputs)) + for i, inp := range pbOp.Inputs { + subvtx, err := load(inp.Digest) + if err != nil { + return "", err + } + subnames[i] = strconv.Quote(subvtx.Name()) + } + return "merge " + strings.Join(subnames, " + "), nil default: - return "unknown" + return "unknown", nil } } @@ -309,6 +324,10 @@ func ValidateOp(op *pb.Op) error { if op.Build == nil { return errors.Errorf("invalid nil build op") } + case *pb.Op_Merge: + if op.Merge == nil { + return errors.Errorf("invalid nil merge op") + } } return nil } diff --git a/solver/pb/caps.go b/solver/pb/caps.go index 361f6f6d6ee6..5847b1754fd0 100644 --- a/solver/pb/caps.go +++ b/solver/pb/caps.go @@ -66,6 +66,8 @@ const ( CapMetaExportCache apicaps.CapID = "meta.exportcache" CapRemoteCacheGHA apicaps.CapID = "cache.gha" + + CapMergeOp apicaps.CapID = "mergeop" ) func init() { @@ -367,4 +369,9 @@ func init() { Enabled: true, Status: apicaps.CapStatusExperimental, }) + Caps.Init(apicaps.Cap{ + ID: CapMergeOp, + Enabled: true, + Status: apicaps.CapStatusExperimental, + }) } diff --git a/solver/pb/ops.pb.go b/solver/pb/ops.pb.go index ecc7345f0f9b..a40aec3277e0 100644 --- a/solver/pb/ops.pb.go +++ b/solver/pb/ops.pb.go @@ -158,6 +158,7 @@ type Op struct { // *Op_Source // *Op_File // *Op_Build + // *Op_Merge Op isOp_Op `protobuf_oneof:"op"` Platform *Platform `protobuf:"bytes,10,opt,name=platform,proto3" json:"platform,omitempty"` Constraints *WorkerConstraints `protobuf:"bytes,11,opt,name=constraints,proto3" json:"constraints,omitempty"` @@ -210,11 +211,15 @@ type Op_File struct { type Op_Build struct { Build *BuildOp `protobuf:"bytes,5,opt,name=build,proto3,oneof" json:"build,omitempty"` } +type Op_Merge struct { + Merge *MergeOp `protobuf:"bytes,6,opt,name=merge,proto3,oneof" json:"merge,omitempty"` +} func (*Op_Exec) isOp_Op() {} func (*Op_Source) isOp_Op() {} func (*Op_File) isOp_Op() {} func (*Op_Build) isOp_Op() {} +func (*Op_Merge) isOp_Op() {} func (m *Op) GetOp() isOp_Op { if m != nil { @@ -258,6 +263,13 @@ func (m *Op) GetBuild() *BuildOp { return nil } +func (m *Op) GetMerge() *MergeOp { + if x, ok := m.GetOp().(*Op_Merge); ok { + return x.Merge + } + return nil +} + func (m *Op) GetPlatform() *Platform { if m != nil { return m.Platform @@ -279,6 +291,7 @@ func (*Op) XXX_OneofWrappers() []interface{} { (*Op_Source)(nil), (*Op_File)(nil), (*Op_Build)(nil), + (*Op_Merge)(nil), } } @@ -2430,6 +2443,79 @@ func (m *NamedUserOpt) GetName() string { return "" } +type MergeInput struct { + Input InputIndex `protobuf:"varint,1,opt,name=input,proto3,customtype=InputIndex" json:"input"` +} + +func (m *MergeInput) Reset() { *m = MergeInput{} } +func (m *MergeInput) String() string { return proto.CompactTextString(m) } +func (*MergeInput) ProtoMessage() {} +func (*MergeInput) Descriptor() ([]byte, []int) { + return fileDescriptor_8de16154b2733812, []int{35} +} +func (m *MergeInput) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *MergeInput) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeInput.Merge(m, src) +} +func (m *MergeInput) XXX_Size() int { + return m.Size() +} +func (m *MergeInput) XXX_DiscardUnknown() { + xxx_messageInfo_MergeInput.DiscardUnknown(m) +} + +var xxx_messageInfo_MergeInput proto.InternalMessageInfo + +type MergeOp struct { + Inputs []*MergeInput `protobuf:"bytes,1,rep,name=inputs,proto3" json:"inputs,omitempty"` +} + +func (m *MergeOp) Reset() { *m = MergeOp{} } +func (m *MergeOp) String() string { return proto.CompactTextString(m) } +func (*MergeOp) ProtoMessage() {} +func (*MergeOp) Descriptor() ([]byte, []int) { + return fileDescriptor_8de16154b2733812, []int{36} +} +func (m *MergeOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *MergeOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeOp.Merge(m, src) +} +func (m *MergeOp) XXX_Size() int { + return m.Size() +} +func (m *MergeOp) XXX_DiscardUnknown() { + xxx_messageInfo_MergeOp.DiscardUnknown(m) +} + +var xxx_messageInfo_MergeOp proto.InternalMessageInfo + +func (m *MergeOp) GetInputs() []*MergeInput { + if m != nil { + return m.Inputs + } + return nil +} + func init() { proto.RegisterEnum("pb.NetMode", NetMode_name, NetMode_value) proto.RegisterEnum("pb.SecurityMode", SecurityMode_name, SecurityMode_value) @@ -2477,159 +2563,164 @@ func init() { proto.RegisterType((*ChownOpt)(nil), "pb.ChownOpt") proto.RegisterType((*UserOpt)(nil), "pb.UserOpt") proto.RegisterType((*NamedUserOpt)(nil), "pb.NamedUserOpt") + proto.RegisterType((*MergeInput)(nil), "pb.MergeInput") + proto.RegisterType((*MergeOp)(nil), "pb.MergeOp") } func init() { proto.RegisterFile("ops.proto", fileDescriptor_8de16154b2733812) } var fileDescriptor_8de16154b2733812 = []byte{ - // 2349 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xcd, 0x6f, 0x1c, 0xb7, - 0x15, 0xd7, 0x7e, 0xef, 0xbc, 0x5d, 0xc9, 0x5b, 0xc6, 0x49, 0x36, 0xaa, 0x2b, 0x29, 0x93, 0x34, - 0x90, 0x65, 0x7b, 0x05, 0x28, 0x40, 0x1c, 0x04, 0x45, 0x51, 0xed, 0x87, 0xa1, 0x4d, 0x6c, 0xad, - 0xc0, 0x95, 0xed, 0xde, 0x8c, 0xd1, 0x2c, 0x57, 0x1a, 0x68, 0x76, 0x38, 0x98, 0xe1, 0xda, 0xda, - 0x1e, 0x7a, 0xc8, 0xbd, 0x40, 0x80, 0x02, 0x45, 0x2f, 0x6d, 0xd1, 0xff, 0xa1, 0xd7, 0xde, 0x73, - 0xcc, 0xa1, 0x87, 0xa0, 0x87, 0xb4, 0xb0, 0xff, 0x8d, 0x16, 0x28, 0xde, 0x23, 0xe7, 0x63, 0x65, - 0x19, 0xb6, 0xd1, 0xa2, 0xa7, 0x21, 0x7f, 0xef, 0xc7, 0xc7, 0x47, 0xf2, 0x47, 0xf2, 0x71, 0xc0, - 0x92, 0x61, 0xdc, 0x09, 0x23, 0xa9, 0x24, 0x2b, 0x86, 0x27, 0xeb, 0x77, 0x4e, 0x3d, 0x75, 0x36, - 0x3f, 0xe9, 0xb8, 0x72, 0xb6, 0x7b, 0x2a, 0x4f, 0xe5, 0x2e, 0x99, 0x4e, 0xe6, 0x53, 0xaa, 0x51, - 0x85, 0x4a, 0xba, 0x89, 0xfd, 0xe7, 0x22, 0x14, 0x47, 0x21, 0xfb, 0x10, 0xaa, 0x5e, 0x10, 0xce, - 0x55, 0xdc, 0x2e, 0x6c, 0x95, 0xb6, 0x1b, 0x7b, 0x56, 0x27, 0x3c, 0xe9, 0x0c, 0x11, 0xe1, 0xc6, - 0xc0, 0xb6, 0xa0, 0x2c, 0x2e, 0x84, 0xdb, 0x2e, 0x6e, 0x15, 0xb6, 0x1b, 0x7b, 0x80, 0x84, 0xc1, - 0x85, 0x70, 0x47, 0xe1, 0xc1, 0x0a, 0x27, 0x0b, 0xfb, 0x04, 0xaa, 0xb1, 0x9c, 0x47, 0xae, 0x68, - 0x97, 0x88, 0xd3, 0x44, 0xce, 0x98, 0x10, 0x62, 0x19, 0x2b, 0x7a, 0x9a, 0x7a, 0xbe, 0x68, 0x97, - 0x33, 0x4f, 0xf7, 0x3c, 0x5f, 0x73, 0xc8, 0xc2, 0x3e, 0x82, 0xca, 0xc9, 0xdc, 0xf3, 0x27, 0xed, - 0x0a, 0x51, 0x1a, 0x48, 0xe9, 0x22, 0x40, 0x1c, 0x6d, 0x63, 0xdb, 0x50, 0x0f, 0x7d, 0x47, 0x4d, - 0x65, 0x34, 0x6b, 0x43, 0xd6, 0xe1, 0x91, 0xc1, 0x78, 0x6a, 0x65, 0x77, 0xa1, 0xe1, 0xca, 0x20, - 0x56, 0x91, 0xe3, 0x05, 0x2a, 0x6e, 0x37, 0x88, 0xfc, 0x2e, 0x92, 0x1f, 0xcb, 0xe8, 0x5c, 0x44, - 0xbd, 0xcc, 0xc8, 0xf3, 0xcc, 0x6e, 0x19, 0x8a, 0x32, 0xb4, 0x7f, 0x57, 0x80, 0x7a, 0xe2, 0x95, - 0xd9, 0xd0, 0xdc, 0x8f, 0xdc, 0x33, 0x4f, 0x09, 0x57, 0xcd, 0x23, 0xd1, 0x2e, 0x6c, 0x15, 0xb6, - 0x2d, 0xbe, 0x84, 0xb1, 0x35, 0x28, 0x8e, 0xc6, 0x34, 0x51, 0x16, 0x2f, 0x8e, 0xc6, 0xac, 0x0d, - 0xb5, 0x47, 0x4e, 0xe4, 0x39, 0x81, 0xa2, 0x99, 0xb1, 0x78, 0x52, 0x65, 0x37, 0xc0, 0x1a, 0x8d, - 0x1f, 0x89, 0x28, 0xf6, 0x64, 0x40, 0xf3, 0x61, 0xf1, 0x0c, 0x60, 0x1b, 0x00, 0xa3, 0xf1, 0x3d, - 0xe1, 0xa0, 0xd3, 0xb8, 0x5d, 0xd9, 0x2a, 0x6d, 0x5b, 0x3c, 0x87, 0xd8, 0xbf, 0x86, 0x0a, 0xad, - 0x11, 0xfb, 0x12, 0xaa, 0x13, 0xef, 0x54, 0xc4, 0x4a, 0x87, 0xd3, 0xdd, 0xfb, 0xf6, 0x87, 0xcd, - 0x95, 0xbf, 0xff, 0xb0, 0xb9, 0x93, 0x13, 0x83, 0x0c, 0x45, 0xe0, 0xca, 0x40, 0x39, 0x5e, 0x20, - 0xa2, 0x78, 0xf7, 0x54, 0xde, 0xd1, 0x4d, 0x3a, 0x7d, 0xfa, 0x70, 0xe3, 0x81, 0xdd, 0x84, 0x8a, - 0x17, 0x4c, 0xc4, 0x05, 0xc5, 0x5f, 0xea, 0xbe, 0x63, 0x5c, 0x35, 0x46, 0x73, 0x15, 0xce, 0xd5, - 0x10, 0x4d, 0x5c, 0x33, 0xec, 0x3f, 0x16, 0xa0, 0xaa, 0x35, 0xc0, 0x6e, 0x40, 0x79, 0x26, 0x94, - 0x43, 0xfd, 0x37, 0xf6, 0xea, 0x38, 0xb7, 0x0f, 0x84, 0x72, 0x38, 0xa1, 0x28, 0xaf, 0x99, 0x9c, - 0xe3, 0xdc, 0x17, 0x33, 0x79, 0x3d, 0x40, 0x84, 0x1b, 0x03, 0xfb, 0x29, 0xd4, 0x02, 0xa1, 0x9e, - 0xc9, 0xe8, 0x9c, 0xe6, 0x68, 0x4d, 0x2f, 0xfa, 0xa1, 0x50, 0x0f, 0xe4, 0x44, 0xf0, 0xc4, 0xc6, - 0x6e, 0x43, 0x3d, 0x16, 0xee, 0x3c, 0xf2, 0xd4, 0x82, 0xe6, 0x6b, 0x6d, 0xaf, 0x45, 0x2a, 0x33, - 0x18, 0x91, 0x53, 0x86, 0xfd, 0x9b, 0x22, 0x94, 0x31, 0x0c, 0xc6, 0xa0, 0xec, 0x44, 0xa7, 0x5a, - 0xdd, 0x16, 0xa7, 0x32, 0x6b, 0x41, 0x49, 0x04, 0x4f, 0x29, 0x22, 0x8b, 0x63, 0x11, 0x11, 0xf7, - 0xd9, 0xc4, 0xac, 0x11, 0x16, 0xb1, 0xdd, 0x3c, 0x16, 0x91, 0x59, 0x1a, 0x2a, 0xb3, 0x9b, 0x60, - 0x85, 0x91, 0xbc, 0x58, 0x3c, 0xc1, 0xd6, 0x95, 0x9c, 0xf0, 0x10, 0x1c, 0x04, 0x4f, 0x79, 0x3d, - 0x34, 0x25, 0xb6, 0x03, 0x20, 0x2e, 0x54, 0xe4, 0x1c, 0xc8, 0x58, 0xc5, 0xed, 0x2a, 0x8d, 0x9d, - 0xf4, 0x8e, 0xc0, 0xf0, 0x88, 0xe7, 0xac, 0x6c, 0x1d, 0xea, 0x67, 0x32, 0x56, 0x81, 0x33, 0x13, - 0xed, 0x1a, 0x75, 0x97, 0xd6, 0x99, 0x0d, 0xd5, 0xb9, 0xef, 0xcd, 0x3c, 0xd5, 0xb6, 0x32, 0x1f, - 0x0f, 0x09, 0xe1, 0xc6, 0x82, 0xc2, 0x74, 0x4f, 0x23, 0x39, 0x0f, 0x8f, 0x9c, 0x48, 0x04, 0x8a, - 0xb6, 0x84, 0xc5, 0x97, 0x30, 0xfb, 0x36, 0x54, 0x75, 0xcf, 0x38, 0x30, 0x2c, 0x19, 0xf9, 0x52, - 0x19, 0x65, 0x3b, 0x3c, 0x4a, 0x64, 0x3b, 0x3c, 0xb2, 0xfb, 0x50, 0xd5, 0x7d, 0x20, 0xfb, 0x10, - 0xe3, 0x32, 0x6c, 0x2c, 0x23, 0x36, 0x96, 0x53, 0xa5, 0x65, 0xc2, 0xa9, 0x4c, 0x5e, 0x9d, 0x48, - 0xcf, 0x60, 0x89, 0x53, 0xd9, 0xfe, 0x7d, 0x09, 0x2a, 0xb4, 0xd4, 0x6c, 0x1b, 0x95, 0x15, 0xce, - 0x75, 0xa7, 0xa5, 0x2e, 0x33, 0xca, 0x02, 0xd2, 0x70, 0x2a, 0x2c, 0xd4, 0xf3, 0x3a, 0xae, 0xb2, - 0x2f, 0x5c, 0x25, 0x23, 0x13, 0x4f, 0x5a, 0xc7, 0x3e, 0x26, 0xa8, 0x74, 0xbd, 0x4a, 0x54, 0x66, - 0xb7, 0xa0, 0x2a, 0x49, 0x9e, 0xb4, 0x50, 0xaf, 0x10, 0xad, 0xa1, 0xa0, 0xf3, 0x48, 0x38, 0x13, - 0x19, 0xf8, 0x0b, 0x5a, 0xbe, 0x3a, 0x4f, 0xeb, 0xec, 0x16, 0x58, 0xa4, 0xc7, 0xe3, 0x45, 0x28, - 0xda, 0x55, 0xd2, 0xd7, 0x6a, 0xaa, 0x55, 0x04, 0x79, 0x66, 0xc7, 0x03, 0xe8, 0x78, 0x16, 0x4e, - 0xe3, 0x51, 0xa8, 0xda, 0xef, 0x64, 0x3a, 0x48, 0x30, 0x9e, 0x5a, 0x91, 0xe9, 0x3a, 0xee, 0x99, - 0x40, 0xe6, 0xf5, 0x8c, 0xd9, 0x33, 0x18, 0x4f, 0xad, 0x18, 0x40, 0x2c, 0xdc, 0x48, 0x28, 0xa4, - 0xbe, 0x4b, 0xd4, 0x55, 0x23, 0x70, 0x0d, 0xf2, 0xcc, 0x8e, 0xb2, 0x18, 0x8f, 0x0f, 0x90, 0xf9, - 0x5e, 0x76, 0x94, 0x6a, 0x84, 0x1b, 0x8b, 0x1e, 0x6d, 0x3c, 0xf7, 0xd5, 0xb0, 0xdf, 0x7e, 0x5f, - 0x4f, 0x65, 0x52, 0xb7, 0x37, 0xb2, 0x01, 0xe0, 0xb4, 0xc6, 0xde, 0xaf, 0xf4, 0x12, 0x97, 0x38, - 0x95, 0xed, 0x21, 0xd4, 0x93, 0x10, 0x49, 0x1c, 0x7d, 0x23, 0x80, 0xe2, 0xb0, 0xcf, 0xee, 0x40, - 0x2d, 0x3e, 0x73, 0x22, 0x2f, 0x38, 0xa5, 0x15, 0x5a, 0xdb, 0x7b, 0x27, 0x1d, 0xd1, 0x58, 0xe3, - 0x18, 0x45, 0xc2, 0xb1, 0x25, 0x58, 0xe9, 0x10, 0x5e, 0xf2, 0xd5, 0x82, 0xd2, 0xdc, 0x9b, 0x90, - 0x9f, 0x55, 0x8e, 0x45, 0x44, 0x4e, 0x3d, 0xad, 0xa3, 0x55, 0x8e, 0x45, 0x8c, 0x6f, 0x26, 0x27, - 0xfa, 0xd2, 0x58, 0xe5, 0x54, 0xc6, 0xb1, 0xc9, 0x50, 0x79, 0x32, 0x70, 0xfc, 0x64, 0x25, 0x93, - 0xba, 0xed, 0x27, 0x73, 0xf3, 0x7f, 0xe9, 0xed, 0xb7, 0x05, 0xa8, 0x27, 0x37, 0x1d, 0x1e, 0xdb, - 0xde, 0x44, 0x04, 0xca, 0x9b, 0x7a, 0x22, 0x32, 0x1d, 0xe7, 0x10, 0x76, 0x07, 0x2a, 0x8e, 0x52, - 0x51, 0x72, 0x18, 0xbe, 0x9f, 0xbf, 0x26, 0x3b, 0xfb, 0x68, 0x19, 0x04, 0x2a, 0x5a, 0x70, 0xcd, - 0x5a, 0xff, 0x1c, 0x20, 0x03, 0x31, 0xd6, 0x73, 0xb1, 0x30, 0x5e, 0xb1, 0xc8, 0xae, 0x43, 0xe5, - 0xa9, 0xe3, 0xcf, 0x85, 0xd9, 0x29, 0xba, 0xf2, 0x45, 0xf1, 0xf3, 0x82, 0xfd, 0xd7, 0x22, 0xd4, - 0xcc, 0xb5, 0xc9, 0x6e, 0x43, 0x8d, 0xae, 0x4d, 0x13, 0xd1, 0xd5, 0xdb, 0x2f, 0xa1, 0xb0, 0xdd, - 0x34, 0x1f, 0xc8, 0xc5, 0x68, 0x5c, 0xe9, 0xbc, 0xc0, 0xc4, 0x98, 0x65, 0x07, 0xa5, 0x89, 0x98, - 0x9a, 0x8b, 0x7f, 0x0d, 0xd9, 0x7d, 0x31, 0xf5, 0x02, 0x0f, 0xe7, 0x87, 0xa3, 0x89, 0xdd, 0x4e, - 0x46, 0x5d, 0x26, 0x8f, 0xef, 0xe5, 0x3d, 0xbe, 0x3c, 0xe8, 0x21, 0x34, 0x72, 0xdd, 0x5c, 0x31, - 0xea, 0x8f, 0xf3, 0xa3, 0x36, 0x5d, 0x92, 0x3b, 0x9d, 0xb5, 0x64, 0xb3, 0xf0, 0x5f, 0xcc, 0xdf, - 0x67, 0x00, 0x99, 0xcb, 0x37, 0x3f, 0xbe, 0xec, 0xaf, 0x4b, 0x00, 0xa3, 0x10, 0x2f, 0x9e, 0x89, - 0x43, 0xb7, 0x5f, 0xd3, 0x3b, 0x0d, 0x64, 0x24, 0x9e, 0xd0, 0x36, 0xa7, 0xf6, 0x75, 0xde, 0xd0, - 0x18, 0xed, 0x18, 0xb6, 0x0f, 0x8d, 0x89, 0x88, 0xdd, 0xc8, 0x23, 0x41, 0x99, 0x49, 0xdf, 0xc4, - 0x31, 0x65, 0x7e, 0x3a, 0xfd, 0x8c, 0xa1, 0xe7, 0x2a, 0xdf, 0x86, 0xed, 0x41, 0x53, 0x5c, 0x84, - 0x32, 0x52, 0xa6, 0x17, 0x9d, 0x5d, 0x5d, 0xd3, 0x79, 0x1a, 0xe2, 0xd4, 0x13, 0x6f, 0x88, 0xac, - 0xc2, 0x1c, 0x28, 0xbb, 0x4e, 0xa8, 0x53, 0x8b, 0xc6, 0x5e, 0xfb, 0x52, 0x7f, 0x3d, 0x27, 0xd4, - 0x93, 0xd6, 0xfd, 0x14, 0xc7, 0xfa, 0xf5, 0x3f, 0x36, 0x6f, 0xe5, 0xf2, 0x89, 0x99, 0x3c, 0x59, - 0xec, 0x92, 0x5e, 0xce, 0x3d, 0xb5, 0x3b, 0x57, 0x9e, 0xbf, 0xeb, 0x84, 0x1e, 0xba, 0xc3, 0x86, - 0xc3, 0x3e, 0x27, 0xd7, 0xeb, 0x3f, 0x87, 0xd6, 0xe5, 0xb8, 0xdf, 0x66, 0x0d, 0xd6, 0xef, 0x82, - 0x95, 0xc6, 0xf1, 0xba, 0x86, 0xf5, 0xfc, 0xe2, 0xfd, 0xa5, 0x00, 0x55, 0xbd, 0xab, 0xd8, 0x5d, - 0xb0, 0x7c, 0xe9, 0x3a, 0x18, 0x40, 0x92, 0xe0, 0x7e, 0x90, 0x6d, 0xba, 0xce, 0xfd, 0xc4, 0xa6, - 0x67, 0x35, 0xe3, 0xa2, 0xc8, 0xbc, 0x60, 0x2a, 0x93, 0x5d, 0xb0, 0x96, 0x35, 0x1a, 0x06, 0x53, - 0xc9, 0xb5, 0x71, 0xfd, 0x2b, 0x58, 0x5b, 0x76, 0x71, 0x45, 0x9c, 0x1f, 0x2d, 0xcb, 0x95, 0xce, - 0xf4, 0xb4, 0x51, 0x3e, 0xec, 0xbb, 0x60, 0xa5, 0x38, 0xdb, 0x79, 0x39, 0xf0, 0x66, 0xbe, 0x65, - 0x2e, 0x56, 0xdb, 0x07, 0xc8, 0x42, 0xc3, 0xc3, 0x0a, 0x33, 0xe9, 0x20, 0xbb, 0xb5, 0xd3, 0x3a, - 0xdd, 0xa0, 0x8e, 0x72, 0x28, 0x94, 0x26, 0xa7, 0x32, 0xeb, 0x00, 0x4c, 0xd2, 0x0d, 0xfb, 0x8a, - 0x6d, 0x9c, 0x63, 0xd8, 0x23, 0xa8, 0x27, 0x41, 0xb0, 0x2d, 0x68, 0xc4, 0xa6, 0x67, 0xcc, 0x1b, - 0xb1, 0xbb, 0x0a, 0xcf, 0x43, 0x98, 0xff, 0x45, 0x4e, 0x70, 0x2a, 0x96, 0xf2, 0x3f, 0x8e, 0x08, - 0x37, 0x06, 0xfb, 0x31, 0x54, 0x08, 0xc0, 0x6d, 0x16, 0x2b, 0x27, 0x52, 0x26, 0x95, 0xd4, 0xa9, - 0x95, 0x8c, 0xa9, 0xdb, 0x6e, 0x19, 0x85, 0xc8, 0x35, 0x81, 0x7d, 0x8c, 0x09, 0xdc, 0xc4, 0xcc, - 0xe8, 0x55, 0x3c, 0x34, 0xdb, 0x3f, 0x83, 0x7a, 0x02, 0xe3, 0xc8, 0xef, 0x7b, 0x81, 0x30, 0x21, - 0x52, 0x19, 0x53, 0xf0, 0xde, 0x99, 0x13, 0x39, 0xae, 0x12, 0x3a, 0xd9, 0xa8, 0xf0, 0x0c, 0xb0, - 0x3f, 0x82, 0x46, 0x6e, 0xf7, 0xa0, 0xdc, 0x1e, 0xd1, 0x32, 0xea, 0x3d, 0xac, 0x2b, 0xf6, 0x9f, - 0xf0, 0x81, 0x90, 0xe4, 0x7c, 0x3f, 0x01, 0x38, 0x53, 0x2a, 0x7c, 0x42, 0x49, 0xa0, 0x99, 0x7b, - 0x0b, 0x11, 0x62, 0xb0, 0x4d, 0x68, 0x60, 0x25, 0x36, 0x76, 0xad, 0x77, 0x6a, 0x11, 0x6b, 0xc2, - 0x8f, 0xc1, 0x9a, 0xa6, 0xcd, 0x4b, 0x66, 0xe9, 0x92, 0xd6, 0x1f, 0x40, 0x3d, 0x90, 0xc6, 0xa6, - 0x73, 0xd2, 0x5a, 0x20, 0xd3, 0x76, 0x8e, 0xef, 0x1b, 0x5b, 0x45, 0xb7, 0x73, 0x7c, 0x9f, 0x8c, - 0xf6, 0x2d, 0xf8, 0xd1, 0x4b, 0x4f, 0x1d, 0xf6, 0x1e, 0x54, 0xa7, 0x9e, 0xaf, 0xe8, 0x46, 0xc0, - 0x1c, 0xd8, 0xd4, 0xec, 0x7f, 0x17, 0x00, 0xb2, 0x65, 0x47, 0x31, 0xe3, 0xd1, 0x8e, 0x9c, 0xa6, - 0x3e, 0xca, 0x7d, 0xa8, 0xcf, 0xcc, 0x21, 0x61, 0x16, 0xf4, 0xc6, 0xb2, 0x54, 0x3a, 0xc9, 0x19, - 0xa2, 0x8f, 0x8f, 0x3d, 0x73, 0x7c, 0xbc, 0xcd, 0x73, 0x24, 0xed, 0x81, 0xb2, 0x9c, 0xfc, 0xb3, - 0x12, 0xb2, 0x5d, 0xc8, 0x8d, 0x65, 0xfd, 0x2b, 0x58, 0x5d, 0xea, 0xf2, 0x0d, 0x2f, 0x8c, 0xec, - 0xb0, 0xcb, 0x6f, 0xc1, 0x3d, 0xa8, 0xea, 0xf7, 0x28, 0xdb, 0x86, 0x9a, 0xe3, 0xea, 0xdd, 0x97, - 0x3b, 0x01, 0xd0, 0xb8, 0x4f, 0x30, 0x4f, 0xcc, 0xf6, 0xdf, 0x8a, 0x00, 0x19, 0xfe, 0x16, 0xa9, - 0xee, 0x17, 0xb0, 0x16, 0x0b, 0x57, 0x06, 0x13, 0x27, 0x5a, 0x90, 0xd5, 0xbc, 0xbb, 0xae, 0x6a, - 0x72, 0x89, 0x99, 0x4b, 0x7b, 0x4b, 0xaf, 0x4f, 0x7b, 0xb7, 0xa1, 0xec, 0xca, 0x70, 0x61, 0xee, - 0x05, 0xb6, 0x3c, 0x90, 0x9e, 0x0c, 0x17, 0xf8, 0xfa, 0x46, 0x06, 0xeb, 0x40, 0x75, 0x76, 0x4e, - 0x2f, 0x74, 0xfd, 0xba, 0xb9, 0xbe, 0xcc, 0x7d, 0x70, 0x8e, 0x65, 0x7c, 0xcf, 0x6b, 0x16, 0xbb, - 0x05, 0x95, 0xd9, 0xf9, 0xc4, 0x8b, 0x28, 0x61, 0x6e, 0xe8, 0x44, 0x30, 0x4f, 0xef, 0x7b, 0x11, - 0xbe, 0xda, 0x89, 0xc3, 0x6c, 0x28, 0x46, 0x33, 0x7a, 0xe0, 0x34, 0xf4, 0xd3, 0x2d, 0x37, 0x9b, - 0xb3, 0x83, 0x15, 0x5e, 0x8c, 0x66, 0xdd, 0x3a, 0x54, 0xf5, 0xbc, 0xda, 0xff, 0x2a, 0xc1, 0xda, - 0x72, 0x94, 0xb8, 0xb2, 0x71, 0xe4, 0x26, 0x2b, 0x1b, 0x47, 0x6e, 0xfa, 0x22, 0x28, 0xe6, 0x5e, - 0x04, 0x36, 0x54, 0xe4, 0xb3, 0x40, 0x44, 0xf9, 0x5f, 0x11, 0xbd, 0x33, 0xf9, 0x2c, 0xc0, 0xac, - 0x54, 0x9b, 0x96, 0x92, 0xbc, 0x8a, 0x49, 0xf2, 0x3e, 0x86, 0xd5, 0xa9, 0xf4, 0x7d, 0xf9, 0x6c, - 0xbc, 0x98, 0xf9, 0x5e, 0x70, 0x6e, 0x32, 0xbd, 0x65, 0x90, 0x6d, 0xc3, 0xb5, 0x89, 0x17, 0x61, - 0x38, 0x3d, 0x19, 0x28, 0x11, 0xd0, 0xe3, 0x0e, 0x79, 0x97, 0x61, 0xf6, 0x25, 0x6c, 0x39, 0x4a, - 0x89, 0x59, 0xa8, 0x1e, 0x06, 0xa1, 0xe3, 0x9e, 0xf7, 0xa5, 0x4b, 0xbb, 0x70, 0x16, 0x3a, 0xca, - 0x3b, 0xf1, 0x7c, 0x7c, 0xc7, 0xd6, 0xa8, 0xe9, 0x6b, 0x79, 0xec, 0x13, 0x58, 0x73, 0x23, 0xe1, - 0x28, 0xd1, 0x17, 0xb1, 0x3a, 0x72, 0xd4, 0x59, 0xbb, 0x4e, 0x2d, 0x2f, 0xa1, 0x38, 0x06, 0x07, - 0xa3, 0x7d, 0xec, 0xf9, 0x13, 0x17, 0x9f, 0x63, 0x96, 0x1e, 0xc3, 0x12, 0xc8, 0x3a, 0xc0, 0x08, - 0x18, 0xcc, 0x42, 0xb5, 0x48, 0xa9, 0x40, 0xd4, 0x2b, 0x2c, 0x78, 0x4e, 0x2a, 0x6f, 0x26, 0x62, - 0xe5, 0xcc, 0x42, 0xfa, 0x85, 0x52, 0xe2, 0x19, 0xc0, 0x6e, 0x42, 0xcb, 0x0b, 0x5c, 0x7f, 0x3e, - 0x11, 0x4f, 0x42, 0x1c, 0x48, 0x14, 0xc4, 0xed, 0x26, 0x9d, 0x2a, 0xd7, 0x0c, 0x7e, 0x64, 0x60, - 0xa4, 0x8a, 0x8b, 0x4b, 0xd4, 0x55, 0x4d, 0x35, 0x78, 0x42, 0xb5, 0xbf, 0x29, 0x40, 0xeb, 0xb2, - 0xf0, 0x70, 0xd9, 0x42, 0x1c, 0xbc, 0x79, 0x8c, 0x62, 0x39, 0x5d, 0xca, 0x62, 0x6e, 0x29, 0x93, - 0x6b, 0xae, 0x94, 0xbb, 0xe6, 0x52, 0x59, 0x94, 0x5f, 0x2d, 0x8b, 0xa5, 0x81, 0x56, 0x2e, 0x0d, - 0xd4, 0xfe, 0x43, 0x01, 0xae, 0x5d, 0x12, 0xf7, 0x1b, 0x47, 0xb4, 0x05, 0x8d, 0x99, 0x73, 0x2e, - 0xf4, 0x63, 0x3c, 0xa6, 0xc0, 0xea, 0x3c, 0x0f, 0xfd, 0x0f, 0xe2, 0x0b, 0xa0, 0x99, 0xdf, 0x51, - 0x57, 0xc6, 0x96, 0x08, 0xe4, 0x50, 0xaa, 0x7b, 0x72, 0x6e, 0xae, 0xd0, 0x44, 0x20, 0x09, 0xf8, - 0xb2, 0x8c, 0x4a, 0x57, 0xc8, 0xc8, 0x3e, 0x84, 0x7a, 0x12, 0x20, 0xdb, 0x34, 0x7f, 0x4b, 0x0a, - 0xd9, 0x5f, 0xbb, 0x87, 0xb1, 0x88, 0x30, 0x76, 0xfd, 0xeb, 0xe4, 0x43, 0xa8, 0xd0, 0xef, 0x08, - 0x73, 0x06, 0x2f, 0x31, 0xb4, 0xc5, 0x1e, 0x43, 0xcd, 0x20, 0x6c, 0x07, 0xaa, 0x27, 0x8b, 0xf4, - 0xbf, 0x83, 0x39, 0x2e, 0xb0, 0x3e, 0x31, 0x0c, 0x3c, 0x83, 0x34, 0x83, 0x5d, 0x87, 0xf2, 0xc9, - 0x62, 0xd8, 0xd7, 0xaf, 0x3a, 0x3c, 0xc9, 0xb0, 0xd6, 0xad, 0xea, 0x80, 0xec, 0xfb, 0xd0, 0xcc, - 0xb7, 0xc3, 0x49, 0xc9, 0x65, 0x46, 0x54, 0xce, 0x8e, 0xec, 0xe2, 0x6b, 0x8e, 0xec, 0x9d, 0x6d, - 0xa8, 0x99, 0xff, 0x52, 0xcc, 0x82, 0xca, 0xc3, 0xc3, 0xf1, 0xe0, 0xb8, 0xb5, 0xc2, 0xea, 0x50, - 0x3e, 0x18, 0x8d, 0x8f, 0x5b, 0x05, 0x2c, 0x1d, 0x8e, 0x0e, 0x07, 0xad, 0xe2, 0xce, 0x4d, 0x68, - 0xe6, 0xff, 0x4c, 0xb1, 0x06, 0xd4, 0xc6, 0xfb, 0x87, 0xfd, 0xee, 0xe8, 0x97, 0xad, 0x15, 0xd6, - 0x84, 0xfa, 0xf0, 0x70, 0x3c, 0xe8, 0x3d, 0xe4, 0x83, 0x56, 0x61, 0xe7, 0x17, 0x60, 0xa5, 0x3f, - 0x19, 0xd0, 0x43, 0x77, 0x78, 0xd8, 0x6f, 0xad, 0x30, 0x80, 0xea, 0x78, 0xd0, 0xe3, 0x03, 0xf4, - 0x5b, 0x83, 0xd2, 0x78, 0x7c, 0xd0, 0x2a, 0x62, 0xaf, 0xbd, 0xfd, 0xde, 0xc1, 0xa0, 0x55, 0xc2, - 0xe2, 0xf1, 0x83, 0xa3, 0x7b, 0xe3, 0x56, 0x79, 0xe7, 0x33, 0xb8, 0x76, 0xe9, 0xf9, 0x4d, 0xad, - 0x0f, 0xf6, 0xf9, 0x00, 0x3d, 0x35, 0xa0, 0x76, 0xc4, 0x87, 0x8f, 0xf6, 0x8f, 0x07, 0xad, 0x02, - 0x1a, 0xee, 0x8f, 0x7a, 0x5f, 0x0d, 0xfa, 0xad, 0x62, 0xf7, 0xc6, 0xb7, 0xcf, 0x37, 0x0a, 0xdf, - 0x3d, 0xdf, 0x28, 0x7c, 0xff, 0x7c, 0xa3, 0xf0, 0xcf, 0xe7, 0x1b, 0x85, 0x6f, 0x5e, 0x6c, 0xac, - 0x7c, 0xf7, 0x62, 0x63, 0xe5, 0xfb, 0x17, 0x1b, 0x2b, 0x27, 0x55, 0xfa, 0x4f, 0xfc, 0xe9, 0x7f, - 0x02, 0x00, 0x00, 0xff, 0xff, 0x69, 0xc4, 0xf1, 0xa8, 0x67, 0x16, 0x00, 0x00, + // 2387 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xcf, 0x6f, 0x1b, 0xc7, + 0xf5, 0x17, 0x7f, 0x93, 0x8f, 0x14, 0xcd, 0xef, 0xd8, 0x49, 0x18, 0x7d, 0x5d, 0x49, 0xd9, 0xa4, + 0x81, 0x2c, 0xdb, 0x14, 0xaa, 0x00, 0x71, 0x10, 0x14, 0x45, 0xc5, 0x1f, 0x86, 0x98, 0xd8, 0xa2, + 0x30, 0x94, 0x9d, 0xde, 0x8c, 0xd5, 0x72, 0x48, 0x2d, 0xb4, 0xdc, 0x59, 0xec, 0x0e, 0x63, 0xb1, + 0x87, 0x1e, 0x72, 0x2f, 0x10, 0xa0, 0x40, 0xd1, 0x4b, 0xdb, 0x7f, 0xa2, 0xd7, 0x5e, 0x8b, 0x1c, + 0x73, 0xe8, 0x21, 0xe8, 0x21, 0x2d, 0x9c, 0x7f, 0xa3, 0x05, 0x8a, 0xf7, 0x66, 0xf6, 0x07, 0x69, + 0x05, 0x8e, 0xd1, 0xa2, 0xa7, 0x9d, 0xf9, 0xbc, 0xcf, 0xbc, 0x79, 0x33, 0xef, 0xcd, 0xec, 0x7b, + 0x03, 0x35, 0x19, 0x44, 0x9d, 0x20, 0x94, 0x4a, 0xb2, 0x7c, 0x70, 0xbe, 0x75, 0x7f, 0xe6, 0xaa, + 0x8b, 0xc5, 0x79, 0xc7, 0x91, 0xf3, 0x83, 0x99, 0x9c, 0xc9, 0x03, 0x12, 0x9d, 0x2f, 0xa6, 0xd4, + 0xa3, 0x0e, 0xb5, 0xf4, 0x10, 0xeb, 0x2f, 0x79, 0xc8, 0x8f, 0x02, 0xf6, 0x0e, 0x94, 0x5d, 0x3f, + 0x58, 0xa8, 0xa8, 0x9d, 0xdb, 0x2d, 0xec, 0xd5, 0x0f, 0x6b, 0x9d, 0xe0, 0xbc, 0x33, 0x44, 0x84, + 0x1b, 0x01, 0xdb, 0x85, 0xa2, 0xb8, 0x12, 0x4e, 0x3b, 0xbf, 0x9b, 0xdb, 0xab, 0x1f, 0x02, 0x12, + 0x06, 0x57, 0xc2, 0x19, 0x05, 0xc7, 0x1b, 0x9c, 0x24, 0xec, 0x7d, 0x28, 0x47, 0x72, 0x11, 0x3a, + 0xa2, 0x5d, 0x20, 0x4e, 0x03, 0x39, 0x63, 0x42, 0x88, 0x65, 0xa4, 0xa8, 0x69, 0xea, 0x7a, 0xa2, + 0x5d, 0x4c, 0x35, 0x3d, 0x74, 0x3d, 0xcd, 0x21, 0x09, 0x7b, 0x17, 0x4a, 0xe7, 0x0b, 0xd7, 0x9b, + 0xb4, 0x4b, 0x44, 0xa9, 0x23, 0xa5, 0x8b, 0x00, 0x71, 0xb4, 0x0c, 0x49, 0x73, 0x11, 0xce, 0x44, + 0xbb, 0x9c, 0x92, 0x1e, 0x23, 0xa0, 0x49, 0x24, 0x63, 0x7b, 0x50, 0x0d, 0x3c, 0x5b, 0x4d, 0x65, + 0x38, 0x6f, 0x43, 0x6a, 0xd5, 0xa9, 0xc1, 0x78, 0x22, 0x65, 0x0f, 0xa0, 0xee, 0x48, 0x3f, 0x52, + 0xa1, 0xed, 0xfa, 0x2a, 0x6a, 0xd7, 0x89, 0xfc, 0x06, 0x92, 0x3f, 0x93, 0xe1, 0xa5, 0x08, 0x7b, + 0xa9, 0x90, 0x67, 0x99, 0xdd, 0x22, 0xe4, 0x65, 0x60, 0xfd, 0x36, 0x07, 0xd5, 0x58, 0x2b, 0xb3, + 0xa0, 0x71, 0x14, 0x3a, 0x17, 0xae, 0x12, 0x8e, 0x5a, 0x84, 0xa2, 0x9d, 0xdb, 0xcd, 0xed, 0xd5, + 0xf8, 0x0a, 0xc6, 0x9a, 0x90, 0x1f, 0x8d, 0x69, 0x37, 0x6b, 0x3c, 0x3f, 0x1a, 0xb3, 0x36, 0x54, + 0x9e, 0xda, 0xa1, 0x6b, 0xfb, 0x8a, 0xb6, 0xaf, 0xc6, 0xe3, 0x2e, 0xbb, 0x0d, 0xb5, 0xd1, 0xf8, + 0xa9, 0x08, 0x23, 0x57, 0xfa, 0xb4, 0x69, 0x35, 0x9e, 0x02, 0x6c, 0x1b, 0x60, 0x34, 0x7e, 0x28, + 0x6c, 0x54, 0x1a, 0xb5, 0x4b, 0xbb, 0x85, 0xbd, 0x1a, 0xcf, 0x20, 0xd6, 0xaf, 0xa0, 0x44, 0x8e, + 0x64, 0x9f, 0x40, 0x79, 0xe2, 0xce, 0x44, 0xa4, 0xb4, 0x39, 0xdd, 0xc3, 0xaf, 0xbe, 0xdd, 0xd9, + 0xf8, 0xdb, 0xb7, 0x3b, 0xfb, 0x99, 0x88, 0x91, 0x81, 0xf0, 0x1d, 0xe9, 0x2b, 0xdb, 0xf5, 0x45, + 0x18, 0x1d, 0xcc, 0xe4, 0x7d, 0x3d, 0xa4, 0xd3, 0xa7, 0x0f, 0x37, 0x1a, 0xd8, 0x1d, 0x28, 0xb9, + 0xfe, 0x44, 0x5c, 0x91, 0xfd, 0x85, 0xee, 0x4d, 0xa3, 0xaa, 0x3e, 0x5a, 0xa8, 0x60, 0xa1, 0x86, + 0x28, 0xe2, 0x9a, 0x61, 0xfd, 0x21, 0x07, 0x65, 0x1d, 0x28, 0xec, 0x36, 0x14, 0xe7, 0x42, 0xd9, + 0x34, 0x7f, 0xfd, 0xb0, 0xaa, 0x1d, 0xa6, 0x6c, 0x4e, 0x28, 0xc6, 0xe0, 0x5c, 0x2e, 0x70, 0xef, + 0xf3, 0x69, 0x0c, 0x3e, 0x46, 0x84, 0x1b, 0x01, 0xfb, 0x31, 0x54, 0x7c, 0xa1, 0x9e, 0xcb, 0xf0, + 0x92, 0xf6, 0xa8, 0xa9, 0x9d, 0x7e, 0x22, 0xd4, 0x63, 0x39, 0x11, 0x3c, 0x96, 0xb1, 0x7b, 0x50, + 0x8d, 0x84, 0xb3, 0x08, 0x5d, 0xb5, 0xa4, 0xfd, 0x6a, 0x1e, 0xb6, 0x28, 0x14, 0x0d, 0x46, 0xe4, + 0x84, 0x61, 0xfd, 0x3a, 0x0f, 0x45, 0x34, 0x83, 0x31, 0x28, 0xda, 0xe1, 0x4c, 0x1f, 0x81, 0x1a, + 0xa7, 0x36, 0x6b, 0x41, 0x41, 0xf8, 0x9f, 0x93, 0x45, 0x35, 0x8e, 0x4d, 0x44, 0x9c, 0xe7, 0x13, + 0xe3, 0x23, 0x6c, 0xe2, 0xb8, 0x45, 0x24, 0x42, 0xe3, 0x1a, 0x6a, 0xb3, 0x3b, 0x50, 0x0b, 0x42, + 0x79, 0xb5, 0x7c, 0x86, 0xa3, 0x4b, 0x99, 0xc0, 0x43, 0x70, 0xe0, 0x7f, 0xce, 0xab, 0x81, 0x69, + 0xb1, 0x7d, 0x00, 0x71, 0xa5, 0x42, 0xfb, 0x58, 0x46, 0x2a, 0x6a, 0x97, 0x69, 0xed, 0x74, 0x28, + 0x10, 0x18, 0x9e, 0xf2, 0x8c, 0x94, 0x6d, 0x41, 0xf5, 0x42, 0x46, 0xca, 0xb7, 0xe7, 0xa2, 0x5d, + 0xa1, 0xe9, 0x92, 0x3e, 0xb3, 0xa0, 0xbc, 0xf0, 0xdc, 0xb9, 0xab, 0xda, 0xb5, 0x54, 0xc7, 0x13, + 0x42, 0xb8, 0x91, 0x60, 0x60, 0x3a, 0xb3, 0x50, 0x2e, 0x82, 0x53, 0x3b, 0x14, 0xbe, 0xa2, 0x23, + 0x51, 0xe3, 0x2b, 0x98, 0x75, 0x0f, 0xca, 0x7a, 0x66, 0x5c, 0x18, 0xb6, 0x4c, 0xf8, 0x52, 0x1b, + 0xc3, 0x76, 0x78, 0x1a, 0x87, 0xed, 0xf0, 0xd4, 0xea, 0x43, 0x59, 0xcf, 0x81, 0xec, 0x13, 0xb4, + 0xcb, 0xb0, 0xb1, 0x8d, 0xd8, 0x58, 0x4e, 0x95, 0x0e, 0x13, 0x4e, 0x6d, 0xd2, 0x6a, 0x87, 0x7a, + 0x07, 0x0b, 0x9c, 0xda, 0xd6, 0xef, 0x0a, 0x50, 0x22, 0x57, 0xb3, 0x3d, 0x8c, 0xac, 0x60, 0xa1, + 0x27, 0x2d, 0x74, 0x99, 0x89, 0x2c, 0xa0, 0x18, 0x4e, 0x02, 0x0b, 0xe3, 0x79, 0x0b, 0xbd, 0xec, + 0x09, 0x47, 0xc9, 0xd0, 0xd8, 0x93, 0xf4, 0x71, 0x8e, 0x09, 0x46, 0xba, 0xf6, 0x12, 0xb5, 0xd9, + 0x5d, 0x28, 0x4b, 0x0a, 0x4f, 0x72, 0xd4, 0xf7, 0x04, 0xad, 0xa1, 0xa0, 0xf2, 0x50, 0xd8, 0x13, + 0xe9, 0x7b, 0x4b, 0x72, 0x5f, 0x95, 0x27, 0x7d, 0x76, 0x17, 0x6a, 0x14, 0x8f, 0x67, 0xcb, 0x40, + 0x5f, 0x3e, 0xcd, 0xc3, 0xcd, 0x24, 0x56, 0x11, 0xe4, 0xa9, 0x1c, 0x2f, 0xa0, 0xb3, 0x79, 0x30, + 0x8d, 0x46, 0x81, 0x6a, 0xdf, 0x4c, 0xe3, 0x20, 0xc6, 0x78, 0x22, 0x45, 0xa6, 0x63, 0x3b, 0x17, + 0x02, 0x99, 0xb7, 0x52, 0x66, 0xcf, 0x60, 0x3c, 0x91, 0xa2, 0x01, 0x91, 0x70, 0x42, 0xa1, 0x90, + 0xfa, 0x06, 0x51, 0x37, 0x4d, 0x80, 0x6b, 0x90, 0xa7, 0x72, 0x0c, 0x8b, 0xf1, 0xf8, 0x18, 0x99, + 0x6f, 0xa6, 0xf7, 0xad, 0x46, 0xb8, 0x91, 0xe8, 0xd5, 0x46, 0x0b, 0x4f, 0x0d, 0xfb, 0xed, 0xb7, + 0xf4, 0x56, 0xc6, 0x7d, 0x6b, 0x3b, 0x5d, 0x00, 0x6e, 0x6b, 0xe4, 0xfe, 0x52, 0xbb, 0xb8, 0xc0, + 0xa9, 0x6d, 0x0d, 0xa1, 0x1a, 0x9b, 0x48, 0xc1, 0xd1, 0x37, 0x01, 0x90, 0x1f, 0xf6, 0xd9, 0x7d, + 0xa8, 0x44, 0x17, 0x76, 0xe8, 0xfa, 0x33, 0xf2, 0x50, 0xf3, 0xf0, 0x66, 0xb2, 0xa2, 0xb1, 0xc6, + 0xd1, 0x8a, 0x98, 0x63, 0x49, 0xa8, 0x25, 0x4b, 0x78, 0x49, 0x57, 0x0b, 0x0a, 0x0b, 0x77, 0x42, + 0x7a, 0x36, 0x39, 0x36, 0x11, 0x99, 0xb9, 0x3a, 0x8e, 0x36, 0x39, 0x36, 0xd1, 0xbe, 0xb9, 0x9c, + 0xe8, 0x3f, 0xcb, 0x26, 0xa7, 0x36, 0xae, 0x4d, 0x06, 0xca, 0x95, 0xbe, 0xed, 0xc5, 0x9e, 0x8c, + 0xfb, 0x96, 0x17, 0xef, 0xcd, 0xff, 0x64, 0xb6, 0xdf, 0xe4, 0xa0, 0x1a, 0xff, 0x0e, 0xf1, 0xda, + 0x76, 0x27, 0xc2, 0x57, 0xee, 0xd4, 0x15, 0xa1, 0x99, 0x38, 0x83, 0xb0, 0xfb, 0x50, 0xb2, 0x95, + 0x0a, 0xe3, 0xcb, 0xf0, 0xad, 0xec, 0xbf, 0xb4, 0x73, 0x84, 0x92, 0x81, 0xaf, 0xc2, 0x25, 0xd7, + 0xac, 0xad, 0x8f, 0x00, 0x52, 0x10, 0x6d, 0xbd, 0x14, 0x4b, 0xa3, 0x15, 0x9b, 0xec, 0x16, 0x94, + 0x3e, 0xb7, 0xbd, 0x85, 0x30, 0x27, 0x45, 0x77, 0x3e, 0xce, 0x7f, 0x94, 0xb3, 0xfe, 0x9c, 0x87, + 0x8a, 0xf9, 0xb7, 0xb2, 0x7b, 0x50, 0xa1, 0x7f, 0xab, 0xb1, 0xe8, 0xfa, 0xe3, 0x17, 0x53, 0xd8, + 0x41, 0x92, 0x34, 0x64, 0x6c, 0x34, 0xaa, 0x74, 0xf2, 0x60, 0x6c, 0x4c, 0x53, 0x88, 0xc2, 0x44, + 0x4c, 0x4d, 0x76, 0xd0, 0x44, 0x76, 0x5f, 0x4c, 0x5d, 0xdf, 0xc5, 0xfd, 0xe1, 0x28, 0x62, 0xf7, + 0xe2, 0x55, 0x17, 0x49, 0xe3, 0x9b, 0x59, 0x8d, 0x2f, 0x2f, 0x7a, 0x08, 0xf5, 0xcc, 0x34, 0xd7, + 0xac, 0xfa, 0xbd, 0xec, 0xaa, 0xcd, 0x94, 0xa4, 0x4e, 0xa7, 0x36, 0xe9, 0x2e, 0xfc, 0x07, 0xfb, + 0xf7, 0x21, 0x40, 0xaa, 0xf2, 0x87, 0x5f, 0x5f, 0xd6, 0x17, 0x05, 0x80, 0x51, 0x80, 0x3f, 0x9e, + 0x89, 0x4d, 0x7f, 0xbf, 0x86, 0x3b, 0xf3, 0x65, 0x28, 0x9e, 0xd1, 0x31, 0xa7, 0xf1, 0x55, 0x5e, + 0xd7, 0x18, 0x9d, 0x18, 0x76, 0x04, 0xf5, 0x89, 0x88, 0x9c, 0xd0, 0xa5, 0x80, 0x32, 0x9b, 0xbe, + 0x83, 0x6b, 0x4a, 0xf5, 0x74, 0xfa, 0x29, 0x43, 0xef, 0x55, 0x76, 0x0c, 0x3b, 0x84, 0x86, 0xb8, + 0x0a, 0x64, 0xa8, 0xcc, 0x2c, 0x3a, 0x05, 0xbb, 0xa1, 0x93, 0x39, 0xc4, 0x69, 0x26, 0x5e, 0x17, + 0x69, 0x87, 0xd9, 0x50, 0x74, 0xec, 0x40, 0xa7, 0x16, 0xf5, 0xc3, 0xf6, 0xda, 0x7c, 0x3d, 0x3b, + 0xd0, 0x9b, 0xd6, 0xfd, 0x00, 0xd7, 0xfa, 0xc5, 0xdf, 0x77, 0xee, 0x66, 0xf2, 0x89, 0xb9, 0x3c, + 0x5f, 0x1e, 0x50, 0xbc, 0x5c, 0xba, 0xea, 0x60, 0xa1, 0x5c, 0xef, 0xc0, 0x0e, 0x5c, 0x54, 0x87, + 0x03, 0x87, 0x7d, 0x4e, 0xaa, 0xb7, 0x7e, 0x06, 0xad, 0x75, 0xbb, 0x5f, 0xc7, 0x07, 0x5b, 0x0f, + 0xa0, 0x96, 0xd8, 0xf1, 0xaa, 0x81, 0xd5, 0xac, 0xf3, 0xfe, 0x94, 0x83, 0xb2, 0x3e, 0x55, 0xec, + 0x01, 0xd4, 0x3c, 0xe9, 0xd8, 0x68, 0x40, 0x9c, 0x05, 0xbf, 0x9d, 0x1e, 0xba, 0xce, 0xa3, 0x58, + 0xa6, 0x77, 0x35, 0xe5, 0x62, 0x90, 0xb9, 0xfe, 0x54, 0xc6, 0xa7, 0xa0, 0x99, 0x0e, 0x1a, 0xfa, + 0x53, 0xc9, 0xb5, 0x70, 0xeb, 0x53, 0x68, 0xae, 0xaa, 0xb8, 0xc6, 0xce, 0x77, 0x57, 0xc3, 0x95, + 0xee, 0xf4, 0x64, 0x50, 0xd6, 0xec, 0x07, 0x50, 0x4b, 0x70, 0xb6, 0xff, 0xb2, 0xe1, 0x8d, 0xec, + 0xc8, 0x8c, 0xad, 0x96, 0x07, 0x90, 0x9a, 0x86, 0x97, 0x15, 0xa6, 0xdb, 0x7e, 0xfa, 0xd7, 0x4e, + 0xfa, 0xf4, 0x07, 0xb5, 0x95, 0x4d, 0xa6, 0x34, 0x38, 0xb5, 0x59, 0x07, 0x60, 0x92, 0x1c, 0xd8, + 0xef, 0x39, 0xc6, 0x19, 0x86, 0x35, 0x82, 0x6a, 0x6c, 0x04, 0xdb, 0x85, 0x7a, 0x64, 0x66, 0xc6, + 0xbc, 0x11, 0xa7, 0x2b, 0xf1, 0x2c, 0x84, 0xf9, 0x5f, 0x68, 0xfb, 0x33, 0xb1, 0x92, 0xff, 0x71, + 0x44, 0xb8, 0x11, 0x58, 0x9f, 0x41, 0x89, 0x00, 0x3c, 0x66, 0x91, 0xb2, 0x43, 0x65, 0x52, 0x49, + 0x9d, 0x5a, 0xc9, 0x88, 0xa6, 0xed, 0x16, 0x31, 0x10, 0xb9, 0x26, 0xb0, 0xf7, 0x30, 0x81, 0x9b, + 0x98, 0x1d, 0xbd, 0x8e, 0x87, 0x62, 0xeb, 0xa7, 0x50, 0x8d, 0x61, 0x5c, 0xf9, 0x23, 0xd7, 0x17, + 0xc6, 0x44, 0x6a, 0x63, 0x0a, 0xde, 0xbb, 0xb0, 0x43, 0xdb, 0x51, 0x42, 0x27, 0x1b, 0x25, 0x9e, + 0x02, 0xd6, 0xbb, 0x50, 0xcf, 0x9c, 0x1e, 0x0c, 0xb7, 0xa7, 0xe4, 0x46, 0x7d, 0x86, 0x75, 0xc7, + 0xfa, 0x23, 0x16, 0x08, 0x71, 0xce, 0xf7, 0x23, 0x80, 0x0b, 0xa5, 0x82, 0x67, 0x94, 0x04, 0x9a, + 0xbd, 0xaf, 0x21, 0x42, 0x0c, 0xb6, 0x03, 0x75, 0xec, 0x44, 0x46, 0xae, 0xe3, 0x9d, 0x46, 0x44, + 0x9a, 0xf0, 0xff, 0x50, 0x9b, 0x26, 0xc3, 0x0b, 0xc6, 0x75, 0xf1, 0xe8, 0xb7, 0xa1, 0xea, 0x4b, + 0x23, 0xd3, 0x39, 0x69, 0xc5, 0x97, 0xc9, 0x38, 0xdb, 0xf3, 0x8c, 0xac, 0xa4, 0xc7, 0xd9, 0x9e, + 0x47, 0x42, 0xeb, 0x2e, 0xfc, 0xdf, 0x4b, 0xa5, 0x0e, 0x7b, 0x13, 0xca, 0x53, 0xd7, 0x53, 0xf4, + 0x47, 0xc0, 0x1c, 0xd8, 0xf4, 0xac, 0x7f, 0xe5, 0x00, 0x52, 0xb7, 0x63, 0x30, 0xe3, 0xd5, 0x8e, + 0x9c, 0x86, 0xbe, 0xca, 0x3d, 0xa8, 0xce, 0xcd, 0x25, 0x61, 0x1c, 0x7a, 0x7b, 0x35, 0x54, 0x3a, + 0xf1, 0x1d, 0xa2, 0xaf, 0x8f, 0x43, 0x73, 0x7d, 0xbc, 0x4e, 0x39, 0x92, 0xcc, 0x40, 0x59, 0x4e, + 0xb6, 0xf6, 0x84, 0xf4, 0x14, 0x72, 0x23, 0xd9, 0xfa, 0x14, 0x36, 0x57, 0xa6, 0xfc, 0x81, 0x3f, + 0x8c, 0xf4, 0xb2, 0xcb, 0x1e, 0xc1, 0x43, 0x28, 0xeb, 0xa2, 0x95, 0xed, 0x41, 0xc5, 0x76, 0xf4, + 0xe9, 0xcb, 0xdc, 0x00, 0x28, 0x3c, 0x22, 0x98, 0xc7, 0x62, 0xeb, 0xaf, 0x79, 0x80, 0x14, 0x7f, + 0x8d, 0x54, 0xf7, 0x63, 0x68, 0x46, 0xc2, 0x91, 0xfe, 0xc4, 0x0e, 0x97, 0x24, 0x35, 0x75, 0xd7, + 0x75, 0x43, 0xd6, 0x98, 0x99, 0xb4, 0xb7, 0xf0, 0xea, 0xb4, 0x77, 0x0f, 0x8a, 0x8e, 0x0c, 0x96, + 0xe6, 0xbf, 0xc0, 0x56, 0x17, 0xd2, 0x93, 0xc1, 0x12, 0x4b, 0x74, 0x64, 0xb0, 0x0e, 0x94, 0xe7, + 0x97, 0x54, 0xc6, 0xeb, 0xea, 0xe6, 0xd6, 0x2a, 0xf7, 0xf1, 0x25, 0xb6, 0xb1, 0xe8, 0xd7, 0x2c, + 0x76, 0x17, 0x4a, 0xf3, 0xcb, 0x89, 0x1b, 0x9a, 0x6a, 0xfd, 0xe6, 0x3a, 0xbd, 0xef, 0x86, 0x54, + 0xb5, 0x23, 0x87, 0x59, 0x90, 0x0f, 0xe7, 0x54, 0xe0, 0xd4, 0x75, 0xe9, 0x96, 0xd9, 0xcd, 0xf9, + 0xf1, 0x06, 0xcf, 0x87, 0xf3, 0x6e, 0x15, 0xca, 0x7a, 0x5f, 0xad, 0x7f, 0x16, 0xa0, 0xb9, 0x6a, + 0x25, 0x7a, 0x36, 0x0a, 0x9d, 0xd8, 0xb3, 0x51, 0xe8, 0x24, 0x15, 0x41, 0x3e, 0x53, 0x11, 0x58, + 0x50, 0x92, 0xcf, 0x7d, 0x11, 0x66, 0xdf, 0x2b, 0x7a, 0x17, 0xf2, 0xb9, 0x8f, 0x59, 0xa9, 0x16, + 0xad, 0x24, 0x79, 0x25, 0x93, 0xe4, 0xbd, 0x07, 0x9b, 0x53, 0xe9, 0x79, 0xf2, 0xf9, 0x78, 0x39, + 0xf7, 0x5c, 0xff, 0xd2, 0x64, 0x7a, 0xab, 0x20, 0xdb, 0x83, 0x1b, 0x13, 0x37, 0x44, 0x73, 0x7a, + 0xd2, 0x57, 0xc2, 0xa7, 0xe2, 0x0e, 0x79, 0xeb, 0x30, 0xfb, 0x04, 0x76, 0x6d, 0xa5, 0xc4, 0x3c, + 0x50, 0x4f, 0xfc, 0xc0, 0x76, 0x2e, 0xfb, 0xd2, 0xa1, 0x53, 0x38, 0x0f, 0x6c, 0xe5, 0x9e, 0xbb, + 0x1e, 0xd6, 0xb1, 0x15, 0x1a, 0xfa, 0x4a, 0x1e, 0x7b, 0x1f, 0x9a, 0x4e, 0x28, 0x6c, 0x25, 0xfa, + 0x22, 0x52, 0xa7, 0xb6, 0xba, 0x68, 0x57, 0x69, 0xe4, 0x1a, 0x8a, 0x6b, 0xb0, 0xd1, 0xda, 0xcf, + 0x5c, 0x6f, 0xe2, 0x60, 0x39, 0x56, 0xd3, 0x6b, 0x58, 0x01, 0x59, 0x07, 0x18, 0x01, 0x83, 0x79, + 0xa0, 0x96, 0x09, 0x15, 0x88, 0x7a, 0x8d, 0x04, 0xef, 0x49, 0xe5, 0xce, 0x45, 0xa4, 0xec, 0x79, + 0x40, 0x4f, 0x28, 0x05, 0x9e, 0x02, 0xec, 0x0e, 0xb4, 0x5c, 0xdf, 0xf1, 0x16, 0x13, 0xf1, 0x2c, + 0xc0, 0x85, 0x84, 0x7e, 0xd4, 0x6e, 0xd0, 0xad, 0x72, 0xc3, 0xe0, 0xa7, 0x06, 0x46, 0xaa, 0xb8, + 0x5a, 0xa3, 0x6e, 0x6a, 0xaa, 0xc1, 0x63, 0xaa, 0xf5, 0x65, 0x0e, 0x5a, 0xeb, 0x81, 0x87, 0x6e, + 0x0b, 0x70, 0xf1, 0xa6, 0x18, 0xc5, 0x76, 0xe2, 0xca, 0x7c, 0xc6, 0x95, 0xf1, 0x6f, 0xae, 0x90, + 0xf9, 0xcd, 0x25, 0x61, 0x51, 0xfc, 0xfe, 0xb0, 0x58, 0x59, 0x68, 0x69, 0x6d, 0xa1, 0xd6, 0xef, + 0x73, 0x70, 0x63, 0x2d, 0xb8, 0x7f, 0xb0, 0x45, 0xbb, 0x50, 0x9f, 0xdb, 0x97, 0x42, 0x17, 0xe3, + 0x11, 0x19, 0x56, 0xe5, 0x59, 0xe8, 0xbf, 0x60, 0x9f, 0x0f, 0x8d, 0xec, 0x89, 0xba, 0xd6, 0xb6, + 0x38, 0x40, 0x4e, 0xa4, 0x7a, 0x28, 0x17, 0xe6, 0x17, 0x1a, 0x07, 0x48, 0x0c, 0xbe, 0x1c, 0x46, + 0x85, 0x6b, 0xc2, 0xc8, 0x3a, 0x81, 0x6a, 0x6c, 0x20, 0xdb, 0x31, 0xaf, 0x25, 0xb9, 0xf4, 0xd5, + 0xee, 0x49, 0x24, 0x42, 0xb4, 0x5d, 0x3f, 0x9d, 0xbc, 0x03, 0x25, 0x7a, 0x8e, 0x30, 0x77, 0xf0, + 0x0a, 0x43, 0x4b, 0xac, 0x31, 0x54, 0x0c, 0xc2, 0xf6, 0xa1, 0x7c, 0xbe, 0x4c, 0xde, 0x1d, 0xcc, + 0x75, 0x81, 0xfd, 0x89, 0x61, 0xe0, 0x1d, 0xa4, 0x19, 0xec, 0x16, 0x14, 0xcf, 0x97, 0xc3, 0xbe, + 0xae, 0xea, 0xf0, 0x26, 0xc3, 0x5e, 0xb7, 0xac, 0x0d, 0xb2, 0x1e, 0x41, 0x23, 0x3b, 0x0e, 0x37, + 0x25, 0x93, 0x19, 0x51, 0x3b, 0xbd, 0xb2, 0xf3, 0xaf, 0x4a, 0xef, 0x3f, 0x04, 0xa0, 0xc7, 0xc8, + 0xd7, 0x2d, 0x0b, 0x7e, 0x02, 0x15, 0xf3, 0x88, 0xc9, 0xde, 0x5f, 0x7b, 0x94, 0x6d, 0x26, 0x2f, + 0x9c, 0x2b, 0x2f, 0xb3, 0xfb, 0x7b, 0x50, 0x31, 0x4f, 0x60, 0xac, 0x06, 0xa5, 0x27, 0x27, 0xe3, + 0xc1, 0x59, 0x6b, 0x83, 0x55, 0xa1, 0x78, 0x3c, 0x1a, 0x9f, 0xb5, 0x72, 0xd8, 0x3a, 0x19, 0x9d, + 0x0c, 0x5a, 0xf9, 0xfd, 0x3b, 0xd0, 0xc8, 0x3e, 0x82, 0xb1, 0x3a, 0x54, 0xc6, 0x47, 0x27, 0xfd, + 0xee, 0xe8, 0x17, 0xad, 0x0d, 0xd6, 0x80, 0xea, 0xf0, 0x64, 0x3c, 0xe8, 0x3d, 0xe1, 0x83, 0x56, + 0x6e, 0xff, 0xe7, 0x50, 0x4b, 0xde, 0x33, 0x50, 0x43, 0x77, 0x78, 0xd2, 0x6f, 0x6d, 0x30, 0x80, + 0xf2, 0x78, 0xd0, 0xe3, 0x03, 0xd4, 0x5b, 0x81, 0xc2, 0x78, 0x7c, 0xdc, 0xca, 0xe3, 0xac, 0xbd, + 0xa3, 0xde, 0xf1, 0xa0, 0x55, 0xc0, 0xe6, 0xd9, 0xe3, 0xd3, 0x87, 0xe3, 0x56, 0x71, 0xff, 0x43, + 0xb8, 0xb1, 0x56, 0xe9, 0xd3, 0xe8, 0xe3, 0x23, 0x3e, 0x40, 0x4d, 0x75, 0xa8, 0x9c, 0xf2, 0xe1, + 0xd3, 0xa3, 0xb3, 0x41, 0x2b, 0x87, 0x82, 0x47, 0xa3, 0xde, 0xa7, 0x83, 0x7e, 0x2b, 0xdf, 0xbd, + 0xfd, 0xd5, 0x8b, 0xed, 0xdc, 0xd7, 0x2f, 0xb6, 0x73, 0xdf, 0xbc, 0xd8, 0xce, 0xfd, 0xe3, 0xc5, + 0x76, 0xee, 0xcb, 0xef, 0xb6, 0x37, 0xbe, 0xfe, 0x6e, 0x7b, 0xe3, 0x9b, 0xef, 0xb6, 0x37, 0xce, + 0xcb, 0xf4, 0x6e, 0xfd, 0xc1, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf6, 0x06, 0xb0, 0x47, 0xf7, + 0x16, 0x00, 0x00, } func (m *Op) Marshal() (dAtA []byte, err error) { @@ -2786,6 +2877,27 @@ func (m *Op_Build) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *Op_Merge) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Op_Merge) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Merge != nil { + { + size, err := m.Merge.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintOps(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} func (m *Platform) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4790,6 +4902,71 @@ func (m *NamedUserOpt) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *MergeInput) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MergeInput) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MergeInput) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Input != 0 { + i = encodeVarintOps(dAtA, i, uint64(m.Input)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *MergeOp) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MergeOp) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MergeOp) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Inputs) > 0 { + for iNdEx := len(m.Inputs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Inputs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintOps(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func encodeVarintOps(dAtA []byte, offset int, v uint64) int { offset -= sovOps(v) base := offset @@ -4875,6 +5052,18 @@ func (m *Op_Build) Size() (n int) { } return n } +func (m *Op_Merge) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Merge != nil { + l = m.Merge.Size() + n += 1 + l + sovOps(uint64(l)) + } + return n +} func (m *Platform) Size() (n int) { if m == nil { return 0 @@ -5738,6 +5927,33 @@ func (m *NamedUserOpt) Size() (n int) { return n } +func (m *MergeInput) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Input != 0 { + n += 1 + sovOps(uint64(m.Input)) + } + return n +} + +func (m *MergeOp) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Inputs) > 0 { + for _, e := range m.Inputs { + l = e.Size() + n += 1 + l + sovOps(uint64(l)) + } + } + return n +} + func sovOps(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -5947,6 +6163,41 @@ func (m *Op) Unmarshal(dAtA []byte) error { } m.Op = &Op_Build{v} iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Merge", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOps + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthOps + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthOps + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &MergeOp{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Op = &Op_Merge{v} + iNdEx = postIndex case 10: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Platform", wireType) @@ -11815,6 +12066,159 @@ func (m *NamedUserOpt) Unmarshal(dAtA []byte) error { } return nil } +func (m *MergeInput) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOps + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MergeInput: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MergeInput: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Input", wireType) + } + m.Input = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOps + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Input |= InputIndex(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipOps(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthOps + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MergeOp) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOps + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MergeOp: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MergeOp: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Inputs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOps + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthOps + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthOps + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Inputs = append(m.Inputs, &MergeInput{}) + if err := m.Inputs[len(m.Inputs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipOps(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthOps + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipOps(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/solver/pb/ops.proto b/solver/pb/ops.proto index 0a88882b0dac..51bf876fe8a1 100644 --- a/solver/pb/ops.proto +++ b/solver/pb/ops.proto @@ -17,6 +17,7 @@ message Op { SourceOp source = 3; FileOp file = 4; BuildOp build = 5; + MergeOp merge = 6; } Platform platform = 10; WorkerConstraints constraints = 11; @@ -118,7 +119,7 @@ message TmpfsOpt { message CacheOpt { // ID is an optional namespace for the mount string ID = 1; - // Sharing is the sharing mode for the mount + // Sharing is the sharing mode for the mount CacheSharingOpt sharing = 2; } @@ -191,11 +192,11 @@ message OpMetadata { // ignore_cache specifies to ignore the cache for this Op. bool ignore_cache = 1; // Description can be used for keeping any text fields that builder doesn't parse - map description = 2; + map description = 2; // index 3 reserved for WorkerConstraint in previous versions // WorkerConstraint worker_constraint = 3; ExportCache export_cache = 4; - + map caps = 5 [(gogoproto.castkey) = "github.com/moby/buildkit/util/apicaps.CapID", (gogoproto.nullable) = false]; } @@ -212,7 +213,7 @@ message Locations { // Source info contains the shared metadata of a source mapping message SourceInfo { - string filename = 1; + string filename = 1; bytes data = 2; Definition definition = 3; } @@ -363,3 +364,11 @@ message NamedUserOpt { string name = 1; int64 input = 2 [(gogoproto.customtype) = "InputIndex", (gogoproto.nullable) = false]; } + +message MergeInput { + int64 input = 1 [(gogoproto.customtype) = "InputIndex", (gogoproto.nullable) = false]; +} + +message MergeOp { + repeated MergeInput inputs = 1; +} diff --git a/source/containerimage/pull.go b/source/containerimage/pull.go index 1162f1c338de..c55c28b5761f 100644 --- a/source/containerimage/pull.go +++ b/source/containerimage/pull.go @@ -33,7 +33,7 @@ import ( "github.com/moby/buildkit/util/progress/controller" "github.com/moby/buildkit/util/pull" "github.com/moby/buildkit/util/resolver" - "github.com/opencontainers/go-digest" + digest "github.com/opencontainers/go-digest" "github.com/opencontainers/image-spec/identity" ocispecs "github.com/opencontainers/image-spec/specs-go/v1" "github.com/pkg/errors" diff --git a/source/git/gitsource_test.go b/source/git/gitsource_test.go index bf3be616c1dc..962dd1934a67 100644 --- a/source/git/gitsource_test.go +++ b/source/git/gitsource_test.go @@ -11,6 +11,8 @@ import ( "testing" "github.com/containerd/containerd/content/local" + "github.com/containerd/containerd/diff/apply" + "github.com/containerd/containerd/diff/walking" ctdmetadata "github.com/containerd/containerd/metadata" "github.com/containerd/containerd/namespaces" "github.com/containerd/containerd/snapshots" @@ -22,6 +24,7 @@ import ( containerdsnapshot "github.com/moby/buildkit/snapshot/containerd" "github.com/moby/buildkit/source" "github.com/moby/buildkit/util/leaseutil" + "github.com/moby/buildkit/util/winlayers" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -456,12 +459,17 @@ func setupGitSource(t *testing.T, tmpdir string) source.Source { md, err := metadata.NewStore(filepath.Join(tmpdir, "metadata.db")) require.NoError(t, err) lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") + c := mdb.ContentStore() + applier := winlayers.NewFileSystemApplierWithWindows(c, apply.NewFileSystemApplier(c)) + differ := winlayers.NewWalkingDiffWithWindows(c, walking.NewWalkingDiff(c)) cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter("native", containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter("native")), nil), MetadataStore: md, LeaseManager: lm, - ContentStore: mdb.ContentStore(), + ContentStore: c, + Applier: applier, + Differ: differ, GarbageCollect: mdb.GarbageCollect, }) require.NoError(t, err) diff --git a/source/http/httpsource_test.go b/source/http/httpsource_test.go index c3c5bd069288..b291f6ea17ec 100644 --- a/source/http/httpsource_test.go +++ b/source/http/httpsource_test.go @@ -9,6 +9,8 @@ import ( "testing" "github.com/containerd/containerd/content/local" + "github.com/containerd/containerd/diff/apply" + "github.com/containerd/containerd/diff/walking" ctdmetadata "github.com/containerd/containerd/metadata" "github.com/containerd/containerd/snapshots" "github.com/containerd/containerd/snapshots/native" @@ -20,6 +22,7 @@ import ( "github.com/moby/buildkit/source" "github.com/moby/buildkit/util/leaseutil" "github.com/moby/buildkit/util/testutil/httpserver" + "github.com/moby/buildkit/util/winlayers" digest "github.com/opencontainers/go-digest" "github.com/stretchr/testify/require" bolt "go.etcd.io/bbolt" @@ -359,12 +362,17 @@ func newHTTPSource(tmpdir string) (source.Source, error) { return nil, err } lm := leaseutil.WithNamespace(ctdmetadata.NewLeaseManager(mdb), "buildkit") + c := mdb.ContentStore() + applier := winlayers.NewFileSystemApplierWithWindows(c, apply.NewFileSystemApplier(c)) + differ := winlayers.NewWalkingDiffWithWindows(c, walking.NewWalkingDiff(c)) cm, err := cache.NewManager(cache.ManagerOpt{ Snapshotter: snapshot.FromContainerdSnapshotter("native", containerdsnapshot.NSSnapshotter("buildkit", mdb.Snapshotter("native")), nil), MetadataStore: md, LeaseManager: lm, - ContentStore: mdb.ContentStore(), + ContentStore: c, + Applier: applier, + Differ: differ, GarbageCollect: mdb.GarbageCollect, }) if err != nil { diff --git a/source/local/local.go b/source/local/local.go index 89f49fa373d1..225622066c69 100644 --- a/source/local/local.go +++ b/source/local/local.go @@ -117,6 +117,8 @@ func (ls *localSourceHandler) snapshot(ctx context.Context, s session.Group, cal bklog.G(ctx).Debugf("reusing ref for local: %s", m.ID()) mutable = m break + } else { + bklog.G(ctx).Debugf("not reusing ref %s for local: %v", si.ID(), err) } } diff --git a/util/overlay/overlay_linux.go b/util/overlay/overlay_linux.go index 4b979ef9d0a7..fcf646cc1cba 100644 --- a/util/overlay/overlay_linux.go +++ b/util/overlay/overlay_linux.go @@ -174,7 +174,8 @@ func Changes(ctx context.Context, changeFn fs.ChangeFunc, upperdir, upperdirView if isDelete { // This is a deleted entry. kind = fs.ChangeKindDelete - f = nil + // Leave f set to the FileInfo for the whiteout device in case the caller wants it, e.g. + // the merge code uses it to hardlink in the whiteout device to merged snapshots } else if baseF, err := os.Lstat(filepath.Join(base, path)); err == nil { // File exists in the base layer. Thus this is modified. kind = fs.ChangeKindModify @@ -185,11 +186,11 @@ func Changes(ctx context.Context, changeFn fs.ChangeFunc, upperdir, upperdirView } else if err != nil { return err } - } else if os.IsNotExist(err) { + } else if os.IsNotExist(err) || errors.Is(err, unix.ENOTDIR) { // File doesn't exist in the base layer. Thus this is added. kind = fs.ChangeKindAdd } else if err != nil { - return err + return errors.Wrap(err, "failed to stat base file during overlay diff") } if !skipRecord { diff --git a/worker/base/worker.go b/worker/base/worker.go index f3370df63916..ead1172c57fa 100644 --- a/worker/base/worker.go +++ b/worker/base/worker.go @@ -267,6 +267,8 @@ func (w *Worker) ResolveOp(v solver.Vertex, s frontend.FrontendLLBBridge, sm *se return ops.NewFileOp(v, op, w.CacheMgr, w.ParallelismSem, w) case *pb.Op_Build: return ops.NewBuildOp(v, op, s, w) + case *pb.Op_Merge: + return ops.NewMergeOp(v, op, w) default: return nil, errors.Errorf("no support for %T", op) }