Plumb op metadata to recreate failed ops with gateway exec

Signed-off-by: Edgar Lee <edgarl@netflix.com>
v0.8
Edgar Lee 2020-10-28 14:07:44 -07:00
parent 6861f17f15
commit 7ce58c31ba
25 changed files with 2935 additions and 252 deletions

View File

@ -44,6 +44,9 @@ func TestClientGatewayIntegration(t *testing.T) {
testClientGatewayContainerExecTty,
testClientSlowCacheRootfsRef,
testClientGatewayContainerPlatformPATH,
testClientGatewayExecError,
testClientGatewaySlowCacheExecError,
testClientGatewayFileActionError,
}, integration.WithMirroredImages(integration.OfficialImages("busybox:latest")))
}
@ -935,6 +938,7 @@ func testClientGatewayContainerExecTty(t *testing.T, sb integration.Sandbox) {
func testClientSlowCacheRootfsRef(t *testing.T, sb integration.Sandbox) {
requiresLinux(t)
ctx := context.TODO()
c, err := New(ctx, sb.Address())
@ -1069,6 +1073,278 @@ func testClientGatewayContainerPlatformPATH(t *testing.T, sb integration.Sandbox
checkAllReleasable(t, c, sb, true)
}
// testClientGatewayExecError is testing gateway exec to recreate the container
// process for a failed execop.
func testClientGatewayExecError(t *testing.T, sb integration.Sandbox) {
requiresLinux(t)
ctx := context.TODO()
c, err := New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
id := identity.NewID()
st := llb.Image("busybox:latest").Run(
llb.Dir("/src"),
llb.AddMount("/src", llb.Scratch()),
llb.Shlexf("sh -c \"echo %s > output && fail\"", id),
).Root()
def, err := st.Marshal(ctx)
require.NoError(t, err)
b := func(ctx context.Context, c client.Client) (*client.Result, error) {
_, solveErr := c.Solve(ctx, client.SolveRequest{
Evaluate: true,
Definition: def.ToPB(),
})
require.Error(t, solveErr)
var se *errdefs.SolveError
require.True(t, errors.As(solveErr, &se))
solveExec, ok := se.Solve.Op.Op.(*pb.Op_Exec)
require.True(t, ok)
exec := solveExec.Exec
var mounts []client.Mount
for _, mnt := range exec.Mounts {
mounts = append(mounts, client.Mount{
Selector: mnt.Selector,
Dest: mnt.Dest,
ResultID: se.Solve.OutputIDs[mnt.Output],
Readonly: mnt.Readonly,
MountType: mnt.MountType,
CacheOpt: mnt.CacheOpt,
SecretOpt: mnt.SecretOpt,
SSHOpt: mnt.SSHOpt,
})
}
ctr, err := c.NewContainer(ctx, client.NewContainerRequest{
Mounts: mounts,
NetMode: exec.Network,
})
require.NoError(t, err)
defer ctr.Release(ctx)
meta := exec.Meta
output := bytes.NewBuffer(nil)
proc, err := ctr.Start(ctx, client.StartRequest{
Args: []string{"cat", "output"},
Env: meta.Env,
User: meta.User,
Cwd: meta.Cwd,
Stdout: &nopCloser{output},
SecurityMode: exec.Security,
})
require.NoError(t, err)
err = proc.Wait()
require.NoError(t, err)
require.Equal(t, id, strings.TrimSpace(output.String()))
return nil, solveErr
}
_, err = c.Build(ctx, SolveOpt{}, "buildkit_test", b, nil)
require.Error(t, err)
checkAllReleasable(t, c, sb, true)
}
// testClientGatewaySlowCacheExecError is testing gateway exec into the ref
// that failed to mount during an execop.
func testClientGatewaySlowCacheExecError(t *testing.T, sb integration.Sandbox) {
requiresLinux(t)
ctx := context.TODO()
c, err := New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
id := identity.NewID()
input := llb.Scratch().File(
llb.Mkdir("/found", 0700).
Mkfile("/found/data", 0600, []byte(id)),
)
st := llb.Image("busybox:latest").Run(
llb.Shlexf("echo hello"),
// Only readonly mounts trigger slow cache errors.
llb.AddMount("/src", input, llb.SourcePath("/notfound"), llb.Readonly),
).Root()
def, err := st.Marshal(ctx)
require.NoError(t, err)
b := func(ctx context.Context, c client.Client) (*client.Result, error) {
_, solveErr := c.Solve(ctx, client.SolveRequest{
Evaluate: true,
Definition: def.ToPB(),
})
require.Error(t, solveErr)
var se *errdefs.SolveError
require.True(t, errors.As(solveErr, &se))
_, ok := se.Solve.Op.Op.(*pb.Op_Exec)
require.True(t, ok)
_, ok = se.Solve.Subject.(*errdefs.Solve_Cache)
require.True(t, ok)
// Slow cache errors should only have exactly one input and no outputs.
require.Len(t, se.Solve.InputIDs, 1)
require.Len(t, se.Solve.OutputIDs, 0)
st := llb.Image("busybox:latest")
def, err := st.Marshal(ctx)
require.NoError(t, err)
res, err := c.Solve(ctx, client.SolveRequest{
Definition: def.ToPB(),
})
require.NoError(t, err)
ref, err := res.SingleRef()
require.NoError(t, err)
ctr, err := c.NewContainer(ctx, client.NewContainerRequest{
Mounts: []client.Mount{{
Dest: "/",
MountType: pb.MountType_BIND,
Ref: ref,
}, {
Dest: "/problem",
MountType: pb.MountType_BIND,
ResultID: se.Solve.InputIDs[0],
}},
})
require.NoError(t, err)
defer ctr.Release(ctx)
output := bytes.NewBuffer(nil)
proc, err := ctr.Start(ctx, client.StartRequest{
Args: []string{"cat", "/problem/found/data"},
Stdout: &nopCloser{output},
})
require.NoError(t, err)
err = proc.Wait()
require.NoError(t, err)
require.Equal(t, id, strings.TrimSpace(output.String()))
return nil, solveErr
}
_, err = c.Build(ctx, SolveOpt{}, "buildkit_test", b, nil)
require.Error(t, err)
checkAllReleasable(t, c, sb, true)
}
// testClientGatewayFileActionError is testing gateway exec into the modified
// mount of a failed fileop action during a solve.
func testClientGatewayFileActionError(t *testing.T, sb integration.Sandbox) {
requiresLinux(t)
ctx := context.TODO()
c, err := New(ctx, sb.Address())
require.NoError(t, err)
defer c.Close()
id := identity.NewID()
st := llb.Scratch().File(
llb.Mkdir("/found", 0700).
Mkfile("/found/foo", 0600, []byte(id)).
Mkfile("/notfound/foo", 0600, []byte(id)),
)
def, err := st.Marshal(ctx)
require.NoError(t, err)
b := func(ctx context.Context, c client.Client) (*client.Result, error) {
_, solveErr := c.Solve(ctx, client.SolveRequest{
Evaluate: true,
Definition: def.ToPB(),
})
require.Error(t, solveErr)
var se *errdefs.SolveError
require.True(t, errors.As(solveErr, &se))
// There are no inputs because rootfs is scratch.
require.Len(t, se.Solve.InputIDs, 0)
// There is one output for the mutable mount used for the fileop actions.
require.Len(t, se.Solve.OutputIDs, 1)
op, ok := se.Solve.Op.Op.(*pb.Op_File)
require.True(t, ok)
subject, ok := se.Solve.Subject.(*errdefs.Solve_File)
require.True(t, ok)
idx := subject.File.Index
require.Greater(t, len(op.File.Actions), int(idx))
// Verify the index is pointing to the action that failed.
action := op.File.Actions[subject.File.Index]
mkfile, ok := action.Action.(*pb.FileAction_Mkfile)
require.True(t, ok)
require.Equal(t, mkfile.Mkfile.Path, "/notfound/foo")
st := llb.Image("busybox:latest")
def, err := st.Marshal(ctx)
require.NoError(t, err)
res, err := c.Solve(ctx, client.SolveRequest{
Definition: def.ToPB(),
})
require.NoError(t, err)
ref, err := res.SingleRef()
require.NoError(t, err)
ctr, err := c.NewContainer(ctx, client.NewContainerRequest{
Mounts: []client.Mount{{
Dest: "/",
MountType: pb.MountType_BIND,
Ref: ref,
}, {
Dest: "/problem",
MountType: pb.MountType_BIND,
ResultID: se.Solve.OutputIDs[0],
}},
})
require.NoError(t, err)
defer ctr.Release(ctx)
// Verify that other actions have succeeded.
output := bytes.NewBuffer(nil)
proc, err := ctr.Start(ctx, client.StartRequest{
Args: []string{"cat", "/problem/found/foo"},
Cwd: "/",
Stdout: &nopCloser{output},
})
require.NoError(t, err)
err = proc.Wait()
require.NoError(t, err)
require.Equal(t, id, strings.TrimSpace(output.String()))
return nil, solveErr
}
_, err = c.Build(ctx, SolveOpt{}, "buildkit_test", b, nil)
require.Error(t, err)
checkAllReleasable(t, c, sb, true)
}
type nopCloser struct {
io.Writer
}

View File

@ -34,6 +34,7 @@ type NewContainerRequest struct {
type Mount struct {
Selector string
Dest string
ResultID string
Ref Reference
Readonly bool
MountType pb.MountType
@ -104,6 +105,7 @@ type StatRequest struct {
// SolveRequest is same as frontend.SolveRequest but avoiding dependency
type SolveRequest struct {
Evaluate bool
Definition *pb.Definition
Frontend string
FrontendOpt map[string]string

View File

@ -13,7 +13,6 @@ import (
"github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/snapshot"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/llbsolver/mounts"
opspb "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/stack"
@ -40,7 +39,7 @@ type Mount struct {
Selector string
Readonly bool
MountType opspb.MountType
RefProxy solver.ResultProxy
WorkerRef *worker.WorkerRef
CacheOpt *opspb.CacheOpt
SecretOpt *opspb.SecretOpt
SSHOpt *opspb.SSHOpt
@ -93,22 +92,13 @@ func NewContainer(ctx context.Context, e executor.Executor, sm *session.Manager,
mnts := req.Mounts
for i, m := range mnts {
if m.Dest == opspb.RootMount && m.RefProxy != nil {
res, err := m.RefProxy.Result(ctx)
if err != nil {
return nil, stack.Enable(err)
}
workerRef, ok := res.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid reference for exec %T", res.Sys())
}
if m.Dest == opspb.RootMount && m.WorkerRef != nil {
name := fmt.Sprintf("container %s", req.ContainerID)
mm = mounts.NewMountManager(name, workerRef.Worker.CacheManager(), sm, workerRef.Worker.MetadataStore())
mm = mounts.NewMountManager(name, m.WorkerRef.Worker.CacheManager(), sm, m.WorkerRef.Worker.MetadataStore())
ctr.rootFS = mountWithSession(workerRef.ImmutableRef, g)
ctr.rootFS = mountWithSession(m.WorkerRef.ImmutableRef, g)
if !m.Readonly {
ref, err := makeMutable(workerRef.Worker, workerRef.ImmutableRef)
ref, err := makeMutable(m.WorkerRef.Worker, m.WorkerRef.ImmutableRef)
if err != nil {
return nil, stack.Enable(err)
}
@ -128,20 +118,13 @@ func NewContainer(ctx context.Context, e executor.Executor, sm *session.Manager,
for _, m := range mnts {
var ref cache.ImmutableRef
var mountable cache.Mountable
if m.RefProxy != nil {
res, err := m.RefProxy.Result(ctx)
if err != nil {
return nil, stack.Enable(err)
}
workerRef, ok := res.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid reference for exec %T", res.Sys())
}
ref = workerRef.ImmutableRef
if m.WorkerRef != nil {
ref = m.WorkerRef.ImmutableRef
mountable = ref
if !m.Readonly {
mountable, err = makeMutable(workerRef.Worker, ref)
var err error
mountable, err = makeMutable(m.WorkerRef.Worker, ref)
if err != nil {
return nil, stack.Enable(err)
}

View File

@ -15,6 +15,8 @@ import (
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/snapshot"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/errdefs"
llberrdefs "github.com/moby/buildkit/solver/llbsolver/errdefs"
opspb "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/apicaps"
"github.com/moby/buildkit/worker"
@ -31,23 +33,26 @@ func llbBridgeToGatewayClient(ctx context.Context, llbBridge frontend.FrontendLL
sm: sm,
workerInfos: workerInfos,
final: map[*ref]struct{}{},
workerRefByID: make(map[string]*worker.WorkerRef),
}, nil
}
type bridgeClient struct {
frontend.FrontendLLBBridge
mu sync.Mutex
opts map[string]string
inputs map[string]*opspb.Definition
final map[*ref]struct{}
sid string
sm *session.Manager
refs []*ref
workerInfos []clienttypes.WorkerInfo
mu sync.Mutex
opts map[string]string
inputs map[string]*opspb.Definition
final map[*ref]struct{}
sid string
sm *session.Manager
refs []*ref
workerRefByID map[string]*worker.WorkerRef
workerInfos []clienttypes.WorkerInfo
}
func (c *bridgeClient) Solve(ctx context.Context, req client.SolveRequest) (*client.Result, error) {
res, err := c.FrontendLLBBridge.Solve(ctx, frontend.SolveRequest{
Evaluate: req.Evaluate,
Definition: req.Definition,
Frontend: req.Frontend,
FrontendOpt: req.FrontendOpt,
@ -55,13 +60,13 @@ func (c *bridgeClient) Solve(ctx context.Context, req client.SolveRequest) (*cli
CacheImports: req.CacheImports,
}, c.sid)
if err != nil {
return nil, err
return nil, c.wrapSolveError(err)
}
cRes := &client.Result{}
c.mu.Lock()
for k, r := range res.Refs {
rr, err := newRef(r, session.NewGroup(c.sid))
rr, err := c.newRef(r, session.NewGroup(c.sid))
if err != nil {
return nil, err
}
@ -69,7 +74,7 @@ func (c *bridgeClient) Solve(ctx context.Context, req client.SolveRequest) (*cli
cRes.AddRef(k, rr)
}
if r := res.Ref; r != nil {
rr, err := newRef(r, session.NewGroup(c.sid))
rr, err := c.newRef(r, session.NewGroup(c.sid))
if err != nil {
return nil, err
}
@ -113,6 +118,64 @@ func (c *bridgeClient) Inputs(ctx context.Context) (map[string]llb.State, error)
return inputs, nil
}
func (c *bridgeClient) wrapSolveError(solveErr error) error {
var (
ee *llberrdefs.ExecError
fae *llberrdefs.FileActionError
sce *solver.SlowCacheError
inputIDs []string
outputIDs []string
subject errdefs.IsSolve_Subject
)
if errors.As(solveErr, &ee) {
var err error
inputIDs, err = c.registerResultIDs(ee.Inputs...)
if err != nil {
return err
}
outputIDs, err = c.registerResultIDs(ee.Outputs...)
if err != nil {
return err
}
}
if errors.As(solveErr, &fae) {
subject = &errdefs.Solve_File{
File: &errdefs.FileAction{
Index: int64(fae.Index),
},
}
}
if errors.As(solveErr, &sce) {
var err error
inputIDs, err = c.registerResultIDs(sce.Result)
if err != nil {
return err
}
subject = &errdefs.Solve_Cache{
Cache: &errdefs.ContentCache{
Index: int64(sce.Index),
},
}
}
return errdefs.WithSolveError(solveErr, subject, inputIDs, outputIDs)
}
func (c *bridgeClient) registerResultIDs(results ...solver.Result) (ids []string, err error) {
c.mu.Lock()
defer c.mu.Unlock()
ids = make([]string, len(results))
for i, res := range results {
workerRef, ok := res.Sys().(*worker.WorkerRef)
if !ok {
return ids, errors.Errorf("unexpected type for result, got %T", res.Sys())
}
ids[i] = workerRef.ID()
c.workerRefByID[workerRef.ID()] = workerRef
}
return ids, nil
}
func (c *bridgeClient) toFrontendResult(r *client.Result) (*frontend.Result, error) {
if r == nil {
return nil, nil
@ -145,6 +208,10 @@ func (c *bridgeClient) toFrontendResult(r *client.Result) (*frontend.Result, err
}
func (c *bridgeClient) discard(err error) {
for id, workerRef := range c.workerRefByID {
workerRef.ImmutableRef.Release(context.TODO())
delete(c.workerRefByID, id)
}
for _, r := range c.refs {
if r != nil {
if _, ok := c.final[r]; !ok || err != nil {
@ -161,20 +228,35 @@ func (c *bridgeClient) NewContainer(ctx context.Context, req client.NewContainer
}
for _, m := range req.Mounts {
var refProxy solver.ResultProxy
var workerRef *worker.WorkerRef
if m.Ref != nil {
var ok bool
refProxy, ok = m.Ref.(*ref)
refProxy, ok := m.Ref.(*ref)
if !ok {
return nil, errors.Errorf("unexpected Ref type: %T", m.Ref)
}
res, err := refProxy.Result(ctx)
if err != nil {
return nil, err
}
workerRef, ok = res.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", res.Sys())
}
} else if m.ResultID != "" {
var ok bool
workerRef, ok = c.workerRefByID[m.ResultID]
if !ok {
return nil, errors.Errorf("failed to find ref %s for %q mount", m.ResultID, m.Dest)
}
}
ctrReq.Mounts = append(ctrReq.Mounts, gateway.Mount{
Dest: m.Dest,
Selector: m.Selector,
Readonly: m.Readonly,
MountType: m.MountType,
RefProxy: refProxy,
WorkerRef: workerRef,
CacheOpt: m.CacheOpt,
SecretOpt: m.SecretOpt,
SSHOpt: m.SSHOpt,
@ -192,10 +274,11 @@ func (c *bridgeClient) NewContainer(ctx context.Context, req client.NewContainer
type ref struct {
solver.ResultProxy
session session.Group
c *bridgeClient
}
func newRef(r solver.ResultProxy, s session.Group) (*ref, error) {
return &ref{ResultProxy: r, session: s}, nil
func (c *bridgeClient) newRef(r solver.ResultProxy, s session.Group) (*ref, error) {
return &ref{ResultProxy: r, session: s, c: c}, nil
}
func (r *ref) ToState() (st llb.State, err error) {
@ -246,7 +329,7 @@ func (r *ref) StatFile(ctx context.Context, req client.StatRequest) (*fstypes.St
func (r *ref) getMountable(ctx context.Context) (snapshot.Mountable, error) {
rr, err := r.ResultProxy.Result(ctx)
if err != nil {
return nil, err
return nil, r.c.wrapSolveError(err)
}
ref, ok := rr.Sys().(*worker.WorkerRef)
if !ok {

View File

@ -31,6 +31,7 @@ import (
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/errdefs"
llberrdefs "github.com/moby/buildkit/solver/llbsolver/errdefs"
opspb "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/apicaps"
"github.com/moby/buildkit/util/grpcerrors"
@ -270,6 +271,10 @@ func (gf *gatewayFrontend) Solve(ctx context.Context, llbBridge frontend.Fronten
func (lbf *llbBridgeForwarder) Discard() {
lbf.mu.Lock()
defer lbf.mu.Unlock()
for id, workerRef := range lbf.workerRefByID {
workerRef.ImmutableRef.Release(context.TODO())
delete(lbf.workerRefByID, id)
}
for id, r := range lbf.refs {
if lbf.err == nil && lbf.result != nil {
keep := false
@ -331,16 +336,17 @@ func NewBridgeForwarder(ctx context.Context, llbBridge frontend.FrontendLLBBridg
func newBridgeForwarder(ctx context.Context, llbBridge frontend.FrontendLLBBridge, workers frontend.WorkerInfos, inputs map[string]*opspb.Definition, sid string, sm *session.Manager) *llbBridgeForwarder {
lbf := &llbBridgeForwarder{
callCtx: ctx,
llbBridge: llbBridge,
refs: map[string]solver.ResultProxy{},
doneCh: make(chan struct{}),
pipe: newPipe(),
workers: workers,
inputs: inputs,
sid: sid,
sm: sm,
ctrs: map[string]gwclient.Container{},
callCtx: ctx,
llbBridge: llbBridge,
refs: map[string]solver.ResultProxy{},
workerRefByID: map[string]*worker.WorkerRef{},
doneCh: make(chan struct{}),
pipe: newPipe(),
workers: workers,
inputs: inputs,
sid: sid,
sm: sm,
ctrs: map[string]gwclient.Container{},
}
return lbf
}
@ -426,10 +432,11 @@ type LLBBridgeForwarder interface {
}
type llbBridgeForwarder struct {
mu sync.Mutex
callCtx context.Context
llbBridge frontend.FrontendLLBBridge
refs map[string]solver.ResultProxy
mu sync.Mutex
callCtx context.Context
llbBridge frontend.FrontendLLBBridge
refs map[string]solver.ResultProxy
workerRefByID map[string]*worker.WorkerRef
// lastRef solver.CachedResult
// lastRefs map[string]solver.CachedResult
// err error
@ -486,6 +493,56 @@ func translateLegacySolveRequest(req *pb.SolveRequest) error {
return nil
}
func (lbf *llbBridgeForwarder) wrapSolveError(solveErr error) error {
var (
ee *llberrdefs.ExecError
fae *llberrdefs.FileActionError
sce *solver.SlowCacheError
inputIDs []string
outputIDs []string
subject errdefs.IsSolve_Subject
)
if errors.As(solveErr, &ee) {
var err error
inputIDs, err = lbf.registerResultIDs(ee.Inputs...)
if err != nil {
return err
}
outputIDs, err = lbf.registerResultIDs(ee.Outputs...)
if err != nil {
return err
}
}
if errors.As(solveErr, &fae) {
subject = fae.ToSubject()
}
if errors.As(solveErr, &sce) {
var err error
inputIDs, err = lbf.registerResultIDs(sce.Result)
if err != nil {
return err
}
subject = sce.ToSubject()
}
return errdefs.WithSolveError(solveErr, subject, inputIDs, outputIDs)
}
func (lbf *llbBridgeForwarder) registerResultIDs(results ...solver.Result) (ids []string, err error) {
lbf.mu.Lock()
defer lbf.mu.Unlock()
ids = make([]string, len(results))
for i, res := range results {
workerRef, ok := res.Sys().(*worker.WorkerRef)
if !ok {
return ids, errors.Errorf("unexpected type for result, got %T", res.Sys())
}
ids[i] = workerRef.ID()
lbf.workerRefByID[workerRef.ID()] = workerRef
}
return ids, nil
}
func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest) (*pb.SolveResponse, error) {
if err := translateLegacySolveRequest(req); err != nil {
return nil, err
@ -500,6 +557,7 @@ func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest)
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
res, err := lbf.llbBridge.Solve(ctx, frontend.SolveRequest{
Evaluate: req.Evaluate,
Definition: req.Definition,
Frontend: req.Frontend,
FrontendOpt: req.FrontendOpt,
@ -507,7 +565,7 @@ func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest)
CacheImports: cacheImports,
}, lbf.sid)
if err != nil {
return nil, err
return nil, lbf.wrapSolveError(err)
}
if len(res.Refs) > 0 && !req.AllowResultReturn {
@ -594,26 +652,39 @@ func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest)
return resp, nil
}
func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileRequest) (*pb.ReadFileResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
func (lbf *llbBridgeForwarder) getImmutableRef(ctx context.Context, id, path string) (cache.ImmutableRef, error) {
lbf.mu.Lock()
ref, ok := lbf.refs[req.Ref]
ref, ok := lbf.refs[id]
lbf.mu.Unlock()
if !ok {
return nil, errors.Errorf("no such ref: %v", req.Ref)
return nil, errors.Errorf("no such ref: %v", id)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.FilePath)
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", path)
}
r, err := ref.Result(ctx)
if err != nil {
return nil, err
return nil, lbf.wrapSolveError(err)
}
workerRef, ok := r.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", r.Sys())
}
return workerRef.ImmutableRef, nil
}
func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileRequest) (*pb.ReadFileResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
ref, err := lbf.getImmutableRef(ctx, req.Ref, req.FilePath)
if err != nil {
return nil, err
}
newReq := cacheutil.ReadRequest{
Filename: req.FilePath,
}
@ -624,14 +695,14 @@ func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileReq
}
}
m, err := workerRef.ImmutableRef.Mount(ctx, true, session.NewGroup(lbf.sid))
m, err := ref.Mount(ctx, true, session.NewGroup(lbf.sid))
if err != nil {
return nil, err
}
dt, err := cacheutil.ReadFile(ctx, m, newReq)
if err != nil {
return nil, err
return nil, lbf.wrapSolveError(err)
}
return &pb.ReadFileResponse{Data: dt}, nil
@ -639,35 +710,23 @@ func (lbf *llbBridgeForwarder) ReadFile(ctx context.Context, req *pb.ReadFileReq
func (lbf *llbBridgeForwarder) ReadDir(ctx context.Context, req *pb.ReadDirRequest) (*pb.ReadDirResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
lbf.mu.Lock()
ref, ok := lbf.refs[req.Ref]
lbf.mu.Unlock()
if !ok {
return nil, errors.Errorf("no such ref: %v", req.Ref)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.DirPath)
}
r, err := ref.Result(ctx)
ref, err := lbf.getImmutableRef(ctx, req.Ref, req.DirPath)
if err != nil {
return nil, err
}
workerRef, ok := r.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", r.Sys())
}
newReq := cacheutil.ReadDirRequest{
Path: req.DirPath,
IncludePattern: req.IncludePattern,
}
m, err := workerRef.ImmutableRef.Mount(ctx, true, session.NewGroup(lbf.sid))
m, err := ref.Mount(ctx, true, session.NewGroup(lbf.sid))
if err != nil {
return nil, err
}
entries, err := cacheutil.ReadDir(ctx, m, newReq)
if err != nil {
return nil, err
return nil, lbf.wrapSolveError(err)
}
return &pb.ReadDirResponse{Entries: entries}, nil
@ -675,24 +734,12 @@ func (lbf *llbBridgeForwarder) ReadDir(ctx context.Context, req *pb.ReadDirReque
func (lbf *llbBridgeForwarder) StatFile(ctx context.Context, req *pb.StatFileRequest) (*pb.StatFileResponse, error) {
ctx = tracing.ContextWithSpanFromContext(ctx, lbf.callCtx)
lbf.mu.Lock()
ref, ok := lbf.refs[req.Ref]
lbf.mu.Unlock()
if !ok {
return nil, errors.Errorf("no such ref: %v", req.Ref)
}
if ref == nil {
return nil, errors.Wrapf(os.ErrNotExist, "%s not found", req.Path)
}
r, err := ref.Result(ctx)
ref, err := lbf.getImmutableRef(ctx, req.Ref, req.Path)
if err != nil {
return nil, err
}
workerRef, ok := r.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", r.Sys())
}
m, err := workerRef.ImmutableRef.Mount(ctx, true, session.NewGroup(lbf.sid))
m, err := ref.Mount(ctx, true, session.NewGroup(lbf.sid))
if err != nil {
return nil, err
}
@ -788,19 +835,34 @@ func (lbf *llbBridgeForwarder) NewContainer(ctx context.Context, in *pb.NewConta
}
for _, m := range in.Mounts {
var refProxy solver.ResultProxy
var workerRef *worker.WorkerRef
if m.ResultID != "" {
refProxy, err = lbf.convertRef(m.ResultID)
if err != nil {
return nil, errors.Wrapf(err, "failed to find ref %s for %q mount", m.ResultID, m.Dest)
var ok bool
workerRef, ok = lbf.workerRefByID[m.ResultID]
if !ok {
refProxy, err := lbf.convertRef(m.ResultID)
if err != nil {
return nil, errors.Wrapf(err, "failed to find ref %s for %q mount", m.ResultID, m.Dest)
}
res, err := refProxy.Result(ctx)
if err != nil {
return nil, stack.Enable(err)
}
workerRef, ok = res.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid reference %T", res.Sys())
}
}
}
ctrReq.Mounts = append(ctrReq.Mounts, Mount{
Dest: m.Dest,
Selector: m.Selector,
Readonly: m.Readonly,
MountType: m.MountType,
RefProxy: refProxy,
WorkerRef: workerRef,
CacheOpt: m.CacheOpt,
SecretOpt: m.SecretOpt,
SSHOpt: m.SSHOpt,

View File

@ -299,7 +299,7 @@ func (c *grpcClient) requestForRef(ref client.Reference) (*pb.SolveRequest, erro
return req, nil
}
func (c *grpcClient) Solve(ctx context.Context, creq client.SolveRequest) (*client.Result, error) {
func (c *grpcClient) Solve(ctx context.Context, creq client.SolveRequest) (res *client.Result, err error) {
if creq.Definition != nil {
for _, md := range creq.Definition.Metadata {
for cap := range md.Caps {
@ -345,13 +345,45 @@ func (c *grpcClient) Solve(ctx context.Context, creq client.SolveRequest) (*clie
req.ExporterAttr = []byte("{}")
}
if creq.Evaluate {
if c.caps.Supports(pb.CapGatewayEvaluateSolve) == nil {
req.Evaluate = creq.Evaluate
} else {
// If evaluate is not supported, fallback to running Stat(".") in order to
// trigger an evaluation of the result.
defer func() {
if res == nil {
return
}
var (
id string
ref client.Reference
)
ref, err = res.SingleRef()
if err != nil {
for refId := range res.Refs {
id = refId
break
}
} else {
id = ref.(*reference).id
}
_, err = c.client.StatFile(ctx, &pb.StatFileRequest{
Ref: id,
Path: ".",
})
}()
}
}
resp, err := c.client.Solve(ctx, req)
if err != nil {
return nil, err
}
res := &client.Result{}
res = &client.Result{}
if resp.Result == nil {
if id := resp.Ref; id != "" {
c.requests[id] = req
@ -652,7 +684,7 @@ func (c *grpcClient) NewContainer(ctx context.Context, req client.NewContainerRe
id := identity.NewID()
var mounts []*opspb.Mount
for _, m := range req.Mounts {
var resultID string
resultID := m.ResultID
if m.Ref != nil {
ref, ok := m.Ref.(*reference)
if !ok {

View File

@ -42,6 +42,11 @@ const (
// CapFrontendCaps can be used to check that frontends define support for certain capabilities
CapFrontendCaps apicaps.CapID = "frontend.caps"
// CapGatewayEvaluateSolve is a capability to immediately unlazy solve
// results. This is generally used by the client to return and handle solve
// errors.
CapGatewayEvaluateSolve apicaps.CapID = "gateway.solve.evaluate"
)
func init() {
@ -157,4 +162,11 @@ func init() {
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
Caps.Init(apicaps.Cap{
ID: CapGatewayEvaluateSolve,
Name: "gateway evaluate solve",
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
}

View File

@ -622,6 +622,7 @@ type SolveRequest struct {
CacheImports []*CacheOptionsEntry `protobuf:"bytes,12,rep,name=CacheImports,proto3" json:"CacheImports,omitempty"`
// apicaps:CapFrontendInputs
FrontendInputs map[string]*pb.Definition `protobuf:"bytes,13,rep,name=FrontendInputs,proto3" json:"FrontendInputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
Evaluate bool `protobuf:"varint,14,opt,name=Evaluate,proto3" json:"Evaluate,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -730,6 +731,13 @@ func (m *SolveRequest) GetFrontendInputs() map[string]*pb.Definition {
return nil
}
func (m *SolveRequest) GetEvaluate() bool {
if m != nil {
return m.Evaluate
}
return false
}
// CacheOptionsEntry corresponds to the control.CacheOptionsEntry
type CacheOptionsEntry struct {
Type string `protobuf:"bytes,1,opt,name=Type,proto3" json:"Type,omitempty"`
@ -2051,126 +2059,127 @@ func init() {
func init() { proto.RegisterFile("gateway.proto", fileDescriptor_f1a937782ebbded5) }
var fileDescriptor_f1a937782ebbded5 = []byte{
// 1899 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0xcf, 0x6f, 0x1b, 0xc7,
0xf5, 0xd7, 0x8a, 0xa4, 0x48, 0x3e, 0x92, 0x32, 0x33, 0xce, 0x37, 0x5f, 0x7a, 0x11, 0x38, 0xcc,
0x22, 0x55, 0x69, 0x47, 0x59, 0xa6, 0x74, 0x02, 0xb9, 0x72, 0x90, 0xd4, 0xd4, 0x0f, 0x58, 0x8d,
0x24, 0xab, 0xe3, 0x14, 0x06, 0x82, 0x14, 0xe8, 0x8a, 0x3b, 0xa4, 0x17, 0xa6, 0x76, 0xb7, 0xb3,
0x43, 0xcb, 0x4c, 0x2e, 0xed, 0xad, 0xf7, 0x02, 0xbd, 0x16, 0xe8, 0x5f, 0xd0, 0x4b, 0xaf, 0x3d,
0xe7, 0xd8, 0x63, 0xd1, 0x43, 0x50, 0x18, 0xfd, 0x3f, 0x5a, 0xbc, 0xf9, 0x41, 0x2e, 0x29, 0x6a,
0x29, 0xa2, 0x27, 0xce, 0xbc, 0x7d, 0x9f, 0x37, 0x6f, 0x3e, 0xf3, 0xe6, 0xbd, 0x37, 0x84, 0xda,
0xc0, 0x13, 0xec, 0xd2, 0x1b, 0xbb, 0x31, 0x8f, 0x44, 0x44, 0xee, 0x5c, 0x44, 0xe7, 0x63, 0xf7,
0x7c, 0x14, 0x0c, 0xfd, 0x97, 0x81, 0x70, 0x5f, 0xfd, 0xc4, 0xed, 0xf3, 0x28, 0x14, 0x2c, 0xf4,
0xed, 0x8f, 0x06, 0x81, 0x78, 0x31, 0x3a, 0x77, 0x7b, 0xd1, 0x45, 0x7b, 0x10, 0x0d, 0xa2, 0xb6,
0x44, 0x9c, 0x8f, 0xfa, 0x72, 0x26, 0x27, 0x72, 0xa4, 0x2c, 0xd9, 0x9d, 0x79, 0xf5, 0x41, 0x14,
0x0d, 0x86, 0xcc, 0x8b, 0x83, 0x44, 0x0f, 0xdb, 0x3c, 0xee, 0xb5, 0x13, 0xe1, 0x89, 0x51, 0xa2,
0x31, 0xdb, 0x29, 0x0c, 0x3a, 0xd2, 0x36, 0x8e, 0xb4, 0x93, 0x68, 0xf8, 0x8a, 0xf1, 0x76, 0x7c,
0xde, 0x8e, 0x62, 0xa3, 0xdd, 0xbe, 0x56, 0xdb, 0x8b, 0x83, 0xb6, 0x18, 0xc7, 0x2c, 0x69, 0x5f,
0x46, 0xfc, 0x25, 0xe3, 0x1a, 0xf0, 0xe0, 0x5a, 0xc0, 0x48, 0x04, 0x43, 0x44, 0xf5, 0xbc, 0x38,
0xc1, 0x45, 0xf0, 0x57, 0x83, 0xd2, 0xdb, 0x16, 0x51, 0x18, 0x24, 0x22, 0x08, 0x06, 0x41, 0xbb,
0x9f, 0x48, 0x8c, 0x5a, 0x05, 0x37, 0xa1, 0xd4, 0x9d, 0xdf, 0xe7, 0x60, 0x83, 0xb2, 0x64, 0x34,
0x14, 0x64, 0x0b, 0x6a, 0x9c, 0xf5, 0xf7, 0x59, 0xcc, 0x59, 0xcf, 0x13, 0xcc, 0x6f, 0x58, 0x4d,
0xab, 0x55, 0x7e, 0xb2, 0x46, 0x67, 0xc5, 0xe4, 0x97, 0xb0, 0xc9, 0x59, 0x3f, 0x49, 0x29, 0xae,
0x37, 0xad, 0x56, 0xa5, 0xf3, 0xa1, 0x7b, 0xed, 0x61, 0xb8, 0x94, 0xf5, 0x4f, 0xbc, 0x78, 0x0a,
0x79, 0xb2, 0x46, 0xe7, 0x8c, 0x90, 0x0e, 0xe4, 0x38, 0xeb, 0x37, 0x72, 0xd2, 0xd6, 0xdd, 0x6c,
0x5b, 0x4f, 0xd6, 0x28, 0x2a, 0x93, 0x1d, 0xc8, 0xa3, 0x95, 0x46, 0x5e, 0x82, 0xde, 0x5f, 0xea,
0xc0, 0x93, 0x35, 0x2a, 0x01, 0xe4, 0x4b, 0x28, 0x5d, 0x30, 0xe1, 0xf9, 0x9e, 0xf0, 0x1a, 0xd0,
0xcc, 0xb5, 0x2a, 0x9d, 0x76, 0x26, 0x18, 0x09, 0x72, 0x4f, 0x34, 0xe2, 0x20, 0x14, 0x7c, 0x4c,
0x27, 0x06, 0xec, 0x47, 0x50, 0x9b, 0xf9, 0x44, 0xea, 0x90, 0x7b, 0xc9, 0xc6, 0x8a, 0x3f, 0x8a,
0x43, 0xf2, 0x36, 0x14, 0x5e, 0x79, 0xc3, 0x11, 0x93, 0x54, 0x55, 0xa9, 0x9a, 0xec, 0xae, 0x3f,
0xb4, 0xba, 0x25, 0xd8, 0xe0, 0xd2, 0xbc, 0xf3, 0x47, 0x0b, 0xea, 0xf3, 0x3c, 0x91, 0x23, 0xbd,
0x43, 0x4b, 0x3a, 0xf9, 0xe9, 0x0a, 0x14, 0xa3, 0x20, 0x51, 0xae, 0x4a, 0x13, 0xf6, 0x0e, 0x94,
0x27, 0xa2, 0x65, 0x2e, 0x96, 0x53, 0x2e, 0x3a, 0x3b, 0x90, 0xa3, 0xac, 0x4f, 0x36, 0x61, 0x3d,
0xd0, 0x41, 0x41, 0xd7, 0x03, 0x9f, 0x34, 0x21, 0xe7, 0xb3, 0xbe, 0x3e, 0xfc, 0x4d, 0x37, 0x3e,
0x77, 0xf7, 0x59, 0x3f, 0x08, 0x03, 0x11, 0x44, 0x21, 0xc5, 0x4f, 0xce, 0x9f, 0x2d, 0x0c, 0x2e,
0x74, 0x8b, 0x7c, 0x31, 0xb3, 0x8f, 0xe5, 0xa1, 0x72, 0xc5, 0xfb, 0xe7, 0xd9, 0xde, 0x7f, 0x92,
0xf6, 0x7e, 0x69, 0xfc, 0xa4, 0x77, 0x27, 0xa0, 0x46, 0x99, 0x18, 0xf1, 0x90, 0xb2, 0xdf, 0x8c,
0x58, 0x22, 0xc8, 0x4f, 0xcd, 0x89, 0x48, 0xfb, 0xcb, 0xc2, 0x0a, 0x15, 0xa9, 0x06, 0x90, 0x16,
0x14, 0x18, 0xe7, 0x11, 0xd7, 0x5e, 0x10, 0x57, 0x65, 0x0e, 0x97, 0xc7, 0x3d, 0xf7, 0x99, 0xcc,
0x1c, 0x54, 0x29, 0x38, 0x75, 0xd8, 0x34, 0xab, 0x26, 0x71, 0x14, 0x26, 0xcc, 0xb9, 0x05, 0xb5,
0xa3, 0x30, 0x1e, 0x89, 0x44, 0xfb, 0xe1, 0xfc, 0xcd, 0x82, 0x4d, 0x23, 0x51, 0x3a, 0xe4, 0x1b,
0xa8, 0x4c, 0x39, 0x36, 0x64, 0xee, 0x66, 0xf8, 0x37, 0x8b, 0x4f, 0x1d, 0x90, 0xe6, 0x36, 0x6d,
0xce, 0x3e, 0x85, 0xfa, 0xbc, 0xc2, 0x02, 0xa6, 0x3f, 0x98, 0x65, 0x7a, 0xfe, 0xe0, 0x53, 0xcc,
0xfe, 0xc1, 0x82, 0x3b, 0x94, 0xc9, 0x54, 0x78, 0x74, 0xe1, 0x0d, 0xd8, 0x5e, 0x14, 0xf6, 0x83,
0x81, 0xa1, 0xb9, 0x2e, 0xa3, 0xca, 0x58, 0xc6, 0x00, 0x6b, 0x41, 0xe9, 0x6c, 0xe8, 0x89, 0x7e,
0xc4, 0x2f, 0xb4, 0xf1, 0x2a, 0x1a, 0x37, 0x32, 0x3a, 0xf9, 0x4a, 0x9a, 0x50, 0xd1, 0x86, 0x4f,
0x22, 0x9f, 0xc9, 0x9c, 0x51, 0xa6, 0x69, 0x11, 0x69, 0x40, 0xf1, 0x38, 0x1a, 0x9c, 0x7a, 0x17,
0x4c, 0x26, 0x87, 0x32, 0x35, 0x53, 0xe7, 0xb7, 0x16, 0xd8, 0x8b, 0xbc, 0xd2, 0x14, 0xff, 0x1c,
0x36, 0xf6, 0x83, 0x01, 0x4b, 0xd4, 0xe9, 0x97, 0xbb, 0x9d, 0xef, 0x7f, 0x78, 0x6f, 0xed, 0x9f,
0x3f, 0xbc, 0x77, 0x3f, 0x95, 0x57, 0xa3, 0x98, 0x85, 0xbd, 0x28, 0x14, 0x5e, 0x10, 0x32, 0x8e,
0xe5, 0xe1, 0x23, 0x5f, 0x42, 0x5c, 0x85, 0xa4, 0xda, 0x02, 0x79, 0x07, 0x36, 0x94, 0x75, 0x7d,
0xed, 0xf5, 0xcc, 0xf9, 0x47, 0x01, 0xaa, 0xcf, 0xd0, 0x01, 0xc3, 0x85, 0x0b, 0x30, 0xa5, 0x50,
0x87, 0xdd, 0x3c, 0xb1, 0x29, 0x0d, 0x62, 0x43, 0xe9, 0x50, 0x1f, 0xb1, 0xbe, 0xae, 0x93, 0x39,
0xf9, 0x1a, 0x2a, 0x66, 0xfc, 0x34, 0x16, 0x8d, 0x9c, 0x8c, 0x91, 0x87, 0x19, 0x31, 0x92, 0xf6,
0xc4, 0x4d, 0x41, 0x75, 0x84, 0xa4, 0x24, 0xe4, 0x33, 0xb8, 0x73, 0x74, 0x11, 0x47, 0x5c, 0xec,
0x79, 0xbd, 0x17, 0x8c, 0xce, 0x56, 0x81, 0x7c, 0x33, 0xd7, 0x2a, 0xd3, 0xeb, 0x15, 0xc8, 0x36,
0xbc, 0xe5, 0x0d, 0x87, 0xd1, 0xa5, 0xbe, 0x34, 0x32, 0xfc, 0x1b, 0x85, 0xa6, 0xd5, 0x2a, 0xd1,
0xab, 0x1f, 0xc8, 0xc7, 0x70, 0x3b, 0x25, 0x7c, 0xcc, 0xb9, 0x37, 0xc6, 0x78, 0xd9, 0x90, 0xfa,
0x8b, 0x3e, 0x61, 0x06, 0x3b, 0x0c, 0x42, 0x6f, 0xd8, 0x00, 0xa9, 0xa3, 0x26, 0xc4, 0x81, 0xea,
0xc1, 0x6b, 0x74, 0x89, 0xf1, 0xc7, 0x42, 0xf0, 0x46, 0x45, 0x1e, 0xc5, 0x8c, 0x8c, 0x9c, 0x41,
0x55, 0x3a, 0xac, 0x7c, 0x4f, 0x1a, 0x55, 0x49, 0xda, 0x76, 0x06, 0x69, 0x52, 0xfd, 0x69, 0x9c,
0xba, 0x4a, 0x33, 0x16, 0x48, 0x0f, 0x36, 0x0d, 0x71, 0xea, 0x0e, 0x36, 0x6a, 0xd2, 0xe6, 0xa3,
0x55, 0x0f, 0x42, 0xa1, 0xd5, 0x12, 0x73, 0x26, 0xed, 0xcf, 0xa1, 0x3e, 0x7f, 0x5e, 0xab, 0x24,
0x76, 0xfb, 0x17, 0x70, 0x7b, 0xc1, 0x32, 0xff, 0xd3, 0x9d, 0xff, 0x8b, 0x05, 0x6f, 0x5d, 0xe1,
0x86, 0x10, 0xc8, 0x7f, 0x35, 0x8e, 0x99, 0x36, 0x29, 0xc7, 0xe4, 0x04, 0x0a, 0xc8, 0x7d, 0xd2,
0x58, 0x97, 0xc4, 0xec, 0xac, 0x42, 0xb6, 0x2b, 0x91, 0x8a, 0x14, 0x65, 0xc5, 0x7e, 0x08, 0x30,
0x15, 0xae, 0x54, 0xde, 0xbe, 0x81, 0x9a, 0x66, 0x5e, 0xa7, 0x80, 0xba, 0xea, 0x44, 0x34, 0x18,
0xfb, 0x8c, 0x69, 0x49, 0xc8, 0xad, 0x58, 0x12, 0x9c, 0xef, 0xe0, 0x16, 0x65, 0x9e, 0x7f, 0x18,
0x0c, 0xd9, 0xf5, 0x99, 0x0f, 0xef, 0x73, 0x30, 0x64, 0x67, 0x9e, 0x78, 0x31, 0xb9, 0xcf, 0x7a,
0x4e, 0x76, 0xa1, 0x40, 0xbd, 0x70, 0xc0, 0xf4, 0xd2, 0x1f, 0x64, 0x2c, 0x2d, 0x17, 0x41, 0x5d,
0xaa, 0x20, 0xce, 0x23, 0x28, 0x4f, 0x64, 0x98, 0x8d, 0x9e, 0xf6, 0xfb, 0x09, 0x53, 0x99, 0x2d,
0x47, 0xf5, 0x0c, 0xe5, 0xc7, 0x2c, 0x1c, 0xe8, 0xa5, 0x73, 0x54, 0xcf, 0x9c, 0x2d, 0x6c, 0x47,
0x8c, 0xe7, 0x9a, 0x1a, 0x02, 0xf9, 0x7d, 0xec, 0x99, 0x2c, 0x79, 0x89, 0xe4, 0xd8, 0xf1, 0xb1,
0x94, 0x79, 0xfe, 0x7e, 0xc0, 0xaf, 0xdf, 0x60, 0x03, 0x8a, 0xfb, 0x01, 0x4f, 0xed, 0xcf, 0x4c,
0xc9, 0x16, 0x16, 0xb9, 0xde, 0x70, 0xe4, 0xe3, 0x6e, 0x05, 0xe3, 0xa1, 0xce, 0xe6, 0x73, 0x52,
0xe7, 0x0b, 0xc5, 0xa3, 0x5c, 0x45, 0x3b, 0xb3, 0x0d, 0x45, 0x16, 0x0a, 0x1e, 0x30, 0x53, 0x09,
0x89, 0xab, 0xda, 0x5c, 0x57, 0xb6, 0xb9, 0xb2, 0xe2, 0x52, 0xa3, 0xe2, 0xec, 0xc0, 0x2d, 0x14,
0x64, 0x1f, 0x04, 0x81, 0x7c, 0xca, 0x49, 0x39, 0x76, 0x76, 0xa1, 0x3e, 0x05, 0xea, 0xa5, 0xb7,
0x20, 0x8f, 0x4d, 0xb4, 0x4e, 0xd5, 0x8b, 0xd6, 0x95, 0xdf, 0x9d, 0x1a, 0x54, 0xce, 0x82, 0xd0,
0xd4, 0x3c, 0xe7, 0x8d, 0x05, 0xd5, 0xb3, 0x28, 0x9c, 0x56, 0x9b, 0x33, 0xb8, 0x65, 0x6e, 0xe0,
0xe3, 0xb3, 0xa3, 0x3d, 0x2f, 0x36, 0x5b, 0x69, 0x5e, 0x3d, 0x66, 0xdd, 0xef, 0xbb, 0x4a, 0xb1,
0x9b, 0xc7, 0xc2, 0x44, 0xe7, 0xe1, 0xe4, 0x67, 0x50, 0x3c, 0x3e, 0xee, 0x4a, 0x4b, 0xeb, 0x2b,
0x59, 0x32, 0x30, 0xf2, 0x39, 0x14, 0x9f, 0xcb, 0x67, 0x48, 0xa2, 0x8b, 0xc7, 0x82, 0x90, 0x53,
0x1b, 0x55, 0x6a, 0x94, 0xf5, 0x22, 0xee, 0x53, 0x03, 0x72, 0xfe, 0x6d, 0xc1, 0xed, 0x53, 0x76,
0xb9, 0x67, 0x0a, 0xa4, 0x61, 0xbb, 0x09, 0x95, 0x89, 0xec, 0x68, 0x5f, 0xb3, 0x9e, 0x16, 0x91,
0xf7, 0x61, 0xe3, 0x24, 0x1a, 0x85, 0xc2, 0xb8, 0x5e, 0xc6, 0x3c, 0x23, 0x25, 0x54, 0x7f, 0x20,
0x3f, 0x82, 0xe2, 0x29, 0x13, 0xf8, 0x4c, 0x92, 0x71, 0xb2, 0xd9, 0xa9, 0xa0, 0xce, 0x29, 0x13,
0x58, 0xf5, 0xa9, 0xf9, 0x86, 0xad, 0x44, 0x6c, 0x5a, 0x89, 0xfc, 0xa2, 0x56, 0xc2, 0x7c, 0x25,
0x3b, 0x50, 0xe9, 0x45, 0x61, 0x22, 0xb8, 0x17, 0xe0, 0xc2, 0x05, 0xa9, 0xfc, 0x7f, 0xa8, 0xac,
0xf6, 0xb3, 0x37, 0xfd, 0x48, 0xd3, 0x9a, 0xce, 0x3b, 0xf0, 0xf6, 0xec, 0x2e, 0x75, 0x1f, 0xf7,
0x08, 0xfe, 0x9f, 0xb2, 0x21, 0xf3, 0x12, 0xb6, 0x3a, 0x03, 0x8e, 0x0d, 0x8d, 0xab, 0x60, 0x6d,
0xf8, 0xaf, 0x39, 0xa8, 0x1c, 0xbc, 0x66, 0xbd, 0x13, 0x96, 0x24, 0xde, 0x80, 0x91, 0x77, 0xa1,
0x7c, 0xc6, 0xa3, 0x1e, 0x4b, 0x92, 0x89, 0xad, 0xa9, 0x80, 0x7c, 0x06, 0xf9, 0xa3, 0x30, 0x10,
0x3a, 0x63, 0x6f, 0x65, 0xf6, 0x88, 0x81, 0xd0, 0x36, 0xf1, 0x7d, 0x84, 0x53, 0xb2, 0x0b, 0x79,
0x8c, 0xf7, 0x9b, 0xe4, 0x1c, 0x3f, 0x85, 0x45, 0x0c, 0xe9, 0xca, 0x17, 0x65, 0xf0, 0x2d, 0xd3,
0xcc, 0xb7, 0xb2, 0x93, 0x65, 0xf0, 0x2d, 0x9b, 0x5a, 0xd0, 0x48, 0x72, 0x00, 0xc5, 0x67, 0xc2,
0xe3, 0xd8, 0x56, 0xa8, 0x13, 0xb9, 0x97, 0x55, 0x37, 0x95, 0xe6, 0xd4, 0x8a, 0xc1, 0x22, 0x09,
0x07, 0xaf, 0x03, 0x21, 0x9b, 0x86, 0x6c, 0x12, 0x50, 0x2d, 0xb5, 0x11, 0x9c, 0x22, 0x7a, 0x3f,
0x0a, 0x59, 0xa3, 0xb8, 0x14, 0x8d, 0x6a, 0x29, 0x34, 0x4e, 0xbb, 0x45, 0x28, 0xc8, 0xa2, 0xea,
0xfc, 0xc9, 0x82, 0x4a, 0x8a, 0xe3, 0x1b, 0xdc, 0x83, 0x77, 0x21, 0x8f, 0x0f, 0x4a, 0x7d, 0x76,
0x25, 0x79, 0x0b, 0x98, 0xf0, 0xa8, 0x94, 0x62, 0xd6, 0x3a, 0xf4, 0xd5, 0xdd, 0xac, 0x51, 0x1c,
0xa2, 0xe4, 0x2b, 0x31, 0x96, 0x74, 0x97, 0x28, 0x0e, 0xc9, 0x36, 0x94, 0x9e, 0xb1, 0xde, 0x88,
0x07, 0x62, 0x2c, 0x09, 0xdc, 0xec, 0xd4, 0xd1, 0x8a, 0x91, 0xc9, 0xcb, 0x32, 0xd1, 0x70, 0xbe,
0xc4, 0xc0, 0x9a, 0x3a, 0x48, 0x20, 0xbf, 0x87, 0x6d, 0x35, 0x7a, 0x56, 0xa3, 0x72, 0x8c, 0x2f,
0x9b, 0x83, 0x65, 0x2f, 0x9b, 0x03, 0xf3, 0xb2, 0x99, 0x3d, 0x10, 0x4c, 0x82, 0x29, 0x82, 0x9c,
0xc7, 0x50, 0x9e, 0x04, 0x0d, 0x3e, 0x2a, 0x0f, 0x7d, 0xbd, 0xd2, 0xfa, 0xa1, 0x8f, 0x5b, 0x39,
0x78, 0x7a, 0x28, 0x57, 0x29, 0x51, 0x1c, 0x4e, 0x4a, 0x4e, 0x2e, 0x55, 0x72, 0x76, 0xf0, 0xcd,
0x96, 0x8a, 0x1c, 0x54, 0xa2, 0xd1, 0x65, 0x62, 0x5c, 0xc6, 0xb1, 0xda, 0xc6, 0x30, 0x91, 0xb6,
0xe4, 0x36, 0x86, 0x49, 0xe7, 0x3f, 0x25, 0x28, 0x1f, 0x1f, 0x77, 0xbb, 0x3c, 0xf0, 0x07, 0x8c,
0xfc, 0xce, 0x02, 0x72, 0xf5, 0x29, 0x40, 0x3e, 0xc9, 0x0e, 0xd8, 0xc5, 0xef, 0x19, 0xfb, 0xd3,
0x15, 0x51, 0xba, 0x02, 0x7c, 0x0d, 0x05, 0xd9, 0x7d, 0x90, 0x1f, 0xdf, 0xb0, 0x33, 0xb4, 0x5b,
0xcb, 0x15, 0xb5, 0xed, 0x1e, 0x94, 0x4c, 0x05, 0x27, 0xf7, 0x33, 0xdd, 0x9b, 0x69, 0x50, 0xec,
0x0f, 0x6f, 0xa4, 0xab, 0x17, 0xf9, 0x35, 0x14, 0x75, 0x61, 0x26, 0xf7, 0x96, 0xe0, 0xa6, 0x2d,
0x82, 0x7d, 0xff, 0x26, 0xaa, 0xd3, 0x6d, 0x98, 0x02, 0x9c, 0xb9, 0x8d, 0xb9, 0xf2, 0x9e, 0xb9,
0x8d, 0x2b, 0x15, 0xfd, 0x39, 0xe4, 0xb1, 0x52, 0x93, 0xac, 0x6b, 0x9e, 0x2a, 0xe5, 0x76, 0xd6,
0x71, 0xcd, 0x94, 0xf8, 0x5f, 0x61, 0x3a, 0x94, 0x2f, 0x9a, 0xec, 0x44, 0x98, 0xfa, 0x0b, 0xc2,
0xbe, 0x77, 0x03, 0xcd, 0xa9, 0x79, 0xd5, 0xbb, 0x67, 0x9a, 0x9f, 0xf9, 0x67, 0x21, 0xd3, 0xfc,
0xdc, 0x3f, 0x0e, 0x11, 0x54, 0xd3, 0x55, 0x8e, 0xb8, 0x19, 0xd0, 0x05, 0x45, 0xdf, 0x6e, 0xdf,
0x58, 0x5f, 0x2f, 0xf8, 0x1d, 0x76, 0x9d, 0xb3, 0x15, 0x90, 0x74, 0x32, 0xe9, 0x58, 0x58, 0x6b,
0xed, 0x07, 0x2b, 0x61, 0xf4, 0xe2, 0x9e, 0xaa, 0xb0, 0xba, 0x8a, 0x92, 0xec, 0x82, 0x31, 0xa9,
0xc4, 0xf6, 0x0d, 0xf5, 0x5a, 0xd6, 0xc7, 0x56, 0xb7, 0xfa, 0xfd, 0x9b, 0xbb, 0xd6, 0xdf, 0xdf,
0xdc, 0xb5, 0xfe, 0xf5, 0xe6, 0xae, 0x75, 0xbe, 0x21, 0xff, 0x85, 0x7d, 0xf0, 0xdf, 0x00, 0x00,
0x00, 0xff, 0xff, 0x8e, 0x01, 0x01, 0xc8, 0xd7, 0x16, 0x00, 0x00,
// 1909 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7,
0x15, 0xd7, 0x8a, 0xa4, 0x48, 0x3e, 0xfe, 0x31, 0x33, 0x4e, 0x53, 0x7a, 0x11, 0x38, 0xcc, 0x22,
0x55, 0x69, 0x47, 0x59, 0xa6, 0x74, 0x02, 0xb9, 0x72, 0x90, 0xd4, 0x94, 0x28, 0x58, 0x8d, 0x24,
0xab, 0xe3, 0x14, 0x06, 0x82, 0x14, 0xe8, 0x8a, 0x3b, 0xa4, 0x17, 0xa6, 0x76, 0xb7, 0xb3, 0x43,
0xcb, 0x4c, 0x2e, 0xed, 0xad, 0xf7, 0x02, 0xbd, 0x16, 0xe8, 0x27, 0xe8, 0xa5, 0xd7, 0x9e, 0x73,
0xec, 0xb9, 0x07, 0xa3, 0x10, 0xfa, 0x11, 0x7a, 0x6f, 0xf1, 0x66, 0x67, 0xc8, 0x25, 0x45, 0x2d,
0x49, 0xe4, 0xc4, 0x99, 0xb7, 0xef, 0xf7, 0xe6, 0xfd, 0x9b, 0xf7, 0xde, 0x10, 0x2a, 0x03, 0x47,
0xb0, 0x4b, 0x67, 0x6c, 0x87, 0x3c, 0x10, 0x01, 0xb9, 0x73, 0x11, 0x9c, 0x8f, 0xed, 0xf3, 0x91,
0x37, 0x74, 0x5f, 0x7a, 0xc2, 0x7e, 0xf5, 0x33, 0xbb, 0xcf, 0x03, 0x5f, 0x30, 0xdf, 0x35, 0x3f,
0x1a, 0x78, 0xe2, 0xc5, 0xe8, 0xdc, 0xee, 0x05, 0x17, 0xad, 0x41, 0x30, 0x08, 0x5a, 0x12, 0x71,
0x3e, 0xea, 0xcb, 0x9d, 0xdc, 0xc8, 0x55, 0x2c, 0xc9, 0x6c, 0xcf, 0xb3, 0x0f, 0x82, 0x60, 0x30,
0x64, 0x4e, 0xe8, 0x45, 0x6a, 0xd9, 0xe2, 0x61, 0xaf, 0x15, 0x09, 0x47, 0x8c, 0x22, 0x85, 0xd9,
0x49, 0x60, 0x50, 0x91, 0x96, 0x56, 0xa4, 0x15, 0x05, 0xc3, 0x57, 0x8c, 0xb7, 0xc2, 0xf3, 0x56,
0x10, 0x6a, 0xee, 0xd6, 0x8d, 0xdc, 0x4e, 0xe8, 0xb5, 0xc4, 0x38, 0x64, 0x51, 0xeb, 0x32, 0xe0,
0x2f, 0x19, 0x57, 0x80, 0x07, 0x37, 0x02, 0x46, 0xc2, 0x1b, 0x22, 0xaa, 0xe7, 0x84, 0x11, 0x1e,
0x82, 0xbf, 0x0a, 0x94, 0x34, 0x5b, 0x04, 0xbe, 0x17, 0x09, 0xcf, 0x1b, 0x78, 0xad, 0x7e, 0x24,
0x31, 0xf1, 0x29, 0x68, 0x44, 0xcc, 0x6e, 0xfd, 0x31, 0x03, 0x5b, 0x94, 0x45, 0xa3, 0xa1, 0x20,
0xdb, 0x50, 0xe1, 0xac, 0x7f, 0xc0, 0x42, 0xce, 0x7a, 0x8e, 0x60, 0x6e, 0xdd, 0x68, 0x18, 0xcd,
0xe2, 0x93, 0x0d, 0x3a, 0x4b, 0x26, 0xbf, 0x86, 0x2a, 0x67, 0xfd, 0x28, 0xc1, 0xb8, 0xd9, 0x30,
0x9a, 0xa5, 0xf6, 0x87, 0xf6, 0x8d, 0xc1, 0xb0, 0x29, 0xeb, 0x9f, 0x38, 0xe1, 0x14, 0xf2, 0x64,
0x83, 0xce, 0x09, 0x21, 0x6d, 0xc8, 0x70, 0xd6, 0xaf, 0x67, 0xa4, 0xac, 0xbb, 0xe9, 0xb2, 0x9e,
0x6c, 0x50, 0x64, 0x26, 0xbb, 0x90, 0x45, 0x29, 0xf5, 0xac, 0x04, 0xbd, 0xbf, 0x54, 0x81, 0x27,
0x1b, 0x54, 0x02, 0xc8, 0x97, 0x50, 0xb8, 0x60, 0xc2, 0x71, 0x1d, 0xe1, 0xd4, 0xa1, 0x91, 0x69,
0x96, 0xda, 0xad, 0x54, 0x30, 0x3a, 0xc8, 0x3e, 0x51, 0x88, 0xae, 0x2f, 0xf8, 0x98, 0x4e, 0x04,
0x98, 0x8f, 0xa0, 0x32, 0xf3, 0x89, 0xd4, 0x20, 0xf3, 0x92, 0x8d, 0x63, 0xff, 0x51, 0x5c, 0x92,
0xb7, 0x21, 0xf7, 0xca, 0x19, 0x8e, 0x98, 0x74, 0x55, 0x99, 0xc6, 0x9b, 0xbd, 0xcd, 0x87, 0x46,
0xa7, 0x00, 0x5b, 0x5c, 0x8a, 0xb7, 0xfe, 0x6c, 0x40, 0x6d, 0xde, 0x4f, 0xe4, 0x48, 0x59, 0x68,
0x48, 0x25, 0x3f, 0x5d, 0xc3, 0xc5, 0x48, 0x88, 0x62, 0x55, 0xa5, 0x08, 0x73, 0x17, 0x8a, 0x13,
0xd2, 0x32, 0x15, 0x8b, 0x09, 0x15, 0xad, 0x5d, 0xc8, 0x50, 0xd6, 0x27, 0x55, 0xd8, 0xf4, 0x54,
0x52, 0xd0, 0x4d, 0xcf, 0x25, 0x0d, 0xc8, 0xb8, 0xac, 0xaf, 0x82, 0x5f, 0xb5, 0xc3, 0x73, 0xfb,
0x80, 0xf5, 0x3d, 0xdf, 0x13, 0x5e, 0xe0, 0x53, 0xfc, 0x64, 0xfd, 0xd5, 0xc0, 0xe4, 0x42, 0xb5,
0xc8, 0x17, 0x33, 0x76, 0x2c, 0x4f, 0x95, 0x6b, 0xda, 0x3f, 0x4f, 0xd7, 0xfe, 0x93, 0xa4, 0xf6,
0x4b, 0xf3, 0x27, 0x69, 0x9d, 0x80, 0x0a, 0x65, 0x62, 0xc4, 0x7d, 0xca, 0x7e, 0x37, 0x62, 0x91,
0x20, 0x3f, 0xd7, 0x11, 0x91, 0xf2, 0x97, 0xa5, 0x15, 0x32, 0x52, 0x05, 0x20, 0x4d, 0xc8, 0x31,
0xce, 0x03, 0xae, 0xb4, 0x20, 0x76, 0x5c, 0x39, 0x6c, 0x1e, 0xf6, 0xec, 0x67, 0xb2, 0x72, 0xd0,
0x98, 0xc1, 0xaa, 0x41, 0x55, 0x9f, 0x1a, 0x85, 0x81, 0x1f, 0x31, 0xeb, 0x16, 0x54, 0x8e, 0xfc,
0x70, 0x24, 0x22, 0xa5, 0x87, 0xf5, 0x0f, 0x03, 0xaa, 0x9a, 0x12, 0xf3, 0x90, 0x6f, 0xa0, 0x34,
0xf5, 0xb1, 0x76, 0xe6, 0x5e, 0x8a, 0x7e, 0xb3, 0xf8, 0x44, 0x80, 0x94, 0x6f, 0x93, 0xe2, 0xcc,
0x53, 0xa8, 0xcd, 0x33, 0x2c, 0xf0, 0xf4, 0x07, 0xb3, 0x9e, 0x9e, 0x0f, 0x7c, 0xc2, 0xb3, 0x7f,
0x32, 0xe0, 0x0e, 0x65, 0xb2, 0x14, 0x1e, 0x5d, 0x38, 0x03, 0xb6, 0x1f, 0xf8, 0x7d, 0x6f, 0xa0,
0xdd, 0x5c, 0x93, 0x59, 0xa5, 0x25, 0x63, 0x82, 0x35, 0xa1, 0x70, 0x36, 0x74, 0x44, 0x3f, 0xe0,
0x17, 0x4a, 0x78, 0x19, 0x85, 0x6b, 0x1a, 0x9d, 0x7c, 0x25, 0x0d, 0x28, 0x29, 0xc1, 0x27, 0x81,
0xcb, 0x64, 0xcd, 0x28, 0xd2, 0x24, 0x89, 0xd4, 0x21, 0x7f, 0x1c, 0x0c, 0x4e, 0x9d, 0x0b, 0x26,
0x8b, 0x43, 0x91, 0xea, 0xad, 0xf5, 0x7b, 0x03, 0xcc, 0x45, 0x5a, 0x29, 0x17, 0xff, 0x12, 0xb6,
0x0e, 0xbc, 0x01, 0x8b, 0xe2, 0xe8, 0x17, 0x3b, 0xed, 0xef, 0xdf, 0xbc, 0xb7, 0xf1, 0xaf, 0x37,
0xef, 0xdd, 0x4f, 0xd4, 0xd5, 0x20, 0x64, 0x7e, 0x2f, 0xf0, 0x85, 0xe3, 0xf9, 0x8c, 0x63, 0x7b,
0xf8, 0xc8, 0x95, 0x10, 0x3b, 0x46, 0x52, 0x25, 0x81, 0xbc, 0x03, 0x5b, 0xb1, 0x74, 0x75, 0xed,
0xd5, 0xce, 0xfa, 0x6f, 0x0e, 0xca, 0xcf, 0x50, 0x01, 0xed, 0x0b, 0x1b, 0x60, 0xea, 0x42, 0x95,
0x76, 0xf3, 0x8e, 0x4d, 0x70, 0x10, 0x13, 0x0a, 0x87, 0x2a, 0xc4, 0xea, 0xba, 0x4e, 0xf6, 0xe4,
0x6b, 0x28, 0xe9, 0xf5, 0xd3, 0x50, 0xd4, 0x33, 0x32, 0x47, 0x1e, 0xa6, 0xe4, 0x48, 0x52, 0x13,
0x3b, 0x01, 0x55, 0x19, 0x92, 0xa0, 0x90, 0xcf, 0xe0, 0xce, 0xd1, 0x45, 0x18, 0x70, 0xb1, 0xef,
0xf4, 0x5e, 0x30, 0x3a, 0xdb, 0x05, 0xb2, 0x8d, 0x4c, 0xb3, 0x48, 0x6f, 0x66, 0x20, 0x3b, 0xf0,
0x96, 0x33, 0x1c, 0x06, 0x97, 0xea, 0xd2, 0xc8, 0xf4, 0xaf, 0xe7, 0x1a, 0x46, 0xb3, 0x40, 0xaf,
0x7f, 0x20, 0x1f, 0xc3, 0xed, 0x04, 0xf1, 0x31, 0xe7, 0xce, 0x18, 0xf3, 0x65, 0x4b, 0xf2, 0x2f,
0xfa, 0x84, 0x15, 0xec, 0xd0, 0xf3, 0x9d, 0x61, 0x1d, 0x24, 0x4f, 0xbc, 0x21, 0x16, 0x94, 0xbb,
0xaf, 0x51, 0x25, 0xc6, 0x1f, 0x0b, 0xc1, 0xeb, 0x25, 0x19, 0x8a, 0x19, 0x1a, 0x39, 0x83, 0xb2,
0x54, 0x38, 0xd6, 0x3d, 0xaa, 0x97, 0xa5, 0xd3, 0x76, 0x52, 0x9c, 0x26, 0xd9, 0x9f, 0x86, 0x89,
0xab, 0x34, 0x23, 0x81, 0xf4, 0xa0, 0xaa, 0x1d, 0x17, 0xdf, 0xc1, 0x7a, 0x45, 0xca, 0x7c, 0xb4,
0x6e, 0x20, 0x62, 0x74, 0x7c, 0xc4, 0x9c, 0x48, 0x4c, 0x83, 0x2e, 0x5e, 0x37, 0x47, 0xb0, 0x7a,
0x55, 0xda, 0x3c, 0xd9, 0x9b, 0x9f, 0x43, 0x6d, 0x3e, 0x96, 0xeb, 0x14, 0x7d, 0xf3, 0x57, 0x70,
0x7b, 0x81, 0x0a, 0x3f, 0xa8, 0x1e, 0xfc, 0xcd, 0x80, 0xb7, 0xae, 0xf9, 0x8d, 0x10, 0xc8, 0x7e,
0x35, 0x0e, 0x99, 0x12, 0x29, 0xd7, 0xe4, 0x04, 0x72, 0x18, 0x97, 0xa8, 0xbe, 0x29, 0x9d, 0xb6,
0xbb, 0x4e, 0x20, 0x6c, 0x89, 0x8c, 0x1d, 0x16, 0x4b, 0x31, 0x1f, 0x02, 0x4c, 0x89, 0x6b, 0xb5,
0xbe, 0x6f, 0xa0, 0xa2, 0xa2, 0xa2, 0xca, 0x43, 0x2d, 0x9e, 0x52, 0x14, 0x18, 0x67, 0x90, 0x69,
0xbb, 0xc8, 0xac, 0xd9, 0x2e, 0xac, 0xef, 0xe0, 0x16, 0x65, 0x8e, 0x7b, 0xe8, 0x0d, 0xd9, 0xcd,
0x55, 0x11, 0xef, 0xba, 0x37, 0x64, 0x67, 0x8e, 0x78, 0x31, 0xb9, 0xeb, 0x6a, 0x4f, 0xf6, 0x20,
0x47, 0x1d, 0x7f, 0xc0, 0xd4, 0xd1, 0x1f, 0xa4, 0x1c, 0x2d, 0x0f, 0x41, 0x5e, 0x1a, 0x43, 0xac,
0x47, 0x50, 0x9c, 0xd0, 0xb0, 0x52, 0x3d, 0xed, 0xf7, 0x23, 0x16, 0x57, 0xbd, 0x0c, 0x55, 0x3b,
0xa4, 0x1f, 0x33, 0x7f, 0xa0, 0x8e, 0xce, 0x50, 0xb5, 0xb3, 0xb6, 0x71, 0x54, 0xd1, 0x9a, 0x2b,
0xd7, 0x10, 0xc8, 0x1e, 0xe0, 0x3c, 0x65, 0xc8, 0x0b, 0x26, 0xd7, 0x96, 0x8b, 0x6d, 0xce, 0x71,
0x0f, 0x3c, 0x7e, 0xb3, 0x81, 0x75, 0xc8, 0x1f, 0x78, 0x3c, 0x61, 0x9f, 0xde, 0x92, 0x6d, 0x6c,
0x80, 0xbd, 0xe1, 0xc8, 0x45, 0x6b, 0x05, 0xe3, 0xbe, 0xaa, 0xf4, 0x73, 0x54, 0xeb, 0x8b, 0xd8,
0x8f, 0xf2, 0x14, 0xa5, 0xcc, 0x0e, 0xe4, 0x99, 0x2f, 0xb8, 0xc7, 0x74, 0x97, 0x24, 0x76, 0x3c,
0x02, 0xdb, 0x72, 0x04, 0x96, 0xdd, 0x98, 0x6a, 0x16, 0x6b, 0x17, 0x6e, 0x21, 0x21, 0x3d, 0x10,
0x04, 0xb2, 0x09, 0x25, 0xe5, 0xda, 0xda, 0x83, 0xda, 0x14, 0xa8, 0x8e, 0xde, 0x86, 0x2c, 0x0e,
0xd8, 0xaa, 0x8c, 0x2f, 0x3a, 0x57, 0x7e, 0xb7, 0x2a, 0x50, 0x3a, 0xf3, 0x7c, 0xdd, 0x0f, 0xad,
0x2b, 0x03, 0xca, 0x67, 0x81, 0x3f, 0xed, 0x44, 0x67, 0x70, 0x4b, 0xdf, 0xc0, 0xc7, 0x67, 0x47,
0xfb, 0x4e, 0xa8, 0x4d, 0x69, 0x5c, 0x0f, 0xb3, 0x7a, 0x0b, 0xd8, 0x31, 0x63, 0x27, 0x8b, 0x4d,
0x8b, 0xce, 0xc3, 0xc9, 0x2f, 0x20, 0x7f, 0x7c, 0xdc, 0x91, 0x92, 0x36, 0xd7, 0x92, 0xa4, 0x61,
0xe4, 0x73, 0xc8, 0x3f, 0x97, 0x4f, 0x94, 0x48, 0x35, 0x96, 0x05, 0x29, 0x17, 0x1b, 0x1a, 0xb3,
0x51, 0xd6, 0x0b, 0xb8, 0x4b, 0x35, 0xc8, 0xfa, 0x8f, 0x01, 0xb7, 0x4f, 0xd9, 0xe5, 0xbe, 0x6e,
0x9e, 0xda, 0xdb, 0x0d, 0x28, 0x4d, 0x68, 0x47, 0x07, 0xca, 0xeb, 0x49, 0x12, 0x79, 0x1f, 0xb6,
0x4e, 0x82, 0x91, 0x2f, 0xb4, 0xea, 0x45, 0xac, 0x33, 0x92, 0x42, 0xd5, 0x07, 0xf2, 0x13, 0xc8,
0x9f, 0x32, 0x81, 0x4f, 0x28, 0x99, 0x27, 0xd5, 0x76, 0x09, 0x79, 0x4e, 0x99, 0xc0, 0x89, 0x80,
0xea, 0x6f, 0x38, 0x66, 0x84, 0x7a, 0xcc, 0xc8, 0x2e, 0x1a, 0x33, 0xf4, 0x57, 0xb2, 0x0b, 0xa5,
0x5e, 0xe0, 0x47, 0x82, 0x3b, 0x1e, 0x1e, 0x9c, 0x93, 0xcc, 0x3f, 0x42, 0xe6, 0xd8, 0x9e, 0xfd,
0xe9, 0x47, 0x9a, 0xe4, 0xb4, 0xde, 0x81, 0xb7, 0x67, 0xad, 0x54, 0x33, 0xde, 0x23, 0xf8, 0x31,
0x65, 0x43, 0xe6, 0x44, 0x6c, 0x7d, 0x0f, 0x58, 0x26, 0xd4, 0xaf, 0x83, 0x95, 0xe0, 0xbf, 0x67,
0xa0, 0xd4, 0x7d, 0xcd, 0x7a, 0x27, 0x2c, 0x8a, 0x9c, 0x01, 0x23, 0xef, 0x42, 0xf1, 0x8c, 0x07,
0x3d, 0x16, 0x45, 0x13, 0x59, 0x53, 0x02, 0xf9, 0x0c, 0xb2, 0x47, 0xbe, 0x27, 0x54, 0xc5, 0xde,
0x4e, 0x9d, 0x1f, 0x3d, 0xa1, 0x64, 0xe2, 0xdb, 0x09, 0xb7, 0x64, 0x0f, 0xb2, 0x98, 0xef, 0xab,
0xd4, 0x1c, 0x37, 0x81, 0x45, 0x0c, 0xe9, 0xc8, 0xd7, 0xa6, 0xf7, 0x2d, 0x53, 0x9e, 0x6f, 0xa6,
0x17, 0x4b, 0xef, 0x5b, 0x36, 0x95, 0xa0, 0x90, 0xa4, 0x0b, 0xf9, 0x67, 0xc2, 0xe1, 0x38, 0x72,
0xc4, 0x11, 0xb9, 0x97, 0xd6, 0x53, 0x63, 0xce, 0xa9, 0x14, 0x8d, 0x45, 0x27, 0x74, 0x5f, 0x7b,
0x42, 0x0e, 0x14, 0xe9, 0x4e, 0x40, 0xb6, 0x84, 0x21, 0xb8, 0x45, 0xf4, 0x41, 0xe0, 0xb3, 0x7a,
0x7e, 0x29, 0x1a, 0xd9, 0x12, 0x68, 0xdc, 0x76, 0xf2, 0x90, 0x93, 0x4d, 0xd5, 0xfa, 0x8b, 0x01,
0xa5, 0x84, 0x8f, 0x57, 0xb8, 0x07, 0xef, 0x42, 0x16, 0x1f, 0x9b, 0x2a, 0x76, 0x05, 0x79, 0x0b,
0x98, 0x70, 0xa8, 0xa4, 0x62, 0xd5, 0x3a, 0x74, 0xe3, 0xbb, 0x59, 0xa1, 0xb8, 0x44, 0xca, 0x57,
0x62, 0x2c, 0xdd, 0x5d, 0xa0, 0xb8, 0x24, 0x3b, 0x50, 0x78, 0xc6, 0x7a, 0x23, 0xee, 0x89, 0xb1,
0x74, 0x60, 0xb5, 0x5d, 0x43, 0x29, 0x9a, 0x26, 0x2f, 0xcb, 0x84, 0xc3, 0xfa, 0x12, 0x13, 0x6b,
0xaa, 0x20, 0x81, 0xec, 0x3e, 0x8e, 0xdc, 0xa8, 0x59, 0x85, 0xca, 0x35, 0xbe, 0x7a, 0xba, 0xcb,
0x5e, 0x3d, 0x5d, 0xfd, 0xea, 0x99, 0x0d, 0x08, 0x16, 0xc1, 0x84, 0x83, 0xac, 0xc7, 0x50, 0x9c,
0x24, 0x0d, 0x3e, 0x38, 0x0f, 0x5d, 0x75, 0xd2, 0xe6, 0xa1, 0x8b, 0xa6, 0x74, 0x9f, 0x1e, 0xca,
0x53, 0x0a, 0x14, 0x97, 0x93, 0x96, 0x93, 0x49, 0xb4, 0x9c, 0x5d, 0x7c, 0xcf, 0x25, 0x32, 0x07,
0x99, 0x68, 0x70, 0x19, 0x69, 0x95, 0x71, 0x1d, 0x9b, 0x31, 0x8c, 0xa4, 0x2c, 0x69, 0xc6, 0x30,
0x6a, 0xff, 0xaf, 0x00, 0xc5, 0xe3, 0xe3, 0x4e, 0x87, 0x7b, 0xee, 0x80, 0x91, 0x3f, 0x18, 0x40,
0xae, 0x3f, 0x13, 0xc8, 0x27, 0xe9, 0x09, 0xbb, 0xf8, 0xad, 0x63, 0x7e, 0xba, 0x26, 0x4a, 0x75,
0x80, 0xaf, 0x21, 0x27, 0xa7, 0x0f, 0xf2, 0xd3, 0x15, 0xa7, 0x46, 0xb3, 0xb9, 0x9c, 0x51, 0xc9,
0xee, 0x41, 0x41, 0x77, 0x70, 0x72, 0x3f, 0x55, 0xbd, 0x99, 0x01, 0xc5, 0xfc, 0x70, 0x25, 0x5e,
0x75, 0xc8, 0x6f, 0x21, 0xaf, 0x1a, 0x33, 0xb9, 0xb7, 0x04, 0x37, 0x1d, 0x11, 0xcc, 0xfb, 0xab,
0xb0, 0x4e, 0xcd, 0xd0, 0x0d, 0x38, 0xd5, 0x8c, 0xb9, 0xf6, 0x9e, 0x6a, 0xc6, 0xb5, 0x8e, 0xfe,
0x1c, 0xb2, 0xd8, 0xa9, 0x49, 0xda, 0x35, 0x4f, 0xb4, 0x72, 0x33, 0x2d, 0x5c, 0x33, 0x2d, 0xfe,
0x37, 0x58, 0x0e, 0xe5, 0x6b, 0x27, 0xbd, 0x10, 0x26, 0xfe, 0x9e, 0x30, 0xef, 0xad, 0xc0, 0x39,
0x15, 0xaf, 0x5e, 0x0a, 0xcd, 0x15, 0xfe, 0x23, 0x58, 0x2e, 0x7e, 0xee, 0xdf, 0x88, 0x00, 0xca,
0xc9, 0x2e, 0x47, 0xec, 0x14, 0xe8, 0x82, 0xa6, 0x6f, 0xb6, 0x56, 0xe6, 0x57, 0x07, 0x7e, 0x87,
0x53, 0xe7, 0x6c, 0x07, 0x24, 0xed, 0x54, 0x77, 0x2c, 0xec, 0xb5, 0xe6, 0x83, 0xb5, 0x30, 0xea,
0x70, 0x27, 0xee, 0xb0, 0xaa, 0x8b, 0x92, 0xf4, 0x86, 0x31, 0xe9, 0xc4, 0xe6, 0x8a, 0x7c, 0x4d,
0xe3, 0x63, 0xa3, 0x53, 0xfe, 0xfe, 0xea, 0xae, 0xf1, 0xcf, 0xab, 0xbb, 0xc6, 0xbf, 0xaf, 0xee,
0x1a, 0xe7, 0x5b, 0xf2, 0x1f, 0xda, 0x07, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0xe7, 0x9d, 0x1a,
0x7c, 0xf3, 0x16, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -3220,6 +3229,16 @@ func (m *SolveRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i -= len(m.XXX_unrecognized)
copy(dAtA[i:], m.XXX_unrecognized)
}
if m.Evaluate {
i--
if m.Evaluate {
dAtA[i] = 1
} else {
dAtA[i] = 0
}
i--
dAtA[i] = 0x70
}
if len(m.FrontendInputs) > 0 {
for k := range m.FrontendInputs {
v := m.FrontendInputs[k]
@ -4758,6 +4777,9 @@ func (m *SolveRequest) Size() (n int) {
n += mapEntrySize + 1 + sovGateway(uint64(mapEntrySize))
}
}
if m.Evaluate {
n += 2
}
if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized)
}
@ -7313,6 +7335,26 @@ func (m *SolveRequest) Unmarshal(dAtA []byte) error {
}
m.FrontendInputs[mapkey] = mapvalue
iNdEx = postIndex
case 14:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Evaluate", wireType)
}
var v int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
v |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
m.Evaluate = bool(v != 0)
default:
iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:])

View File

@ -107,6 +107,8 @@ message SolveRequest {
// apicaps:CapFrontendInputs
map<string, pb.Definition> FrontendInputs = 13;
bool Evaluate = 14;
}
// CacheOptionsEntry corresponds to the control.CacheOptionsEntry

View File

@ -181,29 +181,219 @@ func (m *Subrequest) GetName() string {
return ""
}
type Solve struct {
InputIDs []string `protobuf:"bytes,1,rep,name=inputIDs,proto3" json:"inputIDs,omitempty"`
OutputIDs []string `protobuf:"bytes,2,rep,name=outputIDs,proto3" json:"outputIDs,omitempty"`
Op *pb.Op `protobuf:"bytes,3,opt,name=op,proto3" json:"op,omitempty"`
// Types that are valid to be assigned to Subject:
// *Solve_File
// *Solve_Cache
Subject isSolve_Subject `protobuf_oneof:"subject"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *Solve) Reset() { *m = Solve{} }
func (m *Solve) String() string { return proto.CompactTextString(m) }
func (*Solve) ProtoMessage() {}
func (*Solve) Descriptor() ([]byte, []int) {
return fileDescriptor_689dc58a5060aff5, []int{4}
}
func (m *Solve) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Solve.Unmarshal(m, b)
}
func (m *Solve) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Solve.Marshal(b, m, deterministic)
}
func (m *Solve) XXX_Merge(src proto.Message) {
xxx_messageInfo_Solve.Merge(m, src)
}
func (m *Solve) XXX_Size() int {
return xxx_messageInfo_Solve.Size(m)
}
func (m *Solve) XXX_DiscardUnknown() {
xxx_messageInfo_Solve.DiscardUnknown(m)
}
var xxx_messageInfo_Solve proto.InternalMessageInfo
type isSolve_Subject interface {
isSolve_Subject()
}
type Solve_File struct {
File *FileAction `protobuf:"bytes,4,opt,name=file,proto3,oneof" json:"file,omitempty"`
}
type Solve_Cache struct {
Cache *ContentCache `protobuf:"bytes,5,opt,name=cache,proto3,oneof" json:"cache,omitempty"`
}
func (*Solve_File) isSolve_Subject() {}
func (*Solve_Cache) isSolve_Subject() {}
func (m *Solve) GetSubject() isSolve_Subject {
if m != nil {
return m.Subject
}
return nil
}
func (m *Solve) GetInputIDs() []string {
if m != nil {
return m.InputIDs
}
return nil
}
func (m *Solve) GetOutputIDs() []string {
if m != nil {
return m.OutputIDs
}
return nil
}
func (m *Solve) GetOp() *pb.Op {
if m != nil {
return m.Op
}
return nil
}
func (m *Solve) GetFile() *FileAction {
if x, ok := m.GetSubject().(*Solve_File); ok {
return x.File
}
return nil
}
func (m *Solve) GetCache() *ContentCache {
if x, ok := m.GetSubject().(*Solve_Cache); ok {
return x.Cache
}
return nil
}
// XXX_OneofWrappers is for the internal use of the proto package.
func (*Solve) XXX_OneofWrappers() []interface{} {
return []interface{}{
(*Solve_File)(nil),
(*Solve_Cache)(nil),
}
}
type FileAction struct {
// Index of the file action that failed the exec.
Index int64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *FileAction) Reset() { *m = FileAction{} }
func (m *FileAction) String() string { return proto.CompactTextString(m) }
func (*FileAction) ProtoMessage() {}
func (*FileAction) Descriptor() ([]byte, []int) {
return fileDescriptor_689dc58a5060aff5, []int{5}
}
func (m *FileAction) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FileAction.Unmarshal(m, b)
}
func (m *FileAction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_FileAction.Marshal(b, m, deterministic)
}
func (m *FileAction) XXX_Merge(src proto.Message) {
xxx_messageInfo_FileAction.Merge(m, src)
}
func (m *FileAction) XXX_Size() int {
return xxx_messageInfo_FileAction.Size(m)
}
func (m *FileAction) XXX_DiscardUnknown() {
xxx_messageInfo_FileAction.DiscardUnknown(m)
}
var xxx_messageInfo_FileAction proto.InternalMessageInfo
func (m *FileAction) GetIndex() int64 {
if m != nil {
return m.Index
}
return 0
}
type ContentCache struct {
// Original index of result that failed the slow cache calculation.
Index int64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ContentCache) Reset() { *m = ContentCache{} }
func (m *ContentCache) String() string { return proto.CompactTextString(m) }
func (*ContentCache) ProtoMessage() {}
func (*ContentCache) Descriptor() ([]byte, []int) {
return fileDescriptor_689dc58a5060aff5, []int{6}
}
func (m *ContentCache) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ContentCache.Unmarshal(m, b)
}
func (m *ContentCache) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ContentCache.Marshal(b, m, deterministic)
}
func (m *ContentCache) XXX_Merge(src proto.Message) {
xxx_messageInfo_ContentCache.Merge(m, src)
}
func (m *ContentCache) XXX_Size() int {
return xxx_messageInfo_ContentCache.Size(m)
}
func (m *ContentCache) XXX_DiscardUnknown() {
xxx_messageInfo_ContentCache.DiscardUnknown(m)
}
var xxx_messageInfo_ContentCache proto.InternalMessageInfo
func (m *ContentCache) GetIndex() int64 {
if m != nil {
return m.Index
}
return 0
}
func init() {
proto.RegisterType((*Vertex)(nil), "errdefs.Vertex")
proto.RegisterType((*Source)(nil), "errdefs.Source")
proto.RegisterType((*FrontendCap)(nil), "errdefs.FrontendCap")
proto.RegisterType((*Subrequest)(nil), "errdefs.Subrequest")
proto.RegisterType((*Solve)(nil), "errdefs.Solve")
proto.RegisterType((*FileAction)(nil), "errdefs.FileAction")
proto.RegisterType((*ContentCache)(nil), "errdefs.ContentCache")
}
func init() { proto.RegisterFile("errdefs.proto", fileDescriptor_689dc58a5060aff5) }
var fileDescriptor_689dc58a5060aff5 = []byte{
// 213 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x8e, 0xc1, 0x4a, 0x03, 0x31,
0x10, 0x86, 0xa9, 0x96, 0x48, 0x67, 0xd1, 0x43, 0x0e, 0x52, 0x3c, 0x6d, 0x73, 0xea, 0x41, 0x36,
0x50, 0x1f, 0x41, 0x10, 0x3c, 0x09, 0x5b, 0xf0, 0xbe, 0x69, 0x66, 0xd7, 0x60, 0x37, 0x13, 0x27,
0x89, 0xe8, 0xdb, 0xcb, 0xc6, 0x1c, 0x7b, 0x9b, 0x7f, 0xbe, 0x6f, 0x98, 0x1f, 0x6e, 0x91, 0xd9,
0xe2, 0x18, 0xbb, 0xc0, 0x94, 0x48, 0xde, 0xd4, 0xf8, 0xf0, 0x38, 0xb9, 0xf4, 0x91, 0x4d, 0x77,
0xa2, 0x59, 0xcf, 0x64, 0x7e, 0xb5, 0xc9, 0xee, 0x6c, 0x3f, 0x5d, 0xd2, 0x91, 0xce, 0xdf, 0xc8,
0x3a, 0x18, 0x4d, 0xa1, 0x9e, 0xa9, 0x16, 0xc4, 0x3b, 0x72, 0xc2, 0x1f, 0x79, 0x0f, 0xc2, 0xba,
0x09, 0x63, 0xda, 0xae, 0xda, 0xd5, 0x7e, 0xd3, 0xd7, 0xa4, 0xde, 0x40, 0x1c, 0x29, 0xf3, 0x09,
0xa5, 0x82, 0xb5, 0xf3, 0x23, 0x15, 0xde, 0x1c, 0xee, 0xba, 0x60, 0xba, 0x7f, 0xf2, 0xea, 0x47,
0xea, 0x0b, 0x93, 0x3b, 0x10, 0x3c, 0xf8, 0x09, 0xe3, 0xf6, 0xaa, 0xbd, 0xde, 0x37, 0x87, 0xcd,
0x62, 0xf5, 0xcb, 0xa6, 0xaf, 0x40, 0xed, 0xa0, 0x79, 0x61, 0xf2, 0x09, 0xbd, 0x7d, 0x1e, 0x82,
0x94, 0xb0, 0xf6, 0xc3, 0x8c, 0xf5, 0x6b, 0x99, 0x55, 0x0b, 0x70, 0xcc, 0x86, 0xf1, 0x2b, 0x63,
0x4c, 0x97, 0x0c, 0x23, 0x4a, 0xfd, 0xa7, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x75, 0x4b, 0xfe,
0xad, 0x06, 0x01, 0x00, 0x00,
// 349 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x91, 0xcd, 0x8e, 0xd3, 0x30,
0x14, 0x85, 0x27, 0xbf, 0x43, 0x6e, 0x81, 0x85, 0x81, 0x51, 0x34, 0x62, 0x91, 0xb1, 0x58, 0x14,
0x09, 0x12, 0xa9, 0x3c, 0x01, 0x14, 0x55, 0xed, 0xaa, 0x92, 0x2b, 0xb1, 0x8f, 0x93, 0x9b, 0xd6,
0x90, 0xda, 0xc6, 0xb1, 0x51, 0x79, 0x39, 0x9e, 0x0d, 0xc5, 0x0d, 0x2d, 0x8b, 0xee, 0x72, 0xee,
0xf7, 0xe5, 0x26, 0xc7, 0x86, 0x17, 0x68, 0x4c, 0x8b, 0xdd, 0x50, 0x6a, 0xa3, 0xac, 0x22, 0xf7,
0x53, 0x7c, 0xfc, 0xb0, 0x17, 0xf6, 0xe0, 0x78, 0xd9, 0xa8, 0x63, 0x75, 0x54, 0xfc, 0x77, 0xc5,
0x9d, 0xe8, 0xdb, 0x1f, 0xc2, 0x56, 0x83, 0xea, 0x7f, 0xa1, 0xa9, 0x34, 0xaf, 0x94, 0x9e, 0x5e,
0xa3, 0x05, 0xa4, 0xdf, 0xd0, 0x58, 0x3c, 0x91, 0x07, 0x48, 0x5b, 0xb1, 0xc7, 0xc1, 0xe6, 0x41,
0x11, 0xcc, 0x33, 0x36, 0x25, 0xba, 0x85, 0x74, 0xa7, 0x9c, 0x69, 0x90, 0x50, 0x88, 0x85, 0xec,
0x94, 0xe7, 0xb3, 0xc5, 0xcb, 0x52, 0xf3, 0xf2, 0x4c, 0x36, 0xb2, 0x53, 0xcc, 0x33, 0xf2, 0x04,
0xa9, 0xa9, 0xe5, 0x1e, 0x87, 0x3c, 0x2c, 0xa2, 0xf9, 0x6c, 0x91, 0x8d, 0x16, 0x1b, 0x27, 0x6c,
0x02, 0xf4, 0x09, 0x66, 0x2b, 0xa3, 0xa4, 0x45, 0xd9, 0x2e, 0x6b, 0x4d, 0x08, 0xc4, 0xb2, 0x3e,
0xe2, 0xf4, 0x55, 0xff, 0x4c, 0x0b, 0x80, 0x9d, 0xe3, 0x06, 0x7f, 0x3a, 0x1c, 0xec, 0x4d, 0xe3,
0x4f, 0x00, 0xc9, 0x6e, 0xec, 0x43, 0x1e, 0xe1, 0x99, 0x90, 0xda, 0xd9, 0xcd, 0xd7, 0x21, 0x0f,
0x8a, 0x68, 0x9e, 0xb1, 0x4b, 0x26, 0x6f, 0x21, 0x53, 0xce, 0x4e, 0x30, 0xf4, 0xf0, 0x3a, 0x20,
0x0f, 0x10, 0x2a, 0x9d, 0x47, 0xbe, 0x4d, 0x3a, 0xfe, 0xe7, 0x56, 0xb3, 0x50, 0x69, 0xf2, 0x1e,
0xe2, 0x4e, 0xf4, 0x98, 0xc7, 0x9e, 0xbc, 0x2a, 0xff, 0x1d, 0xf4, 0x4a, 0xf4, 0xf8, 0xb9, 0xb1,
0x42, 0xc9, 0xf5, 0x1d, 0xf3, 0x0a, 0xf9, 0x08, 0x49, 0x53, 0x37, 0x07, 0xcc, 0x13, 0xef, 0xbe,
0xb9, 0xb8, 0x4b, 0x5f, 0xd0, 0x2e, 0x47, 0xb8, 0xbe, 0x63, 0x67, 0xeb, 0x4b, 0x06, 0xf7, 0x83,
0xe3, 0xdf, 0xb1, 0xb1, 0x94, 0x02, 0x5c, 0xf7, 0x91, 0xd7, 0x90, 0x08, 0xd9, 0xe2, 0xc9, 0x77,
0x8c, 0xd8, 0x39, 0xd0, 0x77, 0xf0, 0xfc, 0xff, 0x3d, 0xb7, 0x2d, 0x9e, 0xfa, 0x9b, 0xfc, 0xf4,
0x37, 0x00, 0x00, 0xff, 0xff, 0x2c, 0x24, 0x2d, 0xe3, 0x11, 0x02, 0x00, 0x00,
}

View File

@ -20,3 +20,24 @@ message FrontendCap {
message Subrequest {
string name = 1;
}
message Solve {
repeated string inputIDs = 1;
repeated string outputIDs = 2;
pb.Op op = 3;
oneof subject {
FileAction file = 4;
ContentCache cache = 5;
}
}
message FileAction {
// Index of the file action that failed the exec.
int64 index = 1;
}
message ContentCache {
// Original index of result that failed the slow cache calculation.
int64 index = 1;
}

View File

@ -1,6 +1,6 @@
package errdefs
import fmt "fmt"
import "fmt"
// ExitError will be returned when the container process exits with a non-zero
// exit code.

19
solver/errdefs/op.go Normal file
View File

@ -0,0 +1,19 @@
package errdefs
import "github.com/moby/buildkit/solver/pb"
type OpError struct {
error
Op *pb.Op
}
func (e *OpError) Unwrap() error {
return e.error
}
func WithOp(err error, op *pb.Op) error {
if err == nil {
return nil
}
return &OpError{error: err, Op: op}
}

73
solver/errdefs/solve.go Normal file
View File

@ -0,0 +1,73 @@
package errdefs
import (
"bytes"
"errors"
"github.com/containerd/typeurl"
"github.com/golang/protobuf/jsonpb"
"github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/grpcerrors"
)
func init() {
typeurl.Register((*Solve)(nil), "github.com/moby/buildkit", "errdefs.Solve+json")
}
type IsSolve_Subject isSolve_Subject
// SolveError will be returned when an error is encountered during a solve that
// has an exec op.
type SolveError struct {
Solve
Err error
}
func (e *SolveError) Error() string {
return e.Err.Error()
}
func (e *SolveError) Unwrap() error {
return e.Err
}
func (e *SolveError) ToProto() grpcerrors.TypedErrorProto {
return &e.Solve
}
func WithSolveError(err error, subject IsSolve_Subject, inputIDs, outputIDs []string) error {
if err == nil {
return nil
}
var (
oe *OpError
op *pb.Op
)
if errors.As(err, &oe) {
op = oe.Op
}
return &SolveError{
Err: err,
Solve: Solve{
InputIDs: inputIDs,
OutputIDs: outputIDs,
Op: op,
Subject: subject,
},
}
}
func (v *Solve) WrapError(err error) error {
return &SolveError{Err: err, Solve: *v}
}
func (v *Solve) MarshalJSON() ([]byte, error) {
m := jsonpb.Marshaler{}
buf := new(bytes.Buffer)
err := m.Marshal(buf, v)
return buf.Bytes(), err
}
func (v *Solve) UnmarshalJSON(b []byte) error {
return jsonpb.Unmarshal(bytes.NewReader(b), v)
}

View File

@ -617,9 +617,11 @@ func (s *sharedOp) LoadCache(ctx context.Context, rec *CacheRecord) (Result, err
return res, err
}
// CalcSlowCache computes the digest of an input that is ready and has been
// evaluated, hence "slow" cache.
func (s *sharedOp) CalcSlowCache(ctx context.Context, index Index, p PreprocessFunc, f ResultBasedCacheFunc, res Result) (dgst digest.Digest, err error) {
defer func() {
err = errdefs.WrapVertex(err, s.st.origDigest)
err = errdefs.WrapVertex(WrapSlowCache(err, index, NewSharedResult(res).Clone()), s.st.origDigest)
}()
key, err := s.g.Do(ctx, fmt.Sprintf("slow-compute-%d", index), func(ctx context.Context) (interface{}, error) {
s.slowMu.Lock()
@ -880,3 +882,28 @@ func notifyCompleted(ctx context.Context, v *client.Vertex, err error, cached bo
}
pw.Write(v.Digest.String(), *v)
}
type SlowCacheError struct {
error
Index Index
Result Result
}
func (e *SlowCacheError) Unwrap() error {
return e.error
}
func (e *SlowCacheError) ToSubject() errdefs.IsSolve_Subject {
return &errdefs.Solve_Cache{
Cache: &errdefs.ContentCache{
Index: int64(e.Index),
},
}
}
func WrapSlowCache(err error, index Index, res Result) error {
if err == nil {
return nil
}
return &SlowCacheError{Index: index, Result: res, error: err}
}

View File

@ -125,6 +125,10 @@ func (b *llbBridge) Solve(ctx context.Context, req frontend.SolveRequest, sid st
if req.Definition != nil && req.Definition.Def != nil {
res = &frontend.Result{Ref: newResultProxy(b, req)}
if req.Evaluate {
_, err := res.Ref.Result(ctx)
return res, err
}
} else if req.Frontend != "" {
f, ok := b.frontends[req.Frontend]
if !ok {
@ -196,6 +200,21 @@ func (rp *resultProxy) wrapError(err error) error {
}
}
}
var op *pb.Op
for _, dt := range rp.def.Def {
var curr pb.Op
if err := (&curr).Unmarshal(dt); err != nil {
return errors.Wrap(err, "failed to parse llb proto op")
}
if ve.Digest == digest.FromBytes(dt).String() {
op = &curr
break
}
}
if op != nil {
err = errdefs.WithOp(err, op)
}
}
return err
}

View File

@ -0,0 +1,41 @@
package errdefs
import (
"github.com/moby/buildkit/solver"
)
// ExecError will be returned when an error is encountered when evaluating an op.
type ExecError struct {
error
Inputs []solver.Result
Outputs []solver.Result
}
func (e *ExecError) Unwrap() error {
return e.error
}
func (e *ExecError) EachRef(fn func(solver.Result) error) (err error) {
for _, res := range e.Inputs {
if err1 := fn(res); err1 != nil && err == nil {
err = err1
}
}
for _, res := range e.Outputs {
if err1 := fn(res); err1 != nil && err == nil {
err = err1
}
}
return err
}
func WithExecError(err error, inputs, outputs []solver.Result) error {
if err == nil {
return nil
}
return &ExecError{
error: err,
Inputs: inputs,
Outputs: outputs,
}
}

View File

@ -0,0 +1,34 @@
package errdefs
import (
serrdefs "github.com/moby/buildkit/solver/errdefs"
)
// FileActionError will be returned when an error is encountered when solving
// a fileop.
type FileActionError struct {
error
Index int
}
func (e *FileActionError) Unwrap() error {
return e.error
}
func (e *FileActionError) ToSubject() serrdefs.IsSolve_Subject {
return &serrdefs.Solve_File{
File: &serrdefs.FileAction{
Index: int64(e.Index),
},
}
}
func WithFileActionError(err error, idx int) error {
if err == nil {
return nil
}
return &FileActionError{
error: err,
Index: idx,
}
}

View File

@ -19,6 +19,7 @@ import (
"github.com/moby/buildkit/snapshot"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/llbsolver"
"github.com/moby/buildkit/solver/llbsolver/errdefs"
"github.com/moby/buildkit/solver/llbsolver/mounts"
"github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/progress/logs"
@ -397,9 +398,12 @@ func (e *execOp) Exec(ctx context.Context, g session.Group, inputs []solver.Resu
defer stdout.Close()
defer stderr.Close()
if err := e.exec.Run(ctx, "", mountWithSession(root, g), mounts, executor.ProcessInfo{Meta: meta, Stdin: nil, Stdout: stdout, Stderr: stderr}, nil); err != nil {
return nil, errors.Wrapf(err, "executor failed running %v", meta.Args)
}
execErr := e.exec.Run(ctx, "", mountWithSession(root, g), mounts, executor.ProcessInfo{
Meta: meta,
Stdin: nil,
Stdout: stdout,
Stderr: stderr,
}, nil)
refs := []solver.Result{}
for i, out := range outputs {
@ -414,6 +418,10 @@ func (e *execOp) Exec(ctx context.Context, g session.Group, inputs []solver.Resu
}
outputs[i] = nil
}
if execErr != nil {
return nil, errdefs.WithExecError(errors.Wrapf(execErr, "executor failed running %v", meta.Args), inputs, refs)
}
return refs, nil
}

View File

@ -15,6 +15,7 @@ import (
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/llbsolver"
"github.com/moby/buildkit/solver/llbsolver/errdefs"
"github.com/moby/buildkit/solver/llbsolver/file"
"github.com/moby/buildkit/solver/llbsolver/ops/fileoptypes"
"github.com/moby/buildkit/solver/pb"
@ -44,7 +45,7 @@ func NewFileOp(v solver.Vertex, op *pb.Op_File, cm cache.Manager, md *metadata.S
md: md,
numInputs: len(v.Inputs()),
w: w,
solver: NewFileOpSolver(&file.Backend{}, file.NewRefManager(cm)),
solver: NewFileOpSolver(w, &file.Backend{}, file.NewRefManager(cm)),
}, nil
}
@ -159,6 +160,7 @@ func (f *fileOp) Exec(ctx context.Context, g session.Group, inputs []solver.Resu
outs, err := f.solver.Solve(ctx, inpRefs, f.op.Actions, g)
if err != nil {
return nil, err
// return nil, errdefs.WithExecError(err, inputs, nil)
}
outResults := make([]solver.Result, 0, len(outs))
@ -258,8 +260,9 @@ func processOwner(chopt *pb.ChownOpt, selectors map[int]map[llbsolver.Selector]s
return nil
}
func NewFileOpSolver(b fileoptypes.Backend, r fileoptypes.RefManager) *FileOpSolver {
func NewFileOpSolver(w worker.Worker, b fileoptypes.Backend, r fileoptypes.RefManager) *FileOpSolver {
return &FileOpSolver{
w: w,
b: b,
r: r,
outs: map[int]int{},
@ -268,6 +271,7 @@ func NewFileOpSolver(b fileoptypes.Backend, r fileoptypes.RefManager) *FileOpSol
}
type FileOpSolver struct {
w worker.Worker
b fileoptypes.Backend
r fileoptypes.RefManager
@ -343,7 +347,7 @@ func (s *FileOpSolver) Solve(ctx context.Context, inputs []fileoptypes.Ref, acti
}
inp, err := s.getInput(ctx, idx, inputs, actions, g)
if err != nil {
return err
return errdefs.WithFileActionError(err, idx)
}
outs[i] = inp.ref
return nil
@ -406,8 +410,22 @@ func (s *FileOpSolver) getInput(ctx context.Context, idx int, inputs []fileoptyp
for _, m := range toRelease {
m.Release(context.TODO())
}
if err != nil && inpMount != nil && inpMountPrepared {
inpMount.Release(context.TODO())
if err != nil && inpMount != nil {
if inpMountPrepared {
inpMount.Release(context.TODO())
}
var (
inputRes []solver.Result
outputRes []solver.Result
)
for _, input := range inputs {
inputRes = append(inputRes, worker.NewWorkerRefResult(input.(cache.ImmutableRef), s.w))
}
ref, cerr := s.r.Commit(ctx, inpMount)
if cerr == nil {
outputRes = append(outputRes, worker.NewWorkerRefResult(ref.(cache.ImmutableRef), s.w))
}
err = errdefs.WithExecError(err, inputRes, outputRes)
}
}()

View File

@ -16,6 +16,7 @@ import (
"github.com/moby/buildkit/frontend/gateway"
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/llbsolver/errdefs"
"github.com/moby/buildkit/util/compression"
"github.com/moby/buildkit/util/entitlements"
"github.com/moby/buildkit/util/progress"
@ -142,6 +143,19 @@ func (s *Solver) Solve(ctx context.Context, id string, sessionID string, req fro
res.EachRef(func(ref solver.ResultProxy) error {
eg.Go(func() error {
_, err := ref.Result(ctx2)
if err != nil {
var ee *errdefs.ExecError
if errors.As(err, &ee) {
ee.EachRef(func(res solver.Result) error {
workerRef, ok := res.Sys().(*worker.WorkerRef)
if !ok {
return nil
}
return workerRef.ImmutableRef.Release(ctx)
})
}
}
return err
})
return nil

1290
vendor/github.com/golang/protobuf/jsonpb/jsonpb.go generated vendored Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,338 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: google/protobuf/struct.proto
package structpb
import (
fmt "fmt"
proto "github.com/golang/protobuf/proto"
math "math"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
// A compilation error at this line likely means your copy of the
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
// `NullValue` is a singleton enumeration to represent the null value for the
// `Value` type union.
//
// The JSON representation for `NullValue` is JSON `null`.
type NullValue int32
const (
// Null value.
NullValue_NULL_VALUE NullValue = 0
)
var NullValue_name = map[int32]string{
0: "NULL_VALUE",
}
var NullValue_value = map[string]int32{
"NULL_VALUE": 0,
}
func (x NullValue) String() string {
return proto.EnumName(NullValue_name, int32(x))
}
func (NullValue) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_df322afd6c9fb402, []int{0}
}
func (NullValue) XXX_WellKnownType() string { return "NullValue" }
// `Struct` represents a structured data value, consisting of fields
// which map to dynamically typed values. In some languages, `Struct`
// might be supported by a native representation. For example, in
// scripting languages like JS a struct is represented as an
// object. The details of that representation are described together
// with the proto support for the language.
//
// The JSON representation for `Struct` is JSON object.
type Struct struct {
// Unordered map of dynamically typed values.
Fields map[string]*Value `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *Struct) Reset() { *m = Struct{} }
func (m *Struct) String() string { return proto.CompactTextString(m) }
func (*Struct) ProtoMessage() {}
func (*Struct) Descriptor() ([]byte, []int) {
return fileDescriptor_df322afd6c9fb402, []int{0}
}
func (*Struct) XXX_WellKnownType() string { return "Struct" }
func (m *Struct) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Struct.Unmarshal(m, b)
}
func (m *Struct) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Struct.Marshal(b, m, deterministic)
}
func (m *Struct) XXX_Merge(src proto.Message) {
xxx_messageInfo_Struct.Merge(m, src)
}
func (m *Struct) XXX_Size() int {
return xxx_messageInfo_Struct.Size(m)
}
func (m *Struct) XXX_DiscardUnknown() {
xxx_messageInfo_Struct.DiscardUnknown(m)
}
var xxx_messageInfo_Struct proto.InternalMessageInfo
func (m *Struct) GetFields() map[string]*Value {
if m != nil {
return m.Fields
}
return nil
}
// `Value` represents a dynamically typed value which can be either
// null, a number, a string, a boolean, a recursive struct value, or a
// list of values. A producer of value is expected to set one of that
// variants, absence of any variant indicates an error.
//
// The JSON representation for `Value` is JSON value.
type Value struct {
// The kind of value.
//
// Types that are valid to be assigned to Kind:
// *Value_NullValue
// *Value_NumberValue
// *Value_StringValue
// *Value_BoolValue
// *Value_StructValue
// *Value_ListValue
Kind isValue_Kind `protobuf_oneof:"kind"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *Value) Reset() { *m = Value{} }
func (m *Value) String() string { return proto.CompactTextString(m) }
func (*Value) ProtoMessage() {}
func (*Value) Descriptor() ([]byte, []int) {
return fileDescriptor_df322afd6c9fb402, []int{1}
}
func (*Value) XXX_WellKnownType() string { return "Value" }
func (m *Value) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Value.Unmarshal(m, b)
}
func (m *Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Value.Marshal(b, m, deterministic)
}
func (m *Value) XXX_Merge(src proto.Message) {
xxx_messageInfo_Value.Merge(m, src)
}
func (m *Value) XXX_Size() int {
return xxx_messageInfo_Value.Size(m)
}
func (m *Value) XXX_DiscardUnknown() {
xxx_messageInfo_Value.DiscardUnknown(m)
}
var xxx_messageInfo_Value proto.InternalMessageInfo
type isValue_Kind interface {
isValue_Kind()
}
type Value_NullValue struct {
NullValue NullValue `protobuf:"varint,1,opt,name=null_value,json=nullValue,proto3,enum=google.protobuf.NullValue,oneof"`
}
type Value_NumberValue struct {
NumberValue float64 `protobuf:"fixed64,2,opt,name=number_value,json=numberValue,proto3,oneof"`
}
type Value_StringValue struct {
StringValue string `protobuf:"bytes,3,opt,name=string_value,json=stringValue,proto3,oneof"`
}
type Value_BoolValue struct {
BoolValue bool `protobuf:"varint,4,opt,name=bool_value,json=boolValue,proto3,oneof"`
}
type Value_StructValue struct {
StructValue *Struct `protobuf:"bytes,5,opt,name=struct_value,json=structValue,proto3,oneof"`
}
type Value_ListValue struct {
ListValue *ListValue `protobuf:"bytes,6,opt,name=list_value,json=listValue,proto3,oneof"`
}
func (*Value_NullValue) isValue_Kind() {}
func (*Value_NumberValue) isValue_Kind() {}
func (*Value_StringValue) isValue_Kind() {}
func (*Value_BoolValue) isValue_Kind() {}
func (*Value_StructValue) isValue_Kind() {}
func (*Value_ListValue) isValue_Kind() {}
func (m *Value) GetKind() isValue_Kind {
if m != nil {
return m.Kind
}
return nil
}
func (m *Value) GetNullValue() NullValue {
if x, ok := m.GetKind().(*Value_NullValue); ok {
return x.NullValue
}
return NullValue_NULL_VALUE
}
func (m *Value) GetNumberValue() float64 {
if x, ok := m.GetKind().(*Value_NumberValue); ok {
return x.NumberValue
}
return 0
}
func (m *Value) GetStringValue() string {
if x, ok := m.GetKind().(*Value_StringValue); ok {
return x.StringValue
}
return ""
}
func (m *Value) GetBoolValue() bool {
if x, ok := m.GetKind().(*Value_BoolValue); ok {
return x.BoolValue
}
return false
}
func (m *Value) GetStructValue() *Struct {
if x, ok := m.GetKind().(*Value_StructValue); ok {
return x.StructValue
}
return nil
}
func (m *Value) GetListValue() *ListValue {
if x, ok := m.GetKind().(*Value_ListValue); ok {
return x.ListValue
}
return nil
}
// XXX_OneofWrappers is for the internal use of the proto package.
func (*Value) XXX_OneofWrappers() []interface{} {
return []interface{}{
(*Value_NullValue)(nil),
(*Value_NumberValue)(nil),
(*Value_StringValue)(nil),
(*Value_BoolValue)(nil),
(*Value_StructValue)(nil),
(*Value_ListValue)(nil),
}
}
// `ListValue` is a wrapper around a repeated field of values.
//
// The JSON representation for `ListValue` is JSON array.
type ListValue struct {
// Repeated field of dynamically typed values.
Values []*Value `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ListValue) Reset() { *m = ListValue{} }
func (m *ListValue) String() string { return proto.CompactTextString(m) }
func (*ListValue) ProtoMessage() {}
func (*ListValue) Descriptor() ([]byte, []int) {
return fileDescriptor_df322afd6c9fb402, []int{2}
}
func (*ListValue) XXX_WellKnownType() string { return "ListValue" }
func (m *ListValue) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListValue.Unmarshal(m, b)
}
func (m *ListValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ListValue.Marshal(b, m, deterministic)
}
func (m *ListValue) XXX_Merge(src proto.Message) {
xxx_messageInfo_ListValue.Merge(m, src)
}
func (m *ListValue) XXX_Size() int {
return xxx_messageInfo_ListValue.Size(m)
}
func (m *ListValue) XXX_DiscardUnknown() {
xxx_messageInfo_ListValue.DiscardUnknown(m)
}
var xxx_messageInfo_ListValue proto.InternalMessageInfo
func (m *ListValue) GetValues() []*Value {
if m != nil {
return m.Values
}
return nil
}
func init() {
proto.RegisterEnum("google.protobuf.NullValue", NullValue_name, NullValue_value)
proto.RegisterType((*Struct)(nil), "google.protobuf.Struct")
proto.RegisterMapType((map[string]*Value)(nil), "google.protobuf.Struct.FieldsEntry")
proto.RegisterType((*Value)(nil), "google.protobuf.Value")
proto.RegisterType((*ListValue)(nil), "google.protobuf.ListValue")
}
func init() {
proto.RegisterFile("google/protobuf/struct.proto", fileDescriptor_df322afd6c9fb402)
}
var fileDescriptor_df322afd6c9fb402 = []byte{
// 417 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x92, 0x41, 0x8b, 0xd3, 0x40,
0x14, 0xc7, 0x3b, 0xc9, 0x36, 0x98, 0x17, 0x59, 0x97, 0x11, 0xb4, 0xac, 0xa2, 0xa1, 0x7b, 0x09,
0x22, 0x29, 0xd6, 0x8b, 0x18, 0x2f, 0x06, 0xd6, 0x5d, 0x30, 0x2c, 0x31, 0xba, 0x15, 0xbc, 0x94,
0x26, 0x4d, 0x63, 0xe8, 0x74, 0x26, 0x24, 0x33, 0x4a, 0x8f, 0x7e, 0x0b, 0xcf, 0x1e, 0x3d, 0xfa,
0xe9, 0x3c, 0xca, 0xcc, 0x24, 0xa9, 0xb4, 0xf4, 0x94, 0xbc, 0xf7, 0x7e, 0xef, 0x3f, 0xef, 0xff,
0x66, 0xe0, 0x71, 0xc1, 0x58, 0x41, 0xf2, 0x49, 0x55, 0x33, 0xce, 0x52, 0xb1, 0x9a, 0x34, 0xbc,
0x16, 0x19, 0xf7, 0x55, 0x8c, 0xef, 0xe9, 0xaa, 0xdf, 0x55, 0xc7, 0x3f, 0x11, 0x58, 0x1f, 0x15,
0x81, 0x03, 0xb0, 0x56, 0x65, 0x4e, 0x96, 0xcd, 0x08, 0xb9, 0xa6, 0xe7, 0x4c, 0x2f, 0xfc, 0x3d,
0xd8, 0xd7, 0xa0, 0xff, 0x4e, 0x51, 0x97, 0x94, 0xd7, 0xdb, 0xa4, 0x6d, 0x39, 0xff, 0x00, 0xce,
0x7f, 0x69, 0x7c, 0x06, 0xe6, 0x3a, 0xdf, 0x8e, 0x90, 0x8b, 0x3c, 0x3b, 0x91, 0xbf, 0xf8, 0x39,
0x0c, 0xbf, 0x2d, 0x88, 0xc8, 0x47, 0x86, 0x8b, 0x3c, 0x67, 0xfa, 0xe0, 0x40, 0x7c, 0x26, 0xab,
0x89, 0x86, 0x5e, 0x1b, 0xaf, 0xd0, 0xf8, 0x8f, 0x01, 0x43, 0x95, 0xc4, 0x01, 0x00, 0x15, 0x84,
0xcc, 0xb5, 0x80, 0x14, 0x3d, 0x9d, 0x9e, 0x1f, 0x08, 0xdc, 0x08, 0x42, 0x14, 0x7f, 0x3d, 0x48,
0x6c, 0xda, 0x05, 0xf8, 0x02, 0xee, 0x52, 0xb1, 0x49, 0xf3, 0x7a, 0xbe, 0x3b, 0x1f, 0x5d, 0x0f,
0x12, 0x47, 0x67, 0x7b, 0xa8, 0xe1, 0x75, 0x49, 0x8b, 0x16, 0x32, 0xe5, 0xe0, 0x12, 0xd2, 0x59,
0x0d, 0x3d, 0x05, 0x48, 0x19, 0xeb, 0xc6, 0x38, 0x71, 0x91, 0x77, 0x47, 0x1e, 0x25, 0x73, 0x1a,
0x78, 0xa3, 0x54, 0x44, 0xc6, 0x5b, 0x64, 0xa8, 0xac, 0x3e, 0x3c, 0xb2, 0xc7, 0x56, 0x5e, 0x64,
0xbc, 0x77, 0x49, 0xca, 0xa6, 0xeb, 0xb5, 0x54, 0xef, 0xa1, 0xcb, 0xa8, 0x6c, 0x78, 0xef, 0x92,
0x74, 0x41, 0x68, 0xc1, 0xc9, 0xba, 0xa4, 0xcb, 0x71, 0x00, 0x76, 0x4f, 0x60, 0x1f, 0x2c, 0x25,
0xd6, 0xdd, 0xe8, 0xb1, 0xa5, 0xb7, 0xd4, 0xb3, 0x47, 0x60, 0xf7, 0x4b, 0xc4, 0xa7, 0x00, 0x37,
0xb7, 0x51, 0x34, 0x9f, 0xbd, 0x8d, 0x6e, 0x2f, 0xcf, 0x06, 0xe1, 0x0f, 0x04, 0xf7, 0x33, 0xb6,
0xd9, 0x97, 0x08, 0x1d, 0xed, 0x26, 0x96, 0x71, 0x8c, 0xbe, 0xbc, 0x28, 0x4a, 0xfe, 0x55, 0xa4,
0x7e, 0xc6, 0x36, 0x93, 0x82, 0x91, 0x05, 0x2d, 0x76, 0x4f, 0xb1, 0xe2, 0xdb, 0x2a, 0x6f, 0xda,
0x17, 0x19, 0xe8, 0x4f, 0x95, 0xfe, 0x45, 0xe8, 0x97, 0x61, 0x5e, 0xc5, 0xe1, 0x6f, 0xe3, 0xc9,
0x95, 0x16, 0x8f, 0xbb, 0xf9, 0x3e, 0xe7, 0x84, 0xbc, 0xa7, 0xec, 0x3b, 0xfd, 0x24, 0x3b, 0x53,
0x4b, 0x49, 0xbd, 0xfc, 0x17, 0x00, 0x00, 0xff, 0xff, 0xe8, 0x1b, 0x59, 0xf8, 0xe5, 0x02, 0x00,
0x00,
}

View File

@ -0,0 +1,95 @@
// Protocol Buffers - Google's data interchange format
// Copyright 2008 Google Inc. All rights reserved.
// https://developers.google.com/protocol-buffers/
//
// Redistribution and use in source and binary forms, with or without
// modification, are permitted provided that the following conditions are
// met:
//
// * Redistributions of source code must retain the above copyright
// notice, this list of conditions and the following disclaimer.
// * Redistributions in binary form must reproduce the above
// copyright notice, this list of conditions and the following disclaimer
// in the documentation and/or other materials provided with the
// distribution.
// * Neither the name of Google Inc. nor the names of its
// contributors may be used to endorse or promote products derived from
// this software without specific prior written permission.
//
// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
syntax = "proto3";
package google.protobuf;
option csharp_namespace = "Google.Protobuf.WellKnownTypes";
option cc_enable_arenas = true;
option go_package = "github.com/golang/protobuf/ptypes/struct;structpb";
option java_package = "com.google.protobuf";
option java_outer_classname = "StructProto";
option java_multiple_files = true;
option objc_class_prefix = "GPB";
// `Struct` represents a structured data value, consisting of fields
// which map to dynamically typed values. In some languages, `Struct`
// might be supported by a native representation. For example, in
// scripting languages like JS a struct is represented as an
// object. The details of that representation are described together
// with the proto support for the language.
//
// The JSON representation for `Struct` is JSON object.
message Struct {
// Unordered map of dynamically typed values.
map<string, Value> fields = 1;
}
// `Value` represents a dynamically typed value which can be either
// null, a number, a string, a boolean, a recursive struct value, or a
// list of values. A producer of value is expected to set one of that
// variants, absence of any variant indicates an error.
//
// The JSON representation for `Value` is JSON value.
message Value {
// The kind of value.
oneof kind {
// Represents a null value.
NullValue null_value = 1;
// Represents a double value.
double number_value = 2;
// Represents a string value.
string string_value = 3;
// Represents a boolean value.
bool bool_value = 4;
// Represents a structured value.
Struct struct_value = 5;
// Represents a repeated `Value`.
ListValue list_value = 6;
}
}
// `NullValue` is a singleton enumeration to represent the null value for the
// `Value` type union.
//
// The JSON representation for `NullValue` is JSON `null`.
enum NullValue {
// Null value.
NULL_VALUE = 0;
}
// `ListValue` is a wrapper around a repeated field of values.
//
// The JSON representation for `ListValue` is JSON array.
message ListValue {
// Repeated field of dynamically typed values.
repeated Value values = 1;
}

2
vendor/modules.txt vendored
View File

@ -223,10 +223,12 @@ github.com/gogo/protobuf/types
# github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e
github.com/golang/groupcache/lru
# github.com/golang/protobuf v1.4.2 => github.com/golang/protobuf v1.3.5
github.com/golang/protobuf/jsonpb
github.com/golang/protobuf/proto
github.com/golang/protobuf/ptypes
github.com/golang/protobuf/ptypes/any
github.com/golang/protobuf/ptypes/duration
github.com/golang/protobuf/ptypes/struct
github.com/golang/protobuf/ptypes/timestamp
# github.com/google/crfs v0.0.0-20191108021818-71d77da419c9
github.com/google/crfs/stargz