Merge pull request #173 from dagger/push-container

Push Container Support
This commit is contained in:
Andrea Luzzardi 2021-03-12 15:52:52 -08:00 committed by GitHub
commit 9d916a2fc2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 454 additions and 529 deletions

View File

@ -1,67 +0,0 @@
package dagger
import (
"context"
"errors"
"fmt"
"path"
"path/filepath"
cueerrors "cuelang.org/go/cue/errors"
cueload "cuelang.org/go/cue/load"
"github.com/rs/zerolog/log"
"dagger.io/go/dagger/compiler"
"dagger.io/go/stdlib"
)
// Build a cue configuration tree from the files in fs.
func CueBuild(ctx context.Context, fs FS, args ...string) (*compiler.Value, error) {
var (
err error
lg = log.Ctx(ctx)
)
buildConfig := &cueload.Config{
// The CUE overlay needs to be prefixed by a non-conflicting path with the
// local filesystem, otherwise Cue will merge the Overlay with whatever Cue
// files it finds locally.
Dir: "/config",
}
// Start by creating an overlay with the stdlib
buildConfig.Overlay, err = stdlib.Overlay(buildConfig.Dir)
if err != nil {
return nil, err
}
// Add the config files on top of the overlay
err = fs.Walk(ctx, func(p string, f Stat) error {
lg.Debug().Str("path", p).Msg("load")
if f.IsDir() {
return nil
}
if filepath.Ext(p) != ".cue" {
return nil
}
contents, err := fs.ReadFile(ctx, p)
if err != nil {
return fmt.Errorf("%s: %w", p, err)
}
overlayPath := path.Join(buildConfig.Dir, p)
buildConfig.Overlay[overlayPath] = cueload.FromBytes(contents)
return nil
})
if err != nil {
return nil, err
}
instances := cueload.Instances(args, buildConfig)
if len(instances) != 1 {
return nil, errors.New("only one package is supported at a time")
}
inst, err := compiler.Cue().Build(instances[0])
if err != nil {
return nil, errors.New(cueerrors.Details(err, &cueerrors.Config{}))
}
return compiler.Wrap(inst.Value(), inst), nil
}

View File

