Merge pull request #2434 from sipsma/diffop

DiffOp
master
Tõnis Tiigi 2022-01-06 12:33:03 -08:00 committed by GitHub
commit a2528b9772
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 3058 additions and 345 deletions

96
cache/blobs.go vendored
View File

@ -47,27 +47,40 @@ func (sr *immutableRef) computeBlobChain(ctx context.Context, createIfNeeded boo
ctx = winlayers.UseWindowsLayerMode(ctx)
}
return computeBlobChain(ctx, sr, createIfNeeded, compressionType, forceCompression, s)
// filter keeps track of which layers should actually be included in the blob chain.
// This is required for diff refs, which can include only a single layer from their parent
// refs rather than every single layer present among their ancestors.
filter := sr.layerSet()
return computeBlobChain(ctx, sr, createIfNeeded, compressionType, forceCompression, s, filter)
}
type compressor func(dest io.Writer, requiredMediaType string) (io.WriteCloser, error)
func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool, compressionType compression.Type, forceCompression bool, s session.Group) error {
func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool, compressionType compression.Type, forceCompression bool, s session.Group, filter map[string]struct{}) error {
eg, ctx := errgroup.WithContext(ctx)
switch sr.kind() {
case Merge:
for _, parent := range sr.mergeParents {
parent := parent
eg.Go(func() error {
return computeBlobChain(ctx, parent, createIfNeeded, compressionType, forceCompression, s)
return computeBlobChain(ctx, parent, createIfNeeded, compressionType, forceCompression, s, filter)
})
}
case Diff:
if _, ok := filter[sr.ID()]; !ok && sr.diffParents.upper != nil {
// This diff is just re-using the upper blob, compute that
eg.Go(func() error {
return computeBlobChain(ctx, sr.diffParents.upper, createIfNeeded, compressionType, forceCompression, s, filter)
})
}
case Layer:
eg.Go(func() error {
return computeBlobChain(ctx, sr.layerParent, createIfNeeded, compressionType, forceCompression, s)
return computeBlobChain(ctx, sr.layerParent, createIfNeeded, compressionType, forceCompression, s, filter)
})
fallthrough
case BaseLayer:
}
if _, ok := filter[sr.ID()]; ok {
eg.Go(func() error {
_, err := g.Do(ctx, fmt.Sprintf("%s-%t", sr.ID(), createIfNeeded), func(ctx context.Context) (interface{}, error) {
if sr.getBlob() != "" {
@ -95,9 +108,16 @@ func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool
return nil, errors.Errorf("unknown layer compression type: %q", compressionType)
}
var lowerRef *immutableRef
switch sr.kind() {
case Diff:
lowerRef = sr.diffParents.lower
case Layer:
lowerRef = sr.layerParent
}
var lower []mount.Mount
if sr.layerParent != nil {
m, err := sr.layerParent.Mount(ctx, true, s)
if lowerRef != nil {
m, err := lowerRef.Mount(ctx, true, s)
if err != nil {
return nil, err
}
@ -110,22 +130,36 @@ func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool
defer release()
}
}
m, err := sr.Mount(ctx, true, s)
var upperRef *immutableRef
switch sr.kind() {
case Diff:
upperRef = sr.diffParents.upper
default:
upperRef = sr
}
var upper []mount.Mount
if upperRef != nil {
m, err := upperRef.Mount(ctx, true, s)
if err != nil {
return nil, err
}
upper, release, err := m.Mount()
var release func() error
upper, release, err = m.Mount()
if err != nil {
return nil, err
}
if release != nil {
defer release()
}
}
var desc ocispecs.Descriptor
var err error
// 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 != "" {
if forceOvlStr := os.Getenv("BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF"); forceOvlStr != "" && sr.kind() != Diff {
enableOverlay, err = strconv.ParseBool(forceOvlStr)
if err != nil {
return nil, errors.Wrapf(err, "invalid boolean in BUILDKIT_DEBUG_FORCE_OVERLAY_DIFF")
@ -135,8 +169,10 @@ func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool
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
// overlayfs-based snapshotters should support overlay diff except when running an arbitrary diff
// (in which case lower and upper may differ by more than one layer), so print warn log on unexpected
// failure.
logWarnOnErr = sr.kind() != Diff
case "fuse-overlayfs", "native":
// not supported with fuse-overlayfs snapshotter which doesn't provide overlayfs mounts.
// TODO: add support for fuse-overlayfs
@ -220,7 +256,7 @@ func computeBlobChain(ctx context.Context, sr *immutableRef, createIfNeeded bool
if err := eg.Wait(); err != nil {
return err
}
return sr.computeChainMetadata(ctx)
return sr.computeChainMetadata(ctx, filter)
}
// setBlob associates a blob with the cache record.
@ -274,7 +310,7 @@ func (sr *immutableRef) setBlob(ctx context.Context, compressionType compression
return nil
}
func (sr *immutableRef) computeChainMetadata(ctx context.Context) error {
func (sr *immutableRef) computeChainMetadata(ctx context.Context, filter map[string]struct{}) error {
if _, ok := leases.FromContext(ctx); !ok {
return errors.Errorf("missing lease requirement for computeChainMetadata")
}
@ -291,10 +327,17 @@ func (sr *immutableRef) computeChainMetadata(ctx context.Context) error {
switch sr.kind() {
case BaseLayer:
if _, ok := filter[sr.ID()]; !ok {
return nil
}
diffID := sr.getDiffID()
chainID = diffID
blobChainID = imagespecidentity.ChainID([]digest.Digest{digest.Digest(sr.getBlob()), diffID})
case Layer:
if _, ok := filter[sr.ID()]; !ok {
return nil
}
if _, ok := filter[sr.layerParent.ID()]; ok {
if parentChainID := sr.layerParent.getChainID(); parentChainID != "" {
chainID = parentChainID
} else {
@ -305,25 +348,42 @@ func (sr *immutableRef) computeChainMetadata(ctx context.Context) error {
} else {
return errors.Errorf("failed to set blobchain for reference with non-addressable parent %q", sr.layerParent.GetDescription())
}
}
diffID := digest.Digest(sr.getDiffID())
chainID = imagespecidentity.ChainID([]digest.Digest{chainID, diffID})
blobID := imagespecidentity.ChainID([]digest.Digest{digest.Digest(sr.getBlob()), diffID})
blobChainID = imagespecidentity.ChainID([]digest.Digest{blobChainID, blobID})
case Merge:
// Merge chain IDs can re-use the first input chain ID as a base, but after that have to
// be computed one-by-one for each blob in the chain. It should have the same value as
// if you had created the merge by unpacking all the blobs on top of each other with GetByBlob.
baseInput := sr.mergeParents[0]
if _, ok := filter[baseInput.ID()]; !ok {
// not enough information to compute chain at this time
return nil
}
chainID = baseInput.getChainID()
blobChainID = baseInput.getBlobChainID()
for _, mergeParent := range sr.mergeParents[1:] {
for _, layer := range mergeParent.layerChain() {
if _, ok := filter[layer.ID()]; !ok {
// not enough information to compute chain at this time
return nil
}
diffID := digest.Digest(layer.getDiffID())
chainID = imagespecidentity.ChainID([]digest.Digest{chainID, diffID})
blobID := imagespecidentity.ChainID([]digest.Digest{digest.Digest(layer.getBlob()), diffID})
blobChainID = imagespecidentity.ChainID([]digest.Digest{blobChainID, blobID})
}
}
case Diff:
if _, ok := filter[sr.ID()]; ok {
// this diff is its own blob
diffID := sr.getDiffID()
chainID = diffID
blobChainID = imagespecidentity.ChainID([]digest.Digest{digest.Digest(sr.getBlob()), diffID})
} else {
// re-using upper blob
chainID = sr.diffParents.upper.getChainID()
blobChainID = sr.diffParents.upper.getBlobChainID()
}
}
sr.queueChainID(chainID)

238
cache/manager.go vendored
View File

@ -2,6 +2,7 @@ package cache
import (
"context"
"fmt"
"sort"
"strings"
"sync"
@ -56,6 +57,7 @@ type Accessor interface {
GetMutable(ctx context.Context, id string, opts ...RefOption) (MutableRef, error) // Rebase?
IdentityMapping() *idtools.IdentityMapping
Merge(ctx context.Context, parents []ImmutableRef, opts ...RefOption) (ImmutableRef, error)
Diff(ctx context.Context, lower, upper ImmutableRef, opts ...RefOption) (ImmutableRef, error)
}
type Controller interface {
@ -477,6 +479,26 @@ func (cm *cacheManager) parentsOf(ctx context.Context, md *cacheMetadata, opts .
}
ps.mergeParents = append(ps.mergeParents, p)
}
if lowerParentID := md.getLowerDiffParent(); lowerParentID != "" {
p, err := cm.get(ctx, lowerParentID, append(opts, NoUpdateLastUsed))
if err != nil {
return ps, err
}
if ps.diffParents == nil {
ps.diffParents = &diffParents{}
}
ps.diffParents.lower = p
}
if upperParentID := md.getUpperDiffParent(); upperParentID != "" {
p, err := cm.get(ctx, upperParentID, append(opts, NoUpdateLastUsed))
if err != nil {
return ps, err
}
if ps.diffParents == nil {
ps.diffParents = &diffParents{}
}
ps.diffParents.upper = p
}
return ps, nil
}
@ -627,8 +649,6 @@ func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef,
// 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() {
@ -659,7 +679,7 @@ func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef,
for _, grandparent := range parent.mergeParents {
parents.mergeParents = append(parents.mergeParents, grandparent.clone())
}
case Layer, BaseLayer:
default:
parents.mergeParents = append(parents.mergeParents, parent.clone())
}
for dgst, handler := range parent.descHandlers {
@ -667,6 +687,15 @@ func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef,
}
}
mergeRef, err := cm.createMergeRef(ctx, parents, dhs, opts...)
if err != nil {
parents.release(context.TODO())
return nil, err
}
return mergeRef, nil
}
func (cm *cacheManager) createMergeRef(ctx context.Context, parents parentRefs, dhs DescHandlers, opts ...RefOption) (ir *immutableRef, rerr error) {
if len(parents.mergeParents) == 0 {
// merge of nothing is nothing
return nil, nil
@ -686,6 +715,7 @@ func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef,
defer cm.mu.Unlock()
// Build the new ref
id := identity.NewID()
md, _ := cm.getMetadata(id)
rec := &cacheRecord{
@ -740,6 +770,177 @@ func (cm *cacheManager) Merge(ctx context.Context, inputParents []ImmutableRef,
return rec.ref(true, dhs), nil
}
func (cm *cacheManager) Diff(ctx context.Context, lower, upper ImmutableRef, opts ...RefOption) (ir ImmutableRef, rerr error) {
if lower == nil {
return nil, errors.New("lower ref for diff cannot be nil")
}
var dps diffParents
parents := parentRefs{diffParents: &dps}
dhs := make(map[digest.Digest]*DescHandler)
defer func() {
if rerr != nil {
parents.release(context.TODO())
}
}()
for i, inputParent := range []ImmutableRef{lower, upper} {
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())
}
if i == 0 {
dps.lower = parent
} else {
dps.upper = parent
}
for dgst, handler := range parent.descHandlers {
dhs[dgst] = handler
}
}
// Check to see if lower is an ancestor of upper. If so, define the diff as a merge
// of the layers separating the two. This can result in a different diff than just
// running the differ directly on lower and upper, but this is chosen as a default
// behavior in order to maximize layer re-use in the default case. We may add an
// option for controlling this behavior in the future if it's needed.
if dps.upper != nil {
lowerLayers := dps.lower.layerChain()
upperLayers := dps.upper.layerChain()
var lowerIsAncestor bool
// when upper is only 1 layer different than lower, we can skip this as we
// won't need a merge in order to get optimal behavior.
if len(upperLayers) > len(lowerLayers)+1 {
lowerIsAncestor = true
for i, lowerLayer := range lowerLayers {
if lowerLayer.ID() != upperLayers[i].ID() {
lowerIsAncestor = false
break
}
}
}
if lowerIsAncestor {
mergeParents := parentRefs{mergeParents: make([]*immutableRef, len(upperLayers)-len(lowerLayers))}
defer func() {
if rerr != nil {
mergeParents.release(context.TODO())
}
}()
for i := len(lowerLayers); i < len(upperLayers); i++ {
subUpper := upperLayers[i]
subLower := subUpper.layerParent
if subLower == nil {
mergeParents.mergeParents[i-len(lowerLayers)] = subUpper.clone()
} else {
subParents := parentRefs{diffParents: &diffParents{lower: subLower.clone(), upper: subUpper.clone()}}
diffRef, err := cm.createDiffRef(ctx, subParents, subUpper.descHandlers,
WithDescription(fmt.Sprintf("diff %q -> %q", subLower.ID(), subUpper.ID())))
if err != nil {
subParents.release(context.TODO())
return nil, err
}
mergeParents.mergeParents[i-len(lowerLayers)] = diffRef
}
}
mergeRef, err := cm.createMergeRef(ctx, mergeParents, dhs)
if err != nil {
return nil, err
}
parents.release(context.TODO())
return mergeRef, nil
}
}
diffRef, err := cm.createDiffRef(ctx, parents, dhs, opts...)
if err != nil {
parents.release(context.TODO())
return nil, err
}
return diffRef, nil
}
func (cm *cacheManager) createDiffRef(ctx context.Context, parents parentRefs, dhs DescHandlers, opts ...RefOption) (ir *immutableRef, rerr error) {
dps := parents.diffParents
if err := dps.lower.Finalize(ctx); err != nil {
return nil, errors.Wrapf(err, "failed to finalize lower parent during diff")
}
if dps.upper != nil {
if err := dps.upper.Finalize(ctx); err != nil {
return nil, errors.Wrapf(err, "failed to finalize upper parent during diff")
}
}
id := identity.NewID()
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
}
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
}
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()
@ -963,6 +1164,14 @@ func (cm *cacheManager) prune(ctx context.Context, ch chan client.UsageInfo, opt
for i, p := range cr.mergeParents {
c.Parents[i] = p.ID()
}
case Diff:
c.Parents = make([]string, 0, 2)
if cr.diffParents.lower != nil {
c.Parents = append(c.Parents, cr.diffParents.lower.ID())
}
if cr.diffParents.upper != nil {
c.Parents = append(c.Parents, cr.diffParents.upper.ID())
}
}
if c.Size == sizeUnknown && cr.equalImmutable != nil {
c.Size = cr.equalImmutable.getSize() // benefit from DiskUsage calc
@ -1089,6 +1298,13 @@ func (cm *cacheManager) DiskUsage(ctx context.Context, opt client.DiskUsageInfo)
for i, p := range cr.mergeParents {
c.parents[i] = p.ID()
}
case Diff:
if cr.diffParents.lower != nil {
c.parents = append(c.parents, cr.diffParents.lower.ID())
}
if cr.diffParents.upper != nil {
c.parents = append(c.parents, cr.diffParents.upper.ID())
}
}
if cr.mutable && c.refs > 0 {
c.size = 0 // size can not be determined because it is changing
@ -1251,11 +1467,12 @@ func initializeMetadata(m *cacheMetadata, parents parentRefs, opts ...RefOption)
return nil
}
if parents.layerParent != nil {
switch {
case parents.layerParent != nil:
if err := m.queueParent(parents.layerParent.ID()); err != nil {
return err
}
} else if len(parents.mergeParents) > 0 {
case len(parents.mergeParents) > 0:
var ids []string
for _, p := range parents.mergeParents {
ids = append(ids, p.ID())
@ -1263,6 +1480,17 @@ func initializeMetadata(m *cacheMetadata, parents parentRefs, opts ...RefOption)
if err := m.queueMergeParents(ids); err != nil {
return err
}
case parents.diffParents != nil:
if parents.diffParents.lower != nil {
if err := m.queueLowerDiffParent(parents.diffParents.lower.ID()); err != nil {
return err
}
}
if parents.diffParents.upper != nil {
if err := m.queueUpperDiffParent(parents.diffParents.upper.ID()); err != nil {
return err
}
}
}
if err := m.queueCreatedAt(time.Now()); err != nil {

View File

@ -602,7 +602,7 @@ func TestExtractOnMutable(t *testing.T) {
err = snap.(*immutableRef).setBlob(leaseCtx, compressionType, desc)
done(context.TODO())
require.NoError(t, err)
err = snap.(*immutableRef).computeChainMetadata(leaseCtx)
err = snap.(*immutableRef).computeChainMetadata(leaseCtx, map[string]struct{}{snap.ID(): {}})
require.NoError(t, err)
snap2, err := cm.GetByBlob(ctx, desc2, snap)
@ -725,7 +725,7 @@ func TestSetBlob(t *testing.T) {
}
err = snap.(*immutableRef).setBlob(ctx, compressionType, desc)
require.NoError(t, err)
err = snap.(*immutableRef).computeChainMetadata(ctx)
err = snap.(*immutableRef).computeChainMetadata(ctx, map[string]struct{}{snap.ID(): {}})
require.NoError(t, err)
snapRef = snap.(*immutableRef)
@ -755,7 +755,7 @@ func TestSetBlob(t *testing.T) {
}
err = snap2.(*immutableRef).setBlob(ctx, compressionType2, desc2)
require.NoError(t, err)
err = snap2.(*immutableRef).computeChainMetadata(ctx)
err = snap2.(*immutableRef).computeChainMetadata(ctx, map[string]struct{}{snap.ID(): {}, snap2.ID(): {}})
require.NoError(t, err)
snapRef2 := snap2.(*immutableRef)

18
cache/metadata.go vendored
View File

@ -25,6 +25,8 @@ const keyRecordType = "cache.recordType"
const keyCommitted = "snapshot.committed"
const keyParent = "cache.parent"
const keyMergeParents = "cache.mergeParents"
const keyLowerDiffParent = "cache.lowerDiffParent"
const keyUpperDiffParent = "cache.upperDiffParent"
const keyDiffID = "cache.diffID"
const keyChainID = "cache.chainID"
const keyBlobChainID = "cache.blobChainID"
@ -315,6 +317,22 @@ func (md *cacheMetadata) getMergeParents() []string {
return md.getStringSlice(keyMergeParents)
}
func (md *cacheMetadata) queueLowerDiffParent(parent string) error {
return md.queueValue(keyLowerDiffParent, parent, "")
}
func (md *cacheMetadata) getLowerDiffParent() string {
return md.GetString(keyLowerDiffParent)
}
func (md *cacheMetadata) queueUpperDiffParent(parent string) error {
return md.queueValue(keyUpperDiffParent, parent, "")
}
func (md *cacheMetadata) getUpperDiffParent() string {
return md.GetString(keyUpperDiffParent)
}
func (md *cacheMetadata) queueSize(s int64) error {
return md.queueValue(keySize, s, "")
}

139
cache/refs.go vendored
View File

@ -114,15 +114,22 @@ func (cr *cacheRecord) mref(triggerLastUsed bool, descHandlers DescHandlers) *mu
type parentRefs struct {
layerParent *immutableRef
mergeParents []*immutableRef
diffParents *diffParents
}
type diffParents struct {
lower *immutableRef
upper *immutableRef
}
// caller must hold cacheManager.mu
func (p parentRefs) release(ctx context.Context) (rerr error) {
if p.layerParent != nil {
switch {
case p.layerParent != nil:
p.layerParent.mu.Lock()
defer p.layerParent.mu.Unlock()
rerr = p.layerParent.release(ctx)
} else if len(p.mergeParents) > 0 {
case len(p.mergeParents) > 0:
for i, parent := range p.mergeParents {
if parent == nil {
continue
@ -135,19 +142,49 @@ func (p parentRefs) release(ctx context.Context) (rerr error) {
}
parent.mu.Unlock()
}
case p.diffParents != nil:
if p.diffParents.lower != nil {
p.diffParents.lower.mu.Lock()
defer p.diffParents.lower.mu.Unlock()
if err := p.diffParents.lower.release(ctx); err != nil {
rerr = multierror.Append(rerr, err).ErrorOrNil()
} else {
p.diffParents.lower = nil
}
}
if p.diffParents.upper != nil {
p.diffParents.upper.mu.Lock()
defer p.diffParents.upper.mu.Unlock()
if err := p.diffParents.upper.release(ctx); err != nil {
rerr = multierror.Append(rerr, err).ErrorOrNil()
} else {
p.diffParents.upper = nil
}
}
}
return rerr
}
func (p parentRefs) clone() parentRefs {
if p.layerParent != nil {
switch {
case p.layerParent != nil:
p.layerParent = p.layerParent.clone()
} else if len(p.mergeParents) > 0 {
case len(p.mergeParents) > 0:
newParents := make([]*immutableRef, len(p.mergeParents))
for i, p := range p.mergeParents {
newParents[i] = p.clone()
}
p.mergeParents = newParents
case p.diffParents != nil:
newDiffParents := &diffParents{}
if p.diffParents.lower != nil {
newDiffParents.lower = p.diffParents.lower.clone()
}
if p.diffParents.upper != nil {
newDiffParents.upper = p.diffParents.upper.clone()
}
p.diffParents = newDiffParents
}
return p
}
@ -158,12 +195,16 @@ const (
BaseLayer refKind = iota
Layer
Merge
Diff
)
func (cr *cacheRecord) kind() refKind {
if len(cr.mergeParents) > 0 {
return Merge
}
if cr.diffParents != nil {
return Diff
}
if cr.layerParent != nil {
return Layer
}
@ -177,8 +218,8 @@ func (cr *cacheRecord) isDead() bool {
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
// walkAncestors calls the provided func on cr and each of its ancestors, counting layer,
// diff, 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.
@ -200,6 +241,13 @@ func (cr *cacheRecord) walkAncestors(f func(*cacheRecord) error) error {
for _, p := range cur.mergeParents {
curs = append(curs, p.cacheRecord)
}
case Diff:
if cur.diffParents.lower != nil {
curs = append(curs, cur.diffParents.lower.cacheRecord)
}
if cur.diffParents.upper != nil {
curs = append(curs, cur.diffParents.upper.cacheRecord)
}
}
}
return nil
@ -416,6 +464,19 @@ func (sr *immutableRef) layerChain() []*immutableRef {
return layers
}
// returns the set of cache record IDs for each layer in sr's layer chain
func (sr *immutableRef) layerSet() map[string]struct{} {
var count int
sr.layerWalk(func(*immutableRef) {
count++
})
set := make(map[string]struct{}, count)
sr.layerWalk(func(sr *immutableRef) {
set[sr.ID()] = struct{}{}
})
return set
}
// 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.
@ -425,6 +486,21 @@ func (sr *immutableRef) layerWalk(f func(*immutableRef)) {
for _, parent := range sr.mergeParents {
parent.layerWalk(f)
}
case Diff:
lower := sr.diffParents.lower
upper := sr.diffParents.upper
// If upper is only one blob different from lower, then re-use that blob
switch {
case upper != nil && lower == nil && upper.kind() == BaseLayer:
// upper is a single layer being diffed with scratch
f(upper)
case upper != nil && lower != nil && upper.kind() == Layer && upper.layerParent.ID() == lower.ID():
// upper is a single layer on top of lower
f(upper)
default:
// otherwise, the diff will be computed and turned into its own single blob
f(sr)
}
case Layer:
sr.layerParent.layerWalk(f)
fallthrough
@ -439,6 +515,13 @@ func (cr *cacheRecord) layerDigestChain() []digest.Digest {
return cr.layerDigestChainCache
}
switch cr.kind() {
case Diff:
if cr.getBlob() == "" {
// this diff just reuses the upper blob
cr.layerDigestChainCache = cr.diffParents.upper.layerDigestChain()
} else {
cr.layerDigestChainCache = append(cr.layerDigestChainCache, cr.getBlob())
}
case Merge:
for _, parent := range cr.mergeParents {
cr.layerDigestChainCache = append(cr.layerDigestChainCache, parent.layerDigestChain()...)
@ -870,8 +953,8 @@ func (sr *immutableRef) unlazy(ctx context.Context, dhs DescHandlers, s session.
}
switch sr.kind() {
case Merge:
return nil, sr.unlazyMerge(ctx, dhs, s)
case Merge, Diff:
return nil, sr.unlazyDiffMerge(ctx, dhs, s)
case Layer, BaseLayer:
return nil, sr.unlazyLayer(ctx, dhs, s)
}
@ -881,25 +964,39 @@ func (sr *immutableRef) unlazy(ctx context.Context, dhs DescHandlers, s session.
}
// 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 {
func (sr *immutableRef) unlazyDiffMerge(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 {
var diff snapshot.Diff
switch sr.kind() {
case Diff:
if sr.diffParents.lower != nil {
diff.Lower = sr.diffParents.lower.getSnapshotID()
eg.Go(func() error {
return sr.diffParents.lower.unlazy(egctx, dhs, s)
})
}
if sr.diffParents.upper != nil {
diff.Upper = sr.diffParents.upper.getSnapshotID()
eg.Go(func() error {
return sr.diffParents.upper.unlazy(egctx, dhs, s)
})
}
case Layer:
diff.Lower = sr.layerParent.getSnapshotID()
fallthrough
case BaseLayer:
diff.Upper = sr.getSnapshotID()
eg.Go(func() error {
return sr.unlazy(egctx, dhs, s)
})
}
diffs = append(diffs, diff)
})
if err := eg.Wait(); err != nil {
return err
}
return sr.cm.Snapshotter.Merge(ctx, sr.getSnapshotID(), diffs)
}
@ -1089,7 +1186,7 @@ func (cr *cacheRecord) finalize(ctx context.Context) error {
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 to %s", mutable.getSnapshotID(), cr.getSnapshotID())
return errors.Wrapf(err, "failed to commit %s to %s during finalize", mutable.getSnapshotID(), cr.getSnapshotID())
}
cr.mountCache = nil

View File

@ -72,7 +72,7 @@ func TestIntegration(t *testing.T) {
mirroredImages["tonistiigi/test:nolayers"] = "docker.io/tonistiigi/test:nolayers"
mirrors := integration.WithMirroredImages(mirroredImages)
integration.Run(t, integration.TestFuncs(
tests := integration.TestFuncs(
testCacheExportCacheKeyLoop,
testRelativeWorkDir,
testFileOpMkdirMkfile,
@ -141,7 +141,9 @@ func TestIntegration(t *testing.T) {
testMergeOpCache,
testRmSymlink,
testMoveParentDir,
), mirrors)
)
tests = append(tests, diffOpTestCases()...)
integration.Run(t, tests, mirrors)
integration.Run(t, integration.TestFuncs(
testSecurityMode,
@ -4023,93 +4025,19 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
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...)))
// verify the image exports+imports the same
var target string
var exports []ExportEntry
if os.Getenv("TEST_DOCKERD") == "1" {
// use a fake url so we can guarantee when we do the build with the image
// we are using the local image in dockerd, not pulling from a registry
target = "fake.invalid:33333/testmergeop:latest"
exports = []ExportEntry{{
Type: "moby",
Attrs: map[string]string{
"name": target,
},
}}
} else {
ctx := sb.Context()
registry, err := sb.NewRegistry()
if errors.Is(err, integration.ErrorRequirements) {
return
}
require.NoError(t, err)
target = registry + "/testmergeop:latest"
exports = []ExportEntry{{
Type: ExporterImage,
Attrs: map[string]string{
"name": target,
"push": "true",
},
}}
}
_, err = c.Solve(sb.Context(), def, SolveOpt{Exports: exports}, nil)
require.NoError(t, err)
cdAddress := sb.ContainerdAddress()
if cdAddress != "" {
ctx := namespaces.WithNamespace(sb.Context(), "buildkit")
client, err := newContainerd(cdAddress)
require.NoError(t, err)
defer client.Close()
imageService := client.ImageService()
imageList, err := imageService.List(ctx)
require.NoError(t, err)
for _, img := range imageList {
err = imageService.Delete(ctx, img.Name, images.SynchronousDelete())
if !errors.Is(err, integration.ErrorRequirements) {
require.NoError(t, err)
}
}
checkAllReleasable(t, c, sb, true)
img := llb.Image(target, llb.ResolveModePreferLocal)
def, err = img.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...)))
var imageTarget string
if os.Getenv("TEST_DOCKERD") == "1" {
// do image export but use a fake url as the image should just end up in moby's
// local store
imageTarget = "fake.invalid:33333/buildkit/testmergeop:latest"
} else if registry != "" {
imageTarget = registry + "/buildkit/testmergeop:latest"
}
stateA := llb.Scratch().
@ -4128,7 +4056,7 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
File(llb.Mkfile("/bar/A", 0400, []byte("C")))
mergeA := llb.Merge([]llb.State{stateA, stateC})
requireContents(mergeA,
requireContents(ctx, t, c, sb, mergeA, nil, nil, imageTarget,
fstest.CreateFile("foo", []byte("C"), 0755),
fstest.CreateFile("c", []byte("C"), 0755),
fstest.CreateDir("bar", 0755),
@ -4137,7 +4065,7 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
)
mergeB := llb.Merge([]llb.State{stateC, stateB})
requireContents(mergeB,
requireContents(ctx, t, c, sb, mergeB, nil, nil, imageTarget,
fstest.CreateFile("a", []byte("A"), 0755),
fstest.CreateFile("b", []byte("B"), 0755),
fstest.CreateFile("c", []byte("C"), 0755),
@ -4148,7 +4076,7 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
stateD := llb.Scratch().File(llb.Mkdir("/qaz", 0755))
mergeC := llb.Merge([]llb.State{mergeA, mergeB, stateD})
requireContents(mergeC,
requireContents(ctx, t, c, sb, mergeC, nil, nil, imageTarget,
fstest.CreateFile("a", []byte("A"), 0755),
fstest.CreateFile("b", []byte("B"), 0755),
fstest.CreateFile("c", []byte("C"), 0755),
@ -4158,8 +4086,7 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
fstest.CreateDir("qaz", 0755),
)
runA := llb.Merge([]llb.State{llb.Image("busybox"), mergeC}).
Run(llb.Shlex("sh -c -e -x '" + strings.Join([]string{
runA := runShell(llb.Merge([]llb.State{llb.Image("alpine"), mergeC}),
// turn /a file into a dir, mv b and c into it
"rm /a",
"mkdir /a",
@ -4171,31 +4098,21 @@ func testMergeOp(t *testing.T, sb integration.Sandbox) {
// 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", 0755, []byte("E"))).
File(llb.Mkdir("/bar", 0755)).
File(llb.Mkfile("/bar/A", 0755, []byte("A"))).
File(llb.Mkfile("/bar/E", 0755, nil))
mergeD := llb.Merge([]llb.State{stateE, runA})
requireContents(mergeD,
fstest.CreateDir("a", 0755),
fstest.CreateFile("a/b", []byte("B"), 0755),
fstest.CreateFile("a/c", []byte("C"), 0755),
fstest.CreateDir("bar", 0755),
fstest.CreateFile("bar/D", []byte("D"), 0644),
fstest.CreateFile("bar/E", nil, 0755), // exists because opaques dirs are converted to explicit whiteouts
fstest.CreateFile("qaz", nil, 0644),
requireEqualContents(ctx, t, c, mergeD, llb.Image("alpine").
File(llb.Mkdir("a", 0755)).
File(llb.Mkfile("a/b", 0755, []byte("B"))).
File(llb.Mkfile("a/c", 0755, []byte("C"))).
File(llb.Mkdir("bar", 0755)).
File(llb.Mkfile("bar/D", 0644, []byte("D"))).
File(llb.Mkfile("bar/E", 0755, nil)).
File(llb.Mkfile("qaz", 0644, nil)),
// /foo from stateE is not here because it is deleted in stateB, which is part of a submerge of mergeD
)
}
@ -4276,10 +4193,9 @@ func testMergeOpCache(t *testing.T, sb integration.Sandbox) {
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{
input2 := runShell(llb.Image("alpine:latest"),
"mkdir /dir",
"cat /dev/urandom | head -c 100 | sha256sum > /dir/2",
}, " && ")})).Root()
"cat /dev/urandom | head -c 100 | sha256sum > /dir/2")
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})
@ -4467,6 +4383,111 @@ func testMergeOpCache(t *testing.T, sb integration.Sandbox) {
require.Equalf(t, bar2Contents, newBar2Contents, "bar/2 contents changed")
}
func requireContents(ctx context.Context, t *testing.T, c *Client, sb integration.Sandbox, state llb.State, cacheImports, cacheExports []CacheOptionsEntry, imageTarget string, files ...fstest.Applier) {
t.Helper()
def, err := state.Marshal(ctx)
require.NoError(t, err)
destDir, err := ioutil.TempDir("", "buildkit")
require.NoError(t, err)
defer os.RemoveAll(destDir)
_, err = c.Solve(ctx, def, SolveOpt{
Exports: []ExportEntry{
{
Type: ExporterLocal,
OutputDir: destDir,
},
},
CacheImports: cacheImports,
CacheExports: cacheExports,
}, nil)
require.NoError(t, err)
require.NoError(t, fstest.CheckDirectoryEqualWithApplier(destDir, fstest.Apply(files...)))
if imageTarget != "" {
var exports []ExportEntry
if os.Getenv("TEST_DOCKERD") == "1" {
exports = []ExportEntry{{
Type: "moby",
Attrs: map[string]string{
"name": imageTarget,
},
}}
} else {
exports = []ExportEntry{{
Type: ExporterImage,
Attrs: map[string]string{
"name": imageTarget,
"push": "true",
},
}}
}
_, err = c.Solve(ctx, def, SolveOpt{Exports: exports, CacheImports: cacheImports, CacheExports: cacheExports}, nil)
require.NoError(t, err)
resetState(t, c, sb)
requireContents(ctx, t, c, sb, llb.Image(imageTarget, llb.ResolveModePreferLocal), cacheImports, nil, "", files...)
}
}
func requireEqualContents(ctx context.Context, t *testing.T, c *Client, stateA, stateB llb.State) {
t.Helper()
defA, err := stateA.Marshal(ctx)
require.NoError(t, err)
destDirA, err := ioutil.TempDir("", "buildkit")
require.NoError(t, err)
defer os.RemoveAll(destDirA)
_, err = c.Solve(ctx, defA, SolveOpt{
Exports: []ExportEntry{
{
Type: ExporterLocal,
OutputDir: destDirA,
},
},
}, nil)
require.NoError(t, err)
defB, err := stateB.Marshal(ctx)
require.NoError(t, err)
destDirB, err := ioutil.TempDir("", "buildkit")
require.NoError(t, err)
defer os.RemoveAll(destDirB)
_, err = c.Solve(ctx, defB, SolveOpt{
Exports: []ExportEntry{
{
Type: ExporterLocal,
OutputDir: destDirB,
},
},
}, nil)
require.NoError(t, err)
require.NoError(t, fstest.CheckDirectoryEqual(destDirA, destDirB))
}
func runShellExecState(base llb.State, cmds ...string) llb.ExecState {
return base.Run(llb.Args([]string{"sh", "-c", strings.Join(cmds, " && ")}))
}
func runShell(base llb.State, cmds ...string) llb.State {
return runShellExecState(base, cmds...).Root()
}
func chainRunShells(base llb.State, cmdss ...[]string) llb.State {
for _, cmds := range cmdss {
base = runShell(base, cmds...)
}
return base
}
func requiresLinux(t *testing.T) {
if runtime.GOOS != "linux" {
t.Skipf("unsupported GOOS: %s", runtime.GOOS)

108
client/llb/diff.go Normal file
View File

@ -0,0 +1,108 @@
package llb
import (
"context"
"github.com/moby/buildkit/solver/pb"
digest "github.com/opencontainers/go-digest"
)
type DiffOp struct {
MarshalCache
lower Output
upper Output
output Output
constraints Constraints
}
func NewDiff(lower, upper State, c Constraints) *DiffOp {
addCap(&c, pb.CapDiffOp)
op := &DiffOp{
lower: lower.Output(),
upper: upper.Output(),
constraints: c,
}
op.output = &output{vertex: op}
return op
}
func (m *DiffOp) Validate(ctx context.Context, constraints *Constraints) error {
return nil
}
func (m *DiffOp) 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
}
proto, md := MarshalConstraints(constraints, &m.constraints)
proto.Platform = nil // diff op is not platform specific
op := &pb.DiffOp{}
op.Lower = &pb.LowerDiffInput{Input: pb.InputIndex(len(proto.Inputs))}
if m.lower == nil {
op.Lower.Input = pb.Empty
} else {
pbLowerInput, err := m.lower.ToInput(ctx, constraints)
if err != nil {
return "", nil, nil, nil, err
}
proto.Inputs = append(proto.Inputs, pbLowerInput)
}
op.Upper = &pb.UpperDiffInput{Input: pb.InputIndex(len(proto.Inputs))}
if m.upper == nil {
op.Upper.Input = pb.Empty
} else {
pbUpperInput, err := m.upper.ToInput(ctx, constraints)
if err != nil {
return "", nil, nil, nil, err
}
proto.Inputs = append(proto.Inputs, pbUpperInput)
}
proto.Op = &pb.Op_Diff{Diff: op}
dt, err := proto.Marshal()
if err != nil {
return "", nil, nil, nil, err
}
m.Store(dt, md, m.constraints.SourceLocations, constraints)
return m.Load()
}
func (m *DiffOp) Output() Output {
return m.output
}
func (m *DiffOp) Inputs() (out []Output) {
if m.lower != nil {
out = append(out, m.lower)
}
if m.upper != nil {
out = append(out, m.upper)
}
return out
}
func Diff(lower, upper State, opts ...ConstraintsOpt) State {
if lower.Output() == nil {
if upper.Output() == nil {
// diff of scratch and scratch is scratch
return Scratch()
}
// diff of scratch and upper is just upper
return upper
}
var c Constraints
for _, o := range opts {
o.SetConstraintsOption(&c)
}
return NewState(NewDiff(lower, upper, c).Output())
}

View File

@ -0,0 +1,26 @@
//go:build linux
// +build linux
package client
import (
"os"
"path/filepath"
"github.com/containerd/continuity/fs/fstest"
"golang.org/x/sys/unix"
)
func mknod(path string, mode os.FileMode, maj, min uint32) fstest.Applier {
return applyFn(func(root string) error {
return unix.Mknod(filepath.Join(root, path), uint32(mode), int(unix.Mkdev(maj, min)))
})
}
func mkfifo(path string, mode os.FileMode) fstest.Applier {
return mknod(path, mode|unix.S_IFIFO, 0, 0)
}
func mkchardev(path string, mode os.FileMode, maj, min uint32) fstest.Applier {
return mknod(path, mode|unix.S_IFCHR, maj, min)
}

View File

@ -0,0 +1,29 @@
//go:build !linux
// +build !linux
package client
import (
"os"
"github.com/containerd/continuity/fs/fstest"
"github.com/pkg/errors"
)
func mknod(path string, mode os.FileMode, maj, min uint32) fstest.Applier {
return applyFn(func(string) error {
return errors.New("mknod applier not implemented yet on this platform")
})
}
func mkfifo(path string, mode os.FileMode) fstest.Applier {
return applyFn(func(string) error {
return errors.New("mkfifo applier not implemented yet on this platform")
})
}
func mkchardev(path string, mode os.FileMode, maj, min uint32) fstest.Applier {
return applyFn(func(string) error {
return errors.New("mkchardev applier not implemented yet on this platform")
})
}

1323
client/mergediff_test.go Normal file

File diff suppressed because it is too large Load Diff

View File

@ -113,6 +113,8 @@ func attr(dgst digest.Digest, op pb.Op) (string, string) {
return "build", "box3d"
case *pb.Op_Merge:
return "merge", "invtriangle"
case *pb.Op_Diff:
return "diff", "doublecircle"
case *pb.Op_File:
names := []string{}

View File

@ -99,11 +99,15 @@ func (sn *mergeSnapshotter) Merge(ctx context.Context, key string, diffs []Diff,
// Diff("", A) -> Diff(A, B) -> Diff(B, C), etc.
var baseIndex int
for i, diff := range diffs {
var parentKey string
if diff.Upper != "" {
info, err := sn.Stat(ctx, diff.Upper)
if err != nil {
return err
}
if info.Parent != diff.Lower {
parentKey = info.Parent
}
if parentKey != diff.Lower {
break
}
if diff.Lower != baseKey {

View File

@ -136,7 +136,7 @@ func (e *exporter) ExportTo(ctx context.Context, t CacheExporterTarget, opt Cach
return nil, err
}
res.Release(context.TODO())
if remote == nil {
if remote == nil && len(remotes) > 0 {
remote, remotes = remotes[0], remotes[1:] // pop the first element
}
if opt.CompressionOpt != nil {

View File

@ -0,0 +1,130 @@
package ops
import (
"context"
"encoding/json"
"fmt"
"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 diffCacheType = "buildkit.diff.v0"
type diffOp struct {
op *pb.DiffOp
worker worker.Worker
}
func NewDiffOp(v solver.Vertex, op *pb.Op_Diff, w worker.Worker) (solver.Op, error) {
if err := llbsolver.ValidateOp(&pb.Op{Op: op}); err != nil {
return nil, err
}
return &diffOp{
op: op.Diff,
worker: w,
}, nil
}
func (d *diffOp) CacheMap(ctx context.Context, group session.Group, index int) (*solver.CacheMap, bool, error) {
dt, err := json.Marshal(struct {
Type string
Diff *pb.DiffOp
}{
Type: diffCacheType,
Diff: d.op,
})
if err != nil {
return nil, false, err
}
var depCount int
if d.op.Lower.Input != pb.Empty {
depCount++
}
if d.op.Upper.Input != pb.Empty {
depCount++
}
cm := &solver.CacheMap{
Digest: digest.Digest(dt),
Deps: make([]struct {
Selector digest.Digest
ComputeDigestFunc solver.ResultBasedCacheFunc
PreprocessFunc solver.PreprocessFunc
}, depCount),
}
return cm, true, nil
}
func (d *diffOp) Exec(ctx context.Context, g session.Group, inputs []solver.Result) ([]solver.Result, error) {
var curInput int
var lowerRef cache.ImmutableRef
var lowerRefID string
if d.op.Lower.Input != pb.Empty {
if lowerInp := inputs[curInput]; lowerInp != nil {
wref, ok := lowerInp.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid lower reference for diff op %T", lowerInp.Sys())
}
lowerRef = wref.ImmutableRef
if lowerRef != nil {
lowerRefID = wref.ImmutableRef.ID()
}
} else {
return nil, errors.New("invalid nil lower input for diff op")
}
curInput++
}
var upperRef cache.ImmutableRef
var upperRefID string
if d.op.Upper.Input != pb.Empty {
if upperInp := inputs[curInput]; upperInp != nil {
wref, ok := upperInp.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid upper reference for diff op %T", upperInp.Sys())
}
upperRef = wref.ImmutableRef
if upperRef != nil {
upperRefID = wref.ImmutableRef.ID()
}
} else {
return nil, errors.New("invalid nil upper input for diff op")
}
}
if lowerRef == nil {
if upperRef == nil {
// The diff of nothing and nothing is nothing. Just return an empty ref.
return []solver.Result{worker.NewWorkerRefResult(nil, d.worker)}, nil
}
// The diff of nothing and upper is upper. Just return a clone of upper
return []solver.Result{worker.NewWorkerRefResult(upperRef.Clone(), d.worker)}, nil
}
if upperRef != nil && lowerRef.ID() == upperRef.ID() {
// The diff of a ref and itself is nothing, return an empty ref.
return []solver.Result{worker.NewWorkerRefResult(nil, d.worker)}, nil
}
diffRef, err := d.worker.CacheManager().Diff(ctx, lowerRef, upperRef,
cache.WithDescription(fmt.Sprintf("diff %q -> %q", lowerRefID, upperRefID)))
if err != nil {
return nil, err
}
return []solver.Result{worker.NewWorkerRefResult(diffRef, d.worker)}, nil
}
func (d *diffOp) Acquire(ctx context.Context) (release solver.ReleaseFunc, err error) {
return func() {}, nil
}

View File

@ -70,6 +70,9 @@ func (m *mergeOp) Exec(ctx context.Context, g session.Group, inputs []solver.Res
if !ok {
return nil, errors.Errorf("invalid reference for merge %T", inp.Sys())
}
if wref.ImmutableRef == nil {
continue
}
refs[index] = wref.ImmutableRef
ids[index] = wref.ImmutableRef.ID()
index++

View File

@ -274,6 +274,28 @@ func llbOpName(pbOp *pb.Op, load func(digest.Digest) (solver.Vertex, error)) (st
subnames[i] = strconv.Quote(subvtx.Name())
}
return "merge " + strings.Join(subnames, " + "), nil
case *pb.Op_Diff:
var lowerName string
if op.Diff.Lower.Input == -1 {
lowerName = "scratch"
} else {
lowerVtx, err := load(pbOp.Inputs[op.Diff.Lower.Input].Digest)
if err != nil {
return "", err
}
lowerName = strconv.Quote(lowerVtx.Name())
}
var upperName string
if op.Diff.Upper.Input == -1 {
upperName = "scratch"
} else {
upperVtx, err := load(pbOp.Inputs[op.Diff.Upper.Input].Digest)
if err != nil {
return "", err
}
upperName = strconv.Quote(upperVtx.Name())
}
return "diff " + lowerName + " -> " + upperName, nil
default:
return "unknown", nil
}
@ -328,6 +350,10 @@ func ValidateOp(op *pb.Op) error {
if op.Merge == nil {
return errors.Errorf("invalid nil merge op")
}
case *pb.Op_Diff:
if op.Diff == nil {
return errors.Errorf("invalid nil diff op")
}
}
return nil
}

View File

@ -69,6 +69,7 @@ const (
CapRemoteCacheGHA apicaps.CapID = "cache.gha"
CapMergeOp apicaps.CapID = "mergeop"
CapDiffOp apicaps.CapID = "diffop"
)
func init() {
@ -381,4 +382,9 @@ func init() {
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
Caps.Init(apicaps.Cap{
ID: CapDiffOp,
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
}

View File

@ -159,6 +159,7 @@ type Op struct {
// *Op_File
// *Op_Build
// *Op_Merge
// *Op_Diff
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"`
@ -214,12 +215,16 @@ type Op_Build struct {
type Op_Merge struct {
Merge *MergeOp `protobuf:"bytes,6,opt,name=merge,proto3,oneof" json:"merge,omitempty"`
}
type Op_Diff struct {
Diff *DiffOp `protobuf:"bytes,7,opt,name=diff,proto3,oneof" json:"diff,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 (*Op_Diff) isOp_Op() {}
func (m *Op) GetOp() isOp_Op {
if m != nil {
@ -270,6 +275,13 @@ func (m *Op) GetMerge() *MergeOp {
return nil
}
func (m *Op) GetDiff() *DiffOp {
if x, ok := m.GetOp().(*Op_Diff); ok {
return x.Diff
}
return nil
}
func (m *Op) GetPlatform() *Platform {
if m != nil {
return m.Platform
@ -292,6 +304,7 @@ func (*Op) XXX_OneofWrappers() []interface{} {
(*Op_File)(nil),
(*Op_Build)(nil),
(*Op_Merge)(nil),
(*Op_Diff)(nil),
}
}
@ -2516,6 +2529,120 @@ func (m *MergeOp) GetInputs() []*MergeInput {
return nil
}
type LowerDiffInput struct {
Input InputIndex `protobuf:"varint,1,opt,name=input,proto3,customtype=InputIndex" json:"input"`
}
func (m *LowerDiffInput) Reset() { *m = LowerDiffInput{} }
func (m *LowerDiffInput) String() string { return proto.CompactTextString(m) }
func (*LowerDiffInput) ProtoMessage() {}
func (*LowerDiffInput) Descriptor() ([]byte, []int) {
return fileDescriptor_8de16154b2733812, []int{37}
}
func (m *LowerDiffInput) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LowerDiffInput) 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 *LowerDiffInput) XXX_Merge(src proto.Message) {
xxx_messageInfo_LowerDiffInput.Merge(m, src)
}
func (m *LowerDiffInput) XXX_Size() int {
return m.Size()
}
func (m *LowerDiffInput) XXX_DiscardUnknown() {
xxx_messageInfo_LowerDiffInput.DiscardUnknown(m)
}
var xxx_messageInfo_LowerDiffInput proto.InternalMessageInfo
type UpperDiffInput struct {
Input InputIndex `protobuf:"varint,1,opt,name=input,proto3,customtype=InputIndex" json:"input"`
}
func (m *UpperDiffInput) Reset() { *m = UpperDiffInput{} }
func (m *UpperDiffInput) String() string { return proto.CompactTextString(m) }
func (*UpperDiffInput) ProtoMessage() {}
func (*UpperDiffInput) Descriptor() ([]byte, []int) {
return fileDescriptor_8de16154b2733812, []int{38}
}
func (m *UpperDiffInput) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *UpperDiffInput) 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 *UpperDiffInput) XXX_Merge(src proto.Message) {
xxx_messageInfo_UpperDiffInput.Merge(m, src)
}
func (m *UpperDiffInput) XXX_Size() int {
return m.Size()
}
func (m *UpperDiffInput) XXX_DiscardUnknown() {
xxx_messageInfo_UpperDiffInput.DiscardUnknown(m)
}
var xxx_messageInfo_UpperDiffInput proto.InternalMessageInfo
type DiffOp struct {
Lower *LowerDiffInput `protobuf:"bytes,1,opt,name=lower,proto3" json:"lower,omitempty"`
Upper *UpperDiffInput `protobuf:"bytes,2,opt,name=upper,proto3" json:"upper,omitempty"`
}
func (m *DiffOp) Reset() { *m = DiffOp{} }
func (m *DiffOp) String() string { return proto.CompactTextString(m) }
func (*DiffOp) ProtoMessage() {}
func (*DiffOp) Descriptor() ([]byte, []int) {
return fileDescriptor_8de16154b2733812, []int{39}
}
func (m *DiffOp) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *DiffOp) 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 *DiffOp) XXX_Merge(src proto.Message) {
xxx_messageInfo_DiffOp.Merge(m, src)
}
func (m *DiffOp) XXX_Size() int {
return m.Size()
}
func (m *DiffOp) XXX_DiscardUnknown() {
xxx_messageInfo_DiffOp.DiscardUnknown(m)
}
var xxx_messageInfo_DiffOp proto.InternalMessageInfo
func (m *DiffOp) GetLower() *LowerDiffInput {
if m != nil {
return m.Lower
}
return nil
}
func (m *DiffOp) GetUpper() *UpperDiffInput {
if m != nil {
return m.Upper
}
return nil
}
func init() {
proto.RegisterEnum("pb.NetMode", NetMode_name, NetMode_value)
proto.RegisterEnum("pb.SecurityMode", SecurityMode_name, SecurityMode_value)
@ -2565,162 +2692,169 @@ func init() {
proto.RegisterType((*NamedUserOpt)(nil), "pb.NamedUserOpt")
proto.RegisterType((*MergeInput)(nil), "pb.MergeInput")
proto.RegisterType((*MergeOp)(nil), "pb.MergeOp")
proto.RegisterType((*LowerDiffInput)(nil), "pb.LowerDiffInput")
proto.RegisterType((*UpperDiffInput)(nil), "pb.UpperDiffInput")
proto.RegisterType((*DiffOp)(nil), "pb.DiffOp")
}
func init() { proto.RegisterFile("ops.proto", fileDescriptor_8de16154b2733812) }
var fileDescriptor_8de16154b2733812 = []byte{
// 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,
// 2454 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4d, 0x6f, 0x1c, 0xc7,
0xd1, 0xe6, 0x7e, 0xef, 0xd4, 0x92, 0xab, 0x7d, 0x5b, 0xb2, 0xbd, 0xe6, 0xab, 0x50, 0xf4, 0xd8,
0x31, 0x28, 0x4a, 0x22, 0x11, 0x1a, 0xb0, 0x0c, 0x21, 0x08, 0xc2, 0xfd, 0x10, 0xb8, 0xb6, 0xc4,
0x25, 0x7a, 0x25, 0x39, 0x87, 0x00, 0xc2, 0x70, 0xb6, 0x77, 0x39, 0xe0, 0xec, 0xf4, 0x60, 0xa6,
0xd7, 0xe4, 0xe6, 0x90, 0x83, 0xef, 0x01, 0x0c, 0x04, 0x08, 0x72, 0x49, 0xf2, 0x27, 0x72, 0xcd,
0xdd, 0x47, 0x1f, 0x72, 0x30, 0x72, 0x70, 0x02, 0xf9, 0x92, 0x1f, 0x91, 0x00, 0x41, 0x55, 0xf7,
0x7c, 0xec, 0x8a, 0x86, 0xac, 0x24, 0xc8, 0x69, 0x6a, 0x9e, 0x7a, 0xba, 0xba, 0xba, 0xab, 0xba,
0xbb, 0xba, 0xc1, 0x92, 0x61, 0xbc, 0x17, 0x46, 0x52, 0x49, 0x56, 0x0c, 0x4f, 0x37, 0xef, 0x4d,
0x3d, 0x75, 0x36, 0x3f, 0xdd, 0x73, 0xe5, 0x6c, 0x7f, 0x2a, 0xa7, 0x72, 0x9f, 0x54, 0xa7, 0xf3,
0x09, 0xfd, 0xd1, 0x0f, 0x49, 0xba, 0x89, 0xfd, 0xf7, 0x22, 0x14, 0x87, 0x21, 0x7b, 0x07, 0xaa,
0x5e, 0x10, 0xce, 0x55, 0xdc, 0x2e, 0x6c, 0x97, 0x76, 0x1a, 0x07, 0xd6, 0x5e, 0x78, 0xba, 0x37,
0x40, 0x84, 0x1b, 0x05, 0xdb, 0x86, 0xb2, 0xb8, 0x14, 0x6e, 0xbb, 0xb8, 0x5d, 0xd8, 0x69, 0x1c,
0x00, 0x12, 0xfa, 0x97, 0xc2, 0x1d, 0x86, 0x47, 0x6b, 0x9c, 0x34, 0xec, 0x7d, 0xa8, 0xc6, 0x72,
0x1e, 0xb9, 0xa2, 0x5d, 0x22, 0xce, 0x3a, 0x72, 0x46, 0x84, 0x10, 0xcb, 0x68, 0xd1, 0xd2, 0xc4,
0xf3, 0x45, 0xbb, 0x9c, 0x59, 0x7a, 0xe8, 0xf9, 0x9a, 0x43, 0x1a, 0xf6, 0x2e, 0x54, 0x4e, 0xe7,
0x9e, 0x3f, 0x6e, 0x57, 0x88, 0xd2, 0x40, 0x4a, 0x07, 0x01, 0xe2, 0x68, 0x1d, 0x92, 0x66, 0x22,
0x9a, 0x8a, 0x76, 0x35, 0x23, 0x3d, 0x46, 0x40, 0x93, 0x48, 0x87, 0x7d, 0x8d, 0xbd, 0xc9, 0xa4,
0x5d, 0xcb, 0xfa, 0xea, 0x79, 0x93, 0x89, 0xee, 0x0b, 0x35, 0x6c, 0x07, 0xea, 0xa1, 0xef, 0xa8,
0x89, 0x8c, 0x66, 0x6d, 0xc8, 0xfc, 0x3e, 0x31, 0x18, 0x4f, 0xb5, 0xec, 0x3e, 0x34, 0x5c, 0x19,
0xc4, 0x2a, 0x72, 0xbc, 0x40, 0xc5, 0xed, 0x06, 0x91, 0xdf, 0x40, 0xf2, 0xa7, 0x32, 0x3a, 0x17,
0x51, 0x37, 0x53, 0xf2, 0x3c, 0xb3, 0x53, 0x86, 0xa2, 0x0c, 0xed, 0xdf, 0x14, 0xa0, 0x9e, 0x58,
0x65, 0x36, 0xac, 0x1f, 0x46, 0xee, 0x99, 0xa7, 0x84, 0xab, 0xe6, 0x91, 0x68, 0x17, 0xb6, 0x0b,
0x3b, 0x16, 0x5f, 0xc2, 0x58, 0x13, 0x8a, 0xc3, 0x11, 0xcd, 0xb7, 0xc5, 0x8b, 0xc3, 0x11, 0x6b,
0x43, 0xed, 0x99, 0x13, 0x79, 0x4e, 0xa0, 0x68, 0x82, 0x2d, 0x9e, 0xfc, 0xb2, 0x9b, 0x60, 0x0d,
0x47, 0xcf, 0x44, 0x14, 0x7b, 0x32, 0xa0, 0x69, 0xb5, 0x78, 0x06, 0xb0, 0x2d, 0x80, 0xe1, 0xe8,
0xa1, 0x70, 0xd0, 0x68, 0xdc, 0xae, 0x6c, 0x97, 0x76, 0x2c, 0x9e, 0x43, 0xec, 0x5f, 0x42, 0x85,
0x42, 0xcd, 0x3e, 0x86, 0xea, 0xd8, 0x9b, 0x8a, 0x58, 0x69, 0x77, 0x3a, 0x07, 0x5f, 0x7e, 0x73,
0x6b, 0xed, 0x2f, 0xdf, 0xdc, 0xda, 0xcd, 0xe5, 0x94, 0x0c, 0x45, 0xe0, 0xca, 0x40, 0x39, 0x5e,
0x20, 0xa2, 0x78, 0x7f, 0x2a, 0xef, 0xe9, 0x26, 0x7b, 0x3d, 0xfa, 0x70, 0x63, 0x81, 0xdd, 0x86,
0x8a, 0x17, 0x8c, 0xc5, 0x25, 0xf9, 0x5f, 0xea, 0x5c, 0x37, 0xa6, 0x1a, 0xc3, 0xb9, 0x0a, 0xe7,
0x6a, 0x80, 0x2a, 0xae, 0x19, 0xf6, 0xef, 0x0b, 0x50, 0xd5, 0xa9, 0xc4, 0x6e, 0x42, 0x79, 0x26,
0x94, 0x43, 0xfd, 0x37, 0x0e, 0xea, 0x3a, 0xa4, 0xca, 0xe1, 0x84, 0x62, 0x96, 0xce, 0xe4, 0x1c,
0xe7, 0xbe, 0x98, 0x65, 0xe9, 0x63, 0x44, 0xb8, 0x51, 0xb0, 0x1f, 0x42, 0x2d, 0x10, 0xea, 0x42,
0x46, 0xe7, 0x34, 0x47, 0x4d, 0x9d, 0x16, 0xc7, 0x42, 0x3d, 0x96, 0x63, 0xc1, 0x13, 0x1d, 0xbb,
0x0b, 0xf5, 0x58, 0xb8, 0xf3, 0xc8, 0x53, 0x0b, 0x9a, 0xaf, 0xe6, 0x41, 0x8b, 0x92, 0xd5, 0x60,
0x44, 0x4e, 0x19, 0xf6, 0xaf, 0x8a, 0x50, 0x46, 0x37, 0x18, 0x83, 0xb2, 0x13, 0x4d, 0xf5, 0x22,
0xb1, 0x38, 0xc9, 0xac, 0x05, 0x25, 0x11, 0x7c, 0x46, 0x1e, 0x59, 0x1c, 0x45, 0x44, 0xdc, 0x8b,
0xb1, 0x89, 0x11, 0x8a, 0xd8, 0x6e, 0x1e, 0x8b, 0xc8, 0x84, 0x86, 0x64, 0x76, 0x1b, 0xac, 0x30,
0x92, 0x97, 0x8b, 0xe7, 0xd8, 0xba, 0x92, 0x4b, 0x3c, 0x04, 0xfb, 0xc1, 0x67, 0xbc, 0x1e, 0x1a,
0x89, 0xed, 0x02, 0x88, 0x4b, 0x15, 0x39, 0x47, 0x32, 0x56, 0x71, 0xbb, 0x4a, 0x63, 0xa7, 0x54,
0x46, 0x60, 0x70, 0xc2, 0x73, 0x5a, 0xb6, 0x09, 0xf5, 0x33, 0x19, 0xab, 0xc0, 0x99, 0x09, 0x4a,
0x7a, 0x8b, 0xa7, 0xff, 0xcc, 0x86, 0xea, 0xdc, 0xf7, 0x66, 0x9e, 0x6a, 0x5b, 0x99, 0x8d, 0xa7,
0x84, 0x70, 0xa3, 0xc1, 0xc4, 0x74, 0xa7, 0x91, 0x9c, 0x87, 0x27, 0x4e, 0x24, 0x02, 0x45, 0x4b,
0xc2, 0xe2, 0x4b, 0x98, 0x7d, 0x17, 0xaa, 0xba, 0x67, 0x1c, 0x18, 0x4a, 0x26, 0x7d, 0x49, 0xc6,
0xb4, 0x1d, 0x9c, 0x24, 0x69, 0x3b, 0x38, 0xb1, 0x7b, 0x50, 0xd5, 0x7d, 0x20, 0xfb, 0x18, 0xfd,
0x32, 0x6c, 0x94, 0x11, 0x1b, 0xc9, 0x89, 0xd2, 0x69, 0xc2, 0x49, 0x26, 0xab, 0x4e, 0xa4, 0x67,
0xb0, 0xc4, 0x49, 0xb6, 0x7f, 0x5b, 0x82, 0x0a, 0x85, 0x9a, 0xed, 0x60, 0x66, 0x85, 0x73, 0xdd,
0x69, 0xa9, 0xc3, 0x4c, 0x66, 0x01, 0xe5, 0x70, 0x9a, 0x58, 0x98, 0xcf, 0x9b, 0x18, 0x65, 0x5f,
0xb8, 0x4a, 0x46, 0xc6, 0x9f, 0xf4, 0x1f, 0xfb, 0x18, 0x63, 0xa6, 0xeb, 0x28, 0x91, 0xcc, 0xee,
0x40, 0x55, 0x52, 0x7a, 0x52, 0xa0, 0xbe, 0x23, 0x69, 0x0d, 0x05, 0x8d, 0x47, 0xc2, 0x19, 0xcb,
0xc0, 0x5f, 0x50, 0xf8, 0xea, 0x3c, 0xfd, 0x67, 0x77, 0xc0, 0xa2, 0x7c, 0x7c, 0xb2, 0x08, 0xf5,
0xf6, 0xd4, 0x3c, 0xd8, 0x48, 0x73, 0x15, 0x41, 0x9e, 0xe9, 0x71, 0x03, 0x7a, 0x32, 0x0b, 0x27,
0xf1, 0x30, 0x54, 0xed, 0xeb, 0x59, 0x1e, 0x24, 0x18, 0x4f, 0xb5, 0xc8, 0x74, 0x1d, 0xf7, 0x4c,
0x20, 0xf3, 0x46, 0xc6, 0xec, 0x1a, 0x8c, 0xa7, 0x5a, 0x74, 0x20, 0x16, 0x6e, 0x24, 0x14, 0x52,
0xdf, 0x20, 0xea, 0x86, 0x49, 0x70, 0x0d, 0xf2, 0x4c, 0x8f, 0x69, 0x31, 0x1a, 0x1d, 0x21, 0xf3,
0xcd, 0x6c, 0x97, 0xd4, 0x08, 0x37, 0x1a, 0x3d, 0xda, 0x78, 0xee, 0xab, 0x41, 0xaf, 0xfd, 0x96,
0x9e, 0xca, 0xe4, 0xdf, 0xde, 0xca, 0x06, 0x80, 0xd3, 0x1a, 0x7b, 0xbf, 0xd0, 0x21, 0x2e, 0x71,
0x92, 0xed, 0x01, 0xd4, 0x13, 0x17, 0x29, 0x39, 0x7a, 0x26, 0x01, 0x8a, 0x83, 0x1e, 0xbb, 0x07,
0xb5, 0xf8, 0xcc, 0x89, 0xbc, 0x60, 0x4a, 0x11, 0x6a, 0x1e, 0x5c, 0x4f, 0x47, 0x34, 0xd2, 0x38,
0x7a, 0x91, 0x70, 0x6c, 0x09, 0x56, 0x3a, 0x84, 0x97, 0x6c, 0xb5, 0xa0, 0x34, 0xf7, 0xc6, 0x64,
0x67, 0x83, 0xa3, 0x88, 0xc8, 0xd4, 0xd3, 0x79, 0xb4, 0xc1, 0x51, 0x44, 0xff, 0x66, 0x72, 0xac,
0xcf, 0x9e, 0x0d, 0x4e, 0x32, 0x8e, 0x4d, 0x86, 0xca, 0x93, 0x81, 0xe3, 0x27, 0x91, 0x4c, 0xfe,
0x6d, 0x3f, 0x99, 0x9b, 0xff, 0x49, 0x6f, 0xbf, 0x2e, 0x40, 0x3d, 0x39, 0x30, 0x71, 0xdb, 0xf6,
0xc6, 0x22, 0x50, 0xde, 0xc4, 0x13, 0x91, 0xe9, 0x38, 0x87, 0xb0, 0x7b, 0x50, 0x71, 0x94, 0x8a,
0x92, 0xcd, 0xf0, 0xad, 0xfc, 0x69, 0xbb, 0x77, 0x88, 0x9a, 0x7e, 0xa0, 0xa2, 0x05, 0xd7, 0xac,
0xcd, 0x8f, 0x00, 0x32, 0x10, 0x7d, 0x3d, 0x17, 0x0b, 0x63, 0x15, 0x45, 0x76, 0x03, 0x2a, 0x9f,
0x39, 0xfe, 0x5c, 0x98, 0x95, 0xa2, 0x7f, 0x1e, 0x14, 0x3f, 0x2a, 0xd8, 0x7f, 0x2a, 0x42, 0xcd,
0x9c, 0xbe, 0xec, 0x2e, 0xd4, 0xe8, 0xf4, 0x35, 0x1e, 0x5d, 0xbd, 0xfc, 0x12, 0x0a, 0xdb, 0x4f,
0xcb, 0x8a, 0x9c, 0x8f, 0xc6, 0x94, 0x2e, 0x2f, 0x8c, 0x8f, 0x59, 0x91, 0x51, 0x1a, 0x8b, 0x89,
0xa9, 0x1f, 0x9a, 0x74, 0x5a, 0x8b, 0x89, 0x17, 0x78, 0x38, 0x3f, 0x1c, 0x55, 0xec, 0x6e, 0x32,
0xea, 0x32, 0x59, 0x7c, 0x33, 0x6f, 0xf1, 0xe5, 0x41, 0x0f, 0xa0, 0x91, 0xeb, 0xe6, 0x8a, 0x51,
0xbf, 0x97, 0x1f, 0xb5, 0xe9, 0x92, 0xcc, 0xe9, 0xe2, 0x27, 0x9b, 0x85, 0xff, 0x60, 0xfe, 0x3e,
0x04, 0xc8, 0x4c, 0x7e, 0xff, 0xed, 0xcb, 0xfe, 0xbc, 0x04, 0x30, 0x0c, 0xf1, 0xe0, 0x19, 0x3b,
0x74, 0xfa, 0xad, 0x7b, 0xd3, 0x40, 0x46, 0xe2, 0x39, 0x2d, 0x73, 0x6a, 0x5f, 0xe7, 0x0d, 0x8d,
0xd1, 0x8a, 0x61, 0x87, 0xd0, 0x18, 0x8b, 0xd8, 0x8d, 0x3c, 0x4a, 0x28, 0x33, 0xe9, 0xb7, 0x70,
0x4c, 0x99, 0x9d, 0xbd, 0x5e, 0xc6, 0xd0, 0x73, 0x95, 0x6f, 0xc3, 0x0e, 0x60, 0x5d, 0x5c, 0x86,
0x32, 0x52, 0xa6, 0x17, 0x5d, 0xa4, 0x5d, 0xd3, 0xe5, 0x1e, 0xe2, 0xd4, 0x13, 0x6f, 0x88, 0xec,
0x87, 0x39, 0x50, 0x76, 0x9d, 0x50, 0x97, 0x16, 0x8d, 0x83, 0xf6, 0x4a, 0x7f, 0x5d, 0x27, 0xd4,
0x93, 0xd6, 0xf9, 0x00, 0xc7, 0xfa, 0xf9, 0x5f, 0x6f, 0xdd, 0xc9, 0xd5, 0x13, 0x33, 0x79, 0xba,
0xd8, 0xa7, 0x7c, 0x39, 0xf7, 0xd4, 0xfe, 0x5c, 0x79, 0xfe, 0xbe, 0x13, 0x7a, 0x68, 0x0e, 0x1b,
0x0e, 0x7a, 0x9c, 0x4c, 0x6f, 0xfe, 0x04, 0x5a, 0xab, 0x7e, 0xbf, 0x4e, 0x0c, 0x36, 0xef, 0x83,
0x95, 0xfa, 0xf1, 0xaa, 0x86, 0xf5, 0x7c, 0xf0, 0xfe, 0x58, 0x80, 0xaa, 0x5e, 0x55, 0xec, 0x3e,
0x58, 0xbe, 0x74, 0x1d, 0x74, 0x20, 0xa9, 0x93, 0xdf, 0xce, 0x16, 0xdd, 0xde, 0xa3, 0x44, 0xa7,
0x67, 0x35, 0xe3, 0x62, 0x92, 0x79, 0xc1, 0x44, 0x26, 0xab, 0xa0, 0x99, 0x35, 0x1a, 0x04, 0x13,
0xc9, 0xb5, 0x72, 0xf3, 0x13, 0x68, 0x2e, 0x9b, 0xb8, 0xc2, 0xcf, 0x77, 0x97, 0xd3, 0x95, 0xf6,
0xf4, 0xb4, 0x51, 0xde, 0xed, 0xfb, 0x60, 0xa5, 0x38, 0xdb, 0x7d, 0xd9, 0xf1, 0xf5, 0x7c, 0xcb,
0x9c, 0xaf, 0xb6, 0x0f, 0x90, 0xb9, 0x86, 0x9b, 0x15, 0x16, 0xe4, 0x41, 0x76, 0x6a, 0xa7, 0xff,
0x74, 0x82, 0x3a, 0xca, 0x21, 0x57, 0xd6, 0x39, 0xc9, 0x6c, 0x0f, 0x60, 0x9c, 0x2e, 0xd8, 0xef,
0x58, 0xc6, 0x39, 0x86, 0x3d, 0x84, 0x7a, 0xe2, 0x04, 0xdb, 0x86, 0x46, 0x6c, 0x7a, 0xc6, 0xba,
0x11, 0xbb, 0xab, 0xf0, 0x3c, 0x84, 0xf5, 0x5f, 0xe4, 0x04, 0x53, 0xb1, 0x54, 0xff, 0x71, 0x44,
0xb8, 0x51, 0xd8, 0x9f, 0x42, 0x85, 0x00, 0x5c, 0x66, 0xb1, 0x72, 0x22, 0x65, 0x4a, 0x49, 0x5d,
0x5a, 0xc9, 0x98, 0xba, 0xed, 0x94, 0x31, 0x11, 0xb9, 0x26, 0xb0, 0xf7, 0xb0, 0x80, 0x1b, 0x9b,
0x19, 0xbd, 0x8a, 0x87, 0x6a, 0xfb, 0xc7, 0x50, 0x4f, 0x60, 0x1c, 0xf9, 0x23, 0x2f, 0x10, 0xc6,
0x45, 0x92, 0xb1, 0x04, 0xef, 0x9e, 0x39, 0x91, 0xe3, 0x2a, 0xa1, 0x8b, 0x8d, 0x0a, 0xcf, 0x00,
0xfb, 0x5d, 0x68, 0xe4, 0x56, 0x0f, 0xa6, 0xdb, 0x33, 0x0a, 0xa3, 0x5e, 0xc3, 0xfa, 0xc7, 0xfe,
0x03, 0x5e, 0x10, 0x92, 0x9a, 0xef, 0x07, 0x00, 0x67, 0x4a, 0x85, 0xcf, 0xa9, 0x08, 0x34, 0x73,
0x6f, 0x21, 0x42, 0x0c, 0x76, 0x0b, 0x1a, 0xf8, 0x13, 0x1b, 0xbd, 0xce, 0x77, 0x6a, 0x11, 0x6b,
0xc2, 0xff, 0x83, 0x35, 0x49, 0x9b, 0x97, 0x4c, 0xe8, 0x92, 0xd6, 0x6f, 0x43, 0x3d, 0x90, 0x46,
0xa7, 0x6b, 0xd2, 0x5a, 0x20, 0xd3, 0x76, 0x8e, 0xef, 0x1b, 0x5d, 0x45, 0xb7, 0x73, 0x7c, 0x9f,
0x94, 0xf6, 0x1d, 0xf8, 0xbf, 0x97, 0xae, 0x3a, 0xec, 0x4d, 0xa8, 0x4e, 0x3c, 0x5f, 0xd1, 0x89,
0x80, 0x35, 0xb0, 0xf9, 0xb3, 0xff, 0x59, 0x00, 0xc8, 0xc2, 0x8e, 0xc9, 0x8c, 0x5b, 0x3b, 0x72,
0xd6, 0xf5, 0x56, 0xee, 0x43, 0x7d, 0x66, 0x36, 0x09, 0x13, 0xd0, 0x9b, 0xcb, 0xa9, 0xb2, 0x97,
0xec, 0x21, 0x7a, 0xfb, 0x38, 0x30, 0xdb, 0xc7, 0xeb, 0x5c, 0x47, 0xd2, 0x1e, 0xa8, 0xca, 0xc9,
0xdf, 0x4e, 0x21, 0x5b, 0x85, 0xdc, 0x68, 0x36, 0x3f, 0x81, 0x8d, 0xa5, 0x2e, 0xbf, 0xe7, 0x81,
0x91, 0x6d, 0x76, 0xf9, 0x25, 0x78, 0x00, 0x55, 0x7d, 0xad, 0x65, 0x3b, 0x50, 0x73, 0x5c, 0xbd,
0xfa, 0x72, 0x3b, 0x00, 0x2a, 0x0f, 0x09, 0xe6, 0x89, 0xda, 0xfe, 0x73, 0x11, 0x20, 0xc3, 0x5f,
0xa3, 0xd4, 0x7d, 0x00, 0xcd, 0x58, 0xb8, 0x32, 0x18, 0x3b, 0xd1, 0x82, 0xb4, 0xe6, 0xde, 0x75,
0x55, 0x93, 0x15, 0x66, 0xae, 0xec, 0x2d, 0xbd, 0xba, 0xec, 0xdd, 0x81, 0xb2, 0x2b, 0xc3, 0x85,
0x39, 0x17, 0xd8, 0xf2, 0x40, 0xba, 0x32, 0x5c, 0xe0, 0xc5, 0x1a, 0x19, 0x6c, 0x0f, 0xaa, 0xb3,
0x73, 0xba, 0xe8, 0xeb, 0xdb, 0xcd, 0x8d, 0x65, 0xee, 0xe3, 0x73, 0x94, 0x8f, 0xd6, 0xb8, 0x61,
0xb1, 0x3b, 0x50, 0x99, 0x9d, 0x8f, 0xbd, 0xc8, 0xdc, 0xe7, 0xaf, 0xaf, 0xd2, 0x7b, 0x5e, 0x44,
0xf7, 0x7a, 0xe4, 0x30, 0x1b, 0x8a, 0xd1, 0xcc, 0xdc, 0xea, 0x5b, 0x2b, 0xb3, 0x39, 0x3b, 0x5a,
0xe3, 0xc5, 0x68, 0xd6, 0xa9, 0x43, 0x55, 0xcf, 0xab, 0xfd, 0x8f, 0x12, 0x34, 0x97, 0xbd, 0xc4,
0xc8, 0xc6, 0x91, 0x9b, 0x44, 0x36, 0x8e, 0xdc, 0xf4, 0x46, 0x50, 0xcc, 0xdd, 0x08, 0x6c, 0xa8,
0xc8, 0x8b, 0x40, 0x44, 0xf9, 0x17, 0x8d, 0xee, 0x99, 0xbc, 0x08, 0xb0, 0x2a, 0xd5, 0xaa, 0xa5,
0x22, 0xaf, 0x62, 0x8a, 0xbc, 0xf7, 0x60, 0x63, 0x22, 0x7d, 0x5f, 0x5e, 0x8c, 0x16, 0x33, 0xdf,
0x0b, 0xce, 0x4d, 0xa5, 0xb7, 0x0c, 0xb2, 0x1d, 0xb8, 0x36, 0xf6, 0x22, 0x74, 0xa7, 0x2b, 0x03,
0x25, 0x02, 0xba, 0xdc, 0x21, 0x6f, 0x15, 0x66, 0x1f, 0xc3, 0xb6, 0xa3, 0x94, 0x98, 0x85, 0xea,
0x69, 0x10, 0x3a, 0xee, 0x79, 0x4f, 0xba, 0xb4, 0x0a, 0x67, 0xa1, 0xa3, 0xbc, 0x53, 0xcf, 0xc7,
0x7b, 0x6c, 0x8d, 0x9a, 0xbe, 0x92, 0xc7, 0xde, 0x87, 0xa6, 0x1b, 0x09, 0x47, 0x89, 0x9e, 0x88,
0xd5, 0x89, 0xa3, 0xce, 0xda, 0x75, 0x6a, 0xb9, 0x82, 0xe2, 0x18, 0x1c, 0xf4, 0xf6, 0x53, 0xcf,
0x1f, 0xbb, 0x78, 0x1d, 0xb3, 0xf4, 0x18, 0x96, 0x40, 0xb6, 0x07, 0x8c, 0x80, 0xfe, 0x2c, 0x54,
0x8b, 0x94, 0x0a, 0x44, 0xbd, 0x42, 0x83, 0xfb, 0xa4, 0xf2, 0x66, 0x22, 0x56, 0xce, 0x2c, 0xa4,
0x27, 0x94, 0x12, 0xcf, 0x00, 0x76, 0x1b, 0x5a, 0x5e, 0xe0, 0xfa, 0xf3, 0xb1, 0x78, 0x1e, 0xe2,
0x40, 0xa2, 0x20, 0x6e, 0xaf, 0xd3, 0xae, 0x72, 0xcd, 0xe0, 0x27, 0x06, 0x46, 0xaa, 0xb8, 0x5c,
0xa1, 0x6e, 0x68, 0xaa, 0xc1, 0x13, 0xaa, 0xfd, 0x45, 0x01, 0x5a, 0xab, 0x89, 0x87, 0x61, 0x0b,
0x71, 0xf0, 0xe6, 0x32, 0x8a, 0x72, 0x1a, 0xca, 0x62, 0x2e, 0x94, 0xc9, 0x31, 0x57, 0xca, 0x1d,
0x73, 0x69, 0x5a, 0x94, 0xbf, 0x3b, 0x2d, 0x96, 0x06, 0x5a, 0x59, 0x19, 0xa8, 0xfd, 0xbb, 0x02,
0x5c, 0x5b, 0x49, 0xee, 0xef, 0xed, 0xd1, 0x36, 0x34, 0x66, 0xce, 0xb9, 0xd0, 0x97, 0xf1, 0x98,
0x1c, 0xab, 0xf3, 0x3c, 0xf4, 0x5f, 0xf0, 0x2f, 0x80, 0xf5, 0xfc, 0x8a, 0xba, 0xd2, 0xb7, 0x24,
0x41, 0x8e, 0xa5, 0x7a, 0x28, 0xe7, 0xe6, 0x08, 0x4d, 0x12, 0x24, 0x01, 0x5f, 0x4e, 0xa3, 0xd2,
0x15, 0x69, 0x64, 0x1f, 0x43, 0x3d, 0x71, 0x90, 0xdd, 0x32, 0xaf, 0x25, 0x85, 0xec, 0x5d, 0xef,
0x69, 0x2c, 0x22, 0xf4, 0x5d, 0x3f, 0x9d, 0xbc, 0x03, 0x15, 0x7a, 0x8e, 0x30, 0x7b, 0xf0, 0x12,
0x43, 0x6b, 0xec, 0x11, 0xd4, 0x0c, 0xc2, 0x76, 0xa1, 0x7a, 0xba, 0x48, 0xdf, 0x1d, 0xcc, 0x76,
0x81, 0xff, 0x63, 0xc3, 0xc0, 0x3d, 0x48, 0x33, 0xd8, 0x0d, 0x28, 0x9f, 0x2e, 0x06, 0x3d, 0x7d,
0xab, 0xc3, 0x9d, 0x0c, 0xff, 0x3a, 0x55, 0xed, 0x90, 0xfd, 0x08, 0xd6, 0xf3, 0xed, 0x70, 0x52,
0x72, 0x95, 0x11, 0xc9, 0xd9, 0x96, 0x5d, 0x7c, 0x55, 0x79, 0xff, 0x21, 0x00, 0x3d, 0x57, 0xbe,
0xee, 0xb5, 0xe0, 0x47, 0x50, 0x33, 0xcf, 0x9c, 0xec, 0xfd, 0x95, 0x67, 0xdb, 0x66, 0xfa, 0x06,
0xba, 0xf4, 0x76, 0x6b, 0x3f, 0xc0, 0xd2, 0xf2, 0x42, 0x44, 0x3d, 0x6f, 0x32, 0x79, 0xdd, 0xee,
0x1e, 0x40, 0xf3, 0x69, 0x18, 0xfe, 0x7b, 0x6d, 0x7f, 0x0e, 0x55, 0xfd, 0xda, 0x8a, 0x6d, 0x7c,
0xf4, 0xc0, 0xc4, 0x80, 0xe9, 0xf2, 0x33, 0xef, 0x12, 0xd7, 0x04, 0x64, 0xce, 0xb1, 0x3f, 0x13,
0x5c, 0x62, 0x2e, 0x3b, 0xc0, 0x35, 0x61, 0x77, 0x07, 0x6a, 0xe6, 0x61, 0x8f, 0x59, 0x50, 0x79,
0x7a, 0x3c, 0xea, 0x3f, 0x69, 0xad, 0xb1, 0x3a, 0x94, 0x8f, 0x86, 0xa3, 0x27, 0xad, 0x02, 0x4a,
0xc7, 0xc3, 0xe3, 0x7e, 0xab, 0xb8, 0x7b, 0x1b, 0xd6, 0xf3, 0x4f, 0x7b, 0xac, 0x01, 0xb5, 0xd1,
0xe1, 0x71, 0xaf, 0x33, 0xfc, 0x59, 0x6b, 0x8d, 0xad, 0x43, 0x7d, 0x70, 0x3c, 0xea, 0x77, 0x9f,
0xf2, 0x7e, 0xab, 0xb0, 0xfb, 0x53, 0xb0, 0xd2, 0x57, 0x1a, 0xb4, 0xd0, 0x19, 0x1c, 0xf7, 0x5a,
0x6b, 0x0c, 0xa0, 0x3a, 0xea, 0x77, 0x79, 0x1f, 0xed, 0xd6, 0xa0, 0x34, 0x1a, 0x1d, 0xb5, 0x8a,
0xd8, 0x6b, 0xf7, 0xb0, 0x7b, 0xd4, 0x6f, 0x95, 0x50, 0x7c, 0xf2, 0xf8, 0xe4, 0xe1, 0xa8, 0x55,
0xde, 0xfd, 0x10, 0xae, 0xad, 0xbc, 0x5f, 0x50, 0xeb, 0xa3, 0x43, 0xde, 0x47, 0x4b, 0x0d, 0xa8,
0x9d, 0xf0, 0xc1, 0xb3, 0xc3, 0x27, 0xfd, 0x56, 0x01, 0x15, 0x8f, 0x86, 0xdd, 0x4f, 0xfa, 0xbd,
0x56, 0xb1, 0x73, 0xf3, 0xcb, 0x17, 0x5b, 0x85, 0xaf, 0x5e, 0x6c, 0x15, 0xbe, 0x7e, 0xb1, 0x55,
0xf8, 0xdb, 0x8b, 0xad, 0xc2, 0x17, 0xdf, 0x6e, 0xad, 0x7d, 0xf5, 0xed, 0xd6, 0xda, 0xd7, 0xdf,
0x6e, 0xad, 0x9d, 0x56, 0xe9, 0xbd, 0xfe, 0x83, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x33, 0xe0,
0x88, 0xd4, 0xef, 0x17, 0x00, 0x00,
}
func (m *Op) Marshal() (dAtA []byte, err error) {
@ -2898,6 +3032,27 @@ func (m *Op_Merge) MarshalToSizedBuffer(dAtA []byte) (int, error) {
}
return len(dAtA) - i, nil
}
func (m *Op_Diff) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *Op_Diff) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
if m.Diff != nil {
{
size, err := m.Diff.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintOps(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x3a
}
return len(dAtA) - i, nil
}
func (m *Platform) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -4967,6 +5122,109 @@ func (m *MergeOp) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LowerDiffInput) 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 *LowerDiffInput) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LowerDiffInput) 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 *UpperDiffInput) 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 *UpperDiffInput) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *UpperDiffInput) 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 *DiffOp) 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 *DiffOp) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *DiffOp) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if m.Upper != nil {
{
size, err := m.Upper.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintOps(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
}
if m.Lower != nil {
{
size, err := m.Lower.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
@ -5064,6 +5322,18 @@ func (m *Op_Merge) Size() (n int) {
}
return n
}
func (m *Op_Diff) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Diff != nil {
l = m.Diff.Size()
n += 1 + l + sovOps(uint64(l))
}
return n
}
func (m *Platform) Size() (n int) {
if m == nil {
return 0
@ -5954,6 +6224,47 @@ func (m *MergeOp) Size() (n int) {
return n
}
func (m *LowerDiffInput) 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 *UpperDiffInput) 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 *DiffOp) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Lower != nil {
l = m.Lower.Size()
n += 1 + l + sovOps(uint64(l))
}
if m.Upper != nil {
l = m.Upper.Size()
n += 1 + l + sovOps(uint64(l))
}
return n
}
func sovOps(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}
@ -6198,6 +6509,41 @@ func (m *Op) Unmarshal(dAtA []byte) error {
}
m.Op = &Op_Merge{v}
iNdEx = postIndex
case 7:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Diff", 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 := &DiffOp{}
if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
m.Op = &Op_Diff{v}
iNdEx = postIndex
case 10:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Platform", wireType)
@ -12219,6 +12565,266 @@ func (m *MergeOp) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LowerDiffInput) 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: LowerDiffInput: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LowerDiffInput: 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 *UpperDiffInput) 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: UpperDiffInput: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: UpperDiffInput: 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 *DiffOp) 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: DiffOp: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: DiffOp: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Lower", 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
}
if m.Lower == nil {
m.Lower = &LowerDiffInput{}
}
if err := m.Lower.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Upper", 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
}
if m.Upper == nil {
m.Upper = &UpperDiffInput{}
}
if err := m.Upper.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

View File

@ -18,6 +18,7 @@ message Op {
FileOp file = 4;
BuildOp build = 5;
MergeOp merge = 6;
DiffOp diff = 7;
}
Platform platform = 10;
WorkerConstraints constraints = 11;
@ -372,3 +373,16 @@ message MergeInput {
message MergeOp {
repeated MergeInput inputs = 1;
}
message LowerDiffInput {
int64 input = 1 [(gogoproto.customtype) = "InputIndex", (gogoproto.nullable) = false];
}
message UpperDiffInput {
int64 input = 1 [(gogoproto.customtype) = "InputIndex", (gogoproto.nullable) = false];
}
message DiffOp {
LowerDiffInput lower = 1;
UpperDiffInput upper = 2;
}

View File

@ -221,6 +221,11 @@ func (w *Worker) LoadRef(ctx context.Context, id string, hidden bool) (cache.Imm
if hidden {
opts = append(opts, cache.NoUpdateLastUsed)
}
if id == "" {
// results can have nil refs if they are optimized out to be equal to scratch,
// i.e. Diff(A,A) == scratch
return nil, nil
}
ref, err := w.CacheMgr.Get(ctx, id, opts...)
var needsRemoteProviders cache.NeedsRemoteProvidersError
@ -269,6 +274,8 @@ func (w *Worker) ResolveOp(v solver.Vertex, s frontend.FrontendLLBBridge, sm *se
return ops.NewBuildOp(v, op, s, w)
case *pb.Op_Merge:
return ops.NewMergeOp(v, op, w)
case *pb.Op_Diff:
return ops.NewDiffOp(v, op, w)
default:
return nil, errors.Errorf("no support for %T", op)
}

View File

@ -75,7 +75,9 @@ func (s *cacheResultStorage) LoadRemotes(ctx context.Context, res solver.CacheRe
if err != nil {
return nil, err
}
if ref != nil {
defer ref.Release(context.TODO())
}
wref := WorkerRef{ref, w}
all := true // load as many compression blobs as possible
if compressionopt == nil {

View File

@ -34,6 +34,9 @@ func (wr *WorkerRef) GetRemotes(ctx context.Context, createIfNeeded bool, compre
}); ok {
return w.GetRemotes(ctx, wr.ImmutableRef, createIfNeeded, compressionopt, all, g)
}
if wr.ImmutableRef == nil {
return nil, nil
}
return wr.ImmutableRef.GetRemotes(ctx, createIfNeeded, compressionopt, all, g)
}