Browse Source

Merge pull request #33492 from simonferquel/typed-builder-commands-squashed

[Builder] Introduce a typed command system and 2 phase parse/dispatch build
Akihiro Suda 7 years ago
parent
commit
22e15721b1

+ 20 - 0
builder/dockerfile/buildargs.go

@@ -42,6 +42,26 @@ func newBuildArgs(argsFromOptions map[string]*string) *buildArgs {
 	}
 }
 
+func (b *buildArgs) Clone() *buildArgs {
+	result := newBuildArgs(b.argsFromOptions)
+	for k, v := range b.allowedBuildArgs {
+		result.allowedBuildArgs[k] = v
+	}
+	for k, v := range b.allowedMetaArgs {
+		result.allowedMetaArgs[k] = v
+	}
+	for k := range b.referencedArgs {
+		result.referencedArgs[k] = struct{}{}
+	}
+	return result
+}
+
+func (b *buildArgs) MergeReferencedArgs(other *buildArgs) {
+	for k := range other.referencedArgs {
+		b.referencedArgs[k] = struct{}{}
+	}
+}
+
 // WarnOnUnusedBuildArgs checks if there are any leftover build-args that were
 // passed but not consumed during build. Print a warning, if there are any.
 func (b *buildArgs) WarnOnUnusedBuildArgs(out io.Writer) {

+ 115 - 88
builder/dockerfile/builder.go

@@ -13,7 +13,7 @@ import (
 	"github.com/docker/docker/api/types/backend"
 	"github.com/docker/docker/api/types/container"
 	"github.com/docker/docker/builder"
-	"github.com/docker/docker/builder/dockerfile/command"
+	"github.com/docker/docker/builder/dockerfile/instructions"
 	"github.com/docker/docker/builder/dockerfile/parser"
 	"github.com/docker/docker/builder/fscache"
 	"github.com/docker/docker/builder/remotecontext"
@@ -41,6 +41,10 @@ var validCommitCommands = map[string]bool{
 	"workdir":     true,
 }
 
+const (
+	stepFormat = "Step %d/%d : %v"
+)
+
 // SessionGetter is object used to get access to a session by uuid
 type SessionGetter interface {
 	Get(ctx context.Context, uuid string) (session.Caller, error)
@@ -176,9 +180,7 @@ type Builder struct {
 	clientCtx context.Context
 
 	idMappings       *idtools.IDMappings
-	buildStages      *buildStages
 	disableCommit    bool
-	buildArgs        *buildArgs
 	imageSources     *imageSources
 	pathCache        pathCache
 	containerManager *containerManager
@@ -218,8 +220,6 @@ func newBuilder(clientCtx context.Context, options builderOptions) *Builder {
 		Output:           options.ProgressWriter.Output,
 		docker:           options.Backend,
 		idMappings:       options.IDMappings,
-		buildArgs:        newBuildArgs(config.BuildArgs),
-		buildStages:      newBuildStages(),
 		imageSources:     newImageSources(clientCtx, options),
 		pathCache:        options.PathCache,
 		imageProber:      newImageProber(options.Backend, config.CacheFrom, options.Platform, config.NoCache),
@@ -237,24 +237,27 @@ func (b *Builder) build(source builder.Source, dockerfile *parser.Result) (*buil
 
 	addNodesForLabelOption(dockerfile.AST, b.options.Labels)
 
-	if err := checkDispatchDockerfile(dockerfile.AST); err != nil {
-		buildsFailed.WithValues(metricsDockerfileSyntaxError).Inc()
+	stages, metaArgs, err := instructions.Parse(dockerfile.AST)
+	if err != nil {
+		if instructions.IsUnknownInstruction(err) {
+			buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
+		}
 		return nil, validationError{err}
 	}
+	if b.options.Target != "" {
+		targetIx, found := instructions.HasStage(stages, b.options.Target)
+		if !found {
+			buildsFailed.WithValues(metricsBuildTargetNotReachableError).Inc()
+			return nil, errors.Errorf("failed to reach build target %s in Dockerfile", b.options.Target)
+		}
+		stages = stages[:targetIx+1]
+	}
 
-	dispatchState, err := b.dispatchDockerfileWithCancellation(dockerfile, source)
+	dockerfile.PrintWarnings(b.Stderr)
+	dispatchState, err := b.dispatchDockerfileWithCancellation(stages, metaArgs, dockerfile.EscapeToken, source)
 	if err != nil {
 		return nil, err
 	}
-
-	if b.options.Target != "" && !dispatchState.isCurrentStage(b.options.Target) {
-		buildsFailed.WithValues(metricsBuildTargetNotReachableError).Inc()
-		return nil, errors.Errorf("failed to reach build target %s in Dockerfile", b.options.Target)
-	}
-
-	dockerfile.PrintWarnings(b.Stderr)
-	b.buildArgs.WarnOnUnusedBuildArgs(b.Stderr)
-
 	if dispatchState.imageID == "" {
 		buildsFailed.WithValues(metricsDockerfileEmptyError).Inc()
 		return nil, errors.New("No image was generated. Is your Dockerfile empty?")
@@ -269,61 +272,91 @@ func emitImageID(aux *streamformatter.AuxFormatter, state *dispatchState) error
 	return aux.Emit(types.BuildResult{ID: state.imageID})
 }
 
-func (b *Builder) dispatchDockerfileWithCancellation(dockerfile *parser.Result, source builder.Source) (*dispatchState, error) {
-	shlex := NewShellLex(dockerfile.EscapeToken)
-	state := newDispatchState()
-	total := len(dockerfile.AST.Children)
-	var err error
-	for i, n := range dockerfile.AST.Children {
-		select {
-		case <-b.clientCtx.Done():
-			logrus.Debug("Builder: build cancelled!")
-			fmt.Fprint(b.Stdout, "Build cancelled")
-			buildsFailed.WithValues(metricsBuildCanceled).Inc()
-			return nil, errors.New("Build cancelled")
-		default:
-			// Not cancelled yet, keep going...
-		}
+func processMetaArg(meta instructions.ArgCommand, shlex *ShellLex, args *buildArgs) error {
+	// ShellLex currently only support the concatenated string format
+	envs := convertMapToEnvList(args.GetAllAllowed())
+	if err := meta.Expand(func(word string) (string, error) {
+		return shlex.ProcessWord(word, envs)
+	}); err != nil {
+		return err
+	}
+	args.AddArg(meta.Key, meta.Value)
+	args.AddMetaArg(meta.Key, meta.Value)
+	return nil
+}
 
-		// If this is a FROM and we have a previous image then
-		// emit an aux message for that image since it is the
-		// end of the previous stage
-		if n.Value == command.From {
-			if err := emitImageID(b.Aux, state); err != nil {
-				return nil, err
-			}
+func printCommand(out io.Writer, currentCommandIndex int, totalCommands int, cmd interface{}) int {
+	fmt.Fprintf(out, stepFormat, currentCommandIndex, totalCommands, cmd)
+	fmt.Fprintln(out)
+	return currentCommandIndex + 1
+}
+
+func (b *Builder) dispatchDockerfileWithCancellation(parseResult []instructions.Stage, metaArgs []instructions.ArgCommand, escapeToken rune, source builder.Source) (*dispatchState, error) {
+	dispatchRequest := dispatchRequest{}
+	buildArgs := newBuildArgs(b.options.BuildArgs)
+	totalCommands := len(metaArgs) + len(parseResult)
+	currentCommandIndex := 1
+	for _, stage := range parseResult {
+		totalCommands += len(stage.Commands)
+	}
+	shlex := NewShellLex(escapeToken)
+	for _, meta := range metaArgs {
+		currentCommandIndex = printCommand(b.Stdout, currentCommandIndex, totalCommands, &meta)
+
+		err := processMetaArg(meta, shlex, buildArgs)
+		if err != nil {
+			return nil, err
 		}
+	}
+
+	stagesResults := newStagesBuildResults()
 
-		if n.Value == command.From && state.isCurrentStage(b.options.Target) {
-			break
+	for _, stage := range parseResult {
+		if err := stagesResults.checkStageNameAvailable(stage.Name); err != nil {
+			return nil, err
 		}
+		dispatchRequest = newDispatchRequest(b, escapeToken, source, buildArgs, stagesResults)
 
-		opts := dispatchOptions{
-			state:   state,
-			stepMsg: formatStep(i, total),
-			node:    n,
-			shlex:   shlex,
-			source:  source,
+		currentCommandIndex = printCommand(b.Stdout, currentCommandIndex, totalCommands, stage.SourceCode)
+		if err := initializeStage(dispatchRequest, &stage); err != nil {
+			return nil, err
 		}
-		if state, err = b.dispatch(opts); err != nil {
-			if b.options.ForceRemove {
-				b.containerManager.RemoveAll(b.Stdout)
+		dispatchRequest.state.updateRunConfig()
+		fmt.Fprintf(b.Stdout, " ---> %s\n", stringid.TruncateID(dispatchRequest.state.imageID))
+		for _, cmd := range stage.Commands {
+			select {
+			case <-b.clientCtx.Done():
+				logrus.Debug("Builder: build cancelled!")
+				fmt.Fprint(b.Stdout, "Build cancelled\n")
+				buildsFailed.WithValues(metricsBuildCanceled).Inc()
+				return nil, errors.New("Build cancelled")
+			default:
+				// Not cancelled yet, keep going...
 			}
+
+			currentCommandIndex = printCommand(b.Stdout, currentCommandIndex, totalCommands, cmd)
+
+			if err := dispatch(dispatchRequest, cmd); err != nil {
+				return nil, err
+			}
+
+			dispatchRequest.state.updateRunConfig()
+			fmt.Fprintf(b.Stdout, " ---> %s\n", stringid.TruncateID(dispatchRequest.state.imageID))
+
+		}
+		if err := emitImageID(b.Aux, dispatchRequest.state); err != nil {
 			return nil, err
 		}
-
-		fmt.Fprintf(b.Stdout, " ---> %s\n", stringid.TruncateID(state.imageID))
-		if b.options.Remove {
-			b.containerManager.RemoveAll(b.Stdout)
+		buildArgs.MergeReferencedArgs(dispatchRequest.state.buildArgs)
+		if err := commitStage(dispatchRequest.state, stagesResults); err != nil {
+			return nil, err
 		}
 	}
-
-	// Emit a final aux message for the final image
-	if err := emitImageID(b.Aux, state); err != nil {
-		return nil, err
+	if b.options.Remove {
+		b.containerManager.RemoveAll(b.Stdout)
 	}
-
-	return state, nil
+	buildArgs.WarnOnUnusedBuildArgs(b.Stdout)
+	return dispatchRequest.state, nil
 }
 
 func addNodesForLabelOption(dockerfile *parser.Node, labels map[string]string) {
@@ -380,39 +413,33 @@ func BuildFromConfig(config *container.Config, changes []string) (*container.Con
 	b.Stderr = ioutil.Discard
 	b.disableCommit = true
 
-	if err := checkDispatchDockerfile(dockerfile.AST); err != nil {
-		return nil, validationError{err}
+	commands := []instructions.Command{}
+	for _, n := range dockerfile.AST.Children {
+		cmd, err := instructions.ParseCommand(n)
+		if err != nil {
+			return nil, validationError{err}
+		}
+		commands = append(commands, cmd)
 	}
-	dispatchState := newDispatchState()
-	dispatchState.runConfig = config
-	return dispatchFromDockerfile(b, dockerfile, dispatchState, nil)
-}
 
-func checkDispatchDockerfile(dockerfile *parser.Node) error {
-	for _, n := range dockerfile.Children {
-		if err := checkDispatch(n); err != nil {
-			return errors.Wrapf(err, "Dockerfile parse error line %d", n.StartLine)
+	dispatchRequest := newDispatchRequest(b, dockerfile.EscapeToken, nil, newBuildArgs(b.options.BuildArgs), newStagesBuildResults())
+	dispatchRequest.state.runConfig = config
+	dispatchRequest.state.imageID = config.Image
+	for _, cmd := range commands {
+		err := dispatch(dispatchRequest, cmd)
+		if err != nil {
+			return nil, validationError{err}
 		}
+		dispatchRequest.state.updateRunConfig()
 	}
-	return nil
+
+	return dispatchRequest.state.runConfig, nil
 }
 
-func dispatchFromDockerfile(b *Builder, result *parser.Result, dispatchState *dispatchState, source builder.Source) (*container.Config, error) {
-	shlex := NewShellLex(result.EscapeToken)
-	ast := result.AST
-	total := len(ast.Children)
-
-	for i, n := range ast.Children {
-		opts := dispatchOptions{
-			state:   dispatchState,
-			stepMsg: formatStep(i, total),
-			node:    n,
-			shlex:   shlex,
-			source:  source,
-		}
-		if _, err := b.dispatch(opts); err != nil {
-			return nil, err
-		}
+func convertMapToEnvList(m map[string]string) []string {
+	result := []string{}
+	for k, v := range m {
+		result = append(result, k+"="+v)
 	}
-	return dispatchState.runConfig, nil
+	return result
 }

+ 186 - 543
builder/dockerfile/dispatchers.go

@@ -10,17 +10,15 @@ package dockerfile
 import (
 	"bytes"
 	"fmt"
-	"regexp"
 	"runtime"
 	"sort"
-	"strconv"
 	"strings"
-	"time"
 
 	"github.com/docker/docker/api"
 	"github.com/docker/docker/api/types/container"
 	"github.com/docker/docker/api/types/strslice"
 	"github.com/docker/docker/builder"
+	"github.com/docker/docker/builder/dockerfile/instructions"
 	"github.com/docker/docker/builder/dockerfile/parser"
 	"github.com/docker/docker/image"
 	"github.com/docker/docker/pkg/jsonmessage"
@@ -36,32 +34,14 @@ import (
 // Sets the environment variable foo to bar, also makes interpolation
 // in the dockerfile available from the next statement on via ${foo}.
 //
-func env(req dispatchRequest) error {
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("ENV")
-	}
-
-	if len(req.args)%2 != 0 {
-		// should never get here, but just in case
-		return errTooManyArguments("ENV")
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	runConfig := req.state.runConfig
+func dispatchEnv(d dispatchRequest, c *instructions.EnvCommand) error {
+	runConfig := d.state.runConfig
 	commitMessage := bytes.NewBufferString("ENV")
+	for _, e := range c.Env {
+		name := e.Key
+		newVar := e.String()
 
-	for j := 0; j < len(req.args); j += 2 {
-		if len(req.args[j]) == 0 {
-			return errBlankCommandNames("ENV")
-		}
-		name := req.args[j]
-		value := req.args[j+1]
-		newVar := name + "=" + value
 		commitMessage.WriteString(" " + newVar)
-
 		gotOne := false
 		for i, envVar := range runConfig.Env {
 			envParts := strings.SplitN(envVar, "=", 2)
@@ -76,64 +56,32 @@ func env(req dispatchRequest) error {
 			runConfig.Env = append(runConfig.Env, newVar)
 		}
 	}
-
-	return req.builder.commit(req.state, commitMessage.String())
+	return d.builder.commit(d.state, commitMessage.String())
 }
 
 // MAINTAINER some text <maybe@an.email.address>
 //
 // Sets the maintainer metadata.
-func maintainer(req dispatchRequest) error {
-	if len(req.args) != 1 {
-		return errExactlyOneArgument("MAINTAINER")
-	}
+func dispatchMaintainer(d dispatchRequest, c *instructions.MaintainerCommand) error {
 
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	maintainer := req.args[0]
-	req.state.maintainer = maintainer
-	return req.builder.commit(req.state, "MAINTAINER "+maintainer)
+	d.state.maintainer = c.Maintainer
+	return d.builder.commit(d.state, "MAINTAINER "+c.Maintainer)
 }
 
 // LABEL some json data describing the image
 //
 // Sets the Label variable foo to bar,
 //
-func label(req dispatchRequest) error {
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("LABEL")
-	}
-	if len(req.args)%2 != 0 {
-		// should never get here, but just in case
-		return errTooManyArguments("LABEL")
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
+func dispatchLabel(d dispatchRequest, c *instructions.LabelCommand) error {
+	if d.state.runConfig.Labels == nil {
+		d.state.runConfig.Labels = make(map[string]string)
 	}
-
 	commitStr := "LABEL"
-	runConfig := req.state.runConfig
-
-	if runConfig.Labels == nil {
-		runConfig.Labels = map[string]string{}
-	}
-
-	for j := 0; j < len(req.args); j++ {
-		name := req.args[j]
-		if name == "" {
-			return errBlankCommandNames("LABEL")
-		}
-
-		value := req.args[j+1]
-		commitStr += " " + name + "=" + value
-
-		runConfig.Labels[name] = value
-		j++
+	for _, v := range c.Labels {
+		d.state.runConfig.Labels[v.Key] = v.Value
+		commitStr += " " + v.String()
 	}
-	return req.builder.commit(req.state, commitStr)
+	return d.builder.commit(d.state, commitStr)
 }
 
 // ADD foo /path
@@ -141,257 +89,172 @@ func label(req dispatchRequest) error {
 // 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(req dispatchRequest) error {
-	if len(req.args) < 2 {
-		return errAtLeastTwoArguments("ADD")
-	}
-
-	flChown := req.flags.AddString("chown", "")
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	downloader := newRemoteSourceDownloader(req.builder.Output, req.builder.Stdout)
-	copier := copierFromDispatchRequest(req, downloader, nil)
+func dispatchAdd(d dispatchRequest, c *instructions.AddCommand) error {
+	downloader := newRemoteSourceDownloader(d.builder.Output, d.builder.Stdout)
+	copier := copierFromDispatchRequest(d, downloader, nil)
 	defer copier.Cleanup()
-	copyInstruction, err := copier.createCopyInstruction(req.args, "ADD")
+
+	copyInstruction, err := copier.createCopyInstruction(c.SourcesAndDest, "ADD")
 	if err != nil {
 		return err
 	}
-	copyInstruction.chownStr = flChown.Value
+	copyInstruction.chownStr = c.Chown
 	copyInstruction.allowLocalDecompression = true
 
-	return req.builder.performCopy(req.state, copyInstruction)
+	return d.builder.performCopy(d.state, copyInstruction)
 }
 
 // COPY foo /path
 //
 // Same as 'ADD' but without the tar and remote url handling.
 //
-func dispatchCopy(req dispatchRequest) error {
-	if len(req.args) < 2 {
-		return errAtLeastTwoArguments("COPY")
-	}
-
-	flFrom := req.flags.AddString("from", "")
-	flChown := req.flags.AddString("chown", "")
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	im, err := req.builder.getImageMount(flFrom)
-	if err != nil {
-		return errors.Wrapf(err, "invalid from flag value %s", flFrom.Value)
+func dispatchCopy(d dispatchRequest, c *instructions.CopyCommand) error {
+	var im *imageMount
+	var err error
+	if c.From != "" {
+		im, err = d.getImageMount(c.From)
+		if err != nil {
+			return errors.Wrapf(err, "invalid from flag value %s", c.From)
+		}
 	}
-
-	copier := copierFromDispatchRequest(req, errOnSourceDownload, im)
+	copier := copierFromDispatchRequest(d, errOnSourceDownload, im)
 	defer copier.Cleanup()
-	copyInstruction, err := copier.createCopyInstruction(req.args, "COPY")
+	copyInstruction, err := copier.createCopyInstruction(c.SourcesAndDest, "COPY")
 	if err != nil {
 		return err
 	}
-	copyInstruction.chownStr = flChown.Value
+	copyInstruction.chownStr = c.Chown
 
-	return req.builder.performCopy(req.state, copyInstruction)
+	return d.builder.performCopy(d.state, copyInstruction)
 }
 
-func (b *Builder) getImageMount(fromFlag *Flag) (*imageMount, error) {
-	if !fromFlag.IsUsed() {
+func (d *dispatchRequest) getImageMount(imageRefOrID string) (*imageMount, error) {
+	if imageRefOrID == "" {
 		// TODO: this could return the source in the default case as well?
 		return nil, nil
 	}
 
 	var localOnly bool
-	imageRefOrID := fromFlag.Value
-	stage, err := b.buildStages.get(fromFlag.Value)
+	stage, err := d.stages.get(imageRefOrID)
 	if err != nil {
 		return nil, err
 	}
 	if stage != nil {
-		imageRefOrID = stage.ImageID()
+		imageRefOrID = stage.Image
 		localOnly = true
 	}
-	return b.imageSources.Get(imageRefOrID, localOnly)
+	return d.builder.imageSources.Get(imageRefOrID, localOnly)
 }
 
 // FROM imagename[:tag | @digest] [AS build-stage-name]
 //
-func from(req dispatchRequest) error {
-	stageName, err := parseBuildStageName(req.args)
-	if err != nil {
-		return err
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	req.builder.imageProber.Reset()
-	image, err := req.builder.getFromImage(req.shlex, req.args[0])
+func initializeStage(d dispatchRequest, cmd *instructions.Stage) error {
+	d.builder.imageProber.Reset()
+	image, err := d.getFromImage(d.shlex, cmd.BaseName)
 	if err != nil {
 		return err
 	}
-	if err := req.builder.buildStages.add(stageName, image); err != nil {
-		return err
-	}
-	req.state.beginStage(stageName, image)
-	req.builder.buildArgs.ResetAllowed()
-	if image.ImageID() == "" {
-		// Typically this means they used "FROM scratch"
-		return nil
+	state := d.state
+	state.beginStage(cmd.Name, image)
+	if len(state.runConfig.OnBuild) > 0 {
+		triggers := state.runConfig.OnBuild
+		state.runConfig.OnBuild = nil
+		return dispatchTriggeredOnBuild(d, triggers)
 	}
-
-	return processOnBuild(req)
+	return nil
 }
 
-func parseBuildStageName(args []string) (string, error) {
-	stageName := ""
-	switch {
-	case len(args) == 3 && strings.EqualFold(args[1], "as"):
-		stageName = strings.ToLower(args[2])
-		if ok, _ := regexp.MatchString("^[a-z][a-z0-9-_\\.]*$", stageName); !ok {
-			return "", errors.Errorf("invalid name for build stage: %q, name can't start with a number or contain symbols", stageName)
+func dispatchTriggeredOnBuild(d dispatchRequest, triggers []string) error {
+	fmt.Fprintf(d.builder.Stdout, "# Executing %d build trigger", len(triggers))
+	if len(triggers) > 1 {
+		fmt.Fprint(d.builder.Stdout, "s")
+	}
+	fmt.Fprintln(d.builder.Stdout)
+	for _, trigger := range triggers {
+		d.state.updateRunConfig()
+		ast, err := parser.Parse(strings.NewReader(trigger))
+		if err != nil {
+			return err
+		}
+		if len(ast.AST.Children) != 1 {
+			return errors.New("onbuild trigger should be a single expression")
+		}
+		cmd, err := instructions.ParseCommand(ast.AST.Children[0])
+		if err != nil {
+			if instructions.IsUnknownInstruction(err) {
+				buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
+			}
+			return err
+		}
+		err = dispatch(d, cmd)
+		if err != nil {
+			return err
 		}
-	case len(args) != 1:
-		return "", errors.New("FROM requires either one or three arguments")
 	}
-
-	return stageName, nil
+	return nil
 }
 
-// scratchImage is used as a token for the empty base image.
+// scratchImage is used as a token for the empty base image. It uses buildStage
+// as a convenient implementation of builder.Image, but is not actually a
+// buildStage.
 var scratchImage builder.Image = &image.Image{}
 
-func (b *Builder) getFromImage(shlex *ShellLex, name string) (builder.Image, error) {
+func (d *dispatchRequest) getExpandedImageName(shlex *ShellLex, name string) (string, error) {
 	substitutionArgs := []string{}
-	for key, value := range b.buildArgs.GetAllMeta() {
+	for key, value := range d.state.buildArgs.GetAllMeta() {
 		substitutionArgs = append(substitutionArgs, key+"="+value)
 	}
 
 	name, err := shlex.ProcessWord(name, substitutionArgs)
 	if err != nil {
-		return nil, err
+		return "", err
 	}
-
+	return name, nil
+}
+func (d *dispatchRequest) getImageOrStage(name string) (builder.Image, error) {
 	var localOnly bool
-	if stage, ok := b.buildStages.getByName(name); ok {
-		name = stage.ImageID()
+	if im, ok := d.stages.getByName(name); ok {
+		name = im.Image
 		localOnly = true
 	}
 
 	// Windows cannot support a container with no base image unless it is LCOW.
 	if name == api.NoBaseImageSpecifier {
 		if runtime.GOOS == "windows" {
-			if b.platform == "windows" || (b.platform != "windows" && !system.LCOWSupported()) {
+			if d.builder.platform == "windows" || (d.builder.platform != "windows" && !system.LCOWSupported()) {
 				return nil, errors.New("Windows does not support FROM scratch")
 			}
 		}
 		return scratchImage, nil
 	}
-	imageMount, err := b.imageSources.Get(name, localOnly)
+	imageMount, err := d.builder.imageSources.Get(name, localOnly)
 	if err != nil {
 		return nil, err
 	}
 	return imageMount.Image(), nil
 }
-
-func processOnBuild(req dispatchRequest) error {
-	dispatchState := req.state
-	// Process ONBUILD triggers if they exist
-	if nTriggers := len(dispatchState.runConfig.OnBuild); nTriggers != 0 {
-		word := "trigger"
-		if nTriggers > 1 {
-			word = "triggers"
-		}
-		fmt.Fprintf(req.builder.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 := dispatchState.runConfig.OnBuild
-	dispatchState.runConfig.OnBuild = []string{}
-
-	// Reset stdin settings as all build actions run without stdin
-	dispatchState.runConfig.OpenStdin = false
-	dispatchState.runConfig.StdinOnce = false
-
-	// parse the ONBUILD triggers by invoking the parser
-	for _, step := range onBuildTriggers {
-		dockerfile, err := parser.Parse(strings.NewReader(step))
-		if err != nil {
-			return err
-		}
-
-		for _, n := range dockerfile.AST.Children {
-			if err := checkDispatch(n); err != nil {
-				return err
-			}
-
-			upperCasedCmd := strings.ToUpper(n.Value)
-			switch upperCasedCmd {
-			case "ONBUILD":
-				return errors.New("Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed")
-			case "MAINTAINER", "FROM":
-				return errors.Errorf("%s isn't allowed as an ONBUILD trigger", upperCasedCmd)
-			}
-		}
-
-		if _, err := dispatchFromDockerfile(req.builder, dockerfile, dispatchState, req.source); err != nil {
-			return err
-		}
+func (d *dispatchRequest) getFromImage(shlex *ShellLex, name string) (builder.Image, error) {
+	name, err := d.getExpandedImageName(shlex, name)
+	if err != nil {
+		return nil, err
 	}
-	return nil
+	return d.getImageOrStage(name)
 }
 
-// ONBUILD RUN echo yo
-//
-// ONBUILD triggers run when the image is used in a FROM statement.
-//
-// ONBUILD handling has a lot of special-case functionality, the heading in
-// evaluator.go and comments around dispatch() in the same file explain the
-// special cases. search for 'OnBuild' in internals.go for additional special
-// cases.
-//
-func onbuild(req dispatchRequest) error {
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("ONBUILD")
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	triggerInstruction := strings.ToUpper(strings.TrimSpace(req.args[0]))
-	switch triggerInstruction {
-	case "ONBUILD":
-		return errors.New("Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed")
-	case "MAINTAINER", "FROM":
-		return fmt.Errorf("%s isn't allowed as an ONBUILD trigger", triggerInstruction)
-	}
+func dispatchOnbuild(d dispatchRequest, c *instructions.OnbuildCommand) error {
 
-	runConfig := req.state.runConfig
-	original := regexp.MustCompile(`(?i)^\s*ONBUILD\s*`).ReplaceAllString(req.original, "")
-	runConfig.OnBuild = append(runConfig.OnBuild, original)
-	return req.builder.commit(req.state, "ONBUILD "+original)
+	d.state.runConfig.OnBuild = append(d.state.runConfig.OnBuild, c.Expression)
+	return d.builder.commit(d.state, "ONBUILD "+c.Expression)
 }
 
 // WORKDIR /tmp
 //
 // Set the working directory for future RUN/CMD/etc statements.
 //
-func workdir(req dispatchRequest) error {
-	if len(req.args) != 1 {
-		return errExactlyOneArgument("WORKDIR")
-	}
-
-	err := req.flags.Parse()
-	if err != nil {
-		return err
-	}
-
-	runConfig := req.state.runConfig
-	// This is from the Dockerfile and will not necessarily be in platform
-	// specific semantics, hence ensure it is converted.
-	runConfig.WorkingDir, err = normalizeWorkdir(req.builder.platform, runConfig.WorkingDir, req.args[0])
+func dispatchWorkdir(d dispatchRequest, c *instructions.WorkdirCommand) error {
+	runConfig := d.state.runConfig
+	var err error
+	runConfig.WorkingDir, err = normalizeWorkdir(d.builder.platform, runConfig.WorkingDir, c.Path)
 	if err != nil {
 		return err
 	}
@@ -400,23 +263,31 @@ func workdir(req dispatchRequest) error {
 	// This avoids having an unnecessary expensive mount/unmount calls
 	// (on Windows in particular) during each container create.
 	// Prior to 1.13, the mkdir was deferred and not executed at this step.
-	if req.builder.disableCommit {
+	if d.builder.disableCommit {
 		// Don't call back into the daemon if we're going through docker commit --change "WORKDIR /foo".
 		// We've already updated the runConfig and that's enough.
 		return nil
 	}
 
 	comment := "WORKDIR " + runConfig.WorkingDir
-	runConfigWithCommentCmd := copyRunConfig(runConfig, withCmdCommentString(comment, req.builder.platform))
-	containerID, err := req.builder.probeAndCreate(req.state, runConfigWithCommentCmd)
+	runConfigWithCommentCmd := copyRunConfig(runConfig, withCmdCommentString(comment, d.builder.platform))
+	containerID, err := d.builder.probeAndCreate(d.state, runConfigWithCommentCmd)
 	if err != nil || containerID == "" {
 		return err
 	}
-	if err := req.builder.docker.ContainerCreateWorkdir(containerID); err != nil {
+	if err := d.builder.docker.ContainerCreateWorkdir(containerID); err != nil {
 		return err
 	}
 
-	return req.builder.commitContainer(req.state, containerID, runConfigWithCommentCmd)
+	return d.builder.commitContainer(d.state, containerID, runConfigWithCommentCmd)
+}
+
+func resolveCmdLine(cmd instructions.ShellDependantCmdLine, runConfig *container.Config, platform string) []string {
+	result := cmd.CmdLine
+	if cmd.PrependShell && result != nil {
+		result = append(getShell(runConfig, platform), result...)
+	}
+	return result
 }
 
 // RUN some command yo
@@ -429,32 +300,21 @@ func workdir(req dispatchRequest) error {
 // RUN echo hi          # cmd /S /C echo hi   (Windows)
 // RUN [ "echo", "hi" ] # echo hi
 //
-func run(req dispatchRequest) error {
-	if !req.state.hasFromImage() {
-		return errors.New("Please provide a source image with `from` prior to run")
-	}
+func dispatchRun(d dispatchRequest, c *instructions.RunCommand) error {
 
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	stateRunConfig := req.state.runConfig
-	args := handleJSONArgs(req.args, req.attributes)
-	if !req.attributes["json"] {
-		args = append(getShell(stateRunConfig, req.builder.platform), args...)
-	}
-	cmdFromArgs := strslice.StrSlice(args)
-	buildArgs := req.builder.buildArgs.FilterAllowed(stateRunConfig.Env)
+	stateRunConfig := d.state.runConfig
+	cmdFromArgs := resolveCmdLine(c.ShellDependantCmdLine, stateRunConfig, d.builder.platform)
+	buildArgs := d.state.buildArgs.FilterAllowed(stateRunConfig.Env)
 
 	saveCmd := cmdFromArgs
 	if len(buildArgs) > 0 {
-		saveCmd = prependEnvOnCmd(req.builder.buildArgs, buildArgs, cmdFromArgs)
+		saveCmd = prependEnvOnCmd(d.state.buildArgs, buildArgs, cmdFromArgs)
 	}
 
 	runConfigForCacheProbe := copyRunConfig(stateRunConfig,
 		withCmd(saveCmd),
 		withEntrypointOverride(saveCmd, nil))
-	hit, err := req.builder.probeCache(req.state, runConfigForCacheProbe)
+	hit, err := d.builder.probeCache(d.state, runConfigForCacheProbe)
 	if err != nil || hit {
 		return err
 	}
@@ -468,11 +328,11 @@ func run(req dispatchRequest) error {
 	runConfig.ArgsEscaped = true
 
 	logrus.Debugf("[BUILDER] Command to be executed: %v", runConfig.Cmd)
-	cID, err := req.builder.create(runConfig)
+	cID, err := d.builder.create(runConfig)
 	if err != nil {
 		return err
 	}
-	if err := req.builder.containerManager.Run(req.builder.clientCtx, cID, req.builder.Stdout, req.builder.Stderr); err != nil {
+	if err := d.builder.containerManager.Run(d.builder.clientCtx, cID, d.builder.Stdout, d.builder.Stderr); err != nil {
 		if err, ok := err.(*statusCodeError); ok {
 			// TODO: change error type, because jsonmessage.JSONError assumes HTTP
 			return &jsonmessage.JSONError{
@@ -485,7 +345,7 @@ func run(req dispatchRequest) error {
 		return err
 	}
 
-	return req.builder.commitContainer(req.state, cID, runConfigForCacheProbe)
+	return d.builder.commitContainer(d.state, cID, runConfigForCacheProbe)
 }
 
 // Derive the command to use for probeCache() and to commit in this container.
@@ -518,139 +378,39 @@ func prependEnvOnCmd(buildArgs *buildArgs, buildArgVars []string, cmd strslice.S
 // Set the default command to run in the container (which may be empty).
 // Argument handling is the same as RUN.
 //
-func cmd(req dispatchRequest) error {
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	runConfig := req.state.runConfig
-	cmdSlice := handleJSONArgs(req.args, req.attributes)
-	if !req.attributes["json"] {
-		cmdSlice = append(getShell(runConfig, req.builder.platform), cmdSlice...)
-	}
-
-	runConfig.Cmd = strslice.StrSlice(cmdSlice)
+func dispatchCmd(d dispatchRequest, c *instructions.CmdCommand) error {
+	runConfig := d.state.runConfig
+	cmd := resolveCmdLine(c.ShellDependantCmdLine, runConfig, d.builder.platform)
+	runConfig.Cmd = cmd
 	// set config as already being escaped, this prevents double escaping on windows
 	runConfig.ArgsEscaped = true
 
-	if err := req.builder.commit(req.state, fmt.Sprintf("CMD %q", cmdSlice)); err != nil {
+	if err := d.builder.commit(d.state, fmt.Sprintf("CMD %q", cmd)); err != nil {
 		return err
 	}
 
-	if len(req.args) != 0 {
-		req.state.cmdSet = true
+	if len(c.ShellDependantCmdLine.CmdLine) != 0 {
+		d.state.cmdSet = true
 	}
 
 	return nil
 }
 
-// parseOptInterval(flag) is the duration of flag.Value, or 0 if
-// empty. An error is reported if the value is given and less than minimum duration.
-func parseOptInterval(f *Flag) (time.Duration, error) {
-	s := f.Value
-	if s == "" {
-		return 0, nil
-	}
-	d, err := time.ParseDuration(s)
-	if err != nil {
-		return 0, err
-	}
-	if d < container.MinimumDuration {
-		return 0, fmt.Errorf("Interval %#v cannot be less than %s", f.name, container.MinimumDuration)
-	}
-	return d, nil
-}
-
 // HEALTHCHECK foo
 //
 // Set the default healthcheck command to run in the container (which may be empty).
 // Argument handling is the same as RUN.
 //
-func healthcheck(req dispatchRequest) error {
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("HEALTHCHECK")
-	}
-	runConfig := req.state.runConfig
-	typ := strings.ToUpper(req.args[0])
-	args := req.args[1:]
-	if typ == "NONE" {
-		if len(args) != 0 {
-			return errors.New("HEALTHCHECK NONE takes no arguments")
-		}
-		test := strslice.StrSlice{typ}
-		runConfig.Healthcheck = &container.HealthConfig{
-			Test: test,
-		}
-	} else {
-		if runConfig.Healthcheck != nil {
-			oldCmd := runConfig.Healthcheck.Test
-			if len(oldCmd) > 0 && oldCmd[0] != "NONE" {
-				fmt.Fprintf(req.builder.Stdout, "Note: overriding previous HEALTHCHECK: %v\n", oldCmd)
-			}
-		}
-
-		healthcheck := container.HealthConfig{}
-
-		flInterval := req.flags.AddString("interval", "")
-		flTimeout := req.flags.AddString("timeout", "")
-		flStartPeriod := req.flags.AddString("start-period", "")
-		flRetries := req.flags.AddString("retries", "")
-
-		if err := req.flags.Parse(); err != nil {
-			return err
-		}
-
-		switch typ {
-		case "CMD":
-			cmdSlice := handleJSONArgs(args, req.attributes)
-			if len(cmdSlice) == 0 {
-				return errors.New("Missing command after HEALTHCHECK CMD")
-			}
-
-			if !req.attributes["json"] {
-				typ = "CMD-SHELL"
-			}
-
-			healthcheck.Test = strslice.StrSlice(append([]string{typ}, cmdSlice...))
-		default:
-			return fmt.Errorf("Unknown type %#v in HEALTHCHECK (try CMD)", typ)
-		}
-
-		interval, err := parseOptInterval(flInterval)
-		if err != nil {
-			return err
-		}
-		healthcheck.Interval = interval
-
-		timeout, err := parseOptInterval(flTimeout)
-		if err != nil {
-			return err
-		}
-		healthcheck.Timeout = timeout
-
-		startPeriod, err := parseOptInterval(flStartPeriod)
-		if err != nil {
-			return err
-		}
-		healthcheck.StartPeriod = startPeriod
-
-		if flRetries.Value != "" {
-			retries, err := strconv.ParseInt(flRetries.Value, 10, 32)
-			if err != nil {
-				return err
-			}
-			if retries < 1 {
-				return fmt.Errorf("--retries must be at least 1 (not %d)", retries)
-			}
-			healthcheck.Retries = int(retries)
-		} else {
-			healthcheck.Retries = 0
+func dispatchHealthcheck(d dispatchRequest, c *instructions.HealthCheckCommand) error {
+	runConfig := d.state.runConfig
+	if runConfig.Healthcheck != nil {
+		oldCmd := runConfig.Healthcheck.Test
+		if len(oldCmd) > 0 && oldCmd[0] != "NONE" {
+			fmt.Fprintf(d.builder.Stdout, "Note: overriding previous HEALTHCHECK: %v\n", oldCmd)
 		}
-
-		runConfig.Healthcheck = &healthcheck
 	}
-
-	return req.builder.commit(req.state, fmt.Sprintf("HEALTHCHECK %q", runConfig.Healthcheck))
+	runConfig.Healthcheck = c.Health
+	return d.builder.commit(d.state, fmt.Sprintf("HEALTHCHECK %q", runConfig.Healthcheck))
 }
 
 // ENTRYPOINT /usr/sbin/nginx
@@ -661,33 +421,15 @@ func healthcheck(req dispatchRequest) error {
 // Handles command processing similar to CMD and RUN, only req.runConfig.Entrypoint
 // is initialized at newBuilder time instead of through argument parsing.
 //
-func entrypoint(req dispatchRequest) error {
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	runConfig := req.state.runConfig
-	parsed := handleJSONArgs(req.args, req.attributes)
-
-	switch {
-	case req.attributes["json"]:
-		// ENTRYPOINT ["echo", "hi"]
-		runConfig.Entrypoint = strslice.StrSlice(parsed)
-	case len(parsed) == 0:
-		// ENTRYPOINT []
-		runConfig.Entrypoint = nil
-	default:
-		// ENTRYPOINT echo hi
-		runConfig.Entrypoint = strslice.StrSlice(append(getShell(runConfig, req.builder.platform), parsed[0]))
-	}
-
-	// when setting the entrypoint if a CMD was not explicitly set then
-	// set the command to nil
-	if !req.state.cmdSet {
+func dispatchEntrypoint(d dispatchRequest, c *instructions.EntrypointCommand) error {
+	runConfig := d.state.runConfig
+	cmd := resolveCmdLine(c.ShellDependantCmdLine, runConfig, d.builder.platform)
+	runConfig.Entrypoint = cmd
+	if !d.state.cmdSet {
 		runConfig.Cmd = nil
 	}
 
-	return req.builder.commit(req.state, fmt.Sprintf("ENTRYPOINT %q", runConfig.Entrypoint))
+	return d.builder.commit(d.state, fmt.Sprintf("ENTRYPOINT %q", runConfig.Entrypoint))
 }
 
 // EXPOSE 6667/tcp 7000/tcp
@@ -695,41 +437,33 @@ func entrypoint(req dispatchRequest) error {
 // Expose ports for links and port mappings. This all ends up in
 // req.runConfig.ExposedPorts for runconfig.
 //
-func expose(req dispatchRequest) error {
-	portsTab := req.args
-
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("EXPOSE")
+func dispatchExpose(d dispatchRequest, c *instructions.ExposeCommand, envs []string) error {
+	// custom multi word expansion
+	// expose $FOO with FOO="80 443" is expanded as EXPOSE [80,443]. This is the only command supporting word to words expansion
+	// so the word processing has been de-generalized
+	ports := []string{}
+	for _, p := range c.Ports {
+		ps, err := d.shlex.ProcessWords(p, envs)
+		if err != nil {
+			return err
+		}
+		ports = append(ports, ps...)
 	}
+	c.Ports = ports
 
-	if err := req.flags.Parse(); err != nil {
+	ps, _, err := nat.ParsePortSpecs(ports)
+	if err != nil {
 		return err
 	}
 
-	runConfig := req.state.runConfig
-	if runConfig.ExposedPorts == nil {
-		runConfig.ExposedPorts = make(nat.PortSet)
+	if d.state.runConfig.ExposedPorts == nil {
+		d.state.runConfig.ExposedPorts = make(nat.PortSet)
 	}
-
-	ports, _, err := nat.ParsePortSpecs(portsTab)
-	if err != nil {
-		return err
+	for p := range ps {
+		d.state.runConfig.ExposedPorts[p] = struct{}{}
 	}
 
-	// instead of using ports directly, we build a list of ports and sort it so
-	// the order is consistent. This prevents cache burst where map ordering
-	// changes between builds
-	portList := make([]string, len(ports))
-	var i int
-	for port := range ports {
-		if _, exists := runConfig.ExposedPorts[port]; !exists {
-			runConfig.ExposedPorts[port] = struct{}{}
-		}
-		portList[i] = string(port)
-		i++
-	}
-	sort.Strings(portList)
-	return req.builder.commit(req.state, "EXPOSE "+strings.Join(portList, " "))
+	return d.builder.commit(d.state, "EXPOSE "+strings.Join(c.Ports, " "))
 }
 
 // USER foo
@@ -737,62 +471,39 @@ func expose(req dispatchRequest) error {
 // Set the user to 'foo' for future commands and when running the
 // ENTRYPOINT/CMD at container run time.
 //
-func user(req dispatchRequest) error {
-	if len(req.args) != 1 {
-		return errExactlyOneArgument("USER")
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-
-	req.state.runConfig.User = req.args[0]
-	return req.builder.commit(req.state, fmt.Sprintf("USER %v", req.args))
+func dispatchUser(d dispatchRequest, c *instructions.UserCommand) error {
+	d.state.runConfig.User = c.User
+	return d.builder.commit(d.state, fmt.Sprintf("USER %v", c.User))
 }
 
 // VOLUME /foo
 //
 // Expose the volume /foo for use. Will also accept the JSON array form.
 //
-func volume(req dispatchRequest) error {
-	if len(req.args) == 0 {
-		return errAtLeastOneArgument("VOLUME")
-	}
-
-	if err := req.flags.Parse(); err != nil {
-		return err
+func dispatchVolume(d dispatchRequest, c *instructions.VolumeCommand) error {
+	if d.state.runConfig.Volumes == nil {
+		d.state.runConfig.Volumes = map[string]struct{}{}
 	}
-
-	runConfig := req.state.runConfig
-	if runConfig.Volumes == nil {
-		runConfig.Volumes = map[string]struct{}{}
-	}
-	for _, v := range req.args {
-		v = strings.TrimSpace(v)
+	for _, v := range c.Volumes {
 		if v == "" {
 			return errors.New("VOLUME specified can not be an empty string")
 		}
-		runConfig.Volumes[v] = struct{}{}
+		d.state.runConfig.Volumes[v] = struct{}{}
 	}
-	return req.builder.commit(req.state, fmt.Sprintf("VOLUME %v", req.args))
+	return d.builder.commit(d.state, fmt.Sprintf("VOLUME %v", c.Volumes))
 }
 
 // STOPSIGNAL signal
 //
 // Set the signal that will be used to kill the container.
-func stopSignal(req dispatchRequest) error {
-	if len(req.args) != 1 {
-		return errExactlyOneArgument("STOPSIGNAL")
-	}
+func dispatchStopSignal(d dispatchRequest, c *instructions.StopSignalCommand) error {
 
-	sig := req.args[0]
-	_, err := signal.ParseSignal(sig)
+	_, err := signal.ParseSignal(c.Signal)
 	if err != nil {
 		return validationError{err}
 	}
-
-	req.state.runConfig.StopSignal = sig
-	return req.builder.commit(req.state, fmt.Sprintf("STOPSIGNAL %v", req.args))
+	d.state.runConfig.StopSignal = c.Signal
+	return d.builder.commit(d.state, fmt.Sprintf("STOPSIGNAL %v", c.Signal))
 }
 
 // ARG name[=value]
@@ -800,89 +511,21 @@ func stopSignal(req dispatchRequest) error {
 // Adds the variable foo to the trusted list of variables that can be passed
 // to builder using the --build-arg flag for expansion/substitution or passing to 'run'.
 // Dockerfile author may optionally set a default value of this variable.
-func arg(req dispatchRequest) error {
-	if len(req.args) != 1 {
-		return errExactlyOneArgument("ARG")
-	}
-
-	var (
-		name       string
-		newValue   string
-		hasDefault bool
-	)
-
-	arg := req.args[0]
-	// 'arg' can just be a name or name-value pair. Note that this is different
-	// from 'env' that handles the split of name and value at the parser level.
-	// The reason for doing it differently for 'arg' is that we support just
-	// defining an arg and not assign it a value (while 'env' always expects a
-	// name-value pair). If possible, it will be good to harmonize the two.
-	if strings.Contains(arg, "=") {
-		parts := strings.SplitN(arg, "=", 2)
-		if len(parts[0]) == 0 {
-			return errBlankCommandNames("ARG")
-		}
-
-		name = parts[0]
-		newValue = parts[1]
-		hasDefault = true
-	} else {
-		name = arg
-		hasDefault = false
-	}
+func dispatchArg(d dispatchRequest, c *instructions.ArgCommand) error {
 
-	var value *string
-	if hasDefault {
-		value = &newValue
+	commitStr := "ARG " + c.Key
+	if c.Value != nil {
+		commitStr += "=" + *c.Value
 	}
-	req.builder.buildArgs.AddArg(name, value)
 
-	// Arg before FROM doesn't add a layer
-	if !req.state.hasFromImage() {
-		req.builder.buildArgs.AddMetaArg(name, value)
-		return nil
-	}
-	return req.builder.commit(req.state, "ARG "+arg)
+	d.state.buildArgs.AddArg(c.Key, c.Value)
+	return d.builder.commit(d.state, commitStr)
 }
 
 // SHELL powershell -command
 //
 // Set the non-default shell to use.
-func shell(req dispatchRequest) error {
-	if err := req.flags.Parse(); err != nil {
-		return err
-	}
-	shellSlice := handleJSONArgs(req.args, req.attributes)
-	switch {
-	case len(shellSlice) == 0:
-		// SHELL []
-		return errAtLeastOneArgument("SHELL")
-	case req.attributes["json"]:
-		// SHELL ["powershell", "-command"]
-		req.state.runConfig.Shell = strslice.StrSlice(shellSlice)
-	default:
-		// SHELL powershell -command - not JSON
-		return errNotJSON("SHELL", req.original)
-	}
-	return req.builder.commit(req.state, fmt.Sprintf("SHELL %v", shellSlice))
-}
-
-func errAtLeastOneArgument(command string) error {
-	return fmt.Errorf("%s requires at least one argument", command)
-}
-
-func errExactlyOneArgument(command string) error {
-	return fmt.Errorf("%s requires exactly one argument", command)
-}
-
-func errAtLeastTwoArguments(command string) error {
-	return fmt.Errorf("%s requires at least two arguments", command)
-}
-
-func errBlankCommandNames(command string) error {
-	return fmt.Errorf("%s names can not be blank", command)
-}
-
-func errTooManyArguments(command string) error {
-	return fmt.Errorf("Bad input to %s, too many arguments", command)
+func dispatchShell(d dispatchRequest, c *instructions.ShellCommand) error {
+	d.state.runConfig.Shell = c.Shell
+	return d.builder.commit(d.state, fmt.Sprintf("SHELL %v", d.state.runConfig.Shell))
 }

+ 192 - 245
builder/dockerfile/dispatchers_test.go

@@ -1,60 +1,29 @@
 package dockerfile
 
 import (
-	"fmt"
-	"runtime"
-	"testing"
-
 	"bytes"
 	"context"
+	"runtime"
+	"testing"
 
 	"github.com/docker/docker/api/types"
 	"github.com/docker/docker/api/types/backend"
 	"github.com/docker/docker/api/types/container"
 	"github.com/docker/docker/api/types/strslice"
 	"github.com/docker/docker/builder"
-	"github.com/docker/docker/builder/dockerfile/parser"
-	"github.com/docker/docker/internal/testutil"
+	"github.com/docker/docker/builder/dockerfile/instructions"
 	"github.com/docker/docker/pkg/system"
 	"github.com/docker/go-connections/nat"
 	"github.com/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
 )
 
-type commandWithFunction struct {
-	name     string
-	function func(args []string) error
-}
-
-func withArgs(f dispatcher) func([]string) error {
-	return func(args []string) error {
-		return f(dispatchRequest{args: args})
-	}
-}
-
-func withBuilderAndArgs(builder *Builder, f dispatcher) func([]string) error {
-	return func(args []string) error {
-		return f(defaultDispatchReq(builder, args...))
-	}
-}
-
-func defaultDispatchReq(builder *Builder, args ...string) dispatchRequest {
-	return dispatchRequest{
-		builder: builder,
-		args:    args,
-		flags:   NewBFlags(),
-		shlex:   NewShellLex(parser.DefaultEscapeToken),
-		state:   &dispatchState{runConfig: &container.Config{}},
-	}
-}
-
 func newBuilderWithMockBackend() *Builder {
 	mockBackend := &MockBackend{}
 	ctx := context.Background()
 	b := &Builder{
 		options:       &types.ImageBuildOptions{},
 		docker:        mockBackend,
-		buildArgs:     newBuildArgs(make(map[string]*string)),
 		Stdout:        new(bytes.Buffer),
 		clientCtx:     ctx,
 		disableCommit: true,
@@ -62,137 +31,84 @@ func newBuilderWithMockBackend() *Builder {
 			Options: &types.ImageBuildOptions{},
 			Backend: mockBackend,
 		}),
-		buildStages:      newBuildStages(),
 		imageProber:      newImageProber(mockBackend, nil, runtime.GOOS, false),
 		containerManager: newContainerManager(mockBackend),
 	}
 	return b
 }
 
-func TestCommandsExactlyOneArgument(t *testing.T) {
-	commands := []commandWithFunction{
-		{"MAINTAINER", withArgs(maintainer)},
-		{"WORKDIR", withArgs(workdir)},
-		{"USER", withArgs(user)},
-		{"STOPSIGNAL", withArgs(stopSignal)},
-	}
-
-	for _, command := range commands {
-		err := command.function([]string{})
-		assert.EqualError(t, err, errExactlyOneArgument(command.name).Error())
-	}
-}
-
-func TestCommandsAtLeastOneArgument(t *testing.T) {
-	commands := []commandWithFunction{
-		{"ENV", withArgs(env)},
-		{"LABEL", withArgs(label)},
-		{"ONBUILD", withArgs(onbuild)},
-		{"HEALTHCHECK", withArgs(healthcheck)},
-		{"EXPOSE", withArgs(expose)},
-		{"VOLUME", withArgs(volume)},
-	}
-
-	for _, command := range commands {
-		err := command.function([]string{})
-		assert.EqualError(t, err, errAtLeastOneArgument(command.name).Error())
-	}
-}
-
-func TestCommandsAtLeastTwoArguments(t *testing.T) {
-	commands := []commandWithFunction{
-		{"ADD", withArgs(add)},
-		{"COPY", withArgs(dispatchCopy)}}
-
-	for _, command := range commands {
-		err := command.function([]string{"arg1"})
-		assert.EqualError(t, err, errAtLeastTwoArguments(command.name).Error())
-	}
-}
-
-func TestCommandsTooManyArguments(t *testing.T) {
-	commands := []commandWithFunction{
-		{"ENV", withArgs(env)},
-		{"LABEL", withArgs(label)}}
-
-	for _, command := range commands {
-		err := command.function([]string{"arg1", "arg2", "arg3"})
-		assert.EqualError(t, err, errTooManyArguments(command.name).Error())
-	}
-}
-
-func TestCommandsBlankNames(t *testing.T) {
-	builder := newBuilderWithMockBackend()
-	commands := []commandWithFunction{
-		{"ENV", withBuilderAndArgs(builder, env)},
-		{"LABEL", withBuilderAndArgs(builder, label)},
-	}
-
-	for _, command := range commands {
-		err := command.function([]string{"", ""})
-		assert.EqualError(t, err, errBlankCommandNames(command.name).Error())
-	}
-}
-
 func TestEnv2Variables(t *testing.T) {
 	b := newBuilderWithMockBackend()
-
-	args := []string{"var1", "val1", "var2", "val2"}
-	req := defaultDispatchReq(b, args...)
-	err := env(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	envCommand := &instructions.EnvCommand{
+		Env: instructions.KeyValuePairs{
+			instructions.KeyValuePair{Key: "var1", Value: "val1"},
+			instructions.KeyValuePair{Key: "var2", Value: "val2"},
+		},
+	}
+	err := dispatch(sb, envCommand)
 	require.NoError(t, err)
 
 	expected := []string{
-		fmt.Sprintf("%s=%s", args[0], args[1]),
-		fmt.Sprintf("%s=%s", args[2], args[3]),
+		"var1=val1",
+		"var2=val2",
 	}
-	assert.Equal(t, expected, req.state.runConfig.Env)
+	assert.Equal(t, expected, sb.state.runConfig.Env)
 }
 
 func TestEnvValueWithExistingRunConfigEnv(t *testing.T) {
 	b := newBuilderWithMockBackend()
-
-	args := []string{"var1", "val1"}
-	req := defaultDispatchReq(b, args...)
-	req.state.runConfig.Env = []string{"var1=old", "var2=fromenv"}
-	err := env(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	sb.state.runConfig.Env = []string{"var1=old", "var2=fromenv"}
+	envCommand := &instructions.EnvCommand{
+		Env: instructions.KeyValuePairs{
+			instructions.KeyValuePair{Key: "var1", Value: "val1"},
+		},
+	}
+	err := dispatch(sb, envCommand)
 	require.NoError(t, err)
-
 	expected := []string{
-		fmt.Sprintf("%s=%s", args[0], args[1]),
+		"var1=val1",
 		"var2=fromenv",
 	}
-	assert.Equal(t, expected, req.state.runConfig.Env)
+	assert.Equal(t, expected, sb.state.runConfig.Env)
 }
 
 func TestMaintainer(t *testing.T) {
 	maintainerEntry := "Some Maintainer <maintainer@example.com>"
-
 	b := newBuilderWithMockBackend()
-	req := defaultDispatchReq(b, maintainerEntry)
-	err := maintainer(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	cmd := &instructions.MaintainerCommand{Maintainer: maintainerEntry}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, maintainerEntry, req.state.maintainer)
+	assert.Equal(t, maintainerEntry, sb.state.maintainer)
 }
 
 func TestLabel(t *testing.T) {
 	labelName := "label"
 	labelValue := "value"
 
-	labelEntry := []string{labelName, labelValue}
 	b := newBuilderWithMockBackend()
-	req := defaultDispatchReq(b, labelEntry...)
-	err := label(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	cmd := &instructions.LabelCommand{
+		Labels: instructions.KeyValuePairs{
+			instructions.KeyValuePair{Key: labelName, Value: labelValue},
+		},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
-	require.Contains(t, req.state.runConfig.Labels, labelName)
-	assert.Equal(t, req.state.runConfig.Labels[labelName], labelValue)
+	require.Contains(t, sb.state.runConfig.Labels, labelName)
+	assert.Equal(t, sb.state.runConfig.Labels[labelName], labelValue)
 }
 
 func TestFromScratch(t *testing.T) {
 	b := newBuilderWithMockBackend()
-	req := defaultDispatchReq(b, "scratch")
-	err := from(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	cmd := &instructions.Stage{
+		BaseName: "scratch",
+	}
+	err := initializeStage(sb, cmd)
 
 	if runtime.GOOS == "windows" && !system.LCOWSupported() {
 		assert.EqualError(t, err, "Windows does not support FROM scratch")
@@ -200,14 +116,14 @@ func TestFromScratch(t *testing.T) {
 	}
 
 	require.NoError(t, err)
-	assert.True(t, req.state.hasFromImage())
-	assert.Equal(t, "", req.state.imageID)
+	assert.True(t, sb.state.hasFromImage())
+	assert.Equal(t, "", sb.state.imageID)
 	// Windows does not set the default path. TODO @jhowardmsft LCOW support. This will need revisiting as we get further into the implementation
 	expected := "PATH=" + system.DefaultPathEnv(runtime.GOOS)
 	if runtime.GOOS == "windows" {
 		expected = ""
 	}
-	assert.Equal(t, []string{expected}, req.state.runConfig.Env)
+	assert.Equal(t, []string{expected}, sb.state.runConfig.Env)
 }
 
 func TestFromWithArg(t *testing.T) {
@@ -219,16 +135,27 @@ func TestFromWithArg(t *testing.T) {
 	}
 	b := newBuilderWithMockBackend()
 	b.docker.(*MockBackend).getImageFunc = getImage
+	args := newBuildArgs(make(map[string]*string))
 
-	require.NoError(t, arg(defaultDispatchReq(b, "THETAG="+tag)))
-	req := defaultDispatchReq(b, "alpine${THETAG}")
-	err := from(req)
+	val := "sometag"
+	metaArg := instructions.ArgCommand{
+		Key:   "THETAG",
+		Value: &val,
+	}
+	cmd := &instructions.Stage{
+		BaseName: "alpine:${THETAG}",
+	}
+	err := processMetaArg(metaArg, NewShellLex('\\'), args)
 
+	sb := newDispatchRequest(b, '\\', nil, args, newStagesBuildResults())
 	require.NoError(t, err)
-	assert.Equal(t, expected, req.state.imageID)
-	assert.Equal(t, expected, req.state.baseImage.ImageID())
-	assert.Len(t, b.buildArgs.GetAllAllowed(), 0)
-	assert.Len(t, b.buildArgs.GetAllMeta(), 1)
+	err = initializeStage(sb, cmd)
+	require.NoError(t, err)
+
+	assert.Equal(t, expected, sb.state.imageID)
+	assert.Equal(t, expected, sb.state.baseImage.ImageID())
+	assert.Len(t, sb.state.buildArgs.GetAllAllowed(), 0)
+	assert.Len(t, sb.state.buildArgs.GetAllMeta(), 1)
 }
 
 func TestFromWithUndefinedArg(t *testing.T) {
@@ -240,74 +167,74 @@ func TestFromWithUndefinedArg(t *testing.T) {
 	}
 	b := newBuilderWithMockBackend()
 	b.docker.(*MockBackend).getImageFunc = getImage
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+
 	b.options.BuildArgs = map[string]*string{"THETAG": &tag}
 
-	req := defaultDispatchReq(b, "alpine${THETAG}")
-	err := from(req)
+	cmd := &instructions.Stage{
+		BaseName: "alpine${THETAG}",
+	}
+	err := initializeStage(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, expected, req.state.imageID)
+	assert.Equal(t, expected, sb.state.imageID)
 }
 
-func TestFromMultiStageWithScratchNamedStage(t *testing.T) {
-	if runtime.GOOS == "windows" {
-		t.Skip("Windows does not support scratch")
-	}
+func TestFromMultiStageWithNamedStage(t *testing.T) {
 	b := newBuilderWithMockBackend()
-	req := defaultDispatchReq(b, "scratch", "AS", "base")
-
-	require.NoError(t, from(req))
-	assert.True(t, req.state.hasFromImage())
-
-	req.args = []string{"base"}
-	require.NoError(t, from(req))
-	assert.True(t, req.state.hasFromImage())
-}
-
-func TestOnbuildIllegalTriggers(t *testing.T) {
-	triggers := []struct{ command, expectedError string }{
-		{"ONBUILD", "Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed"},
-		{"MAINTAINER", "MAINTAINER isn't allowed as an ONBUILD trigger"},
-		{"FROM", "FROM isn't allowed as an ONBUILD trigger"}}
-
-	for _, trigger := range triggers {
-		b := newBuilderWithMockBackend()
-
-		err := onbuild(defaultDispatchReq(b, trigger.command))
-		testutil.ErrorContains(t, err, trigger.expectedError)
-	}
+	firstFrom := &instructions.Stage{BaseName: "someimg", Name: "base"}
+	secondFrom := &instructions.Stage{BaseName: "base"}
+	previousResults := newStagesBuildResults()
+	firstSB := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), previousResults)
+	secondSB := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), previousResults)
+	err := initializeStage(firstSB, firstFrom)
+	require.NoError(t, err)
+	assert.True(t, firstSB.state.hasFromImage())
+	previousResults.indexed["base"] = firstSB.state.runConfig
+	previousResults.flat = append(previousResults.flat, firstSB.state.runConfig)
+	err = initializeStage(secondSB, secondFrom)
+	require.NoError(t, err)
+	assert.True(t, secondSB.state.hasFromImage())
 }
 
 func TestOnbuild(t *testing.T) {
 	b := newBuilderWithMockBackend()
-
-	req := defaultDispatchReq(b, "ADD", ".", "/app/src")
-	req.original = "ONBUILD ADD . /app/src"
-	req.state.runConfig = &container.Config{}
-
-	err := onbuild(req)
+	sb := newDispatchRequest(b, '\\', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	cmd := &instructions.OnbuildCommand{
+		Expression: "ADD . /app/src",
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, "ADD . /app/src", req.state.runConfig.OnBuild[0])
+	assert.Equal(t, "ADD . /app/src", sb.state.runConfig.OnBuild[0])
 }
 
 func TestWorkdir(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 	workingDir := "/app"
 	if runtime.GOOS == "windows" {
-		workingDir = "C:\app"
+		workingDir = "C:\\app"
+	}
+	cmd := &instructions.WorkdirCommand{
+		Path: workingDir,
 	}
 
-	req := defaultDispatchReq(b, workingDir)
-	err := workdir(req)
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, workingDir, req.state.runConfig.WorkingDir)
+	assert.Equal(t, workingDir, sb.state.runConfig.WorkingDir)
 }
 
 func TestCmd(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 	command := "./executable"
 
-	req := defaultDispatchReq(b, command)
-	err := cmd(req)
+	cmd := &instructions.CmdCommand{
+		ShellDependantCmdLine: instructions.ShellDependantCmdLine{
+			CmdLine:      strslice.StrSlice{command},
+			PrependShell: true,
+		},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
 	var expectedCommand strslice.StrSlice
@@ -317,42 +244,56 @@ func TestCmd(t *testing.T) {
 		expectedCommand = strslice.StrSlice(append([]string{"/bin/sh"}, "-c", command))
 	}
 
-	assert.Equal(t, expectedCommand, req.state.runConfig.Cmd)
-	assert.True(t, req.state.cmdSet)
+	assert.Equal(t, expectedCommand, sb.state.runConfig.Cmd)
+	assert.True(t, sb.state.cmdSet)
 }
 
 func TestHealthcheckNone(t *testing.T) {
 	b := newBuilderWithMockBackend()
-
-	req := defaultDispatchReq(b, "NONE")
-	err := healthcheck(req)
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	cmd := &instructions.HealthCheckCommand{
+		Health: &container.HealthConfig{
+			Test: []string{"NONE"},
+		},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
-	require.NotNil(t, req.state.runConfig.Healthcheck)
-	assert.Equal(t, []string{"NONE"}, req.state.runConfig.Healthcheck.Test)
+	require.NotNil(t, sb.state.runConfig.Healthcheck)
+	assert.Equal(t, []string{"NONE"}, sb.state.runConfig.Healthcheck.Test)
 }
 
 func TestHealthcheckCmd(t *testing.T) {
-	b := newBuilderWithMockBackend()
 
-	args := []string{"CMD", "curl", "-f", "http://localhost/", "||", "exit", "1"}
-	req := defaultDispatchReq(b, args...)
-	err := healthcheck(req)
+	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	expectedTest := []string{"CMD-SHELL", "curl -f http://localhost/ || exit 1"}
+	cmd := &instructions.HealthCheckCommand{
+		Health: &container.HealthConfig{
+			Test: expectedTest,
+		},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
-	require.NotNil(t, req.state.runConfig.Healthcheck)
-	expectedTest := []string{"CMD-SHELL", "curl -f http://localhost/ || exit 1"}
-	assert.Equal(t, expectedTest, req.state.runConfig.Healthcheck.Test)
+	require.NotNil(t, sb.state.runConfig.Healthcheck)
+	assert.Equal(t, expectedTest, sb.state.runConfig.Healthcheck.Test)
 }
 
 func TestEntrypoint(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 	entrypointCmd := "/usr/sbin/nginx"
 
-	req := defaultDispatchReq(b, entrypointCmd)
-	err := entrypoint(req)
+	cmd := &instructions.EntrypointCommand{
+		ShellDependantCmdLine: instructions.ShellDependantCmdLine{
+			CmdLine:      strslice.StrSlice{entrypointCmd},
+			PrependShell: true,
+		},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	require.NotNil(t, req.state.runConfig.Entrypoint)
+	require.NotNil(t, sb.state.runConfig.Entrypoint)
 
 	var expectedEntrypoint strslice.StrSlice
 	if runtime.GOOS == "windows" {
@@ -360,99 +301,99 @@ func TestEntrypoint(t *testing.T) {
 	} else {
 		expectedEntrypoint = strslice.StrSlice(append([]string{"/bin/sh"}, "-c", entrypointCmd))
 	}
-	assert.Equal(t, expectedEntrypoint, req.state.runConfig.Entrypoint)
+	assert.Equal(t, expectedEntrypoint, sb.state.runConfig.Entrypoint)
 }
 
 func TestExpose(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 
 	exposedPort := "80"
-	req := defaultDispatchReq(b, exposedPort)
-	err := expose(req)
+	cmd := &instructions.ExposeCommand{
+		Ports: []string{exposedPort},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
-	require.NotNil(t, req.state.runConfig.ExposedPorts)
-	require.Len(t, req.state.runConfig.ExposedPorts, 1)
+	require.NotNil(t, sb.state.runConfig.ExposedPorts)
+	require.Len(t, sb.state.runConfig.ExposedPorts, 1)
 
 	portsMapping, err := nat.ParsePortSpec(exposedPort)
 	require.NoError(t, err)
-	assert.Contains(t, req.state.runConfig.ExposedPorts, portsMapping[0].Port)
+	assert.Contains(t, sb.state.runConfig.ExposedPorts, portsMapping[0].Port)
 }
 
 func TestUser(t *testing.T) {
 	b := newBuilderWithMockBackend()
-	userCommand := "foo"
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 
-	req := defaultDispatchReq(b, userCommand)
-	err := user(req)
+	cmd := &instructions.UserCommand{
+		User: "test",
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, userCommand, req.state.runConfig.User)
+	assert.Equal(t, "test", sb.state.runConfig.User)
 }
 
 func TestVolume(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 
 	exposedVolume := "/foo"
 
-	req := defaultDispatchReq(b, exposedVolume)
-	err := volume(req)
+	cmd := &instructions.VolumeCommand{
+		Volumes: []string{exposedVolume},
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-
-	require.NotNil(t, req.state.runConfig.Volumes)
-	assert.Len(t, req.state.runConfig.Volumes, 1)
-	assert.Contains(t, req.state.runConfig.Volumes, exposedVolume)
+	require.NotNil(t, sb.state.runConfig.Volumes)
+	assert.Len(t, sb.state.runConfig.Volumes, 1)
+	assert.Contains(t, sb.state.runConfig.Volumes, exposedVolume)
 }
 
 func TestStopSignal(t *testing.T) {
+	if runtime.GOOS == "windows" {
+		t.Skip("Windows does not support stopsignal")
+		return
+	}
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 	signal := "SIGKILL"
 
-	req := defaultDispatchReq(b, signal)
-	err := stopSignal(req)
+	cmd := &instructions.StopSignalCommand{
+		Signal: signal,
+	}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
-	assert.Equal(t, signal, req.state.runConfig.StopSignal)
+	assert.Equal(t, signal, sb.state.runConfig.StopSignal)
 }
 
 func TestArg(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 
 	argName := "foo"
 	argVal := "bar"
-	argDef := fmt.Sprintf("%s=%s", argName, argVal)
-
-	err := arg(defaultDispatchReq(b, argDef))
+	cmd := &instructions.ArgCommand{Key: argName, Value: &argVal}
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
 	expected := map[string]string{argName: argVal}
-	assert.Equal(t, expected, b.buildArgs.GetAllAllowed())
+	assert.Equal(t, expected, sb.state.buildArgs.GetAllAllowed())
 }
 
 func TestShell(t *testing.T) {
 	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', nil, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
 
 	shellCmd := "powershell"
-	req := defaultDispatchReq(b, shellCmd)
-	req.attributes = map[string]bool{"json": true}
+	cmd := &instructions.ShellCommand{Shell: strslice.StrSlice{shellCmd}}
 
-	err := shell(req)
+	err := dispatch(sb, cmd)
 	require.NoError(t, err)
 
 	expectedShell := strslice.StrSlice([]string{shellCmd})
-	assert.Equal(t, expectedShell, req.state.runConfig.Shell)
-}
-
-func TestParseOptInterval(t *testing.T) {
-	flInterval := &Flag{
-		name:     "interval",
-		flagType: stringType,
-		Value:    "50ns",
-	}
-	_, err := parseOptInterval(flInterval)
-	testutil.ErrorContains(t, err, "cannot be less than 1ms")
-
-	flInterval.Value = "1ms"
-	_, err = parseOptInterval(flInterval)
-	require.NoError(t, err)
+	assert.Equal(t, expectedShell, sb.state.runConfig.Shell)
 }
 
 func TestPrependEnvOnCmd(t *testing.T) {
@@ -469,8 +410,10 @@ func TestPrependEnvOnCmd(t *testing.T) {
 
 func TestRunWithBuildArgs(t *testing.T) {
 	b := newBuilderWithMockBackend()
-	b.buildArgs.argsFromOptions["HTTP_PROXY"] = strPtr("FOO")
+	args := newBuildArgs(make(map[string]*string))
+	args.argsFromOptions["HTTP_PROXY"] = strPtr("FOO")
 	b.disableCommit = false
+	sb := newDispatchRequest(b, '`', nil, args, newStagesBuildResults())
 
 	runConfig := &container.Config{}
 	origCmd := strslice.StrSlice([]string{"cmd", "in", "from", "image"})
@@ -512,14 +455,18 @@ func TestRunWithBuildArgs(t *testing.T) {
 		assert.Equal(t, strslice.StrSlice(nil), cfg.Config.Entrypoint)
 		return "", nil
 	}
-
-	req := defaultDispatchReq(b, "abcdef")
-	require.NoError(t, from(req))
-	b.buildArgs.AddArg("one", strPtr("two"))
-
-	req.args = []string{"echo foo"}
-	require.NoError(t, run(req))
+	from := &instructions.Stage{BaseName: "abcdef"}
+	err := initializeStage(sb, from)
+	require.NoError(t, err)
+	sb.state.buildArgs.AddArg("one", strPtr("two"))
+	run := &instructions.RunCommand{
+		ShellDependantCmdLine: instructions.ShellDependantCmdLine{
+			CmdLine:      strslice.StrSlice{"echo foo"},
+			PrependShell: true,
+		},
+	}
+	require.NoError(t, dispatch(sb, run))
 
 	// Check that runConfig.Cmd has not been modified by run
-	assert.Equal(t, origCmd, req.state.runConfig.Cmd)
+	assert.Equal(t, origCmd, sb.state.runConfig.Cmd)
 }

+ 0 - 5
builder/dockerfile/dispatchers_unix.go

@@ -4,7 +4,6 @@ package dockerfile
 
 import (
 	"errors"
-	"fmt"
 	"os"
 	"path/filepath"
 )
@@ -23,10 +22,6 @@ func normalizeWorkdir(_ string, current string, requested string) (string, error
 	return requested, nil
 }
 
-func errNotJSON(command, _ string) error {
-	return fmt.Errorf("%s requires the arguments to be in JSON form", command)
-}
-
 // equalEnvKeys compare two strings and returns true if they are equal. On
 // Windows this comparison is case insensitive.
 func equalEnvKeys(from, to string) bool {

+ 0 - 19
builder/dockerfile/dispatchers_windows.go

@@ -94,25 +94,6 @@ func normalizeWorkdirWindows(current string, requested string) (string, error) {
 	return (strings.ToUpper(string(requested[0])) + requested[1:]), nil
 }
 
-func errNotJSON(command, original string) error {
-	// For Windows users, give a hint if it looks like it might contain
-	// a path which hasn't been escaped such as ["c:\windows\system32\prog.exe", "-param"],
-	// as JSON must be escaped. Unfortunate...
-	//
-	// Specifically looking for quote-driveletter-colon-backslash, there's no
-	// double backslash and a [] pair. No, this is not perfect, but it doesn't
-	// have to be. It's simply a hint to make life a little easier.
-	extra := ""
-	original = filepath.FromSlash(strings.ToLower(strings.Replace(strings.ToLower(original), strings.ToLower(command)+" ", "", -1)))
-	if len(regexp.MustCompile(`"[a-z]:\\.*`).FindStringSubmatch(original)) > 0 &&
-		!strings.Contains(original, `\\`) &&
-		strings.Contains(original, "[") &&
-		strings.Contains(original, "]") {
-		extra = fmt.Sprintf(`. It looks like '%s' includes a file path without an escaped back-slash. JSON requires back-slashes to be escaped such as ["c:\\path\\to\\file.exe", "/parameter"]`, original)
-	}
-	return fmt.Errorf("%s requires the arguments to be in JSON form%s", command, extra)
-}
-
 // equalEnvKeys compare two strings and returns true if they are equal. On
 // Windows this comparison is case insensitive.
 func equalEnvKeys(from, to string) bool {

+ 137 - 221
builder/dockerfile/evaluator.go

@@ -20,183 +20,178 @@
 package dockerfile
 
 import (
-	"bytes"
-	"fmt"
+	"reflect"
 	"runtime"
+	"strconv"
 	"strings"
 
 	"github.com/docker/docker/api/types/container"
 	"github.com/docker/docker/builder"
-	"github.com/docker/docker/builder/dockerfile/command"
-	"github.com/docker/docker/builder/dockerfile/parser"
+	"github.com/docker/docker/builder/dockerfile/instructions"
 	"github.com/docker/docker/pkg/system"
 	"github.com/docker/docker/runconfig/opts"
 	"github.com/pkg/errors"
 )
 
-// Environment variable interpolation will happen on these statements only.
-var replaceEnvAllowed = map[string]bool{
-	command.Env:        true,
-	command.Label:      true,
-	command.Add:        true,
-	command.Copy:       true,
-	command.Workdir:    true,
-	command.Expose:     true,
-	command.Volume:     true,
-	command.User:       true,
-	command.StopSignal: true,
-	command.Arg:        true,
-}
+func dispatch(d dispatchRequest, cmd instructions.Command) error {
+	if c, ok := cmd.(instructions.PlatformSpecific); ok {
+		err := c.CheckPlatform(d.builder.platform)
+		if err != nil {
+			return validationError{err}
+		}
+	}
+	runConfigEnv := d.state.runConfig.Env
+	envs := append(runConfigEnv, d.state.buildArgs.FilterAllowed(runConfigEnv)...)
 
-// Certain commands are allowed to have their args split into more
-// words after env var replacements. Meaning:
-//   ENV foo="123 456"
-//   EXPOSE $foo
-// should result in the same thing as:
-//   EXPOSE 123 456
-// and not treat "123 456" as a single word.
-// Note that: EXPOSE "$foo" and EXPOSE $foo are not the same thing.
-// Quotes will cause it to still be treated as single word.
-var allowWordExpansion = map[string]bool{
-	command.Expose: true,
-}
+	if ex, ok := cmd.(instructions.SupportsSingleWordExpansion); ok {
+		err := ex.Expand(func(word string) (string, error) {
+			return d.shlex.ProcessWord(word, envs)
+		})
+		if err != nil {
+			return validationError{err}
+		}
+	}
 
-type dispatchRequest struct {
-	builder    *Builder // TODO: replace this with a smaller interface
-	args       []string
-	attributes map[string]bool
-	flags      *BFlags
-	original   string
-	shlex      *ShellLex
-	state      *dispatchState
-	source     builder.Source
+	if d.builder.options.ForceRemove {
+		defer d.builder.containerManager.RemoveAll(d.builder.Stdout)
+	}
+
+	switch c := cmd.(type) {
+	case *instructions.EnvCommand:
+		return dispatchEnv(d, c)
+	case *instructions.MaintainerCommand:
+		return dispatchMaintainer(d, c)
+	case *instructions.LabelCommand:
+		return dispatchLabel(d, c)
+	case *instructions.AddCommand:
+		return dispatchAdd(d, c)
+	case *instructions.CopyCommand:
+		return dispatchCopy(d, c)
+	case *instructions.OnbuildCommand:
+		return dispatchOnbuild(d, c)
+	case *instructions.WorkdirCommand:
+		return dispatchWorkdir(d, c)
+	case *instructions.RunCommand:
+		return dispatchRun(d, c)
+	case *instructions.CmdCommand:
+		return dispatchCmd(d, c)
+	case *instructions.HealthCheckCommand:
+		return dispatchHealthcheck(d, c)
+	case *instructions.EntrypointCommand:
+		return dispatchEntrypoint(d, c)
+	case *instructions.ExposeCommand:
+		return dispatchExpose(d, c, envs)
+	case *instructions.UserCommand:
+		return dispatchUser(d, c)
+	case *instructions.VolumeCommand:
+		return dispatchVolume(d, c)
+	case *instructions.StopSignalCommand:
+		return dispatchStopSignal(d, c)
+	case *instructions.ArgCommand:
+		return dispatchArg(d, c)
+	case *instructions.ShellCommand:
+		return dispatchShell(d, c)
+	}
+	return errors.Errorf("unsupported command type: %v", reflect.TypeOf(cmd))
 }
 
-func newDispatchRequestFromOptions(options dispatchOptions, builder *Builder, args []string) dispatchRequest {
-	return dispatchRequest{
-		builder:    builder,
-		args:       args,
-		attributes: options.node.Attributes,
-		original:   options.node.Original,
-		flags:      NewBFlagsWithArgs(options.node.Flags),
-		shlex:      options.shlex,
-		state:      options.state,
-		source:     options.source,
-	}
+// dispatchState is a data object which is modified by dispatchers
+type dispatchState struct {
+	runConfig  *container.Config
+	maintainer string
+	cmdSet     bool
+	imageID    string
+	baseImage  builder.Image
+	stageName  string
+	buildArgs  *buildArgs
 }
 
-type dispatcher func(dispatchRequest) error
+func newDispatchState(baseArgs *buildArgs) *dispatchState {
+	args := baseArgs.Clone()
+	args.ResetAllowed()
+	return &dispatchState{runConfig: &container.Config{}, buildArgs: args}
+}
 
-var evaluateTable map[string]dispatcher
+type stagesBuildResults struct {
+	flat    []*container.Config
+	indexed map[string]*container.Config
+}
 
-func init() {
-	evaluateTable = map[string]dispatcher{
-		command.Add:         add,
-		command.Arg:         arg,
-		command.Cmd:         cmd,
-		command.Copy:        dispatchCopy, // copy() is a go builtin
-		command.Entrypoint:  entrypoint,
-		command.Env:         env,
-		command.Expose:      expose,
-		command.From:        from,
-		command.Healthcheck: healthcheck,
-		command.Label:       label,
-		command.Maintainer:  maintainer,
-		command.Onbuild:     onbuild,
-		command.Run:         run,
-		command.Shell:       shell,
-		command.StopSignal:  stopSignal,
-		command.User:        user,
-		command.Volume:      volume,
-		command.Workdir:     workdir,
+func newStagesBuildResults() *stagesBuildResults {
+	return &stagesBuildResults{
+		indexed: make(map[string]*container.Config),
 	}
 }
 
-func formatStep(stepN int, stepTotal int) string {
-	return fmt.Sprintf("%d/%d", stepN+1, stepTotal)
+func (r *stagesBuildResults) getByName(name string) (*container.Config, bool) {
+	c, ok := r.indexed[strings.ToLower(name)]
+	return c, ok
 }
 
-// This method is the entrypoint to all statement handling routines.
-//
-// Almost all nodes will have this structure:
-// Child[Node, Node, Node] where Child is from parser.Node.Children and each
-// node comes from parser.Node.Next. This forms a "line" with a statement and
-// arguments and we process them in this normalized form by hitting
-// evaluateTable with the leaf nodes of the command and the Builder object.
-//
-// ONBUILD is a special case; in this case the parser will emit:
-// Child[Node, Child[Node, Node...]] where the first node is the literal
-// "onbuild" and the child entrypoint is the command of the ONBUILD statement,
-// 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(options dispatchOptions) (*dispatchState, error) {
-	node := options.node
-	cmd := node.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.
-	if err := platformSupports(strings.ToLower(cmd)); err != nil {
-		buildsFailed.WithValues(metricsCommandNotSupportedError).Inc()
-		return nil, validationError{err}
+func (r *stagesBuildResults) validateIndex(i int) error {
+	if i == len(r.flat) {
+		return errors.New("refers to current build stage")
 	}
-
-	msg := bytes.NewBufferString(fmt.Sprintf("Step %s : %s%s",
-		options.stepMsg, upperCasedCmd, formatFlags(node.Flags)))
-
-	args := []string{}
-	ast := node
-	if cmd == command.Onbuild {
-		var err error
-		ast, args, err = handleOnBuildNode(node, msg)
-		if err != nil {
-			return nil, validationError{err}
-		}
+	if i < 0 || i > len(r.flat) {
+		return errors.New("index out of bounds")
 	}
+	return nil
+}
 
-	runConfigEnv := options.state.runConfig.Env
-	envs := append(runConfigEnv, b.buildArgs.FilterAllowed(runConfigEnv)...)
-	processFunc := createProcessWordFunc(options.shlex, cmd, envs)
-	words, err := getDispatchArgsFromNode(ast, processFunc, msg)
+func (r *stagesBuildResults) get(nameOrIndex string) (*container.Config, error) {
+	if c, ok := r.getByName(nameOrIndex); ok {
+		return c, nil
+	}
+	ix, err := strconv.ParseInt(nameOrIndex, 10, 0)
 	if err != nil {
-		buildsFailed.WithValues(metricsErrorProcessingCommandsError).Inc()
-		return nil, validationError{err}
+		return nil, nil
 	}
-	args = append(args, words...)
+	if err := r.validateIndex(int(ix)); err != nil {
+		return nil, err
+	}
+	return r.flat[ix], nil
+}
 
-	fmt.Fprintln(b.Stdout, msg.String())
+func (r *stagesBuildResults) checkStageNameAvailable(name string) error {
+	if name != "" {
+		if _, ok := r.getByName(name); ok {
+			return errors.Errorf("%s stage name already used", name)
+		}
+	}
+	return nil
+}
 
-	f, ok := evaluateTable[cmd]
-	if !ok {
-		buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
-		return nil, validationError{errors.Errorf("unknown instruction: %s", upperCasedCmd)}
+func (r *stagesBuildResults) commitStage(name string, config *container.Config) error {
+	if name != "" {
+		if _, ok := r.getByName(name); ok {
+			return errors.Errorf("%s stage name already used", name)
+		}
+		r.indexed[strings.ToLower(name)] = config
 	}
-	options.state.updateRunConfig()
-	err = f(newDispatchRequestFromOptions(options, b, args))
-	return options.state, err
+	r.flat = append(r.flat, config)
+	return nil
+}
+
+func commitStage(state *dispatchState, stages *stagesBuildResults) error {
+	return stages.commitStage(state.stageName, state.runConfig)
 }
 
-type dispatchOptions struct {
+type dispatchRequest struct {
 	state   *dispatchState
-	stepMsg string
-	node    *parser.Node
 	shlex   *ShellLex
+	builder *Builder
 	source  builder.Source
+	stages  *stagesBuildResults
 }
 
-// dispatchState is a data object which is modified by dispatchers
-type dispatchState struct {
-	runConfig  *container.Config
-	maintainer string
-	cmdSet     bool
-	imageID    string
-	baseImage  builder.Image
-	stageName  string
-}
-
-func newDispatchState() *dispatchState {
-	return &dispatchState{runConfig: &container.Config{}}
+func newDispatchRequest(builder *Builder, escapeToken rune, source builder.Source, buildArgs *buildArgs, stages *stagesBuildResults) dispatchRequest {
+	return dispatchRequest{
+		state:   newDispatchState(buildArgs),
+		shlex:   NewShellLex(escapeToken),
+		builder: builder,
+		source:  source,
+		stages:  stages,
+	}
 }
 
 func (s *dispatchState) updateRunConfig() {
@@ -220,12 +215,14 @@ func (s *dispatchState) beginStage(stageName string, image builder.Image) {
 	s.imageID = image.ImageID()
 
 	if image.RunConfig() != nil {
-		s.runConfig = image.RunConfig()
+		s.runConfig = copyRunConfig(image.RunConfig()) // copy avoids referencing the same instance when 2 stages have the same base
 	} else {
 		s.runConfig = &container.Config{}
 	}
 	s.baseImage = image
 	s.setDefaultPath()
+	s.runConfig.OpenStdin = false
+	s.runConfig.StdinOnce = false
 }
 
 // Add the default PATH to runConfig.ENV if one exists for the platform and there
@@ -244,84 +241,3 @@ func (s *dispatchState) setDefaultPath() {
 		s.runConfig.Env = append(s.runConfig.Env, "PATH="+system.DefaultPathEnv(platform))
 	}
 }
-
-func handleOnBuildNode(ast *parser.Node, msg *bytes.Buffer) (*parser.Node, []string, error) {
-	if ast.Next == nil {
-		return nil, nil, validationError{errors.New("ONBUILD requires at least one argument")}
-	}
-	ast = ast.Next.Children[0]
-	msg.WriteString(" " + ast.Value + formatFlags(ast.Flags))
-	return ast, []string{ast.Value}, nil
-}
-
-func formatFlags(flags []string) string {
-	if len(flags) > 0 {
-		return " " + strings.Join(flags, " ")
-	}
-	return ""
-}
-
-func getDispatchArgsFromNode(ast *parser.Node, processFunc processWordFunc, msg *bytes.Buffer) ([]string, error) {
-	args := []string{}
-	for i := 0; ast.Next != nil; i++ {
-		ast = ast.Next
-		words, err := processFunc(ast.Value)
-		if err != nil {
-			return nil, err
-		}
-		args = append(args, words...)
-		msg.WriteString(" " + ast.Value)
-	}
-	return args, nil
-}
-
-type processWordFunc func(string) ([]string, error)
-
-func createProcessWordFunc(shlex *ShellLex, cmd string, envs []string) processWordFunc {
-	switch {
-	case !replaceEnvAllowed[cmd]:
-		return func(word string) ([]string, error) {
-			return []string{word}, nil
-		}
-	case allowWordExpansion[cmd]:
-		return func(word string) ([]string, error) {
-			return shlex.ProcessWords(word, envs)
-		}
-	default:
-		return func(word string) ([]string, error) {
-			word, err := shlex.ProcessWord(word, envs)
-			return []string{word}, err
-		}
-	}
-}
-
-// checkDispatch does a simple check for syntax errors of the Dockerfile.
-// Because some of the instructions can only be validated through runtime,
-// arg, env, etc., this syntax check will not be complete and could not replace
-// the runtime check. Instead, this function is only a helper that allows
-// user to find out the obvious error in Dockerfile earlier on.
-func checkDispatch(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.
-	if err := platformSupports(strings.ToLower(cmd)); err != nil {
-		return err
-	}
-
-	// The instruction itself is ONBUILD, we will make sure it follows with at
-	// least one argument
-	if upperCasedCmd == "ONBUILD" {
-		if ast.Next == nil {
-			buildsFailed.WithValues(metricsMissingOnbuildArgumentsError).Inc()
-			return errors.New("ONBUILD requires at least one argument")
-		}
-	}
-
-	if _, ok := evaluateTable[cmd]; ok {
-		return nil
-	}
-	buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
-	return errors.Errorf("unknown instruction: %s", upperCasedCmd)
-}

+ 48 - 111
builder/dockerfile/evaluator_test.go

@@ -1,13 +1,9 @@
 package dockerfile
 
 import (
-	"io/ioutil"
-	"strings"
 	"testing"
 
-	"github.com/docker/docker/api/types"
-	"github.com/docker/docker/api/types/container"
-	"github.com/docker/docker/builder/dockerfile/parser"
+	"github.com/docker/docker/builder/dockerfile/instructions"
 	"github.com/docker/docker/builder/remotecontext"
 	"github.com/docker/docker/internal/testutil"
 	"github.com/docker/docker/pkg/archive"
@@ -15,8 +11,9 @@ import (
 )
 
 type dispatchTestCase struct {
-	name, dockerfile, expectedError string
-	files                           map[string]string
+	name, expectedError string
+	cmd                 instructions.Command
+	files               map[string]string
 }
 
 func init() {
@@ -24,108 +21,73 @@ func init() {
 }
 
 func initDispatchTestCases() []dispatchTestCase {
-	dispatchTestCases := []dispatchTestCase{{
-		name: "copyEmptyWhitespace",
-		dockerfile: `COPY
-	quux \
-      bar`,
-		expectedError: "COPY requires at least two arguments",
-	},
-		{
-			name:          "ONBUILD forbidden FROM",
-			dockerfile:    "ONBUILD FROM scratch",
-			expectedError: "FROM isn't allowed as an ONBUILD trigger",
-			files:         nil,
-		},
-		{
-			name:          "ONBUILD forbidden MAINTAINER",
-			dockerfile:    "ONBUILD MAINTAINER docker.io",
-			expectedError: "MAINTAINER isn't allowed as an ONBUILD trigger",
-			files:         nil,
-		},
-		{
-			name:          "ARG two arguments",
-			dockerfile:    "ARG foo bar",
-			expectedError: "ARG requires exactly one argument",
-			files:         nil,
-		},
-		{
-			name:          "MAINTAINER unknown flag",
-			dockerfile:    "MAINTAINER --boo joe@example.com",
-			expectedError: "Unknown flag: boo",
-			files:         nil,
-		},
-		{
-			name:          "ADD multiple files to file",
-			dockerfile:    "ADD file1.txt file2.txt test",
-			expectedError: "When using ADD with more than one source file, the destination must be a directory and end with a /",
-			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
-		},
-		{
-			name:          "JSON ADD multiple files to file",
-			dockerfile:    `ADD ["file1.txt", "file2.txt", "test"]`,
-			expectedError: "When using ADD with more than one source file, the destination must be a directory and end with a /",
-			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
-		},
-		{
-			name:          "Wildcard ADD multiple files to file",
-			dockerfile:    "ADD file*.txt test",
+	dispatchTestCases := []dispatchTestCase{
+		{
+			name: "ADD multiple files to file",
+			cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"file1.txt",
+				"file2.txt",
+				"test",
+			}},
 			expectedError: "When using ADD with more than one source file, the destination must be a directory and end with a /",
 			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
 		},
 		{
-			name:          "Wildcard JSON ADD multiple files to file",
-			dockerfile:    `ADD ["file*.txt", "test"]`,
+			name: "Wildcard ADD multiple files to file",
+			cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"file*.txt",
+				"test",
+			}},
 			expectedError: "When using ADD with more than one source file, the destination must be a directory and end with a /",
 			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
 		},
 		{
-			name:          "COPY multiple files to file",
-			dockerfile:    "COPY file1.txt file2.txt test",
-			expectedError: "When using COPY with more than one source file, the destination must be a directory and end with a /",
-			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
-		},
-		{
-			name:          "JSON COPY multiple files to file",
-			dockerfile:    `COPY ["file1.txt", "file2.txt", "test"]`,
+			name: "COPY multiple files to file",
+			cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"file1.txt",
+				"file2.txt",
+				"test",
+			}},
 			expectedError: "When using COPY with more than one source file, the destination must be a directory and end with a /",
 			files:         map[string]string{"file1.txt": "test1", "file2.txt": "test2"},
 		},
 		{
-			name:          "ADD multiple files to file with whitespace",
-			dockerfile:    `ADD [ "test file1.txt", "test file2.txt", "test" ]`,
+			name: "ADD multiple files to file with whitespace",
+			cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"test file1.txt",
+				"test file2.txt",
+				"test",
+			}},
 			expectedError: "When using ADD with more than one source file, the destination must be a directory and end with a /",
 			files:         map[string]string{"test file1.txt": "test1", "test file2.txt": "test2"},
 		},
 		{
-			name:          "COPY multiple files to file with whitespace",
-			dockerfile:    `COPY [ "test file1.txt", "test file2.txt", "test" ]`,
+			name: "COPY multiple files to file with whitespace",
+			cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"test file1.txt",
+				"test file2.txt",
+				"test",
+			}},
 			expectedError: "When using COPY with more than one source file, the destination must be a directory and end with a /",
 			files:         map[string]string{"test file1.txt": "test1", "test file2.txt": "test2"},
 		},
 		{
-			name:          "COPY wildcard no files",
-			dockerfile:    `COPY file*.txt /tmp/`,
+			name: "COPY wildcard no files",
+			cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"file*.txt",
+				"/tmp/",
+			}},
 			expectedError: "COPY failed: no source files were specified",
 			files:         nil,
 		},
 		{
-			name:          "COPY url",
-			dockerfile:    `COPY https://index.docker.io/robots.txt /`,
+			name: "COPY url",
+			cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
+				"https://index.docker.io/robots.txt",
+				"/",
+			}},
 			expectedError: "source can't be a URL for COPY",
 			files:         nil,
-		},
-		{
-			name:          "Chaining ONBUILD",
-			dockerfile:    `ONBUILD ONBUILD RUN touch foobar`,
-			expectedError: "Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed",
-			files:         nil,
-		},
-		{
-			name:          "Invalid instruction",
-			dockerfile:    `foo bar`,
-			expectedError: "unknown instruction: FOO",
-			files:         nil,
 		}}
 
 	return dispatchTestCases
@@ -171,33 +133,8 @@ func executeTestCase(t *testing.T, testCase dispatchTestCase) {
 		}
 	}()
 
-	r := strings.NewReader(testCase.dockerfile)
-	result, err := parser.Parse(r)
-
-	if err != nil {
-		t.Fatalf("Error when parsing Dockerfile: %s", err)
-	}
-
-	options := &types.ImageBuildOptions{
-		BuildArgs: make(map[string]*string),
-	}
-
-	b := &Builder{
-		options:   options,
-		Stdout:    ioutil.Discard,
-		buildArgs: newBuildArgs(options.BuildArgs),
-	}
-
-	shlex := NewShellLex(parser.DefaultEscapeToken)
-	n := result.AST
-	state := &dispatchState{runConfig: &container.Config{}}
-	opts := dispatchOptions{
-		state:   state,
-		stepMsg: formatStep(0, len(n.Children)),
-		node:    n.Children[0],
-		shlex:   shlex,
-		source:  context,
-	}
-	_, err = b.dispatch(opts)
+	b := newBuilderWithMockBackend()
+	sb := newDispatchRequest(b, '`', context, newBuildArgs(make(map[string]*string)), newStagesBuildResults())
+	err = dispatch(sb, testCase.cmd)
 	testutil.ErrorContains(t, err, testCase.expectedError)
 }

+ 0 - 9
builder/dockerfile/evaluator_unix.go

@@ -1,9 +0,0 @@
-// +build !windows
-
-package dockerfile
-
-// platformSupports is a short-term function to give users a quality error
-// message if a Dockerfile uses a command not supported on the platform.
-func platformSupports(command string) error {
-	return nil
-}

+ 0 - 13
builder/dockerfile/evaluator_windows.go

@@ -1,13 +0,0 @@
-package dockerfile
-
-import "fmt"
-
-// platformSupports is gives users a quality error message if a Dockerfile uses
-// a command not supported on the platform.
-func platformSupports(command string) error {
-	switch command {
-	case "stopsignal":
-		return fmt.Errorf("The daemon on this platform does not support the command '%s'", command)
-	}
-	return nil
-}

+ 0 - 76
builder/dockerfile/imagecontext.go

@@ -1,9 +1,6 @@
 package dockerfile
 
 import (
-	"strconv"
-	"strings"
-
 	"github.com/docker/docker/api/types/backend"
 	"github.com/docker/docker/builder"
 	"github.com/docker/docker/builder/remotecontext"
@@ -13,79 +10,6 @@ import (
 	"golang.org/x/net/context"
 )
 
-type buildStage struct {
-	id string
-}
-
-func newBuildStage(imageID string) *buildStage {
-	return &buildStage{id: imageID}
-}
-
-func (b *buildStage) ImageID() string {
-	return b.id
-}
-
-func (b *buildStage) update(imageID string) {
-	b.id = imageID
-}
-
-// buildStages tracks each stage of a build so they can be retrieved by index
-// or by name.
-type buildStages struct {
-	sequence []*buildStage
-	byName   map[string]*buildStage
-}
-
-func newBuildStages() *buildStages {
-	return &buildStages{byName: make(map[string]*buildStage)}
-}
-
-func (s *buildStages) getByName(name string) (*buildStage, bool) {
-	stage, ok := s.byName[strings.ToLower(name)]
-	return stage, ok
-}
-
-func (s *buildStages) get(indexOrName string) (*buildStage, error) {
-	index, err := strconv.Atoi(indexOrName)
-	if err == nil {
-		if err := s.validateIndex(index); err != nil {
-			return nil, err
-		}
-		return s.sequence[index], nil
-	}
-	if im, ok := s.byName[strings.ToLower(indexOrName)]; ok {
-		return im, nil
-	}
-	return nil, nil
-}
-
-func (s *buildStages) validateIndex(i int) error {
-	if i < 0 || i >= len(s.sequence)-1 {
-		if i == len(s.sequence)-1 {
-			return errors.New("refers to current build stage")
-		}
-		return errors.New("index out of bounds")
-	}
-	return nil
-}
-
-func (s *buildStages) add(name string, image builder.Image) error {
-	stage := newBuildStage(image.ImageID())
-	name = strings.ToLower(name)
-	if len(name) > 0 {
-		if _, ok := s.byName[name]; ok {
-			return errors.Errorf("duplicate name %s", name)
-		}
-		s.byName[name] = stage
-	}
-	s.sequence = append(s.sequence, stage)
-	return nil
-}
-
-func (s *buildStages) update(imageID string) {
-	s.sequence[len(s.sequence)-1].update(imageID)
-}
-
 type getAndMountFunc func(string, bool) (builder.Image, builder.ReleaseableLayer, error)
 
 // imageSources mounts images and provides a cache for mounted images. It tracks

+ 1 - 1
builder/dockerfile/bflag.go → builder/dockerfile/instructions/bflag.go

@@ -1,4 +1,4 @@
-package dockerfile
+package instructions
 
 import (
 	"fmt"

+ 1 - 1
builder/dockerfile/bflag_test.go → builder/dockerfile/instructions/bflag_test.go

@@ -1,4 +1,4 @@
-package dockerfile
+package instructions
 
 import (
 	"testing"

+ 396 - 0
builder/dockerfile/instructions/commands.go

@@ -0,0 +1,396 @@
+package instructions
+
+import (
+	"errors"
+
+	"strings"
+
+	"github.com/docker/docker/api/types/container"
+	"github.com/docker/docker/api/types/strslice"
+)
+
+// KeyValuePair represent an arbitrary named value (usefull in slice insted of map[string] string to preserve ordering)
+type KeyValuePair struct {
+	Key   string
+	Value string
+}
+
+func (kvp *KeyValuePair) String() string {
+	return kvp.Key + "=" + kvp.Value
+}
+
+// Command is implemented by every command present in a dockerfile
+type Command interface {
+	Name() string
+}
+
+// KeyValuePairs is a slice of KeyValuePair
+type KeyValuePairs []KeyValuePair
+
+// withNameAndCode is the base of every command in a Dockerfile (String() returns its source code)
+type withNameAndCode struct {
+	code string
+	name string
+}
+
+func (c *withNameAndCode) String() string {
+	return c.code
+}
+
+// Name of the command
+func (c *withNameAndCode) Name() string {
+	return c.name
+}
+
+func newWithNameAndCode(req parseRequest) withNameAndCode {
+	return withNameAndCode{code: strings.TrimSpace(req.original), name: req.command}
+}
+
+// SingleWordExpander is a provider for variable expansion where 1 word => 1 output
+type SingleWordExpander func(word string) (string, error)
+
+// SupportsSingleWordExpansion interface marks a command as supporting variable expansion
+type SupportsSingleWordExpansion interface {
+	Expand(expander SingleWordExpander) error
+}
+
+// PlatformSpecific adds platform checks to a command
+type PlatformSpecific interface {
+	CheckPlatform(platform string) error
+}
+
+func expandKvp(kvp KeyValuePair, expander SingleWordExpander) (KeyValuePair, error) {
+	key, err := expander(kvp.Key)
+	if err != nil {
+		return KeyValuePair{}, err
+	}
+	value, err := expander(kvp.Value)
+	if err != nil {
+		return KeyValuePair{}, err
+	}
+	return KeyValuePair{Key: key, Value: value}, nil
+}
+func expandKvpsInPlace(kvps KeyValuePairs, expander SingleWordExpander) error {
+	for i, kvp := range kvps {
+		newKvp, err := expandKvp(kvp, expander)
+		if err != nil {
+			return err
+		}
+		kvps[i] = newKvp
+	}
+	return nil
+}
+
+func expandSliceInPlace(values []string, expander SingleWordExpander) error {
+	for i, v := range values {
+		newValue, err := expander(v)
+		if err != nil {
+			return err
+		}
+		values[i] = newValue
+	}
+	return nil
+}
+
+// EnvCommand : ENV key1 value1 [keyN valueN...]
+type EnvCommand struct {
+	withNameAndCode
+	Env KeyValuePairs // kvp slice instead of map to preserve ordering
+}
+
+// Expand variables
+func (c *EnvCommand) Expand(expander SingleWordExpander) error {
+	return expandKvpsInPlace(c.Env, expander)
+}
+
+// MaintainerCommand : MAINTAINER maintainer_name
+type MaintainerCommand struct {
+	withNameAndCode
+	Maintainer string
+}
+
+// LabelCommand : LABEL some json data describing the image
+//
+// Sets the Label variable foo to bar,
+//
+type LabelCommand struct {
+	withNameAndCode
+	Labels KeyValuePairs // kvp slice instead of map to preserve ordering
+}
+
+// Expand variables
+func (c *LabelCommand) Expand(expander SingleWordExpander) error {
+	return expandKvpsInPlace(c.Labels, expander)
+}
+
+// SourcesAndDest represent a list of source files and a destination
+type SourcesAndDest []string
+
+// Sources list the source paths
+func (s SourcesAndDest) Sources() []string {
+	res := make([]string, len(s)-1)
+	copy(res, s[:len(s)-1])
+	return res
+}
+
+// Dest path of the operation
+func (s SourcesAndDest) Dest() string {
+	return s[len(s)-1]
+}
+
+// AddCommand : ADD foo /path
+//
+// 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.
+//
+type AddCommand struct {
+	withNameAndCode
+	SourcesAndDest
+	Chown string
+}
+
+// Expand variables
+func (c *AddCommand) Expand(expander SingleWordExpander) error {
+	return expandSliceInPlace(c.SourcesAndDest, expander)
+}
+
+// CopyCommand : COPY foo /path
+//
+// Same as 'ADD' but without the tar and remote url handling.
+//
+type CopyCommand struct {
+	withNameAndCode
+	SourcesAndDest
+	From  string
+	Chown string
+}
+
+// Expand variables
+func (c *CopyCommand) Expand(expander SingleWordExpander) error {
+	return expandSliceInPlace(c.SourcesAndDest, expander)
+}
+
+// OnbuildCommand : ONBUILD <some other command>
+type OnbuildCommand struct {
+	withNameAndCode
+	Expression string
+}
+
+// WorkdirCommand : WORKDIR /tmp
+//
+// Set the working directory for future RUN/CMD/etc statements.
+//
+type WorkdirCommand struct {
+	withNameAndCode
+	Path string
+}
+
+// Expand variables
+func (c *WorkdirCommand) Expand(expander SingleWordExpander) error {
+	p, err := expander(c.Path)
+	if err != nil {
+		return err
+	}
+	c.Path = p
+	return nil
+}
+
+// ShellDependantCmdLine represents a cmdline optionaly prepended with the shell
+type ShellDependantCmdLine struct {
+	CmdLine      strslice.StrSlice
+	PrependShell bool
+}
+
+// RunCommand : RUN some command yo
+//
+// run a command and commit the image. Args are automatically prepended with
+// the current SHELL which defaults to 'sh -c' under linux or 'cmd /S /C' under
+// Windows, in the event there is only one argument The difference in processing:
+//
+// RUN echo hi          # sh -c echo hi       (Linux)
+// RUN echo hi          # cmd /S /C echo hi   (Windows)
+// RUN [ "echo", "hi" ] # echo hi
+//
+type RunCommand struct {
+	withNameAndCode
+	ShellDependantCmdLine
+}
+
+// CmdCommand : CMD foo
+//
+// Set the default command to run in the container (which may be empty).
+// Argument handling is the same as RUN.
+//
+type CmdCommand struct {
+	withNameAndCode
+	ShellDependantCmdLine
+}
+
+// HealthCheckCommand : HEALTHCHECK foo
+//
+// Set the default healthcheck command to run in the container (which may be empty).
+// Argument handling is the same as RUN.
+//
+type HealthCheckCommand struct {
+	withNameAndCode
+	Health *container.HealthConfig
+}
+
+// EntrypointCommand : ENTRYPOINT /usr/sbin/nginx
+//
+// Set the entrypoint to /usr/sbin/nginx. Will accept the CMD as the arguments
+// to /usr/sbin/nginx. Uses the default shell if not in JSON format.
+//
+// Handles command processing similar to CMD and RUN, only req.runConfig.Entrypoint
+// is initialized at newBuilder time instead of through argument parsing.
+//
+type EntrypointCommand struct {
+	withNameAndCode
+	ShellDependantCmdLine
+}
+
+// ExposeCommand : EXPOSE 6667/tcp 7000/tcp
+//
+// Expose ports for links and port mappings. This all ends up in
+// req.runConfig.ExposedPorts for runconfig.
+//
+type ExposeCommand struct {
+	withNameAndCode
+	Ports []string
+}
+
+// UserCommand : USER foo
+//
+// Set the user to 'foo' for future commands and when running the
+// ENTRYPOINT/CMD at container run time.
+//
+type UserCommand struct {
+	withNameAndCode
+	User string
+}
+
+// Expand variables
+func (c *UserCommand) Expand(expander SingleWordExpander) error {
+	p, err := expander(c.User)
+	if err != nil {
+		return err
+	}
+	c.User = p
+	return nil
+}
+
+// VolumeCommand : VOLUME /foo
+//
+// Expose the volume /foo for use. Will also accept the JSON array form.
+//
+type VolumeCommand struct {
+	withNameAndCode
+	Volumes []string
+}
+
+// Expand variables
+func (c *VolumeCommand) Expand(expander SingleWordExpander) error {
+	return expandSliceInPlace(c.Volumes, expander)
+}
+
+// StopSignalCommand : STOPSIGNAL signal
+//
+// Set the signal that will be used to kill the container.
+type StopSignalCommand struct {
+	withNameAndCode
+	Signal string
+}
+
+// Expand variables
+func (c *StopSignalCommand) Expand(expander SingleWordExpander) error {
+	p, err := expander(c.Signal)
+	if err != nil {
+		return err
+	}
+	c.Signal = p
+	return nil
+}
+
+// CheckPlatform checks that the command is supported in the target platform
+func (c *StopSignalCommand) CheckPlatform(platform string) error {
+	if platform == "windows" {
+		return errors.New("The daemon on this platform does not support the command stopsignal")
+	}
+	return nil
+}
+
+// ArgCommand : ARG name[=value]
+//
+// Adds the variable foo to the trusted list of variables that can be passed
+// to builder using the --build-arg flag for expansion/substitution or passing to 'run'.
+// Dockerfile author may optionally set a default value of this variable.
+type ArgCommand struct {
+	withNameAndCode
+	Key   string
+	Value *string
+}
+
+// Expand variables
+func (c *ArgCommand) Expand(expander SingleWordExpander) error {
+	p, err := expander(c.Key)
+	if err != nil {
+		return err
+	}
+	c.Key = p
+	if c.Value != nil {
+		p, err = expander(*c.Value)
+		if err != nil {
+			return err
+		}
+		c.Value = &p
+	}
+	return nil
+}
+
+// ShellCommand : SHELL powershell -command
+//
+// Set the non-default shell to use.
+type ShellCommand struct {
+	withNameAndCode
+	Shell strslice.StrSlice
+}
+
+// Stage represents a single stage in a multi-stage build
+type Stage struct {
+	Name       string
+	Commands   []Command
+	BaseName   string
+	SourceCode string
+}
+
+// AddCommand to the stage
+func (s *Stage) AddCommand(cmd Command) {
+	// todo: validate cmd type
+	s.Commands = append(s.Commands, cmd)
+}
+
+// IsCurrentStage check if the stage name is the current stage
+func IsCurrentStage(s []Stage, name string) bool {
+	if len(s) == 0 {
+		return false
+	}
+	return s[len(s)-1].Name == name
+}
+
+// CurrentStage return the last stage in a slice
+func CurrentStage(s []Stage) (*Stage, error) {
+	if len(s) == 0 {
+		return nil, errors.New("No build stage in current context")
+	}
+	return &s[len(s)-1], nil
+}
+
+// HasStage looks for the presence of a given stage name
+func HasStage(s []Stage, name string) (int, bool) {
+	for i, stage := range s {
+		if stage.Name == name {
+			return i, true
+		}
+	}
+	return -1, false
+}

+ 9 - 0
builder/dockerfile/instructions/errors_unix.go

@@ -0,0 +1,9 @@
+// +build !windows
+
+package instructions
+
+import "fmt"
+
+func errNotJSON(command, _ string) error {
+	return fmt.Errorf("%s requires the arguments to be in JSON form", command)
+}

+ 27 - 0
builder/dockerfile/instructions/errors_windows.go

@@ -0,0 +1,27 @@
+package instructions
+
+import (
+	"fmt"
+	"path/filepath"
+	"regexp"
+	"strings"
+)
+
+func errNotJSON(command, original string) error {
+	// For Windows users, give a hint if it looks like it might contain
+	// a path which hasn't been escaped such as ["c:\windows\system32\prog.exe", "-param"],
+	// as JSON must be escaped. Unfortunate...
+	//
+	// Specifically looking for quote-driveletter-colon-backslash, there's no
+	// double backslash and a [] pair. No, this is not perfect, but it doesn't
+	// have to be. It's simply a hint to make life a little easier.
+	extra := ""
+	original = filepath.FromSlash(strings.ToLower(strings.Replace(strings.ToLower(original), strings.ToLower(command)+" ", "", -1)))
+	if len(regexp.MustCompile(`"[a-z]:\\.*`).FindStringSubmatch(original)) > 0 &&
+		!strings.Contains(original, `\\`) &&
+		strings.Contains(original, "[") &&
+		strings.Contains(original, "]") {
+		extra = fmt.Sprintf(`. It looks like '%s' includes a file path without an escaped back-slash. JSON requires back-slashes to be escaped such as ["c:\\path\\to\\file.exe", "/parameter"]`, original)
+	}
+	return fmt.Errorf("%s requires the arguments to be in JSON form%s", command, extra)
+}

+ 635 - 0
builder/dockerfile/instructions/parse.go

@@ -0,0 +1,635 @@
+package instructions
+
+import (
+	"fmt"
+	"regexp"
+	"sort"
+	"strconv"
+	"strings"
+	"time"
+
+	"github.com/docker/docker/api/types/container"
+	"github.com/docker/docker/api/types/strslice"
+	"github.com/docker/docker/builder/dockerfile/command"
+	"github.com/docker/docker/builder/dockerfile/parser"
+	"github.com/pkg/errors"
+)
+
+type parseRequest struct {
+	command    string
+	args       []string
+	attributes map[string]bool
+	flags      *BFlags
+	original   string
+}
+
+func nodeArgs(node *parser.Node) []string {
+	result := []string{}
+	for ; node.Next != nil; node = node.Next {
+		arg := node.Next
+		if len(arg.Children) == 0 {
+			result = append(result, arg.Value)
+		} else if len(arg.Children) == 1 {
+			//sub command
+			result = append(result, arg.Children[0].Value)
+			result = append(result, nodeArgs(arg.Children[0])...)
+		}
+	}
+	return result
+}
+
+func newParseRequestFromNode(node *parser.Node) parseRequest {
+	return parseRequest{
+		command:    node.Value,
+		args:       nodeArgs(node),
+		attributes: node.Attributes,
+		original:   node.Original,
+		flags:      NewBFlagsWithArgs(node.Flags),
+	}
+}
+
+// ParseInstruction converts an AST to a typed instruction (either a command or a build stage beginning when encountering a `FROM` statement)
+func ParseInstruction(node *parser.Node) (interface{}, error) {
+	req := newParseRequestFromNode(node)
+	switch node.Value {
+	case command.Env:
+		return parseEnv(req)
+	case command.Maintainer:
+		return parseMaintainer(req)
+	case command.Label:
+		return parseLabel(req)
+	case command.Add:
+		return parseAdd(req)
+	case command.Copy:
+		return parseCopy(req)
+	case command.From:
+		return parseFrom(req)
+	case command.Onbuild:
+		return parseOnBuild(req)
+	case command.Workdir:
+		return parseWorkdir(req)
+	case command.Run:
+		return parseRun(req)
+	case command.Cmd:
+		return parseCmd(req)
+	case command.Healthcheck:
+		return parseHealthcheck(req)
+	case command.Entrypoint:
+		return parseEntrypoint(req)
+	case command.Expose:
+		return parseExpose(req)
+	case command.User:
+		return parseUser(req)
+	case command.Volume:
+		return parseVolume(req)
+	case command.StopSignal:
+		return parseStopSignal(req)
+	case command.Arg:
+		return parseArg(req)
+	case command.Shell:
+		return parseShell(req)
+	}
+
+	return nil, &UnknownInstruction{Instruction: node.Value, Line: node.StartLine}
+}
+
+// ParseCommand converts an AST to a typed Command
+func ParseCommand(node *parser.Node) (Command, error) {
+	s, err := ParseInstruction(node)
+	if err != nil {
+		return nil, err
+	}
+	if c, ok := s.(Command); ok {
+		return c, nil
+	}
+	return nil, errors.Errorf("%T is not a command type", s)
+}
+
+// UnknownInstruction represents an error occuring when a command is unresolvable
+type UnknownInstruction struct {
+	Line        int
+	Instruction string
+}
+
+func (e *UnknownInstruction) Error() string {
+	return fmt.Sprintf("unknown instruction: %s", strings.ToUpper(e.Instruction))
+}
+
+// IsUnknownInstruction checks if the error is an UnknownInstruction or a parseError containing an UnknownInstruction
+func IsUnknownInstruction(err error) bool {
+	_, ok := err.(*UnknownInstruction)
+	if !ok {
+		var pe *parseError
+		if pe, ok = err.(*parseError); ok {
+			_, ok = pe.inner.(*UnknownInstruction)
+		}
+	}
+	return ok
+}
+
+type parseError struct {
+	inner error
+	node  *parser.Node
+}
+
+func (e *parseError) Error() string {
+	return fmt.Sprintf("Dockerfile parse error line %d: %v", e.node.StartLine, e.inner.Error())
+}
+
+// Parse a docker file into a collection of buildable stages
+func Parse(ast *parser.Node) (stages []Stage, metaArgs []ArgCommand, err error) {
+	for _, n := range ast.Children {
+		cmd, err := ParseInstruction(n)
+		if err != nil {
+			return nil, nil, &parseError{inner: err, node: n}
+		}
+		if len(stages) == 0 {
+			// meta arg case
+			if a, isArg := cmd.(*ArgCommand); isArg {
+				metaArgs = append(metaArgs, *a)
+				continue
+			}
+		}
+		switch c := cmd.(type) {
+		case *Stage:
+			stages = append(stages, *c)
+		case Command:
+			stage, err := CurrentStage(stages)
+			if err != nil {
+				return nil, nil, err
+			}
+			stage.AddCommand(c)
+		default:
+			return nil, nil, errors.Errorf("%T is not a command type", cmd)
+		}
+
+	}
+	return stages, metaArgs, nil
+}
+
+func parseKvps(args []string, cmdName string) (KeyValuePairs, error) {
+	if len(args) == 0 {
+		return nil, errAtLeastOneArgument(cmdName)
+	}
+	if len(args)%2 != 0 {
+		// should never get here, but just in case
+		return nil, errTooManyArguments(cmdName)
+	}
+	var res KeyValuePairs
+	for j := 0; j < len(args); j += 2 {
+		if len(args[j]) == 0 {
+			return nil, errBlankCommandNames(cmdName)
+		}
+		name := args[j]
+		value := args[j+1]
+		res = append(res, KeyValuePair{Key: name, Value: value})
+	}
+	return res, nil
+}
+
+func parseEnv(req parseRequest) (*EnvCommand, error) {
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	envs, err := parseKvps(req.args, "ENV")
+	if err != nil {
+		return nil, err
+	}
+	return &EnvCommand{
+		Env:             envs,
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseMaintainer(req parseRequest) (*MaintainerCommand, error) {
+	if len(req.args) != 1 {
+		return nil, errExactlyOneArgument("MAINTAINER")
+	}
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &MaintainerCommand{
+		Maintainer:      req.args[0],
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseLabel(req parseRequest) (*LabelCommand, error) {
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+
+	labels, err := parseKvps(req.args, "LABEL")
+	if err != nil {
+		return nil, err
+	}
+
+	return &LabelCommand{
+		Labels:          labels,
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseAdd(req parseRequest) (*AddCommand, error) {
+	if len(req.args) < 2 {
+		return nil, errAtLeastTwoArguments("ADD")
+	}
+	flChown := req.flags.AddString("chown", "")
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &AddCommand{
+		SourcesAndDest:  SourcesAndDest(req.args),
+		withNameAndCode: newWithNameAndCode(req),
+		Chown:           flChown.Value,
+	}, nil
+}
+
+func parseCopy(req parseRequest) (*CopyCommand, error) {
+	if len(req.args) < 2 {
+		return nil, errAtLeastTwoArguments("COPY")
+	}
+	flChown := req.flags.AddString("chown", "")
+	flFrom := req.flags.AddString("from", "")
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &CopyCommand{
+		SourcesAndDest:  SourcesAndDest(req.args),
+		From:            flFrom.Value,
+		withNameAndCode: newWithNameAndCode(req),
+		Chown:           flChown.Value,
+	}, nil
+}
+
+func parseFrom(req parseRequest) (*Stage, error) {
+	stageName, err := parseBuildStageName(req.args)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	code := strings.TrimSpace(req.original)
+
+	return &Stage{
+		BaseName:   req.args[0],
+		Name:       stageName,
+		SourceCode: code,
+		Commands:   []Command{},
+	}, nil
+
+}
+
+func parseBuildStageName(args []string) (string, error) {
+	stageName := ""
+	switch {
+	case len(args) == 3 && strings.EqualFold(args[1], "as"):
+		stageName = strings.ToLower(args[2])
+		if ok, _ := regexp.MatchString("^[a-z][a-z0-9-_\\.]*$", stageName); !ok {
+			return "", errors.Errorf("invalid name for build stage: %q, name can't start with a number or contain symbols", stageName)
+		}
+	case len(args) != 1:
+		return "", errors.New("FROM requires either one or three arguments")
+	}
+
+	return stageName, nil
+}
+
+func parseOnBuild(req parseRequest) (*OnbuildCommand, error) {
+	if len(req.args) == 0 {
+		return nil, errAtLeastOneArgument("ONBUILD")
+	}
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+
+	triggerInstruction := strings.ToUpper(strings.TrimSpace(req.args[0]))
+	switch strings.ToUpper(triggerInstruction) {
+	case "ONBUILD":
+		return nil, errors.New("Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed")
+	case "MAINTAINER", "FROM":
+		return nil, fmt.Errorf("%s isn't allowed as an ONBUILD trigger", triggerInstruction)
+	}
+
+	original := regexp.MustCompile(`(?i)^\s*ONBUILD\s*`).ReplaceAllString(req.original, "")
+	return &OnbuildCommand{
+		Expression:      original,
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+
+}
+
+func parseWorkdir(req parseRequest) (*WorkdirCommand, error) {
+	if len(req.args) != 1 {
+		return nil, errExactlyOneArgument("WORKDIR")
+	}
+
+	err := req.flags.Parse()
+	if err != nil {
+		return nil, err
+	}
+	return &WorkdirCommand{
+		Path:            req.args[0],
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+
+}
+
+func parseShellDependentCommand(req parseRequest, emptyAsNil bool) ShellDependantCmdLine {
+	args := handleJSONArgs(req.args, req.attributes)
+	cmd := strslice.StrSlice(args)
+	if emptyAsNil && len(cmd) == 0 {
+		cmd = nil
+	}
+	return ShellDependantCmdLine{
+		CmdLine:      cmd,
+		PrependShell: !req.attributes["json"],
+	}
+}
+
+func parseRun(req parseRequest) (*RunCommand, error) {
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &RunCommand{
+		ShellDependantCmdLine: parseShellDependentCommand(req, false),
+		withNameAndCode:       newWithNameAndCode(req),
+	}, nil
+
+}
+
+func parseCmd(req parseRequest) (*CmdCommand, error) {
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &CmdCommand{
+		ShellDependantCmdLine: parseShellDependentCommand(req, false),
+		withNameAndCode:       newWithNameAndCode(req),
+	}, nil
+
+}
+
+func parseEntrypoint(req parseRequest) (*EntrypointCommand, error) {
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+
+	cmd := &EntrypointCommand{
+		ShellDependantCmdLine: parseShellDependentCommand(req, true),
+		withNameAndCode:       newWithNameAndCode(req),
+	}
+
+	return cmd, nil
+}
+
+// parseOptInterval(flag) is the duration of flag.Value, or 0 if
+// empty. An error is reported if the value is given and less than minimum duration.
+func parseOptInterval(f *Flag) (time.Duration, error) {
+	s := f.Value
+	if s == "" {
+		return 0, nil
+	}
+	d, err := time.ParseDuration(s)
+	if err != nil {
+		return 0, err
+	}
+	if d < container.MinimumDuration {
+		return 0, fmt.Errorf("Interval %#v cannot be less than %s", f.name, container.MinimumDuration)
+	}
+	return d, nil
+}
+func parseHealthcheck(req parseRequest) (*HealthCheckCommand, error) {
+	if len(req.args) == 0 {
+		return nil, errAtLeastOneArgument("HEALTHCHECK")
+	}
+	cmd := &HealthCheckCommand{
+		withNameAndCode: newWithNameAndCode(req),
+	}
+
+	typ := strings.ToUpper(req.args[0])
+	args := req.args[1:]
+	if typ == "NONE" {
+		if len(args) != 0 {
+			return nil, errors.New("HEALTHCHECK NONE takes no arguments")
+		}
+		test := strslice.StrSlice{typ}
+		cmd.Health = &container.HealthConfig{
+			Test: test,
+		}
+	} else {
+
+		healthcheck := container.HealthConfig{}
+
+		flInterval := req.flags.AddString("interval", "")
+		flTimeout := req.flags.AddString("timeout", "")
+		flStartPeriod := req.flags.AddString("start-period", "")
+		flRetries := req.flags.AddString("retries", "")
+
+		if err := req.flags.Parse(); err != nil {
+			return nil, err
+		}
+
+		switch typ {
+		case "CMD":
+			cmdSlice := handleJSONArgs(args, req.attributes)
+			if len(cmdSlice) == 0 {
+				return nil, errors.New("Missing command after HEALTHCHECK CMD")
+			}
+
+			if !req.attributes["json"] {
+				typ = "CMD-SHELL"
+			}
+
+			healthcheck.Test = strslice.StrSlice(append([]string{typ}, cmdSlice...))
+		default:
+			return nil, fmt.Errorf("Unknown type %#v in HEALTHCHECK (try CMD)", typ)
+		}
+
+		interval, err := parseOptInterval(flInterval)
+		if err != nil {
+			return nil, err
+		}
+		healthcheck.Interval = interval
+
+		timeout, err := parseOptInterval(flTimeout)
+		if err != nil {
+			return nil, err
+		}
+		healthcheck.Timeout = timeout
+
+		startPeriod, err := parseOptInterval(flStartPeriod)
+		if err != nil {
+			return nil, err
+		}
+		healthcheck.StartPeriod = startPeriod
+
+		if flRetries.Value != "" {
+			retries, err := strconv.ParseInt(flRetries.Value, 10, 32)
+			if err != nil {
+				return nil, err
+			}
+			if retries < 1 {
+				return nil, fmt.Errorf("--retries must be at least 1 (not %d)", retries)
+			}
+			healthcheck.Retries = int(retries)
+		} else {
+			healthcheck.Retries = 0
+		}
+
+		cmd.Health = &healthcheck
+	}
+	return cmd, nil
+}
+
+func parseExpose(req parseRequest) (*ExposeCommand, error) {
+	portsTab := req.args
+
+	if len(req.args) == 0 {
+		return nil, errAtLeastOneArgument("EXPOSE")
+	}
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+
+	sort.Strings(portsTab)
+	return &ExposeCommand{
+		Ports:           portsTab,
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseUser(req parseRequest) (*UserCommand, error) {
+	if len(req.args) != 1 {
+		return nil, errExactlyOneArgument("USER")
+	}
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	return &UserCommand{
+		User:            req.args[0],
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseVolume(req parseRequest) (*VolumeCommand, error) {
+	if len(req.args) == 0 {
+		return nil, errAtLeastOneArgument("VOLUME")
+	}
+
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+
+	cmd := &VolumeCommand{
+		withNameAndCode: newWithNameAndCode(req),
+	}
+
+	for _, v := range req.args {
+		v = strings.TrimSpace(v)
+		if v == "" {
+			return nil, errors.New("VOLUME specified can not be an empty string")
+		}
+		cmd.Volumes = append(cmd.Volumes, v)
+	}
+	return cmd, nil
+
+}
+
+func parseStopSignal(req parseRequest) (*StopSignalCommand, error) {
+	if len(req.args) != 1 {
+		return nil, errExactlyOneArgument("STOPSIGNAL")
+	}
+	sig := req.args[0]
+
+	cmd := &StopSignalCommand{
+		Signal:          sig,
+		withNameAndCode: newWithNameAndCode(req),
+	}
+	return cmd, nil
+
+}
+
+func parseArg(req parseRequest) (*ArgCommand, error) {
+	if len(req.args) != 1 {
+		return nil, errExactlyOneArgument("ARG")
+	}
+
+	var (
+		name     string
+		newValue *string
+	)
+
+	arg := req.args[0]
+	// 'arg' can just be a name or name-value pair. Note that this is different
+	// from 'env' that handles the split of name and value at the parser level.
+	// The reason for doing it differently for 'arg' is that we support just
+	// defining an arg and not assign it a value (while 'env' always expects a
+	// name-value pair). If possible, it will be good to harmonize the two.
+	if strings.Contains(arg, "=") {
+		parts := strings.SplitN(arg, "=", 2)
+		if len(parts[0]) == 0 {
+			return nil, errBlankCommandNames("ARG")
+		}
+
+		name = parts[0]
+		newValue = &parts[1]
+	} else {
+		name = arg
+	}
+
+	return &ArgCommand{
+		Key:             name,
+		Value:           newValue,
+		withNameAndCode: newWithNameAndCode(req),
+	}, nil
+}
+
+func parseShell(req parseRequest) (*ShellCommand, error) {
+	if err := req.flags.Parse(); err != nil {
+		return nil, err
+	}
+	shellSlice := handleJSONArgs(req.args, req.attributes)
+	switch {
+	case len(shellSlice) == 0:
+		// SHELL []
+		return nil, errAtLeastOneArgument("SHELL")
+	case req.attributes["json"]:
+		// SHELL ["powershell", "-command"]
+
+		return &ShellCommand{
+			Shell:           strslice.StrSlice(shellSlice),
+			withNameAndCode: newWithNameAndCode(req),
+		}, nil
+	default:
+		// SHELL powershell -command - not JSON
+		return nil, errNotJSON("SHELL", req.original)
+	}
+}
+
+func errAtLeastOneArgument(command string) error {
+	return errors.Errorf("%s requires at least one argument", command)
+}
+
+func errExactlyOneArgument(command string) error {
+	return errors.Errorf("%s requires exactly one argument", command)
+}
+
+func errAtLeastTwoArguments(command string) error {
+	return errors.Errorf("%s requires at least two arguments", command)
+}
+
+func errBlankCommandNames(command string) error {
+	return errors.Errorf("%s names can not be blank", command)
+}
+
+func errTooManyArguments(command string) error {
+	return errors.Errorf("Bad input to %s, too many arguments", command)
+}

+ 204 - 0
builder/dockerfile/instructions/parse_test.go

@@ -0,0 +1,204 @@
+package instructions
+
+import (
+	"strings"
+	"testing"
+
+	"github.com/docker/docker/builder/dockerfile/command"
+	"github.com/docker/docker/builder/dockerfile/parser"
+	"github.com/docker/docker/internal/testutil"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/require"
+)
+
+func TestCommandsExactlyOneArgument(t *testing.T) {
+	commands := []string{
+		"MAINTAINER",
+		"WORKDIR",
+		"USER",
+		"STOPSIGNAL",
+	}
+
+	for _, command := range commands {
+		ast, err := parser.Parse(strings.NewReader(command))
+		require.NoError(t, err)
+		_, err = ParseInstruction(ast.AST.Children[0])
+		assert.EqualError(t, err, errExactlyOneArgument(command).Error())
+	}
+}
+
+func TestCommandsAtLeastOneArgument(t *testing.T) {
+	commands := []string{
+		"ENV",
+		"LABEL",
+		"ONBUILD",
+		"HEALTHCHECK",
+		"EXPOSE",
+		"VOLUME",
+	}
+
+	for _, command := range commands {
+		ast, err := parser.Parse(strings.NewReader(command))
+		require.NoError(t, err)
+		_, err = ParseInstruction(ast.AST.Children[0])
+		assert.EqualError(t, err, errAtLeastOneArgument(command).Error())
+	}
+}
+
+func TestCommandsAtLeastTwoArgument(t *testing.T) {
+	commands := []string{
+		"ADD",
+		"COPY",
+	}
+
+	for _, command := range commands {
+		ast, err := parser.Parse(strings.NewReader(command + " arg1"))
+		require.NoError(t, err)
+		_, err = ParseInstruction(ast.AST.Children[0])
+		assert.EqualError(t, err, errAtLeastTwoArguments(command).Error())
+	}
+}
+
+func TestCommandsTooManyArguments(t *testing.T) {
+	commands := []string{
+		"ENV",
+		"LABEL",
+	}
+
+	for _, command := range commands {
+		node := &parser.Node{
+			Original: command + "arg1 arg2 arg3",
+			Value:    strings.ToLower(command),
+			Next: &parser.Node{
+				Value: "arg1",
+				Next: &parser.Node{
+					Value: "arg2",
+					Next: &parser.Node{
+						Value: "arg3",
+					},
+				},
+			},
+		}
+		_, err := ParseInstruction(node)
+		assert.EqualError(t, err, errTooManyArguments(command).Error())
+	}
+}
+
+func TestCommandsBlankNames(t *testing.T) {
+	commands := []string{
+		"ENV",
+		"LABEL",
+	}
+
+	for _, command := range commands {
+		node := &parser.Node{
+			Original: command + " =arg2",
+			Value:    strings.ToLower(command),
+			Next: &parser.Node{
+				Value: "",
+				Next: &parser.Node{
+					Value: "arg2",
+				},
+			},
+		}
+		_, err := ParseInstruction(node)
+		assert.EqualError(t, err, errBlankCommandNames(command).Error())
+	}
+}
+
+func TestHealthCheckCmd(t *testing.T) {
+	node := &parser.Node{
+		Value: command.Healthcheck,
+		Next: &parser.Node{
+			Value: "CMD",
+			Next: &parser.Node{
+				Value: "hello",
+				Next: &parser.Node{
+					Value: "world",
+				},
+			},
+		},
+	}
+	cmd, err := ParseInstruction(node)
+	assert.NoError(t, err)
+	hc, ok := cmd.(*HealthCheckCommand)
+	assert.True(t, ok)
+	expected := []string{"CMD-SHELL", "hello world"}
+	assert.Equal(t, expected, hc.Health.Test)
+}
+
+func TestParseOptInterval(t *testing.T) {
+	flInterval := &Flag{
+		name:     "interval",
+		flagType: stringType,
+		Value:    "50ns",
+	}
+	_, err := parseOptInterval(flInterval)
+	testutil.ErrorContains(t, err, "cannot be less than 1ms")
+
+	flInterval.Value = "1ms"
+	_, err = parseOptInterval(flInterval)
+	require.NoError(t, err)
+}
+
+func TestErrorCases(t *testing.T) {
+	cases := []struct {
+		name          string
+		dockerfile    string
+		expectedError string
+	}{
+		{
+			name: "copyEmptyWhitespace",
+			dockerfile: `COPY	
+		quux \
+      bar`,
+			expectedError: "COPY requires at least two arguments",
+		},
+		{
+			name:          "ONBUILD forbidden FROM",
+			dockerfile:    "ONBUILD FROM scratch",
+			expectedError: "FROM isn't allowed as an ONBUILD trigger",
+		},
+		{
+			name:          "ONBUILD forbidden MAINTAINER",
+			dockerfile:    "ONBUILD MAINTAINER docker.io",
+			expectedError: "MAINTAINER isn't allowed as an ONBUILD trigger",
+		},
+		{
+			name:          "ARG two arguments",
+			dockerfile:    "ARG foo bar",
+			expectedError: "ARG requires exactly one argument",
+		},
+		{
+			name:          "MAINTAINER unknown flag",
+			dockerfile:    "MAINTAINER --boo joe@example.com",
+			expectedError: "Unknown flag: boo",
+		},
+		{
+			name:          "Chaining ONBUILD",
+			dockerfile:    `ONBUILD ONBUILD RUN touch foobar`,
+			expectedError: "Chaining ONBUILD via `ONBUILD ONBUILD` isn't allowed",
+		},
+		{
+			name:          "Invalid instruction",
+			dockerfile:    `foo bar`,
+			expectedError: "unknown instruction: FOO",
+		},
+	}
+	for _, c := range cases {
+		r := strings.NewReader(c.dockerfile)
+		ast, err := parser.Parse(r)
+
+		if err != nil {
+			t.Fatalf("Error when parsing Dockerfile: %s", err)
+		}
+		n := ast.AST.Children[0]
+		_, err = ParseInstruction(n)
+		if err != nil {
+			testutil.ErrorContains(t, err, c.expectedError)
+			return
+		}
+		t.Fatalf("No error when executing test %s", c.name)
+	}
+
+}

+ 1 - 1
builder/dockerfile/support.go → builder/dockerfile/instructions/support.go

@@ -1,4 +1,4 @@
-package dockerfile
+package instructions
 
 import "strings"
 

+ 1 - 1
builder/dockerfile/support_test.go → builder/dockerfile/instructions/support_test.go

@@ -1,4 +1,4 @@
-package dockerfile
+package instructions
 
 import "testing"
 

+ 0 - 3
builder/dockerfile/internals.go

@@ -124,7 +124,6 @@ func (b *Builder) commitContainer(dispatchState *dispatchState, id string, conta
 	}
 
 	dispatchState.imageID = imageID
-	b.buildStages.update(imageID)
 	return nil
 }
 
@@ -164,7 +163,6 @@ func (b *Builder) exportImage(state *dispatchState, imageMount *imageMount, runC
 
 	state.imageID = exportedImage.ImageID()
 	b.imageSources.Add(newImageMount(exportedImage, newLayer))
-	b.buildStages.update(state.imageID)
 	return nil
 }
 
@@ -460,7 +458,6 @@ func (b *Builder) probeCache(dispatchState *dispatchState, runConfig *container.
 	fmt.Fprint(b.Stdout, " ---> Using cache\n")
 
 	dispatchState.imageID = cachedID
-	b.buildStages.update(dispatchState.imageID)
 	return true, nil
 }
 

+ 76 - 0
integration-cli/docker_api_build_test.go

@@ -439,6 +439,82 @@ func (s *DockerSuite) TestBuildChownOnCopy(c *check.C) {
 	assert.Contains(c, string(out), "Successfully built")
 }
 
+func (s *DockerSuite) TestBuildCopyCacheOnFileChange(c *check.C) {
+
+	dockerfile := `FROM busybox
+COPY file /file`
+
+	ctx1 := fakecontext.New(c, "",
+		fakecontext.WithDockerfile(dockerfile),
+		fakecontext.WithFile("file", "foo"))
+	ctx2 := fakecontext.New(c, "",
+		fakecontext.WithDockerfile(dockerfile),
+		fakecontext.WithFile("file", "bar"))
+
+	var build = func(ctx *fakecontext.Fake) string {
+		res, body, err := request.Post("/build",
+			request.RawContent(ctx.AsTarReader(c)),
+			request.ContentType("application/x-tar"))
+
+		require.NoError(c, err)
+		assert.Equal(c, http.StatusOK, res.StatusCode)
+
+		out, err := request.ReadBody(body)
+
+		ids := getImageIDsFromBuild(c, out)
+		return ids[len(ids)-1]
+	}
+
+	id1 := build(ctx1)
+	id2 := build(ctx1)
+	id3 := build(ctx2)
+
+	if id1 != id2 {
+		c.Fatal("didn't use the cache")
+	}
+	if id1 == id3 {
+		c.Fatal("COPY With different source file should not share same cache")
+	}
+}
+
+func (s *DockerSuite) TestBuildAddCacheOnFileChange(c *check.C) {
+
+	dockerfile := `FROM busybox
+ADD file /file`
+
+	ctx1 := fakecontext.New(c, "",
+		fakecontext.WithDockerfile(dockerfile),
+		fakecontext.WithFile("file", "foo"))
+	ctx2 := fakecontext.New(c, "",
+		fakecontext.WithDockerfile(dockerfile),
+		fakecontext.WithFile("file", "bar"))
+
+	var build = func(ctx *fakecontext.Fake) string {
+		res, body, err := request.Post("/build",
+			request.RawContent(ctx.AsTarReader(c)),
+			request.ContentType("application/x-tar"))
+
+		require.NoError(c, err)
+		assert.Equal(c, http.StatusOK, res.StatusCode)
+
+		out, err := request.ReadBody(body)
+
+		ids := getImageIDsFromBuild(c, out)
+		return ids[len(ids)-1]
+	}
+
+	id1 := build(ctx1)
+	id2 := build(ctx1)
+	id3 := build(ctx2)
+
+	if id1 != id2 {
+		c.Fatal("didn't use the cache")
+	}
+	if id1 == id3 {
+		c.Fatal("COPY With different source file should not share same cache")
+	}
+}
+
 func (s *DockerSuite) TestBuildWithSession(c *check.C) {
 	testRequires(c, ExperimentalDaemon)
 

+ 11 - 7
integration-cli/docker_cli_build_test.go

@@ -1173,12 +1173,13 @@ func (s *DockerSuite) TestBuildForceRm(c *check.C) {
 	containerCountBefore := getContainerCount(c)
 	name := "testbuildforcerm"
 
-	buildImage(name, cli.WithFlags("--force-rm"), build.WithBuildContext(c,
-		build.WithFile("Dockerfile", `FROM `+minimalBaseImage()+`
+	r := buildImage(name, cli.WithFlags("--force-rm"), build.WithBuildContext(c,
+		build.WithFile("Dockerfile", `FROM busybox
 	RUN true
-	RUN thiswillfail`))).Assert(c, icmd.Expected{
-		ExitCode: 1,
-	})
+	RUN thiswillfail`)))
+	if r.ExitCode != 1 && r.ExitCode != 127 { // different on Linux / Windows
+		c.Fatalf("Wrong exit code")
+	}
 
 	containerCountAfter := getContainerCount(c)
 	if containerCountBefore != containerCountAfter {
@@ -4542,7 +4543,6 @@ func (s *DockerSuite) TestBuildBuildTimeArgOverrideEnvDefinedBeforeArg(c *check.
 }
 
 func (s *DockerSuite) TestBuildBuildTimeArgExpansion(c *check.C) {
-	testRequires(c, DaemonIsLinux) // Windows does not support ARG
 	imgName := "bldvarstest"
 
 	wdVar := "WDIR"
@@ -4559,6 +4559,10 @@ func (s *DockerSuite) TestBuildBuildTimeArgExpansion(c *check.C) {
 	userVal := "testUser"
 	volVar := "VOL"
 	volVal := "/testVol/"
+	if DaemonIsWindows() {
+		volVal = "C:\\testVol"
+		wdVal = "C:\\tmp"
+	}
 
 	buildImageSuccessfully(c, imgName,
 		cli.WithFlags(
@@ -4594,7 +4598,7 @@ func (s *DockerSuite) TestBuildBuildTimeArgExpansion(c *check.C) {
 	)
 
 	res := inspectField(c, imgName, "Config.WorkingDir")
-	c.Check(res, check.Equals, filepath.ToSlash(wdVal))
+	c.Check(filepath.ToSlash(res), check.Equals, filepath.ToSlash(wdVal))
 
 	var resArr []string
 	inspectFieldAndUnmarshall(c, imgName, "Config.Env", &resArr)