@ -20,6 +20,7 @@ import (
// buildkit // buildkit
bk "github.com/moby/buildkit/client" bk "github.com/moby/buildkit/client"
_ "github.com/moby/buildkit/client/connhelper/dockercontainer" // import the container connection driver _ "github.com/moby/buildkit/client/connhelper/dockercontainer" // import the container connection driver
"github.com/moby/buildkit/client/llb"
bkgw "github.com/moby/buildkit/frontend/gateway/client" bkgw "github.com/moby/buildkit/frontend/gateway/client"
// docker output // docker output
@ -127,8 +128,8 @@ func (c *Client) buildfn(ctx context.Context, env *Env, ch chan *bk.SolveStatus,
Interface("attrs", opts.FrontendAttrs). Interface("attrs", opts.FrontendAttrs).
Msg("spawning buildkit job") Msg("spawning buildkit job")
resp, err := c.c.Build(ctx, opts, "", func(ctx context.Context, c bkgw.Client) (*bkgw.Result, error) { resp, err := c.c.Build(ctx, opts, "", func(ctx context.Context, gw bkgw.Client) (*bkgw.Result, error) {
s := NewSolver(c) s := NewSolver(c.c, gw, ch)
lg.Debug().Msg("loading configuration") lg.Debug().Msg("loading configuration")
if err := env.Update(ctx, s); err != nil { if err := env.Update(ctx, s); err != nil {
@ -142,14 +143,22 @@ func (c *Client) buildfn(ctx context.Context, env *Env, ch chan *bk.SolveStatus,
} }
// Export env to a cue directory // Export env to a cue directory
// FIXME: this should be elsewhere
lg.Debug().Msg("exporting env") lg.Debug().Msg("exporting env")
outdir, err := env.Export(ctx, s.Scratch()) span, _ := opentracing.StartSpanFromContext(ctx, "Env.Export")
defer span.Finish()
st := llb.Scratch().File(
llb.Mkfile("state.cue", 0600, env.State().JSON()),
llb.WithCustomName("[internal] serializing state to JSON"),
)
ref, err := s.Solve(ctx, st)
if err != nil { if err != nil {
return nil, err return nil, err
} }
res := bkgw.NewResult()
// Wrap cue directory in buildkit result res.SetRef(ref)
return outdir.Result(ctx) return res, nil
}, ch) }, ch)
if err != nil { if err != nil {
return fmt.Errorf("buildkit solve: %w", bkCleanError(err)) return fmt.Errorf("buildkit solve: %w", bkCleanError(err))

63
dagger/compiler/build.go Normal file
View File

@ -0,0 +1,63 @@
package compiler
import (
"errors"
"fmt"
"io/fs"
"path"
"path/filepath"
cueerrors "cuelang.org/go/cue/errors"
cueload "cuelang.org/go/cue/load"
)
// Build a cue configuration tree from the files in fs.
func Build(sources map[string]fs.FS, args ...string) (*Value, error) {
buildConfig := &cueload.Config{
// The CUE overlay needs to be prefixed by a non-conflicting path with the
// local filesystem, otherwise Cue will merge the Overlay with whatever Cue
// files it finds locally.
Dir: "/config",
Overlay: map[string]cueload.Source{},
}
// Map the source files into the overlay
for mnt, f := range sources {
f := f
mnt := mnt
err := fs.WalkDir(f, ".", func(p string, entry fs.DirEntry, err error) error {
if err != nil {
return err
}
if !entry.Type().IsRegular() {
return nil
}
if filepath.Ext(entry.Name()) != ".cue" {
return nil
}
contents, err := fs.ReadFile(f, p)
if err != nil {
return fmt.Errorf("%s: %w", p, err)
}
overlayPath := path.Join(buildConfig.Dir, mnt, p)
buildConfig.Overlay[overlayPath] = cueload.FromBytes(contents)
return nil
})
if err != nil {
return nil, err
}
}
instances := cueload.Instances(args, buildConfig)
if len(instances) != 1 {
return nil, errors.New("only one package is supported at a time")
}
inst, err := Cue().Build(instances[0])
if err != nil {
return nil, errors.New(cueerrors.Details(err, &cueerrors.Config{}))
}
return Wrap(inst.Value(), inst), nil
}

View File

@ -3,12 +3,14 @@ package dagger
import ( import (
"context" "context"
"fmt" "fmt"
"io/fs"
"strings" "strings"
"time" "time"
"cuelang.org/go/cue" "cuelang.org/go/cue"
cueflow "cuelang.org/go/tools/flow" cueflow "cuelang.org/go/tools/flow"
"dagger.io/go/dagger/compiler" "dagger.io/go/dagger/compiler"
"dagger.io/go/stdlib"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/ext" "github.com/opentracing/opentracing-go/ext"
@ -98,9 +100,12 @@ func (env *Env) Update(ctx context.Context, s Solver) error {
return err return err
} }
// load cue files produced by updater // Build a Cue config by overlaying the source with the stdlib
// FIXME: BuildAll() to force all files (no required package..) sources := map[string]fs.FS{
base, err := CueBuild(ctx, p.FS()) stdlib.Path: stdlib.FS,
"/": p.FS(),
}
base, err := compiler.Build(sources)
if err != nil { if err != nil {
return fmt.Errorf("base config: %w", err) return fmt.Errorf("base config: %w", err)
} }
@ -190,33 +195,6 @@ func (env *Env) mergeState() error {
return nil return nil
} }
// Export env to a directory of cue files
// (Use with FS.Change)
func (env *Env) Export(ctx context.Context, fs FS) (FS, error) {
span, _ := opentracing.StartSpanFromContext(ctx, "Env.Export")
defer span.Finish()
// FIXME: we serialize as JSON to guarantee a self-contained file.
// compiler.Value.Save() leaks imports, so requires a shared cue.mod with
// client which is undesirable.
// Once compiler.Value.Save() resolves non-builtin imports with a tree shake,
// we can use it here.
// FIXME: Exporting base/input/output separately causes merge errors.
// For instance, `foo: string | *"default foo"` gets serialized as
// `{"foo":"default foo"}`, which will fail to merge if output contains
// a different definition of `foo`.
//
// fs = env.base.SaveJSON(fs, "base.cue")
// fs = env.input.SaveJSON(fs, "input.cue")
// if env.output != nil {
// fs = env.output.SaveJSON(fs, "output.cue")
// }
// For now, export a single `state.cue` containing the combined output.
fs = fs.WriteValueJSON("state.cue", env.state)
return fs, nil
}
// Compute missing values in env configuration, and write them to state. // Compute missing values in env configuration, and write them to state.
func (env *Env) Compute(ctx context.Context, s Solver) error { func (env *Env) Compute(ctx context.Context, s Solver) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "Env.Compute") span, ctx := opentracing.StartSpanFromContext(ctx, "Env.Compute")

View File

@ -3,208 +3,102 @@ package dagger
import ( import (
"context" "context"
"errors" "errors"
"os" "io/fs"
"path" "time"
"strings"
"github.com/moby/buildkit/client/llb"
bkgw "github.com/moby/buildkit/frontend/gateway/client" bkgw "github.com/moby/buildkit/frontend/gateway/client"
bkpb "github.com/moby/buildkit/solver/pb"
fstypes "github.com/tonistiigi/fsutil/types" fstypes "github.com/tonistiigi/fsutil/types"
"dagger.io/go/dagger/compiler"
) )
type Stat struct { // BuildkitFS is a io/fs.FS adapter for Buildkit
*fstypes.Stat // BuildkitFS implements the ReadFileFS, StatFS and ReadDirFS interfaces.
type BuildkitFS struct {
ref bkgw.Reference
} }
type FS struct { func NewBuildkitFS(ref bkgw.Reference) *BuildkitFS {
// Before last solve return &BuildkitFS{
input llb.State ref: ref,
// After last solve }
output bkgw.Reference
// How to produce the output
s Solver
} }
func (fs FS) WriteValueJSON(filename string, v *compiler.Value) FS { // Open is not supported.
return fs.Change(func(st llb.State) llb.State { func (f *BuildkitFS) Open(name string) (fs.File, error) {
return st.File( return nil, errors.New("not implemented")
llb.Mkfile(filename, 0600, v.JSON()),
llb.WithCustomName("[internal] serializing state to JSON"),
)
})
} }
func (fs FS) WriteValueCUE(filename string, v *compiler.Value) (FS, error) { func (f *BuildkitFS) Stat(name string) (fs.FileInfo, error) {
src, err := v.Source() st, err := f.ref.StatFile(context.TODO(), bkgw.StatRequest{
if err != nil { Path: name,
return fs, err
}
return fs.Change(func(st llb.State) llb.State {
return st.File(
llb.Mkfile(filename, 0600, src),
llb.WithCustomName("[internal] serializing state to CUE"),
)
}), nil
}
func (fs FS) Solver() Solver {
return fs.s
}
// Compute output from input, if not done already.
// This method uses a pointer receiver to simplify
// calling it, since it is called in almost every
// other method.
func (fs *FS) solve(ctx context.Context) error {
if fs.output != nil {
return nil
}
output, err := fs.s.Solve(ctx, fs.input)
if err != nil {
return bkCleanError(err)
}
fs.output = output
return nil
}
func (fs FS) ReadFile(ctx context.Context, filename string) ([]byte, error) {
// Lazy solve
if err := (&fs).solve(ctx); err != nil {
return nil, err
}
// NOTE: llb.Scratch is represented by a `nil` reference. If solve result is
// Scratch, then `fs.output` is `nil`.
if fs.output == nil {
return nil, os.ErrNotExist
}
contents, err := fs.output.ReadFile(ctx, bkgw.ReadRequest{Filename: filename})
if err != nil {
return nil, bkCleanError(err)
}
return contents, nil
}
func (fs FS) ReadDir(ctx context.Context, dir string) ([]Stat, error) {
// Lazy solve
if err := (&fs).solve(ctx); err != nil {
return nil, err
}
// NOTE: llb.Scratch is represented by a `nil` reference. If solve result is
// Scratch, then `fs.output` is `nil`.
if fs.output == nil {
return []Stat{}, nil
}
st, err := fs.output.ReadDir(ctx, bkgw.ReadDirRequest{
Path: dir,
}) })
if err != nil { if err != nil {
return nil, bkCleanError(err) return nil, err
} }
out := make([]Stat, len(st)) return bkFileInfo{st}, nil
for i := range st {
out[i] = Stat{
Stat: st[i],
}
}
return out, nil
} }
func (fs FS) walk(ctx context.Context, p string, fn WalkFunc) error { func (f *BuildkitFS) ReadDir(name string) ([]fs.DirEntry, error) {
files, err := fs.ReadDir(ctx, p) entries, err := f.ref.ReadDir(context.TODO(), bkgw.ReadDirRequest{
if err != nil { Path: name,
return err })
}
for _, f := range files {
fPath := path.Join(p, f.GetPath())
if err := fn(fPath, f); err != nil {
return err
}
if f.IsDir() {
if err := fs.walk(ctx, fPath, fn); err != nil {
return err
}
}
}
return nil
}
type WalkFunc func(string, Stat) error
func (fs FS) Walk(ctx context.Context, fn WalkFunc) error {
return fs.walk(ctx, "/", fn)
}
type ChangeFunc func(llb.State) llb.State
func (fs FS) Change(changes ...ChangeFunc) FS {
for _, change := range changes {
fs = fs.Set(change(fs.input))
}
return fs
}
func (fs FS) Set(st llb.State) FS {
fs.input = st
fs.output = nil
return fs
}
func (fs FS) Solve(ctx context.Context) (FS, error) {
if err := (&fs).solve(ctx); err != nil {
return fs, err
}
return fs, nil
}
func (fs FS) LLB() llb.State {
return fs.input
}
func (fs FS) Def(ctx context.Context) (*bkpb.Definition, error) {
def, err := fs.LLB().Marshal(ctx, llb.LinuxAmd64)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return def.ToPB(), nil res := make([]fs.DirEntry, 0, len(entries))
} for _, st := range entries {
res = append(res, bkDirEntry{
func (fs FS) Ref(ctx context.Context) (bkgw.Reference, error) { bkFileInfo: bkFileInfo{
if err := (&fs).solve(ctx); err != nil { st: st,
return nil, err },
})
} }
return fs.output, nil
}
func (fs FS) Result(ctx context.Context) (*bkgw.Result, error) {
res := bkgw.NewResult()
ref, err := fs.Ref(ctx)
if err != nil {
return nil, err
}
res.SetRef(ref)
return res, nil return res, nil
} }
// A helper to remove noise from buildkit error messages. func (f *BuildkitFS) ReadFile(name string) ([]byte, error) {
// FIXME: Obviously a cleaner solution would be nice. return f.ref.ReadFile(context.TODO(), bkgw.ReadRequest{
func bkCleanError(err error) error { Filename: name,
noise := []string{ })
"executor failed running ", }
"buildkit-runc did not terminate successfully",
"rpc error: code = Unknown desc = ", // bkFileInfo is a fs.FileInfo adapter for fstypes.Stat
"failed to solve: ", type bkFileInfo struct {
} st *fstypes.Stat
}
msg := err.Error()
func (s bkFileInfo) Name() string {
for _, s := range noise { return s.st.GetPath()
msg = strings.ReplaceAll(msg, s, "") }
}
func (s bkFileInfo) Size() int64 {
return errors.New(msg) return s.st.GetSize_()
}
func (s bkFileInfo) IsDir() bool {
return s.st.IsDir()
}
func (s bkFileInfo) ModTime() time.Time {
return time.Unix(s.st.GetModTime(), 0)
}
func (s bkFileInfo) Mode() fs.FileMode {
return fs.FileMode(s.st.Mode)
}
func (s bkFileInfo) Sys() interface{} {
return s.st
}
// bkDirEntry is a fs.DirEntry adapter for fstypes.Stat
type bkDirEntry struct {
bkFileInfo
}
func (s bkDirEntry) Info() (fs.FileInfo, error) {
return s.bkFileInfo, nil
}
func (s bkDirEntry) Type() fs.FileMode {
return s.Mode()
} }

View File

@ -9,6 +9,7 @@ import (
"strings" "strings"
"github.com/docker/distribution/reference" "github.com/docker/distribution/reference"
bk "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb" "github.com/moby/buildkit/client/llb"
dockerfilebuilder "github.com/moby/buildkit/frontend/dockerfile/builder" dockerfilebuilder "github.com/moby/buildkit/frontend/dockerfile/builder"
bkgw "github.com/moby/buildkit/frontend/gateway/client" bkgw "github.com/moby/buildkit/frontend/gateway/client"
@ -21,23 +22,32 @@ import (
// An execution pipeline // An execution pipeline
type Pipeline struct { type Pipeline struct {
name string name string
s Solver s Solver
fs FS state llb.State
out *Fillable result bkgw.Reference
out *Fillable
} }
func NewPipeline(name string, s Solver, out *Fillable) *Pipeline { func NewPipeline(name string, s Solver, out *Fillable) *Pipeline {
return &Pipeline{ return &Pipeline{
name: name, name: name,
s: s, s: s,
fs: s.Scratch(), state: llb.Scratch(),
out: out, out: out,
} }
} }
func (p *Pipeline) FS() FS { func (p *Pipeline) State() llb.State {
return p.fs return p.state
}
func (p *Pipeline) Result() bkgw.Reference {
return p.result
}
func (p *Pipeline) FS() fs.FS {
return NewBuildkitFS(p.result)
} }
func isComponent(v *compiler.Value) bool { func isComponent(v *compiler.Value) bool {
@ -128,52 +138,54 @@ func (p *Pipeline) Do(ctx context.Context, code ...*compiler.Value) error {
Msg("pipeline was partially executed because of missing inputs") Msg("pipeline was partially executed because of missing inputs")
return nil return nil
} }
if err := p.doOp(ctx, op); err != nil { p.state, err = p.doOp(ctx, op, p.state)
if err != nil {
return err return err
} }
// Force a buildkit solve request at each operation, // Force a buildkit solve request at each operation,
// so that errors map to the correct cue path. // so that errors map to the correct cue path.
// FIXME: might as well change FS to make every operation // FIXME: might as well change FS to make every operation
// synchronous. // synchronous.
fs, err := p.fs.Solve(ctx) p.result, err = p.s.Solve(ctx, p.state)
if err != nil { if err != nil {
return err return err
} }
p.fs = fs
} }
return nil return nil
} }
func (p *Pipeline) doOp(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) doOp(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
do, err := op.Get("do").String() do, err := op.Get("do").String()
if err != nil { if err != nil {
return err return st, err
} }
switch do { switch do {
case "copy": case "copy":
return p.Copy(ctx, op) return p.Copy(ctx, op, st)
case "exec": case "exec":
return p.Exec(ctx, op) return p.Exec(ctx, op, st)
case "export": case "export":
return p.Export(ctx, op) return p.Export(ctx, op, st)
case "fetch-container": case "fetch-container":
return p.FetchContainer(ctx, op) return p.FetchContainer(ctx, op, st)
case "push-container":
return p.PushContainer(ctx, op, st)
case "fetch-git": case "fetch-git":
return p.FetchGit(ctx, op) return p.FetchGit(ctx, op, st)
case "local": case "local":
return p.Local(ctx, op) return p.Local(ctx, op, st)
case "load": case "load":
return p.Load(ctx, op) return p.Load(ctx, op, st)
case "subdir": case "subdir":
return p.Subdir(ctx, op) return p.Subdir(ctx, op, st)
case "docker-build": case "docker-build":
return p.DockerBuild(ctx, op) return p.DockerBuild(ctx, op, st)
case "write-file": case "write-file":
return p.WriteFile(ctx, op) return p.WriteFile(ctx, op, st)
case "mkdir": case "mkdir":
return p.Mkdir(ctx, op) return p.Mkdir(ctx, op, st)
default: default:
return fmt.Errorf("invalid operation: %s", op.JSON()) return st, fmt.Errorf("invalid operation: %s", op.JSON())
} }
} }
@ -183,80 +195,72 @@ func (p *Pipeline) vertexNamef(format string, a ...interface{}) string {
return prefix + " " + name return prefix + " " + name
} }
// Spawn a temporary pipeline with the same solver. func (p *Pipeline) Subdir(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
// Output values are discarded: the parent pipeline's values are not modified.
func (p *Pipeline) Tmp(name string) *Pipeline {
return NewPipeline(name, p.s, nil)
}
func (p *Pipeline) Subdir(ctx context.Context, op *compiler.Value) error {
// FIXME: this could be more optimized by carrying subdir path as metadata, // FIXME: this could be more optimized by carrying subdir path as metadata,
// and using it in copy, load or mount. // and using it in copy, load or mount.
dir, err := op.Get("dir").String() dir, err := op.Get("dir").String()
if err != nil { if err != nil {
return err return st, err
} }
p.fs = p.fs.Change(func(st llb.State) llb.State { return st.File(
return st.File( llb.Copy(
llb.Copy( st,
p.fs.LLB(), dir,
dir, "/",
"/", &llb.CopyInfo{
&llb.CopyInfo{ CopyDirContentsOnly: true,
CopyDirContentsOnly: true, },
}, ),
), llb.WithCustomName(p.vertexNamef("Subdir %s", dir)),
llb.WithCustomName(p.vertexNamef("Subdir %s", dir)), ), nil
)
})
return nil
} }
func (p *Pipeline) Copy(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Copy(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
// Decode copy options // Decode copy options
src, err := op.Get("src").String() src, err := op.Get("src").String()
if err != nil { if err != nil {
return err return st, err
} }
dest, err := op.Get("dest").String() dest, err := op.Get("dest").String()
if err != nil { if err != nil {
return err return st, err
} }
// Execute 'from' in a tmp pipeline, and use the resulting fs // Execute 'from' in a tmp pipeline, and use the resulting fs
from := p.Tmp(op.Get("from").Path().String()) from := NewPipeline(op.Get("from").Path().String(), p.s, nil)
if err := from.Do(ctx, op.Get("from")); err != nil { if err := from.Do(ctx, op.Get("from")); err != nil {
return err return st, err
} }
p.fs = p.fs.Change(func(st llb.State) llb.State { fromResult, err := from.Result().ToState()
return st.File( if err != nil {
llb.Copy( return st, err
from.FS().LLB(), }
src, return st.File(
dest, llb.Copy(
// FIXME: allow more configurable llb options fromResult,
// For now we define the following convenience presets: src,
&llb.CopyInfo{ dest,
CopyDirContentsOnly: true, // FIXME: allow more configurable llb options
CreateDestPath: true, // For now we define the following convenience presets:
AllowWildcard: true, &llb.CopyInfo{
}, CopyDirContentsOnly: true,
), CreateDestPath: true,
llb.WithCustomName(p.vertexNamef("Copy %s %s", src, dest)), AllowWildcard: true,
) },
}) ),
return nil llb.WithCustomName(p.vertexNamef("Copy %s %s", src, dest)),
), nil
} }
func (p *Pipeline) Local(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Local(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
dir, err := op.Get("dir").String() dir, err := op.Get("dir").String()
if err != nil { if err != nil {
return err return st, err
} }
var include []string var include []string
if inc := op.Get("include"); inc.Exists() { if inc := op.Get("include"); inc.Exists() {
if err := inc.Decode(&include); err != nil { if err := inc.Decode(&include); err != nil {
return err return st, err
} }
} }
// FIXME: Remove the `Copy` and use `Local` directly. // FIXME: Remove the `Copy` and use `Local` directly.
@ -267,30 +271,26 @@ func (p *Pipeline) Local(ctx context.Context, op *compiler.Value) error {
// //
// By wrapping `llb.Local` inside `llb.Copy`, we get the same digest for // By wrapping `llb.Local` inside `llb.Copy`, we get the same digest for
// the same content. // the same content.
p.fs = p.fs.Change(func(st llb.State) llb.State { return st.File(
return st.File( llb.Copy(
llb.Copy( llb.Local(
llb.Local( dir,
dir, llb.FollowPaths(include),
llb.FollowPaths(include), llb.WithCustomName(p.vertexNamef("Local %s [transfer]", dir)),
llb.WithCustomName(p.vertexNamef("Local %s [transfer]", dir)),
// Without hint, multiple `llb.Local` operations on the // Without hint, multiple `llb.Local` operations on the
// same path get a different digest. // same path get a different digest.
llb.SessionID(p.s.SessionID()), llb.SessionID(p.s.SessionID()),
llb.SharedKeyHint(dir), llb.SharedKeyHint(dir),
),
"/",
"/",
), ),
llb.WithCustomName(p.vertexNamef("Local %s [copy]", dir)), "/",
) "/",
}) ),
llb.WithCustomName(p.vertexNamef("Local %s [copy]", dir)),
return nil ), nil
} }
func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
opts := []llb.RunOption{} opts := []llb.RunOption{}
var cmd struct { var cmd struct {
Args []string Args []string
@ -300,7 +300,7 @@ func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value) error {
} }
if err := op.Decode(&cmd); err != nil { if err := op.Decode(&cmd); err != nil {
return err return st, err
} }
// args // args
opts = append(opts, llb.Args(cmd.Args)) opts = append(opts, llb.Args(cmd.Args))
@ -315,7 +315,7 @@ func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value) error {
if cmd.Always { if cmd.Always {
cacheBuster, err := randomID(8) cacheBuster, err := randomID(8)
if err != nil { if err != nil {
return err return st, err
} }
opts = append(opts, llb.AddEnv("DAGGER_CACHEBUSTER", cacheBuster)) opts = append(opts, llb.AddEnv("DAGGER_CACHEBUSTER", cacheBuster))
} }
@ -323,7 +323,7 @@ func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value) error {
if mounts := op.Lookup("mount"); mounts.Exists() { if mounts := op.Lookup("mount"); mounts.Exists() {
mntOpts, err := p.mountAll(ctx, mounts) mntOpts, err := p.mountAll(ctx, mounts)
if err != nil { if err != nil {
return err return st, err
} }
opts = append(opts, mntOpts...) opts = append(opts, mntOpts...)
} }
@ -337,10 +337,7 @@ func (p *Pipeline) Exec(ctx context.Context, op *compiler.Value) error {
opts = append(opts, llb.WithCustomName(p.vertexNamef("Exec [%s]", strings.Join(args, ", ")))) opts = append(opts, llb.WithCustomName(p.vertexNamef("Exec [%s]", strings.Join(args, ", "))))
// --> Execute // --> Execute
p.fs = p.fs.Change(func(st llb.State) llb.State { return st.Run(opts...).Root(), nil
return st.Run(opts...).Root()
})
return nil
} }
func (p *Pipeline) mountAll(ctx context.Context, mounts *compiler.Value) ([]llb.RunOption, error) { func (p *Pipeline) mountAll(ctx context.Context, mounts *compiler.Value) ([]llb.RunOption, error) {
@ -380,10 +377,14 @@ func (p *Pipeline) mount(ctx context.Context, dest string, mnt *compiler.Value)
} }
} }
// eg. mount: "/foo": { from: www.source } // eg. mount: "/foo": { from: www.source }
from := p.Tmp(mnt.Get("from").Path().String()) from := NewPipeline(mnt.Get("from").Path().String(), p.s, nil)
if err := from.Do(ctx, mnt.Get("from")); err != nil { if err := from.Do(ctx, mnt.Get("from")); err != nil {
return nil, err return nil, err
} }
fromResult, err := from.Result().ToState()
if err != nil {
return nil, err
}
// possibly construct mount options for LLB from // possibly construct mount options for LLB from
var mo []llb.MountOption var mo []llb.MountOption
// handle "path" option // handle "path" option
@ -394,21 +395,21 @@ func (p *Pipeline) mount(ctx context.Context, dest string, mnt *compiler.Value)
} }
mo = append(mo, llb.SourcePath(mps)) mo = append(mo, llb.SourcePath(mps))
} }
return llb.AddMount(dest, from.FS().LLB(), mo...), nil return llb.AddMount(dest, fromResult, mo...), nil
} }
func (p *Pipeline) Export(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Export(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
source, err := op.Get("source").String() source, err := op.Get("source").String()
if err != nil { if err != nil {
return err return st, err
} }
format, err := op.Get("format").String() format, err := op.Get("format").String()
if err != nil { if err != nil {
return err return st, err
} }
contents, err := p.fs.ReadFile(ctx, source) contents, err := fs.ReadFile(p.FS(), source)
if err != nil { if err != nil {
return fmt.Errorf("export %s: %w", source, err) return st, fmt.Errorf("export %s: %w", source, err)
} }
switch format { switch format {
case "string": case "string":
@ -419,13 +420,13 @@ func (p *Pipeline) Export(ctx context.Context, op *compiler.Value) error {
Msg("exporting string") Msg("exporting string")
if err := p.out.Fill(string(contents)); err != nil { if err := p.out.Fill(string(contents)); err != nil {
return err return st, err
} }
case "json": case "json":
var o interface{} var o interface{}
o, err := unmarshalAnything(contents, json.Unmarshal) o, err := unmarshalAnything(contents, json.Unmarshal)
if err != nil { if err != nil {
return err return st, err
} }
log. log.
@ -435,13 +436,13 @@ func (p *Pipeline) Export(ctx context.Context, op *compiler.Value) error {
Msg("exporting json") Msg("exporting json")
if err := p.out.Fill(o); err != nil { if err := p.out.Fill(o); err != nil {
return err return st, err
} }
case "yaml": case "yaml":
var o interface{} var o interface{}
o, err := unmarshalAnything(contents, yaml.Unmarshal) o, err := unmarshalAnything(contents, yaml.Unmarshal)
if err != nil { if err != nil {
return err return st, err
} }
log. log.
@ -451,12 +452,12 @@ func (p *Pipeline) Export(ctx context.Context, op *compiler.Value) error {
Msg("exporting yaml") Msg("exporting yaml")
if err := p.out.Fill(o); err != nil { if err := p.out.Fill(o); err != nil {
return err return st, err
} }
default: default:
return fmt.Errorf("unsupported export format: %q", format) return st, fmt.Errorf("unsupported export format: %q", format)
} }
return nil return st, nil
} }
type unmarshaller func([]byte, interface{}) error type unmarshaller func([]byte, interface{}) error
@ -478,31 +479,29 @@ func unmarshalAnything(data []byte, fn unmarshaller) (interface{}, error) {
return o, err return o, err
} }
func (p *Pipeline) Load(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Load(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
// Execute 'from' in a tmp pipeline, and use the resulting fs // Execute 'from' in a tmp pipeline, and use the resulting fs
from := p.Tmp(op.Get("from").Path().String()) from := NewPipeline(op.Get("from").Path().String(), p.s, nil)
if err := from.Do(ctx, op.Get("from")); err != nil { if err := from.Do(ctx, op.Get("from")); err != nil {
return err return st, err
} }
return from.Result().ToState()
p.fs = p.fs.Set(from.FS().LLB())
return nil
} }
func (p *Pipeline) FetchContainer(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) FetchContainer(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
rawRef, err := op.Get("ref").String() rawRef, err := op.Get("ref").String()
if err != nil { if err != nil {
return err return st, err
} }
ref, err := reference.ParseNormalizedNamed(rawRef) ref, err := reference.ParseNormalizedNamed(rawRef)
if err != nil { if err != nil {
return fmt.Errorf("failed to parse ref %s: %w", rawRef, err) return st, fmt.Errorf("failed to parse ref %s: %w", rawRef, err)
} }
// Add the default tag "latest" to a reference if it only has a repo name. // Add the default tag "latest" to a reference if it only has a repo name.
ref = reference.TagNameOnly(ref) ref = reference.TagNameOnly(ref)
state := llb.Image( st = llb.Image(
ref.String(), ref.String(),
llb.WithCustomName(p.vertexNamef("FetchContainer %s", rawRef)), llb.WithCustomName(p.vertexNamef("FetchContainer %s", rawRef)),
) )
@ -514,21 +513,20 @@ func (p *Pipeline) FetchContainer(ctx context.Context, op *compiler.Value) error
LogName: p.vertexNamef("load metadata for %s", ref.String()), LogName: p.vertexNamef("load metadata for %s", ref.String()),
}) })
if err != nil { if err != nil {
return err return st, err
} }
for _, env := range image.Config.Env { for _, env := range image.Config.Env {
k, v := parseKeyValue(env) k, v := parseKeyValue(env)
state = state.AddEnv(k, v) st = st.AddEnv(k, v)
} }
if image.Config.WorkingDir != "" { if image.Config.WorkingDir != "" {
state = state.Dir(image.Config.WorkingDir) st = st.Dir(image.Config.WorkingDir)
} }
if image.Config.User != "" { if image.Config.User != "" {
state = state.User(image.Config.User) st = st.User(image.Config.User)
} }
p.fs = p.fs.Set(state) return st, nil
return nil
} }
func parseKeyValue(env string) (string, string) { func parseKeyValue(env string) (string, string) {
@ -541,22 +539,52 @@ func parseKeyValue(env string) (string, string) {
return parts[0], v return parts[0], v
} }
func (p *Pipeline) FetchGit(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) PushContainer(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
rawRef, err := op.Get("ref").String()
if err != nil {
return st, err
}
ref, err := reference.ParseNormalizedNamed(rawRef)
if err != nil {
return st, fmt.Errorf("failed to parse ref %s: %w", rawRef, err)
}
// Add the default tag "latest" to a reference if it only has a repo name.
ref = reference.TagNameOnly(ref)
pushSt, err := p.Result().ToState()
if err != nil {
return st, err
}
_, err = p.s.Export(ctx, pushSt, bk.ExportEntry{
Type: bk.ExporterImage,
Attrs: map[string]string{
"name": ref.String(),
"push": "true",
},
})
return st, err
}
func (p *Pipeline) FetchGit(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
remote, err := op.Get("remote").String() remote, err := op.Get("remote").String()
if err != nil { if err != nil {
return err return st, err
} }
ref, err := op.Get("ref").String() ref, err := op.Get("ref").String()
if err != nil { if err != nil {
return err return st, err
} }
p.fs = p.fs.Set( return llb.Git(
llb.Git(remote, ref, llb.WithCustomName(p.vertexNamef("FetchGit %s@%s", remote, ref))), remote,
) ref,
return nil llb.WithCustomName(p.vertexNamef("FetchGit %s@%s", remote, ref)),
), nil
} }
func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
var ( var (
context = op.Lookup("context") context = op.Lookup("context")
dockerfile = op.Lookup("dockerfile") dockerfile = op.Lookup("dockerfile")
@ -568,19 +596,23 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
) )
if !context.Exists() && !dockerfile.Exists() { if !context.Exists() && !dockerfile.Exists() {
return errors.New("context or dockerfile required") return st, errors.New("context or dockerfile required")
} }
// docker build context. This can come from another component, so we need to // docker build context. This can come from another component, so we need to
// compute it first. // compute it first.
if context.Exists() { if context.Exists() {
from := p.Tmp(op.Lookup("context").Path().String()) from := NewPipeline(op.Lookup("context").Path().String(), p.s, nil)
if err := from.Do(ctx, context); err != nil { if err := from.Do(ctx, context); err != nil {
return err return st, err
} }
contextDef, err = from.FS().Def(ctx) fromResult, err := from.Result().ToState()
if err != nil { if err != nil {
return err return st, err
}
contextDef, err = p.s.Marshal(ctx, fromResult)
if err != nil {
return st, err
} }
dockerfileDef = contextDef dockerfileDef = contextDef
} }
@ -589,15 +621,15 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
if dockerfile.Exists() { if dockerfile.Exists() {
content, err := dockerfile.String() content, err := dockerfile.String()
if err != nil { if err != nil {
return err return st, err
} }
dockerfileDef, err = p.s.Scratch().Set( dockerfileDef, err = p.s.Marshal(ctx,
llb.Scratch().File( llb.Scratch().File(
llb.Mkfile("/Dockerfile", 0644, []byte(content)), llb.Mkfile("/Dockerfile", 0644, []byte(content)),
), ),
).Def(ctx) )
if err != nil { if err != nil {
return err return st, err
} }
if contextDef == nil { if contextDef == nil {
contextDef = dockerfileDef contextDef = dockerfileDef
@ -616,7 +648,7 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
if dockerfilePath := op.Lookup("dockerfilePath"); dockerfilePath.Exists() { if dockerfilePath := op.Lookup("dockerfilePath"); dockerfilePath.Exists() {
filename, err := dockerfilePath.String() filename, err := dockerfilePath.String()
if err != nil { if err != nil {
return err return st, err
} }
req.FrontendOpt["filename"] = filename req.FrontendOpt["filename"] = filename
} }
@ -631,7 +663,7 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
return nil return nil
}) })
if err != nil { if err != nil {
return err return st, err
} }
} }
@ -645,7 +677,7 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
return nil return nil
}) })
if err != nil { if err != nil {
return err return st, err
} }
} }
@ -653,13 +685,13 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
p := []string{} p := []string{}
list, err := platforms.List() list, err := platforms.List()
if err != nil { if err != nil {
return err return st, err
} }
for _, platform := range list { for _, platform := range list {
s, err := platform.String() s, err := platform.String()
if err != nil { if err != nil {
return err return st, err
} }
p = append(p, s) p = append(p, s)
} }
@ -674,65 +706,51 @@ func (p *Pipeline) DockerBuild(ctx context.Context, op *compiler.Value) error {
res, err := p.s.SolveRequest(ctx, req) res, err := p.s.SolveRequest(ctx, req)
if err != nil { if err != nil {
return err return st, err
} }
st, err := res.ToState() return res.ToState()
if err != nil {
return err
}
p.fs = p.fs.Set(st)
return nil
} }
func (p *Pipeline) WriteFile(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) WriteFile(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
content, err := op.Get("content").String() content, err := op.Get("content").String()
if err != nil { if err != nil {
return err return st, err
} }
dest, err := op.Get("dest").String() dest, err := op.Get("dest").String()
if err != nil { if err != nil {
return err return st, err
} }
mode, err := op.Get("mode").Int64() mode, err := op.Get("mode").Int64()
if err != nil { if err != nil {
return err return st, err
} }
p.fs = p.fs.Change(func(st llb.State) llb.State { return st.File(
return st.File( llb.Mkfile(dest, fs.FileMode(mode), []byte(content)),
llb.Mkfile(dest, fs.FileMode(mode), []byte(content)), llb.WithCustomName(p.vertexNamef("WriteFile %s", dest)),
llb.WithCustomName(p.vertexNamef("WriteFile %s", dest)), ), nil
)
})
return nil
} }
func (p *Pipeline) Mkdir(ctx context.Context, op *compiler.Value) error { func (p *Pipeline) Mkdir(ctx context.Context, op *compiler.Value, st llb.State) (llb.State, error) {
path, err := op.Get("path").String() path, err := op.Get("path").String()
if err != nil { if err != nil {
return err return st, err
} }
dir, err := op.Get("dir").String() dir, err := op.Get("dir").String()
if err != nil { if err != nil {
return err return st, err
} }
mode, err := op.Get("mode").Int64() mode, err := op.Get("mode").Int64()
if err != nil { if err != nil {
return err return st, err
} }
p.fs = p.fs.Change(func(st llb.State) llb.State { return st.Dir(dir).File(
return st.Dir(dir).File( llb.Mkdir(path, fs.FileMode(mode)),
llb.Mkdir(path, fs.FileMode(mode)), llb.WithCustomName(p.vertexNamef("Mkdir %s", path)),
llb.WithCustomName(p.vertexNamef("Mkdir %s", path)), ), nil
)
})
return nil
} }

