Merge pull request #1286 from hinshun/llbstate-from-result

Allow previous solve results to be used in new solves
v0.7
Tõnis Tiigi 2020-01-23 16:41:55 -08:00 committed by GitHub
commit 490b277c91
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 704 additions and 163 deletions

View File

@ -94,6 +94,7 @@ func TestIntegration(t *testing.T) {
testExtraHosts, testExtraHosts,
testNetworkMode, testNetworkMode,
testFrontendMetadataReturn, testFrontendMetadataReturn,
testFrontendUseSolveResults,
testSSHMount, testSSHMount,
testStdinClosed, testStdinClosed,
testHostnameLookup, testHostnameLookup,
@ -1461,6 +1462,72 @@ func testFrontendMetadataReturn(t *testing.T, sb integration.Sandbox) {
checkAllReleasable(t, c, sb, true) checkAllReleasable(t, c, sb, true)
} }
func testFrontendUseSolveResults(t *testing.T, sb integration.Sandbox) {
requiresLinux(t)
c, err := New(context.TODO(), sb.Address())
require.NoError(t, err)
defer c.Close()
frontend := func(ctx context.Context, c gateway.Client) (*gateway.Result, error) {
st := llb.Scratch().File(
llb.Mkfile("foo", 0600, []byte("data")),
)
def, err := st.Marshal()
if err != nil {
return nil, err
}
res, err := c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
if err != nil {
return nil, err
}
ref, err := res.SingleRef()
if err != nil {
return nil, err
}
st2, err := ref.ToState()
if err != nil {
return nil, err
}
st = llb.Scratch().File(
llb.Copy(st2, "foo", "foo2"),
)
def, err = st.Marshal()
if err != nil {
return nil, err
}
return c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
}
destDir, err := ioutil.TempDir("", "buildkit")
require.NoError(t, err)
defer os.RemoveAll(destDir)
_, err = c.Build(context.TODO(), SolveOpt{
Exports: []ExportEntry{
{
Type: ExporterLocal,
OutputDir: destDir,
},
},
}, "", frontend, nil)
require.NoError(t, err)
dt, err := ioutil.ReadFile(filepath.Join(destDir, "foo2"))
require.NoError(t, err)
require.Equal(t, dt, []byte("data"))
}
func testExporterTargetExists(t *testing.T, sb integration.Sandbox) { func testExporterTargetExists(t *testing.T, sb integration.Sandbox) {
requiresLinux(t) requiresLinux(t)
c, err := New(context.TODO(), sb.Address()) c, err := New(context.TODO(), sb.Address())

158
client/llb/definition.go Normal file
View File

@ -0,0 +1,158 @@
package llb
import (
"github.com/moby/buildkit/solver/pb"
digest "github.com/opencontainers/go-digest"
specs "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
// DefinitionOp implements llb.Vertex using a marshalled definition.
//
// For example, after marshalling a LLB state and sending over the wire, the
// LLB state can be reconstructed from the definition.
type DefinitionOp struct {
MarshalCache
ops map[digest.Digest]*pb.Op
defs map[digest.Digest][]byte
metas map[digest.Digest]pb.OpMetadata
platforms map[digest.Digest]*specs.Platform
dgst digest.Digest
index pb.OutputIndex
}
// NewDefinitionOp returns a new operation from a marshalled definition.
func NewDefinitionOp(def *pb.Definition) (*DefinitionOp, error) {
ops := make(map[digest.Digest]*pb.Op)
defs := make(map[digest.Digest][]byte)
var dgst digest.Digest
for _, dt := range def.Def {
var op pb.Op
if err := (&op).Unmarshal(dt); err != nil {
return nil, errors.Wrap(err, "failed to parse llb proto op")
}
dgst = digest.FromBytes(dt)
ops[dgst] = &op
defs[dgst] = dt
}
if dgst != "" {
dgst = ops[dgst].Inputs[0].Digest
}
return &DefinitionOp{
ops: ops,
defs: defs,
metas: def.Metadata,
platforms: make(map[digest.Digest]*specs.Platform),
dgst: dgst,
}, nil
}
func (d *DefinitionOp) ToInput(c *Constraints) (*pb.Input, error) {
return d.Output().ToInput(c)
}
func (d *DefinitionOp) Vertex() Vertex {
return d
}
func (d *DefinitionOp) Validate() error {
// Scratch state has no digest, ops or metas.
if d.dgst == "" {
return nil
}
if len(d.ops) == 0 || len(d.defs) == 0 || len(d.metas) == 0 {
return errors.Errorf("invalid definition op with no ops %d %d", len(d.ops), len(d.metas))
}
_, ok := d.ops[d.dgst]
if !ok {
return errors.Errorf("invalid definition op with unknown op %q", d.dgst)
}
_, ok = d.defs[d.dgst]
if !ok {
return errors.Errorf("invalid definition op with unknown def %q", d.dgst)
}
_, ok = d.metas[d.dgst]
if !ok {
return errors.Errorf("invalid definition op with unknown metas %q", d.dgst)
}
// It is possible for d.index >= len(d.ops[d.dgst]) when depending on scratch
// images.
if d.index < 0 {
return errors.Errorf("invalid definition op with invalid index")
}
return nil
}
func (d *DefinitionOp) Marshal(c *Constraints) (digest.Digest, []byte, *pb.OpMetadata, error) {
if d.dgst == "" {
return "", nil, nil, errors.Errorf("cannot marshal empty definition op")
}
if err := d.Validate(); err != nil {
return "", nil, nil, err
}
meta := d.metas[d.dgst]
return d.dgst, d.defs[d.dgst], &meta, nil
}
func (d *DefinitionOp) Output() Output {
if d.dgst == "" {
return nil
}
return &output{vertex: d, platform: d.platform(), getIndex: func() (pb.OutputIndex, error) {
return d.index, nil
}}
}
func (d *DefinitionOp) Inputs() []Output {
if d.dgst == "" {
return nil
}
var inputs []Output
op := d.ops[d.dgst]
for _, input := range op.Inputs {
vtx := &DefinitionOp{
ops: d.ops,
defs: d.defs,
metas: d.metas,
dgst: input.Digest,
index: input.Index,
}
inputs = append(inputs, &output{vertex: vtx, platform: d.platform(), getIndex: func() (pb.OutputIndex, error) {
return pb.OutputIndex(vtx.index), nil
}})
}
return inputs
}
func (d *DefinitionOp) platform() *specs.Platform {
platform, ok := d.platforms[d.dgst]
if ok {
return platform
}
op := d.ops[d.dgst]
if op.Platform != nil {
spec := op.Platform.Spec()
platform = &spec
}
d.platforms[d.dgst] = platform
return platform
}

View File

@ -0,0 +1,65 @@
package llb
import (
"bytes"
"testing"
"github.com/containerd/containerd/platforms"
"github.com/stretchr/testify/require"
)
func TestDefinitionEquivalence(t *testing.T) {
for _, tc := range []struct {
name string
state State
}{
{"scratch", Scratch()},
{"image op", Image("ref")},
{"exec op", Image("ref").Run(Shlex("args")).Root()},
{"local op", Local("name")},
{"git op", Git("remote", "ref")},
{"http op", HTTP("url")},
{"file op", Scratch().File(Mkdir("foo", 0600).Mkfile("foo/bar", 0600, []byte("data")).Copy(Scratch(), "src", "dst"))},
{"platform constraint", Image("ref", LinuxArm64)},
} {
tc := tc
t.Run(tc.name, func(t *testing.T) {
t.Parallel()
def, err := tc.state.Marshal()
require.NoError(t, err)
op, err := NewDefinitionOp(def.ToPB())
require.NoError(t, err)
err = op.Validate()
require.NoError(t, err)
st2 := NewState(op.Output())
def2, err := st2.Marshal()
require.NoError(t, err)
require.Equal(t, len(def.Def), len(def2.Def))
require.Equal(t, len(def.Metadata), len(def2.Metadata))
for i := 0; i < len(def.Def); i++ {
res := bytes.Compare(def.Def[i], def2.Def[i])
require.Equal(t, res, 0)
}
for dgst := range def.Metadata {
require.Equal(t, def.Metadata[dgst], def2.Metadata[dgst])
}
expectedPlatform := tc.state.GetPlatform()
actualPlatform := st2.GetPlatform()
if expectedPlatform == nil && actualPlatform != nil {
defaultPlatform := platforms.Normalize(platforms.DefaultSpec())
expectedPlatform = &defaultPlatform
}
require.Equal(t, expectedPlatform, actualPlatform)
})
}
}

View File

@ -10,7 +10,6 @@ import (
"github.com/containerd/containerd/remotes/docker" "github.com/containerd/containerd/remotes/docker"
"github.com/docker/docker/pkg/locker" "github.com/docker/docker/pkg/locker"
"github.com/moby/buildkit/client/llb" "github.com/moby/buildkit/client/llb"
gw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/util/contentutil" "github.com/moby/buildkit/util/contentutil"
"github.com/moby/buildkit/util/imageutil" "github.com/moby/buildkit/util/imageutil"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
@ -72,7 +71,7 @@ type resolveResult struct {
dgst digest.Digest dgst digest.Digest
} }
func (imr *imageMetaResolver) ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt) (digest.Digest, []byte, error) { func (imr *imageMetaResolver) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt) (digest.Digest, []byte, error) {
imr.locker.Lock(ref) imr.locker.Lock(ref)
defer imr.locker.Unlock(ref) defer imr.locker.Unlock(ref)

View File

@ -3,8 +3,8 @@ package llb
import ( import (
"context" "context"
gw "github.com/moby/buildkit/frontend/gateway/client"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
specs "github.com/opencontainers/image-spec/specs-go/v1"
) )
// WithMetaResolver adds a metadata resolver to an image // WithMetaResolver adds a metadata resolver to an image
@ -16,5 +16,11 @@ func WithMetaResolver(mr ImageMetaResolver) ImageOption {
// ImageMetaResolver can resolve image config metadata from a reference // ImageMetaResolver can resolve image config metadata from a reference
type ImageMetaResolver interface { type ImageMetaResolver interface {
ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt) (digest.Digest, []byte, error) ResolveImageConfig(ctx context.Context, ref string, opt ResolveImageConfigOpt) (digest.Digest, []byte, error)
}
type ResolveImageConfigOpt struct {
Platform *specs.Platform
ResolveMode string
LogName string
} }

View File

@ -9,7 +9,6 @@ import (
"strings" "strings"
"github.com/docker/distribution/reference" "github.com/docker/distribution/reference"
gw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/solver/pb" "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/apicaps" "github.com/moby/buildkit/util/apicaps"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
@ -119,7 +118,7 @@ func Image(ref string, opts ...ImageOption) State {
src.err = err src.err = err
} }
if info.metaResolver != nil { if info.metaResolver != nil {
_, dt, err := info.metaResolver.ResolveImageConfig(context.TODO(), ref, gw.ResolveImageConfigOpt{ _, dt, err := info.metaResolver.ResolveImageConfig(context.TODO(), ref, ResolveImageConfigOpt{
Platform: info.Constraints.Platform, Platform: info.Constraints.Platform,
ResolveMode: info.resolveMode.String(), ResolveMode: info.resolveMode.String(),
}) })

View File

@ -22,7 +22,6 @@ import (
"github.com/moby/buildkit/frontend/dockerfile/instructions" "github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/buildkit/frontend/dockerfile/parser" "github.com/moby/buildkit/frontend/dockerfile/parser"
"github.com/moby/buildkit/frontend/dockerfile/shell" "github.com/moby/buildkit/frontend/dockerfile/shell"
gw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/solver/pb" "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/apicaps" "github.com/moby/buildkit/util/apicaps"
"github.com/moby/buildkit/util/system" "github.com/moby/buildkit/util/system"
@ -240,7 +239,7 @@ func Dockerfile2LLB(ctx context.Context, dt []byte, opt ConvertOpt) (*llb.State,
prefix += platforms.Format(*platform) + " " prefix += platforms.Format(*platform) + " "
} }
prefix += "internal]" prefix += "internal]"
dgst, dt, err := metaResolver.ResolveImageConfig(ctx, d.stage.BaseName, gw.ResolveImageConfigOpt{ dgst, dt, err := metaResolver.ResolveImageConfig(ctx, d.stage.BaseName, llb.ResolveImageConfigOpt{
Platform: platform, Platform: platform,
ResolveMode: opt.ImageResolveMode.String(), ResolveMode: opt.ImageResolveMode.String(),
LogName: fmt.Sprintf("%s load metadata for %s", prefix, d.stage.BaseName), LogName: fmt.Sprintf("%s load metadata for %s", prefix, d.stage.BaseName),

View File

@ -31,6 +31,7 @@ import (
"github.com/moby/buildkit/client/llb" "github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/frontend/dockerfile/builder" "github.com/moby/buildkit/frontend/dockerfile/builder"
"github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb" "github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb"
gateway "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/identity" "github.com/moby/buildkit/identity"
"github.com/moby/buildkit/session" "github.com/moby/buildkit/session"
"github.com/moby/buildkit/session/upload/uploadprovider" "github.com/moby/buildkit/session/upload/uploadprovider"
@ -93,6 +94,7 @@ var allTests = []integration.Test{
testEnvEmptyFormatting, testEnvEmptyFormatting,
testCacheMultiPlatformImportExport, testCacheMultiPlatformImportExport,
testOnBuildCleared, testOnBuildCleared,
testFrontendUseForwardedSolveResults,
} }
var fileOpTests = []integration.Test{ var fileOpTests = []integration.Test{
@ -4400,6 +4402,77 @@ COPY foo bar
require.Equal(t, string(dt), "contents") require.Equal(t, string(dt), "contents")
} }
func testFrontendUseForwardedSolveResults(t *testing.T, sb integration.Sandbox) {
c, err := client.New(context.TODO(), sb.Address())
require.NoError(t, err)
defer c.Close()
dockerfile := []byte(`
FROM scratch
COPY foo foo2
`)
dir, err := tmpdir(
fstest.CreateFile("Dockerfile", dockerfile, 0600),
fstest.CreateFile("foo", []byte("data"), 0600),
)
require.NoError(t, err)
defer os.RemoveAll(dir)
frontend := func(ctx context.Context, c gateway.Client) (*gateway.Result, error) {
res, err := c.Solve(ctx, gateway.SolveRequest{
Frontend: "dockerfile.v0",
})
if err != nil {
return nil, err
}
ref, err := res.SingleRef()
if err != nil {
return nil, err
}
st2, err := ref.ToState()
if err != nil {
return nil, err
}
st := llb.Scratch().File(
llb.Copy(st2, "foo2", "foo3"),
)
def, err := st.Marshal()
if err != nil {
return nil, err
}
return c.Solve(ctx, gateway.SolveRequest{
Definition: def.ToPB(),
})
}
destDir, err := ioutil.TempDir("", "buildkit")
require.NoError(t, err)
defer os.RemoveAll(destDir)
_, err = c.Build(context.TODO(), client.SolveOpt{
Exports: []client.ExportEntry{
{
Type: client.ExporterLocal,
OutputDir: destDir,
},
},
LocalDirs: map[string]string{
builder.DefaultLocalNameDockerfile: dir,
builder.DefaultLocalNameContext: dir,
},
}, "", frontend, nil)
require.NoError(t, err)
dt, err := ioutil.ReadFile(filepath.Join(destDir, "foo3"))
require.NoError(t, err)
require.Equal(t, dt, []byte("data"))
}
func tmpdir(appliers ...fstest.Applier) (string, error) { func tmpdir(appliers ...fstest.Applier) (string, error) {
tmpdir, err := ioutil.TempDir("", "buildkit-dockerfile") tmpdir, err := ioutil.TempDir("", "buildkit-dockerfile")
if err != nil { if err != nil {

View File

@ -6,6 +6,7 @@ import (
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
"github.com/moby/buildkit/client" "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/executor" "github.com/moby/buildkit/executor"
gw "github.com/moby/buildkit/frontend/gateway/client" gw "github.com/moby/buildkit/frontend/gateway/client"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
@ -17,7 +18,7 @@ type Frontend interface {
type FrontendLLBBridge interface { type FrontendLLBBridge interface {
Solve(ctx context.Context, req SolveRequest) (*Result, error) Solve(ctx context.Context, req SolveRequest) (*Result, error)
ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt) (digest.Digest, []byte, error) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt) (digest.Digest, []byte, error)
Exec(ctx context.Context, meta executor.Meta, rootfs cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error Exec(ctx context.Context, meta executor.Meta, rootfs cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error
} }

View File

@ -3,6 +3,7 @@ package client
import ( import (
"context" "context"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/solver/pb" "github.com/moby/buildkit/solver/pb"
"github.com/moby/buildkit/util/apicaps" "github.com/moby/buildkit/util/apicaps"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
@ -12,11 +13,12 @@ import (
type Client interface { type Client interface {
Solve(ctx context.Context, req SolveRequest) (*Result, error) Solve(ctx context.Context, req SolveRequest) (*Result, error)
ResolveImageConfig(ctx context.Context, ref string, opt ResolveImageConfigOpt) (digest.Digest, []byte, error) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt) (digest.Digest, []byte, error)
BuildOpts() BuildOpts BuildOpts() BuildOpts
} }
type Reference interface { type Reference interface {
ToState() (llb.State, error)
ReadFile(ctx context.Context, req ReadRequest) ([]byte, error) ReadFile(ctx context.Context, req ReadRequest) ([]byte, error)
StatFile(ctx context.Context, req StatRequest) (*fstypes.Stat, error) StatFile(ctx context.Context, req StatRequest) (*fstypes.Stat, error)
ReadDir(ctx context.Context, req ReadDirRequest) ([]*fstypes.Stat, error) ReadDir(ctx context.Context, req ReadDirRequest) ([]*fstypes.Stat, error)
@ -68,9 +70,3 @@ type BuildOpts struct {
LLBCaps apicaps.CapSet LLBCaps apicaps.CapSet
Caps apicaps.CapSet Caps apicaps.CapSet
} }
type ResolveImageConfigOpt struct {
Platform *specs.Platform
ResolveMode string
LogName string
}

View File

@ -7,6 +7,7 @@ import (
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
cacheutil "github.com/moby/buildkit/cache/util" cacheutil "github.com/moby/buildkit/cache/util"
clienttypes "github.com/moby/buildkit/client" clienttypes "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/frontend" "github.com/moby/buildkit/frontend"
"github.com/moby/buildkit/frontend/gateway/client" "github.com/moby/buildkit/frontend/gateway/client"
gwpb "github.com/moby/buildkit/frontend/gateway/pb" gwpb "github.com/moby/buildkit/frontend/gateway/pb"
@ -54,12 +55,18 @@ func (c *bridgeClient) Solve(ctx context.Context, req client.SolveRequest) (*cli
cRes := &client.Result{} cRes := &client.Result{}
c.mu.Lock() c.mu.Lock()
for k, r := range res.Refs { for k, r := range res.Refs {
rr := &ref{r} rr, err := newRef(r)
if err != nil {
return nil, err
}
c.refs = append(c.refs, rr) c.refs = append(c.refs, rr)
cRes.AddRef(k, rr) cRes.AddRef(k, rr)
} }
if r := res.Ref; r != nil { if r := res.Ref; r != nil {
rr := &ref{r} rr, err := newRef(r)
if err != nil {
return nil, err
}
c.refs = append(c.refs, rr) c.refs = append(c.refs, rr)
cRes.SetRef(rr) cRes.SetRef(rr)
} }
@ -131,6 +138,24 @@ func (c *bridgeClient) discard(err error) {
type ref struct { type ref struct {
solver.CachedResult solver.CachedResult
def *opspb.Definition
}
func newRef(r solver.CachedResult) (*ref, error) {
wref, ok := r.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", r.Sys())
}
return &ref{CachedResult: r, def: wref.Definition}, nil
}
func (r *ref) ToState() (st llb.State, err error) {
defop, err := llb.NewDefinitionOp(r.def)
if err != nil {
return st, err
}
return llb.NewState(defop), nil
} }
func (r *ref) ReadFile(ctx context.Context, req client.ReadRequest) ([]byte, error) { func (r *ref) ReadFile(ctx context.Context, req client.ReadRequest) ([]byte, error) {

View File

@ -21,7 +21,6 @@ import (
"github.com/moby/buildkit/executor" "github.com/moby/buildkit/executor"
"github.com/moby/buildkit/exporter/containerimage/exptypes" "github.com/moby/buildkit/exporter/containerimage/exptypes"
"github.com/moby/buildkit/frontend" "github.com/moby/buildkit/frontend"
gw "github.com/moby/buildkit/frontend/gateway/client"
pb "github.com/moby/buildkit/frontend/gateway/pb" pb "github.com/moby/buildkit/frontend/gateway/pb"
"github.com/moby/buildkit/identity" "github.com/moby/buildkit/identity"
"github.com/moby/buildkit/session" "github.com/moby/buildkit/session"
@ -113,7 +112,7 @@ func (gf *gatewayFrontend) Solve(ctx context.Context, llbBridge frontend.Fronten
return nil, err return nil, err
} }
dgst, config, err := llbBridge.ResolveImageConfig(ctx, reference.TagNameOnly(sourceRef).String(), gw.ResolveImageConfigOpt{}) dgst, config, err := llbBridge.ResolveImageConfig(ctx, reference.TagNameOnly(sourceRef).String(), llb.ResolveImageConfigOpt{})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -405,7 +404,7 @@ func (lbf *llbBridgeForwarder) ResolveImageConfig(ctx context.Context, req *pb.R
OSFeatures: p.OSFeatures, OSFeatures: p.OSFeatures,
} }
} }
dgst, dt, err := lbf.llbBridge.ResolveImageConfig(ctx, req.Ref, gw.ResolveImageConfigOpt{ dgst, dt, err := lbf.llbBridge.ResolveImageConfig(ctx, req.Ref, llb.ResolveImageConfigOpt{
Platform: platform, Platform: platform,
ResolveMode: req.ResolveMode, ResolveMode: req.ResolveMode,
LogName: req.LogName, LogName: req.LogName,
@ -466,6 +465,7 @@ func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest)
lbf.mu.Lock() lbf.mu.Lock()
if res.Refs != nil { if res.Refs != nil {
ids := make(map[string]string, len(res.Refs)) ids := make(map[string]string, len(res.Refs))
defs := make(map[string]*opspb.Definition, len(res.Refs))
for k, ref := range res.Refs { for k, ref := range res.Refs {
id := identity.NewID() id := identity.NewID()
if ref == nil { if ref == nil {
@ -474,28 +474,43 @@ func (lbf *llbBridgeForwarder) Solve(ctx context.Context, req *pb.SolveRequest)
lbf.refs[id] = ref lbf.refs[id] = ref
} }
ids[k] = id ids[k] = id
wref, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
defs[k] = wref.Definition
} }
if req.AllowResultArrayRef { if req.AllowResultArrayRef {
refMap := make(map[string]*pb.Ref, len(res.Refs)) refMap := make(map[string]*pb.Ref, len(res.Refs))
for k, id := range ids { for k, id := range ids {
refMap[k] = pb.NewRef(id) refMap[k] = pb.NewRef(id, defs[k])
} }
pbRes.Result = &pb.Result_Refs{Refs: &pb.RefMap{Refs: refMap}} pbRes.Result = &pb.Result_Refs{Refs: &pb.RefMap{Refs: refMap}}
} else { } else {
pbRes.Result = &pb.Result_RefsDeprecated{RefsDeprecated: &pb.RefMapDeprecated{Refs: ids}} pbRes.Result = &pb.Result_RefsDeprecated{RefsDeprecated: &pb.RefMapDeprecated{Refs: ids}}
} }
} else { } else {
ref := res.Ref
id := identity.NewID() id := identity.NewID()
if res.Ref == nil {
var def *opspb.Definition
if ref == nil {
id = "" id = ""
} else { } else {
lbf.refs[id] = res.Ref wref, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
def = wref.Definition
lbf.refs[id] = ref
} }
defaultID = id defaultID = id
if req.AllowResultArrayRef { if req.AllowResultArrayRef {
pbRes.Result = &pb.Result_Ref{Ref: pb.NewRef(id)} pbRes.Result = &pb.Result_Ref{Ref: pb.NewRef(id, def)}
} else { } else {
pbRes.Result = &pb.Result_RefDeprecated{RefDeprecated: id} pbRes.Result = &pb.Result_RefDeprecated{RefDeprecated: id}
} }

View File

@ -10,6 +10,7 @@ import (
"time" "time"
"github.com/gogo/googleapis/google/rpc" "github.com/gogo/googleapis/google/rpc"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/frontend/gateway/client" "github.com/moby/buildkit/frontend/gateway/client"
pb "github.com/moby/buildkit/frontend/gateway/pb" pb "github.com/moby/buildkit/frontend/gateway/pb"
opspb "github.com/moby/buildkit/solver/pb" opspb "github.com/moby/buildkit/solver/pb"
@ -68,15 +69,15 @@ func current() (GrpcClient, error) {
return New(ctx, opts(), sessionID(), product(), pb.NewLLBBridgeClient(conn), workers()) return New(ctx, opts(), sessionID(), product(), pb.NewLLBBridgeClient(conn), workers())
} }
func convertRef(ref client.Reference) (string, error) { func convertRef(ref client.Reference) (*pb.Ref, error) {
if ref == nil { if ref == nil {
return "", nil return &pb.Ref{}, nil
} }
r, ok := ref.(*reference) r, ok := ref.(*reference)
if !ok { if !ok {
return "", errors.Errorf("invalid return reference type %T", ref) return nil, errors.Errorf("invalid return reference type %T", ref)
} }
return r.id, nil return &pb.Ref{Ids: []string{r.id}, Defs: []*opspb.Definition{r.def}}, nil
} }
func RunFromEnvironment(ctx context.Context, f client.BuildFunc) error { func RunFromEnvironment(ctx context.Context, f client.BuildFunc) error {
@ -108,34 +109,47 @@ func (c *grpcClient) Run(ctx context.Context, f client.BuildFunc) (retError erro
if c.caps.Supports(pb.CapProtoRefArray) == nil { if c.caps.Supports(pb.CapProtoRefArray) == nil {
m := map[string]*pb.Ref{} m := map[string]*pb.Ref{}
for k, r := range res.Refs { for k, r := range res.Refs {
id, err := convertRef(r) pbRef, err := convertRef(r)
if err != nil { if err != nil {
retError = err retError = err
continue continue
} }
m[k] = pb.NewRef(id) m[k] = pbRef
} }
pbRes.Result = &pb.Result_Refs{Refs: &pb.RefMap{Refs: m}} pbRes.Result = &pb.Result_Refs{Refs: &pb.RefMap{Refs: m}}
} else { } else {
// Server doesn't support the new wire format for refs, so we construct
// a deprecated result ref map.
m := map[string]string{} m := map[string]string{}
for k, r := range res.Refs { for k, r := range res.Refs {
id, err := convertRef(r) pbRef, err := convertRef(r)
if err != nil { if err != nil {
retError = err retError = err
continue continue
} }
var id string
if len(pbRef.Ids) > 0 {
id = pbRef.Ids[0]
}
m[k] = id m[k] = id
} }
pbRes.Result = &pb.Result_RefsDeprecated{RefsDeprecated: &pb.RefMapDeprecated{Refs: m}} pbRes.Result = &pb.Result_RefsDeprecated{RefsDeprecated: &pb.RefMapDeprecated{Refs: m}}
} }
} else { } else {
id, err := convertRef(res.Ref) pbRef, err := convertRef(res.Ref)
if err != nil { if err != nil {
retError = err retError = err
} else { } else {
if c.caps.Supports(pb.CapProtoRefArray) == nil { if c.caps.Supports(pb.CapProtoRefArray) == nil {
pbRes.Result = &pb.Result_Ref{Ref: pb.NewRef(id)} pbRes.Result = &pb.Result_Ref{Ref: pbRef}
} else { } else {
// Server doesn't support the new wire format for refs, so we construct
// a deprecated result ref.
var id string
if len(pbRef.Ids) > 0 {
id = pbRef.Ids[0]
}
pbRes.Result = &pb.Result_RefDeprecated{RefDeprecated: id} pbRes.Result = &pb.Result_RefDeprecated{RefDeprecated: id}
} }
} }
@ -343,19 +357,21 @@ func (c *grpcClient) Solve(ctx context.Context, creq client.SolveRequest) (*clie
case *pb.Result_Ref: case *pb.Result_Ref:
ids := pbRes.Ref.Ids ids := pbRes.Ref.Ids
if len(ids) > 0 { if len(ids) > 0 {
if len(ids) > 1 { ref, err := newReference(c, pbRes.Ref)
return nil, errors.Errorf("solve returned multi-result array") if err != nil {
return nil, err
} }
res.SetRef(&reference{id: ids[0], c: c})
res.SetRef(ref)
} }
case *pb.Result_Refs: case *pb.Result_Refs:
for k, v := range pbRes.Refs.Refs { for k, v := range pbRes.Refs.Refs {
var ref *reference var ref *reference
if len(v.Ids) > 0 { if len(v.Ids) > 0 {
if len(v.Ids) > 1 { ref, err = newReference(c, v)
return nil, errors.Errorf("solve returned multi-result array") if err != nil {
return nil, err
} }
ref = &reference{id: v.Ids[0], c: c}
} }
res.AddRef(k, ref) res.AddRef(k, ref)
} }
@ -365,7 +381,7 @@ func (c *grpcClient) Solve(ctx context.Context, creq client.SolveRequest) (*clie
return res, nil return res, nil
} }
func (c *grpcClient) ResolveImageConfig(ctx context.Context, ref string, opt client.ResolveImageConfigOpt) (digest.Digest, []byte, error) { func (c *grpcClient) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt) (digest.Digest, []byte, error) {
var p *opspb.Platform var p *opspb.Platform
if platform := opt.Platform; platform != nil { if platform := opt.Platform; platform != nil {
p = &opspb.Platform{ p = &opspb.Platform{
@ -395,8 +411,44 @@ func (c *grpcClient) BuildOpts() client.BuildOpts {
} }
type reference struct { type reference struct {
id string c *grpcClient
c *grpcClient id string
def *opspb.Definition
output llb.Output
}
func newReference(c *grpcClient, ref *pb.Ref) (*reference, error) {
if len(ref.Ids) == 0 {
return nil, errors.Errorf("reference has no ids")
}
if len(ref.Ids) > 1 {
return nil, errors.Errorf("cannot create multi-result array reference")
}
if len(ref.Ids) != len(ref.Defs) {
return nil, errors.Errorf("reference ids and definitions mismatch length")
}
return &reference{c: c, id: ref.Ids[0], def: ref.Defs[0]}, nil
}
func (r *reference) ToState() (st llb.State, err error) {
err = r.c.caps.Supports(pb.CapReferenceOutput)
if err != nil {
return st, err
}
if r.def == nil {
return st, errors.Errorf("gateway did not return reference with definition")
}
defop, err := llb.NewDefinitionOp(r.def)
if err != nil {
return st, err
}
return llb.NewState(defop), nil
} }
func (r *reference) ReadFile(ctx context.Context, req client.ReadRequest) ([]byte, error) { func (r *reference) ReadFile(ctx context.Context, req client.ReadRequest) ([]byte, error) {

View File

@ -24,6 +24,10 @@ const (
// refs. This capability is only for the wire format change and shouldn't be // refs. This capability is only for the wire format change and shouldn't be
// used in frontends for feature detection. // used in frontends for feature detection.
CapProtoRefArray apicaps.CapID = "proto.refarray" CapProtoRefArray apicaps.CapID = "proto.refarray"
// CapReferenceOutput is a capability to use a reference of a solved result as
// an llb.Output.
CapReferenceOutput apicaps.CapID = "reference.output"
) )
func init() { func init() {
@ -104,4 +108,11 @@ func init() {
Enabled: true, Enabled: true,
Status: apicaps.CapStatusExperimental, Status: apicaps.CapStatusExperimental,
}) })
Caps.Init(apicaps.Cap{
ID: CapReferenceOutput,
Name: "reference output",
Enabled: true,
Status: apicaps.CapStatusExperimental,
})
} }

View File

@ -50,7 +50,7 @@ func (m *Result) Reset() { *m = Result{} }
func (m *Result) String() string { return proto.CompactTextString(m) } func (m *Result) String() string { return proto.CompactTextString(m) }
func (*Result) ProtoMessage() {} func (*Result) ProtoMessage() {}
func (*Result) Descriptor() ([]byte, []int) { func (*Result) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{0} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{0}
} }
func (m *Result) XXX_Unmarshal(b []byte) error { func (m *Result) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -264,7 +264,7 @@ func (m *RefMapDeprecated) Reset() { *m = RefMapDeprecated{} }
func (m *RefMapDeprecated) String() string { return proto.CompactTextString(m) } func (m *RefMapDeprecated) String() string { return proto.CompactTextString(m) }
func (*RefMapDeprecated) ProtoMessage() {} func (*RefMapDeprecated) ProtoMessage() {}
func (*RefMapDeprecated) Descriptor() ([]byte, []int) { func (*RefMapDeprecated) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{1} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{1}
} }
func (m *RefMapDeprecated) XXX_Unmarshal(b []byte) error { func (m *RefMapDeprecated) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -301,17 +301,18 @@ func (m *RefMapDeprecated) GetRefs() map[string]string {
} }
type Ref struct { type Ref struct {
Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` Defs []*pb.Definition `protobuf:"bytes,2,rep,name=defs,proto3" json:"defs,omitempty"`
XXX_unrecognized []byte `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
} }
func (m *Ref) Reset() { *m = Ref{} } func (m *Ref) Reset() { *m = Ref{} }
func (m *Ref) String() string { return proto.CompactTextString(m) } func (m *Ref) String() string { return proto.CompactTextString(m) }
func (*Ref) ProtoMessage() {} func (*Ref) ProtoMessage() {}
func (*Ref) Descriptor() ([]byte, []int) { func (*Ref) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{2} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{2}
} }
func (m *Ref) XXX_Unmarshal(b []byte) error { func (m *Ref) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -347,6 +348,13 @@ func (m *Ref) GetIds() []string {
return nil return nil
} }
func (m *Ref) GetDefs() []*pb.Definition {
if m != nil {
return m.Defs
}
return nil
}
type RefMap struct { type RefMap struct {
Refs map[string]*Ref `protobuf:"bytes,1,rep,name=refs,proto3" json:"refs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` Refs map[string]*Ref `protobuf:"bytes,1,rep,name=refs,proto3" json:"refs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -358,7 +366,7 @@ func (m *RefMap) Reset() { *m = RefMap{} }
func (m *RefMap) String() string { return proto.CompactTextString(m) } func (m *RefMap) String() string { return proto.CompactTextString(m) }
func (*RefMap) ProtoMessage() {} func (*RefMap) ProtoMessage() {}
func (*RefMap) Descriptor() ([]byte, []int) { func (*RefMap) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{3} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{3}
} }
func (m *RefMap) XXX_Unmarshal(b []byte) error { func (m *RefMap) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -406,7 +414,7 @@ func (m *ReturnRequest) Reset() { *m = ReturnRequest{} }
func (m *ReturnRequest) String() string { return proto.CompactTextString(m) } func (m *ReturnRequest) String() string { return proto.CompactTextString(m) }
func (*ReturnRequest) ProtoMessage() {} func (*ReturnRequest) ProtoMessage() {}
func (*ReturnRequest) Descriptor() ([]byte, []int) { func (*ReturnRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{4} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{4}
} }
func (m *ReturnRequest) XXX_Unmarshal(b []byte) error { func (m *ReturnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -459,7 +467,7 @@ func (m *ReturnResponse) Reset() { *m = ReturnResponse{} }
func (m *ReturnResponse) String() string { return proto.CompactTextString(m) } func (m *ReturnResponse) String() string { return proto.CompactTextString(m) }
func (*ReturnResponse) ProtoMessage() {} func (*ReturnResponse) ProtoMessage() {}
func (*ReturnResponse) Descriptor() ([]byte, []int) { func (*ReturnResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{5} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{5}
} }
func (m *ReturnResponse) XXX_Unmarshal(b []byte) error { func (m *ReturnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -502,7 +510,7 @@ func (m *ResolveImageConfigRequest) Reset() { *m = ResolveImageConfigReq
func (m *ResolveImageConfigRequest) String() string { return proto.CompactTextString(m) } func (m *ResolveImageConfigRequest) String() string { return proto.CompactTextString(m) }
func (*ResolveImageConfigRequest) ProtoMessage() {} func (*ResolveImageConfigRequest) ProtoMessage() {}
func (*ResolveImageConfigRequest) Descriptor() ([]byte, []int) { func (*ResolveImageConfigRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{6} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{6}
} }
func (m *ResolveImageConfigRequest) XXX_Unmarshal(b []byte) error { func (m *ResolveImageConfigRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -571,7 +579,7 @@ func (m *ResolveImageConfigResponse) Reset() { *m = ResolveImageConfigRe
func (m *ResolveImageConfigResponse) String() string { return proto.CompactTextString(m) } func (m *ResolveImageConfigResponse) String() string { return proto.CompactTextString(m) }
func (*ResolveImageConfigResponse) ProtoMessage() {} func (*ResolveImageConfigResponse) ProtoMessage() {}
func (*ResolveImageConfigResponse) Descriptor() ([]byte, []int) { func (*ResolveImageConfigResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{7} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{7}
} }
func (m *ResolveImageConfigResponse) XXX_Unmarshal(b []byte) error { func (m *ResolveImageConfigResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -633,7 +641,7 @@ func (m *SolveRequest) Reset() { *m = SolveRequest{} }
func (m *SolveRequest) String() string { return proto.CompactTextString(m) } func (m *SolveRequest) String() string { return proto.CompactTextString(m) }
func (*SolveRequest) ProtoMessage() {} func (*SolveRequest) ProtoMessage() {}
func (*SolveRequest) Descriptor() ([]byte, []int) { func (*SolveRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{8} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{8}
} }
func (m *SolveRequest) XXX_Unmarshal(b []byte) error { func (m *SolveRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -738,7 +746,7 @@ func (m *CacheOptionsEntry) Reset() { *m = CacheOptionsEntry{} }
func (m *CacheOptionsEntry) String() string { return proto.CompactTextString(m) } func (m *CacheOptionsEntry) String() string { return proto.CompactTextString(m) }
func (*CacheOptionsEntry) ProtoMessage() {} func (*CacheOptionsEntry) ProtoMessage() {}
func (*CacheOptionsEntry) Descriptor() ([]byte, []int) { func (*CacheOptionsEntry) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{9} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{9}
} }
func (m *CacheOptionsEntry) XXX_Unmarshal(b []byte) error { func (m *CacheOptionsEntry) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -795,7 +803,7 @@ func (m *SolveResponse) Reset() { *m = SolveResponse{} }
func (m *SolveResponse) String() string { return proto.CompactTextString(m) } func (m *SolveResponse) String() string { return proto.CompactTextString(m) }
func (*SolveResponse) ProtoMessage() {} func (*SolveResponse) ProtoMessage() {}
func (*SolveResponse) Descriptor() ([]byte, []int) { func (*SolveResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{10} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{10}
} }
func (m *SolveResponse) XXX_Unmarshal(b []byte) error { func (m *SolveResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -851,7 +859,7 @@ func (m *ReadFileRequest) Reset() { *m = ReadFileRequest{} }
func (m *ReadFileRequest) String() string { return proto.CompactTextString(m) } func (m *ReadFileRequest) String() string { return proto.CompactTextString(m) }
func (*ReadFileRequest) ProtoMessage() {} func (*ReadFileRequest) ProtoMessage() {}
func (*ReadFileRequest) Descriptor() ([]byte, []int) { func (*ReadFileRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{11} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{11}
} }
func (m *ReadFileRequest) XXX_Unmarshal(b []byte) error { func (m *ReadFileRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -913,7 +921,7 @@ func (m *FileRange) Reset() { *m = FileRange{} }
func (m *FileRange) String() string { return proto.CompactTextString(m) } func (m *FileRange) String() string { return proto.CompactTextString(m) }
func (*FileRange) ProtoMessage() {} func (*FileRange) ProtoMessage() {}
func (*FileRange) Descriptor() ([]byte, []int) { func (*FileRange) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{12} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{12}
} }
func (m *FileRange) XXX_Unmarshal(b []byte) error { func (m *FileRange) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -967,7 +975,7 @@ func (m *ReadFileResponse) Reset() { *m = ReadFileResponse{} }
func (m *ReadFileResponse) String() string { return proto.CompactTextString(m) } func (m *ReadFileResponse) String() string { return proto.CompactTextString(m) }
func (*ReadFileResponse) ProtoMessage() {} func (*ReadFileResponse) ProtoMessage() {}
func (*ReadFileResponse) Descriptor() ([]byte, []int) { func (*ReadFileResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{13} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{13}
} }
func (m *ReadFileResponse) XXX_Unmarshal(b []byte) error { func (m *ReadFileResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1016,7 +1024,7 @@ func (m *ReadDirRequest) Reset() { *m = ReadDirRequest{} }
func (m *ReadDirRequest) String() string { return proto.CompactTextString(m) } func (m *ReadDirRequest) String() string { return proto.CompactTextString(m) }
func (*ReadDirRequest) ProtoMessage() {} func (*ReadDirRequest) ProtoMessage() {}
func (*ReadDirRequest) Descriptor() ([]byte, []int) { func (*ReadDirRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{14} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{14}
} }
func (m *ReadDirRequest) XXX_Unmarshal(b []byte) error { func (m *ReadDirRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1077,7 +1085,7 @@ func (m *ReadDirResponse) Reset() { *m = ReadDirResponse{} }
func (m *ReadDirResponse) String() string { return proto.CompactTextString(m) } func (m *ReadDirResponse) String() string { return proto.CompactTextString(m) }
func (*ReadDirResponse) ProtoMessage() {} func (*ReadDirResponse) ProtoMessage() {}
func (*ReadDirResponse) Descriptor() ([]byte, []int) { func (*ReadDirResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{15} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{15}
} }
func (m *ReadDirResponse) XXX_Unmarshal(b []byte) error { func (m *ReadDirResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1125,7 +1133,7 @@ func (m *StatFileRequest) Reset() { *m = StatFileRequest{} }
func (m *StatFileRequest) String() string { return proto.CompactTextString(m) } func (m *StatFileRequest) String() string { return proto.CompactTextString(m) }
func (*StatFileRequest) ProtoMessage() {} func (*StatFileRequest) ProtoMessage() {}
func (*StatFileRequest) Descriptor() ([]byte, []int) { func (*StatFileRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{16} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{16}
} }
func (m *StatFileRequest) XXX_Unmarshal(b []byte) error { func (m *StatFileRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1179,7 +1187,7 @@ func (m *StatFileResponse) Reset() { *m = StatFileResponse{} }
func (m *StatFileResponse) String() string { return proto.CompactTextString(m) } func (m *StatFileResponse) String() string { return proto.CompactTextString(m) }
func (*StatFileResponse) ProtoMessage() {} func (*StatFileResponse) ProtoMessage() {}
func (*StatFileResponse) Descriptor() ([]byte, []int) { func (*StatFileResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{17} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{17}
} }
func (m *StatFileResponse) XXX_Unmarshal(b []byte) error { func (m *StatFileResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1225,7 +1233,7 @@ func (m *PingRequest) Reset() { *m = PingRequest{} }
func (m *PingRequest) String() string { return proto.CompactTextString(m) } func (m *PingRequest) String() string { return proto.CompactTextString(m) }
func (*PingRequest) ProtoMessage() {} func (*PingRequest) ProtoMessage() {}
func (*PingRequest) Descriptor() ([]byte, []int) { func (*PingRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{18} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{18}
} }
func (m *PingRequest) XXX_Unmarshal(b []byte) error { func (m *PingRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1267,7 +1275,7 @@ func (m *PongResponse) Reset() { *m = PongResponse{} }
func (m *PongResponse) String() string { return proto.CompactTextString(m) } func (m *PongResponse) String() string { return proto.CompactTextString(m) }
func (*PongResponse) ProtoMessage() {} func (*PongResponse) ProtoMessage() {}
func (*PongResponse) Descriptor() ([]byte, []int) { func (*PongResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_gateway_f66b6a1c7ba3b979, []int{19} return fileDescriptor_gateway_b7e6dbfe1a84b4d2, []int{19}
} }
func (m *PongResponse) XXX_Unmarshal(b []byte) error { func (m *PongResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -1794,6 +1802,18 @@ func (m *Ref) MarshalTo(dAtA []byte) (int, error) {
i += copy(dAtA[i:], s) i += copy(dAtA[i:], s)
} }
} }
if len(m.Defs) > 0 {
for _, msg := range m.Defs {
dAtA[i] = 0x12
i++
i = encodeVarintGateway(dAtA, i, uint64(msg.Size()))
n, err := msg.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n
}
}
if m.XXX_unrecognized != nil { if m.XXX_unrecognized != nil {
i += copy(dAtA[i:], m.XXX_unrecognized) i += copy(dAtA[i:], m.XXX_unrecognized)
} }
@ -2620,6 +2640,12 @@ func (m *Ref) Size() (n int) {
n += 1 + l + sovGateway(uint64(l)) n += 1 + l + sovGateway(uint64(l))
} }
} }
if len(m.Defs) > 0 {
for _, e := range m.Defs {
l = e.Size()
n += 1 + l + sovGateway(uint64(l))
}
}
if m.XXX_unrecognized != nil { if m.XXX_unrecognized != nil {
n += len(m.XXX_unrecognized) n += len(m.XXX_unrecognized)
} }
@ -3541,6 +3567,37 @@ func (m *Ref) Unmarshal(dAtA []byte) error {
} }
m.Ids = append(m.Ids, string(dAtA[iNdEx:postIndex])) m.Ids = append(m.Ids, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Defs", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowGateway
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthGateway
}
postIndex := iNdEx + msglen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Defs = append(m.Defs, &pb.Definition{})
if err := m.Defs[len(m.Defs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default: default:
iNdEx = preIndex iNdEx = preIndex
skippy, err := skipGateway(dAtA[iNdEx:]) skippy, err := skipGateway(dAtA[iNdEx:])
@ -5906,91 +5963,92 @@ var (
ErrIntOverflowGateway = fmt.Errorf("proto: integer overflow") ErrIntOverflowGateway = fmt.Errorf("proto: integer overflow")
) )
func init() { proto.RegisterFile("gateway.proto", fileDescriptor_gateway_f66b6a1c7ba3b979) } func init() { proto.RegisterFile("gateway.proto", fileDescriptor_gateway_b7e6dbfe1a84b4d2) }
var fileDescriptor_gateway_f66b6a1c7ba3b979 = []byte{ var fileDescriptor_gateway_b7e6dbfe1a84b4d2 = []byte{
// 1321 bytes of a gzipped FileDescriptorProto // 1334 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x57, 0x5f, 0x6f, 0x1b, 0x45, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x57, 0xcd, 0x6e, 0xdb, 0xc6,
0x10, 0xcf, 0xc5, 0xce, 0x1f, 0x4f, 0x9c, 0xd4, 0x5d, 0x10, 0xb8, 0xf7, 0x90, 0x86, 0x13, 0x0a, 0x16, 0x36, 0x2d, 0xc9, 0xb6, 0x8e, 0x65, 0x47, 0x99, 0x7b, 0x71, 0xa1, 0x70, 0xe1, 0xf8, 0x12,
0x6e, 0x9b, 0xde, 0x41, 0xda, 0x2a, 0xa5, 0x45, 0x2d, 0x75, 0xd3, 0x2a, 0x81, 0x84, 0x5a, 0x5b, 0x17, 0xbe, 0x4a, 0xe2, 0x90, 0xad, 0x93, 0xc0, 0x69, 0x52, 0x24, 0x8d, 0xe2, 0x04, 0x76, 0x6b,
0x50, 0xa5, 0x0a, 0x24, 0xce, 0xbe, 0xbd, 0xeb, 0xaa, 0xe7, 0xdb, 0x63, 0x6f, 0xdd, 0x62, 0xf1, 0x37, 0xc2, 0xa4, 0x45, 0x80, 0xa0, 0x05, 0x4a, 0x89, 0x43, 0x66, 0x10, 0x8a, 0xc3, 0x0e, 0x47,
0x02, 0x6f, 0xbc, 0x23, 0xf1, 0x8c, 0xf8, 0x04, 0x7c, 0x8c, 0xbe, 0xc1, 0x33, 0x0f, 0x15, 0xca, 0x49, 0x85, 0x6e, 0xda, 0x5d, 0xf7, 0x05, 0xba, 0x2e, 0xfa, 0x04, 0x7d, 0x8c, 0xec, 0xda, 0x75,
0x27, 0x41, 0xfb, 0xe7, 0xe2, 0xb3, 0x93, 0x9c, 0xe3, 0x27, 0xef, 0xce, 0xcd, 0x6f, 0xe6, 0x37, 0x17, 0x41, 0xe1, 0x27, 0x29, 0xe6, 0x87, 0x16, 0x25, 0xdb, 0x94, 0xb5, 0xd2, 0xcc, 0xf0, 0x7c,
0xb3, 0x33, 0xb3, 0x6b, 0x58, 0x8d, 0x7c, 0x41, 0x5e, 0xfb, 0x43, 0x37, 0xe5, 0x4c, 0x30, 0x74, 0xe7, 0x7c, 0xe7, 0x77, 0x46, 0xb0, 0x16, 0xf9, 0x82, 0xbc, 0xf5, 0x47, 0x6e, 0xca, 0x99, 0x60,
0xa9, 0xcf, 0xba, 0x43, 0xb7, 0x3b, 0xa0, 0x71, 0xf0, 0x92, 0x0a, 0xf7, 0xd5, 0x27, 0x6e, 0xc8, 0xe8, 0xca, 0x80, 0xf5, 0x46, 0x6e, 0x6f, 0x48, 0xe3, 0xe0, 0x35, 0x15, 0xee, 0x9b, 0x0f, 0xdd,
0x59, 0x22, 0x48, 0x12, 0xd8, 0xd7, 0x23, 0x2a, 0x5e, 0x0c, 0xba, 0x6e, 0x8f, 0xf5, 0xbd, 0x88, 0x90, 0xb3, 0x44, 0x90, 0x24, 0xb0, 0x6f, 0x46, 0x54, 0xbc, 0x1a, 0xf6, 0xdc, 0x3e, 0x1b, 0x78,
0x45, 0xcc, 0x53, 0x88, 0xee, 0x20, 0x54, 0x3b, 0xb5, 0x51, 0x2b, 0x6d, 0xc9, 0xde, 0x9e, 0x54, 0x11, 0x8b, 0x98, 0xa7, 0x10, 0xbd, 0x61, 0xa8, 0x76, 0x6a, 0xa3, 0x56, 0x5a, 0x93, 0xbd, 0x33,
0x8f, 0x18, 0x8b, 0x62, 0xe2, 0xa7, 0x34, 0x33, 0x4b, 0x8f, 0xa7, 0x3d, 0x2f, 0x13, 0xbe, 0x18, 0x2d, 0x1e, 0x31, 0x16, 0xc5, 0xc4, 0x4f, 0x69, 0x66, 0x96, 0x1e, 0x4f, 0xfb, 0x5e, 0x26, 0x7c,
0x64, 0x06, 0xb3, 0x55, 0xc0, 0x48, 0x22, 0x5e, 0x4e, 0xc4, 0xcb, 0x58, 0xfc, 0x8a, 0x70, 0x2f, 0x31, 0xcc, 0x0c, 0x66, 0xbb, 0x80, 0x91, 0x44, 0xbc, 0x9c, 0x88, 0x97, 0xb1, 0xf8, 0x0d, 0xe1,
0xed, 0x7a, 0x2c, 0xcd, 0xb5, 0xbd, 0x33, 0xb5, 0xfd, 0x94, 0x7a, 0x62, 0x98, 0x92, 0xcc, 0x7b, 0x5e, 0xda, 0xf3, 0x58, 0x9a, 0x4b, 0x7b, 0xe7, 0x4a, 0xfb, 0x29, 0xf5, 0xc4, 0x28, 0x25, 0x99,
0xcd, 0xf8, 0x4b, 0xc2, 0x0d, 0xe0, 0xc6, 0x99, 0x80, 0x81, 0xa0, 0xb1, 0x44, 0xf5, 0xfc, 0x34, 0xf7, 0x96, 0xf1, 0xd7, 0x84, 0x1b, 0xc0, 0xad, 0x73, 0x01, 0x43, 0x41, 0x63, 0x89, 0xea, 0xfb,
0x93, 0x4e, 0xe4, 0xaf, 0x01, 0x15, 0xc3, 0x16, 0x2c, 0xa1, 0x99, 0xa0, 0x34, 0xa2, 0x5e, 0x98, 0x69, 0x26, 0x8d, 0xc8, 0x5f, 0x03, 0x2a, 0xba, 0x2d, 0x58, 0x42, 0x33, 0x41, 0x69, 0x44, 0xbd,
0x29, 0x8c, 0xf6, 0x22, 0x83, 0xd0, 0xea, 0xce, 0xaf, 0x15, 0x58, 0xc4, 0x24, 0x1b, 0xc4, 0x02, 0x30, 0x53, 0x18, 0x6d, 0x45, 0x3a, 0xa1, 0xc5, 0x9d, 0x9f, 0x2a, 0xb0, 0x84, 0x49, 0x36, 0x8c,
0x6d, 0xc2, 0x2a, 0x27, 0xe1, 0x2e, 0x49, 0x39, 0xe9, 0xf9, 0x82, 0x04, 0x4d, 0x6b, 0xc3, 0x6a, 0x05, 0xda, 0x82, 0x35, 0x4e, 0xc2, 0x3d, 0x92, 0x72, 0xd2, 0xf7, 0x05, 0x09, 0x5a, 0xd6, 0xa6,
0xd5, 0xf6, 0xe6, 0xf0, 0xb8, 0x18, 0x7d, 0x03, 0x6b, 0x9c, 0x84, 0x59, 0x41, 0x71, 0x7e, 0xc3, 0xd5, 0xae, 0xef, 0x2f, 0xe0, 0xc9, 0x63, 0xf4, 0x25, 0xac, 0x73, 0x12, 0x66, 0x05, 0xc1, 0xc5,
0x6a, 0xad, 0x6c, 0x5f, 0x73, 0xcf, 0x3c, 0x0c, 0x17, 0x93, 0xf0, 0xd0, 0x4f, 0x47, 0x90, 0xbd, 0x4d, 0xab, 0xbd, 0xba, 0x73, 0xc3, 0x3d, 0x37, 0x19, 0x2e, 0x26, 0xe1, 0x91, 0x9f, 0x8e, 0x21,
0x39, 0x3c, 0x61, 0x04, 0x6d, 0x43, 0x85, 0x93, 0xb0, 0x59, 0x51, 0xb6, 0xd6, 0xcb, 0x6d, 0xed, 0xfb, 0x0b, 0x78, 0x4a, 0x09, 0xda, 0x81, 0x0a, 0x27, 0x61, 0xab, 0xa2, 0x74, 0x6d, 0x94, 0xeb,
0xcd, 0x61, 0xa9, 0x8c, 0x76, 0xa0, 0x2a, 0xad, 0x34, 0xab, 0x0a, 0xf4, 0xc1, 0x54, 0x02, 0x7b, 0xda, 0x5f, 0xc0, 0x52, 0x18, 0xed, 0x42, 0x55, 0x6a, 0x69, 0x55, 0x15, 0xe8, 0xbf, 0x33, 0x09,
0x73, 0x58, 0x01, 0xd0, 0x97, 0xb0, 0xdc, 0x27, 0xc2, 0x0f, 0x7c, 0xe1, 0x37, 0x61, 0xa3, 0xd2, 0xec, 0x2f, 0x60, 0x05, 0x40, 0x9f, 0xc1, 0xca, 0x80, 0x08, 0x3f, 0xf0, 0x85, 0xdf, 0x82, 0xcd,
0x5a, 0xd9, 0xf6, 0x4a, 0xc1, 0x32, 0x41, 0xee, 0xa1, 0x41, 0x3c, 0x4a, 0x04, 0x1f, 0xe2, 0x63, 0x4a, 0x7b, 0x75, 0xc7, 0x2b, 0x05, 0xcb, 0x00, 0xb9, 0x47, 0x06, 0xf1, 0x24, 0x11, 0x7c, 0x84,
0x03, 0xf6, 0x5d, 0x58, 0x1d, 0xfb, 0x84, 0x1a, 0x50, 0x79, 0x49, 0x86, 0x3a, 0x7f, 0x58, 0x2e, 0x4f, 0x14, 0xd8, 0xf7, 0x61, 0x6d, 0xe2, 0x13, 0x6a, 0x42, 0xe5, 0x35, 0x19, 0xe9, 0xf8, 0x61,
0xd1, 0xbb, 0xb0, 0xf0, 0xca, 0x8f, 0x07, 0x44, 0xa5, 0xaa, 0x8e, 0xf5, 0xe6, 0xce, 0xfc, 0x6d, 0xb9, 0x44, 0xff, 0x86, 0xda, 0x1b, 0x3f, 0x1e, 0x12, 0x15, 0xaa, 0x06, 0xd6, 0x9b, 0x7b, 0x8b,
0xab, 0xbd, 0x0c, 0x8b, 0x5c, 0x99, 0x77, 0x7e, 0xb7, 0xa0, 0x31, 0x99, 0x27, 0xb4, 0x6f, 0x22, 0x77, 0xad, 0xce, 0x0a, 0x2c, 0x71, 0xa5, 0xde, 0xf9, 0xc5, 0x82, 0xe6, 0x74, 0x9c, 0xd0, 0x81,
0xb4, 0x14, 0xc9, 0x5b, 0x33, 0xa4, 0x58, 0x0a, 0x32, 0x4d, 0x55, 0x99, 0xb0, 0x77, 0xa0, 0x76, 0xf1, 0xd0, 0x52, 0x24, 0xef, 0xcc, 0x11, 0x62, 0x79, 0x90, 0x69, 0xaa, 0x4a, 0x85, 0xbd, 0x0b,
0x2c, 0x9a, 0x46, 0xb1, 0x56, 0xa0, 0xe8, 0xbc, 0x0f, 0x15, 0x4c, 0x42, 0x09, 0xa1, 0x81, 0x66, 0xf5, 0x93, 0xa3, 0x59, 0x14, 0xeb, 0x05, 0x8a, 0xce, 0x7d, 0xa8, 0x60, 0x12, 0x4a, 0x08, 0x0d,
0x52, 0xc3, 0x72, 0xe9, 0xfc, 0x69, 0xc9, 0xe2, 0x91, 0x6e, 0xd1, 0xfd, 0x31, 0x9e, 0xd3, 0x4b, 0x34, 0x93, 0x3a, 0x96, 0x4b, 0xe4, 0x40, 0x35, 0x90, 0xe4, 0x16, 0x15, 0xb9, 0x75, 0x37, 0xed,
0xe1, 0x04, 0xbb, 0x67, 0xe5, 0xec, 0x6e, 0x16, 0xd9, 0x4d, 0xad, 0x8f, 0x22, 0x7b, 0x01, 0xab, 0xb9, 0x7b, 0x24, 0xa4, 0x09, 0x15, 0x94, 0x25, 0x58, 0x7d, 0x73, 0x7e, 0xb3, 0x64, 0x81, 0x49,
0x98, 0x88, 0x01, 0x4f, 0x30, 0xf9, 0x61, 0x40, 0x32, 0x81, 0x3e, 0xcd, 0x33, 0xae, 0xec, 0x4f, 0x6a, 0xe8, 0xe1, 0x84, 0x2f, 0xb3, 0xcb, 0xe5, 0x94, 0x07, 0x2f, 0xca, 0x3d, 0xb8, 0x5d, 0xf4,
0x2b, 0x1b, 0xa9, 0x88, 0x0d, 0x00, 0xb5, 0x60, 0x81, 0x70, 0xce, 0xb8, 0x61, 0x81, 0x5c, 0x3d, 0x60, 0x66, 0x0d, 0x15, 0x3d, 0x14, 0xb0, 0x86, 0x89, 0x18, 0xf2, 0x04, 0x93, 0x6f, 0x87, 0x24,
0x19, 0x5c, 0x9e, 0xf6, 0xdc, 0xa7, 0x6a, 0x32, 0x60, 0xad, 0xe0, 0x34, 0x60, 0x2d, 0xf7, 0x9a, 0x13, 0xe8, 0xa3, 0x3c, 0x2b, 0x4a, 0xff, 0xac, 0xd2, 0x92, 0x82, 0xd8, 0x00, 0x50, 0x1b, 0x6a,
0xa5, 0x2c, 0xc9, 0x88, 0xf3, 0x9b, 0x05, 0x97, 0x30, 0x51, 0x83, 0x61, 0xbf, 0xef, 0x47, 0xe4, 0x84, 0x73, 0xc6, 0x0d, 0x0b, 0xe4, 0xea, 0xe9, 0xe1, 0xf2, 0xb4, 0xef, 0x3e, 0x57, 0xd3, 0x03,
0x21, 0x4b, 0x42, 0x1a, 0xe5, 0xa4, 0x1a, 0x2a, 0xc7, 0x79, 0xc4, 0x32, 0xdd, 0x2d, 0x58, 0xee, 0x6b, 0x01, 0xa7, 0x09, 0xeb, 0xb9, 0xd5, 0x2c, 0x65, 0x49, 0x46, 0x9c, 0x9f, 0x2d, 0xb8, 0x82,
0xc4, 0xbe, 0x08, 0x19, 0xef, 0x1b, 0x77, 0x75, 0x37, 0xed, 0xba, 0xb9, 0x0c, 0x1f, 0x7f, 0x45, 0x89, 0x1a, 0x1e, 0x07, 0x03, 0x3f, 0x22, 0x8f, 0x59, 0x12, 0xd2, 0x28, 0x27, 0xd5, 0x54, 0x79,
0x1b, 0xb0, 0x62, 0x0c, 0x1f, 0xb2, 0x80, 0xa8, 0x0e, 0xaa, 0xe1, 0xa2, 0x08, 0x35, 0x61, 0xe9, 0xc8, 0x3d, 0x96, 0x29, 0x69, 0xc3, 0x4a, 0x37, 0xf6, 0x45, 0xc8, 0xf8, 0xc0, 0x98, 0x6b, 0xc8,
0x80, 0x45, 0x5f, 0xf9, 0x7d, 0xa2, 0x5a, 0xa5, 0x86, 0xf3, 0xad, 0xf3, 0xb3, 0x05, 0xf6, 0x69, 0x24, 0xe4, 0x67, 0xf8, 0xe4, 0x2b, 0xda, 0x84, 0x55, 0xa3, 0xf8, 0x88, 0x05, 0x44, 0x75, 0x59,
0xac, 0x34, 0x69, 0xf4, 0x05, 0x2c, 0xee, 0xd2, 0x88, 0x64, 0x3a, 0x57, 0xb5, 0xf6, 0xf6, 0x9b, 0x1d, 0x17, 0x8f, 0x50, 0x0b, 0x96, 0x0f, 0x59, 0xf4, 0xb9, 0x3f, 0x20, 0xaa, 0x9d, 0xea, 0x38,
0xb7, 0x97, 0xe7, 0xfe, 0x7d, 0x7b, 0xf9, 0x6a, 0x61, 0xca, 0xb0, 0x94, 0x24, 0x3d, 0x96, 0x08, 0xdf, 0x3a, 0x3f, 0x58, 0x60, 0x9f, 0xc5, 0x4a, 0x93, 0x46, 0x9f, 0xc2, 0xd2, 0x1e, 0x8d, 0x48,
0x9f, 0x26, 0x84, 0xcb, 0x61, 0x79, 0x3d, 0x50, 0x10, 0x57, 0x23, 0xb1, 0xb1, 0x80, 0xde, 0x83, 0xa6, 0x63, 0x55, 0xef, 0xec, 0xbc, 0x7b, 0x7f, 0x75, 0xe1, 0xaf, 0xf7, 0x57, 0xaf, 0x17, 0x26,
0x45, 0x6d, 0xdd, 0x34, 0x81, 0xd9, 0x39, 0x7f, 0x54, 0xa1, 0xfe, 0x54, 0x12, 0xc8, 0x73, 0xe1, 0x11, 0x4b, 0x49, 0xd2, 0x67, 0x89, 0xf0, 0x69, 0x42, 0xb8, 0x1c, 0xa8, 0x37, 0x03, 0x05, 0x71,
0x02, 0xec, 0x92, 0x90, 0x26, 0x54, 0x50, 0x96, 0x98, 0x43, 0x5a, 0x93, 0xb1, 0x8f, 0xa4, 0xb8, 0x35, 0x12, 0x1b, 0x0d, 0xe8, 0x3f, 0xb0, 0xa4, 0xb5, 0x9b, 0x46, 0x31, 0x3b, 0xe7, 0xd7, 0x2a,
0xa0, 0x81, 0x6c, 0x58, 0x7e, 0x6c, 0x0e, 0xcc, 0x14, 0xef, 0xf1, 0x1e, 0x3d, 0x87, 0x95, 0x7c, 0x34, 0x9e, 0x4b, 0x02, 0x79, 0x2c, 0x5c, 0x80, 0x71, 0xa9, 0x99, 0x24, 0x4d, 0x17, 0x60, 0x41,
0xfd, 0x24, 0x15, 0xcd, 0x8a, 0x2a, 0xcf, 0xdb, 0x25, 0x27, 0x5e, 0x64, 0xe2, 0x16, 0xa0, 0xba, 0x02, 0xd9, 0xb0, 0xf2, 0xd4, 0x24, 0xcc, 0x14, 0xf8, 0xc9, 0x1e, 0xbd, 0x84, 0xd5, 0x7c, 0xfd,
0x56, 0x8b, 0xc6, 0xd0, 0x67, 0x70, 0x69, 0xbf, 0x9f, 0x32, 0x2e, 0x1e, 0xfa, 0xbd, 0x17, 0x04, 0x2c, 0x15, 0xad, 0x8a, 0x2a, 0xcf, 0xbb, 0x25, 0x19, 0x2f, 0x32, 0x71, 0x0b, 0x50, 0x5d, 0xab,
0x8f, 0xcf, 0xc4, 0xaa, 0x6a, 0x93, 0xb3, 0x15, 0xd0, 0x16, 0x5c, 0xf4, 0xe3, 0x98, 0xbd, 0x36, 0x45, 0x65, 0xe8, 0x63, 0xb8, 0x72, 0x30, 0x48, 0x19, 0x17, 0x8f, 0xfd, 0xfe, 0x2b, 0x82, 0x27,
0x25, 0xa6, 0x8a, 0xa5, 0xb9, 0xb0, 0x61, 0xb5, 0x96, 0xf1, 0xc9, 0x0f, 0xe8, 0x63, 0x78, 0xa7, 0xe7, 0x66, 0x55, 0xb5, 0xd2, 0xf9, 0x02, 0x68, 0x1b, 0x2e, 0xfb, 0x71, 0xcc, 0xde, 0x9a, 0x12,
0x20, 0x7c, 0xc0, 0xb9, 0x3f, 0x94, 0xf5, 0xb2, 0xa8, 0xf4, 0x4f, 0xfb, 0x24, 0xfb, 0xf9, 0x31, 0x53, 0xc5, 0xd2, 0xaa, 0x6d, 0x5a, 0xed, 0x15, 0x7c, 0xfa, 0x03, 0xfa, 0x00, 0xfe, 0x55, 0x38,
0x4d, 0xfc, 0xb8, 0x09, 0x4a, 0x47, 0x6f, 0x90, 0x03, 0xf5, 0x47, 0x3f, 0x4a, 0x4a, 0x84, 0x3f, 0x7c, 0xc4, 0xb9, 0x3f, 0x92, 0xf5, 0xb2, 0xa4, 0xe4, 0xcf, 0xfa, 0x24, 0x7b, 0xfe, 0x29, 0x4d,
0x10, 0x82, 0x37, 0x57, 0xd4, 0x51, 0x8c, 0xc9, 0x50, 0x07, 0xea, 0x8a, 0xb0, 0xe6, 0x9e, 0x35, 0xfc, 0xb8, 0x05, 0x4a, 0x46, 0x6f, 0x90, 0x03, 0x8d, 0x27, 0xdf, 0x49, 0x4a, 0x84, 0x3f, 0x12,
0xeb, 0x2a, 0x69, 0x5b, 0x25, 0x49, 0x53, 0xea, 0x4f, 0x52, 0x79, 0x1a, 0xa6, 0xa9, 0xc7, 0x2c, 0x82, 0xb7, 0x56, 0x55, 0x2a, 0x26, 0xce, 0x50, 0x17, 0x1a, 0x8a, 0xb0, 0xe6, 0x9e, 0xb5, 0x1a,
0xd8, 0xf7, 0xa0, 0x31, 0x99, 0xca, 0x99, 0x26, 0xd0, 0x5f, 0x16, 0x5c, 0x3c, 0xe1, 0x03, 0x21, 0x2a, 0x68, 0xdb, 0x25, 0x41, 0x53, 0xe2, 0xcf, 0x52, 0x99, 0x0d, 0xd3, 0xd4, 0x13, 0x1a, 0xec,
0xa8, 0x7e, 0x3d, 0x4c, 0x89, 0x31, 0xa1, 0xd6, 0xe8, 0x10, 0x16, 0x64, 0x0c, 0x59, 0x73, 0x5e, 0x07, 0xd0, 0x9c, 0x0e, 0xe5, 0x5c, 0x53, 0xea, 0x77, 0x0b, 0x2e, 0x9f, 0xb2, 0x81, 0x10, 0x54,
0x91, 0xde, 0x99, 0x85, 0xb4, 0xab, 0x90, 0x9a, 0xbf, 0xb6, 0x62, 0xdf, 0x06, 0x18, 0x09, 0x67, 0xbf, 0x18, 0xa5, 0xc4, 0xa8, 0x50, 0x6b, 0x74, 0x04, 0x35, 0xe9, 0x43, 0x3e, 0xb7, 0x76, 0xe7,
0xa2, 0xfc, 0x2d, 0xac, 0x9a, 0x52, 0x32, 0xad, 0xd4, 0xd0, 0xf7, 0x9b, 0x01, 0xcb, 0xdb, 0x6b, 0x21, 0xed, 0x2a, 0xa4, 0xe6, 0xaf, 0xb5, 0xd8, 0x77, 0x01, 0xc6, 0x87, 0x73, 0x51, 0xfe, 0x0a,
0x34, 0x88, 0x2a, 0x33, 0x0e, 0x22, 0xe7, 0x27, 0xb8, 0x80, 0x89, 0x1f, 0x3c, 0xa6, 0x31, 0x39, 0xd6, 0x4c, 0x29, 0x99, 0x56, 0x6a, 0xea, 0x3b, 0xd0, 0x80, 0xe5, 0x0d, 0x37, 0x1e, 0x44, 0x95,
0x7b, 0x82, 0xc8, 0xbe, 0xa0, 0x31, 0xe9, 0xf8, 0xe2, 0xc5, 0x71, 0x5f, 0x98, 0x3d, 0xba, 0x03, 0x39, 0x07, 0x91, 0xf3, 0x3d, 0x5c, 0xc2, 0xc4, 0x0f, 0x9e, 0xd2, 0x98, 0x9c, 0x3f, 0x41, 0x64,
0x0b, 0xd8, 0x4f, 0x22, 0x62, 0x5c, 0x7f, 0x58, 0xe2, 0x5a, 0x39, 0x91, 0xba, 0x58, 0x43, 0x9c, 0x5f, 0xd0, 0x98, 0x74, 0x7d, 0xf1, 0xea, 0xa4, 0x2f, 0xcc, 0x1e, 0xdd, 0x83, 0x1a, 0xf6, 0x93,
0xbb, 0x50, 0x3b, 0x96, 0xc9, 0xae, 0x7e, 0x12, 0x86, 0x19, 0xd1, 0x13, 0xa2, 0x82, 0xcd, 0x4e, 0x88, 0x18, 0xd3, 0xff, 0x2b, 0x31, 0xad, 0x8c, 0x48, 0x59, 0xac, 0x21, 0xce, 0x7d, 0xa8, 0x9f,
0xca, 0x0f, 0x48, 0x12, 0x19, 0xd7, 0x15, 0x6c, 0x76, 0xce, 0xa6, 0xbc, 0xe4, 0x72, 0xe6, 0x26, 0x9c, 0xc9, 0xae, 0x7e, 0x16, 0x86, 0x19, 0xd1, 0x13, 0xa2, 0x82, 0xcd, 0x4e, 0x9e, 0x1f, 0x92,
0x35, 0x08, 0xaa, 0xbb, 0xf2, 0x26, 0xb6, 0x54, 0x31, 0xaa, 0xb5, 0x13, 0xc8, 0x01, 0xea, 0x07, 0x24, 0x32, 0xa6, 0x2b, 0xd8, 0xec, 0x9c, 0x2d, 0x79, 0x11, 0xe6, 0xcc, 0x4d, 0x68, 0x10, 0x54,
0xbb, 0x94, 0x9f, 0x1d, 0x60, 0x13, 0x96, 0x76, 0x29, 0x2f, 0xc4, 0x97, 0x6f, 0xd1, 0x26, 0xac, 0xf7, 0xe4, 0x6d, 0x6d, 0xa9, 0x62, 0x54, 0x6b, 0x27, 0x90, 0x03, 0xd4, 0x0f, 0xf6, 0x28, 0x3f,
0xed, 0x27, 0xbd, 0x78, 0x10, 0xc8, 0x68, 0x05, 0xe1, 0x89, 0x99, 0x8a, 0x13, 0x52, 0xe7, 0xbe, 0xdf, 0xc1, 0x16, 0x2c, 0xef, 0x51, 0x5e, 0xf0, 0x2f, 0xdf, 0xa2, 0x2d, 0x58, 0x3f, 0x48, 0xfa,
0xce, 0xa3, 0xf2, 0x62, 0xc8, 0x6c, 0xc1, 0x12, 0x49, 0x04, 0xa7, 0x24, 0xbf, 0xcc, 0x90, 0xab, 0xf1, 0x30, 0x90, 0xde, 0x0a, 0xc2, 0x13, 0x33, 0x15, 0xa7, 0x4e, 0x9d, 0x87, 0x3a, 0x8e, 0xca,
0x1f, 0x4f, 0xae, 0x7a, 0x3c, 0xa9, 0x39, 0x8f, 0x73, 0x15, 0x67, 0x07, 0x2e, 0x48, 0x41, 0xf9, 0x8a, 0x21, 0xb3, 0x0d, 0xcb, 0x24, 0x11, 0x9c, 0x92, 0xfc, 0x32, 0x43, 0xae, 0x7e, 0x60, 0xb9,
0x41, 0x20, 0xa8, 0x16, 0x48, 0xaa, 0xb5, 0x73, 0x07, 0x1a, 0x23, 0xa0, 0x71, 0xbd, 0x09, 0x55, 0xea, 0x81, 0xa5, 0xe6, 0x3c, 0xce, 0x45, 0x9c, 0x5d, 0xb8, 0x24, 0x0f, 0xca, 0x13, 0x81, 0xa0,
0xf9, 0x34, 0x33, 0x23, 0xef, 0x34, 0xbf, 0xea, 0xbb, 0xb3, 0x0a, 0x2b, 0x1d, 0x9a, 0xe4, 0x77, 0x5a, 0x20, 0xa9, 0xd6, 0xce, 0x3d, 0x68, 0x8e, 0x81, 0xc6, 0xf4, 0x16, 0x54, 0xe5, 0xf3, 0xcd,
0x87, 0x73, 0x64, 0x41, 0xbd, 0xc3, 0x92, 0xd1, 0xd4, 0xee, 0xc0, 0x85, 0xbc, 0xdd, 0x1e, 0x74, 0x8c, 0xbc, 0xb3, 0xec, 0xaa, 0xef, 0xce, 0x1a, 0xac, 0x76, 0x69, 0x92, 0xdf, 0x1d, 0xce, 0xb1,
0xf6, 0x1f, 0xfa, 0x69, 0x1e, 0xca, 0xc6, 0xc9, 0x63, 0x36, 0xaf, 0x48, 0x57, 0x2b, 0xb6, 0xab, 0x05, 0x8d, 0x2e, 0x4b, 0xc6, 0x53, 0xbb, 0x0b, 0x97, 0xf2, 0x76, 0x7b, 0xd4, 0x3d, 0x78, 0xec,
0x72, 0xc0, 0xe3, 0x49, 0x38, 0xfa, 0x1c, 0x96, 0x0e, 0x0e, 0xda, 0xca, 0xd2, 0xfc, 0x4c, 0x96, 0xa7, 0xb9, 0x2b, 0x9b, 0xa7, 0xd3, 0x6c, 0x5e, 0x9a, 0xae, 0x16, 0xec, 0x54, 0xe5, 0x80, 0xc7,
0x72, 0x18, 0xba, 0x07, 0x4b, 0xcf, 0xd4, 0xe3, 0x36, 0x33, 0x43, 0xf8, 0x94, 0x92, 0xd3, 0x81, 0xd3, 0x70, 0xf4, 0x09, 0x2c, 0x1f, 0x1e, 0x76, 0x94, 0xa6, 0xc5, 0xb9, 0x34, 0xe5, 0x30, 0xf4,
0x6a, 0x35, 0x4c, 0x7a, 0x8c, 0x07, 0x38, 0x07, 0x6d, 0xff, 0xbd, 0x00, 0xb5, 0x83, 0x83, 0x76, 0x00, 0x96, 0x5f, 0xa8, 0x07, 0x70, 0x66, 0x86, 0xf0, 0x19, 0x25, 0xa7, 0x1d, 0xd5, 0x62, 0x98,
0x9b, 0xd3, 0x20, 0x22, 0xe8, 0x17, 0x0b, 0xd0, 0xc9, 0x6b, 0x0b, 0xdd, 0x2c, 0xef, 0xa0, 0xd3, 0xf4, 0x19, 0x0f, 0x70, 0x0e, 0xda, 0xf9, 0xa3, 0x06, 0xf5, 0xc3, 0xc3, 0x4e, 0x87, 0xd3, 0x20,
0xef, 0x5e, 0xfb, 0xd6, 0x8c, 0x28, 0x93, 0xe5, 0xe7, 0xb0, 0xa0, 0x3a, 0x1c, 0x7d, 0x74, 0xce, 0x22, 0xe8, 0x47, 0x0b, 0xd0, 0xe9, 0x6b, 0x0b, 0xdd, 0x2e, 0xef, 0xa0, 0xb3, 0xef, 0x5e, 0xfb,
0xeb, 0xc4, 0x6e, 0x4d, 0x57, 0x34, 0xb6, 0x7b, 0xb0, 0x9c, 0x77, 0x09, 0xba, 0x5a, 0x4a, 0x6f, 0xce, 0x9c, 0x28, 0x13, 0xe5, 0x97, 0x50, 0x53, 0x1d, 0x8e, 0xfe, 0x7f, 0xc1, 0xeb, 0xc4, 0x6e,
0x6c, 0x08, 0xd8, 0xd7, 0xce, 0xa5, 0x6b, 0x9c, 0x7c, 0x0f, 0x4b, 0xa6, 0xf8, 0xd1, 0x95, 0x29, 0xcf, 0x16, 0x34, 0xba, 0xfb, 0xb0, 0x92, 0x77, 0x09, 0xba, 0x5e, 0x4a, 0x6f, 0x62, 0x08, 0xd8,
0xb8, 0x51, 0x1b, 0xda, 0x57, 0xcf, 0xa3, 0x3a, 0x0a, 0x23, 0x2f, 0xf2, 0xd2, 0x30, 0x26, 0x5a, 0x37, 0x2e, 0x24, 0x6b, 0x8c, 0x7c, 0x03, 0xcb, 0xa6, 0xf8, 0xd1, 0xb5, 0x19, 0xb8, 0x71, 0x1b,
0xa8, 0x34, 0x8c, 0x13, 0x5d, 0xf3, 0x0c, 0xaa, 0xb2, 0x1b, 0xd0, 0x66, 0x09, 0xa8, 0xd0, 0x2e, 0xda, 0xd7, 0x2f, 0x22, 0x3a, 0x76, 0x23, 0x2f, 0xf2, 0x52, 0x37, 0xa6, 0x5a, 0xa8, 0xd4, 0x8d,
0x76, 0xd9, 0x71, 0x8d, 0xb5, 0xd1, 0x77, 0xf2, 0x75, 0xab, 0x6e, 0xdf, 0x56, 0x69, 0xcc, 0x85, 0x53, 0x5d, 0xf3, 0x02, 0xaa, 0xb2, 0x1b, 0xd0, 0x56, 0x09, 0xa8, 0xd0, 0x2e, 0x76, 0x59, 0xba,
0xc7, 0xa5, 0x7d, 0xe5, 0x1c, 0x9a, 0xda, 0x7c, 0xbb, 0xfe, 0xe6, 0x68, 0xdd, 0xfa, 0xe7, 0x68, 0x26, 0xda, 0xe8, 0x6b, 0xf9, 0xba, 0x55, 0xb7, 0x6f, 0xbb, 0xd4, 0xe7, 0xc2, 0xe3, 0xd2, 0xbe,
0xdd, 0xfa, 0xef, 0x68, 0xdd, 0xea, 0x2e, 0xaa, 0xff, 0x63, 0x37, 0xfe, 0x0f, 0x00, 0x00, 0xff, 0x76, 0x01, 0x49, 0xad, 0xbe, 0xd3, 0x78, 0x77, 0xbc, 0x61, 0xfd, 0x79, 0xbc, 0x61, 0xfd, 0x7d,
0xff, 0xbd, 0xb4, 0x1b, 0xf0, 0xe1, 0x0e, 0x00, 0x00, 0xbc, 0x61, 0xf5, 0x96, 0xd4, 0x7f, 0xb6, 0x5b, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x6d, 0x93,
0xbb, 0x7e, 0x05, 0x0f, 0x00, 0x00,
} }

View File

@ -46,6 +46,7 @@ message RefMapDeprecated {
message Ref { message Ref {
repeated string ids = 1; repeated string ids = 1;
repeated pb.Definition defs = 2;
} }
message RefMap { message RefMap {

View File

@ -1,9 +1,14 @@
package moby_buildkit_v1_frontend package moby_buildkit_v1_frontend
func NewRef(id string) *Ref { import "github.com/moby/buildkit/solver/pb"
func NewRef(id string, def *pb.Definition) *Ref {
var ref Ref var ref Ref
if id != "" { if id != "" {
ref.Ids = append(ref.Ids, id) ref.Ids = append(ref.Ids, id)
} }
if def != nil {
ref.Defs = append(ref.Defs, def)
}
return &ref return &ref
} }

View File

@ -1,6 +1,8 @@
package frontend package frontend
import "github.com/moby/buildkit/solver" import (
"github.com/moby/buildkit/solver"
)
type Result struct { type Result struct {
Ref solver.CachedResult Ref solver.CachedResult

View File

@ -11,6 +11,7 @@ import (
"github.com/containerd/containerd/platforms" "github.com/containerd/containerd/platforms"
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
"github.com/moby/buildkit/cache/remotecache" "github.com/moby/buildkit/cache/remotecache"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/executor" "github.com/moby/buildkit/executor"
"github.com/moby/buildkit/frontend" "github.com/moby/buildkit/frontend"
gw "github.com/moby/buildkit/frontend/gateway/client" gw "github.com/moby/buildkit/frontend/gateway/client"
@ -116,6 +117,12 @@ func (b *llbBridge) Solve(ctx context.Context, req frontend.SolveRequest) (res *
return nil, errors.Wrap(err, "failed to build LLB") return nil, errors.Wrap(err, "failed to build LLB")
} }
wref, ok := ref.Sys().(*worker.WorkerRef)
if !ok {
return nil, errors.Errorf("invalid ref: %T", ref.Sys())
}
wref.Definition = req.Definition
res = &frontend.Result{Ref: ref} res = &frontend.Result{Ref: ref}
} else if req.Frontend != "" { } else if req.Frontend != "" {
f, ok := b.frontends[req.Frontend] f, ok := b.frontends[req.Frontend]
@ -158,7 +165,7 @@ func (s *llbBridge) Exec(ctx context.Context, meta executor.Meta, root cache.Imm
return err return err
} }
func (s *llbBridge) ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt) (dgst digest.Digest, config []byte, err error) { func (s *llbBridge) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt) (dgst digest.Digest, config []byte, err error) {
w, err := s.resolveWorker() w, err := s.resolveWorker()
if err != nil { if err != nil {
return "", nil, err return "", nil, err

View File

@ -13,7 +13,7 @@ import (
"github.com/containerd/containerd/platforms" "github.com/containerd/containerd/platforms"
"github.com/docker/distribution/reference" "github.com/docker/distribution/reference"
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
gw "github.com/moby/buildkit/frontend/gateway/client" "github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/session" "github.com/moby/buildkit/session"
"github.com/moby/buildkit/snapshot" "github.com/moby/buildkit/snapshot"
"github.com/moby/buildkit/source" "github.com/moby/buildkit/source"
@ -60,7 +60,7 @@ func (is *imageSource) ID() string {
return source.DockerImageScheme return source.DockerImageScheme
} }
func (is *imageSource) ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) { func (is *imageSource) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) {
type t struct { type t struct {
dgst digest.Digest dgst digest.Digest
dt []byte dt []byte

View File

@ -21,6 +21,7 @@ import (
"github.com/moby/buildkit/cache/blobs" "github.com/moby/buildkit/cache/blobs"
"github.com/moby/buildkit/cache/metadata" "github.com/moby/buildkit/cache/metadata"
"github.com/moby/buildkit/client" "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/executor" "github.com/moby/buildkit/executor"
"github.com/moby/buildkit/exporter" "github.com/moby/buildkit/exporter"
imageexporter "github.com/moby/buildkit/exporter/containerimage" imageexporter "github.com/moby/buildkit/exporter/containerimage"
@ -28,7 +29,6 @@ import (
ociexporter "github.com/moby/buildkit/exporter/oci" ociexporter "github.com/moby/buildkit/exporter/oci"
tarexporter "github.com/moby/buildkit/exporter/tar" tarexporter "github.com/moby/buildkit/exporter/tar"
"github.com/moby/buildkit/frontend" "github.com/moby/buildkit/frontend"
gw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/identity" "github.com/moby/buildkit/identity"
"github.com/moby/buildkit/session" "github.com/moby/buildkit/session"
"github.com/moby/buildkit/snapshot" "github.com/moby/buildkit/snapshot"
@ -274,7 +274,7 @@ func (w *Worker) PruneCacheMounts(ctx context.Context, ids []string) error {
return nil return nil
} }
func (w *Worker) ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) { func (w *Worker) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) {
// ImageSource is typically source/containerimage // ImageSource is typically source/containerimage
resolveImageConfig, ok := w.ImageSource.(resolveImageConfig) resolveImageConfig, ok := w.ImageSource.(resolveImageConfig)
if !ok { if !ok {
@ -284,7 +284,7 @@ func (w *Worker) ResolveImageConfig(ctx context.Context, ref string, opt gw.Reso
} }
type resolveImageConfig interface { type resolveImageConfig interface {
ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error)
} }
func (w *Worker) Exec(ctx context.Context, meta executor.Meta, rootFS cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error { func (w *Worker) Exec(ctx context.Context, meta executor.Meta, rootFS cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error {

View File

@ -5,6 +5,7 @@ import (
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
"github.com/moby/buildkit/solver" "github.com/moby/buildkit/solver"
"github.com/moby/buildkit/solver/pb"
) )
func NewWorkerRefResult(ref cache.ImmutableRef, worker Worker) solver.Result { func NewWorkerRefResult(ref cache.ImmutableRef, worker Worker) solver.Result {
@ -14,6 +15,7 @@ func NewWorkerRefResult(ref cache.ImmutableRef, worker Worker) solver.Result {
type WorkerRef struct { type WorkerRef struct {
ImmutableRef cache.ImmutableRef ImmutableRef cache.ImmutableRef
Worker Worker Worker Worker
Definition *pb.Definition
} }
func (wr *WorkerRef) ID() string { func (wr *WorkerRef) ID() string {

View File

@ -7,10 +7,10 @@ import (
"github.com/containerd/containerd/content" "github.com/containerd/containerd/content"
"github.com/moby/buildkit/cache" "github.com/moby/buildkit/cache"
"github.com/moby/buildkit/client" "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/executor" "github.com/moby/buildkit/executor"
"github.com/moby/buildkit/exporter" "github.com/moby/buildkit/exporter"
"github.com/moby/buildkit/frontend" "github.com/moby/buildkit/frontend"
gw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/session" "github.com/moby/buildkit/session"
"github.com/moby/buildkit/solver" "github.com/moby/buildkit/solver"
digest "github.com/opencontainers/go-digest" digest "github.com/opencontainers/go-digest"
@ -26,7 +26,7 @@ type Worker interface {
LoadRef(id string, hidden bool) (cache.ImmutableRef, error) LoadRef(id string, hidden bool) (cache.ImmutableRef, error)
// ResolveOp resolves Vertex.Sys() to Op implementation. // ResolveOp resolves Vertex.Sys() to Op implementation.
ResolveOp(v solver.Vertex, s frontend.FrontendLLBBridge, sm *session.Manager) (solver.Op, error) ResolveOp(v solver.Vertex, s frontend.FrontendLLBBridge, sm *session.Manager) (solver.Op, error)
ResolveImageConfig(ctx context.Context, ref string, opt gw.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error) ResolveImageConfig(ctx context.Context, ref string, opt llb.ResolveImageConfigOpt, sm *session.Manager) (digest.Digest, []byte, error)
// Exec is similar to executor.Exec but without []mount.Mount // Exec is similar to executor.Exec but without []mount.Mount
Exec(ctx context.Context, meta executor.Meta, rootFS cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error Exec(ctx context.Context, meta executor.Meta, rootFS cache.ImmutableRef, stdin io.ReadCloser, stdout, stderr io.WriteCloser) error
DiskUsage(ctx context.Context, opt client.DiskUsageInfo) ([]*client.UsageInfo, error) DiskUsage(ctx context.Context, opt client.DiskUsageInfo) ([]*client.UsageInfo, error)