From e0ef11a4c2c6789e08b61df082b5b9aa70898e7a Mon Sep 17 00:00:00 2001 From: Tibor Vass Date: Sun, 6 Sep 2015 13:26:40 -0400 Subject: [PATCH] Abstract builder and implement server-side dockerfile builder This patch creates interfaces in builder/ for building Docker images. It is a first step in a series of patches to remove the daemon dependency on builder and later allow a client-side Dockerfile builder as well as potential builder plugins. It is needed because we cannot remove the /build API endpoint, so we need to keep the server-side Dockerfile builder, but we also want to reuse the same Dockerfile parser and evaluator for both server-side and client-side. builder/dockerfile/ and api/server/builder.go contain implementations of those interfaces as a refactoring of the current code. Signed-off-by: Tibor Vass --- api/client/build.go | 16 +- api/server/router/local/image.go | 124 ++- builder/builder.go | 139 +++ builder/dockerfile/builder.go | 292 +++++++ builder/dockerfile/dispatchers.go | 201 +++-- builder/dockerfile/evaluator.go | 253 +----- builder/dockerfile/internals.go | 823 +++++++----------- builder/dockerfile/internals_unix.go | 7 +- builder/dockerfile/internals_windows.go | 14 - builder/dockerfile/job.go | 376 -------- builder/dockerfile/support.go | 13 +- builder/dockerfile/support_test.go | 41 - builder/dockerignore.go | 47 + builder/git.go | 28 + builder/remote.go | 115 +++ .../job_test.go => remote_test.go} | 38 +- builder/tarsum.go | 165 ++++ daemon/create.go | 31 +- daemon/daemonbuilder/builder.go | 238 +++++ daemon/daemonbuilder/builder_unix.go | 40 + daemon/daemonbuilder/builder_windows.go | 8 + integration-cli/docker_api_build_test.go | 2 +- pkg/ioutils/temp_unix.go | 10 + pkg/ioutils/temp_windows.go | 18 + utils/utils.go | 14 +- utils/utils_test.go | 13 +- 26 files changed, 1723 insertions(+), 1343 deletions(-) create mode 100644 builder/builder.go create mode 100644 builder/dockerfile/builder.go delete mode 100644 builder/dockerfile/job.go delete mode 100644 builder/dockerfile/support_test.go create mode 100644 builder/dockerignore.go create mode 100644 builder/git.go create mode 100644 builder/remote.go rename builder/{dockerfile/job_test.go => remote_test.go} (81%) create mode 100644 builder/tarsum.go create mode 100644 daemon/daemonbuilder/builder.go create mode 100644 daemon/daemonbuilder/builder_unix.go create mode 100644 daemon/daemonbuilder/builder_windows.go create mode 100644 pkg/ioutils/temp_unix.go create mode 100644 pkg/ioutils/temp_windows.go diff --git a/api/client/build.go b/api/client/build.go index 965fbe696c..9408d4495f 100644 --- a/api/client/build.go +++ b/api/client/build.go @@ -12,7 +12,6 @@ import ( "net/url" "os" "os/exec" - "path" "path/filepath" "regexp" "runtime" @@ -131,13 +130,19 @@ func (cli *DockerCli) CmdBuild(args ...string) error { return fmt.Errorf("cannot canonicalize dockerfile path %s: %v", relDockerfile, err) } - var includes = []string{"."} - - excludes, err := utils.ReadDockerIgnore(path.Join(contextDir, ".dockerignore")) - if err != nil { + f, err := os.Open(filepath.Join(contextDir, ".dockerignore")) + if err != nil && !os.IsNotExist(err) { return err } + var excludes []string + if err == nil { + excludes, err = utils.ReadDockerIgnore(f) + if err != nil { + return err + } + } + if err := utils.ValidateContextDirectory(contextDir, excludes); err != nil { return fmt.Errorf("Error checking context: '%s'.", err) } @@ -149,6 +154,7 @@ func (cli *DockerCli) CmdBuild(args ...string) error { // removed. The deamon will remove them for us, if needed, after it // parses the Dockerfile. Ignore errors here, as they will have been // caught by ValidateContextDirectory above. + var includes = []string{"."} keepThem1, _ := fileutils.Matches(".dockerignore", excludes) keepThem2, _ := fileutils.Matches(relDockerfile, excludes) if keepThem1 || keepThem2 { diff --git a/api/server/router/local/image.go b/api/server/router/local/image.go index 331e959344..bd1aac4e49 100644 --- a/api/server/router/local/image.go +++ b/api/server/router/local/image.go @@ -12,13 +12,18 @@ import ( "github.com/Sirupsen/logrus" "github.com/docker/docker/api/server/httputils" "github.com/docker/docker/api/types" + "github.com/docker/docker/builder" "github.com/docker/docker/builder/dockerfile" "github.com/docker/docker/cliconfig" + "github.com/docker/docker/daemon/daemonbuilder" "github.com/docker/docker/graph" + "github.com/docker/docker/graph/tags" "github.com/docker/docker/pkg/ioutils" "github.com/docker/docker/pkg/parsers" + "github.com/docker/docker/pkg/progressreader" "github.com/docker/docker/pkg/streamformatter" "github.com/docker/docker/pkg/ulimit" + "github.com/docker/docker/registry" "github.com/docker/docker/runconfig" "github.com/docker/docker/utils" "golang.org/x/net/context" @@ -56,13 +61,18 @@ func (s *router) postCommit(ctx context.Context, w http.ResponseWriter, r *http. Config: c, } - imgID, err := dockerfile.Commit(cname, s.daemon, commitCfg) + container, err := s.daemon.Get(cname) + if err != nil { + return err + } + + imgID, err := dockerfile.Commit(container, s.daemon, commitCfg) if err != nil { return err } return httputils.WriteJSON(w, http.StatusCreated, &types.ContainerCommitResponse{ - ID: imgID, + ID: string(imgID), }) } @@ -125,7 +135,7 @@ func (s *router) postImagesCreate(ctx context.Context, w http.ResponseWriter, r // generated from the download to be available to the output // stream processing below var newConfig *runconfig.Config - newConfig, err = dockerfile.BuildFromConfig(s.daemon, &runconfig.Config{}, r.Form["changes"]) + newConfig, err = dockerfile.BuildFromConfig(&runconfig.Config{}, r.Form["changes"]) if err != nil { return err } @@ -269,7 +279,7 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R var ( authConfigs = map[string]cliconfig.AuthConfig{} authConfigsEncoded = r.Header.Get("X-Registry-Config") - buildConfig = dockerfile.NewBuildConfig() + buildConfig = &dockerfile.Config{} ) if authConfigsEncoded != "" { @@ -284,6 +294,21 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R w.Header().Set("Content-Type", "application/json") version := httputils.VersionFromContext(ctx) + output := ioutils.NewWriteFlusher(w) + sf := streamformatter.NewJSONStreamFormatter() + errf := func(err error) error { + // Do not write the error in the http output if it's still empty. + // This prevents from writing a 200(OK) when there is an interal error. + if !output.Flushed() { + return err + } + _, err = w.Write(sf.FormatError(errors.New(utils.GetErrorMessage(err)))) + if err != nil { + logrus.Warnf("could not write error response: %v", err) + } + return nil + } + if httputils.BoolValue(r, "forcerm") && version.GreaterThanOrEqualTo("1.12") { buildConfig.Remove = true } else if r.FormValue("rm") == "" && version.GreaterThanOrEqualTo("1.12") { @@ -295,17 +320,22 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R buildConfig.Pull = true } - output := ioutils.NewWriteFlusher(w) - buildConfig.Stdout = output - buildConfig.Context = r.Body + repoName, tag := parsers.ParseRepositoryTag(r.FormValue("t")) + if repoName != "" { + if err := registry.ValidateRepositoryName(repoName); err != nil { + return errf(err) + } + if len(tag) > 0 { + if err := tags.ValidateTagName(tag); err != nil { + return errf(err) + } + } + } - buildConfig.RemoteURL = r.FormValue("remote") buildConfig.DockerfileName = r.FormValue("dockerfile") - buildConfig.RepoName = r.FormValue("t") - buildConfig.SuppressOutput = httputils.BoolValue(r, "q") - buildConfig.NoCache = httputils.BoolValue(r, "nocache") + buildConfig.Verbose = !httputils.BoolValue(r, "q") + buildConfig.UseCache = !httputils.BoolValue(r, "nocache") buildConfig.ForceRemove = httputils.BoolValue(r, "forcerm") - buildConfig.AuthConfigs = authConfigs buildConfig.MemorySwap = httputils.Int64ValueOrZero(r, "memswap") buildConfig.Memory = httputils.Int64ValueOrZero(r, "memory") buildConfig.CPUShares = httputils.Int64ValueOrZero(r, "cpushares") @@ -319,7 +349,7 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R ulimitsJSON := r.FormValue("ulimits") if ulimitsJSON != "" { if err := json.NewDecoder(strings.NewReader(ulimitsJSON)).Decode(&buildUlimits); err != nil { - return err + return errf(err) } buildConfig.Ulimits = buildUlimits } @@ -328,12 +358,50 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R buildArgsJSON := r.FormValue("buildargs") if buildArgsJSON != "" { if err := json.NewDecoder(strings.NewReader(buildArgsJSON)).Decode(&buildArgs); err != nil { - return err + return errf(err) } + buildConfig.BuildArgs = buildArgs } - buildConfig.BuildArgs = buildArgs - // Job cancellation. Note: not all job types support this. + remoteURL := r.FormValue("remote") + + // Currently, only used if context is from a remote url. + // The field `In` is set by DetectContextFromRemoteURL. + // Look at code in DetectContextFromRemoteURL for more information. + pReader := &progressreader.Config{ + // TODO: make progressreader streamformatter-agnostic + Out: output, + Formatter: sf, + Size: r.ContentLength, + NewLines: true, + ID: "Downloading context", + Action: remoteURL, + } + + var ( + context builder.ModifiableContext + dockerfileName string + err error + ) + context, dockerfileName, err = daemonbuilder.DetectContextFromRemoteURL(r.Body, remoteURL, pReader) + if err != nil { + return errf(err) + } + defer func() { + if err := context.Close(); err != nil { + logrus.Debugf("[BUILDER] failed to remove temporary context: %v", err) + } + }() + + docker := daemonbuilder.Docker{s.daemon, output, authConfigs} + + b, err := dockerfile.NewBuilder(buildConfig, docker, builder.DockerIgnoreContext{context}, nil) + if err != nil { + return errf(err) + } + b.Stdout = &streamformatter.StdoutFormatter{Writer: output, StreamFormatter: sf} + b.Stderr = &streamformatter.StderrFormatter{Writer: output, StreamFormatter: sf} + if closeNotifier, ok := w.(http.CloseNotifier); ok { finished := make(chan struct{}) defer close(finished) @@ -342,20 +410,26 @@ func (s *router) postBuild(ctx context.Context, w http.ResponseWriter, r *http.R case <-finished: case <-closeNotifier.CloseNotify(): logrus.Infof("Client disconnected, cancelling job: build") - buildConfig.Cancel() + b.Cancel() } }() } - if err := dockerfile.Build(s.daemon, buildConfig); err != nil { - // Do not write the error in the http output if it's still empty. - // This prevents from writing a 200(OK) when there is an interal error. - if !output.Flushed() { - return err - } - sf := streamformatter.NewJSONStreamFormatter() - w.Write(sf.FormatError(errors.New(utils.GetErrorMessage(err)))) + if len(dockerfileName) > 0 { + b.DockerfileName = dockerfileName } + + imgID, err := b.Build() + if err != nil { + return errf(err) + } + + if repoName != "" { + if err := s.daemon.Repositories().Tag(repoName, tag, string(imgID), true); err != nil { + return errf(err) + } + } + return nil } diff --git a/builder/builder.go b/builder/builder.go new file mode 100644 index 0000000000..f693aef9ba --- /dev/null +++ b/builder/builder.go @@ -0,0 +1,139 @@ +// Package builder defines interfaces for any Docker builder to implement. +// +// Historically, only server-side Dockerfile interpreters existed. +// This package allows for other implementations of Docker builders. +package builder + +import ( + "io" + "os" + + // TODO: remove dependency on daemon + "github.com/docker/docker/daemon" + "github.com/docker/docker/image" + "github.com/docker/docker/runconfig" +) + +// Builder abstracts a Docker builder whose only purpose is to build a Docker image referenced by an imageID. +type Builder interface { + // Build builds a Docker image referenced by an imageID string. + // + // Note: Tagging an image should not be done by a Builder, it should instead be done + // by the caller. + // + // TODO: make this return a reference instead of string + Build() (imageID string) +} + +// Context represents a file system tree. +type Context interface { + // Close allows to signal that the filesystem tree won't be used anymore. + // For Context implementations using a temporary directory, it is recommended to + // delete the temporary directory in Close(). + Close() error + // Stat returns an entry corresponding to path if any. + // It is recommended to return an error if path was not found. + Stat(path string) (FileInfo, error) + // Open opens path from the context and returns a readable stream of it. + Open(path string) (io.ReadCloser, error) + // Walk walks the tree of the context with the function passed to it. + Walk(root string, walkFn WalkFunc) error +} + +// WalkFunc is the type of the function called for each file or directory visited by Context.Walk(). +type WalkFunc func(path string, fi FileInfo, err error) error + +// ModifiableContext represents a modifiable Context. +// TODO: remove this interface once we can get rid of Remove() +type ModifiableContext interface { + Context + // Remove deletes the entry specified by `path`. + // It is usual for directory entries to delete all its subentries. + Remove(path string) error +} + +// FileInfo extends os.FileInfo to allow retrieving an absolute path to the file. +// TODO: remove this interface once pkg/archive exposes a walk function that Context can use. +type FileInfo interface { + os.FileInfo + Path() string +} + +// PathFileInfo is a convenience struct that implements the FileInfo interface. +type PathFileInfo struct { + os.FileInfo + // FilePath holds the absolute path to the file. + FilePath string +} + +// Path returns the absolute path to the file. +func (fi PathFileInfo) Path() string { + return fi.FilePath +} + +// Hashed defines an extra method intended for implementations of os.FileInfo. +type Hashed interface { + // Hash returns the hash of a file. + Hash() string + SetHash(string) +} + +// HashedFileInfo is a convenient struct that augments FileInfo with a field. +type HashedFileInfo struct { + FileInfo + // FileHash represents the hash of a file. + FileHash string +} + +// Hash returns the hash of a file. +func (fi HashedFileInfo) Hash() string { + return fi.FileHash +} + +// SetHash sets the hash of a file. +func (fi *HashedFileInfo) SetHash(h string) { + fi.FileHash = h +} + +// Docker abstracts calls to a Docker Daemon. +type Docker interface { + // TODO: use digest reference instead of name + + // LookupImage looks up a Docker image referenced by `name`. + LookupImage(name string) (*image.Image, error) + // Pull tells Docker to pull image referenced by `name`. + Pull(name string) (*image.Image, error) + + // TODO: move daemon.Container to its own package + + // Container looks up a Docker container referenced by `id`. + Container(id string) (*daemon.Container, error) + // Create creates a new Docker container and returns potential warnings + // TODO: put warnings in the error + Create(*runconfig.Config, *runconfig.HostConfig) (*daemon.Container, []string, error) + // Remove removes a container specified by `id`. + Remove(id string, cfg *daemon.ContainerRmConfig) error + // Commit creates a new Docker image from an existing Docker container. + Commit(*daemon.Container, *daemon.ContainerCommitConfig) (*image.Image, error) + // Copy copies/extracts a source FileInfo to a destination path inside a container + // specified by a container object. + // TODO: make an Extract method instead of passing `decompress` + // TODO: do not pass a FileInfo, instead refactor the archive package to export a Walk function that can be used + // with Context.Walk + Copy(c *daemon.Container, destPath string, src FileInfo, decompress bool) error + + // Retain retains an image avoiding it to be removed or overwritten until a corresponding Release() call. + // TODO: remove + Retain(sessionID, imgID string) + // Release releases a list of images that were retained for the time of a build. + // TODO: remove + Release(sessionID string, activeImages []string) +} + +// ImageCache abstracts an image cache store. +// (parent image, child runconfig) -> child image +type ImageCache interface { + // GetCachedImage returns a reference to a cached image whose parent equals `parent` + // and runconfig equals `cfg`. A cache miss is expected to return an empty ID and a nil error. + GetCachedImage(parentID string, cfg *runconfig.Config) (imageID string, err error) +} diff --git a/builder/dockerfile/builder.go b/builder/dockerfile/builder.go new file mode 100644 index 0000000000..8b8c6e8735 --- /dev/null +++ b/builder/dockerfile/builder.go @@ -0,0 +1,292 @@ +package dockerfile + +import ( + "bytes" + "fmt" + "io" + "io/ioutil" + "os" + "runtime" + "strings" + "sync" + + "github.com/Sirupsen/logrus" + "github.com/docker/docker/builder" + "github.com/docker/docker/builder/dockerfile/parser" + "github.com/docker/docker/daemon" + "github.com/docker/docker/pkg/stringid" + "github.com/docker/docker/pkg/ulimit" + "github.com/docker/docker/runconfig" +) + +var validCommitCommands = map[string]bool{ + "cmd": true, + "entrypoint": true, + "env": true, + "expose": true, + "label": true, + "onbuild": true, + "user": true, + "volume": true, + "workdir": true, +} + +// BuiltinAllowedBuildArgs is list of built-in allowed build args +var BuiltinAllowedBuildArgs = map[string]bool{ + "HTTP_PROXY": true, + "http_proxy": true, + "HTTPS_PROXY": true, + "https_proxy": true, + "FTP_PROXY": true, + "ftp_proxy": true, + "NO_PROXY": true, + "no_proxy": true, +} + +// Config constitutes the configuration for a Dockerfile builder. +type Config struct { + // only used if Dockerfile has to be extracted from Context + DockerfileName string + + Verbose bool + UseCache bool + Remove bool + ForceRemove bool + Pull bool + BuildArgs map[string]string // build-time args received in build context for expansion/substitution and commands in 'run'. + + // resource constraints + // TODO: factor out to be reused with Run ? + + Memory int64 + MemorySwap int64 + CPUShares int64 + CPUPeriod int64 + CPUQuota int64 + CPUSetCpus string + CPUSetMems string + CgroupParent string + Ulimits []*ulimit.Ulimit +} + +// Builder is a Dockerfile builder +// It implements the builder.Builder interface. +type Builder struct { + *Config + + Stdout io.Writer + Stderr io.Writer + + docker builder.Docker + context builder.Context + + dockerfile *parser.Node + runConfig *runconfig.Config // runconfig for cmd, run, entrypoint etc. + flags *BFlags + tmpContainers map[string]struct{} + image string // imageID + noBaseImage bool + maintainer string + cmdSet bool + disableCommit bool + cacheBusted bool + cancelled chan struct{} + cancelOnce sync.Once + allowedBuildArgs map[string]bool // list of build-time args that are allowed for expansion/substitution and passing to commands in 'run'. + + // TODO: remove once docker.Commit can receive a tag + id string + activeImages []string +} + +// NewBuilder creates a new Dockerfile builder from an optional dockerfile and a Config. +// If dockerfile is nil, the Dockerfile specified by Config.DockerfileName, +// will be read from the Context passed to Build(). +func NewBuilder(config *Config, docker builder.Docker, context builder.Context, dockerfile io.ReadCloser) (b *Builder, err error) { + if config == nil { + config = new(Config) + } + if config.BuildArgs == nil { + config.BuildArgs = make(map[string]string) + } + b = &Builder{ + Config: config, + Stdout: os.Stdout, + Stderr: os.Stderr, + docker: docker, + context: context, + runConfig: new(runconfig.Config), + tmpContainers: map[string]struct{}{}, + cancelled: make(chan struct{}), + id: stringid.GenerateNonCryptoID(), + allowedBuildArgs: make(map[string]bool), + } + if dockerfile != nil { + b.dockerfile, err = parser.Parse(dockerfile) + if err != nil { + return nil, err + } + } + + return b, nil +} + +// Build runs the Dockerfile builder from a context and a docker object that allows to make calls +// to Docker. +// +// This will (barring errors): +// +// * read the dockerfile from context +// * parse the dockerfile if not already parsed +// * walk the AST and execute it by dispatching to handlers. If Remove +// or ForceRemove is set, additional cleanup around containers happens after +// processing. +// * Print a happy message and return the image ID. +// * NOT tag the image, that is responsibility of the caller. +// +func (b *Builder) Build() (string, error) { + // TODO: remove once b.docker.Commit can take a tag parameter. + defer func() { + b.docker.Release(b.id, b.activeImages) + }() + + // If Dockerfile was not parsed yet, extract it from the Context + if b.dockerfile == nil { + if err := b.readDockerfile(); err != nil { + return "", err + } + } + + var shortImgID string + for i, n := range b.dockerfile.Children { + select { + case <-b.cancelled: + logrus.Debug("Builder: build cancelled!") + fmt.Fprintf(b.Stdout, "Build cancelled") + return "", fmt.Errorf("Build cancelled") + default: + // Not cancelled yet, keep going... + } + if err := b.dispatch(i, n); err != nil { + if b.ForceRemove { + b.clearTmp() + } + return "", err + } + shortImgID = stringid.TruncateID(b.image) + fmt.Fprintf(b.Stdout, " ---> %s\n", shortImgID) + if b.Remove { + b.clearTmp() + } + } + + // check if there are any leftover build-args that were passed but not + // consumed during build. Return an error, if there are any. + leftoverArgs := []string{} + for arg := range b.BuildArgs { + if !b.isBuildArgAllowed(arg) { + leftoverArgs = append(leftoverArgs, arg) + } + } + if len(leftoverArgs) > 0 { + return "", fmt.Errorf("One or more build-args %v were not consumed, failing build.", leftoverArgs) + } + + if b.image == "" { + return "", fmt.Errorf("No image was generated. Is your Dockerfile empty?") + } + + fmt.Fprintf(b.Stdout, "Successfully built %s\n", shortImgID) + return b.image, nil +} + +// Cancel cancels an ongoing Dockerfile build. +func (b *Builder) Cancel() { + b.cancelOnce.Do(func() { + close(b.cancelled) + }) +} + +// CommitConfig contains build configs for commit operation +type CommitConfig struct { + Pause bool + Repo string + Tag string + Author string + Comment string + Changes []string + Config *runconfig.Config +} + +// BuildFromConfig will do build directly from parameter 'changes', which comes +// from Dockerfile entries, it will: +// - call parse.Parse() to get AST root from Dockerfile entries +// - do build by calling builder.dispatch() to call all entries' handling routines +// TODO: remove? +func BuildFromConfig(config *runconfig.Config, changes []string) (*runconfig.Config, error) { + ast, err := parser.Parse(bytes.NewBufferString(strings.Join(changes, "\n"))) + if err != nil { + return nil, err + } + + // ensure that the commands are valid + for _, n := range ast.Children { + if !validCommitCommands[n.Value] { + return nil, fmt.Errorf("%s is not a valid change command", n.Value) + } + } + + b, err := NewBuilder(nil, nil, nil, nil) + if err != nil { + return nil, err + } + b.runConfig = config + b.Stdout = ioutil.Discard + b.Stderr = ioutil.Discard + b.disableCommit = true + + for i, n := range ast.Children { + if err := b.dispatch(i, n); err != nil { + return nil, err + } + } + + return b.runConfig, nil +} + +// Commit will create a new image from a container's changes +// TODO: remove daemon, make Commit a method on *Builder ? +func Commit(container *daemon.Container, d *daemon.Daemon, c *CommitConfig) (string, error) { + // It is not possible to commit a running container on Windows + if runtime.GOOS == "windows" && container.IsRunning() { + return "", fmt.Errorf("Windows does not support commit of a running container") + } + + if c.Config == nil { + c.Config = &runconfig.Config{} + } + + newConfig, err := BuildFromConfig(c.Config, c.Changes) + if err != nil { + return "", err + } + + if err := runconfig.Merge(newConfig, container.Config); err != nil { + return "", err + } + + commitCfg := &daemon.ContainerCommitConfig{ + Pause: c.Pause, + Repo: c.Repo, + Tag: c.Tag, + Author: c.Author, + Comment: c.Comment, + Config: newConfig, + } + + img, err := d.Commit(container, commitCfg) + if err != nil { + return "", err + } + return img.ID, nil +} diff --git a/builder/dockerfile/dispatchers.go b/builder/dockerfile/dispatchers.go index a1e7396a58..750d19a7a7 100644 --- a/builder/dockerfile/dispatchers.go +++ b/builder/dockerfile/dispatchers.go @@ -19,6 +19,7 @@ import ( "github.com/Sirupsen/logrus" derr "github.com/docker/docker/errors" + "github.com/docker/docker/image" flag "github.com/docker/docker/pkg/mflag" "github.com/docker/docker/pkg/nat" "github.com/docker/docker/pkg/signal" @@ -34,7 +35,7 @@ const ( ) // dispatch with no layer / parsing. This is effectively not a command. -func nullDispatch(b *builder, args []string, attributes map[string]bool, original string) error { +func nullDispatch(b *Builder, args []string, attributes map[string]bool, original string) error { return nil } @@ -43,7 +44,7 @@ func nullDispatch(b *builder, args []string, attributes map[string]bool, origina // Sets the environment variable foo to bar, also makes interpolation // in the dockerfile available from the next statement on via ${foo}. // -func env(b *builder, args []string, attributes map[string]bool, original string) error { +func env(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) == 0 { return derr.ErrorCodeAtLeastOneArg.WithArgs("ENV") } @@ -53,7 +54,7 @@ func env(b *builder, args []string, attributes map[string]bool, original string) return derr.ErrorCodeTooManyArgs.WithArgs("ENV") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -62,10 +63,10 @@ func env(b *builder, args []string, attributes map[string]bool, original string) // context of a builder command. Will remove once we actually add // a builder command to something! /* - flBool1 := b.BuilderFlags.AddBool("bool1", false) - flStr1 := b.BuilderFlags.AddString("str1", "HI") + flBool1 := b.flags.AddBool("bool1", false) + flStr1 := b.flags.AddString("str1", "HI") - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -82,44 +83,44 @@ func env(b *builder, args []string, attributes map[string]bool, original string) commitStr += " " + newVar gotOne := false - for i, envVar := range b.Config.Env { + for i, envVar := range b.runConfig.Env { envParts := strings.SplitN(envVar, "=", 2) if envParts[0] == args[j] { - b.Config.Env[i] = newVar + b.runConfig.Env[i] = newVar gotOne = true break } } if !gotOne { - b.Config.Env = append(b.Config.Env, newVar) + b.runConfig.Env = append(b.runConfig.Env, newVar) } j++ } - return b.commit("", b.Config.Cmd, commitStr) + return b.commit("", b.runConfig.Cmd, commitStr) } // MAINTAINER some text // // Sets the maintainer metadata. -func maintainer(b *builder, args []string, attributes map[string]bool, original string) error { +func maintainer(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return derr.ErrorCodeExactlyOneArg.WithArgs("MAINTAINER") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } b.maintainer = args[0] - return b.commit("", b.Config.Cmd, fmt.Sprintf("MAINTAINER %s", b.maintainer)) + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("MAINTAINER %s", b.maintainer)) } // LABEL some json data describing the image // // Sets the Label variable foo to bar, // -func label(b *builder, args []string, attributes map[string]bool, original string) error { +func label(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) == 0 { return derr.ErrorCodeAtLeastOneArg.WithArgs("LABEL") } @@ -128,14 +129,14 @@ func label(b *builder, args []string, attributes map[string]bool, original strin return derr.ErrorCodeTooManyArgs.WithArgs("LABEL") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } commitStr := "LABEL" - if b.Config.Labels == nil { - b.Config.Labels = map[string]string{} + if b.runConfig.Labels == nil { + b.runConfig.Labels = map[string]string{} } for j := 0; j < len(args); j++ { @@ -144,10 +145,10 @@ func label(b *builder, args []string, attributes map[string]bool, original strin newVar := args[j] + "=" + args[j+1] + "" commitStr += " " + newVar - b.Config.Labels[args[j]] = args[j+1] + b.runConfig.Labels[args[j]] = args[j+1] j++ } - return b.commit("", b.Config.Cmd, commitStr) + return b.commit("", b.runConfig.Cmd, commitStr) } // ADD foo /path @@ -155,12 +156,12 @@ func label(b *builder, args []string, attributes map[string]bool, original strin // Add the file 'foo' to '/path'. Tarball and Remote URL (git, http) handling // exist here. If you do not wish to have this automatic handling, use COPY. // -func add(b *builder, args []string, attributes map[string]bool, original string) error { +func add(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) < 2 { return derr.ErrorCodeAtLeastTwoArgs.WithArgs("ADD") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -171,12 +172,12 @@ func add(b *builder, args []string, attributes map[string]bool, original string) // // Same as 'ADD' but without the tar and remote url handling. // -func dispatchCopy(b *builder, args []string, attributes map[string]bool, original string) error { +func dispatchCopy(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) < 2 { return derr.ErrorCodeAtLeastTwoArgs.WithArgs("COPY") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -187,12 +188,12 @@ func dispatchCopy(b *builder, args []string, attributes map[string]bool, origina // // This sets the image the dockerfile will build on top of. // -func from(b *builder, args []string, attributes map[string]bool, original string) error { +func from(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return derr.ErrorCodeExactlyOneArg.WithArgs("FROM") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -208,25 +209,21 @@ func from(b *builder, args []string, attributes map[string]bool, original string return nil } - image, err := b.Daemon.Repositories().LookupImage(name) - if b.Pull { - image, err = b.pullImage(name) + var ( + image *image.Image + err error + ) + // TODO: don't use `name`, instead resolve it to a digest + if !b.Pull { + image, err = b.docker.LookupImage(name) + // TODO: shouldn't we error out if error is different from "not found" ? + } + if image == nil { + image, err = b.docker.Pull(name) if err != nil { return err } } - if err != nil { - if b.Daemon.Graph().IsNotExist(err, name) { - image, err = b.pullImage(name) - } - - // note that the top level err will still be !nil here if IsNotExist is - // not the error. This approach just simplifies the logic a bit. - if err != nil { - return err - } - } - return b.processImageFrom(image) } @@ -239,12 +236,12 @@ func from(b *builder, args []string, attributes map[string]bool, original string // special cases. search for 'OnBuild' in internals.go for additional special // cases. // -func onbuild(b *builder, args []string, attributes map[string]bool, original string) error { +func onbuild(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) == 0 { return derr.ErrorCodeAtLeastOneArg.WithArgs("ONBUILD") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -258,20 +255,20 @@ func onbuild(b *builder, args []string, attributes map[string]bool, original str original = regexp.MustCompile(`(?i)^\s*ONBUILD\s*`).ReplaceAllString(original, "") - b.Config.OnBuild = append(b.Config.OnBuild, original) - return b.commit("", b.Config.Cmd, fmt.Sprintf("ONBUILD %s", original)) + b.runConfig.OnBuild = append(b.runConfig.OnBuild, original) + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("ONBUILD %s", original)) } // WORKDIR /tmp // // Set the working directory for future RUN/CMD/etc statements. // -func workdir(b *builder, args []string, attributes map[string]bool, original string) error { +func workdir(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return derr.ErrorCodeExactlyOneArg.WithArgs("WORKDIR") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -280,13 +277,13 @@ func workdir(b *builder, args []string, attributes map[string]bool, original str workdir := filepath.FromSlash(args[0]) if !system.IsAbs(workdir) { - current := filepath.FromSlash(b.Config.WorkingDir) + current := filepath.FromSlash(b.runConfig.WorkingDir) workdir = filepath.Join(string(os.PathSeparator), current, workdir) } - b.Config.WorkingDir = workdir + b.runConfig.WorkingDir = workdir - return b.commit("", b.Config.Cmd, fmt.Sprintf("WORKDIR %v", workdir)) + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("WORKDIR %v", workdir)) } // RUN some command yo @@ -299,12 +296,12 @@ func workdir(b *builder, args []string, attributes map[string]bool, original str // RUN echo hi # cmd /S /C echo hi (Windows) // RUN [ "echo", "hi" ] # echo hi // -func run(b *builder, args []string, attributes map[string]bool, original string) error { +func run(b *Builder, args []string, attributes map[string]bool, original string) error { if b.image == "" && !b.noBaseImage { return derr.ErrorCodeMissingFrom } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } @@ -328,13 +325,13 @@ func run(b *builder, args []string, attributes map[string]bool, original string) } // stash the cmd - cmd := b.Config.Cmd - runconfig.Merge(b.Config, config) + cmd := b.runConfig.Cmd + runconfig.Merge(b.runConfig, config) // stash the config environment - env := b.Config.Env + env := b.runConfig.Env - defer func(cmd *stringutils.StrSlice) { b.Config.Cmd = cmd }(cmd) - defer func(env []string) { b.Config.Env = env }(env) + defer func(cmd *stringutils.StrSlice) { b.runConfig.Cmd = cmd }(cmd) + defer func(env []string) { b.runConfig.Env = env }(env) // derive the net build-time environment for this run. We let config // environment override the build time environment. @@ -350,8 +347,8 @@ func run(b *builder, args []string, attributes map[string]bool, original string) // of RUN, without leaking it to the final image. It also aids cache // lookup for same image built with same build time environment. cmdBuildEnv := []string{} - configEnv := runconfig.ConvertKVStringsToMap(b.Config.Env) - for key, val := range b.buildArgs { + configEnv := runconfig.ConvertKVStringsToMap(b.runConfig.Env) + for key, val := range b.BuildArgs { if !b.isBuildArgAllowed(key) { // skip build-args that are not in allowed list, meaning they have // not been defined by an "ARG" Dockerfile command yet. @@ -379,7 +376,7 @@ func run(b *builder, args []string, attributes map[string]bool, original string) saveCmd = stringutils.NewStrSlice(append(tmpEnv, saveCmd.Slice()...)...) } - b.Config.Cmd = saveCmd + b.runConfig.Cmd = saveCmd hit, err := b.probeCache() if err != nil { return err @@ -389,11 +386,11 @@ func run(b *builder, args []string, attributes map[string]bool, original string) } // set Cmd manually, this is special case only for Dockerfiles - b.Config.Cmd = config.Cmd + b.runConfig.Cmd = config.Cmd // set build-time environment for 'run'. - b.Config.Env = append(b.Config.Env, cmdBuildEnv...) + b.runConfig.Env = append(b.runConfig.Env, cmdBuildEnv...) - logrus.Debugf("[BUILDER] Command to be executed: %v", b.Config.Cmd) + logrus.Debugf("[BUILDER] Command to be executed: %v", b.runConfig.Cmd) c, err := b.create() if err != nil { @@ -413,8 +410,8 @@ func run(b *builder, args []string, attributes map[string]bool, original string) // revert to original config environment and set the command string to // have the build-time env vars in it (if any) so that future cache look-ups // properly match it. - b.Config.Env = env - b.Config.Cmd = saveCmd + b.runConfig.Env = env + b.runConfig.Cmd = saveCmd if err := b.commit(c.ID, cmd, "run"); err != nil { return err } @@ -427,8 +424,8 @@ func run(b *builder, args []string, attributes map[string]bool, original string) // Set the default command to run in the container (which may be empty). // Argument handling is the same as RUN. // -func cmd(b *builder, args []string, attributes map[string]bool, original string) error { - if err := b.BuilderFlags.Parse(); err != nil { +func cmd(b *Builder, args []string, attributes map[string]bool, original string) error { + if err := b.flags.Parse(); err != nil { return err } @@ -442,9 +439,9 @@ func cmd(b *builder, args []string, attributes map[string]bool, original string) } } - b.Config.Cmd = stringutils.NewStrSlice(cmdSlice...) + b.runConfig.Cmd = stringutils.NewStrSlice(cmdSlice...) - if err := b.commit("", b.Config.Cmd, fmt.Sprintf("CMD %q", cmdSlice)); err != nil { + if err := b.commit("", b.runConfig.Cmd, fmt.Sprintf("CMD %q", cmdSlice)); err != nil { return err } @@ -460,11 +457,11 @@ func cmd(b *builder, args []string, attributes map[string]bool, original string) // Set the entrypoint (which defaults to sh -c on linux, or cmd /S /C on Windows) to // /usr/sbin/nginx. Will accept the CMD as the arguments to /usr/sbin/nginx. // -// Handles command processing similar to CMD and RUN, only b.Config.Entrypoint +// Handles command processing similar to CMD and RUN, only b.runConfig.Entrypoint // is initialized at NewBuilder time instead of through argument parsing. // -func entrypoint(b *builder, args []string, attributes map[string]bool, original string) error { - if err := b.BuilderFlags.Parse(); err != nil { +func entrypoint(b *Builder, args []string, attributes map[string]bool, original string) error { + if err := b.flags.Parse(); err != nil { return err } @@ -473,26 +470,26 @@ func entrypoint(b *builder, args []string, attributes map[string]bool, original switch { case attributes["json"]: // ENTRYPOINT ["echo", "hi"] - b.Config.Entrypoint = stringutils.NewStrSlice(parsed...) + b.runConfig.Entrypoint = stringutils.NewStrSlice(parsed...) case len(parsed) == 0: // ENTRYPOINT [] - b.Config.Entrypoint = nil + b.runConfig.Entrypoint = nil default: // ENTRYPOINT echo hi if runtime.GOOS != "windows" { - b.Config.Entrypoint = stringutils.NewStrSlice("/bin/sh", "-c", parsed[0]) + b.runConfig.Entrypoint = stringutils.NewStrSlice("/bin/sh", "-c", parsed[0]) } else { - b.Config.Entrypoint = stringutils.NewStrSlice("cmd", "/S", "/C", parsed[0]) + b.runConfig.Entrypoint = stringutils.NewStrSlice("cmd", "/S /C", parsed[0]) } } // when setting the entrypoint if a CMD was not explicitly set then // set the command to nil if !b.cmdSet { - b.Config.Cmd = nil + b.runConfig.Cmd = nil } - if err := b.commit("", b.Config.Cmd, fmt.Sprintf("ENTRYPOINT %q", b.Config.Entrypoint)); err != nil { + if err := b.commit("", b.runConfig.Cmd, fmt.Sprintf("ENTRYPOINT %q", b.runConfig.Entrypoint)); err != nil { return err } @@ -502,21 +499,21 @@ func entrypoint(b *builder, args []string, attributes map[string]bool, original // EXPOSE 6667/tcp 7000/tcp // // Expose ports for links and port mappings. This all ends up in -// b.Config.ExposedPorts for runconfig. +// b.runConfig.ExposedPorts for runconfig. // -func expose(b *builder, args []string, attributes map[string]bool, original string) error { +func expose(b *Builder, args []string, attributes map[string]bool, original string) error { portsTab := args if len(args) == 0 { return derr.ErrorCodeAtLeastOneArg.WithArgs("EXPOSE") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } - if b.Config.ExposedPorts == nil { - b.Config.ExposedPorts = make(nat.PortSet) + if b.runConfig.ExposedPorts == nil { + b.runConfig.ExposedPorts = make(nat.PortSet) } ports, _, err := nat.ParsePortSpecs(portsTab) @@ -530,14 +527,14 @@ func expose(b *builder, args []string, attributes map[string]bool, original stri portList := make([]string, len(ports)) var i int for port := range ports { - if _, exists := b.Config.ExposedPorts[port]; !exists { - b.Config.ExposedPorts[port] = struct{}{} + if _, exists := b.runConfig.ExposedPorts[port]; !exists { + b.runConfig.ExposedPorts[port] = struct{}{} } portList[i] = string(port) i++ } sort.Strings(portList) - return b.commit("", b.Config.Cmd, fmt.Sprintf("EXPOSE %s", strings.Join(portList, " "))) + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("EXPOSE %s", strings.Join(portList, " "))) } // USER foo @@ -545,43 +542,43 @@ func expose(b *builder, args []string, attributes map[string]bool, original stri // Set the user to 'foo' for future commands and when running the // ENTRYPOINT/CMD at container run time. // -func user(b *builder, args []string, attributes map[string]bool, original string) error { +func user(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return derr.ErrorCodeExactlyOneArg.WithArgs("USER") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } - b.Config.User = args[0] - return b.commit("", b.Config.Cmd, fmt.Sprintf("USER %v", args)) + b.runConfig.User = args[0] + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("USER %v", args)) } // VOLUME /foo // // Expose the volume /foo for use. Will also accept the JSON array form. // -func volume(b *builder, args []string, attributes map[string]bool, original string) error { +func volume(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) == 0 { return derr.ErrorCodeAtLeastOneArg.WithArgs("VOLUME") } - if err := b.BuilderFlags.Parse(); err != nil { + if err := b.flags.Parse(); err != nil { return err } - if b.Config.Volumes == nil { - b.Config.Volumes = map[string]struct{}{} + if b.runConfig.Volumes == nil { + b.runConfig.Volumes = map[string]struct{}{} } for _, v := range args { v = strings.TrimSpace(v) if v == "" { return derr.ErrorCodeVolumeEmpty } - b.Config.Volumes[v] = struct{}{} + b.runConfig.Volumes[v] = struct{}{} } - if err := b.commit("", b.Config.Cmd, fmt.Sprintf("VOLUME %v", args)); err != nil { + if err := b.commit("", b.runConfig.Cmd, fmt.Sprintf("VOLUME %v", args)); err != nil { return err } return nil @@ -590,7 +587,7 @@ func volume(b *builder, args []string, attributes map[string]bool, original stri // STOPSIGNAL signal // // Set the signal that will be used to kill the container. -func stopSignal(b *builder, args []string, attributes map[string]bool, original string) error { +func stopSignal(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return fmt.Errorf("STOPSIGNAL requires exactly one argument") } @@ -601,8 +598,8 @@ func stopSignal(b *builder, args []string, attributes map[string]bool, original return err } - b.Config.StopSignal = sig - return b.commit("", b.Config.Cmd, fmt.Sprintf("STOPSIGNAL %v", args)) + b.runConfig.StopSignal = sig + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("STOPSIGNAL %v", args)) } // ARG name[=value] @@ -610,7 +607,7 @@ func stopSignal(b *builder, args []string, attributes map[string]bool, original // Adds the variable foo to the trusted list of variables that can be passed // to builder using the --build-arg flag for expansion/subsitution or passing to 'run'. // Dockerfile author may optionally set a default value of this variable. -func arg(b *builder, args []string, attributes map[string]bool, original string) error { +func arg(b *Builder, args []string, attributes map[string]bool, original string) error { if len(args) != 1 { return fmt.Errorf("ARG requires exactly one argument definition") } @@ -642,9 +639,9 @@ func arg(b *builder, args []string, attributes map[string]bool, original string) // If there is a default value associated with this arg then add it to the // b.buildArgs if one is not already passed to the builder. The args passed // to builder override the defaut value of 'arg'. - if _, ok := b.buildArgs[name]; !ok && hasDefault { - b.buildArgs[name] = value + if _, ok := b.BuildArgs[name]; !ok && hasDefault { + b.BuildArgs[name] = value } - return b.commit("", b.Config.Cmd, fmt.Sprintf("ARG %s", arg)) + return b.commit("", b.runConfig.Cmd, fmt.Sprintf("ARG %s", arg)) } diff --git a/builder/dockerfile/evaluator.go b/builder/dockerfile/evaluator.go index c202273b9e..0629ee9bbc 100644 --- a/builder/dockerfile/evaluator.go +++ b/builder/dockerfile/evaluator.go @@ -21,26 +21,11 @@ package dockerfile import ( "fmt" - "io" - "os" - "path/filepath" "runtime" "strings" - "github.com/Sirupsen/logrus" - "github.com/docker/docker/api" "github.com/docker/docker/builder/dockerfile/command" "github.com/docker/docker/builder/dockerfile/parser" - "github.com/docker/docker/cliconfig" - "github.com/docker/docker/daemon" - "github.com/docker/docker/pkg/fileutils" - "github.com/docker/docker/pkg/streamformatter" - "github.com/docker/docker/pkg/stringid" - "github.com/docker/docker/pkg/symlink" - "github.com/docker/docker/pkg/tarsum" - "github.com/docker/docker/pkg/ulimit" - "github.com/docker/docker/runconfig" - "github.com/docker/docker/utils" ) // Environment variable interpolation will happen on these statements only. @@ -57,10 +42,10 @@ var replaceEnvAllowed = map[string]struct{}{ command.Arg: {}, } -var evaluateTable map[string]func(*builder, []string, map[string]bool, string) error +var evaluateTable map[string]func(*Builder, []string, map[string]bool, string) error func init() { - evaluateTable = map[string]func(*builder, []string, map[string]bool, string) error{ + evaluateTable = map[string]func(*Builder, []string, map[string]bool, string) error{ command.Env: env, command.Label: label, command.Maintainer: maintainer, @@ -80,223 +65,6 @@ func init() { } } -// builder is an internal struct, used to maintain configuration of the Dockerfile's -// processing as it evaluates the parsing result. -type builder struct { - Daemon *daemon.Daemon - - // effectively stdio for the run. Because it is not stdio, I said - // "Effectively". Do not use stdio anywhere in this package for any reason. - OutStream io.Writer - ErrStream io.Writer - - Verbose bool - UtilizeCache bool - cacheBusted bool - - // controls how images and containers are handled between steps. - Remove bool - ForceRemove bool - Pull bool - - // set this to true if we want the builder to not commit between steps. - // This is useful when we only want to use the evaluator table to generate - // the final configs of the Dockerfile but dont want the layers - disableCommit bool - - // Registry server auth configs used to pull images when handling `FROM`. - AuthConfigs map[string]cliconfig.AuthConfig - - // Deprecated, original writer used for ImagePull. To be removed. - OutOld io.Writer - StreamFormatter *streamformatter.StreamFormatter - - Config *runconfig.Config // runconfig for cmd, run, entrypoint etc. - - buildArgs map[string]string // build-time args received in build context for expansion/substitution and commands in 'run'. - allowedBuildArgs map[string]bool // list of build-time args that are allowed for expansion/substitution and passing to commands in 'run'. - - // both of these are controlled by the Remove and ForceRemove options in BuildOpts - TmpContainers map[string]struct{} // a map of containers used for removes - - dockerfileName string // name of Dockerfile - dockerfile *parser.Node // the syntax tree of the dockerfile - image string // image name for commit processing - maintainer string // maintainer name. could probably be removed. - cmdSet bool // indicates is CMD was set in current Dockerfile - BuilderFlags *BFlags // current cmd's BuilderFlags - temporary - context tarsum.TarSum // the context is a tarball that is uploaded by the client - contextPath string // the path of the temporary directory the local context is unpacked to (server side) - noBaseImage bool // indicates that this build does not start from any base image, but is being built from an empty file system. - - // Set resource restrictions for build containers - cpuSetCpus string - cpuSetMems string - cpuShares int64 - cpuPeriod int64 - cpuQuota int64 - cgroupParent string - memory int64 - memorySwap int64 - ulimits []*ulimit.Ulimit - - cancelled <-chan struct{} // When closed, job was cancelled. - - activeImages []string - id string // Used to hold reference images -} - -// Run the builder with the context. This is the lynchpin of this package. This -// will (barring errors): -// -// * call readContext() which will set up the temporary directory and unpack -// the context into it. -// * read the dockerfile -// * parse the dockerfile -// * walk the parse tree and execute it by dispatching to handlers. If Remove -// or ForceRemove is set, additional cleanup around containers happens after -// processing. -// * Print a happy message and return the image ID. -// -func (b *builder) Run(context io.Reader) (string, error) { - if err := b.readContext(context); err != nil { - return "", err - } - - defer func() { - if err := os.RemoveAll(b.contextPath); err != nil { - logrus.Debugf("[BUILDER] failed to remove temporary context: %s", err) - } - }() - - if err := b.readDockerfile(); err != nil { - return "", err - } - - // some initializations that would not have been supplied by the caller. - b.Config = &runconfig.Config{} - - b.TmpContainers = map[string]struct{}{} - - for i, n := range b.dockerfile.Children { - select { - case <-b.cancelled: - logrus.Debug("Builder: build cancelled!") - fmt.Fprintf(b.OutStream, "Build cancelled") - return "", fmt.Errorf("Build cancelled") - default: - // Not cancelled yet, keep going... - } - if err := b.dispatch(i, n); err != nil { - if b.ForceRemove { - b.clearTmp() - } - return "", err - } - fmt.Fprintf(b.OutStream, " ---> %s\n", stringid.TruncateID(b.image)) - if b.Remove { - b.clearTmp() - } - } - - // check if there are any leftover build-args that were passed but not - // consumed during build. Return an error, if there are any. - leftoverArgs := []string{} - for arg := range b.buildArgs { - if !b.isBuildArgAllowed(arg) { - leftoverArgs = append(leftoverArgs, arg) - } - } - if len(leftoverArgs) > 0 { - return "", fmt.Errorf("One or more build-args %v were not consumed, failing build.", leftoverArgs) - } - - if b.image == "" { - return "", fmt.Errorf("No image was generated. Is your Dockerfile empty?") - } - - fmt.Fprintf(b.OutStream, "Successfully built %s\n", stringid.TruncateID(b.image)) - return b.image, nil -} - -// Reads a Dockerfile from the current context. It assumes that the -// 'filename' is a relative path from the root of the context -func (b *builder) readDockerfile() error { - // If no -f was specified then look for 'Dockerfile'. If we can't find - // that then look for 'dockerfile'. If neither are found then default - // back to 'Dockerfile' and use that in the error message. - if b.dockerfileName == "" { - b.dockerfileName = api.DefaultDockerfileName - tmpFN := filepath.Join(b.contextPath, api.DefaultDockerfileName) - if _, err := os.Lstat(tmpFN); err != nil { - tmpFN = filepath.Join(b.contextPath, strings.ToLower(api.DefaultDockerfileName)) - if _, err := os.Lstat(tmpFN); err == nil { - b.dockerfileName = strings.ToLower(api.DefaultDockerfileName) - } - } - } - - origFile := b.dockerfileName - - filename, err := symlink.FollowSymlinkInScope(filepath.Join(b.contextPath, origFile), b.contextPath) - if err != nil { - return fmt.Errorf("The Dockerfile (%s) must be within the build context", origFile) - } - - fi, err := os.Lstat(filename) - if os.IsNotExist(err) { - return fmt.Errorf("Cannot locate specified Dockerfile: %s", origFile) - } - if fi.Size() == 0 { - return fmt.Errorf("The Dockerfile (%s) cannot be empty", origFile) - } - - f, err := os.Open(filename) - if err != nil { - return err - } - - b.dockerfile, err = parser.Parse(f) - f.Close() - - if err != nil { - return err - } - - // After the Dockerfile has been parsed, we need to check the .dockerignore - // file for either "Dockerfile" or ".dockerignore", and if either are - // present then erase them from the build context. These files should never - // have been sent from the client but we did send them to make sure that - // we had the Dockerfile to actually parse, and then we also need the - // .dockerignore file to know whether either file should be removed. - // Note that this assumes the Dockerfile has been read into memory and - // is now safe to be removed. - - excludes, _ := utils.ReadDockerIgnore(filepath.Join(b.contextPath, ".dockerignore")) - if rm, _ := fileutils.Matches(".dockerignore", excludes); rm == true { - os.Remove(filepath.Join(b.contextPath, ".dockerignore")) - b.context.(tarsum.BuilderContext).Remove(".dockerignore") - } - if rm, _ := fileutils.Matches(b.dockerfileName, excludes); rm == true { - os.Remove(filepath.Join(b.contextPath, b.dockerfileName)) - b.context.(tarsum.BuilderContext).Remove(b.dockerfileName) - } - - return nil -} - -// determine if build arg is part of built-in args or user -// defined args in Dockerfile at any point in time. -func (b *builder) isBuildArgAllowed(arg string) bool { - if _, ok := BuiltinAllowedBuildArgs[arg]; ok { - return true - } - if _, ok := b.allowedBuildArgs[arg]; ok { - return true - } - return false -} - // This method is the entrypoint to all statement handling routines. // // Almost all nodes will have this structure: @@ -311,8 +79,9 @@ func (b *builder) isBuildArgAllowed(arg string) bool { // such as `RUN` in ONBUILD RUN foo. There is special case logic in here to // deal with that, at least until it becomes more of a general concern with new // features. -func (b *builder) dispatch(stepN int, ast *parser.Node) error { +func (b *Builder) dispatch(stepN int, ast *parser.Node) error { cmd := ast.Value + upperCasedCmd := strings.ToUpper(cmd) // To ensure the user is given a decent error message if the platform // on which the daemon is running does not support a builder command. @@ -324,7 +93,7 @@ func (b *builder) dispatch(stepN int, ast *parser.Node) error { original := ast.Original flags := ast.Flags strs := []string{} - msg := fmt.Sprintf("Step %d : %s", stepN+1, strings.ToUpper(cmd)) + msg := fmt.Sprintf("Step %d : %s", stepN+1, upperCasedCmd) if len(ast.Flags) > 0 { msg += " " + strings.Join(ast.Flags, " ") @@ -368,8 +137,8 @@ func (b *builder) dispatch(stepN int, ast *parser.Node) error { // stop on the first occurrence of a variable name and not notice // a subsequent one. So, putting the buildArgs list after the Config.Env // list, in 'envs', is safe. - envs := b.Config.Env - for key, val := range b.buildArgs { + envs := b.runConfig.Env + for key, val := range b.BuildArgs { if !b.isBuildArgAllowed(key) { // skip build-args that are not in allowed list, meaning they have // not been defined by an "ARG" Dockerfile command yet. @@ -397,17 +166,17 @@ func (b *builder) dispatch(stepN int, ast *parser.Node) error { } msg += " " + strings.Join(msgList, " ") - fmt.Fprintln(b.OutStream, msg) + fmt.Fprintln(b.Stdout, msg) // XXX yes, we skip any cmds that are not valid; the parser should have // picked these out already. if f, ok := evaluateTable[cmd]; ok { - b.BuilderFlags = NewBFlags() - b.BuilderFlags.Args = flags + b.flags = NewBFlags() + b.flags.Args = flags return f(b, strList, attrs, original) } - return fmt.Errorf("Unknown instruction: %s", strings.ToUpper(cmd)) + return fmt.Errorf("Unknown instruction: %s", upperCasedCmd) } // platformSupports is a short-term function to give users a quality error diff --git a/builder/dockerfile/internals.go b/builder/dockerfile/internals.go index 2e3a96bafd..ddcde4778a 100644 --- a/builder/dockerfile/internals.go +++ b/builder/dockerfile/internals.go @@ -19,83 +19,45 @@ import ( "time" "github.com/Sirupsen/logrus" + "github.com/docker/docker/api" + "github.com/docker/docker/builder" "github.com/docker/docker/builder/dockerfile/parser" - "github.com/docker/docker/cliconfig" "github.com/docker/docker/daemon" - "github.com/docker/docker/graph" "github.com/docker/docker/image" "github.com/docker/docker/pkg/archive" - "github.com/docker/docker/pkg/chrootarchive" "github.com/docker/docker/pkg/httputils" "github.com/docker/docker/pkg/ioutils" "github.com/docker/docker/pkg/jsonmessage" - "github.com/docker/docker/pkg/parsers" "github.com/docker/docker/pkg/progressreader" + "github.com/docker/docker/pkg/streamformatter" "github.com/docker/docker/pkg/stringid" "github.com/docker/docker/pkg/stringutils" - "github.com/docker/docker/pkg/symlink" "github.com/docker/docker/pkg/system" "github.com/docker/docker/pkg/tarsum" "github.com/docker/docker/pkg/urlutil" - "github.com/docker/docker/registry" "github.com/docker/docker/runconfig" ) -func (b *builder) readContext(context io.Reader) (err error) { - tmpdirPath, err := getTempDir("", "docker-build") - if err != nil { - return - } - - // Make sure we clean-up upon error. In the happy case the caller - // is expected to manage the clean-up - defer func() { - if err != nil { - if e := os.RemoveAll(tmpdirPath); e != nil { - logrus.Debugf("[BUILDER] failed to remove temporary context: %s", e) - } - } - }() - - decompressedStream, err := archive.DecompressStream(context) - if err != nil { - return - } - - if b.context, err = tarsum.NewTarSum(decompressedStream, true, tarsum.Version1); err != nil { - return - } - - if err = chrootarchive.Untar(b.context, tmpdirPath, nil); err != nil { - return - } - - b.contextPath = tmpdirPath - return -} - -func (b *builder) commit(id string, autoCmd *stringutils.StrSlice, comment string) error { +func (b *Builder) commit(id string, autoCmd *stringutils.StrSlice, comment string) error { if b.disableCommit { return nil } if b.image == "" && !b.noBaseImage { return fmt.Errorf("Please provide a source image with `from` prior to commit") } - b.Config.Image = b.image + b.runConfig.Image = b.image if id == "" { - cmd := b.Config.Cmd + cmd := b.runConfig.Cmd if runtime.GOOS != "windows" { - b.Config.Cmd = stringutils.NewStrSlice("/bin/sh", "-c", "#(nop) "+comment) + b.runConfig.Cmd = stringutils.NewStrSlice("/bin/sh", "-c", "#(nop) "+comment) } else { - b.Config.Cmd = stringutils.NewStrSlice("cmd", "/S", "/C", "REM (nop) "+comment) + b.runConfig.Cmd = stringutils.NewStrSlice("cmd", "/S /C", "REM (nop) "+comment) } - defer func(cmd *stringutils.StrSlice) { b.Config.Cmd = cmd }(cmd) + defer func(cmd *stringutils.StrSlice) { b.runConfig.Cmd = cmd }(cmd) - hit, err := b.probeCache() - if err != nil { + if hit, err := b.probeCache(); err != nil { return err - } - if hit { + } else if hit { return nil } @@ -110,13 +72,14 @@ func (b *builder) commit(id string, autoCmd *stringutils.StrSlice, comment strin } defer container.Unmount() } - container, err := b.Daemon.Get(id) + + container, err := b.docker.Container(id) if err != nil { return err } // Note: Actually copy the struct - autoConfig := *b.Config + autoConfig := *b.runConfig autoConfig.Cmd = autoCmd commitCfg := &daemon.ContainerCommitConfig{ @@ -126,25 +89,22 @@ func (b *builder) commit(id string, autoCmd *stringutils.StrSlice, comment strin } // Commit the container - image, err := b.Daemon.Commit(container, commitCfg) + image, err := b.docker.Commit(container, commitCfg) if err != nil { return err } - b.Daemon.Graph().Retain(b.id, image.ID) + b.docker.Retain(b.id, image.ID) b.activeImages = append(b.activeImages, image.ID) b.image = image.ID return nil } type copyInfo struct { - origPath string - destPath string - hash string + builder.FileInfo decompress bool - tmpDir string } -func (b *builder) runContextCommand(args []string, allowRemote bool, allowDecompression bool, cmdName string) error { +func (b *Builder) runContextCommand(args []string, allowRemote bool, allowLocalDecompression bool, cmdName string) error { if b.context == nil { return fmt.Errorf("No context given. Impossible to use %s", cmdName) } @@ -156,57 +116,66 @@ func (b *builder) runContextCommand(args []string, allowRemote bool, allowDecomp // Work in daemon-specific filepath semantics dest := filepath.FromSlash(args[len(args)-1]) // last one is always the dest - copyInfos := []*copyInfo{} + b.runConfig.Image = b.image - b.Config.Image = b.image - - defer func() { - for _, ci := range copyInfos { - if ci.tmpDir != "" { - os.RemoveAll(ci.tmpDir) - } - } - }() + var infos []copyInfo // Loop through each src file and calculate the info we need to // do the copy (e.g. hash value if cached). Don't actually do // the copy until we've looked at all src files + var err error for _, orig := range args[0 : len(args)-1] { - if err := calcCopyInfo( - b, - cmdName, - ©Infos, - orig, - dest, - allowRemote, - allowDecompression, - true, - ); err != nil { + var fi builder.FileInfo + decompress := allowLocalDecompression + if urlutil.IsURL(orig) { + if !allowRemote { + return fmt.Errorf("Source can't be a URL for %s", cmdName) + } + fi, err = b.download(orig) + if err != nil { + return err + } + defer os.RemoveAll(filepath.Dir(fi.Path())) + decompress = false + infos = append(infos, copyInfo{fi, decompress}) + continue + } + // not a URL + subInfos, err := b.calcCopyInfo(cmdName, orig, allowLocalDecompression, true) + if err != nil { return err } + + infos = append(infos, subInfos...) } - if len(copyInfos) == 0 { + if len(infos) == 0 { return fmt.Errorf("No source files were specified") } - if len(copyInfos) > 1 && !strings.HasSuffix(dest, string(os.PathSeparator)) { + if len(infos) > 1 && !strings.HasSuffix(dest, string(os.PathSeparator)) { return fmt.Errorf("When using %s with more than one source file, the destination must be a directory and end with a /", cmdName) } - // For backwards compat, if there's just one CI then use it as the + // For backwards compat, if there's just one info then use it as the // cache look-up string, otherwise hash 'em all into one var srcHash string var origPaths string - if len(copyInfos) == 1 { - srcHash = copyInfos[0].hash - origPaths = copyInfos[0].origPath + if len(infos) == 1 { + fi := infos[0].FileInfo + origPaths = fi.Name() + if hfi, ok := fi.(builder.Hashed); ok { + srcHash = hfi.Hash() + } } else { var hashs []string var origs []string - for _, ci := range copyInfos { - hashs = append(hashs, ci.hash) - origs = append(origs, ci.origPath) + for _, info := range infos { + fi := info.FileInfo + origs = append(origs, fi.Name()) + if hfi, ok := fi.(builder.Hashed); ok { + hashs = append(hashs, hfi.Hash()) + } } hasher := sha256.New() hasher.Write([]byte(strings.Join(hashs, ","))) @@ -214,262 +183,225 @@ func (b *builder) runContextCommand(args []string, allowRemote bool, allowDecomp origPaths = strings.Join(origs, " ") } - cmd := b.Config.Cmd + cmd := b.runConfig.Cmd if runtime.GOOS != "windows" { - b.Config.Cmd = stringutils.NewStrSlice("/bin/sh", "-c", fmt.Sprintf("#(nop) %s %s in %s", cmdName, srcHash, dest)) + b.runConfig.Cmd = stringutils.NewStrSlice("/bin/sh", "-c", fmt.Sprintf("#(nop) %s %s in %s", cmdName, srcHash, dest)) } else { - b.Config.Cmd = stringutils.NewStrSlice("cmd", "/S", "/C", fmt.Sprintf("REM (nop) %s %s in %s", cmdName, srcHash, dest)) + b.runConfig.Cmd = stringutils.NewStrSlice("cmd", "/S /C", fmt.Sprintf("REM (nop) %s %s in %s", cmdName, srcHash, dest)) } - defer func(cmd *stringutils.StrSlice) { b.Config.Cmd = cmd }(cmd) + defer func(cmd *stringutils.StrSlice) { b.runConfig.Cmd = cmd }(cmd) - hit, err := b.probeCache() - if err != nil { + if hit, err := b.probeCache(); err != nil { return err - } - - if hit { + } else if hit { return nil } - ccr, err := b.Daemon.ContainerCreate("", b.Config, nil, true) + container, _, err := b.docker.Create(b.runConfig, nil) if err != nil { return err } - container, err := b.Daemon.Get(ccr.ID) - if err != nil { - return err - } - - b.TmpContainers[container.ID] = struct{}{} - - if err := container.Mount(); err != nil { - return err - } defer container.Unmount() + b.tmpContainers[container.ID] = struct{}{} - for _, ci := range copyInfos { - if err := b.addContext(container, ci.origPath, ci.destPath, ci.decompress); err != nil { + comment := fmt.Sprintf("%s %s in %s", cmdName, origPaths, dest) + + // Twiddle the destination when its a relative path - meaning, make it + // relative to the WORKINGDIR + if !system.IsAbs(dest) { + hasSlash := strings.HasSuffix(dest, string(os.PathSeparator)) + dest = filepath.Join(string(os.PathSeparator), filepath.FromSlash(b.runConfig.WorkingDir), dest) + + // Make sure we preserve any trailing slash + if hasSlash { + dest += string(os.PathSeparator) + } + } + + for _, info := range infos { + if err := b.docker.Copy(container, dest, info.FileInfo, info.decompress); err != nil { return err } } - if err := b.commit(container.ID, cmd, fmt.Sprintf("%s %s in %s", cmdName, origPaths, dest)); err != nil { + if err := b.commit(container.ID, cmd, comment); err != nil { return err } return nil } -func calcCopyInfo(b *builder, cmdName string, cInfos *[]*copyInfo, origPath string, destPath string, allowRemote bool, allowDecompression bool, allowWildcards bool) error { +func (b *Builder) download(srcURL string) (fi builder.FileInfo, err error) { + // get filename from URL + u, err := url.Parse(srcURL) + if err != nil { + return + } + path := u.Path + if strings.HasSuffix(path, string(os.PathSeparator)) { + path = path[:len(path)-1] + } + parts := strings.Split(path, string(os.PathSeparator)) + filename := parts[len(parts)-1] + if filename == "" { + err = fmt.Errorf("cannot determine filename from url: %s", u) + return + } - // Work in daemon-specific OS filepath semantics. However, we save - // the the origPath passed in here, as it might also be a URL which - // we need to check for in this function. - passedInOrigPath := origPath + // Initiate the download + resp, err := httputils.Download(srcURL) + if err != nil { + return + } + + // Prepare file in a tmp dir + tmpDir, err := ioutils.TempDir("", "docker-remote") + if err != nil { + return + } + defer func() { + if err != nil { + os.RemoveAll(tmpDir) + } + }() + tmpFileName := filepath.Join(tmpDir, filename) + tmpFile, err := os.OpenFile(tmpFileName, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0600) + if err != nil { + return + } + + // Download and dump result to tmp file + if _, err = io.Copy(tmpFile, progressreader.New(progressreader.Config{ + In: resp.Body, + // TODO: make progressreader streamformatter agnostic + Out: b.Stdout.(*streamformatter.StdoutFormatter).Writer, + Formatter: b.Stdout.(*streamformatter.StdoutFormatter).StreamFormatter, + Size: resp.ContentLength, + NewLines: true, + ID: "", + Action: "Downloading", + })); err != nil { + tmpFile.Close() + return + } + fmt.Fprintln(b.Stdout) + // ignoring error because the file was already opened successfully + tmpFileSt, err := tmpFile.Stat() + if err != nil { + return + } + tmpFile.Close() + + // Set the mtime to the Last-Modified header value if present + // Otherwise just remove atime and mtime + mTime := time.Time{} + + lastMod := resp.Header.Get("Last-Modified") + if lastMod != "" { + // If we can't parse it then just let it default to 'zero' + // otherwise use the parsed time value + if parsedMTime, err := http.ParseTime(lastMod); err == nil { + mTime = parsedMTime + } + } + + if err = system.Chtimes(tmpFileName, time.Time{}, mTime); err != nil { + return + } + + // Calc the checksum, even if we're using the cache + r, err := archive.Tar(tmpFileName, archive.Uncompressed) + if err != nil { + return + } + tarSum, err := tarsum.NewTarSum(r, true, tarsum.Version1) + if err != nil { + return + } + if _, err = io.Copy(ioutil.Discard, tarSum); err != nil { + return + } + hash := tarSum.Sum(nil) + r.Close() + return &builder.HashedFileInfo{FileInfo: builder.PathFileInfo{FileInfo: tmpFileSt, FilePath: tmpFileName}, FileHash: hash}, nil +} + +func (b *Builder) calcCopyInfo(cmdName, origPath string, allowLocalDecompression, allowWildcards bool) ([]copyInfo, error) { + + // Work in daemon-specific OS filepath semantics origPath = filepath.FromSlash(origPath) - destPath = filepath.FromSlash(destPath) if origPath != "" && origPath[0] == os.PathSeparator && len(origPath) > 1 { origPath = origPath[1:] } origPath = strings.TrimPrefix(origPath, "."+string(os.PathSeparator)) - // Twiddle the destPath when its a relative path - meaning, make it - // relative to the WORKINGDIR - if !system.IsAbs(destPath) { - hasSlash := strings.HasSuffix(destPath, string(os.PathSeparator)) - destPath = filepath.Join(string(os.PathSeparator), filepath.FromSlash(b.Config.WorkingDir), destPath) - - // Make sure we preserve any trailing slash - if hasSlash { - destPath += string(os.PathSeparator) - } - } - - // In the remote/URL case, download it and gen its hashcode - if urlutil.IsURL(passedInOrigPath) { - - // As it's a URL, we go back to processing on what was passed in - // to this function - origPath = passedInOrigPath - - if !allowRemote { - return fmt.Errorf("Source can't be a URL for %s", cmdName) - } - - ci := copyInfo{} - ci.origPath = origPath - ci.hash = origPath // default to this but can change - ci.destPath = destPath - ci.decompress = false - *cInfos = append(*cInfos, &ci) - - // Initiate the download - resp, err := httputils.Download(ci.origPath) - if err != nil { - return err - } - - // Create a tmp dir - tmpDirName, err := getTempDir(b.contextPath, "docker-remote") - if err != nil { - return err - } - ci.tmpDir = tmpDirName - - // Create a tmp file within our tmp dir - tmpFileName := filepath.Join(tmpDirName, "tmp") - tmpFile, err := os.OpenFile(tmpFileName, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0600) - if err != nil { - return err - } - - // Download and dump result to tmp file - if _, err := io.Copy(tmpFile, progressreader.New(progressreader.Config{ - In: resp.Body, - Out: b.OutOld, - Formatter: b.StreamFormatter, - Size: resp.ContentLength, - NewLines: true, - ID: "", - Action: "Downloading", - })); err != nil { - tmpFile.Close() - return err - } - fmt.Fprintf(b.OutStream, "\n") - tmpFile.Close() - - // Set the mtime to the Last-Modified header value if present - // Otherwise just remove atime and mtime - mTime := time.Time{} - - lastMod := resp.Header.Get("Last-Modified") - if lastMod != "" { - // If we can't parse it then just let it default to 'zero' - // otherwise use the parsed time value - if parsedMTime, err := http.ParseTime(lastMod); err == nil { - mTime = parsedMTime - } - } - - if err := system.Chtimes(tmpFileName, time.Time{}, mTime); err != nil { - return err - } - - ci.origPath = filepath.Join(filepath.Base(tmpDirName), filepath.Base(tmpFileName)) - - // If the destination is a directory, figure out the filename. - if strings.HasSuffix(ci.destPath, string(os.PathSeparator)) { - u, err := url.Parse(origPath) + // Deal with wildcards + if allowWildcards && containsWildcards(origPath) { + var copyInfos []copyInfo + if err := b.context.Walk("", func(path string, info builder.FileInfo, err error) error { if err != nil { return err } - path := filepath.FromSlash(u.Path) // Ensure in platform semantics - if strings.HasSuffix(path, string(os.PathSeparator)) { - path = path[:len(path)-1] + if info.Name() == "" { + // Why are we doing this check? + return nil } - parts := strings.Split(path, string(os.PathSeparator)) - filename := parts[len(parts)-1] - if filename == "" { - return fmt.Errorf("cannot determine filename from url: %s", u) - } - ci.destPath = ci.destPath + filename - } - - // Calc the checksum, even if we're using the cache - r, err := archive.Tar(tmpFileName, archive.Uncompressed) - if err != nil { - return err - } - tarSum, err := tarsum.NewTarSum(r, true, tarsum.Version1) - if err != nil { - return err - } - if _, err := io.Copy(ioutil.Discard, tarSum); err != nil { - return err - } - ci.hash = tarSum.Sum(nil) - r.Close() - - return nil - } - - // Deal with wildcards - if allowWildcards && containsWildcards(origPath) { - for _, fileInfo := range b.context.GetSums() { - if fileInfo.Name() == "" { - continue - } - match, _ := filepath.Match(origPath, fileInfo.Name()) - if !match { - continue + if match, _ := filepath.Match(origPath, path); !match { + return nil } // Note we set allowWildcards to false in case the name has // a * in it - calcCopyInfo(b, cmdName, cInfos, fileInfo.Name(), destPath, allowRemote, allowDecompression, false) + subInfos, err := b.calcCopyInfo(cmdName, path, allowLocalDecompression, false) + if err != nil { + return err + } + copyInfos = append(copyInfos, subInfos...) + return nil + }); err != nil { + return nil, err } - return nil + return copyInfos, nil } // Must be a dir or a file - if err := b.checkPathForAddition(origPath); err != nil { - return err + fi, err := b.context.Stat(origPath) + if err != nil { + return nil, err } - fi, _ := os.Stat(filepath.Join(b.contextPath, origPath)) - ci := copyInfo{} - ci.origPath = origPath - ci.hash = origPath - ci.destPath = destPath - ci.decompress = allowDecompression - *cInfos = append(*cInfos, &ci) + copyInfos := []copyInfo{{FileInfo: fi, decompress: allowLocalDecompression}} + + hfi, handleHash := fi.(builder.Hashed) + if !handleHash { + return copyInfos, nil + } // Deal with the single file case if !fi.IsDir() { - // This will match first file in sums of the archive - fis := b.context.GetSums().GetFile(ci.origPath) - if fis != nil { - ci.hash = "file:" + fis.Sum() - } - return nil + hfi.SetHash("file:" + hfi.Hash()) + return copyInfos, nil } // Must be a dir + var subfiles []string - absOrigPath := filepath.Join(b.contextPath, ci.origPath) - - // Add a trailing / to make sure we only pick up nested files under - // the dir and not sibling files of the dir that just happen to - // start with the same chars - if !strings.HasSuffix(absOrigPath, string(os.PathSeparator)) { - absOrigPath += string(os.PathSeparator) - } - - // Need path w/o slash too to find matching dir w/o trailing slash - absOrigPathNoSlash := absOrigPath[:len(absOrigPath)-1] - - for _, fileInfo := range b.context.GetSums() { - absFile := filepath.Join(b.contextPath, fileInfo.Name()) - // Any file in the context that starts with the given path will be - // picked up and its hashcode used. However, we'll exclude the - // root dir itself. We do this for a coupel of reasons: - // 1 - ADD/COPY will not copy the dir itself, just its children - // so there's no reason to include it in the hash calc - // 2 - the metadata on the dir will change when any child file - // changes. This will lead to a miss in the cache check if that - // child file is in the .dockerignore list. - if strings.HasPrefix(absFile, absOrigPath) && absFile != absOrigPathNoSlash { - subfiles = append(subfiles, fileInfo.Sum()) + b.context.Walk(origPath, func(path string, info builder.FileInfo, err error) error { + if err != nil { + return err } - } + // we already checked handleHash above + subfiles = append(subfiles, info.(builder.Hashed).Hash()) + return nil + }) + sort.Strings(subfiles) hasher := sha256.New() hasher.Write([]byte(strings.Join(subfiles, ","))) - ci.hash = "dir:" + hex.EncodeToString(hasher.Sum(nil)) + hfi.SetHash("dir:" + hex.EncodeToString(hasher.Sum(nil))) - return nil + return copyInfos, nil } func containsWildcards(name string) bool { @@ -484,68 +416,30 @@ func containsWildcards(name string) bool { return false } -func (b *builder) pullImage(name string) (*image.Image, error) { - remote, tag := parsers.ParseRepositoryTag(name) - if tag == "" { - tag = "latest" - } - - pullRegistryAuth := &cliconfig.AuthConfig{} - if len(b.AuthConfigs) > 0 { - // The request came with a full auth config file, we prefer to use that - repoInfo, err := b.Daemon.RegistryService.ResolveRepository(remote) - if err != nil { - return nil, err - } - - resolvedConfig := registry.ResolveAuthConfig( - &cliconfig.ConfigFile{AuthConfigs: b.AuthConfigs}, - repoInfo.Index, - ) - pullRegistryAuth = &resolvedConfig - } - - imagePullConfig := &graph.ImagePullConfig{ - AuthConfig: pullRegistryAuth, - OutStream: ioutils.NopWriteCloser(b.OutOld), - } - - if err := b.Daemon.Repositories().Pull(remote, tag, imagePullConfig); err != nil { - return nil, err - } - - image, err := b.Daemon.Repositories().LookupImage(name) - if err != nil { - return nil, err - } - - return image, nil -} - -func (b *builder) processImageFrom(img *image.Image) error { +func (b *Builder) processImageFrom(img *image.Image) error { b.image = img.ID if img.Config != nil { - b.Config = img.Config + b.runConfig = img.Config } // The default path will be blank on Windows (set by HCS) - if len(b.Config.Env) == 0 && daemon.DefaultPathEnv != "" { - b.Config.Env = append(b.Config.Env, "PATH="+daemon.DefaultPathEnv) + if len(b.runConfig.Env) == 0 && daemon.DefaultPathEnv != "" { + b.runConfig.Env = append(b.runConfig.Env, "PATH="+daemon.DefaultPathEnv) } // Process ONBUILD triggers if they exist - if nTriggers := len(b.Config.OnBuild); nTriggers != 0 { + if nTriggers := len(b.runConfig.OnBuild); nTriggers != 0 { word := "trigger" if nTriggers > 1 { word = "triggers" } - fmt.Fprintf(b.ErrStream, "# Executing %d build %s...\n", nTriggers, word) + fmt.Fprintf(b.Stderr, "# Executing %d build %s...\n", nTriggers, word) } // Copy the ONBUILD triggers, and remove them from the config, since the config will be committed. - onBuildTriggers := b.Config.OnBuild - b.Config.OnBuild = []string{} + onBuildTriggers := b.runConfig.OnBuild + b.runConfig.OnBuild = []string{} // parse the ONBUILD triggers by invoking the parser for _, step := range onBuildTriggers { @@ -571,86 +465,86 @@ func (b *builder) processImageFrom(img *image.Image) error { return nil } -// probeCache checks to see if image-caching is enabled (`b.UtilizeCache`) -// and if so attempts to look up the current `b.image` and `b.Config` pair -// in the current server `b.Daemon`. If an image is found, probeCache returns -// `(true, nil)`. If no image is found, it returns `(false, nil)`. If there -// is any error, it returns `(false, err)`. -func (b *builder) probeCache() (bool, error) { - if !b.UtilizeCache || b.cacheBusted { +// probeCache checks if `b.docker` implements builder.ImageCache and image-caching +// is enabled (`b.UseCache`). +// If so attempts to look up the current `b.image` and `b.runConfig` pair with `b.docker`. +// If an image is found, probeCache returns `(true, nil)`. +// If no image is found, it returns `(false, nil)`. +// If there is any error, it returns `(false, err)`. +func (b *Builder) probeCache() (bool, error) { + c, ok := b.docker.(builder.ImageCache) + if !ok || !b.UseCache || b.cacheBusted { return false, nil } - - cache, err := b.Daemon.ImageGetCached(b.image, b.Config) + cache, err := c.GetCachedImage(b.image, b.runConfig) if err != nil { return false, err } - if cache == nil { + if len(cache) == 0 { logrus.Debugf("[BUILDER] Cache miss") b.cacheBusted = true return false, nil } - fmt.Fprintf(b.OutStream, " ---> Using cache\n") + fmt.Fprintf(b.Stdout, " ---> Using cache\n") logrus.Debugf("[BUILDER] Use cached version") - b.image = cache.ID - b.Daemon.Graph().Retain(b.id, cache.ID) - b.activeImages = append(b.activeImages, cache.ID) + b.image = string(cache) + + // TODO: remove once Commit can take a tag parameter. + b.docker.Retain(b.id, b.image) + b.activeImages = append(b.activeImages, b.image) + return true, nil } -func (b *builder) create() (*daemon.Container, error) { +func (b *Builder) create() (*daemon.Container, error) { if b.image == "" && !b.noBaseImage { return nil, fmt.Errorf("Please provide a source image with `from` prior to run") } - b.Config.Image = b.image + b.runConfig.Image = b.image + // TODO: why not embed a hostconfig in builder? hostConfig := &runconfig.HostConfig{ - CPUShares: b.cpuShares, - CPUPeriod: b.cpuPeriod, - CPUQuota: b.cpuQuota, - CpusetCpus: b.cpuSetCpus, - CpusetMems: b.cpuSetMems, - CgroupParent: b.cgroupParent, - Memory: b.memory, - MemorySwap: b.memorySwap, - Ulimits: b.ulimits, + CPUShares: b.CPUShares, + CPUPeriod: b.CPUPeriod, + CPUQuota: b.CPUQuota, + CpusetCpus: b.CPUSetCpus, + CpusetMems: b.CPUSetMems, + CgroupParent: b.CgroupParent, + Memory: b.Memory, + MemorySwap: b.MemorySwap, + Ulimits: b.Ulimits, } - config := *b.Config + config := *b.runConfig // Create the container - ccr, err := b.Daemon.ContainerCreate("", b.Config, hostConfig, true) + c, warnings, err := b.docker.Create(b.runConfig, hostConfig) if err != nil { return nil, err } - for _, warning := range ccr.Warnings { - fmt.Fprintf(b.OutStream, " ---> [Warning] %s\n", warning) - } - c, err := b.Daemon.Get(ccr.ID) - if err != nil { - return nil, err + defer c.Unmount() + for _, warning := range warnings { + fmt.Fprintf(b.Stdout, " ---> [Warning] %s\n", warning) } - b.TmpContainers[c.ID] = struct{}{} - fmt.Fprintf(b.OutStream, " ---> Running in %s\n", stringid.TruncateID(c.ID)) + b.tmpContainers[c.ID] = struct{}{} + fmt.Fprintf(b.Stdout, " ---> Running in %s\n", stringid.TruncateID(c.ID)) if config.Cmd.Len() > 0 { // override the entry point that may have been picked up from the base image s := config.Cmd.Slice() c.Path = s[0] c.Args = s[1:] - } else { - config.Cmd = stringutils.NewStrSlice() } return c, nil } -func (b *builder) run(c *daemon.Container) error { +func (b *Builder) run(c *daemon.Container) error { var errCh chan error if b.Verbose { - errCh = c.Attach(nil, b.OutStream, b.ErrStream) + errCh = c.Attach(nil, b.Stdout, b.Stderr) } //start the container @@ -678,8 +572,9 @@ func (b *builder) run(c *daemon.Container) error { // Wait for it to finish if ret, _ := c.WaitStop(-1 * time.Second); ret != 0 { + // TODO: change error type, because jsonmessage.JSONError assumes HTTP return &jsonmessage.JSONError{ - Message: fmt.Sprintf("The command '%s' returned a non-zero code: %d", b.Config.Cmd.ToString(), ret), + Message: fmt.Sprintf("The command '%s' returned a non-zero code: %d", b.runConfig.Cmd.ToString(), ret), Code: ret, } } @@ -687,125 +582,81 @@ func (b *builder) run(c *daemon.Container) error { return nil } -func (b *builder) checkPathForAddition(orig string) error { - origPath := filepath.Join(b.contextPath, orig) - origPath, err := symlink.EvalSymlinks(origPath) - if err != nil { - if os.IsNotExist(err) { - return fmt.Errorf("%s: no such file or directory", orig) - } - return err - } - contextPath, err := symlink.EvalSymlinks(b.contextPath) - if err != nil { - return err - } - if !strings.HasPrefix(origPath, contextPath) { - return fmt.Errorf("Forbidden path outside the build context: %s (%s)", orig, origPath) - } - if _, err := os.Stat(origPath); err != nil { - if os.IsNotExist(err) { - return fmt.Errorf("%s: no such file or directory", orig) - } - return err - } - return nil -} - -func (b *builder) addContext(container *daemon.Container, orig, dest string, decompress bool) error { - var ( - err error - destExists = true - origPath = filepath.Join(b.contextPath, orig) - destPath string - ) - - // Work in daemon-local OS specific file paths - dest = filepath.FromSlash(dest) - - destPath, err = container.GetResourcePath(dest) - if err != nil { - return err - } - - // Preserve the trailing slash - if strings.HasSuffix(dest, string(os.PathSeparator)) || dest == "." { - destPath = destPath + string(os.PathSeparator) - } - - destStat, err := os.Stat(destPath) - if err != nil { - if !os.IsNotExist(err) { - logrus.Errorf("Error performing os.Stat on %s. %s", destPath, err) - return err - } - destExists = false - } - - fi, err := os.Stat(origPath) - if err != nil { - if os.IsNotExist(err) { - return fmt.Errorf("%s: no such file or directory", orig) - } - return err - } - - if fi.IsDir() { - return copyAsDirectory(origPath, destPath, destExists) - } - - // If we are adding a remote file (or we've been told not to decompress), do not try to untar it - if decompress { - // First try to unpack the source as an archive - // to support the untar feature we need to clean up the path a little bit - // because tar is very forgiving. First we need to strip off the archive's - // filename from the path but this is only added if it does not end in slash - tarDest := destPath - if strings.HasSuffix(tarDest, string(os.PathSeparator)) { - tarDest = filepath.Dir(destPath) - } - - // try to successfully untar the orig - if err := chrootarchive.UntarPath(origPath, tarDest); err == nil { - return nil - } else if err != io.EOF { - logrus.Debugf("Couldn't untar %s to %s: %s", origPath, tarDest, err) - } - } - - if err := system.MkdirAll(filepath.Dir(destPath), 0755); err != nil { - return err - } - if err := chrootarchive.CopyWithTar(origPath, destPath); err != nil { - return err - } - - resPath := destPath - if destExists && destStat.IsDir() { - resPath = filepath.Join(destPath, filepath.Base(origPath)) - } - - return fixPermissions(origPath, resPath, 0, 0, destExists) -} - -func copyAsDirectory(source, destination string, destExisted bool) error { - if err := chrootarchive.CopyWithTar(source, destination); err != nil { - return err - } - return fixPermissions(source, destination, 0, 0, destExisted) -} - -func (b *builder) clearTmp() { - for c := range b.TmpContainers { +func (b *Builder) clearTmp() { + for c := range b.tmpContainers { rmConfig := &daemon.ContainerRmConfig{ ForceRemove: true, RemoveVolume: true, } - if err := b.Daemon.ContainerRm(c, rmConfig); err != nil { - fmt.Fprintf(b.OutStream, "Error removing intermediate container %s: %v\n", stringid.TruncateID(c), err) + if err := b.docker.Remove(c, rmConfig); err != nil { + fmt.Fprintf(b.Stdout, "Error removing intermediate container %s: %v\n", stringid.TruncateID(c), err) return } - delete(b.TmpContainers, c) - fmt.Fprintf(b.OutStream, "Removing intermediate container %s\n", stringid.TruncateID(c)) + delete(b.tmpContainers, c) + fmt.Fprintf(b.Stdout, "Removing intermediate container %s\n", stringid.TruncateID(c)) } } + +// readDockerfile reads a Dockerfile from the current context. +func (b *Builder) readDockerfile() error { + // If no -f was specified then look for 'Dockerfile'. If we can't find + // that then look for 'dockerfile'. If neither are found then default + // back to 'Dockerfile' and use that in the error message. + if b.DockerfileName == "" { + b.DockerfileName = api.DefaultDockerfileName + if _, err := b.context.Stat(b.DockerfileName); os.IsNotExist(err) { + lowercase := strings.ToLower(b.DockerfileName) + if _, err := b.context.Stat(lowercase); err == nil { + b.DockerfileName = lowercase + } + } + } + + f, err := b.context.Open(b.DockerfileName) + if err != nil { + if os.IsNotExist(err) { + return fmt.Errorf("Cannot locate specified Dockerfile: %s", b.DockerfileName) + } + return err + } + if f, ok := f.(*os.File); ok { + // ignoring error because Open already succeeded + fi, err := f.Stat() + if err != nil { + return fmt.Errorf("Unexpected error reading Dockerfile: %v", err) + } + if fi.Size() == 0 { + return fmt.Errorf("The Dockerfile (%s) cannot be empty", b.DockerfileName) + } + } + b.dockerfile, err = parser.Parse(f) + f.Close() + if err != nil { + return err + } + + // After the Dockerfile has been parsed, we need to check the .dockerignore + // file for either "Dockerfile" or ".dockerignore", and if either are + // present then erase them from the build context. These files should never + // have been sent from the client but we did send them to make sure that + // we had the Dockerfile to actually parse, and then we also need the + // .dockerignore file to know whether either file should be removed. + // Note that this assumes the Dockerfile has been read into memory and + // is now safe to be removed. + if dockerIgnore, ok := b.context.(builder.DockerIgnoreContext); ok { + dockerIgnore.Process([]string{b.DockerfileName}) + } + return nil +} + +// determine if build arg is part of built-in args or user +// defined args in Dockerfile at any point in time. +func (b *Builder) isBuildArgAllowed(arg string) bool { + if _, ok := BuiltinAllowedBuildArgs[arg]; ok { + return true + } + if _, ok := b.allowedBuildArgs[arg]; ok { + return true + } + return false +} diff --git a/builder/dockerfile/internals_unix.go b/builder/dockerfile/internals_unix.go index 5a35f86c1f..2b2eee5114 100644 --- a/builder/dockerfile/internals_unix.go +++ b/builder/dockerfile/internals_unix.go @@ -1,17 +1,12 @@ -// +build freebsd linux +// +build !windows package dockerfile import ( - "io/ioutil" "os" "path/filepath" ) -func getTempDir(dir, prefix string) (string, error) { - return ioutil.TempDir(dir, prefix) -} - func fixPermissions(source, destination string, uid, gid int, destExisted bool) error { // If the destination didn't already exist, or the destination isn't a // directory, then we should Lchown the destination. Otherwise, we shouldn't diff --git a/builder/dockerfile/internals_windows.go b/builder/dockerfile/internals_windows.go index 8473e48008..78664246df 100644 --- a/builder/dockerfile/internals_windows.go +++ b/builder/dockerfile/internals_windows.go @@ -2,20 +2,6 @@ package dockerfile -import ( - "io/ioutil" - - "github.com/docker/docker/pkg/longpath" -) - -func getTempDir(dir, prefix string) (string, error) { - tempDir, err := ioutil.TempDir(dir, prefix) - if err != nil { - return "", err - } - return longpath.AddPrefix(tempDir), nil -} - func fixPermissions(source, destination string, uid, gid int, destExisted bool) error { // chown is not supported on Windows return nil diff --git a/builder/dockerfile/job.go b/builder/dockerfile/job.go deleted file mode 100644 index 1265f5bb61..0000000000 --- a/builder/dockerfile/job.go +++ /dev/null @@ -1,376 +0,0 @@ -package dockerfile - -import ( - "bytes" - "errors" - "fmt" - "io" - "io/ioutil" - "os" - "runtime" - "strings" - "sync" - - "github.com/docker/docker/api" - "github.com/docker/docker/builder/dockerfile/parser" - "github.com/docker/docker/cliconfig" - "github.com/docker/docker/daemon" - "github.com/docker/docker/graph/tags" - "github.com/docker/docker/pkg/archive" - "github.com/docker/docker/pkg/httputils" - "github.com/docker/docker/pkg/parsers" - "github.com/docker/docker/pkg/progressreader" - "github.com/docker/docker/pkg/streamformatter" - "github.com/docker/docker/pkg/stringid" - "github.com/docker/docker/pkg/ulimit" - "github.com/docker/docker/pkg/urlutil" - "github.com/docker/docker/registry" - "github.com/docker/docker/runconfig" - "github.com/docker/docker/utils" -) - -// When downloading remote contexts, limit the amount (in bytes) -// to be read from the response body in order to detect its Content-Type -const maxPreambleLength = 100 - -// whitelist of commands allowed for a commit/import -var validCommitCommands = map[string]bool{ - "cmd": true, - "entrypoint": true, - "env": true, - "expose": true, - "label": true, - "onbuild": true, - "user": true, - "volume": true, - "workdir": true, -} - -// BuiltinAllowedBuildArgs is list of built-in allowed build args -var BuiltinAllowedBuildArgs = map[string]bool{ - "HTTP_PROXY": true, - "http_proxy": true, - "HTTPS_PROXY": true, - "https_proxy": true, - "FTP_PROXY": true, - "ftp_proxy": true, - "NO_PROXY": true, - "no_proxy": true, -} - -// Config contains all configs for a build job -type Config struct { - DockerfileName string - RemoteURL string - RepoName string - SuppressOutput bool - NoCache bool - Remove bool - ForceRemove bool - Pull bool - Memory int64 - MemorySwap int64 - CPUShares int64 - CPUPeriod int64 - CPUQuota int64 - CPUSetCpus string - CPUSetMems string - CgroupParent string - Ulimits []*ulimit.Ulimit - AuthConfigs map[string]cliconfig.AuthConfig - BuildArgs map[string]string - - Stdout io.Writer - Context io.ReadCloser - // When closed, the job has been cancelled. - // Note: not all jobs implement cancellation. - // See Job.Cancel() and Job.WaitCancelled() - cancelled chan struct{} - cancelOnce sync.Once -} - -// Cancel signals the build job to cancel -func (b *Config) Cancel() { - b.cancelOnce.Do(func() { - close(b.cancelled) - }) -} - -// WaitCancelled returns a channel which is closed ("never blocks") when -// the job is cancelled. -func (b *Config) WaitCancelled() <-chan struct{} { - return b.cancelled -} - -// NewBuildConfig returns a new Config struct -func NewBuildConfig() *Config { - return &Config{ - AuthConfigs: map[string]cliconfig.AuthConfig{}, - cancelled: make(chan struct{}), - } -} - -// Build is the main interface of the package, it gathers the Builder -// struct and calls builder.Run() to do all the real build job. -func Build(d *daemon.Daemon, buildConfig *Config) error { - var ( - repoName string - tag string - context io.ReadCloser - ) - sf := streamformatter.NewJSONStreamFormatter() - - repoName, tag = parsers.ParseRepositoryTag(buildConfig.RepoName) - if repoName != "" { - if err := registry.ValidateRepositoryName(repoName); err != nil { - return err - } - if len(tag) > 0 { - if err := tags.ValidateTagName(tag); err != nil { - return err - } - } - } - - if buildConfig.RemoteURL == "" { - context = ioutil.NopCloser(buildConfig.Context) - } else if urlutil.IsGitURL(buildConfig.RemoteURL) { - root, err := utils.GitClone(buildConfig.RemoteURL) - if err != nil { - return err - } - defer os.RemoveAll(root) - - c, err := archive.Tar(root, archive.Uncompressed) - if err != nil { - return err - } - context = c - } else if urlutil.IsURL(buildConfig.RemoteURL) { - f, err := httputils.Download(buildConfig.RemoteURL) - if err != nil { - return fmt.Errorf("Error downloading remote context %s: %v", buildConfig.RemoteURL, err) - } - defer f.Body.Close() - ct := f.Header.Get("Content-Type") - clen := f.ContentLength - contentType, bodyReader, err := inspectResponse(ct, f.Body, clen) - - defer bodyReader.Close() - - if err != nil { - return fmt.Errorf("Error detecting content type for remote %s: %v", buildConfig.RemoteURL, err) - } - if contentType == httputils.MimeTypes.TextPlain { - dockerFile, err := ioutil.ReadAll(bodyReader) - if err != nil { - return err - } - - // When we're downloading just a Dockerfile put it in - // the default name - don't allow the client to move/specify it - buildConfig.DockerfileName = api.DefaultDockerfileName - - c, err := archive.Generate(buildConfig.DockerfileName, string(dockerFile)) - if err != nil { - return err - } - context = c - } else { - // Pass through - this is a pre-packaged context, presumably - // with a Dockerfile with the right name inside it. - prCfg := progressreader.Config{ - In: bodyReader, - Out: buildConfig.Stdout, - Formatter: sf, - Size: clen, - NewLines: true, - ID: "Downloading context", - Action: buildConfig.RemoteURL, - } - context = progressreader.New(prCfg) - } - } - - defer context.Close() - - builder := &builder{ - Daemon: d, - OutStream: &streamformatter.StdoutFormatter{ - Writer: buildConfig.Stdout, - StreamFormatter: sf, - }, - ErrStream: &streamformatter.StderrFormatter{ - Writer: buildConfig.Stdout, - StreamFormatter: sf, - }, - Verbose: !buildConfig.SuppressOutput, - UtilizeCache: !buildConfig.NoCache, - Remove: buildConfig.Remove, - ForceRemove: buildConfig.ForceRemove, - Pull: buildConfig.Pull, - OutOld: buildConfig.Stdout, - StreamFormatter: sf, - AuthConfigs: buildConfig.AuthConfigs, - dockerfileName: buildConfig.DockerfileName, - cpuShares: buildConfig.CPUShares, - cpuPeriod: buildConfig.CPUPeriod, - cpuQuota: buildConfig.CPUQuota, - cpuSetCpus: buildConfig.CPUSetCpus, - cpuSetMems: buildConfig.CPUSetMems, - cgroupParent: buildConfig.CgroupParent, - memory: buildConfig.Memory, - memorySwap: buildConfig.MemorySwap, - ulimits: buildConfig.Ulimits, - cancelled: buildConfig.WaitCancelled(), - id: stringid.GenerateRandomID(), - buildArgs: buildConfig.BuildArgs, - allowedBuildArgs: make(map[string]bool), - } - - defer func() { - builder.Daemon.Graph().Release(builder.id, builder.activeImages...) - }() - - id, err := builder.Run(context) - if err != nil { - return err - } - if repoName != "" { - return d.Repositories().Tag(repoName, tag, id, true) - } - return nil -} - -// BuildFromConfig will do build directly from parameter 'changes', which comes -// from Dockerfile entries, it will: -// -// - call parse.Parse() to get AST root from Dockerfile entries -// - do build by calling builder.dispatch() to call all entries' handling routines -func BuildFromConfig(d *daemon.Daemon, c *runconfig.Config, changes []string) (*runconfig.Config, error) { - ast, err := parser.Parse(bytes.NewBufferString(strings.Join(changes, "\n"))) - if err != nil { - return nil, err - } - - // ensure that the commands are valid - for _, n := range ast.Children { - if !validCommitCommands[n.Value] { - return nil, fmt.Errorf("%s is not a valid change command", n.Value) - } - } - - builder := &builder{ - Daemon: d, - Config: c, - OutStream: ioutil.Discard, - ErrStream: ioutil.Discard, - disableCommit: true, - } - - for i, n := range ast.Children { - if err := builder.dispatch(i, n); err != nil { - return nil, err - } - } - - return builder.Config, nil -} - -// CommitConfig contains build configs for commit operation -type CommitConfig struct { - Pause bool - Repo string - Tag string - Author string - Comment string - Changes []string - Config *runconfig.Config -} - -// Commit will create a new image from a container's changes -func Commit(name string, d *daemon.Daemon, c *CommitConfig) (string, error) { - container, err := d.Get(name) - if err != nil { - return "", err - } - - // It is not possible to commit a running container on Windows - if runtime.GOOS == "windows" && container.IsRunning() { - return "", fmt.Errorf("Windows does not support commit of a running container") - } - - if c.Config == nil { - c.Config = &runconfig.Config{} - } - - newConfig, err := BuildFromConfig(d, c.Config, c.Changes) - if err != nil { - return "", err - } - - if err := runconfig.Merge(newConfig, container.Config); err != nil { - return "", err - } - - commitCfg := &daemon.ContainerCommitConfig{ - Pause: c.Pause, - Repo: c.Repo, - Tag: c.Tag, - Author: c.Author, - Comment: c.Comment, - Config: newConfig, - } - - img, err := d.Commit(container, commitCfg) - if err != nil { - return "", err - } - - return img.ID, nil -} - -// inspectResponse looks into the http response data at r to determine whether its -// content-type is on the list of acceptable content types for remote build contexts. -// This function returns: -// - a string representation of the detected content-type -// - an io.Reader for the response body -// - an error value which will be non-nil either when something goes wrong while -// reading bytes from r or when the detected content-type is not acceptable. -func inspectResponse(ct string, r io.ReadCloser, clen int64) (string, io.ReadCloser, error) { - plen := clen - if plen <= 0 || plen > maxPreambleLength { - plen = maxPreambleLength - } - - preamble := make([]byte, plen, plen) - rlen, err := r.Read(preamble) - if rlen == 0 { - return ct, r, errors.New("Empty response") - } - if err != nil && err != io.EOF { - return ct, r, err - } - - preambleR := bytes.NewReader(preamble) - bodyReader := ioutil.NopCloser(io.MultiReader(preambleR, r)) - // Some web servers will use application/octet-stream as the default - // content type for files without an extension (e.g. 'Dockerfile') - // so if we receive this value we better check for text content - contentType := ct - if len(ct) == 0 || ct == httputils.MimeTypes.OctetStream { - contentType, _, err = httputils.DetectContentType(preamble) - if err != nil { - return contentType, bodyReader, err - } - } - - contentType = selectAcceptableMIME(contentType) - var cterr error - if len(contentType) == 0 { - cterr = fmt.Errorf("unsupported Content-Type %q", ct) - contentType = ct - } - - return contentType, bodyReader, cterr -} diff --git a/builder/dockerfile/support.go b/builder/dockerfile/support.go index 1d793b17f4..38897b2cad 100644 --- a/builder/dockerfile/support.go +++ b/builder/dockerfile/support.go @@ -1,17 +1,6 @@ package dockerfile -import ( - "regexp" - "strings" -) - -const acceptableRemoteMIME = `(?:application/(?:(?:x\-)?tar|octet\-stream|((?:x\-)?(?:gzip|bzip2?|xz)))|(?:text/plain))` - -var mimeRe = regexp.MustCompile(acceptableRemoteMIME) - -func selectAcceptableMIME(ct string) string { - return mimeRe.FindString(ct) -} +import "strings" func handleJSONArgs(args []string, attributes map[string]bool) []string { if len(args) == 0 { diff --git a/builder/dockerfile/support_test.go b/builder/dockerfile/support_test.go deleted file mode 100644 index 05b44bae7b..0000000000 --- a/builder/dockerfile/support_test.go +++ /dev/null @@ -1,41 +0,0 @@ -package dockerfile - -import ( - "fmt" - "testing" -) - -func TestSelectAcceptableMIME(t *testing.T) { - validMimeStrings := []string{ - "application/x-bzip2", - "application/bzip2", - "application/gzip", - "application/x-gzip", - "application/x-xz", - "application/xz", - "application/tar", - "application/x-tar", - "application/octet-stream", - "text/plain", - } - - invalidMimeStrings := []string{ - "", - "application/octet", - "application/json", - } - - for _, m := range invalidMimeStrings { - if len(selectAcceptableMIME(m)) > 0 { - err := fmt.Errorf("Should not have accepted %q", m) - t.Fatal(err) - } - } - - for _, m := range validMimeStrings { - if str := selectAcceptableMIME(m); str == "" { - err := fmt.Errorf("Should have accepted %q", m) - t.Fatal(err) - } - } -} diff --git a/builder/dockerignore.go b/builder/dockerignore.go new file mode 100644 index 0000000000..7630274c99 --- /dev/null +++ b/builder/dockerignore.go @@ -0,0 +1,47 @@ +package builder + +import ( + "os" + + "github.com/docker/docker/pkg/fileutils" + "github.com/docker/docker/utils" +) + +// DockerIgnoreContext wraps a ModifiableContext to add a method +// for handling the .dockerignore file at the root of the context. +type DockerIgnoreContext struct { + ModifiableContext +} + +// Process reads the .dockerignore file at the root of the embedded context. +// If .dockerignore does not exist in the context, then nil is returned. +// +// It can take a list of files to be removed after .dockerignore is removed. +// This is used for server-side implementations of builders that need to send +// the .dockerignore file as well as the special files specified in filesToRemove, +// but expect them to be excluded from the context after they were processed. +// +// For example, server-side Dockerfile builders are expected to pass in the name +// of the Dockerfile to be removed after it was parsed. +// +// TODO: Don't require a ModifiableContext (use Context instead) and don't remove +// files, instead handle a list of files to be excluded from the context. +func (c DockerIgnoreContext) Process(filesToRemove []string) error { + dockerignore, err := c.Open(".dockerignore") + // Note that a missing .dockerignore file isn't treated as an error + if err != nil { + if os.IsNotExist(err) { + return nil + } + return err + } + excludes, _ := utils.ReadDockerIgnore(dockerignore) + filesToRemove = append([]string{".dockerignore"}, filesToRemove...) + for _, fileToRemove := range filesToRemove { + rm, _ := fileutils.Matches(fileToRemove, excludes) + if rm { + c.Remove(fileToRemove) + } + } + return nil +} diff --git a/builder/git.go b/builder/git.go new file mode 100644 index 0000000000..e857060cc0 --- /dev/null +++ b/builder/git.go @@ -0,0 +1,28 @@ +package builder + +import ( + "os" + + "github.com/docker/docker/pkg/archive" + "github.com/docker/docker/utils" +) + +// MakeGitContext returns a Context from gitURL that is cloned in a temporary directory. +func MakeGitContext(gitURL string) (ModifiableContext, error) { + root, err := utils.GitClone(gitURL) + if err != nil { + return nil, err + } + + c, err := archive.Tar(root, archive.Uncompressed) + if err != nil { + return nil, err + } + + defer func() { + // TODO: print errors? + c.Close() + os.RemoveAll(root) + }() + return MakeTarSumContext(c) +} diff --git a/builder/remote.go b/builder/remote.go new file mode 100644 index 0000000000..b43bd32841 --- /dev/null +++ b/builder/remote.go @@ -0,0 +1,115 @@ +package builder + +import ( + "bytes" + "errors" + "fmt" + "io" + "io/ioutil" + "regexp" + + "github.com/docker/docker/pkg/httputils" +) + +// When downloading remote contexts, limit the amount (in bytes) +// to be read from the response body in order to detect its Content-Type +const maxPreambleLength = 100 + +const acceptableRemoteMIME = `(?:application/(?:(?:x\-)?tar|octet\-stream|((?:x\-)?(?:gzip|bzip2?|xz)))|(?:text/plain))` + +var mimeRe = regexp.MustCompile(acceptableRemoteMIME) + +// MakeRemoteContext downloads a context from remoteURL and returns it. +// +// If contentTypeHandlers is non-nil, then the Content-Type header is read along with a maximum of +// maxPreambleLength bytes from the body to help detecting the MIME type. +// Look at acceptableRemoteMIME for more details. +// +// If a match is found, then the body is sent to the contentType handler and a (potentially compressed) tar stream is expected +// to be returned. If no match is found, it is assumed the body is a tar stream (compressed or not). +// In either case, an (assumed) tar stream is passed to MakeTarSumContext whose result is returned. +func MakeRemoteContext(remoteURL string, contentTypeHandlers map[string]func(io.ReadCloser) (io.ReadCloser, error)) (ModifiableContext, error) { + f, err := httputils.Download(remoteURL) + if err != nil { + return nil, fmt.Errorf("Error downloading remote context %s: %v", remoteURL, err) + } + defer f.Body.Close() + + var contextReader io.ReadCloser + if contentTypeHandlers != nil { + contentType := f.Header.Get("Content-Type") + clen := f.ContentLength + + contentType, contextReader, err = inspectResponse(contentType, f.Body, clen) + if err != nil { + return nil, fmt.Errorf("Error detecting content type for remote %s: %v", remoteURL, err) + } + defer contextReader.Close() + + // This loop tries to find a content-type handler for the detected content-type. + // If it could not find one from the caller-supplied map, it tries the empty content-type `""` + // which is interpreted as a fallback handler (usually used for raw tar contexts). + for _, ct := range []string{contentType, ""} { + if fn, ok := contentTypeHandlers[ct]; ok { + defer contextReader.Close() + if contextReader, err = fn(contextReader); err != nil { + return nil, err + } + break + } + } + } + + // Pass through - this is a pre-packaged context, presumably + // with a Dockerfile with the right name inside it. + return MakeTarSumContext(contextReader) +} + +// inspectResponse looks into the http response data at r to determine whether its +// content-type is on the list of acceptable content types for remote build contexts. +// This function returns: +// - a string representation of the detected content-type +// - an io.Reader for the response body +// - an error value which will be non-nil either when something goes wrong while +// reading bytes from r or when the detected content-type is not acceptable. +func inspectResponse(ct string, r io.ReadCloser, clen int64) (string, io.ReadCloser, error) { + plen := clen + if plen <= 0 || plen > maxPreambleLength { + plen = maxPreambleLength + } + + preamble := make([]byte, plen, plen) + rlen, err := r.Read(preamble) + if rlen == 0 { + return ct, r, errors.New("Empty response") + } + if err != nil && err != io.EOF { + return ct, r, err + } + + preambleR := bytes.NewReader(preamble) + bodyReader := ioutil.NopCloser(io.MultiReader(preambleR, r)) + // Some web servers will use application/octet-stream as the default + // content type for files without an extension (e.g. 'Dockerfile') + // so if we receive this value we better check for text content + contentType := ct + if len(ct) == 0 || ct == httputils.MimeTypes.OctetStream { + contentType, _, err = httputils.DetectContentType(preamble) + if err != nil { + return contentType, bodyReader, err + } + } + + contentType = selectAcceptableMIME(contentType) + var cterr error + if len(contentType) == 0 { + cterr = fmt.Errorf("unsupported Content-Type %q", ct) + contentType = ct + } + + return contentType, bodyReader, cterr +} + +func selectAcceptableMIME(ct string) string { + return mimeRe.FindString(ct) +} diff --git a/builder/dockerfile/job_test.go b/builder/remote_test.go similarity index 81% rename from builder/dockerfile/job_test.go rename to builder/remote_test.go index 19c1846541..b6e846be60 100644 --- a/builder/dockerfile/job_test.go +++ b/builder/remote_test.go @@ -1,7 +1,8 @@ -package dockerfile +package builder import ( "bytes" + "fmt" "io/ioutil" "testing" ) @@ -9,6 +10,41 @@ import ( var textPlainDockerfile = "FROM busybox" var binaryContext = []byte{0xFD, 0x37, 0x7A, 0x58, 0x5A, 0x00} //xz magic +func TestSelectAcceptableMIME(t *testing.T) { + validMimeStrings := []string{ + "application/x-bzip2", + "application/bzip2", + "application/gzip", + "application/x-gzip", + "application/x-xz", + "application/xz", + "application/tar", + "application/x-tar", + "application/octet-stream", + "text/plain", + } + + invalidMimeStrings := []string{ + "", + "application/octet", + "application/json", + } + + for _, m := range invalidMimeStrings { + if len(selectAcceptableMIME(m)) > 0 { + err := fmt.Errorf("Should not have accepted %q", m) + t.Fatal(err) + } + } + + for _, m := range validMimeStrings { + if str := selectAcceptableMIME(m); str == "" { + err := fmt.Errorf("Should have accepted %q", m) + t.Fatal(err) + } + } +} + func TestInspectEmptyResponse(t *testing.T) { ct := "application/octet-stream" br := ioutil.NopCloser(bytes.NewReader([]byte(""))) diff --git a/builder/tarsum.go b/builder/tarsum.go new file mode 100644 index 0000000000..b020972657 --- /dev/null +++ b/builder/tarsum.go @@ -0,0 +1,165 @@ +package builder + +import ( + "fmt" + "io" + "os" + "path/filepath" + "strings" + + "github.com/docker/docker/pkg/archive" + "github.com/docker/docker/pkg/chrootarchive" + "github.com/docker/docker/pkg/ioutils" + "github.com/docker/docker/pkg/symlink" + "github.com/docker/docker/pkg/tarsum" +) + +type tarSumContext struct { + root string + sums tarsum.FileInfoSums +} + +func (c *tarSumContext) Close() error { + return os.RemoveAll(c.root) +} + +func convertPathError(err error, cleanpath string) error { + if err, ok := err.(*os.PathError); ok { + err.Path = cleanpath + return err + } + return err +} + +func (c *tarSumContext) Open(path string) (io.ReadCloser, error) { + cleanpath, fullpath, err := c.normalize(path) + if err != nil { + return nil, err + } + r, err := os.Open(fullpath) + if err != nil { + return nil, convertPathError(err, cleanpath) + } + return r, nil +} + +func (c *tarSumContext) Stat(path string) (fi FileInfo, err error) { + cleanpath, fullpath, err := c.normalize(path) + if err != nil { + return nil, err + } + + st, err := os.Lstat(fullpath) + if err != nil { + return nil, convertPathError(err, cleanpath) + } + + fi = PathFileInfo{st, fullpath} + // we set sum to path by default for the case where GetFile returns nil. + // The usual case is if cleanpath is empty. + sum := path + if tsInfo := c.sums.GetFile(cleanpath); tsInfo != nil { + sum = tsInfo.Sum() + } + fi = &HashedFileInfo{fi, sum} + return fi, nil +} + +// MakeTarSumContext returns a build Context from a tar stream. +// +// It extracts the tar stream to a temporary folder that is deleted as soon as +// the Context is closed. +// As the extraction happens, a tarsum is calculated for every file, and the set of +// all those sums then becomes the source of truth for all operations on this Context. +// +// Closing tarStream has to be done by the caller. +func MakeTarSumContext(tarStream io.Reader) (ModifiableContext, error) { + root, err := ioutils.TempDir("", "docker-builder") + if err != nil { + return nil, err + } + + tsc := &tarSumContext{root: root} + + // Make sure we clean-up upon error. In the happy case the caller + // is expected to manage the clean-up + defer func() { + if err != nil { + tsc.Close() + } + }() + + decompressedStream, err := archive.DecompressStream(tarStream) + if err != nil { + return nil, err + } + + sum, err := tarsum.NewTarSum(decompressedStream, true, tarsum.Version1) + if err != nil { + return nil, err + } + + if err := chrootarchive.Untar(sum, root, nil); err != nil { + return nil, err + } + + tsc.sums = sum.GetSums() + + return tsc, nil +} + +func (c *tarSumContext) normalize(path string) (cleanpath, fullpath string, err error) { + cleanpath = filepath.Clean(string(os.PathSeparator) + path)[1:] + fullpath, err = symlink.FollowSymlinkInScope(filepath.Join(c.root, path), c.root) + if err != nil { + return "", "", fmt.Errorf("Forbidden path outside the build context: %s (%s)", path, fullpath) + } + _, err = os.Stat(fullpath) + if err != nil { + return "", "", convertPathError(err, path) + } + return +} + +func (c *tarSumContext) Walk(root string, walkFn WalkFunc) error { + for _, tsInfo := range c.sums { + path := tsInfo.Name() + path, fullpath, err := c.normalize(path) + if err != nil { + return err + } + + // Any file in the context that starts with the given path will be + // picked up and its hashcode used. However, we'll exclude the + // root dir itself. We do this for a coupel of reasons: + // 1 - ADD/COPY will not copy the dir itself, just its children + // so there's no reason to include it in the hash calc + // 2 - the metadata on the dir will change when any child file + // changes. This will lead to a miss in the cache check if that + // child file is in the .dockerignore list. + if rel, err := filepath.Rel(root, path); err != nil { + return err + } else if rel == "." || strings.HasPrefix(rel, ".."+string(os.PathSeparator)) { + continue + } + + info, err := os.Lstat(fullpath) + if err != nil { + return convertPathError(err, path) + } + // TODO check context breakout? + fi := &HashedFileInfo{PathFileInfo{info, fullpath}, tsInfo.Sum()} + if err := walkFn(path, fi, nil); err != nil { + return err + } + } + return nil +} + +func (c *tarSumContext) Remove(path string) error { + _, fullpath, err := c.normalize(path) + if err != nil { + return err + } + return os.RemoveAll(fullpath) +} diff --git a/daemon/create.go b/daemon/create.go index de8e2bf2fa..13ba5baba7 100644 --- a/daemon/create.go +++ b/daemon/create.go @@ -27,7 +27,7 @@ func (daemon *Daemon) ContainerCreate(name string, config *runconfig.Config, hos daemon.adaptContainerSettings(hostConfig, adjustCPUShares) - container, buildWarnings, err := daemon.Create(config, hostConfig, name) + container, err := daemon.Create(config, hostConfig, name) if err != nil { if daemon.Graph().IsNotExist(err, config.Image) { if strings.Contains(config.Image, "@") { @@ -42,16 +42,13 @@ func (daemon *Daemon) ContainerCreate(name string, config *runconfig.Config, hos return types.ContainerCreateResponse{"", warnings}, err } - warnings = append(warnings, buildWarnings...) - return types.ContainerCreateResponse{container.ID, warnings}, nil } // Create creates a new container from the given configuration with a given name. -func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.HostConfig, name string) (retC *Container, retS []string, retErr error) { +func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.HostConfig, name string) (retC *Container, retErr error) { var ( container *Container - warnings []string img *image.Image imgID string err error @@ -60,16 +57,16 @@ func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.Hos if config.Image != "" { img, err = daemon.repositories.LookupImage(config.Image) if err != nil { - return nil, nil, err + return nil, err } if err = daemon.graph.CheckDepth(img); err != nil { - return nil, nil, err + return nil, err } imgID = img.ID } if err := daemon.mergeAndVerifyConfig(config, img); err != nil { - return nil, nil, err + return nil, err } if hostConfig == nil { @@ -78,11 +75,11 @@ func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.Hos if hostConfig.SecurityOpt == nil { hostConfig.SecurityOpt, err = daemon.generateSecurityOpt(hostConfig.IpcMode, hostConfig.PidMode) if err != nil { - return nil, nil, err + return nil, err } } if container, err = daemon.newContainer(name, config, imgID); err != nil { - return nil, nil, err + return nil, err } defer func() { if retErr != nil { @@ -93,13 +90,13 @@ func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.Hos }() if err := daemon.Register(container); err != nil { - return nil, nil, err + return nil, err } if err := daemon.createRootfs(container); err != nil { - return nil, nil, err + return nil, err } if err := daemon.setHostConfig(container, hostConfig); err != nil { - return nil, nil, err + return nil, err } defer func() { if retErr != nil { @@ -109,20 +106,20 @@ func (daemon *Daemon) Create(config *runconfig.Config, hostConfig *runconfig.Hos } }() if err := container.Mount(); err != nil { - return nil, nil, err + return nil, err } defer container.Unmount() if err := createContainerPlatformSpecificSettings(container, config, hostConfig, img); err != nil { - return nil, nil, err + return nil, err } if err := container.toDiskLocking(); err != nil { logrus.Errorf("Error saving new container to disk: %v", err) - return nil, nil, err + return nil, err } container.logEvent("create") - return container, warnings, nil + return container, nil } func (daemon *Daemon) generateSecurityOpt(ipcMode runconfig.IpcMode, pidMode runconfig.PidMode) ([]string, error) { diff --git a/daemon/daemonbuilder/builder.go b/daemon/daemonbuilder/builder.go new file mode 100644 index 0000000000..829fdfa7e2 --- /dev/null +++ b/daemon/daemonbuilder/builder.go @@ -0,0 +1,238 @@ +package daemonbuilder + +import ( + "fmt" + "io" + "io/ioutil" + "os" + "path/filepath" + "strings" + + "github.com/Sirupsen/logrus" + "github.com/docker/docker/api" + "github.com/docker/docker/builder" + "github.com/docker/docker/cliconfig" + "github.com/docker/docker/daemon" + "github.com/docker/docker/graph" + "github.com/docker/docker/image" + "github.com/docker/docker/pkg/archive" + "github.com/docker/docker/pkg/chrootarchive" + "github.com/docker/docker/pkg/httputils" + "github.com/docker/docker/pkg/ioutils" + "github.com/docker/docker/pkg/parsers" + "github.com/docker/docker/pkg/progressreader" + "github.com/docker/docker/pkg/system" + "github.com/docker/docker/pkg/urlutil" + "github.com/docker/docker/registry" + "github.com/docker/docker/runconfig" +) + +// Docker implements builder.Docker for the docker Daemon object. +type Docker struct { + Daemon *daemon.Daemon + OutOld io.Writer + AuthConfigs map[string]cliconfig.AuthConfig +} + +// ensure Docker implements builder.Docker +var _ builder.Docker = Docker{} + +// LookupImage looks up a Docker image referenced by `name`. +func (d Docker) LookupImage(name string) (*image.Image, error) { + return d.Daemon.Repositories().LookupImage(name) +} + +// Pull tells Docker to pull image referenced by `name`. +func (d Docker) Pull(name string) (*image.Image, error) { + remote, tag := parsers.ParseRepositoryTag(name) + if tag == "" { + tag = "latest" + } + + pullRegistryAuth := &cliconfig.AuthConfig{} + if len(d.AuthConfigs) > 0 { + // The request came with a full auth config file, we prefer to use that + repoInfo, err := d.Daemon.RegistryService.ResolveRepository(remote) + if err != nil { + return nil, err + } + + resolvedConfig := registry.ResolveAuthConfig( + &cliconfig.ConfigFile{AuthConfigs: d.AuthConfigs}, + repoInfo.Index, + ) + pullRegistryAuth = &resolvedConfig + } + + imagePullConfig := &graph.ImagePullConfig{ + AuthConfig: pullRegistryAuth, + OutStream: ioutils.NopWriteCloser(d.OutOld), + } + + if err := d.Daemon.Repositories().Pull(remote, tag, imagePullConfig); err != nil { + return nil, err + } + + return d.Daemon.Repositories().LookupImage(name) +} + +// Container looks up a Docker container referenced by `id`. +func (d Docker) Container(id string) (*daemon.Container, error) { + return d.Daemon.Get(id) +} + +// Create creates a new Docker container and returns potential warnings +func (d Docker) Create(cfg *runconfig.Config, hostCfg *runconfig.HostConfig) (*daemon.Container, []string, error) { + ccr, err := d.Daemon.ContainerCreate("", cfg, hostCfg, true) + if err != nil { + return nil, nil, err + } + container, err := d.Daemon.Get(ccr.ID) + if err != nil { + return nil, ccr.Warnings, err + } + return container, ccr.Warnings, container.Mount() +} + +// Remove removes a container specified by `id`. +func (d Docker) Remove(id string, cfg *daemon.ContainerRmConfig) error { + return d.Daemon.ContainerRm(id, cfg) +} + +// Commit creates a new Docker image from an existing Docker container. +func (d Docker) Commit(c *daemon.Container, cfg *daemon.ContainerCommitConfig) (*image.Image, error) { + return d.Daemon.Commit(c, cfg) +} + +// Retain retains an image avoiding it to be removed or overwritten until a corresponding Release() call. +func (d Docker) Retain(sessionID, imgID string) { + d.Daemon.Graph().Retain(sessionID, imgID) +} + +// Release releases a list of images that were retained for the time of a build. +func (d Docker) Release(sessionID string, activeImages []string) { + d.Daemon.Graph().Release(sessionID, activeImages...) +} + +// Copy copies/extracts a source FileInfo to a destination path inside a container +// specified by a container object. +// TODO: make sure callers don't unnecessarily convert destPath with filepath.FromSlash (Copy does it already). +// Copy should take in abstract paths (with slashes) and the implementation should convert it to OS-specific paths. +func (d Docker) Copy(c *daemon.Container, destPath string, src builder.FileInfo, decompress bool) error { + srcPath := src.Path() + destExists := true + + // Work in daemon-local OS specific file paths + destPath = filepath.FromSlash(destPath) + + dest, err := c.GetResourcePath(destPath) + if err != nil { + return err + } + + // Preserve the trailing slash + // TODO: why are we appending another path separator if there was already one? + if strings.HasSuffix(destPath, string(os.PathSeparator)) || destPath == "." { + dest += string(os.PathSeparator) + } + + destPath = dest + + destStat, err := os.Stat(destPath) + if err != nil { + if !os.IsNotExist(err) { + logrus.Errorf("Error performing os.Stat on %s. %s", destPath, err) + return err + } + destExists = false + } + + if src.IsDir() { + // copy as directory + if err := chrootarchive.CopyWithTar(srcPath, destPath); err != nil { + return err + } + return fixPermissions(srcPath, destPath, 0, 0, destExists) + } + if decompress { + // Only try to untar if it is a file and that we've been told to decompress (when ADD-ing a remote file) + + // First try to unpack the source as an archive + // to support the untar feature we need to clean up the path a little bit + // because tar is very forgiving. First we need to strip off the archive's + // filename from the path but this is only added if it does not end in slash + tarDest := destPath + if strings.HasSuffix(tarDest, string(os.PathSeparator)) { + tarDest = filepath.Dir(destPath) + } + + // try to successfully untar the orig + if err := chrootarchive.UntarPath(srcPath, tarDest); err == nil { + return nil + } else if err != io.EOF { + logrus.Debugf("Couldn't untar to %s: %v", tarDest, err) + } + } + + // only needed for fixPermissions, but might as well put it before CopyFileWithTar + if destExists && destStat.IsDir() { + destPath = filepath.Join(destPath, filepath.Base(srcPath)) + } + + if err := system.MkdirAll(filepath.Dir(destPath), 0755); err != nil { + return err + } + if err := chrootarchive.CopyFileWithTar(srcPath, destPath); err != nil { + return err + } + + return fixPermissions(srcPath, destPath, 0, 0, destExists) +} + +// GetCachedImage returns a reference to a cached image whose parent equals `parent` +// and runconfig equals `cfg`. A cache miss is expected to return an empty ID and a nil error. +func (d Docker) GetCachedImage(imgID string, cfg *runconfig.Config) (string, error) { + cache, err := d.Daemon.ImageGetCached(string(imgID), cfg) + if cache == nil || err != nil { + return "", err + } + return cache.ID, nil +} + +// Following is specific to builder contexts + +// DetectContextFromRemoteURL returns a context and in certain cases the name of the dockerfile to be used +// irrespective of user input. +// progressReader is only used if remoteURL is actually a URL (not empty, and not a Git endpoint). +func DetectContextFromRemoteURL(r io.ReadCloser, remoteURL string, progressReader *progressreader.Config) (context builder.ModifiableContext, dockerfileName string, err error) { + switch { + case remoteURL == "": + context, err = builder.MakeTarSumContext(r) + case urlutil.IsGitURL(remoteURL): + context, err = builder.MakeGitContext(remoteURL) + case urlutil.IsURL(remoteURL): + context, err = builder.MakeRemoteContext(remoteURL, map[string]func(io.ReadCloser) (io.ReadCloser, error){ + httputils.MimeTypes.TextPlain: func(rc io.ReadCloser) (io.ReadCloser, error) { + dockerfile, err := ioutil.ReadAll(rc) + if err != nil { + return nil, err + } + + // dockerfileName is set to signal that the remote was interpreted as a single Dockerfile, in which case the caller + // should use dockerfileName as the new name for the Dockerfile, irrespective of any other user input. + dockerfileName = api.DefaultDockerfileName + + // TODO: return a context without tarsum + return archive.Generate(dockerfileName, string(dockerfile)) + }, + // fallback handler (tar context) + "": func(rc io.ReadCloser) (io.ReadCloser, error) { + progressReader.In = rc + return progressReader, nil + }, + }) + default: + err = fmt.Errorf("remoteURL (%s) could not be recognized as URL", remoteURL) + } + return +} diff --git a/daemon/daemonbuilder/builder_unix.go b/daemon/daemonbuilder/builder_unix.go new file mode 100644 index 0000000000..aa63b33b4c --- /dev/null +++ b/daemon/daemonbuilder/builder_unix.go @@ -0,0 +1,40 @@ +// +build freebsd linux + +package daemonbuilder + +import ( + "os" + "path/filepath" +) + +func fixPermissions(source, destination string, uid, gid int, destExisted bool) error { + // If the destination didn't already exist, or the destination isn't a + // directory, then we should Lchown the destination. Otherwise, we shouldn't + // Lchown the destination. + destStat, err := os.Stat(destination) + if err != nil { + // This should *never* be reached, because the destination must've already + // been created while untar-ing the context. + return err + } + doChownDestination := !destExisted || !destStat.IsDir() + + // We Walk on the source rather than on the destination because we don't + // want to change permissions on things we haven't created or modified. + return filepath.Walk(source, func(fullpath string, info os.FileInfo, err error) error { + // Do not alter the walk root iff. it existed before, as it doesn't fall under + // the domain of "things we should chown". + if !doChownDestination && (source == fullpath) { + return nil + } + + // Path is prefixed by source: substitute with destination instead. + cleaned, err := filepath.Rel(source, fullpath) + if err != nil { + return err + } + + fullpath = filepath.Join(destination, cleaned) + return os.Lchown(fullpath, uid, gid) + }) +} diff --git a/daemon/daemonbuilder/builder_windows.go b/daemon/daemonbuilder/builder_windows.go new file mode 100644 index 0000000000..dded0e400d --- /dev/null +++ b/daemon/daemonbuilder/builder_windows.go @@ -0,0 +1,8 @@ +// +build windows + +package daemonbuilder + +func fixPermissions(source, destination string, uid, gid int, destExisted bool) error { + // chown is not supported on Windows + return nil +} diff --git a/integration-cli/docker_api_build_test.go b/integration-cli/docker_api_build_test.go index 78c538b992..41d64ca191 100644 --- a/integration-cli/docker_api_build_test.go +++ b/integration-cli/docker_api_build_test.go @@ -39,7 +39,7 @@ func (s *DockerSuite) TestBuildApiDockerfilePath(c *check.C) { c.Fatal(err) } - if !strings.Contains(string(out), "must be within the build context") { + if !strings.Contains(string(out), "Forbidden path outside the build context") { c.Fatalf("Didn't complain about leaving build context: %s", out) } } diff --git a/pkg/ioutils/temp_unix.go b/pkg/ioutils/temp_unix.go new file mode 100644 index 0000000000..1539ad21b5 --- /dev/null +++ b/pkg/ioutils/temp_unix.go @@ -0,0 +1,10 @@ +// +build !windows + +package ioutils + +import "io/ioutil" + +// TempDir on Unix systems is equivalent to ioutil.TempDir. +func TempDir(dir, prefix string) (string, error) { + return ioutil.TempDir(dir, prefix) +} diff --git a/pkg/ioutils/temp_windows.go b/pkg/ioutils/temp_windows.go new file mode 100644 index 0000000000..c258e5fdd8 --- /dev/null +++ b/pkg/ioutils/temp_windows.go @@ -0,0 +1,18 @@ +// +build windows + +package ioutils + +import ( + "io/ioutil" + + "github.com/docker/docker/pkg/longpath" +) + +// TempDir is the equivalent of ioutil.TempDir, except that the result is in Windows longpath format. +func TempDir(dir, prefix string) (string, error) { + tempDir, err := ioutil.TempDir(dir, prefix) + if err != nil { + return "", err + } + return longpath.AddPrefix(tempDir), nil +} diff --git a/utils/utils.go b/utils/utils.go index c83c355b1c..a17ab9dcea 100644 --- a/utils/utils.go +++ b/utils/utils.go @@ -247,17 +247,11 @@ func ValidateContextDirectory(srcPath string, excludes []string) error { // ReadDockerIgnore reads a .dockerignore file and returns the list of file patterns // to ignore. Note this will trim whitespace from each line as well // as use GO's "clean" func to get the shortest/cleanest path for each. -func ReadDockerIgnore(path string) ([]string, error) { - // Note that a missing .dockerignore file isn't treated as an error - reader, err := os.Open(path) - if err != nil { - if !os.IsNotExist(err) { - return nil, fmt.Errorf("Error reading '%s': %v", path, err) - } +func ReadDockerIgnore(reader io.ReadCloser) ([]string, error) { + if reader == nil { return nil, nil } defer reader.Close() - scanner := bufio.NewScanner(reader) var excludes []string @@ -269,8 +263,8 @@ func ReadDockerIgnore(path string) ([]string, error) { pattern = filepath.Clean(pattern) excludes = append(excludes, pattern) } - if err = scanner.Err(); err != nil { - return nil, fmt.Errorf("Error reading '%s': %v", path, err) + if err := scanner.Err(); err != nil { + return nil, fmt.Errorf("Error reading .dockerignore: %v", err) } return excludes, nil } diff --git a/utils/utils_test.go b/utils/utils_test.go index 2863009423..9acb8017a6 100644 --- a/utils/utils_test.go +++ b/utils/utils_test.go @@ -63,24 +63,27 @@ func TestReadDockerIgnore(t *testing.T) { } defer os.RemoveAll(tmpDir) - diName := filepath.Join(tmpDir, ".dockerignore") - - di, err := ReadDockerIgnore(diName) + di, err := ReadDockerIgnore(nil) if err != nil { - t.Fatalf("Expected not to have error, got %s", err) + t.Fatalf("Expected not to have error, got %v", err) } if diLen := len(di); diLen != 0 { t.Fatalf("Expected to have zero dockerignore entry, got %d", diLen) } + diName := filepath.Join(tmpDir, ".dockerignore") content := fmt.Sprintf("test1\n/test2\n/a/file/here\n\nlastfile") err = ioutil.WriteFile(diName, []byte(content), 0777) if err != nil { t.Fatal(err) } - di, err = ReadDockerIgnore(diName) + diFd, err := os.Open(diName) + if err != nil { + t.Fatal(err) + } + di, err = ReadDockerIgnore(diFd) if err != nil { t.Fatal(err) }