View File

@ -3,41 +3,46 @@ package dagger
import ( import (
"context" "context"
"encoding/json" "encoding/json"
"errors"
"fmt" "fmt"
"strings"
bk "github.com/moby/buildkit/client"
"github.com/moby/buildkit/client/llb" "github.com/moby/buildkit/client/llb"
"github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb" "github.com/moby/buildkit/frontend/dockerfile/dockerfile2llb"
bkgw "github.com/moby/buildkit/frontend/gateway/client" bkgw "github.com/moby/buildkit/frontend/gateway/client"
"github.com/moby/buildkit/session"
"github.com/moby/buildkit/session/auth/authprovider"
bkpb "github.com/moby/buildkit/solver/pb" bkpb "github.com/moby/buildkit/solver/pb"
"github.com/opencontainers/go-digest" "github.com/opencontainers/go-digest"
"github.com/rs/zerolog/log" "github.com/rs/zerolog/log"
) )
// Polyfill for buildkit gateway client
// Use instead of bkgw.Client
type Solver struct { type Solver struct {
c bkgw.Client events chan *bk.SolveStatus
control *bk.Client
gw bkgw.Client
} }
func NewSolver(c bkgw.Client) Solver { func NewSolver(control *bk.Client, gw bkgw.Client, events chan *bk.SolveStatus) Solver {
return Solver{ return Solver{
c: c, events: events,
control: control,
gw: gw,
} }
} }
func (s Solver) FS(input llb.State) FS { func (s Solver) Marshal(ctx context.Context, st llb.State) (*bkpb.Definition, error) {
return FS{ // FIXME: do not hardcode the platform
s: s, def, err := st.Marshal(ctx, llb.LinuxAmd64)
input: input, if err != nil {
return nil, err
} }
} return def.ToPB(), nil
func (s Solver) Scratch() FS {
return s.FS(llb.Scratch())
} }
func (s Solver) SessionID() string { func (s Solver) SessionID() string {
return s.c.BuildOpts().SessionID return s.gw.BuildOpts().SessionID
} }
func (s Solver) ResolveImageConfig(ctx context.Context, ref string, opts llb.ResolveImageConfigOpt) (dockerfile2llb.Image, error) { func (s Solver) ResolveImageConfig(ctx context.Context, ref string, opts llb.ResolveImageConfigOpt) (dockerfile2llb.Image, error) {
@ -46,7 +51,7 @@ func (s Solver) ResolveImageConfig(ctx context.Context, ref string, opts llb.Res
// Load image metadata and convert to to LLB. // Load image metadata and convert to to LLB.
// Inspired by https://github.com/moby/buildkit/blob/master/frontend/dockerfile/dockerfile2llb/convert.go // Inspired by https://github.com/moby/buildkit/blob/master/frontend/dockerfile/dockerfile2llb/convert.go
// FIXME: this needs to handle platform // FIXME: this needs to handle platform
_, meta, err := s.c.ResolveImageConfig(ctx, ref, opts) _, meta, err := s.gw.ResolveImageConfig(ctx, ref, opts)
if err != nil { if err != nil {
return image, err return image, err
} }
@ -60,7 +65,7 @@ func (s Solver) ResolveImageConfig(ctx context.Context, ref string, opts llb.Res
// Solve will block until the state is solved and returns a Reference. // Solve will block until the state is solved and returns a Reference.
func (s Solver) SolveRequest(ctx context.Context, req bkgw.SolveRequest) (bkgw.Reference, error) { func (s Solver) SolveRequest(ctx context.Context, req bkgw.SolveRequest) (bkgw.Reference, error) {
// call solve // call solve
res, err := s.c.Solve(ctx, req) res, err := s.gw.Solve(ctx, req)
if err != nil { if err != nil {
return nil, bkCleanError(err) return nil, bkCleanError(err)
} }
@ -71,7 +76,7 @@ func (s Solver) SolveRequest(ctx context.Context, req bkgw.SolveRequest) (bkgw.R
// Solve will block until the state is solved and returns a Reference. // Solve will block until the state is solved and returns a Reference.
func (s Solver) Solve(ctx context.Context, st llb.State) (bkgw.Reference, error) { func (s Solver) Solve(ctx context.Context, st llb.State) (bkgw.Reference, error) {
// marshal llb // marshal llb
def, err := st.Marshal(ctx, llb.LinuxAmd64) def, err := s.Marshal(ctx, st)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -89,7 +94,7 @@ func (s Solver) Solve(ctx context.Context, st llb.State) (bkgw.Reference, error)
// call solve // call solve
return s.SolveRequest(ctx, bkgw.SolveRequest{ return s.SolveRequest(ctx, bkgw.SolveRequest{
Definition: def.ToPB(), Definition: def,
// makes Solve() to block until LLB graph is solved. otherwise it will // makes Solve() to block until LLB graph is solved. otherwise it will
// return result (that you can for example use for next build) that // return result (that you can for example use for next build) that
@ -98,13 +103,47 @@ func (s Solver) Solve(ctx context.Context, st llb.State) (bkgw.Reference, error)
}) })
} }
// Export will export `st` to `output`
// FIXME: this is currently impleneted as a hack, starting a new Build session
// within buildkit from the Control API. Ideally the Gateway API should allow to
// Export directly.
func (s Solver) Export(ctx context.Context, st llb.State, output bk.ExportEntry) (*bk.SolveResponse, error) {
def, err := s.Marshal(ctx, st)
if err != nil {
return nil, err
}
opts := bk.SolveOpt{
Exports: []bk.ExportEntry{output},
Session: []session.Attachable{
authprovider.NewDockerAuthProvider(log.Ctx(ctx)),
},
}
ch := make(chan *bk.SolveStatus)
// Forward this build session events to the main events channel, for logging
// purposes.
go func() {
for event := range ch {
s.events <- event
}
}()
return s.control.Build(ctx, opts, "", func(ctx context.Context, c bkgw.Client) (*bkgw.Result, error) {
return c.Solve(ctx, bkgw.SolveRequest{
Definition: def,
})
}, ch)
}
type llbOp struct { type llbOp struct {
Op bkpb.Op Op bkpb.Op
Digest digest.Digest Digest digest.Digest
OpMetadata bkpb.OpMetadata OpMetadata bkpb.OpMetadata
} }
func dumpLLB(def *llb.Definition) ([]byte, error) { func dumpLLB(def *bkpb.Definition) ([]byte, error) {
ops := make([]llbOp, 0, len(def.Def)) ops := make([]llbOp, 0, len(def.Def))
for _, dt := range def.Def { for _, dt := range def.Def {
var op bkpb.Op var op bkpb.Op
@ -117,3 +156,22 @@ func dumpLLB(def *llb.Definition) ([]byte, error) {
} }
return json.Marshal(ops) return json.Marshal(ops)
} }
// A helper to remove noise from buildkit error messages.
// FIXME: Obviously a cleaner solution would be nice.
func bkCleanError(err error) error {
noise := []string{
"executor failed running ",
"buildkit-runc did not terminate successfully",
"rpc error: code = Unknown desc = ",
"failed to solve: ",
}
msg := err.Error()
for _, s := range noise {
msg = strings.ReplaceAll(msg, s, "")
}
return errors.New(msg)
}

1
go.sum
View File

@ -276,6 +276,7 @@ github.com/docker/docker v1.4.2-0.20190924003213-a8608b5b67c7/go.mod h1:eEKB0N0r
github.com/docker/docker v17.12.0-ce-rc1.0.20200730172259-9f28837c1d93+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/docker v17.12.0-ce-rc1.0.20200730172259-9f28837c1d93+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk=
github.com/docker/docker v20.10.0-beta1.0.20201110211921-af34b94a78a1+incompatible h1:J2OhsbfqoBRRT048iD/tqXBvEQWQATQ8vew6LqQmDSU= github.com/docker/docker v20.10.0-beta1.0.20201110211921-af34b94a78a1+incompatible h1:J2OhsbfqoBRRT048iD/tqXBvEQWQATQ8vew6LqQmDSU=
github.com/docker/docker v20.10.0-beta1.0.20201110211921-af34b94a78a1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/docker v20.10.0-beta1.0.20201110211921-af34b94a78a1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk=
github.com/docker/docker-credential-helpers v0.6.3 h1:zI2p9+1NQYdnG6sMU26EX4aVGlqbInSQxQXLvzJ4RPQ=
github.com/docker/docker-credential-helpers v0.6.3/go.mod h1:WRaJzqw3CTB9bk10avuGsjVBZsD05qeibJ1/TYlvc0Y= github.com/docker/docker-credential-helpers v0.6.3/go.mod h1:WRaJzqw3CTB9bk10avuGsjVBZsD05qeibJ1/TYlvc0Y=
github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ=
github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec=

View File

@ -52,6 +52,11 @@ package dagger
ref: string ref: string
} }
#PushContainer: {
do: "push-container"
ref: string
}
#FetchGit: { #FetchGit: {
do: "fetch-git" do: "fetch-git"
remote: string remote: string

View File

@ -2,48 +2,14 @@ package stdlib
import ( import (
"embed" "embed"
"fmt"
"io/fs"
"path" "path"
"path/filepath"
cueload "cuelang.org/go/cue/load"
) )
// FS contains the filesystem of the stdlib. var (
//go:embed **/*.cue **/*/*.cue // FS contains the filesystem of the stdlib.
var FS embed.FS //go:embed **/*.cue **/*/*.cue
FS embed.FS
const ( PackageName = "dagger.io"
stdlibPackageName = "dagger.io" Path = path.Join("cue.mod", "pkg", PackageName)
) )
func Overlay(prefixPath string) (map[string]cueload.Source, error) {
overlay := map[string]cueload.Source{}
err := fs.WalkDir(FS, ".", func(p string, entry fs.DirEntry, err error) error {
if err != nil {
return err
}
if !entry.Type().IsRegular() {
return nil
}
if filepath.Ext(entry.Name()) != ".cue" {
return nil
}
contents, err := FS.ReadFile(p)
if err != nil {
return fmt.Errorf("%s: %w", p, err)
}
overlayPath := path.Join(prefixPath, "cue.mod", "pkg", stdlibPackageName, p)
overlay[overlayPath] = cueload.FromBytes(contents)
return nil
})
return overlay, err
}