Преглед изворни кода

Merge pull request #33116 from dnephin/refactor-builder-copy

[Builder] Refactor COPY/ADD dispatchers
Vincent Demeester пре 8 година
родитељ
комит
23e857cd3d

+ 5 - 6
builder/dockerfile/builder.go

@@ -98,7 +98,6 @@ type Builder struct {
 	Output io.Writer
 
 	docker    builder.Backend
-	source    builder.Source
 	clientCtx context.Context
 
 	tmpContainers map[string]struct{}
@@ -108,6 +107,7 @@ type Builder struct {
 	buildArgs     *buildArgs
 	imageCache    builder.ImageCache
 	imageSources  *imageSources
+	pathCache     pathCache
 }
 
 // newBuilder creates a new Dockerfile builder from an optional dockerfile and a Options.
@@ -128,6 +128,7 @@ func newBuilder(clientCtx context.Context, options builderOptions) *Builder {
 		buildArgs:     newBuildArgs(config.BuildArgs),
 		buildStages:   newBuildStages(),
 		imageSources:  newImageSources(clientCtx, options),
+		pathCache:     options.PathCache,
 	}
 	return b
 }
@@ -144,9 +145,6 @@ func (b *Builder) resetImageCache() {
 func (b *Builder) build(source builder.Source, dockerfile *parser.Result) (*builder.Result, error) {
 	defer b.imageSources.Unmount()
 
-	// TODO: Remove source field from Builder
-	b.source = source
-
 	addNodesForLabelOption(dockerfile.AST, b.options.Labels)
 
 	if err := checkDispatchDockerfile(dockerfile.AST); err != nil {
@@ -154,7 +152,7 @@ func (b *Builder) build(source builder.Source, dockerfile *parser.Result) (*buil
 		return nil, err
 	}
 
-	dispatchState, err := b.dispatchDockerfileWithCancellation(dockerfile)
+	dispatchState, err := b.dispatchDockerfileWithCancellation(dockerfile, source)
 	if err != nil {
 		return nil, err
 	}
@@ -180,7 +178,7 @@ func emitImageID(aux *streamformatter.AuxFormatter, state *dispatchState) error
 	return aux.Emit(types.BuildResult{ID: state.imageID})
 }
 
-func (b *Builder) dispatchDockerfileWithCancellation(dockerfile *parser.Result) (*dispatchState, error) {
+func (b *Builder) dispatchDockerfileWithCancellation(dockerfile *parser.Result, source builder.Source) (*dispatchState, error) {
 	shlex := NewShellLex(dockerfile.EscapeToken)
 	state := newDispatchState()
 	total := len(dockerfile.AST.Children)
@@ -214,6 +212,7 @@ func (b *Builder) dispatchDockerfileWithCancellation(dockerfile *parser.Result)
 			stepMsg: formatStep(i, total),
 			node:    n,
 			shlex:   shlex,
+			source:  source,
 		}
 		if state, err = b.dispatch(opts); err != nil {
 			if b.options.ForceRemove {

+ 360 - 0
builder/dockerfile/copy.go

@@ -0,0 +1,360 @@
+package dockerfile
+
+import (
+	"fmt"
+	"io"
+	"net/http"
+	"net/url"
+	"os"
+	"path/filepath"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/docker/docker/builder"
+	"github.com/docker/docker/builder/remotecontext"
+	"github.com/docker/docker/pkg/httputils"
+	"github.com/docker/docker/pkg/ioutils"
+	"github.com/docker/docker/pkg/progress"
+	"github.com/docker/docker/pkg/streamformatter"
+	"github.com/docker/docker/pkg/system"
+	"github.com/docker/docker/pkg/urlutil"
+	"github.com/pkg/errors"
+)
+
+type pathCache interface {
+	Load(key interface{}) (value interface{}, ok bool)
+	Store(key, value interface{})
+}
+
+// copyInfo is a data object which stores the metadata about each source file in
+// a copyInstruction
+type copyInfo struct {
+	root string
+	path string
+	hash string
+}
+
+func newCopyInfoFromSource(source builder.Source, path string, hash string) copyInfo {
+	return copyInfo{root: source.Root(), path: path, hash: hash}
+}
+
+func newCopyInfos(copyInfos ...copyInfo) []copyInfo {
+	return copyInfos
+}
+
+// copyInstruction is a fully parsed COPY or ADD command that is passed to
+// Builder.performCopy to copy files into the image filesystem
+type copyInstruction struct {
+	cmdName                 string
+	infos                   []copyInfo
+	dest                    string
+	allowLocalDecompression bool
+}
+
+// copier reads a raw COPY or ADD command, fetches remote sources using a downloader,
+// and creates a copyInstruction
+type copier struct {
+	imageSource *imageMount
+	source      builder.Source
+	pathCache   pathCache
+	download    sourceDownloader
+	tmpPaths    []string
+}
+
+func copierFromDispatchRequest(req dispatchRequest, download sourceDownloader, imageSource *imageMount) copier {
+	return copier{
+		source:      req.source,
+		pathCache:   req.builder.pathCache,
+		download:    download,
+		imageSource: imageSource,
+	}
+}
+
+func (o *copier) createCopyInstruction(args []string, cmdName string) (copyInstruction, error) {
+	inst := copyInstruction{cmdName: cmdName}
+	last := len(args) - 1
+
+	// Work in daemon-specific filepath semantics
+	inst.dest = filepath.FromSlash(args[last])
+
+	infos, err := o.getCopyInfosForSourcePaths(args[0:last])
+	if err != nil {
+		return inst, errors.Wrapf(err, "%s failed", cmdName)
+	}
+	if len(infos) > 1 && !strings.HasSuffix(inst.dest, string(os.PathSeparator)) {
+		return inst, errors.Errorf("When using %s with more than one source file, the destination must be a directory and end with a /", cmdName)
+	}
+	inst.infos = infos
+	return inst, nil
+}
+
+// getCopyInfosForSourcePaths iterates over the source files and calculate the info
+// needed to copy (e.g. hash value if cached)
+func (o *copier) getCopyInfosForSourcePaths(sources []string) ([]copyInfo, error) {
+	var infos []copyInfo
+	for _, orig := range sources {
+		subinfos, err := o.getCopyInfoForSourcePath(orig)
+		if err != nil {
+			return nil, err
+		}
+		infos = append(infos, subinfos...)
+	}
+
+	if len(infos) == 0 {
+		return nil, errors.New("no source files were specified")
+	}
+	return infos, nil
+}
+
+func (o *copier) getCopyInfoForSourcePath(orig string) ([]copyInfo, error) {
+	if !urlutil.IsURL(orig) {
+		return o.calcCopyInfo(orig, true)
+	}
+	remote, path, err := o.download(orig)
+	if err != nil {
+		return nil, err
+	}
+	o.tmpPaths = append(o.tmpPaths, remote.Root())
+
+	hash, err := remote.Hash(path)
+	return newCopyInfos(newCopyInfoFromSource(remote, path, hash)), err
+}
+
+// Cleanup removes any temporary directories created as part of downloading
+// remote files.
+func (o *copier) Cleanup() {
+	for _, path := range o.tmpPaths {
+		os.RemoveAll(path)
+	}
+	o.tmpPaths = []string{}
+}
+
+// TODO: allowWildcards can probably be removed by refactoring this function further.
+func (o *copier) calcCopyInfo(origPath string, allowWildcards bool) ([]copyInfo, error) {
+	imageSource := o.imageSource
+	if err := validateCopySourcePath(imageSource, origPath); err != nil {
+		return nil, err
+	}
+
+	// Work in daemon-specific OS filepath semantics
+	origPath = filepath.FromSlash(origPath)
+	origPath = strings.TrimPrefix(origPath, string(os.PathSeparator))
+	origPath = strings.TrimPrefix(origPath, "."+string(os.PathSeparator))
+
+	// TODO: do this when creating copier. Requires validateCopySourcePath
+	// (and other below) to be aware of the difference sources. Why is it only
+	// done on image Source?
+	if imageSource != nil {
+		var err error
+		o.source, err = imageSource.Source()
+		if err != nil {
+			return nil, errors.Wrapf(err, "failed to copy")
+		}
+	}
+
+	if o.source == nil {
+		return nil, errors.Errorf("missing build context")
+	}
+
+	// Deal with wildcards
+	if allowWildcards && containsWildcards(origPath) {
+		return o.copyWithWildcards(origPath)
+	}
+
+	if imageSource != nil && imageSource.ImageID() != "" {
+		// return a cached copy if one exists
+		if h, ok := o.pathCache.Load(imageSource.ImageID() + origPath); ok {
+			return newCopyInfos(newCopyInfoFromSource(o.source, origPath, h.(string))), nil
+		}
+	}
+
+	// Deal with the single file case
+	copyInfo, err := copyInfoForFile(o.source, origPath)
+	switch {
+	case err != nil:
+		return nil, err
+	case copyInfo.hash != "":
+		o.storeInPathCache(imageSource, origPath, copyInfo.hash)
+		return newCopyInfos(copyInfo), err
+	}
+
+	// TODO: remove, handle dirs in Hash()
+	subfiles, err := walkSource(o.source, origPath)
+	if err != nil {
+		return nil, err
+	}
+
+	hash := hashStringSlice("dir", subfiles)
+	o.storeInPathCache(imageSource, origPath, hash)
+	return newCopyInfos(newCopyInfoFromSource(o.source, origPath, hash)), nil
+}
+
+func (o *copier) storeInPathCache(im *imageMount, path string, hash string) {
+	if im != nil {
+		o.pathCache.Store(im.ImageID()+path, hash)
+	}
+}
+
+func (o *copier) copyWithWildcards(origPath string) ([]copyInfo, error) {
+	var copyInfos []copyInfo
+	if err := filepath.Walk(o.source.Root(), func(path string, info os.FileInfo, err error) error {
+		if err != nil {
+			return err
+		}
+		rel, err := remotecontext.Rel(o.source.Root(), path)
+		if err != nil {
+			return err
+		}
+
+		if rel == "." {
+			return nil
+		}
+		if match, _ := filepath.Match(origPath, rel); !match {
+			return nil
+		}
+
+		// Note we set allowWildcards to false in case the name has
+		// a * in it
+		subInfos, err := o.calcCopyInfo(rel, false)
+		if err != nil {
+			return err
+		}
+		copyInfos = append(copyInfos, subInfos...)
+		return nil
+	}); err != nil {
+		return nil, err
+	}
+	return copyInfos, nil
+}
+
+func copyInfoForFile(source builder.Source, path string) (copyInfo, error) {
+	fi, err := remotecontext.StatAt(source, path)
+	if err != nil {
+		return copyInfo{}, err
+	}
+
+	if fi.IsDir() {
+		return copyInfo{}, nil
+	}
+	hash, err := source.Hash(path)
+	if err != nil {
+		return copyInfo{}, err
+	}
+	return newCopyInfoFromSource(source, path, "file:"+hash), nil
+}
+
+// TODO: dedupe with copyWithWildcards()
+func walkSource(source builder.Source, origPath string) ([]string, error) {
+	fp, err := remotecontext.FullPath(source, origPath)
+	if err != nil {
+		return nil, err
+	}
+	// Must be a dir
+	var subfiles []string
+	err = filepath.Walk(fp, func(path string, info os.FileInfo, err error) error {
+		if err != nil {
+			return err
+		}
+		rel, err := remotecontext.Rel(source.Root(), path)
+		if err != nil {
+			return err
+		}
+		if rel == "." {
+			return nil
+		}
+		hash, err := source.Hash(rel)
+		if err != nil {
+			return nil
+		}
+		// we already checked handleHash above
+		subfiles = append(subfiles, hash)
+		return nil
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	sort.Strings(subfiles)
+	return subfiles, nil
+}
+
+type sourceDownloader func(string) (builder.Source, string, error)
+
+func newRemoteSourceDownloader(output, stdout io.Writer) sourceDownloader {
+	return func(url string) (builder.Source, string, error) {
+		return downloadSource(output, stdout, url)
+	}
+}
+
+func errOnSourceDownload(_ string) (builder.Source, string, error) {
+	return nil, "", errors.New("source can't be a URL for COPY")
+}
+
+func downloadSource(output io.Writer, stdout io.Writer, srcURL string) (remote builder.Source, p string, err error) {
+	// get filename from URL
+	u, err := url.Parse(srcURL)
+	if err != nil {
+		return
+	}
+	filename := filepath.Base(filepath.FromSlash(u.Path)) // Ensure in platform semantics
+	if filename == "" {
+		err = errors.Errorf("cannot determine filename from url: %s", u)
+		return
+	}
+
+	// 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
+	}
+
+	progressOutput := streamformatter.NewJSONProgressOutput(output, true)
+	progressReader := progress.NewProgressReader(resp.Body, progressOutput, resp.ContentLength, "", "Downloading")
+	// Download and dump result to tmp file
+	// TODO: add filehash directly
+	if _, err = io.Copy(tmpFile, progressReader); err != nil {
+		tmpFile.Close()
+		return
+	}
+	// TODO: how important is this random blank line to the output?
+	fmt.Fprintln(stdout)
+
+	// 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
+		}
+	}
+
+	tmpFile.Close()
+
+	if err = system.Chtimes(tmpFileName, mTime, mTime); err != nil {
+		return
+	}
+
+	lc, err := remotecontext.NewLazyContext(tmpDir)
+	return lc, filename, err
+}

+ 19 - 2
builder/dockerfile/dispatchers.go

@@ -148,7 +148,16 @@ func add(req dispatchRequest) error {
 		return err
 	}
 
-	return req.builder.runContextCommand(req, true, true, "ADD", nil)
+	downloader := newRemoteSourceDownloader(req.builder.Output, req.builder.Stdout)
+	copier := copierFromDispatchRequest(req, downloader, nil)
+	defer copier.Cleanup()
+	copyInstruction, err := copier.createCopyInstruction(req.args, "ADD")
+	if err != nil {
+		return err
+	}
+	copyInstruction.allowLocalDecompression = true
+
+	return req.builder.performCopy(req.state, copyInstruction)
 }
 
 // COPY foo /path
@@ -169,7 +178,15 @@ func dispatchCopy(req dispatchRequest) error {
 	if err != nil {
 		return errors.Wrapf(err, "invalid from flag value %s", flFrom.Value)
 	}
-	return req.builder.runContextCommand(req, false, false, "COPY", im)
+
+	copier := copierFromDispatchRequest(req, errOnSourceDownload, im)
+	defer copier.Cleanup()
+	copyInstruction, err := copier.createCopyInstruction(req.args, "COPY")
+	if err != nil {
+		return err
+	}
+
+	return req.builder.performCopy(req.state, copyInstruction)
 }
 
 func (b *Builder) getImageMount(fromFlag *Flag) (*imageMount, error) {

+ 3 - 0
builder/dockerfile/evaluator.go

@@ -68,6 +68,7 @@ type dispatchRequest struct {
 	original   string
 	shlex      *ShellLex
 	state      *dispatchState
+	source     builder.Source
 }
 
 func newDispatchRequestFromOptions(options dispatchOptions, builder *Builder, args []string) dispatchRequest {
@@ -79,6 +80,7 @@ func newDispatchRequestFromOptions(options dispatchOptions, builder *Builder, ar
 		flags:      NewBFlagsWithArgs(options.node.Flags),
 		shlex:      options.shlex,
 		state:      options.state,
+		source:     options.source,
 	}
 }
 
@@ -181,6 +183,7 @@ type dispatchOptions struct {
 	stepMsg string
 	node    *parser.Node
 	shlex   *ShellLex
+	source  builder.Source
 }
 
 // dispatchState is a data object which is modified by dispatchers

+ 3 - 3
builder/dockerfile/evaluator_test.go

@@ -105,13 +105,13 @@ func initDispatchTestCases() []dispatchTestCase {
 		{
 			name:          "COPY wildcard no files",
 			dockerfile:    `COPY file*.txt /tmp/`,
-			expectedError: "No source files were specified",
+			expectedError: "COPY failed: no source files were specified",
 			files:         nil,
 		},
 		{
 			name:          "COPY url",
 			dockerfile:    `COPY https://index.docker.io/robots.txt /`,
-			expectedError: "Source can't be a URL for COPY",
+			expectedError: "source can't be a URL for COPY",
 			files:         nil,
 		},
 		{
@@ -184,7 +184,6 @@ func executeTestCase(t *testing.T, testCase dispatchTestCase) {
 	b := &Builder{
 		options:   options,
 		Stdout:    ioutil.Discard,
-		source:    context,
 		buildArgs: newBuildArgs(options.BuildArgs),
 	}
 
@@ -196,6 +195,7 @@ func executeTestCase(t *testing.T, testCase dispatchTestCase) {
 		stepMsg: formatStep(0, len(n.Children)),
 		node:    n.Children[0],
 		shlex:   shlex,
+		source:  context,
 	}
 	state, err = b.dispatch(opts)
 

+ 4 - 22
builder/dockerfile/imagecontext.go

@@ -13,11 +13,6 @@ import (
 	"golang.org/x/net/context"
 )
 
-type pathCache interface {
-	Load(key interface{}) (value interface{}, ok bool)
-	Store(key, value interface{})
-}
-
 type buildStage struct {
 	id     string
 	config *container.Config
@@ -148,23 +143,6 @@ func (m *imageSources) Unmount() (retErr error) {
 	return
 }
 
-// TODO: remove getCache/setCache from imageSources
-func (m *imageSources) getCache(id, path string) (interface{}, bool) {
-	if m.cache != nil {
-		if id == "" {
-			return nil, false
-		}
-		return m.cache.Load(id + path)
-	}
-	return nil, false
-}
-
-func (m *imageSources) setCache(id, path string, v interface{}) {
-	if m.cache != nil {
-		m.cache.Store(id+path, v)
-	}
-}
-
 // imageMount is a reference to an image that can be used as a builder.Source
 type imageMount struct {
 	image  builder.Image
@@ -208,3 +186,7 @@ func (im *imageMount) unmount() error {
 func (im *imageMount) Image() builder.Image {
 	return im.image
 }
+
+func (im *imageMount) ImageID() string {
+	return im.image.ImageID()
+}

+ 28 - 322
builder/dockerfile/internals.go

@@ -7,31 +7,15 @@ import (
 	"crypto/sha256"
 	"encoding/hex"
 	"fmt"
-	"io"
-	"net/http"
-	"net/url"
-	"os"
-	"path/filepath"
-	"runtime"
-	"sort"
 	"strings"
-	"time"
 
 	"github.com/Sirupsen/logrus"
 	"github.com/docker/docker/api/types"
 	"github.com/docker/docker/api/types/backend"
 	"github.com/docker/docker/api/types/container"
-	"github.com/docker/docker/builder"
-	"github.com/docker/docker/builder/remotecontext"
 	containerpkg "github.com/docker/docker/container"
-	"github.com/docker/docker/pkg/httputils"
-	"github.com/docker/docker/pkg/ioutils"
 	"github.com/docker/docker/pkg/jsonmessage"
-	"github.com/docker/docker/pkg/progress"
-	"github.com/docker/docker/pkg/streamformatter"
 	"github.com/docker/docker/pkg/stringid"
-	"github.com/docker/docker/pkg/system"
-	"github.com/docker/docker/pkg/urlutil"
 	"github.com/pkg/errors"
 )
 
@@ -83,90 +67,14 @@ func (b *Builder) commitContainer(dispatchState *dispatchState, id string, conta
 	return nil
 }
 
-type copyInfo struct {
-	root       string
-	path       string
-	hash       string
-	decompress bool
-}
-
-// TODO: this needs to be split so that a Builder method doesn't accept req
-func (b *Builder) runContextCommand(req dispatchRequest, allowRemote bool, allowLocalDecompression bool, cmdName string, imageSource *imageMount) error {
-	args := req.args
-	if len(args) < 2 {
-		return fmt.Errorf("Invalid %s format - at least two arguments required", cmdName)
-	}
-
-	// Work in daemon-specific filepath semantics
-	dest := filepath.FromSlash(args[len(args)-1]) // last one is always the dest
-
-	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 urlutil.IsURL(orig) {
-			if !allowRemote {
-				return fmt.Errorf("Source can't be a URL for %s", cmdName)
-			}
-			remote, path, err := b.download(orig)
-			if err != nil {
-				return err
-			}
-			defer os.RemoveAll(remote.Root())
-			h, err := remote.Hash(path)
-			if err != nil {
-				return err
-			}
-			infos = append(infos, copyInfo{
-				root: remote.Root(),
-				path: path,
-				hash: h,
-			})
-			continue
-		}
-		// not a URL
-		subInfos, err := b.calcCopyInfo(cmdName, orig, allowLocalDecompression, true, imageSource)
-		if err != nil {
-			return err
-		}
-
-		infos = append(infos, subInfos...)
-	}
-
-	if len(infos) == 0 {
-		return errors.New("No source files were specified")
-	}
-	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 info then use it as the
-	// cache look-up string, otherwise hash 'em all into one
-	var srcHash string
-
-	if len(infos) == 1 {
-		info := infos[0]
-		srcHash = info.hash
-	} else {
-		var hashs []string
-		var origs []string
-		for _, info := range infos {
-			origs = append(origs, info.path)
-			hashs = append(hashs, info.hash)
-		}
-		hasher := sha256.New()
-		hasher.Write([]byte(strings.Join(hashs, ",")))
-		srcHash = "multi:" + hex.EncodeToString(hasher.Sum(nil))
-	}
+func (b *Builder) performCopy(state *dispatchState, inst copyInstruction) error {
+	srcHash := getSourceHashFromInfos(inst.infos)
 
 	// TODO: should this have been using origPaths instead of srcHash in the comment?
 	runConfigWithCommentCmd := copyRunConfig(
-		req.state.runConfig,
-		withCmdCommentString(fmt.Sprintf("%s %s in %s ", cmdName, srcHash, dest)))
-	if hit, err := b.probeCache(req.state, runConfigWithCommentCmd); err != nil || hit {
+		state.runConfig,
+		withCmdCommentString(fmt.Sprintf("%s %s in %s ", inst.cmdName, srcHash, inst.dest)))
+	if hit, err := b.probeCache(state, runConfigWithCommentCmd); err != nil || hit {
 		return err
 	}
 
@@ -182,17 +90,36 @@ func (b *Builder) runContextCommand(req dispatchRequest, allowRemote bool, allow
 
 	// Twiddle the destination when it's a relative path - meaning, make it
 	// relative to the WORKINGDIR
-	if dest, err = normaliseDest(cmdName, req.state.runConfig.WorkingDir, dest); err != nil {
+	dest, err := normaliseDest(inst.cmdName, state.runConfig.WorkingDir, inst.dest)
+	if err != nil {
 		return err
 	}
 
-	for _, info := range infos {
-		if err := b.docker.CopyOnBuild(container.ID, dest, info.root, info.path, info.decompress); err != nil {
+	for _, info := range inst.infos {
+		if err := b.docker.CopyOnBuild(container.ID, dest, info.root, info.path, inst.allowLocalDecompression); err != nil {
 			return err
 		}
 	}
+	return b.commitContainer(state, container.ID, runConfigWithCommentCmd)
+}
+
+// For backwards compat, if there's just one info then use it as the
+// cache look-up string, otherwise hash 'em all into one
+func getSourceHashFromInfos(infos []copyInfo) string {
+	if len(infos) == 1 {
+		return infos[0].hash
+	}
+	var hashs []string
+	for _, info := range infos {
+		hashs = append(hashs, info.hash)
+	}
+	return hashStringSlice("multi", hashs)
+}
 
-	return b.commitContainer(req.state, container.ID, runConfigWithCommentCmd)
+func hashStringSlice(prefix string, slice []string) string {
+	hasher := sha256.New()
+	hasher.Write([]byte(strings.Join(slice, ",")))
+	return prefix + ":" + hex.EncodeToString(hasher.Sum(nil))
 }
 
 type runConfigModifier func(*container.Config)
@@ -259,227 +186,6 @@ func getShell(c *container.Config) []string {
 	return append([]string{}, c.Shell[:]...)
 }
 
-func (b *Builder) download(srcURL string) (remote builder.Source, p string, err error) {
-	// get filename from URL
-	u, err := url.Parse(srcURL)
-	if err != nil {
-		return
-	}
-	path := filepath.FromSlash(u.Path) // Ensure in platform semantics
-	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
-	}
-
-	// 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
-	}
-
-	progressOutput := streamformatter.NewJSONProgressOutput(b.Output, true)
-	progressReader := progress.NewProgressReader(resp.Body, progressOutput, resp.ContentLength, "", "Downloading")
-	// Download and dump result to tmp file
-	// TODO: add filehash directly
-	if _, err = io.Copy(tmpFile, progressReader); err != nil {
-		tmpFile.Close()
-		return
-	}
-	fmt.Fprintln(b.Stdout)
-
-	// 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
-		}
-	}
-
-	tmpFile.Close()
-
-	if err = system.Chtimes(tmpFileName, mTime, mTime); err != nil {
-		return
-	}
-
-	lc, err := remotecontext.NewLazyContext(tmpDir)
-	if err != nil {
-		return
-	}
-
-	return lc, filename, nil
-}
-
-var windowsBlacklist = map[string]bool{
-	"c:\\":        true,
-	"c:\\windows": true,
-}
-
-func (b *Builder) calcCopyInfo(cmdName, origPath string, allowLocalDecompression, allowWildcards bool, imageSource *imageMount) ([]copyInfo, error) {
-
-	// Work in daemon-specific OS filepath semantics
-	origPath = filepath.FromSlash(origPath)
-	// validate windows paths from other images
-	if imageSource != nil && runtime.GOOS == "windows" {
-		p := strings.ToLower(filepath.Clean(origPath))
-		if !filepath.IsAbs(p) {
-			if filepath.VolumeName(p) != "" {
-				if p[len(p)-2:] == ":." { // case where clean returns weird c:. paths
-					p = p[:len(p)-1]
-				}
-				p += "\\"
-			} else {
-				p = filepath.Join("c:\\", p)
-			}
-		}
-		if _, blacklisted := windowsBlacklist[p]; blacklisted {
-			return nil, errors.New("copy from c:\\ or c:\\windows is not allowed on windows")
-		}
-	}
-
-	if origPath != "" && origPath[0] == os.PathSeparator && len(origPath) > 1 {
-		origPath = origPath[1:]
-	}
-	origPath = strings.TrimPrefix(origPath, "."+string(os.PathSeparator))
-
-	source := b.source
-	var err error
-	if imageSource != nil {
-		source, err = imageSource.Source()
-		if err != nil {
-			return nil, errors.Wrapf(err, "failed to copy")
-		}
-	}
-
-	if source == nil {
-		return nil, errors.Errorf("No context given. Impossible to use %s", cmdName)
-	}
-
-	// Deal with wildcards
-	if allowWildcards && containsWildcards(origPath) {
-		var copyInfos []copyInfo
-		if err := filepath.Walk(source.Root(), func(path string, info os.FileInfo, err error) error {
-			if err != nil {
-				return err
-			}
-			rel, err := remotecontext.Rel(source.Root(), path)
-			if err != nil {
-				return err
-			}
-			if rel == "." {
-				return nil
-			}
-			if match, _ := filepath.Match(origPath, rel); !match {
-				return nil
-			}
-
-			// Note we set allowWildcards to false in case the name has
-			// a * in it
-			subInfos, err := b.calcCopyInfo(cmdName, rel, allowLocalDecompression, false, imageSource)
-			if err != nil {
-				return err
-			}
-			copyInfos = append(copyInfos, subInfos...)
-			return nil
-		}); err != nil {
-			return nil, err
-		}
-		return copyInfos, nil
-	}
-
-	// Must be a dir or a file
-	hash, err := source.Hash(origPath)
-	if err != nil {
-		return nil, err
-	}
-
-	fi, err := remotecontext.StatAt(source, origPath)
-	if err != nil {
-		return nil, err
-	}
-
-	// TODO: remove, handle dirs in Hash()
-	copyInfos := []copyInfo{{root: source.Root(), path: origPath, hash: hash, decompress: allowLocalDecompression}}
-
-	if imageSource != nil {
-		// fast-cache based on imageID
-		if h, ok := b.imageSources.getCache(imageSource.Image().ImageID(), origPath); ok {
-			copyInfos[0].hash = h.(string)
-			return copyInfos, nil
-		}
-	}
-
-	// Deal with the single file case
-	if !fi.IsDir() {
-		copyInfos[0].hash = "file:" + copyInfos[0].hash
-		return copyInfos, nil
-	}
-
-	fp, err := remotecontext.FullPath(source, origPath)
-	if err != nil {
-		return nil, err
-	}
-	// Must be a dir
-	var subfiles []string
-	err = filepath.Walk(fp, func(path string, info os.FileInfo, err error) error {
-		if err != nil {
-			return err
-		}
-		rel, err := remotecontext.Rel(source.Root(), path)
-		if err != nil {
-			return err
-		}
-		if rel == "." {
-			return nil
-		}
-		hash, err := source.Hash(rel)
-		if err != nil {
-			return nil
-		}
-		// we already checked handleHash above
-		subfiles = append(subfiles, hash)
-		return nil
-	})
-	if err != nil {
-		return nil, err
-	}
-
-	sort.Strings(subfiles)
-	hasher := sha256.New()
-	hasher.Write([]byte(strings.Join(subfiles, ",")))
-	copyInfos[0].hash = "dir:" + hex.EncodeToString(hasher.Sum(nil))
-	if imageSource != nil {
-		b.imageSources.setCache(imageSource.Image().ImageID(), origPath, copyInfos[0].hash)
-	}
-
-	return copyInfos, nil
-}
-
 // probeCache checks if cache match can be found for current build instruction.
 // If an image is found, probeCache returns `(true, nil)`.
 // If no image is found, it returns `(false, nil)`.

+ 4 - 0
builder/dockerfile/internals_unix.go

@@ -36,3 +36,7 @@ func containsWildcards(name string) bool {
 	}
 	return false
 }
+
+func validateCopySourcePath(imageSource *imageMount, origPath string) error {
+	return nil
+}

+ 29 - 0
builder/dockerfile/internals_windows.go

@@ -7,6 +7,7 @@ import (
 	"strings"
 
 	"github.com/docker/docker/pkg/system"
+	"github.com/pkg/errors"
 )
 
 // normaliseDest normalises the destination of a COPY/ADD command in a
@@ -64,3 +65,31 @@ func containsWildcards(name string) bool {
 	}
 	return false
 }
+
+var pathBlacklist = map[string]bool{
+	"c:\\":        true,
+	"c:\\windows": true,
+}
+
+func validateCopySourcePath(imageSource *imageMount, origPath string) error {
+	// validate windows paths from other images
+	if imageSource == nil {
+		return nil
+	}
+	origPath = filepath.FromSlash(origPath)
+	p := strings.ToLower(filepath.Clean(origPath))
+	if !filepath.IsAbs(p) {
+		if filepath.VolumeName(p) != "" {
+			if p[len(p)-2:] == ":." { // case where clean returns weird c:. paths
+				p = p[:len(p)-1]
+			}
+			p += "\\"
+		} else {
+			p = filepath.Join("c:\\", p)
+		}
+	}
+	if _, blacklisted := pathBlacklist[p]; blacklisted {
+		return errors.New("copy from c:\\ or c:\\windows is not allowed on windows")
+	}
+	return nil
+}