Move builder to daemon/builder

Signed-off-by: Derek McGowan <derek@mcg.dev>
This commit is contained in:
Derek McGowan
2025-07-16 15:06:00 -07:00
parent 6ff9bea6a7
commit 7d48302134
61 changed files with 40 additions and 40 deletions

112
daemon/builder/builder.go Normal file
View File

@@ -0,0 +1,112 @@
// Package builder defines interfaces for any Docker builder to implement.
//
// Historically, only server-side Dockerfile interpreters existed.
// This package allows for other implementations of Docker builders.
package builder
import (
"context"
"io"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/image"
"github.com/docker/docker/layer"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
const (
// DefaultDockerfileName is the Default filename with Docker commands, read by docker build
DefaultDockerfileName = "Dockerfile"
)
// Source defines a location that can be used as a source for the ADD/COPY
// instructions in the builder.
type Source interface {
// Root returns root path for accessing source
Root() string
// Close allows to signal that the filesystem tree won't be used anymore.
// For Context implementations using a temporary directory, it is recommended to
// delete the temporary directory in Close().
Close() error
// Hash returns a checksum for a file
Hash(path string) (string, error)
}
// Backend abstracts calls to a Docker Daemon.
type Backend interface {
ImageBackend
ExecBackend
// CommitBuildStep creates a new Docker image from the config generated by
// a build step.
CommitBuildStep(context.Context, backend.CommitConfig) (image.ID, error)
// ContainerCreateWorkdir creates the workdir
ContainerCreateWorkdir(containerID string) error
CreateImage(ctx context.Context, config []byte, parent string, contentStoreDigest digest.Digest) (Image, error)
ImageCacheBuilder
}
// ImageBackend are the interface methods required from an image component
type ImageBackend interface {
GetImageAndReleasableLayer(ctx context.Context, refOrID string, opts backend.GetImageAndLayerOptions) (Image, ROLayer, error)
}
// ExecBackend contains the interface methods required for executing containers
type ExecBackend interface {
// ContainerAttachRaw attaches to container.
ContainerAttachRaw(cID string, stdin io.ReadCloser, stdout, stderr io.Writer, stream bool, attached chan struct{}) error
// ContainerCreateIgnoreImagesArgsEscaped creates a new Docker container and returns potential warnings
ContainerCreateIgnoreImagesArgsEscaped(ctx context.Context, config backend.ContainerCreateConfig) (container.CreateResponse, error)
// ContainerRm removes a container specified by `id`.
ContainerRm(name string, config *backend.ContainerRmConfig) error
// ContainerStart starts a new container
ContainerStart(ctx context.Context, containerID string, checkpoint string, checkpointDir string) error
// ContainerWait stops processing until the given container is stopped.
ContainerWait(ctx context.Context, name string, condition container.WaitCondition) (<-chan container.StateStatus, error)
}
// Result is the output produced by a Builder
type Result struct {
ImageID string
FromImage Image
}
// ImageCacheBuilder represents a generator for stateful image cache.
type ImageCacheBuilder interface {
// MakeImageCache creates a stateful image cache.
MakeImageCache(ctx context.Context, cacheFrom []string) (ImageCache, error)
}
// ImageCache abstracts an image cache.
// (parent image, child runconfig) -> child image
type ImageCache interface {
// GetCache returns a reference to a cached image whose parent equals `parent`
// and runconfig equals `cfg`. A cache miss is expected to return an empty ID and a nil error.
GetCache(parentID string, cfg *container.Config, platform ocispec.Platform) (imageID string, err error)
}
// Image represents a Docker image used by the builder.
type Image interface {
ImageID() string
RunConfig() *container.Config
MarshalJSON() ([]byte, error)
OperatingSystem() string
}
// ROLayer is a reference to image rootfs layer
type ROLayer interface {
Release() error
NewRWLayer() (RWLayer, error)
DiffID() layer.DiffID
ContentStoreDigest() digest.Digest
}
// RWLayer is active layer that can be read/modified
type RWLayer interface {
Release() error
Root() string
Commit() (ROLayer, error)
}

View File

@@ -0,0 +1,174 @@
package dockerfile
import (
"fmt"
"io"
"sort"
)
// builtinAllowedBuildArgs is list of built-in allowed build args
// these args are considered transparent and are excluded from the image history.
// Filtering from history is implemented in dispatchers.go
var builtinAllowedBuildArgs = map[string]bool{
"HTTP_PROXY": true,
"http_proxy": true,
"HTTPS_PROXY": true,
"https_proxy": true,
"FTP_PROXY": true,
"ftp_proxy": true,
"NO_PROXY": true,
"no_proxy": true,
"ALL_PROXY": true,
"all_proxy": true,
}
// BuildArgs manages arguments used by the builder
type BuildArgs struct {
// args that are allowed for expansion/substitution and passing to commands in 'run'.
allowedBuildArgs map[string]*string
// args defined before the first `FROM` in a Dockerfile
allowedMetaArgs map[string]*string
// args referenced by the Dockerfile
referencedArgs map[string]struct{}
// args provided by the user on the command line
argsFromOptions map[string]*string
}
// NewBuildArgs creates a new BuildArgs type
func NewBuildArgs(argsFromOptions map[string]*string) *BuildArgs {
return &BuildArgs{
allowedBuildArgs: make(map[string]*string),
allowedMetaArgs: make(map[string]*string),
referencedArgs: make(map[string]struct{}),
argsFromOptions: argsFromOptions,
}
}
// Clone returns a copy of the BuildArgs type
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
}
// MergeReferencedArgs merges referenced args from another BuildArgs
// object into the current one
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) {
var leftoverArgs []string
for arg := range b.argsFromOptions {
_, isReferenced := b.referencedArgs[arg]
_, isBuiltin := builtinAllowedBuildArgs[arg]
if !isBuiltin && !isReferenced {
leftoverArgs = append(leftoverArgs, arg)
}
}
if len(leftoverArgs) > 0 {
sort.Strings(leftoverArgs)
fmt.Fprintf(out, "[Warning] One or more build-args %v were not consumed\n", leftoverArgs)
}
}
// ResetAllowed clears the list of args that are allowed to be used by a
// directive
func (b *BuildArgs) ResetAllowed() {
b.allowedBuildArgs = make(map[string]*string)
}
// AddMetaArg adds a new meta arg that can be used by FROM directives
func (b *BuildArgs) AddMetaArg(key string, value *string) {
b.allowedMetaArgs[key] = value
}
// AddArg adds a new arg that can be used by directives
func (b *BuildArgs) AddArg(key string, value *string) {
b.allowedBuildArgs[key] = value
b.referencedArgs[key] = struct{}{}
}
// IsReferencedOrNotBuiltin checks if the key is a built-in arg, or if it has been
// referenced by the Dockerfile. Returns true if the arg is not a builtin or
// if the builtin has been referenced in the Dockerfile.
func (b *BuildArgs) IsReferencedOrNotBuiltin(key string) bool {
_, isBuiltin := builtinAllowedBuildArgs[key]
_, isAllowed := b.allowedBuildArgs[key]
return isAllowed || !isBuiltin
}
// GetAllAllowed returns a mapping with all the allowed args
func (b *BuildArgs) GetAllAllowed() map[string]string {
return b.getAllFromMapping(b.allowedBuildArgs)
}
// GetAllMeta returns a mapping with all the meta args
func (b *BuildArgs) GetAllMeta() map[string]string {
return b.getAllFromMapping(b.allowedMetaArgs)
}
func (b *BuildArgs) getAllFromMapping(source map[string]*string) map[string]string {
m := make(map[string]string)
keys := keysFromMaps(source, builtinAllowedBuildArgs)
for _, key := range keys {
v, ok := b.getBuildArg(key, source)
if ok {
m[key] = v
}
}
return m
}
// FilterAllowed returns all allowed args without the filtered args
func (b *BuildArgs) FilterAllowed(filter []string) []string {
envs := []string{}
configEnv := convertKVStringsToMap(filter)
for key, val := range b.GetAllAllowed() {
if _, ok := configEnv[key]; !ok {
envs = append(envs, fmt.Sprintf("%s=%s", key, val))
}
}
return envs
}
func (b *BuildArgs) getBuildArg(key string, mapping map[string]*string) (string, bool) {
defaultValue, exists := mapping[key]
// Return override from options if one is defined
if v, ok := b.argsFromOptions[key]; ok && v != nil {
return *v, ok
}
if defaultValue == nil {
if v, ok := b.allowedMetaArgs[key]; ok && v != nil {
return *v, ok
}
return "", false
}
return *defaultValue, exists
}
func keysFromMaps(source map[string]*string, builtin map[string]bool) []string {
keys := []string{}
for key := range source {
keys = append(keys, key)
}
for key := range builtin {
keys = append(keys, key)
}
return keys
}

View File

@@ -0,0 +1,104 @@
package dockerfile
import (
"bytes"
"strings"
"testing"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func strPtr(source string) *string {
return &source
}
func TestGetAllAllowed(t *testing.T) {
buildArgs := NewBuildArgs(map[string]*string{
"ArgNotUsedInDockerfile": strPtr("fromopt1"),
"ArgOverriddenByOptions": strPtr("fromopt2"),
"ArgNoDefaultInDockerfileFromOptions": strPtr("fromopt3"),
"HTTP_PROXY": strPtr("theproxy"),
"all_proxy": strPtr("theproxy2"),
})
buildArgs.AddMetaArg("ArgFromMeta", strPtr("frommeta1"))
buildArgs.AddMetaArg("ArgFromMetaOverridden", strPtr("frommeta2"))
buildArgs.AddMetaArg("ArgFromMetaNotUsed", strPtr("frommeta3"))
buildArgs.AddArg("ArgOverriddenByOptions", strPtr("fromdockerfile2"))
buildArgs.AddArg("ArgWithDefaultInDockerfile", strPtr("fromdockerfile1"))
buildArgs.AddArg("ArgNoDefaultInDockerfile", nil)
buildArgs.AddArg("ArgNoDefaultInDockerfileFromOptions", nil)
buildArgs.AddArg("ArgFromMeta", nil)
buildArgs.AddArg("ArgFromMetaOverridden", strPtr("fromdockerfile3"))
all := buildArgs.GetAllAllowed()
expected := map[string]string{
"HTTP_PROXY": "theproxy",
"all_proxy": "theproxy2",
"ArgOverriddenByOptions": "fromopt2",
"ArgWithDefaultInDockerfile": "fromdockerfile1",
"ArgNoDefaultInDockerfileFromOptions": "fromopt3",
"ArgFromMeta": "frommeta1",
"ArgFromMetaOverridden": "fromdockerfile3",
}
assert.Check(t, is.DeepEqual(expected, all))
}
func TestGetAllMeta(t *testing.T) {
buildArgs := NewBuildArgs(map[string]*string{
"ArgNotUsedInDockerfile": strPtr("fromopt1"),
"ArgOverriddenByOptions": strPtr("fromopt2"),
"ArgNoDefaultInMetaFromOptions": strPtr("fromopt3"),
"HTTP_PROXY": strPtr("theproxy"),
})
buildArgs.AddMetaArg("ArgFromMeta", strPtr("frommeta1"))
buildArgs.AddMetaArg("ArgOverriddenByOptions", strPtr("frommeta2"))
buildArgs.AddMetaArg("ArgNoDefaultInMetaFromOptions", nil)
all := buildArgs.GetAllMeta()
expected := map[string]string{
"HTTP_PROXY": "theproxy",
"ArgFromMeta": "frommeta1",
"ArgOverriddenByOptions": "fromopt2",
"ArgNoDefaultInMetaFromOptions": "fromopt3",
}
assert.Check(t, is.DeepEqual(expected, all))
}
func TestWarnOnUnusedBuildArgs(t *testing.T) {
buildArgs := NewBuildArgs(map[string]*string{
"ThisArgIsUsed": strPtr("fromopt1"),
"ThisArgIsNotUsed": strPtr("fromopt2"),
"HTTPS_PROXY": strPtr("referenced builtin"),
"HTTP_PROXY": strPtr("unreferenced builtin"),
})
buildArgs.AddArg("ThisArgIsUsed", nil)
buildArgs.AddArg("HTTPS_PROXY", nil)
buffer := new(bytes.Buffer)
buildArgs.WarnOnUnusedBuildArgs(buffer)
out := buffer.String()
assert.Assert(t, !strings.Contains(out, "ThisArgIsUsed"), out)
assert.Assert(t, !strings.Contains(out, "HTTPS_PROXY"), out)
assert.Assert(t, !strings.Contains(out, "HTTP_PROXY"), out)
assert.Check(t, is.Contains(out, "ThisArgIsNotUsed"))
}
func TestIsUnreferencedBuiltin(t *testing.T) {
buildArgs := NewBuildArgs(map[string]*string{
"ThisArgIsUsed": strPtr("fromopt1"),
"ThisArgIsNotUsed": strPtr("fromopt2"),
"HTTPS_PROXY": strPtr("referenced builtin"),
"HTTP_PROXY": strPtr("unreferenced builtin"),
})
buildArgs.AddArg("ThisArgIsUsed", nil)
buildArgs.AddArg("HTTPS_PROXY", nil)
assert.Check(t, buildArgs.IsReferencedOrNotBuiltin("ThisArgIsUsed"))
assert.Check(t, buildArgs.IsReferencedOrNotBuiltin("ThisArgIsNotUsed"))
assert.Check(t, buildArgs.IsReferencedOrNotBuiltin("HTTPS_PROXY"))
assert.Check(t, !buildArgs.IsReferencedOrNotBuiltin("HTTP_PROXY"))
}

View File

@@ -0,0 +1,389 @@
package dockerfile
import (
"bytes"
"context"
"fmt"
"io"
"sort"
"strings"
"github.com/containerd/log"
"github.com/containerd/platforms"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/build"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/daemon/builder/remotecontext"
"github.com/docker/docker/errdefs"
"github.com/docker/docker/pkg/stringid"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/buildkit/frontend/dockerfile/parser"
"github.com/moby/buildkit/frontend/dockerfile/shell"
"github.com/moby/sys/user"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
"golang.org/x/sync/syncmap"
)
var validCommitCommands = map[string]bool{
"cmd": true,
"entrypoint": true,
"healthcheck": true,
"env": true,
"expose": true,
"label": true,
"onbuild": true,
"stopsignal": true,
"user": true,
"volume": true,
"workdir": true,
}
const (
stepFormat = "Step %d/%d : %v"
)
// BuildManager is shared across all Builder objects
type BuildManager struct {
idMapping user.IdentityMapping
backend builder.Backend
pathCache pathCache // TODO: make this persistent
}
// NewBuildManager creates a BuildManager
func NewBuildManager(b builder.Backend, identityMapping user.IdentityMapping) (*BuildManager, error) {
bm := &BuildManager{
backend: b,
pathCache: &syncmap.Map{},
idMapping: identityMapping,
}
return bm, nil
}
// Build starts a new build from a BuildConfig
func (bm *BuildManager) Build(ctx context.Context, config backend.BuildConfig) (*builder.Result, error) {
buildsTriggered.Inc()
if config.Options.Dockerfile == "" {
config.Options.Dockerfile = builder.DefaultDockerfileName
}
source, dockerfile, err := remotecontext.Detect(config)
if err != nil {
return nil, err
}
defer func() {
if source != nil {
if err := source.Close(); err != nil {
log.G(ctx).Debugf("[BUILDER] failed to remove temporary context: %v", err)
}
}
}()
ctx, cancel := context.WithCancel(ctx)
defer cancel()
b, err := newBuilder(ctx, builderOptions{
Options: config.Options,
ProgressWriter: config.ProgressWriter,
Backend: bm.backend,
PathCache: bm.pathCache,
IDMapping: bm.idMapping,
})
if err != nil {
return nil, err
}
return b.build(ctx, source, dockerfile)
}
// builderOptions are the dependencies required by the builder
type builderOptions struct {
Options *build.ImageBuildOptions
Backend builder.Backend
ProgressWriter backend.ProgressWriter
PathCache pathCache
IDMapping user.IdentityMapping
}
// Builder is a Dockerfile builder
// It implements the builder.Backend interface.
type Builder struct {
options *build.ImageBuildOptions
Stdout io.Writer
Stderr io.Writer
Aux backend.AuxEmitter
Output io.Writer
docker builder.Backend
idMapping user.IdentityMapping
disableCommit bool
imageSources *imageSources
pathCache pathCache
containerManager *containerManager
imageProber ImageProber
platform *ocispec.Platform
}
// newBuilder creates a new Dockerfile builder from an optional dockerfile and a Options.
func newBuilder(ctx context.Context, options builderOptions) (*Builder, error) {
config := options.Options
if config == nil {
config = new(build.ImageBuildOptions)
}
imgProber, err := newImageProber(ctx, options.Backend, config.CacheFrom, config.NoCache)
if err != nil {
return nil, err
}
b := &Builder{
options: config,
Stdout: options.ProgressWriter.StdoutFormatter,
Stderr: options.ProgressWriter.StderrFormatter,
Aux: options.ProgressWriter.AuxFormatter,
Output: options.ProgressWriter.Output,
docker: options.Backend,
idMapping: options.IDMapping,
imageSources: newImageSources(options),
pathCache: options.PathCache,
imageProber: imgProber,
containerManager: newContainerManager(options.Backend),
}
// same as in Builder.Build in builder/builder-next/builder.go
// TODO: remove once config.Platform is of type specs.Platform
if config.Platform != "" {
sp, err := platforms.Parse(config.Platform)
if err != nil {
return nil, errdefs.InvalidParameter(err)
}
b.platform = &sp
}
return b, nil
}
// Build 'LABEL' command(s) from '--label' options and add to the last stage
func buildLabelOptions(labels map[string]string, stages []instructions.Stage) {
var keys []string
for key := range labels {
keys = append(keys, key)
}
// Sort the label to have a repeatable order
sort.Strings(keys)
for _, key := range keys {
value := labels[key]
stages[len(stages)-1].AddCommand(instructions.NewLabelCommand(key, value, true))
}
}
// Build runs the Dockerfile builder by parsing the Dockerfile and executing
// the instructions from the file.
func (b *Builder) build(ctx context.Context, source builder.Source, dockerfile *parser.Result) (*builder.Result, error) {
defer b.imageSources.Unmount()
stages, metaArgs, err := instructions.Parse(dockerfile.AST, nil)
if err != nil {
var uiErr *instructions.UnknownInstructionError
if errors.As(err, &uiErr) {
buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
}
return nil, errdefs.InvalidParameter(err)
}
if b.options.Target != "" {
targetIx, found := instructions.HasStage(stages, b.options.Target)
if !found {
buildsFailed.WithValues(metricsBuildTargetNotReachableError).Inc()
return nil, errdefs.InvalidParameter(errors.Errorf("target stage %q could not be found", b.options.Target))
}
stages = stages[:targetIx+1]
}
// Add 'LABEL' command specified by '--label' option to the last stage
buildLabelOptions(b.options.Labels, stages)
dockerfile.PrintWarnings(b.Stderr)
state, err := b.dispatchDockerfileWithCancellation(ctx, stages, metaArgs, dockerfile.EscapeToken, source)
if err != nil {
return nil, err
}
if state.imageID == "" {
buildsFailed.WithValues(metricsDockerfileEmptyError).Inc()
return nil, errors.New("No image was generated. Is your Dockerfile empty?")
}
return &builder.Result{ImageID: state.imageID, FromImage: state.baseImage}, nil
}
func emitImageID(aux backend.AuxEmitter, state *dispatchState) error {
if aux == nil || state.imageID == "" {
return nil
}
return aux.Emit("", build.Result{ID: state.imageID})
}
func processMetaArg(meta instructions.ArgCommand, shlex *shell.Lex, args *BuildArgs) error {
// shell.Lex currently only support the concatenated string format
envs := shell.EnvsFromSlice(convertMapToEnvList(args.GetAllAllowed()))
if err := meta.Expand(func(word string) (string, error) {
newword, _, err := shlex.ProcessWord(word, envs)
return newword, err
}); err != nil {
return err
}
for _, arg := range meta.Args {
args.AddArg(arg.Key, arg.Value)
args.AddMetaArg(arg.Key, arg.Value)
}
return nil
}
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(ctx context.Context, parseResult []instructions.Stage, metaArgs []instructions.ArgCommand, escapeToken rune, source builder.Source) (*dispatchState, error) {
request := dispatchRequest{}
buildArgs := NewBuildArgs(b.options.BuildArgs)
totalCommands := len(metaArgs) + len(parseResult)
currentCommandIndex := 1
for _, stage := range parseResult {
totalCommands += len(stage.Commands)
}
shlex := shell.NewLex(escapeToken)
for i := range metaArgs {
currentCommandIndex = printCommand(b.Stdout, currentCommandIndex, totalCommands, &metaArgs[i])
err := processMetaArg(metaArgs[i], shlex, buildArgs)
if err != nil {
return nil, err
}
}
stagesResults := newStagesBuildResults()
for _, s := range parseResult {
stage := s
if err := stagesResults.checkStageNameAvailable(stage.Name); err != nil {
return nil, err
}
request = newDispatchRequest(b, escapeToken, source, buildArgs, stagesResults)
currentCommandIndex = printCommand(b.Stdout, currentCommandIndex, totalCommands, stage.SourceCode)
if err := initializeStage(ctx, request, &stage); err != nil {
return nil, err
}
request.state.updateRunConfig()
_, _ = fmt.Fprintf(b.Stdout, " ---> %s\n", stringid.TruncateID(request.state.imageID))
for _, cmd := range stage.Commands {
select {
case <-ctx.Done():
log.G(ctx).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(ctx, request, cmd); err != nil {
return nil, err
}
request.state.updateRunConfig()
_, _ = fmt.Fprintf(b.Stdout, " ---> %s\n", stringid.TruncateID(request.state.imageID))
}
if err := emitImageID(b.Aux, request.state); err != nil {
return nil, err
}
buildArgs.MergeReferencedArgs(request.state.buildArgs)
if err := commitStage(request.state, stagesResults); err != nil {
return nil, err
}
}
buildArgs.WarnOnUnusedBuildArgs(b.Stdout)
return request.state, nil
}
// BuildFromConfig builds directly from `changes`, treating it as if it were the contents of a Dockerfile
// It will:
// - Call parse.Parse() to get an AST root for the concatenated Dockerfile entries.
// - Do build by calling builder.dispatch() to call all entries' handling routines
//
// BuildFromConfig is used by the /commit endpoint, with the changes
// coming from the query parameter of the same name.
//
// TODO: Remove?
func BuildFromConfig(ctx context.Context, config *container.Config, changes []string, os string) (*container.Config, error) {
if len(changes) == 0 {
return config, nil
}
dockerfile, err := parser.Parse(bytes.NewBufferString(strings.Join(changes, "\n")))
if err != nil {
return nil, errdefs.InvalidParameter(err)
}
// ensure that the commands are valid
var commands []instructions.Command
for _, n := range dockerfile.AST.Children {
if !validCommitCommands[strings.ToLower(n.Value)] {
return nil, errdefs.InvalidParameter(errors.Errorf("%s is not a valid change command", n.Value))
}
cmd, err := instructions.ParseCommand(n)
if err != nil {
return nil, errdefs.InvalidParameter(err)
}
commands = append(commands, cmd)
}
b, err := newBuilder(ctx, builderOptions{
Options: &build.ImageBuildOptions{NoCache: true},
})
if err != nil {
return nil, err
}
b.Stdout = io.Discard
b.Stderr = io.Discard
b.disableCommit = true
req := newDispatchRequest(b, dockerfile.EscapeToken, nil, NewBuildArgs(b.options.BuildArgs), newStagesBuildResults())
// We make mutations to the configuration, ensure we have a copy
req.state.runConfig = copyRunConfig(config)
req.state.imageID = config.Image
req.state.operatingSystem = os
for _, cmd := range commands {
err := dispatch(ctx, req, cmd)
if err != nil {
return nil, errdefs.InvalidParameter(err)
}
req.state.updateRunConfig()
}
return req.state.runConfig, nil
}
func convertMapToEnvList(m map[string]string) []string {
result := []string{}
for k, v := range m {
result = append(result, k+"="+v)
}
return result
}
// convertKVStringsToMap converts ["key=value"] to {"key":"value"}
func convertKVStringsToMap(values []string) map[string]string {
result := make(map[string]string, len(values))
for _, value := range values {
k, v, _ := strings.Cut(value, "=")
result[k] = v
}
return result
}

View File

@@ -0,0 +1,7 @@
//go:build !windows
package dockerfile
func defaultShellForOS(os string) []string {
return []string{"/bin/sh", "-c"}
}

View File

@@ -0,0 +1,26 @@
package dockerfile
import (
"os"
"path/filepath"
"strings"
)
func defaultShellForOS(os string) []string {
if os == "linux" {
return []string{"/bin/sh", "-c"}
}
return []string{"cmd", "/S", "/C"}
}
// isAbs is a platform-agnostic wrapper for filepath.IsAbs.
//
// On Windows, golang filepath.IsAbs does not consider a path \windows\system32
// as absolute as it doesn't start with a drive-letter/colon combination. However,
// in docker we need to verify things such as WORKDIR /windows/system32 in
// a Dockerfile (which gets translated to \windows\system32 when being processed
// by the daemon). This SHOULD be treated as absolute from a docker processing
// perspective.
func isAbs(path string) bool {
return filepath.IsAbs(path) || strings.HasPrefix(path, string(os.PathSeparator))
}

View File

@@ -0,0 +1,136 @@
package dockerfile
import (
"context"
"fmt"
"io"
cerrdefs "github.com/containerd/errdefs"
"github.com/containerd/log"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/pkg/stringid"
"github.com/pkg/errors"
)
type containerManager struct {
tmpContainers map[string]struct{}
backend builder.ExecBackend
}
// newContainerManager creates a new container backend
func newContainerManager(docker builder.ExecBackend) *containerManager {
return &containerManager{
backend: docker,
tmpContainers: make(map[string]struct{}),
}
}
// Create a container
func (c *containerManager) Create(ctx context.Context, runConfig *container.Config, hostConfig *container.HostConfig) (container.CreateResponse, error) {
ctr, err := c.backend.ContainerCreateIgnoreImagesArgsEscaped(ctx, backend.ContainerCreateConfig{
Config: runConfig,
HostConfig: hostConfig,
})
if err != nil {
return ctr, err
}
c.tmpContainers[ctr.ID] = struct{}{}
return ctr, nil
}
var errCancelled = errors.New("build cancelled")
// Run a container by ID
func (c *containerManager) Run(ctx context.Context, cID string, stdout, stderr io.Writer) error {
attached := make(chan struct{})
errCh := make(chan error, 1)
go func() {
errCh <- c.backend.ContainerAttachRaw(cID, nil, stdout, stderr, true, attached)
}()
select {
case err := <-errCh:
return err
case <-attached:
}
finished := make(chan struct{})
cancelErrCh := make(chan error, 1)
go func() {
select {
case <-ctx.Done():
log.G(ctx).Debugln("Build cancelled, removing container:", cID)
err := c.backend.ContainerRm(cID, &backend.ContainerRmConfig{ForceRemove: true, RemoveVolume: true})
if err != nil {
_, _ = fmt.Fprintf(stdout, "Removing container %s: %v\n", stringid.TruncateID(cID), err)
}
cancelErrCh <- errCancelled
case <-finished:
cancelErrCh <- nil
}
}()
if err := c.backend.ContainerStart(ctx, cID, "", ""); err != nil {
close(finished)
logCancellationError(cancelErrCh, "error from ContainerStart: "+err.Error())
return err
}
// Block on reading output from container, stop on err or chan closed
if err := <-errCh; err != nil {
close(finished)
logCancellationError(cancelErrCh, "error from errCh: "+err.Error())
return err
}
waitC, err := c.backend.ContainerWait(ctx, cID, container.WaitConditionNotRunning)
if err != nil {
close(finished)
logCancellationError(cancelErrCh, fmt.Sprintf("unable to begin ContainerWait: %s", err))
return err
}
if status := <-waitC; status.ExitCode() != 0 {
close(finished)
logCancellationError(cancelErrCh, fmt.Sprintf("a non-zero code from ContainerWait: %d", status.ExitCode()))
return &statusCodeError{code: status.ExitCode(), err: status.Err()}
}
close(finished)
return <-cancelErrCh
}
func logCancellationError(cancelErrCh chan error, msg string) {
if cancelErr := <-cancelErrCh; cancelErr != nil {
log.G(context.TODO()).Debugf("Build cancelled (%v): %s", cancelErr, msg)
}
}
type statusCodeError struct {
code int
err error
}
func (e *statusCodeError) Error() string {
if e.err == nil {
return ""
}
return e.err.Error()
}
func (e *statusCodeError) StatusCode() int {
return e.code
}
// RemoveAll containers managed by this container manager
func (c *containerManager) RemoveAll(stdout io.Writer) {
for containerID := range c.tmpContainers {
if err := c.backend.ContainerRm(containerID, &backend.ContainerRmConfig{ForceRemove: true, RemoveVolume: true}); err != nil && !cerrdefs.IsNotFound(err) {
_, _ = fmt.Fprintf(stdout, "Removing intermediate container %s: %v\n", stringid.TruncateID(containerID), err)
continue
}
delete(c.tmpContainers, containerID)
_, _ = fmt.Fprintf(stdout, " ---> Removed intermediate container %s\n", stringid.TruncateID(containerID))
}
}

View File

@@ -0,0 +1,552 @@
package dockerfile
import (
"context"
"fmt"
"io"
"mime"
"net/http"
"net/url"
"os"
"path/filepath"
"sort"
"strings"
"time"
"github.com/containerd/log"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/daemon/builder/remotecontext"
"github.com/docker/docker/daemon/builder/remotecontext/urlutil"
"github.com/docker/docker/pkg/longpath"
"github.com/docker/docker/pkg/progress"
"github.com/docker/docker/pkg/streamformatter"
"github.com/docker/docker/pkg/system"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/go-archive"
"github.com/moby/sys/symlink"
"github.com/moby/sys/user"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
const unnamedFilename = "__unnamed__"
type pathCache interface {
Load(key interface{}) (value interface{}, ok bool)
Store(key, value interface{})
}
// copyInfo is a data object which stores the metadata about each source file in
// a copyInstruction
type copyInfo struct {
root string
path string
hash string
noDecompress bool
}
func (c copyInfo) fullPath() (string, error) {
return symlink.FollowSymlinkInScope(filepath.Join(c.root, c.path), c.root)
}
func newCopyInfoFromSource(source builder.Source, path string, hash string) copyInfo {
return copyInfo{root: source.Root(), path: path, hash: hash}
}
func newCopyInfos(copyInfos ...copyInfo) []copyInfo {
return copyInfos
}
// copyInstruction is a fully parsed COPY or ADD command that is passed to
// Builder.performCopy to copy files into the image filesystem
type copyInstruction struct {
cmdName string
infos []copyInfo
dest string
chownStr string
allowLocalDecompression bool
preserveOwnership bool
}
// copier reads a raw COPY or ADD command, fetches remote sources using a downloader,
// and creates a copyInstruction
type copier struct {
imageSource *imageMount
source builder.Source
pathCache pathCache
download sourceDownloader
platform ocispec.Platform
// for cleanup. TODO: having copier.cleanup() is error prone and hard to
// follow. Code calling performCopy should manage the lifecycle of its params.
// Copier should take override source as input, not imageMount.
activeLayer builder.RWLayer
tmpPaths []string
}
func copierFromDispatchRequest(req dispatchRequest, download sourceDownloader, imageSource *imageMount) copier {
platform := req.builder.getPlatform(req.state)
return copier{
source: req.source,
pathCache: req.builder.pathCache,
download: download,
imageSource: imageSource,
platform: platform,
}
}
func (o *copier) createCopyInstruction(sourcesAndDest instructions.SourcesAndDest, cmdName string) (copyInstruction, error) {
inst := copyInstruction{
cmdName: cmdName,
dest: filepath.FromSlash(sourcesAndDest.DestPath),
}
infos, err := o.getCopyInfosForSourcePaths(sourcesAndDest.SourcePaths, inst.dest)
if err != nil {
return inst, errors.Wrapf(err, "%s failed", cmdName)
}
if len(infos) > 1 && !strings.HasSuffix(inst.dest, string(os.PathSeparator)) {
return inst, errors.Errorf("When using %s with more than one source file, the destination must be a directory and end with a /", cmdName)
}
inst.infos = infos
return inst, nil
}
// getCopyInfosForSourcePaths iterates over the source files and calculate the info
// needed to copy (e.g. hash value if cached)
// The dest is used in case source is URL (and ends with "/")
func (o *copier) getCopyInfosForSourcePaths(sources []string, dest string) ([]copyInfo, error) {
var infos []copyInfo
for _, orig := range sources {
subinfos, err := o.getCopyInfoForSourcePath(orig, dest)
if err != nil {
return nil, err
}
infos = append(infos, subinfos...)
}
if len(infos) == 0 {
return nil, errors.New("no source files were specified")
}
return infos, nil
}
func (o *copier) getCopyInfoForSourcePath(orig, dest string) ([]copyInfo, error) {
if !urlutil.IsURL(orig) {
return o.calcCopyInfo(orig, true)
}
remote, path, err := o.download(orig)
if err != nil {
return nil, err
}
// If path == "" then we are unable to determine filename from src
// We have to make sure dest is available
if path == "" {
if strings.HasSuffix(dest, "/") {
return nil, errors.Errorf("cannot determine filename for source %s", orig)
}
path = unnamedFilename
}
o.tmpPaths = append(o.tmpPaths, remote.Root())
hash, err := remote.Hash(path)
ci := newCopyInfoFromSource(remote, path, hash)
ci.noDecompress = true // data from http shouldn't be extracted even on ADD
return newCopyInfos(ci), err
}
// Cleanup removes any temporary directories created as part of downloading
// remote files.
func (o *copier) Cleanup() {
for _, path := range o.tmpPaths {
os.RemoveAll(path)
}
o.tmpPaths = []string{}
if o.activeLayer != nil {
o.activeLayer.Release()
o.activeLayer = nil
}
}
// TODO: allowWildcards can probably be removed by refactoring this function further.
func (o *copier) calcCopyInfo(origPath string, allowWildcards bool) ([]copyInfo, error) {
imageSource := o.imageSource
if err := validateCopySourcePath(imageSource, origPath); err != nil {
return nil, err
}
// TODO: do this when creating copier. Requires validateCopySourcePath
// (and other below) to be aware of the difference sources. Why is it only
// done on image Source?
if imageSource != nil && o.activeLayer == nil {
// this needs to be protected against repeated calls as wildcard copy
// will call it multiple times for a single COPY
var err error
rwLayer, err := imageSource.NewRWLayer()
if err != nil {
return nil, err
}
o.activeLayer = rwLayer
o.source, err = remotecontext.NewLazySource(rwLayer.Root())
if err != nil {
return nil, errors.Wrapf(err, "failed to create context for copy from %s", rwLayer.Root())
}
}
if o.source == nil {
return nil, errors.Errorf("missing build context")
}
// Work in daemon-specific OS filepath semantics
origPath = filepath.FromSlash(origPath)
origPath = strings.TrimPrefix(origPath, string(os.PathSeparator))
origPath = strings.TrimPrefix(origPath, "."+string(os.PathSeparator))
// Deal with wildcards
if allowWildcards && containsWildcards(origPath) {
return o.copyWithWildcards(origPath)
}
if imageSource != nil && imageSource.ImageID() != "" {
// return a cached copy if one exists
if h, ok := o.pathCache.Load(imageSource.ImageID() + origPath); ok {
return newCopyInfos(newCopyInfoFromSource(o.source, origPath, h.(string))), nil
}
}
// Deal with the single file case
info, err := copyInfoForFile(o.source, origPath)
switch {
case imageSource == nil && errors.Is(err, os.ErrNotExist):
return nil, errors.Wrapf(err, "file not found in build context or excluded by .dockerignore")
case err != nil:
return nil, err
case info.hash != "":
o.storeInPathCache(imageSource, origPath, info.hash)
return newCopyInfos(info), err
}
// TODO: remove, handle dirs in Hash()
subfiles, err := walkSource(o.source, origPath)
if err != nil {
return nil, err
}
hash := hashStringSlice("dir", subfiles)
o.storeInPathCache(imageSource, origPath, hash)
return newCopyInfos(newCopyInfoFromSource(o.source, origPath, hash)), nil
}
func (o *copier) storeInPathCache(im *imageMount, path string, hash string) {
if im != nil {
o.pathCache.Store(im.ImageID()+path, hash)
}
}
func (o *copier) copyWithWildcards(origPath string) ([]copyInfo, error) {
root := o.source.Root()
var copyInfos []copyInfo
if err := filepath.WalkDir(root, func(path string, _ os.DirEntry, err error) error {
if err != nil {
return err
}
rel, err := filepath.Rel(root, path)
if err != nil {
return err
}
if rel == "." {
return nil
}
if match, _ := filepath.Match(origPath, rel); !match {
return nil
}
// Note we set allowWildcards to false in case the name has
// a * in it
subInfos, err := o.calcCopyInfo(rel, false)
if err != nil {
return err
}
copyInfos = append(copyInfos, subInfos...)
return nil
}); err != nil {
return nil, err
}
return copyInfos, nil
}
func copyInfoForFile(source builder.Source, path string) (copyInfo, error) {
fi, err := remotecontext.StatAt(source, path)
if err != nil {
if errors.Is(err, os.ErrNotExist) {
// return the relative path in the error, which is more user-friendly than the full path to the tmp-dir
return copyInfo{}, errors.WithStack(&os.PathError{Op: "stat", Path: path, Err: os.ErrNotExist})
}
return copyInfo{}, err
}
if fi.IsDir() {
return copyInfo{}, nil
}
hash, err := source.Hash(path)
if err != nil {
return copyInfo{}, err
}
return newCopyInfoFromSource(source, path, "file:"+hash), nil
}
// TODO: dedupe with copyWithWildcards()
func walkSource(source builder.Source, origPath string) ([]string, error) {
fp, err := remotecontext.FullPath(source, origPath)
if err != nil {
return nil, err
}
// Must be a dir
var subfiles []string
err = filepath.WalkDir(fp, func(path string, _ os.DirEntry, err error) error {
if err != nil {
return err
}
rel, err := filepath.Rel(source.Root(), path)
if err != nil {
return err
}
if rel == "." {
return nil
}
hash, err := source.Hash(rel)
if err != nil {
return nil
}
// we already checked handleHash above
subfiles = append(subfiles, hash)
return nil
})
if err != nil {
return nil, err
}
sort.Strings(subfiles)
return subfiles, nil
}
type sourceDownloader func(string) (builder.Source, string, error)
func newRemoteSourceDownloader(output, stdout io.Writer) sourceDownloader {
return func(url string) (builder.Source, string, error) {
return downloadSource(output, stdout, url)
}
}
func errOnSourceDownload(_ string) (builder.Source, string, error) {
return nil, "", errors.New("source can't be a URL for COPY")
}
func getFilenameForDownload(path string, resp *http.Response) string {
// Guess filename based on source
if path != "" && !strings.HasSuffix(path, "/") {
if filename := filepath.Base(filepath.FromSlash(path)); filename != "" {
return filename
}
}
// Guess filename based on Content-Disposition
if contentDisposition := resp.Header.Get("Content-Disposition"); contentDisposition != "" {
if _, params, err := mime.ParseMediaType(contentDisposition); err == nil {
if params["filename"] != "" && !strings.HasSuffix(params["filename"], "/") {
if filename := filepath.Base(filepath.FromSlash(params["filename"])); filename != "" {
return filename
}
}
}
}
return ""
}
func downloadSource(output io.Writer, stdout io.Writer, srcURL string) (remote builder.Source, p string, retErr error) {
u, err := url.Parse(srcURL)
if err != nil {
return nil, "", err
}
resp, err := remotecontext.GetWithStatusError(srcURL)
if err != nil {
return nil, "", err
}
filename := getFilenameForDownload(u.Path, resp)
// Prepare file in a tmp dir
tmpDir, err := longpath.MkdirTemp("", "docker-remote")
if err != nil {
return nil, "", err
}
defer func() {
if retErr != nil {
if err := os.RemoveAll(tmpDir); err != nil {
log.G(context.TODO()).WithError(err).Debug("error cleaning up temp-directory after failing to download source")
}
}
}()
// If filename is empty, the returned filename will be "" but
// the tmp filename will be created as "__unnamed__"
tmpFileName := filename
if filename == "" {
tmpFileName = unnamedFilename
}
tmpFileName = filepath.Join(tmpDir, tmpFileName)
tmpFile, err := os.OpenFile(tmpFileName, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o600)
if err != nil {
return nil, "", err
}
defer func() {
if retErr != nil {
// Ignore os.ErrClosed errors, as the file may already be closed in this function.
if err := tmpFile.Close(); err != nil && !errors.Is(err, os.ErrClosed) {
log.G(context.TODO()).WithError(err).Debug("error closing temp-file after failing to download source")
}
}
}()
progressOutput := streamformatter.NewJSONProgressOutput(output, true)
progressReader := progress.NewProgressReader(resp.Body, progressOutput, resp.ContentLength, "", "Downloading")
// Download and dump result to tmp file
// TODO: add filehash directly
if _, err = io.Copy(tmpFile, progressReader); err != nil {
return nil, "", err
}
// TODO: how important is this random blank line to the output?
_, _ = fmt.Fprintln(stdout)
// Set the mtime to the Last-Modified header value if present
// Otherwise just remove atime and mtime
mTime := time.Time{}
lastMod := resp.Header.Get("Last-Modified")
if lastMod != "" {
// If we can't parse it then just let it default to 'zero'
// otherwise use the parsed time value
if parsedMTime, err := http.ParseTime(lastMod); err == nil {
mTime = parsedMTime
}
}
// TODO(thaJeztah): was there a reason for this file to be closed _before_ system.Chtimes, or could we unconditionally close this in a defer?
if err := tmpFile.Close(); err != nil {
log.G(context.TODO()).WithError(err).Debug("error closing temp-file before chtimes")
}
if err = system.Chtimes(tmpFileName, mTime, mTime); err != nil {
return nil, "", err
}
lc, err := remotecontext.NewLazySource(tmpDir)
return lc, filename, err
}
type identity struct {
UID int
GID int
SID string
}
type copyFileOptions struct {
decompress bool
identity *identity
archiver *archive.Archiver
}
func performCopyForInfo(dest copyInfo, source copyInfo, options copyFileOptions) error {
srcPath, err := source.fullPath()
if err != nil {
return err
}
destPath, err := dest.fullPath()
if err != nil {
return err
}
archiver := options.archiver
src, err := os.Stat(srcPath)
if err != nil {
return errors.Wrapf(err, "source path not found")
}
if src.IsDir() {
return copyDirectory(archiver, srcPath, destPath, options.identity)
}
if options.decompress && archive.IsArchivePath(srcPath) && !source.noDecompress {
f, err := os.Open(srcPath)
if err != nil {
return err
}
defer f.Close()
return archiver.Untar(f, destPath, &archive.TarOptions{
IDMap: archiver.IDMapping,
BestEffortXattrs: true,
})
}
destExistsAsDir, err := isExistingDirectory(destPath)
if err != nil {
return err
}
// dest.path must be used because destPath has already been cleaned of any
// trailing slash
if destExistsAsDir || strings.HasSuffix(dest.path, string(os.PathSeparator)) {
// source.path must be used to get the correct filename when the source
// is a symlink
destPath = filepath.Join(destPath, filepath.Base(source.path))
}
return copyFile(archiver, srcPath, destPath, options.identity)
}
func copyDirectory(archiver *archive.Archiver, source, dest string, identity *identity) error {
destExists, err := isExistingDirectory(dest)
if err != nil {
return errors.Wrapf(err, "failed to query destination path")
}
if err := archiver.CopyWithTar(source, dest); err != nil {
return errors.Wrapf(err, "failed to copy directory")
}
if identity != nil {
return fixPermissions(source, dest, *identity, !destExists)
}
return nil
}
func copyFile(archiver *archive.Archiver, source, dest string, identity *identity) error {
if identity == nil {
if err := os.MkdirAll(filepath.Dir(dest), 0o755); err != nil {
return err
}
} else {
if err := user.MkdirAllAndChown(filepath.Dir(dest), 0o755, identity.UID, identity.GID, user.WithOnlyNew); err != nil {
return errors.Wrapf(err, "failed to create new directory")
}
}
if err := archiver.CopyFileWithTar(source, dest); err != nil {
return errors.Wrapf(err, "failed to copy file")
}
if identity != nil {
return fixPermissions(source, dest, *identity, false)
}
return nil
}
// isExistingDirectory returns true if the path exists and is a directory
func isExistingDirectory(path string) (bool, error) {
destStat, err := os.Stat(path)
switch {
case errors.Is(err, os.ErrNotExist):
return false, nil
case err != nil:
return false, err
}
return destStat.IsDir(), nil
}

View File

@@ -0,0 +1,148 @@
package dockerfile
import (
"net/http"
"os"
"path/filepath"
"testing"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func TestIsExistingDirectory(t *testing.T) {
tmpDir := t.TempDir()
tmpFile := filepath.Join(tmpDir, "existing-file")
err := os.WriteFile(tmpFile, []byte("something"), 0o644)
assert.NilError(t, err)
testcases := []struct {
doc string
path string
expected bool
}{
{
doc: "directory exists",
path: tmpDir,
expected: true,
},
{
doc: "path doesn't exist",
path: "/bogus/path/does/not/exist",
expected: false,
},
{
doc: "file exists",
path: tmpFile,
expected: false,
},
}
for _, testcase := range testcases {
result, err := isExistingDirectory(testcase.path)
if !assert.Check(t, err) {
continue
}
assert.Check(t, is.Equal(testcase.expected, result), testcase.doc)
}
}
func TestGetFilenameForDownload(t *testing.T) {
testcases := []struct {
path string
disposition string
expected string
}{
{
path: "https://www.example.com/",
expected: "",
},
{
path: "https://www.example.com/xyz",
expected: "xyz",
},
{
path: "https://www.example.com/xyz.html",
expected: "xyz.html",
},
{
path: "https://www.example.com/xyz/",
expected: "",
},
{
path: "https://www.example.com/xyz/uvw",
expected: "uvw",
},
{
path: "https://www.example.com/xyz/uvw.html",
expected: "uvw.html",
},
{
path: "https://www.example.com/xyz/uvw/",
expected: "",
},
{
path: "/",
expected: "",
},
{
path: "/xyz",
expected: "xyz",
},
{
path: "/xyz.html",
expected: "xyz.html",
},
{
path: "/xyz/",
expected: "",
},
{
path: "/xyz/",
disposition: "attachment; filename=xyz.html",
expected: "xyz.html",
},
{
disposition: "",
expected: "",
},
{
disposition: "attachment; filename=xyz",
expected: "xyz",
},
{
disposition: "attachment; filename=xyz.html",
expected: "xyz.html",
},
{
disposition: `attachment; filename="xyz"`,
expected: "xyz",
},
{
disposition: `attachment; filename="xyz.html"`,
expected: "xyz.html",
},
{
disposition: `attachment; filename="/xyz.html"`,
expected: "xyz.html",
},
{
disposition: `attachment; filename="/xyz/uvw"`,
expected: "uvw",
},
{
disposition: `attachment; filename="Naïve file.txt"`,
expected: "Naïve file.txt",
},
}
for _, testcase := range testcases {
resp := http.Response{
Header: make(map[string][]string),
}
if testcase.disposition != "" {
resp.Header.Add("Content-Disposition", testcase.disposition)
}
filename := getFilenameForDownload(testcase.path, &resp)
assert.Check(t, is.Equal(testcase.expected, filename))
}
}

View File

@@ -0,0 +1,75 @@
//go:build !windows
package dockerfile
import (
"os"
"path"
"path/filepath"
"strings"
)
func fixPermissions(source, destination string, id identity, overrideSkip bool) error {
var (
skipChownRoot bool
err error
)
if !overrideSkip {
skipChownRoot, err = isExistingDirectory(destination)
if err != nil {
return err
}
}
// We Walk on the source rather than on the destination because we don't
// want to change permissions on things we haven't created or modified.
return filepath.WalkDir(source, func(fullpath string, _ os.DirEntry, _ error) error {
// Do not alter the walk root iff. it existed before, as it doesn't fall under
// the domain of "things we should chown".
if skipChownRoot && source == fullpath {
return nil
}
// Path is prefixed by source: substitute with destination instead.
cleaned, err := filepath.Rel(source, fullpath)
if err != nil {
return err
}
fullpath = filepath.Join(destination, cleaned)
return os.Lchown(fullpath, id.UID, id.GID)
})
}
// normalizeDest normalises the destination of a COPY/ADD command in a
// platform semantically consistent way.
func normalizeDest(workingDir, requested string) (string, error) {
dest := filepath.FromSlash(requested)
endsInSlash := strings.HasSuffix(dest, string(os.PathSeparator))
if !path.IsAbs(requested) {
dest = path.Join("/", filepath.ToSlash(workingDir), dest)
// Make sure we preserve any trailing slash
if endsInSlash {
dest += "/"
}
}
return dest, nil
}
func containsWildcards(name string) bool {
for i := 0; i < len(name); i++ {
ch := name[i]
switch ch {
case '\\':
i++
case '*', '?', '[':
return true
}
}
return false
}
func validateCopySourcePath(imageSource *imageMount, origPath string) error {
return nil
}

View File

@@ -0,0 +1,157 @@
package dockerfile
import (
"fmt"
"os"
"path/filepath"
"strings"
"github.com/Microsoft/go-winio"
"github.com/docker/docker/pkg/system"
"github.com/moby/sys/reexec"
"github.com/pkg/errors"
"golang.org/x/sys/windows"
)
var pathDenyList = map[string]bool{
`c:\`: true,
`c:\windows`: true,
}
func init() {
reexec.Register("windows-fix-permissions", fixPermissionsReexec)
}
func fixPermissions(source, destination string, id identity, _ bool) error {
if id.SID == "" {
return nil
}
cmd := reexec.Command("windows-fix-permissions", source, destination, id.SID)
output, err := cmd.CombinedOutput()
return errors.Wrapf(err, "failed to exec windows-fix-permissions: %s", output)
}
func fixPermissionsReexec() {
err := fixPermissionsWindows(os.Args[1], os.Args[2], os.Args[3])
if err != nil {
fmt.Fprint(os.Stderr, err)
os.Exit(1)
}
}
func fixPermissionsWindows(source, destination, SID string) error {
privileges := []string{winio.SeRestorePrivilege, seTakeOwnershipPrivilege}
err := winio.EnableProcessPrivileges(privileges)
if err != nil {
return err
}
defer winio.DisableProcessPrivileges(privileges)
sid, err := windows.StringToSid(SID)
if err != nil {
return err
}
// Owners on *nix have read/write/delete/read control and write DAC.
// Add an ACE that grants this to the user/group specified with the
// chown option. Currently Windows is not honoring the owner change,
// however, they are aware of this and it should be fixed at some
// point.
sddlString := system.SddlAdministratorsLocalSystem
sddlString += "(A;OICI;GRGWGXRCWDSD;;;" + SID + ")"
securityDescriptor, err := windows.SecurityDescriptorFromString(sddlString)
if err != nil {
return err
}
dacl, _, err := securityDescriptor.DACL()
if err != nil {
return err
}
return windows.SetNamedSecurityInfo(destination, windows.SE_FILE_OBJECT, windows.OWNER_SECURITY_INFORMATION|windows.DACL_SECURITY_INFORMATION, sid, nil, dacl, nil)
}
// normalizeDest normalises the destination of a COPY/ADD command in a
// platform semantically consistent way.
func normalizeDest(workingDir, requested string) (string, error) {
dest := filepath.FromSlash(requested)
endsInSlash := strings.HasSuffix(dest, string(os.PathSeparator))
// We are guaranteed that the working directory is already consistent,
// However, Windows also has, for now, the limitation that ADD/COPY can
// only be done to the system drive, not any drives that might be present
// as a result of a bind mount.
//
// So... if the path requested is Linux-style absolute (/foo or \\foo),
// we assume it is the system drive. If it is a Windows-style absolute
// (DRIVE:\\foo), error if DRIVE is not C. And finally, ensure we
// strip any configured working directories drive letter so that it
// can be subsequently legitimately converted to a Windows volume-style
// pathname.
// Not a typo - filepath.IsAbs, not system.IsAbs on this next check as
// we only want to validate where the DriveColon part has been supplied.
if filepath.IsAbs(dest) {
if strings.ToUpper(string(dest[0])) != "C" {
return "", fmt.Errorf("Windows does not support destinations not on the system drive (C:)")
}
dest = dest[2:] // Strip the drive letter
}
// Cannot handle relative where WorkingDir is not the system drive.
if len(workingDir) > 0 {
if ((len(workingDir) > 1) && !isAbs(workingDir[2:])) || (len(workingDir) == 1) {
return "", fmt.Errorf("Current WorkingDir %s is not platform consistent", workingDir)
}
if !isAbs(dest) {
if string(workingDir[0]) != "C" {
return "", fmt.Errorf("Windows does not support relative paths when WORKDIR is not the system drive")
}
dest = filepath.Join(string(os.PathSeparator), workingDir[2:], dest)
// Make sure we preserve any trailing slash
if endsInSlash {
dest += string(os.PathSeparator)
}
}
}
return dest, nil
}
func containsWildcards(name string) bool {
for i := 0; i < len(name); i++ {
ch := name[i]
if ch == '*' || ch == '?' || ch == '[' {
return true
}
}
return false
}
func validateCopySourcePath(imageSource *imageMount, origPath string) error {
if imageSource == nil {
return nil
}
origPath = filepath.FromSlash(origPath)
p := strings.ToLower(filepath.Clean(origPath))
if !filepath.IsAbs(p) {
if filepath.VolumeName(p) != "" {
if p[len(p)-2:] == ":." { // case where clean returns weird c:. paths
p = p[:len(p)-1]
}
p += `\`
} else {
p = filepath.Join(`c:\`, p)
}
}
if _, ok := pathDenyList[p]; ok {
return errors.New(`copy from c:\ or c:\windows is not allowed on windows`)
}
return nil
}

View File

@@ -0,0 +1,605 @@
package dockerfile
// This file contains the dispatchers for each command. Note that
// `nullDispatch` is not actually a command, but support for commands we parse
// but do nothing with.
//
// See evaluator.go for a higher level discussion of the whole evaluator
// package.
import (
"bytes"
"context"
"fmt"
"runtime"
"sort"
"strings"
"github.com/containerd/platforms"
"github.com/docker/docker/api"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/errdefs"
"github.com/docker/docker/image"
"github.com/docker/docker/pkg/jsonmessage"
"github.com/docker/go-connections/nat"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/buildkit/frontend/dockerfile/parser"
"github.com/moby/buildkit/frontend/dockerfile/shell"
"github.com/moby/sys/signal"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
// ENV foo bar
//
// Sets the environment variable foo to bar, also makes interpolation
// in the dockerfile available from the next statement on via ${foo}.
func dispatchEnv(ctx context.Context, 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()
commitMessage.WriteString(" " + newVar)
gotOne := false
for i, envVar := range runConfig.Env {
compareFrom, _, _ := strings.Cut(envVar, "=")
if shell.EqualEnvKeys(compareFrom, name) {
runConfig.Env[i] = newVar
gotOne = true
break
}
}
if !gotOne {
runConfig.Env = append(runConfig.Env, newVar)
}
}
return d.builder.commit(ctx, d.state, commitMessage.String())
}
// MAINTAINER some text <maybe@an.email.address>
//
// Sets the maintainer metadata.
func dispatchMaintainer(ctx context.Context, d dispatchRequest, c *instructions.MaintainerCommand) error {
d.state.maintainer = c.Maintainer
return d.builder.commit(ctx, d.state, "MAINTAINER "+c.Maintainer)
}
// LABEL some json data describing the image
//
// Sets the Label variable foo to bar,
func dispatchLabel(ctx context.Context, d dispatchRequest, c *instructions.LabelCommand) error {
if d.state.runConfig.Labels == nil {
d.state.runConfig.Labels = make(map[string]string)
}
commitStr := "LABEL"
for _, v := range c.Labels {
d.state.runConfig.Labels[v.Key] = v.Value
commitStr += " " + v.String()
}
return d.builder.commit(ctx, d.state, commitStr)
}
// ADD foo /path
//
// Add the file 'foo' to '/path'. Tarball and Remote URL (http, https) handling
// exist here. If you do not wish to have this automatic handling, use COPY.
func dispatchAdd(ctx context.Context, d dispatchRequest, c *instructions.AddCommand) error {
if c.Chmod != "" {
return errors.New("the --chmod option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled")
}
downloader := newRemoteSourceDownloader(d.builder.Output, d.builder.Stdout)
cpr := copierFromDispatchRequest(d, downloader, nil)
defer cpr.Cleanup()
instruction, err := cpr.createCopyInstruction(c.SourcesAndDest, "ADD")
if err != nil {
return err
}
instruction.chownStr = c.Chown
instruction.allowLocalDecompression = true
return d.builder.performCopy(ctx, d, instruction)
}
// COPY foo /path
//
// Same as 'ADD' but without the tar and remote url handling.
func dispatchCopy(ctx context.Context, d dispatchRequest, c *instructions.CopyCommand) error {
if c.Chmod != "" {
return errors.New("the --chmod option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled")
}
var im *imageMount
var err error
if c.From != "" {
im, err = d.getImageMount(ctx, c.From)
if err != nil {
return errors.Wrapf(err, "invalid from flag value %s", c.From)
}
}
cpr := copierFromDispatchRequest(d, errOnSourceDownload, im)
defer cpr.Cleanup()
instruction, err := cpr.createCopyInstruction(c.SourcesAndDest, "COPY")
if err != nil {
return err
}
instruction.chownStr = c.Chown
if c.From != "" && instruction.chownStr == "" {
instruction.preserveOwnership = true
}
return d.builder.performCopy(ctx, d, instruction)
}
func (d *dispatchRequest) getImageMount(ctx context.Context, imageRefOrID string) (*imageMount, error) {
if imageRefOrID == "" {
// TODO: this could return the source in the default case as well?
return nil, nil
}
var localOnly bool
stage, err := d.stages.get(imageRefOrID)
if err != nil {
return nil, err
}
if stage != nil {
imageRefOrID = stage.Image
localOnly = true
}
return d.builder.imageSources.Get(ctx, imageRefOrID, localOnly, d.builder.platform)
}
// FROM [--platform=platform] imagename[:tag | @digest] [AS build-stage-name]
func initializeStage(ctx context.Context, d dispatchRequest, cmd *instructions.Stage) error {
err := d.builder.imageProber.Reset(ctx)
if err != nil {
return err
}
var platform *ocispec.Platform
if val := cmd.Platform; val != "" {
v, err := d.getExpandedString(d.shlex, val)
if err != nil {
return errors.Wrapf(err, "failed to process arguments for platform %s", v)
}
p, err := platforms.Parse(v)
if err != nil {
return errors.Wrapf(errdefs.InvalidParameter(err), "failed to parse platform %s", v)
}
platform = &p
}
img, err := d.getFromImage(ctx, d.shlex, cmd.BaseName, platform)
if err != nil {
return err
}
state := d.state
if err := state.beginStage(cmd.Name, img); err != nil {
return err
}
if len(state.runConfig.OnBuild) > 0 {
triggers := state.runConfig.OnBuild
state.runConfig.OnBuild = nil
return dispatchTriggeredOnBuild(ctx, d, triggers)
}
return nil
}
func dispatchTriggeredOnBuild(ctx context.Context, 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 {
var uiErr *instructions.UnknownInstructionError
if errors.As(err, &uiErr) {
buildsFailed.WithValues(metricsUnknownInstructionError).Inc()
}
return err
}
err = dispatch(ctx, d, cmd)
if err != nil {
return err
}
}
return nil
}
func (d *dispatchRequest) getExpandedString(shlex *shell.Lex, str string) (string, error) {
substitutionArgs := []string{}
for key, value := range d.state.buildArgs.GetAllMeta() {
substitutionArgs = append(substitutionArgs, key+"="+value)
}
name, _, err := shlex.ProcessWord(str, shell.EnvsFromSlice(substitutionArgs))
if err != nil {
return "", err
}
return name, nil
}
func (d *dispatchRequest) getImageOrStage(ctx context.Context, name string, platform *ocispec.Platform) (builder.Image, error) {
var localOnly bool
if im, ok := d.stages.getByName(name); ok {
name = im.Image
localOnly = true
}
if platform == nil {
platform = d.builder.platform
}
// Windows cannot support a container with no base image.
if name == api.NoBaseImageSpecifier {
// Windows supports scratch. What is not supported is running containers from it.
if runtime.GOOS == "windows" {
return nil, errors.New("Windows does not support FROM scratch")
}
// TODO: scratch should not have an os. It should be nil image.
imageImage := &image.Image{}
if platform != nil {
imageImage.OS = platform.OS
} else {
imageImage.OS = runtime.GOOS
}
return builder.Image(imageImage), nil
}
imageMount, err := d.builder.imageSources.Get(ctx, name, localOnly, platform)
if err != nil {
return nil, err
}
return imageMount.Image(), nil
}
func (d *dispatchRequest) getFromImage(ctx context.Context, shlex *shell.Lex, basename string, platform *ocispec.Platform) (builder.Image, error) {
name, err := d.getExpandedString(shlex, basename)
if err != nil {
return nil, err
}
// Empty string is interpreted to FROM scratch by images.GetImageAndReleasableLayer,
// so validate expanded result is not empty.
if name == "" {
return nil, errors.Errorf("base name (%s) should not be blank", basename)
}
return d.getImageOrStage(ctx, name, platform)
}
func dispatchOnbuild(ctx context.Context, d dispatchRequest, c *instructions.OnbuildCommand) error {
d.state.runConfig.OnBuild = append(d.state.runConfig.OnBuild, c.Expression)
return d.builder.commit(ctx, d.state, "ONBUILD "+c.Expression)
}
// WORKDIR /tmp
//
// Set the working directory for future RUN/CMD/etc statements.
func dispatchWorkdir(ctx context.Context, d dispatchRequest, c *instructions.WorkdirCommand) error {
runConfig := d.state.runConfig
var err error
runConfig.WorkingDir, err = normalizeWorkdir(d.state.operatingSystem, runConfig.WorkingDir, c.Path)
if err != nil {
return err
}
// For performance reasons, we explicitly do a create/mkdir now
// 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 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, d.state.operatingSystem))
containerID, err := d.builder.probeAndCreate(ctx, d.state, runConfigWithCommentCmd)
if err != nil || containerID == "" {
return err
}
if err := d.builder.docker.ContainerCreateWorkdir(containerID); err != nil {
return err
}
return d.builder.commitContainer(ctx, d.state, containerID, runConfigWithCommentCmd)
}
// 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 and LCOW)
// RUN echo hi # cmd /S /C echo hi (Windows)
// RUN [ "echo", "hi" ] # echo hi
func dispatchRun(ctx context.Context, d dispatchRequest, c *instructions.RunCommand) error {
if err := image.CheckOS(d.state.operatingSystem); err != nil {
return err
}
if len(c.FlagsUsed) > 0 {
// classic builder RUN currently does not support any flags, so fail on the first one
return errors.Errorf("the --%s option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled", c.FlagsUsed[0])
}
stateRunConfig := d.state.runConfig
cmdFromArgs, argsEscaped := resolveCmdLine(c.ShellDependantCmdLine, stateRunConfig, d.state.operatingSystem, c.Name(), c.String())
buildArgs := d.state.buildArgs.FilterAllowed(stateRunConfig.Env)
saveCmd := cmdFromArgs
if len(buildArgs) > 0 {
saveCmd = prependEnvOnCmd(d.state.buildArgs, buildArgs, cmdFromArgs)
}
cacheArgsEscaped := argsEscaped
// ArgsEscaped is not persisted in the committed image on Windows.
// Use the original from previous build steps for cache probing.
if d.state.operatingSystem == "windows" {
cacheArgsEscaped = stateRunConfig.ArgsEscaped
}
runConfigForCacheProbe := copyRunConfig(stateRunConfig,
withCmd(saveCmd),
withArgsEscaped(cacheArgsEscaped),
withEntrypointOverride(saveCmd, nil))
if hit, err := d.builder.probeCache(d.state, runConfigForCacheProbe); err != nil || hit {
return err
}
runConfig := copyRunConfig(stateRunConfig,
withCmd(cmdFromArgs),
withArgsEscaped(argsEscaped),
withEnv(append(stateRunConfig.Env, buildArgs...)),
withEntrypointOverride(saveCmd, []string{""}),
withoutHealthcheck())
cID, err := d.builder.create(ctx, runConfig)
if err != nil {
return err
}
if err := d.builder.containerManager.Run(ctx, cID, d.builder.Stdout, d.builder.Stderr); err != nil {
if err, ok := err.(*statusCodeError); ok {
// TODO: change error type, because jsonmessage.JSONError assumes HTTP
msg := fmt.Sprintf(
"The command '%s' returned a non-zero code: %d",
strings.Join(runConfig.Cmd, " "), err.StatusCode())
if err.Error() != "" {
msg = fmt.Sprintf("%s: %s", msg, err.Error())
}
return &jsonmessage.JSONError{
Message: msg,
Code: err.StatusCode(),
}
}
return err
}
// Don't persist the argsEscaped value in the committed image. Use the original
// from previous build steps (only CMD and ENTRYPOINT persist this).
if d.state.operatingSystem == "windows" {
runConfigForCacheProbe.ArgsEscaped = stateRunConfig.ArgsEscaped
}
return d.builder.commitContainer(ctx, d.state, cID, runConfigForCacheProbe)
}
// Derive the command to use for probeCache() and to commit in this container.
// Note that we only do this if there are any build-time env vars. Also, we
// use the special argument "|#" at the start of the args array. This will
// avoid conflicts with any RUN command since commands can not
// start with | (vertical bar). The "#" (number of build envs) is there to
// help ensure proper cache matches. We don't want a RUN command
// that starts with "foo=abc" to be considered part of a build-time env var.
//
// remove any unreferenced built-in args from the environment variables.
// These args are transparent so resulting image should be the same regardless
// of the value.
func prependEnvOnCmd(buildArgs *BuildArgs, buildArgVars []string, cmd []string) []string {
tmpBuildEnv := make([]string, 0, len(buildArgVars))
for _, env := range buildArgVars {
key, _, _ := strings.Cut(env, "=")
if buildArgs.IsReferencedOrNotBuiltin(key) {
tmpBuildEnv = append(tmpBuildEnv, env)
}
}
sort.Strings(tmpBuildEnv)
tmpEnv := append([]string{fmt.Sprintf("|%d", len(tmpBuildEnv))}, tmpBuildEnv...)
return append(tmpEnv, cmd...)
}
// CMD foo
//
// Set the default command to run in the container (which may be empty).
// Argument handling is the same as RUN.
func dispatchCmd(ctx context.Context, d dispatchRequest, c *instructions.CmdCommand) error {
runConfig := d.state.runConfig
cmd, argsEscaped := resolveCmdLine(c.ShellDependantCmdLine, runConfig, d.state.operatingSystem, c.Name(), c.String())
// We warn here as Windows shell processing operates differently to Linux.
// Linux: /bin/sh -c "echo hello" world --> hello
// Windows: cmd /s /c "echo hello" world --> hello world
if d.state.operatingSystem == "windows" &&
len(runConfig.Entrypoint) > 0 &&
d.state.runConfig.ArgsEscaped != argsEscaped {
fmt.Fprintf(d.builder.Stderr, " ---> [Warning] Shell-form ENTRYPOINT and exec-form CMD may have unexpected results\n")
}
runConfig.Cmd = cmd
runConfig.ArgsEscaped = argsEscaped
if err := d.builder.commit(ctx, d.state, fmt.Sprintf("CMD %q", cmd)); err != nil {
return err
}
if len(c.ShellDependantCmdLine.CmdLine) != 0 {
d.state.cmdSet = true
}
return 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 dispatchHealthcheck(ctx context.Context, 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 = c.Health
return d.builder.commit(ctx, d.state, fmt.Sprintf("HEALTHCHECK %q", runConfig.Healthcheck))
}
// 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.
func dispatchEntrypoint(ctx context.Context, d dispatchRequest, c *instructions.EntrypointCommand) error {
runConfig := d.state.runConfig
cmd, argsEscaped := resolveCmdLine(c.ShellDependantCmdLine, runConfig, d.state.operatingSystem, c.Name(), c.String())
// This warning is a little more complex than in dispatchCmd(), as the Windows base images (similar
// universally to almost every Linux image out there) have a single .Cmd field populated so that
// `docker run --rm image` starts the default shell which would typically be sh on Linux,
// or cmd on Windows. The catch to this is that if a dockerfile had `CMD ["c:\\windows\\system32\\cmd.exe"]`,
// we wouldn't be able to tell the difference. However, that would be highly unlikely, and besides, this
// is only trying to give a helpful warning of possibly unexpected results.
if d.state.operatingSystem == "windows" &&
d.state.runConfig.ArgsEscaped != argsEscaped &&
((len(runConfig.Cmd) == 1 && strings.ToLower(runConfig.Cmd[0]) != `c:\windows\system32\cmd.exe` && len(runConfig.Shell) == 0) || (len(runConfig.Cmd) > 1)) {
fmt.Fprintf(d.builder.Stderr, " ---> [Warning] Shell-form CMD and exec-form ENTRYPOINT may have unexpected results\n")
}
runConfig.Entrypoint = cmd
runConfig.ArgsEscaped = argsEscaped
if !d.state.cmdSet {
runConfig.Cmd = nil
}
return d.builder.commit(ctx, d.state, fmt.Sprintf("ENTRYPOINT %q", runConfig.Entrypoint))
}
// EXPOSE 6667/tcp 7000/tcp
//
// Expose ports for links and port mappings. This all ends up in
// req.runConfig.ExposedPorts for runconfig.
func dispatchExpose(ctx context.Context, d dispatchRequest, c *instructions.ExposeCommand, envs shell.EnvGetter) 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
ps, _, err := nat.ParsePortSpecs(ports)
if err != nil {
return err
}
if d.state.runConfig.ExposedPorts == nil {
d.state.runConfig.ExposedPorts = make(nat.PortSet)
}
for p := range ps {
d.state.runConfig.ExposedPorts[p] = struct{}{}
}
return d.builder.commit(ctx, d.state, "EXPOSE "+strings.Join(c.Ports, " "))
}
// USER foo
//
// Set the user to 'foo' for future commands and when running the
// ENTRYPOINT/CMD at container run time.
func dispatchUser(ctx context.Context, d dispatchRequest, c *instructions.UserCommand) error {
d.state.runConfig.User = c.User
return d.builder.commit(ctx, d.state, fmt.Sprintf("USER %v", c.User))
}
// VOLUME /foo
//
// Expose the volume /foo for use. Will also accept the JSON array form.
func dispatchVolume(ctx context.Context, d dispatchRequest, c *instructions.VolumeCommand) error {
if d.state.runConfig.Volumes == nil {
d.state.runConfig.Volumes = map[string]struct{}{}
}
for _, v := range c.Volumes {
if v == "" {
return errors.New("VOLUME specified can not be an empty string")
}
d.state.runConfig.Volumes[v] = struct{}{}
}
return d.builder.commit(ctx, d.state, fmt.Sprintf("VOLUME %v", c.Volumes))
}
// STOPSIGNAL signal
//
// Set the signal that will be used to kill the container.
func dispatchStopSignal(ctx context.Context, d dispatchRequest, c *instructions.StopSignalCommand) error {
_, err := signal.ParseSignal(c.Signal)
if err != nil {
return errdefs.InvalidParameter(err)
}
d.state.runConfig.StopSignal = c.Signal
return d.builder.commit(ctx, d.state, fmt.Sprintf("STOPSIGNAL %v", c.Signal))
}
// 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.
func dispatchArg(ctx context.Context, d dispatchRequest, c *instructions.ArgCommand) error {
var commitStr strings.Builder
commitStr.WriteString("ARG ")
for i, arg := range c.Args {
if i > 0 {
commitStr.WriteString(" ")
}
commitStr.WriteString(arg.Key)
if arg.Value != nil {
commitStr.WriteString("=")
commitStr.WriteString(*arg.Value)
}
d.state.buildArgs.AddArg(arg.Key, arg.Value)
}
return d.builder.commit(ctx, d.state, commitStr.String())
}
// SHELL powershell -command
//
// Set the non-default shell to use.
func dispatchShell(ctx context.Context, d dispatchRequest, c *instructions.ShellCommand) error {
d.state.runConfig.Shell = c.Shell
return d.builder.commit(ctx, d.state, fmt.Sprintf("SHELL %v", d.state.runConfig.Shell))
}

View File

@@ -0,0 +1,632 @@
package dockerfile
import (
"bytes"
"context"
"fmt"
"runtime"
"strings"
"testing"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/build"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/image"
"github.com/docker/docker/oci"
"github.com/docker/go-connections/nat"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/buildkit/frontend/dockerfile/parser"
"github.com/moby/buildkit/frontend/dockerfile/shell"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func newBuilderWithMockBackend(t *testing.T) *Builder {
t.Helper()
mockBackend := &MockBackend{}
opts := &build.ImageBuildOptions{}
ctx := context.Background()
imageProber, err := newImageProber(ctx, mockBackend, nil, false)
assert.NilError(t, err, "Could not create image prober")
b := &Builder{
options: opts,
docker: mockBackend,
Stdout: new(bytes.Buffer),
disableCommit: true,
imageSources: newImageSources(builderOptions{
Options: opts,
Backend: mockBackend,
}),
imageProber: imageProber,
containerManager: newContainerManager(mockBackend),
}
return b
}
func TestEnv2Variables(t *testing.T) {
b := newBuilderWithMockBackend(t)
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(context.TODO(), sb, envCommand)
assert.NilError(t, err)
expected := []string{
"var1=val1",
"var2=val2",
}
assert.Check(t, is.DeepEqual(sb.state.runConfig.Env, expected))
}
func TestEnvValueWithExistingRunConfigEnv(t *testing.T) {
b := newBuilderWithMockBackend(t)
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(context.TODO(), sb, envCommand)
assert.NilError(t, err)
expected := []string{
"var1=val1",
"var2=fromenv",
}
assert.Check(t, is.DeepEqual(sb.state.runConfig.Env, expected))
}
func TestMaintainer(t *testing.T) {
maintainerEntry := "Some Maintainer <maintainer@example.com>"
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '\\', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.MaintainerCommand{Maintainer: maintainerEntry}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.maintainer, maintainerEntry))
}
func TestLabel(t *testing.T) {
labelName := "label"
labelValue := "value"
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '\\', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.LabelCommand{
Labels: instructions.KeyValuePairs{
instructions.KeyValuePair{Key: labelName, Value: labelValue},
},
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Assert(t, is.Contains(sb.state.runConfig.Labels, labelName))
assert.Check(t, is.Equal(sb.state.runConfig.Labels[labelName], labelValue))
}
func TestFromScratch(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '\\', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.Stage{
BaseName: "scratch",
}
err := initializeStage(context.TODO(), sb, cmd)
if runtime.GOOS == "windows" {
assert.Check(t, is.Error(err, "Windows does not support FROM scratch"))
return
}
assert.NilError(t, err)
assert.Check(t, sb.state.hasFromImage())
assert.Check(t, is.Equal(sb.state.imageID, ""))
// TODO(thaJeztah): use github.com/moby/buildkit/util/system.DefaultPathEnv() once https://github.com/moby/buildkit/pull/3158 is resolved.
expected := []string{"PATH=" + oci.DefaultPathEnv(runtime.GOOS)}
assert.Check(t, is.DeepEqual(sb.state.runConfig.Env, expected))
}
func TestFromWithArg(t *testing.T) {
tag, expected := ":sometag", "expectedthisid"
getImage := func(name string) (builder.Image, builder.ROLayer, error) {
assert.Check(t, is.Equal(name, "alpine"+tag))
return &mockImage{id: "expectedthisid"}, nil, nil
}
b := newBuilderWithMockBackend(t)
b.docker.(*MockBackend).getImageFunc = getImage
args := NewBuildArgs(make(map[string]*string))
val := "sometag"
metaArg := instructions.ArgCommand{Args: []instructions.KeyValuePairOptional{{
Key: "THETAG",
Value: &val,
}}}
cmd := &instructions.Stage{
BaseName: "alpine:${THETAG}",
}
err := processMetaArg(metaArg, shell.NewLex('\\'), args)
sb := newDispatchRequest(b, '\\', nil, args, newStagesBuildResults())
assert.NilError(t, err)
err = initializeStage(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.imageID, expected))
assert.Check(t, is.Equal(sb.state.baseImage.ImageID(), expected))
assert.Check(t, is.Len(sb.state.buildArgs.GetAllAllowed(), 0))
assert.Check(t, is.Len(sb.state.buildArgs.GetAllMeta(), 1))
}
func TestFromWithArgButBuildArgsNotGiven(t *testing.T) {
b := newBuilderWithMockBackend(t)
args := NewBuildArgs(make(map[string]*string))
metaArg := instructions.ArgCommand{}
cmd := &instructions.Stage{
BaseName: "${THETAG}",
}
err := processMetaArg(metaArg, shell.NewLex('\\'), args)
sb := newDispatchRequest(b, '\\', nil, args, newStagesBuildResults())
assert.NilError(t, err)
err = initializeStage(context.TODO(), sb, cmd)
assert.Error(t, err, "base name (${THETAG}) should not be blank")
}
func TestFromWithUndefinedArg(t *testing.T) {
tag, expected := "sometag", "expectedthisid"
getImage := func(name string) (builder.Image, builder.ROLayer, error) {
assert.Check(t, is.Equal(name, "alpine"))
return &mockImage{id: "expectedthisid"}, nil, nil
}
b := newBuilderWithMockBackend(t)
b.docker.(*MockBackend).getImageFunc = getImage
sb := newDispatchRequest(b, '\\', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
b.options.BuildArgs = map[string]*string{"THETAG": &tag}
cmd := &instructions.Stage{
BaseName: "alpine${THETAG}",
}
err := initializeStage(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.imageID, expected))
}
func TestFromMultiStageWithNamedStage(t *testing.T) {
b := newBuilderWithMockBackend(t)
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(context.TODO(), firstSB, firstFrom)
assert.NilError(t, err)
assert.Check(t, firstSB.state.hasFromImage())
previousResults.indexed["base"] = firstSB.state.runConfig
previousResults.flat = append(previousResults.flat, firstSB.state.runConfig)
err = initializeStage(context.TODO(), secondSB, secondFrom)
assert.NilError(t, err)
assert.Check(t, secondSB.state.hasFromImage())
}
func TestOnbuild(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '\\', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.OnbuildCommand{
Expression: "ADD . /app/src",
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.runConfig.OnBuild[0], "ADD . /app/src"))
}
func TestWorkdir(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
sb.state.baseImage = &mockImage{}
workingDir := "/app"
if runtime.GOOS == "windows" {
workingDir = "C:\\app"
}
cmd := &instructions.WorkdirCommand{
Path: workingDir,
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.runConfig.WorkingDir, workingDir))
}
func TestCmd(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
sb.state.baseImage = &mockImage{}
err := dispatch(context.TODO(), sb, &instructions.CmdCommand{
ShellDependantCmdLine: instructions.ShellDependantCmdLine{
CmdLine: []string{"./executable"},
PrependShell: true,
},
})
assert.NilError(t, err)
var expectedCommand []string
if runtime.GOOS == "windows" {
expectedCommand = []string{"cmd", "/S", "/C", "./executable"}
} else {
expectedCommand = []string{"/bin/sh", "-c", "./executable"}
}
assert.Check(t, is.DeepEqual([]string(sb.state.runConfig.Cmd), expectedCommand))
assert.Check(t, sb.state.cmdSet)
}
func TestHealthcheckNone(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.HealthCheckCommand{
Health: &container.HealthConfig{
Test: []string{"NONE"},
},
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Assert(t, sb.state.runConfig.Healthcheck != nil)
assert.Check(t, is.DeepEqual(sb.state.runConfig.Healthcheck.Test, []string{"NONE"}))
}
func TestHealthcheckCmd(t *testing.T) {
b := newBuilderWithMockBackend(t)
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(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Assert(t, sb.state.runConfig.Healthcheck != nil)
assert.Check(t, is.DeepEqual(sb.state.runConfig.Healthcheck.Test, expectedTest))
}
func TestEntrypoint(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
sb.state.baseImage = &mockImage{}
err := dispatch(context.TODO(), sb, &instructions.EntrypointCommand{
ShellDependantCmdLine: instructions.ShellDependantCmdLine{
CmdLine: []string{"/usr/sbin/nginx"},
PrependShell: true,
},
})
assert.NilError(t, err)
var expectedEntrypoint []string
if runtime.GOOS == "windows" {
expectedEntrypoint = []string{"cmd", "/S", "/C", "/usr/sbin/nginx"}
} else {
expectedEntrypoint = []string{"/bin/sh", "-c", "/usr/sbin/nginx"}
}
assert.Check(t, is.DeepEqual([]string(sb.state.runConfig.Entrypoint), expectedEntrypoint))
}
func TestExpose(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
exposedPort := "80"
cmd := &instructions.ExposeCommand{
Ports: []string{exposedPort},
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Assert(t, sb.state.runConfig.ExposedPorts != nil)
assert.Assert(t, is.Len(sb.state.runConfig.ExposedPorts, 1))
portsMapping, err := nat.ParsePortSpec(exposedPort)
assert.NilError(t, err)
assert.Check(t, is.Contains(sb.state.runConfig.ExposedPorts, portsMapping[0].Port))
}
func TestUser(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
cmd := &instructions.UserCommand{
User: "test",
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.runConfig.User, "test"))
}
func TestVolume(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
exposedVolume := "/foo"
cmd := &instructions.VolumeCommand{
Volumes: []string{exposedVolume},
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Assert(t, sb.state.runConfig.Volumes != nil)
assert.Check(t, is.Len(sb.state.runConfig.Volumes, 1))
assert.Check(t, is.Contains(sb.state.runConfig.Volumes, exposedVolume))
}
func TestStopSignal(t *testing.T) {
if runtime.GOOS == "windows" {
t.Skip("Windows does not support stopsignal")
return
}
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
sb.state.baseImage = &mockImage{}
const signal = "SIGKILL"
cmd := &instructions.StopSignalCommand{
Signal: signal,
}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
assert.Check(t, is.Equal(sb.state.runConfig.StopSignal, signal))
}
func TestArg(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
argName := "foo"
argVal := "bar"
cmd := &instructions.ArgCommand{Args: []instructions.KeyValuePairOptional{{Key: argName, Value: &argVal}}}
err := dispatch(context.TODO(), sb, cmd)
assert.NilError(t, err)
expected := map[string]string{argName: argVal}
assert.Check(t, is.DeepEqual(sb.state.buildArgs.GetAllAllowed(), expected))
}
func TestShell(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
shellCmd := []string{"powershell"}
err := dispatch(context.TODO(), sb, &instructions.ShellCommand{
Shell: shellCmd,
})
assert.NilError(t, err)
expected := shellCmd
assert.Check(t, is.DeepEqual([]string(sb.state.runConfig.Shell), expected))
}
func TestPrependEnvOnCmd(t *testing.T) {
buildArgs := NewBuildArgs(nil)
buildArgs.AddArg("NO_PROXY", nil)
args := []string{"sorted=nope", "args=not", "http_proxy=foo", "NO_PROXY=YA"}
cmd := []string{"foo", "bar"}
cmdWithEnv := prependEnvOnCmd(buildArgs, args, cmd)
expected := []string{
"|3", "NO_PROXY=YA", "args=not", "sorted=nope", "foo", "bar",
}
assert.Check(t, is.DeepEqual(cmdWithEnv, expected))
}
func TestRunWithBuildArgs(t *testing.T) {
b := newBuilderWithMockBackend(t)
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 := []string{"cmd", "in", "from", "image"}
var cmdWithShell []string
if runtime.GOOS == "windows" {
cmdWithShell = []string{strings.Join(append(getShell(runConfig, runtime.GOOS), []string{"echo foo"}...), " ")}
} else {
cmdWithShell = append(getShell(runConfig, runtime.GOOS), "echo foo")
}
envVars := []string{"|1", "one=two"}
cachedCmd := append(envVars, cmdWithShell...)
imageCache := &mockImageCache{
getCacheFunc: func(parentID string, cfg *container.Config) (string, error) {
// Check the runConfig.Cmd sent to probeCache()
assert.Check(t, is.DeepEqual([]string(cfg.Cmd), cachedCmd))
assert.Check(t, is.Nil(cfg.Entrypoint))
return "", nil
},
}
mockBackend := b.docker.(*MockBackend)
mockBackend.makeImageCacheFunc = func(_ []string) builder.ImageCache {
return imageCache
}
prober, err := newImageProber(context.TODO(), mockBackend, nil, false)
assert.NilError(t, err, "Could not create image prober")
b.imageProber = prober
mockBackend.getImageFunc = func(_ string) (builder.Image, builder.ROLayer, error) {
return &mockImage{
id: "abcdef",
config: &container.Config{Cmd: origCmd},
}, nil, nil
}
mockBackend.containerCreateFunc = func(config backend.ContainerCreateConfig) (container.CreateResponse, error) {
// Check the runConfig.Cmd sent to create()
assert.Check(t, is.DeepEqual([]string(config.Config.Cmd), cmdWithShell))
assert.Check(t, is.Contains(config.Config.Env, "one=two"))
assert.Check(t, is.DeepEqual([]string(config.Config.Entrypoint), []string{""}))
return container.CreateResponse{ID: "12345"}, nil
}
mockBackend.commitFunc = func(cfg backend.CommitConfig) (image.ID, error) {
// Check the runConfig.Cmd sent to commit()
assert.Check(t, is.DeepEqual([]string(cfg.Config.Cmd), origCmd))
assert.Check(t, is.DeepEqual([]string(cfg.ContainerConfig.Cmd), cachedCmd))
assert.Check(t, is.Nil(cfg.Config.Entrypoint))
return "", nil
}
from := &instructions.Stage{BaseName: "abcdef"}
err = initializeStage(context.TODO(), sb, from)
assert.NilError(t, err)
sb.state.buildArgs.AddArg("one", strPtr("two"))
// This is hugely annoying. On the Windows side, it relies on the
// RunCommand being able to emit String() and Name() (as implemented by
// withNameAndCode). Unfortunately, that is internal, and no way to directly
// set. However, we can fortunately use ParseInstruction in the instructions
// package to parse a fake node which can be used as our instructions.RunCommand
// instead.
node := &parser.Node{
Original: `RUN echo foo`,
Value: "run",
}
runint, err := instructions.ParseInstruction(node)
assert.NilError(t, err)
runinst := runint.(*instructions.RunCommand)
runinst.CmdLine = []string{"echo foo"}
runinst.PrependShell = true
assert.NilError(t, dispatch(context.TODO(), sb, runinst))
// Check that runConfig.Cmd has not been modified by run
assert.Check(t, is.DeepEqual([]string(sb.state.runConfig.Cmd), origCmd))
}
func TestRunIgnoresHealthcheck(t *testing.T) {
b := newBuilderWithMockBackend(t)
args := NewBuildArgs(make(map[string]*string))
sb := newDispatchRequest(b, '`', nil, args, newStagesBuildResults())
b.disableCommit = false
origCmd := []string{"cmd", "in", "from", "image"}
imageCache := &mockImageCache{
getCacheFunc: func(parentID string, cfg *container.Config) (string, error) {
return "", nil
},
}
mockBackend := b.docker.(*MockBackend)
mockBackend.makeImageCacheFunc = func(_ []string) builder.ImageCache {
return imageCache
}
imageProber, err := newImageProber(context.TODO(), mockBackend, nil, false)
assert.NilError(t, err, "Could not create image prober")
b.imageProber = imageProber
mockBackend.getImageFunc = func(_ string) (builder.Image, builder.ROLayer, error) {
return &mockImage{
id: "abcdef",
config: &container.Config{Cmd: origCmd},
}, nil, nil
}
mockBackend.containerCreateFunc = func(config backend.ContainerCreateConfig) (container.CreateResponse, error) {
return container.CreateResponse{ID: "12345"}, nil
}
mockBackend.commitFunc = func(cfg backend.CommitConfig) (image.ID, error) {
return "", nil
}
from := &instructions.Stage{BaseName: "abcdef"}
err = initializeStage(context.TODO(), sb, from)
assert.NilError(t, err)
expectedTest := []string{"CMD-SHELL", "curl -f http://localhost/ || exit 1"}
healthint, err := instructions.ParseInstruction(&parser.Node{
Original: `HEALTHCHECK CMD curl -f http://localhost/ || exit 1`,
Value: "healthcheck",
Next: &parser.Node{
Value: "cmd",
Next: &parser.Node{
Value: `curl -f http://localhost/ || exit 1`,
},
},
})
assert.NilError(t, err)
cmd := healthint.(*instructions.HealthCheckCommand)
assert.NilError(t, dispatch(context.TODO(), sb, cmd))
assert.Assert(t, sb.state.runConfig.Healthcheck != nil)
mockBackend.containerCreateFunc = func(config backend.ContainerCreateConfig) (container.CreateResponse, error) {
// Check the Healthcheck is disabled.
assert.Check(t, is.DeepEqual(config.Config.Healthcheck.Test, []string{"NONE"}))
return container.CreateResponse{ID: "123456"}, nil
}
sb.state.buildArgs.AddArg("one", strPtr("two"))
runint, err := instructions.ParseInstruction(&parser.Node{Original: `RUN echo foo`, Value: "run"})
assert.NilError(t, err)
run := runint.(*instructions.RunCommand)
run.PrependShell = true
assert.NilError(t, dispatch(context.TODO(), sb, run))
assert.Check(t, is.DeepEqual(sb.state.runConfig.Healthcheck.Test, expectedTest))
}
func TestDispatchUnsupportedOptions(t *testing.T) {
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', nil, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
sb.state.baseImage = &mockImage{}
sb.state.operatingSystem = runtime.GOOS
t.Run("ADD with chmod", func(t *testing.T) {
cmd := &instructions.AddCommand{
SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"."},
DestPath: ".",
},
Chmod: "0655",
}
err := dispatch(context.TODO(), sb, cmd)
assert.Error(t, err, "the --chmod option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled")
})
t.Run("COPY with chmod", func(t *testing.T) {
cmd := &instructions.CopyCommand{
SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"."},
DestPath: ".",
},
Chmod: "0655",
}
err := dispatch(context.TODO(), sb, cmd)
assert.Error(t, err, "the --chmod option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled")
})
t.Run("RUN with unsupported options", func(t *testing.T) {
runint, err := instructions.ParseInstruction(&parser.Node{Original: `RUN echo foo`, Value: "run"})
assert.NilError(t, err)
cmd := runint.(*instructions.RunCommand)
// classic builder "RUN" currently doesn't support any flags, but testing
// both "known" flags and "bogus" flags for completeness, and in case
// one or more of these flags will be supported in future
for _, f := range []string{"mount", "network", "security", "any-flag"} {
cmd.FlagsUsed = []string{f}
err := dispatch(context.TODO(), sb, cmd)
assert.Error(t, err, fmt.Sprintf("the --%s option requires BuildKit. Refer to https://docs.docker.com/go/buildkit/ to learn how to build images with BuildKit enabled", f))
}
})
}

View File

@@ -0,0 +1,36 @@
//go:build !windows
package dockerfile
import (
"errors"
"os"
"path/filepath"
"github.com/docker/docker/api/types/container"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
)
// normalizeWorkdir normalizes a user requested working directory in a
// platform semantically consistent way.
func normalizeWorkdir(_ string, current string, requested string) (string, error) {
if requested == "" {
return "", errors.New("cannot normalize nothing")
}
current = filepath.FromSlash(current)
requested = filepath.FromSlash(requested)
if !filepath.IsAbs(requested) {
return filepath.Join(string(os.PathSeparator), current, requested), nil
}
return filepath.Clean(requested), nil
}
// resolveCmdLine takes a command line arg set and optionally prepends a platform-specific
// shell in front of it.
func resolveCmdLine(cmd instructions.ShellDependantCmdLine, runConfig *container.Config, os, _, _ string) ([]string, bool) {
result := cmd.CmdLine
if cmd.PrependShell && result != nil {
result = append(getShell(runConfig, os), result...)
}
return result, false
}

View File

@@ -0,0 +1,34 @@
//go:build !windows
package dockerfile
import (
"runtime"
"testing"
)
func TestNormalizeWorkdir(t *testing.T) {
testCases := []struct{ current, requested, expected, expectedError string }{
{``, ``, ``, `cannot normalize nothing`},
{``, `foo`, `/foo`, ``},
{``, `/foo`, `/foo`, ``},
{`/foo`, `bar`, `/foo/bar`, ``},
{`/foo`, `/bar`, `/bar`, ``},
}
for _, test := range testCases {
normalized, err := normalizeWorkdir(runtime.GOOS, test.current, test.requested)
if test.expectedError != "" && err == nil {
t.Fatalf("NormalizeWorkdir should return an error %s, got nil", test.expectedError)
}
if test.expectedError != "" && err.Error() != test.expectedError {
t.Fatalf("NormalizeWorkdir returned wrong error. Expected %s, got %s", test.expectedError, err.Error())
}
if normalized != test.expected {
t.Fatalf("NormalizeWorkdir error. Expected %s for current %s and requested %s, got %s", test.expected, test.current, test.requested, normalized)
}
}
}

View File

@@ -0,0 +1,139 @@
package dockerfile
import (
"errors"
"fmt"
"os"
"path"
"path/filepath"
"strings"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/internal/lazyregexp"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
)
var pattern = lazyregexp.New(`^[a-zA-Z]:\.$`)
// normalizeWorkdir normalizes a user requested working directory in a
// platform semantically consistent way.
func normalizeWorkdir(platform string, current string, requested string) (string, error) {
if platform == "" {
platform = "windows"
}
if platform == "windows" {
return normalizeWorkdirWindows(current, requested)
}
return normalizeWorkdirUnix(current, requested)
}
// normalizeWorkdirUnix normalizes a user requested working directory in a
// platform semantically consistent way.
func normalizeWorkdirUnix(current string, requested string) (string, error) {
if requested == "" {
return "", errors.New("cannot normalize nothing")
}
current = strings.ReplaceAll(current, string(os.PathSeparator), "/")
requested = strings.ReplaceAll(requested, string(os.PathSeparator), "/")
if !path.IsAbs(requested) {
return path.Join(`/`, current, requested), nil
}
return requested, nil
}
// normalizeWorkdirWindows normalizes a user requested working directory in a
// platform semantically consistent way.
func normalizeWorkdirWindows(current string, requested string) (string, error) {
if requested == "" {
return "", errors.New("cannot normalize nothing")
}
// `filepath.Clean` will replace "" with "." so skip in that case
if current != "" {
current = filepath.Clean(current)
}
if requested != "" {
requested = filepath.Clean(requested)
}
// If either current or requested in Windows is:
// C:
// C:.
// then an error will be thrown as the definition for the above
// refers to `current directory on drive C:`
// Since filepath.Clean() will automatically normalize the above
// to `C:.`, we only need to check the last format
if pattern.MatchString(current) {
return "", fmt.Errorf("%s is not a directory. If you are specifying a drive letter, please add a trailing '\\'", current)
}
if pattern.MatchString(requested) {
return "", fmt.Errorf("%s is not a directory. If you are specifying a drive letter, please add a trailing '\\'", requested)
}
// Target semantics is C:\somefolder, specifically in the format:
// UPPERCASEDriveLetter-Colon-Backslash-FolderName. We are already
// guaranteed that `current`, if set, is consistent. This allows us to
// cope correctly with any of the following in a Dockerfile:
// WORKDIR a --> C:\a
// WORKDIR c:\\foo --> C:\foo
// WORKDIR \\foo --> C:\foo
// WORKDIR /foo --> C:\foo
// WORKDIR c:\\foo \ WORKDIR bar --> C:\foo --> C:\foo\bar
// WORKDIR C:/foo \ WORKDIR bar --> C:\foo --> C:\foo\bar
// WORKDIR C:/foo \ WORKDIR \\bar --> C:\foo --> C:\bar
// WORKDIR /foo \ WORKDIR c:/bar --> C:\foo --> C:\bar
if len(current) == 0 || isAbs(requested) {
if (requested[0] == os.PathSeparator) ||
(len(requested) > 1 && string(requested[1]) != ":") ||
(len(requested) == 1) {
requested = filepath.Join(`C:\`, requested)
}
} else {
requested = filepath.Join(current, requested)
}
// Upper-case drive letter
return (strings.ToUpper(string(requested[0])) + requested[1:]), nil
}
// resolveCmdLine takes a command line arg set and optionally prepends a platform-specific
// shell in front of it. It returns either an array of arguments and an indication that
// the arguments are not yet escaped; Or, an array containing a single command line element
// along with an indication that the arguments are escaped so the runtime shouldn't escape.
//
// A better solution could be made, but it would be exceptionally invasive throughout
// many parts of the daemon which are coded assuming Linux args array only only, not taking
// account of Windows-natural command line semantics and it's argv handling. Put another way,
// while what is here is good-enough, it could be improved, but would be highly invasive.
//
// The commands when this function is called are RUN, ENTRYPOINT and CMD.
func resolveCmdLine(cmd instructions.ShellDependantCmdLine, runConfig *container.Config, os, command, original string) ([]string, bool) {
// Make sure we return an empty array if there is no cmd.CmdLine
if len(cmd.CmdLine) == 0 {
return []string{}, runConfig.ArgsEscaped
}
if os == "windows" { // ie WCOW
if cmd.PrependShell {
// WCOW shell-form. Return a single-element array containing the original command line prepended with the shell.
// Also indicate that it has not been escaped (so will be passed through directly to HCS). Note that
// we go back to the original un-parsed command line in the dockerfile line, strip off both the command part of
// it (RUN/ENTRYPOINT/CMD), and also strip any leading white space. IOW, we deliberately ignore any prior parsing
// so as to ensure it is treated exactly as a command line. For those interested, `RUN mkdir "c:/foo"` is a particularly
// good example of why this is necessary if you fancy debugging how cmd.exe and its builtin mkdir works. (Windows
// doesn't have a mkdir.exe, and I'm guessing cmd.exe has some very long unavoidable and unchangeable historical
// design decisions over how both its built-in echo and mkdir are coded. Probably more too.)
original = original[len(command):] // Strip off the command
original = strings.TrimLeft(original, " \t\v\n") // Strip of leading whitespace
return []string{strings.Join(getShell(runConfig, os), " ") + " " + original}, true
}
// WCOW JSON/"exec" form.
return cmd.CmdLine, false
}
// LCOW - use args as an array, same as LCOL.
if cmd.PrependShell && cmd.CmdLine != nil {
return append(getShell(runConfig, os), cmd.CmdLine...), false
}
return cmd.CmdLine, false
}

View File

@@ -0,0 +1,46 @@
//go:build windows
package dockerfile
import "testing"
func TestNormalizeWorkdir(t *testing.T) {
tests := []struct{ platform, current, requested, expected, etext string }{
{"windows", ``, ``, ``, `cannot normalize nothing`},
{"windows", ``, `C:`, ``, `C:. is not a directory. If you are specifying a drive letter, please add a trailing '\'`},
{"windows", ``, `C:.`, ``, `C:. is not a directory. If you are specifying a drive letter, please add a trailing '\'`},
{"windows", `c:`, `\a`, ``, `c:. is not a directory. If you are specifying a drive letter, please add a trailing '\'`},
{"windows", `c:.`, `\a`, ``, `c:. is not a directory. If you are specifying a drive letter, please add a trailing '\'`},
{"windows", ``, `a`, `C:\a`, ``},
{"windows", ``, `c:\foo`, `C:\foo`, ``},
{"windows", ``, `c:\\foo`, `C:\foo`, ``},
{"windows", ``, `\foo`, `C:\foo`, ``},
{"windows", ``, `\\foo`, `C:\foo`, ``},
{"windows", ``, `/foo`, `C:\foo`, ``},
{"windows", ``, `C:/foo`, `C:\foo`, ``},
{"windows", `C:\foo`, `bar`, `C:\foo\bar`, ``},
{"windows", `C:\foo`, `/bar`, `C:\bar`, ``},
{"windows", `C:\foo`, `\bar`, `C:\bar`, ``},
{"linux", ``, ``, ``, `cannot normalize nothing`},
{"linux", ``, `foo`, `/foo`, ``},
{"linux", ``, `/foo`, `/foo`, ``},
{"linux", `/foo`, `bar`, `/foo/bar`, ``},
{"linux", `/foo`, `/bar`, `/bar`, ``},
{"linux", `\a`, `b\c`, `/a/b/c`, ``},
}
for _, i := range tests {
r, e := normalizeWorkdir(i.platform, i.current, i.requested)
if i.etext != "" && e == nil {
t.Fatalf("TestNormalizeWorkingDir Expected error %s for '%s' '%s', got no error", i.etext, i.current, i.requested)
}
if i.etext != "" && e.Error() != i.etext {
t.Fatalf("TestNormalizeWorkingDir Expected error %s for '%s' '%s', got %s", i.etext, i.current, i.requested, e.Error())
}
if r != i.expected {
t.Fatalf("TestNormalizeWorkingDir Expected '%s' for '%s' '%s', got '%s'", i.expected, i.current, i.requested, r)
}
}
}

View File

@@ -0,0 +1,249 @@
// Package dockerfile is the evaluation step in the Dockerfile parse/evaluate pipeline.
//
// It incorporates a dispatch table based on the parser.Node values (see the
// parser package for more information) that are yielded from the parser itself.
// Calling newBuilder with the BuildOpts struct can be used to customize the
// experience for execution purposes only. Parsing is controlled in the parser
// package, and this division of responsibility should be respected.
//
// Please see the jump table targets for the actual invocations, most of which
// will call out to the functions in internals.go to deal with their tasks.
//
// ONBUILD is a special case, which is covered in the onbuild() func in
// dispatchers.go.
//
// The evaluator uses the concept of "steps", which are usually each processable
// line in the Dockerfile. Each step is numbered and certain actions are taken
// before and after each step, such as creating an image ID and removing temporary
// containers and images. Note that ONBUILD creates a kinda-sorta "sub run" which
// includes its own set of steps (usually only one of them).
package dockerfile
import (
"context"
"reflect"
"strconv"
"strings"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/errdefs"
"github.com/docker/docker/image"
"github.com/docker/docker/oci"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/buildkit/frontend/dockerfile/shell"
"github.com/pkg/errors"
)
func dispatch(ctx context.Context, d dispatchRequest, cmd instructions.Command) (retErr error) {
if c, ok := cmd.(instructions.PlatformSpecific); ok {
err := c.CheckPlatform(d.state.operatingSystem)
if err != nil {
return errdefs.InvalidParameter(err)
}
}
runConfigEnv := d.state.runConfig.Env
envs := shell.EnvsFromSlice(append(runConfigEnv, d.state.buildArgs.FilterAllowed(runConfigEnv)...))
if ex, ok := cmd.(instructions.SupportsSingleWordExpansion); ok {
err := ex.Expand(func(word string) (string, error) {
newword, _, err := d.shlex.ProcessWord(word, envs)
return newword, err
})
if err != nil {
return errdefs.InvalidParameter(err)
}
}
defer func() {
if d.builder.options.ForceRemove {
d.builder.containerManager.RemoveAll(d.builder.Stdout)
return
}
if d.builder.options.Remove && retErr == nil {
d.builder.containerManager.RemoveAll(d.builder.Stdout)
return
}
}()
switch c := cmd.(type) {
case *instructions.EnvCommand:
return dispatchEnv(ctx, d, c)
case *instructions.MaintainerCommand:
return dispatchMaintainer(ctx, d, c)
case *instructions.LabelCommand:
return dispatchLabel(ctx, d, c)
case *instructions.AddCommand:
return dispatchAdd(ctx, d, c)
case *instructions.CopyCommand:
return dispatchCopy(ctx, d, c)
case *instructions.OnbuildCommand:
return dispatchOnbuild(ctx, d, c)
case *instructions.WorkdirCommand:
return dispatchWorkdir(ctx, d, c)
case *instructions.RunCommand:
return dispatchRun(ctx, d, c)
case *instructions.CmdCommand:
return dispatchCmd(ctx, d, c)
case *instructions.HealthCheckCommand:
return dispatchHealthcheck(ctx, d, c)
case *instructions.EntrypointCommand:
return dispatchEntrypoint(ctx, d, c)
case *instructions.ExposeCommand:
return dispatchExpose(ctx, d, c, envs)
case *instructions.UserCommand:
return dispatchUser(ctx, d, c)
case *instructions.VolumeCommand:
return dispatchVolume(ctx, d, c)
case *instructions.StopSignalCommand:
return dispatchStopSignal(ctx, d, c)
case *instructions.ArgCommand:
return dispatchArg(ctx, d, c)
case *instructions.ShellCommand:
return dispatchShell(ctx, d, c)
}
return errors.Errorf("unsupported command type: %v", reflect.TypeOf(cmd))
}
// 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
operatingSystem string
}
func newDispatchState(baseArgs *BuildArgs) *dispatchState {
args := baseArgs.Clone()
args.ResetAllowed()
return &dispatchState{runConfig: &container.Config{}, buildArgs: args}
}
type stagesBuildResults struct {
flat []*container.Config
indexed map[string]*container.Config
}
func newStagesBuildResults() *stagesBuildResults {
return &stagesBuildResults{
indexed: make(map[string]*container.Config),
}
}
func (r *stagesBuildResults) getByName(name string) (*container.Config, bool) {
c, ok := r.indexed[strings.ToLower(name)]
return c, ok
}
func (r *stagesBuildResults) validateIndex(i int) error {
if i == len(r.flat) {
return errors.New("refers to current build stage")
}
if i < 0 || i > len(r.flat) {
return errors.New("index out of bounds")
}
return nil
}
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 {
return nil, nil
}
if err := r.validateIndex(int(ix)); err != nil {
return nil, err
}
return r.flat[ix], nil
}
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
}
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
}
r.flat = append(r.flat, config)
return nil
}
func commitStage(state *dispatchState, stages *stagesBuildResults) error {
return stages.commitStage(state.stageName, state.runConfig)
}
type dispatchRequest struct {
state *dispatchState
shlex *shell.Lex
builder *Builder
source builder.Source
stages *stagesBuildResults
}
func newDispatchRequest(builder *Builder, escapeToken rune, source builder.Source, buildArgs *BuildArgs, stages *stagesBuildResults) dispatchRequest {
return dispatchRequest{
state: newDispatchState(buildArgs),
shlex: shell.NewLex(escapeToken),
builder: builder,
source: source,
stages: stages,
}
}
func (s *dispatchState) updateRunConfig() {
s.runConfig.Image = s.imageID
}
// hasFromImage returns true if the builder has processed a `FROM <image>` line
func (s *dispatchState) hasFromImage() bool {
return s.imageID != "" || (s.baseImage != nil && s.baseImage.ImageID() == "")
}
func (s *dispatchState) beginStage(stageName string, img builder.Image) error {
s.stageName = stageName
s.imageID = img.ImageID()
s.operatingSystem = img.OperatingSystem()
if err := image.CheckOS(s.operatingSystem); err != nil {
return err
}
if img.RunConfig() != nil {
// copy avoids referencing the same instance when 2 stages have the same base
s.runConfig = copyRunConfig(img.RunConfig())
} else {
s.runConfig = &container.Config{}
}
s.baseImage = img
s.setDefaultPath()
s.runConfig.OpenStdin = false
s.runConfig.StdinOnce = false
return nil
}
// Add the default PATH to runConfig.ENV if one exists for the operating system and there
// is no PATH set. Note that Windows containers on Windows won't have one as it's set by HCS
func (s *dispatchState) setDefaultPath() {
// TODO(thaJeztah): use github.com/moby/buildkit/util/system.DefaultPathEnv() once https://github.com/moby/buildkit/pull/3158 is resolved.
defaultPath := oci.DefaultPathEnv(s.operatingSystem)
if defaultPath == "" {
return
}
envMap := convertKVStringsToMap(s.runConfig.Env)
if _, ok := envMap["PATH"]; !ok {
s.runConfig.Env = append(s.runConfig.Env, "PATH="+defaultPath)
}
}

View File

@@ -0,0 +1,137 @@
package dockerfile
import (
"context"
"os"
"runtime"
"testing"
"github.com/docker/docker/daemon/builder/remotecontext"
"github.com/moby/buildkit/frontend/dockerfile/instructions"
"github.com/moby/go-archive"
"github.com/moby/sys/reexec"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
"gotest.tools/v3/skip"
)
type dispatchTestCase struct {
name, expectedError string
cmd instructions.Command
files map[string]string
}
func TestMain(m *testing.M) {
if reexec.Init() {
return
}
os.Exit(m.Run())
}
func TestDispatch(t *testing.T) {
if runtime.GOOS != "windows" {
skip.If(t, os.Getuid() != 0, "skipping test that requires root")
}
testCases := []dispatchTestCase{
{
name: "ADD multiple files to file",
cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"file1.txt", "file2.txt"},
DestPath: "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",
cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"file*.txt"},
DestPath: "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",
cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"file1.txt", "file2.txt"},
DestPath: "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",
cmd: &instructions.AddCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"test file1.txt", "test file2.txt"},
DestPath: "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",
cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"test file1.txt", "test file2.txt"},
DestPath: "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",
cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"file*.txt"},
DestPath: "/tmp/",
}},
expectedError: "COPY failed: no source files were specified",
files: nil,
},
{
name: "COPY url",
cmd: &instructions.CopyCommand{SourcesAndDest: instructions.SourcesAndDest{
SourcePaths: []string{"https://example.com/index.html"},
DestPath: "/",
}},
expectedError: "source can't be a URL for COPY",
files: nil,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
contextDir := t.TempDir()
for filename, content := range tc.files {
createTestTempFile(t, contextDir, filename, content, 0o777)
}
tarStream, err := archive.Tar(contextDir, archive.Uncompressed)
if err != nil {
t.Fatalf("Error when creating tar stream: %s", err)
}
defer func() {
if err = tarStream.Close(); err != nil {
t.Fatalf("Error when closing tar stream: %s", err)
}
}()
buildContext, err := remotecontext.FromArchive(tarStream)
if err != nil {
t.Fatalf("Error when creating tar context: %s", err)
}
defer func() {
if err = buildContext.Close(); err != nil {
t.Fatalf("Error when closing tar context: %s", err)
}
}()
b := newBuilderWithMockBackend(t)
sb := newDispatchRequest(b, '`', buildContext, NewBuildArgs(make(map[string]*string)), newStagesBuildResults())
err = dispatch(context.TODO(), sb, tc.cmd)
assert.Check(t, is.ErrorContains(err, tc.expectedError))
})
}
}

View File

@@ -0,0 +1,132 @@
package dockerfile
import (
"context"
"runtime"
"github.com/containerd/log"
"github.com/containerd/platforms"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/daemon/builder"
dockerimage "github.com/docker/docker/image"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
type getAndMountFunc func(context.Context, string, bool, *ocispec.Platform) (builder.Image, builder.ROLayer, error)
// imageSources mounts images and provides a cache for mounted images. It tracks
// all images so they can be unmounted at the end of the build.
type imageSources struct {
byImageID map[string]*imageMount
mounts []*imageMount
getImage getAndMountFunc
}
func newImageSources(options builderOptions) *imageSources {
getAndMount := func(ctx context.Context, idOrRef string, localOnly bool, platform *ocispec.Platform) (builder.Image, builder.ROLayer, error) {
pullOption := backend.PullOptionNoPull
if !localOnly {
if options.Options.PullParent {
pullOption = backend.PullOptionForcePull
} else {
pullOption = backend.PullOptionPreferLocal
}
}
return options.Backend.GetImageAndReleasableLayer(ctx, idOrRef, backend.GetImageAndLayerOptions{
PullOption: pullOption,
AuthConfig: options.Options.AuthConfigs,
Output: options.ProgressWriter.Output,
Platform: platform,
})
}
return &imageSources{
byImageID: make(map[string]*imageMount),
getImage: getAndMount,
}
}
func (m *imageSources) Get(ctx context.Context, idOrRef string, localOnly bool, platform *ocispec.Platform) (*imageMount, error) {
if im, ok := m.byImageID[idOrRef]; ok {
return im, nil
}
image, layer, err := m.getImage(ctx, idOrRef, localOnly, platform)
if err != nil {
return nil, err
}
im := newImageMount(image, layer)
m.Add(im, platform)
return im, nil
}
func (m *imageSources) Unmount() (retErr error) {
for _, im := range m.mounts {
if err := im.unmount(); err != nil {
log.G(context.TODO()).Error(err)
retErr = err
}
}
return retErr
}
func (m *imageSources) Add(im *imageMount, platform *ocispec.Platform) {
switch im.image {
case nil:
// Set the platform for scratch images
if platform == nil {
p := platforms.DefaultSpec()
platform = &p
}
// Windows does not support scratch except for LCOW
os := platform.OS
if runtime.GOOS == "windows" {
os = "linux"
}
im.image = &dockerimage.Image{V1Image: dockerimage.V1Image{
OS: os,
Architecture: platform.Architecture,
Variant: platform.Variant,
}}
default:
m.byImageID[im.image.ImageID()] = im
}
m.mounts = append(m.mounts, im)
}
// imageMount is a reference to an image that can be used as a builder.Source
type imageMount struct {
image builder.Image
layer builder.ROLayer
}
func newImageMount(image builder.Image, layer builder.ROLayer) *imageMount {
im := &imageMount{image: image, layer: layer}
return im
}
func (im *imageMount) unmount() error {
if im.layer == nil {
return nil
}
if err := im.layer.Release(); err != nil {
return errors.Wrapf(err, "failed to unmount previous build image %s", im.image.ImageID())
}
im.layer = nil
return nil
}
func (im *imageMount) Image() builder.Image {
return im.image
}
func (im *imageMount) NewRWLayer() (builder.RWLayer, error) {
return im.layer.NewRWLayer()
}
func (im *imageMount) ImageID() string {
return im.image.ImageID()
}

View File

@@ -0,0 +1,108 @@
package dockerfile
import (
"context"
"errors"
"runtime"
"testing"
"github.com/containerd/platforms"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/image"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"gotest.tools/v3/assert"
)
func getMockImageSource(getImageImage builder.Image, getImageLayer builder.ROLayer, getImageError error) *imageSources {
return &imageSources{
byImageID: make(map[string]*imageMount),
mounts: []*imageMount{},
getImage: func(_ context.Context, name string, localOnly bool, platform *ocispec.Platform) (builder.Image, builder.ROLayer, error) {
return getImageImage, getImageLayer, getImageError
},
}
}
func getMockImageMount() *imageMount {
return &imageMount{
image: nil,
layer: nil,
}
}
func TestAddScratchImageAddsToMounts(t *testing.T) {
is := getMockImageSource(nil, nil, errors.New("getImage is not implemented"))
im := getMockImageMount()
// We are testing whether the imageMount is added to is.mounts
assert.Equal(t, len(is.mounts), 0)
is.Add(im, nil)
assert.Equal(t, len(is.mounts), 1)
}
func TestAddFromScratchPopulatesPlatform(t *testing.T) {
is := getMockImageSource(nil, nil, errors.New("getImage is not implemented"))
platforms := []*ocispec.Platform{
{
OS: "linux",
Architecture: "amd64",
},
{
OS: "linux",
Architecture: "arm64",
Variant: "v8",
},
}
for i, platform := range platforms {
im := getMockImageMount()
assert.Equal(t, len(is.mounts), i)
is.Add(im, platform)
image, ok := im.image.(*image.Image)
assert.Assert(t, ok)
assert.Equal(t, image.OS, platform.OS)
assert.Equal(t, image.Architecture, platform.Architecture)
assert.Equal(t, image.Variant, platform.Variant)
}
}
func TestAddFromScratchDoesNotModifyArgPlatform(t *testing.T) {
is := getMockImageSource(nil, nil, errors.New("getImage is not implemented"))
im := getMockImageMount()
platform := &ocispec.Platform{
OS: "windows",
Architecture: "amd64",
}
argPlatform := *platform
is.Add(im, &argPlatform)
// The way the code is written right now, this test
// really doesn't do much except on Windows.
assert.DeepEqual(t, *platform, argPlatform)
}
func TestAddFromScratchPopulatesPlatformIfNil(t *testing.T) {
is := getMockImageSource(nil, nil, errors.New("getImage is not implemented"))
im := getMockImageMount()
is.Add(im, nil)
image, ok := im.image.(*image.Image)
assert.Assert(t, ok)
expectedPlatform := platforms.DefaultSpec()
if runtime.GOOS == "windows" {
expectedPlatform.OS = "linux"
}
assert.Equal(t, expectedPlatform.OS, image.OS)
assert.Equal(t, expectedPlatform.Architecture, image.Architecture)
assert.Equal(t, expectedPlatform.Variant, image.Variant)
}
func TestImageSourceGetAddsToMounts(t *testing.T) {
is := getMockImageSource(nil, nil, nil)
ctx := context.Background()
_, err := is.Get(ctx, "test", false, nil)
assert.NilError(t, err)
assert.Equal(t, len(is.mounts), 1)
}

View File

@@ -0,0 +1,80 @@
package dockerfile
import (
"context"
"github.com/containerd/log"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
// ImageProber exposes an Image cache to the Builder. It supports resetting a
// cache.
type ImageProber interface {
Reset(ctx context.Context) error
Probe(parentID string, runConfig *container.Config, platform ocispec.Platform) (string, error)
}
type resetFunc func(context.Context) (builder.ImageCache, error)
type imageProber struct {
cache builder.ImageCache
reset resetFunc
cacheBusted bool
}
func newImageProber(ctx context.Context, cacheBuilder builder.ImageCacheBuilder, cacheFrom []string, noCache bool) (ImageProber, error) {
if noCache {
return &nopProber{}, nil
}
reset := func(ctx context.Context) (builder.ImageCache, error) {
return cacheBuilder.MakeImageCache(ctx, cacheFrom)
}
cache, err := reset(ctx)
if err != nil {
return nil, err
}
return &imageProber{cache: cache, reset: reset}, nil
}
func (c *imageProber) Reset(ctx context.Context) error {
newCache, err := c.reset(ctx)
if err != nil {
return err
}
c.cache = newCache
c.cacheBusted = false
return nil
}
// Probe checks if cache match can be found for current build instruction.
// It returns the cachedID if there is a hit, and the empty string on miss
func (c *imageProber) Probe(parentID string, runConfig *container.Config, platform ocispec.Platform) (string, error) {
if c.cacheBusted {
return "", nil
}
cacheID, err := c.cache.GetCache(parentID, runConfig, platform)
if err != nil {
return "", err
}
if cacheID == "" {
log.G(context.TODO()).Debugf("[BUILDER] Cache miss: %s", runConfig.Cmd)
c.cacheBusted = true
return "", nil
}
log.G(context.TODO()).Debugf("[BUILDER] Use cached version: %s", runConfig.Cmd)
return cacheID, nil
}
type nopProber struct{}
func (c *nopProber) Reset(ctx context.Context) error {
return nil
}
func (c *nopProber) Probe(_ string, _ *container.Config, _ ocispec.Platform) (string, error) {
return "", nil
}

View File

@@ -0,0 +1,414 @@
package dockerfile
// internals for handling commands. Covers many areas and a lot of
// non-contiguous functionality. Please read the comments.
import (
"context"
"crypto/sha256"
"encoding/hex"
"fmt"
"strings"
"github.com/containerd/log"
"github.com/containerd/platforms"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/build"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/api/types/network"
"github.com/docker/docker/daemon/builder"
networkSettings "github.com/docker/docker/daemon/network"
"github.com/docker/docker/image"
"github.com/docker/docker/pkg/stringid"
"github.com/docker/go-connections/nat"
"github.com/moby/go-archive"
"github.com/moby/go-archive/chrootarchive"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
"github.com/pkg/errors"
)
func (b *Builder) getArchiver() *archive.Archiver {
return chrootarchive.NewArchiver(b.idMapping)
}
func (b *Builder) commit(ctx context.Context, dispatchState *dispatchState, comment string) error {
if b.disableCommit {
return nil
}
if !dispatchState.hasFromImage() {
return errors.New("Please provide a source image with `from` prior to commit")
}
runConfigWithCommentCmd := copyRunConfig(dispatchState.runConfig, withCmdComment(comment, dispatchState.operatingSystem))
id, err := b.probeAndCreate(ctx, dispatchState, runConfigWithCommentCmd)
if err != nil || id == "" {
return err
}
return b.commitContainer(ctx, dispatchState, id, runConfigWithCommentCmd)
}
func (b *Builder) commitContainer(ctx context.Context, dispatchState *dispatchState, id string, containerConfig *container.Config) error {
if b.disableCommit {
return nil
}
commitCfg := backend.CommitConfig{
Author: dispatchState.maintainer,
// TODO: this copy should be done by Commit()
Config: copyRunConfig(dispatchState.runConfig),
ContainerConfig: containerConfig,
ContainerID: id,
}
imageID, err := b.docker.CommitBuildStep(ctx, commitCfg)
dispatchState.imageID = string(imageID)
return err
}
func (b *Builder) exportImage(ctx context.Context, state *dispatchState, layer builder.RWLayer, parent builder.Image, runConfig *container.Config) error {
newLayer, err := layer.Commit()
if err != nil {
return err
}
parentImage, ok := parent.(*image.Image)
if !ok {
return errors.Errorf("unexpected image type")
}
platform := &ocispec.Platform{
OS: parentImage.OS,
Architecture: parentImage.Architecture,
Variant: parentImage.Variant,
}
// add an image mount without an image so the layer is properly unmounted
// if there is an error before we can add the full mount with image
b.imageSources.Add(newImageMount(nil, newLayer), platform)
newImage := image.NewChildImage(parentImage, image.ChildConfig{
Author: state.maintainer,
ContainerConfig: runConfig,
DiffID: newLayer.DiffID(),
Config: copyRunConfig(state.runConfig),
}, parentImage.OS)
// TODO: it seems strange to marshal this here instead of just passing in the
// image struct
config, err := newImage.MarshalJSON()
if err != nil {
return errors.Wrap(err, "failed to encode image config")
}
// when writing the new image's manifest, we now need to pass in the new layer's digest.
// before the containerd store work this was unnecessary since we get the layer id
// from the image's RootFS ChainID -- see:
// https://github.com/moby/moby/blob/8cf66ed7322fa885ef99c4c044fa23e1727301dc/image/store.go#L162
// however, with the containerd store we can't do this. An alternative implementation here
// without changing the signature would be to get the layer digest by walking the content store
// and filtering the objects to find the layer with the DiffID we want, but that has performance
// implications that should be called out/investigated
exportedImage, err := b.docker.CreateImage(ctx, config, state.imageID, newLayer.ContentStoreDigest())
if err != nil {
return errors.Wrapf(err, "failed to export image")
}
state.imageID = exportedImage.ImageID()
b.imageSources.Add(newImageMount(exportedImage, newLayer), platform)
return nil
}
func (b *Builder) performCopy(ctx context.Context, req dispatchRequest, inst copyInstruction) error {
state := req.state
srcHash := getSourceHashFromInfos(inst.infos)
var chownComment string
if inst.chownStr != "" {
chownComment = fmt.Sprintf("--chown=%s ", inst.chownStr)
}
commentStr := fmt.Sprintf("%s %s%s in %s ", inst.cmdName, chownComment, srcHash, inst.dest)
// TODO: should this have been using origPaths instead of srcHash in the comment?
runConfigWithCommentCmd := copyRunConfig(state.runConfig, withCmdCommentString(commentStr, state.operatingSystem))
hit, err := b.probeCache(state, runConfigWithCommentCmd)
if err != nil || hit {
return err
}
imgMount, err := b.imageSources.Get(ctx, state.imageID, true, req.builder.platform)
if err != nil {
return errors.Wrapf(err, "failed to get destination image %q", state.imageID)
}
rwLayer, err := imgMount.NewRWLayer()
if err != nil {
return err
}
defer rwLayer.Release()
destInfo, err := createDestInfo(state.runConfig.WorkingDir, inst, rwLayer)
if err != nil {
return err
}
uid, gid := b.idMapping.RootPair()
id := identity{UID: uid, GID: gid}
// if a chown was requested, perform the steps to get the uid, gid
// translated (if necessary because of user namespaces), and replace
// the root pair with the chown pair for copy operations
if inst.chownStr != "" {
id, err = parseChownFlag(ctx, b, state, inst.chownStr, destInfo.root, b.idMapping)
if err != nil {
if b.options.Platform != "windows" {
return errors.Wrapf(err, "unable to convert uid/gid chown string to host mapping")
}
return errors.Wrapf(err, "unable to map container user account name to SID")
}
}
for _, info := range inst.infos {
opts := copyFileOptions{
decompress: inst.allowLocalDecompression,
archiver: b.getArchiver(),
}
if !inst.preserveOwnership {
opts.identity = &id
}
if err := performCopyForInfo(destInfo, info, opts); err != nil {
return errors.Wrapf(err, "failed to copy files")
}
}
return b.exportImage(ctx, state, rwLayer, imgMount.Image(), runConfigWithCommentCmd)
}
func createDestInfo(workingDir string, inst copyInstruction, rwLayer builder.RWLayer) (copyInfo, error) {
// Twiddle the destination when it's a relative path - meaning, make it
// relative to the WORKINGDIR
dest, err := normalizeDest(workingDir, inst.dest)
if err != nil {
return copyInfo{}, errors.Wrapf(err, "invalid %s", inst.cmdName)
}
return copyInfo{root: rwLayer.Root(), path: dest}, nil
}
// For backwards compat, if there's just one info then use it as the
// cache look-up string, otherwise hash 'em all into one
func getSourceHashFromInfos(infos []copyInfo) string {
if len(infos) == 1 {
return infos[0].hash
}
var hashs []string
for _, info := range infos {
hashs = append(hashs, info.hash)
}
return hashStringSlice("multi", hashs)
}
func hashStringSlice(prefix string, slice []string) string {
hasher := sha256.New()
hasher.Write([]byte(strings.Join(slice, ",")))
return prefix + ":" + hex.EncodeToString(hasher.Sum(nil))
}
type runConfigModifier func(*container.Config)
func withCmd(cmd []string) runConfigModifier {
return func(runConfig *container.Config) {
runConfig.Cmd = cmd
}
}
func withArgsEscaped(argsEscaped bool) runConfigModifier {
return func(runConfig *container.Config) {
runConfig.ArgsEscaped = argsEscaped
}
}
// withCmdComment sets Cmd to a nop comment string. See withCmdCommentString for
// why there are two almost identical versions of this.
func withCmdComment(comment string, platform string) runConfigModifier {
return func(runConfig *container.Config) {
runConfig.Cmd = append(getShell(runConfig, platform), "#(nop) ", comment)
}
}
// withCmdCommentString exists to maintain compatibility with older versions.
// A few instructions (workdir, copy, add) used a nop comment that is a single arg
// where as all the other instructions used a two arg comment string. This
// function implements the single arg version.
func withCmdCommentString(comment string, platform string) runConfigModifier {
return func(runConfig *container.Config) {
runConfig.Cmd = append(getShell(runConfig, platform), "#(nop) "+comment)
}
}
func withEnv(env []string) runConfigModifier {
return func(runConfig *container.Config) {
runConfig.Env = env
}
}
// withEntrypointOverride sets an entrypoint on runConfig if the command is
// not empty. The entrypoint is left unmodified if command is empty.
//
// The dockerfile RUN instruction expect to run without an entrypoint
// so the runConfig entrypoint needs to be modified accordingly. ContainerCreate
// will change a []string{""} entrypoint to nil, so we probe the cache with the
// nil entrypoint.
func withEntrypointOverride(cmd []string, entrypoint []string) runConfigModifier {
return func(runConfig *container.Config) {
if len(cmd) > 0 {
runConfig.Entrypoint = entrypoint
}
}
}
// withoutHealthcheck disables healthcheck.
//
// The dockerfile RUN instruction expect to run without healthcheck
// so the runConfig Healthcheck needs to be disabled.
func withoutHealthcheck() runConfigModifier {
return func(runConfig *container.Config) {
runConfig.Healthcheck = &container.HealthConfig{
Test: []string{"NONE"},
}
}
}
func copyRunConfig(runConfig *container.Config, modifiers ...runConfigModifier) *container.Config {
cfgCopy := *runConfig
cfgCopy.Cmd = copyStringSlice(runConfig.Cmd)
cfgCopy.Env = copyStringSlice(runConfig.Env)
cfgCopy.Entrypoint = copyStringSlice(runConfig.Entrypoint)
cfgCopy.OnBuild = copyStringSlice(runConfig.OnBuild)
cfgCopy.Shell = copyStringSlice(runConfig.Shell)
if cfgCopy.Volumes != nil {
cfgCopy.Volumes = make(map[string]struct{}, len(runConfig.Volumes))
for k, v := range runConfig.Volumes {
cfgCopy.Volumes[k] = v
}
}
if cfgCopy.ExposedPorts != nil {
cfgCopy.ExposedPorts = make(nat.PortSet, len(runConfig.ExposedPorts))
for k, v := range runConfig.ExposedPorts {
cfgCopy.ExposedPorts[k] = v
}
}
if cfgCopy.Labels != nil {
cfgCopy.Labels = make(map[string]string, len(runConfig.Labels))
for k, v := range runConfig.Labels {
cfgCopy.Labels[k] = v
}
}
for _, modifier := range modifiers {
modifier(&cfgCopy)
}
return &cfgCopy
}
func copyStringSlice(orig []string) []string {
if orig == nil {
return nil
}
return append([]string{}, orig...)
}
// getShell is a helper function which gets the right shell for prefixing the
// shell-form of RUN, ENTRYPOINT and CMD instructions
func getShell(c *container.Config, os string) []string {
if len(c.Shell) == 0 {
return append([]string{}, defaultShellForOS(os)[:]...)
}
return append([]string{}, c.Shell[:]...)
}
func (b *Builder) probeCache(dispatchState *dispatchState, runConfig *container.Config) (bool, error) {
cachedID, err := b.imageProber.Probe(dispatchState.imageID, runConfig, b.getPlatform(dispatchState))
if cachedID == "" || err != nil {
return false, err
}
_, _ = fmt.Fprintln(b.Stdout, " ---> Using cache")
dispatchState.imageID = cachedID
return true, nil
}
var defaultLogConfig = container.LogConfig{Type: "none"}
func (b *Builder) probeAndCreate(ctx context.Context, dispatchState *dispatchState, runConfig *container.Config) (string, error) {
if hit, err := b.probeCache(dispatchState, runConfig); err != nil || hit {
return "", err
}
return b.create(ctx, runConfig)
}
func (b *Builder) create(ctx context.Context, runConfig *container.Config) (string, error) {
log.G(ctx).Debugf("[BUILDER] Command to be executed: %v", runConfig.Cmd)
hostConfig := hostConfigFromOptions(b.options)
ctr, err := b.containerManager.Create(ctx, runConfig, hostConfig)
if err != nil {
return "", err
}
for _, warning := range ctr.Warnings {
_, _ = fmt.Fprintf(b.Stdout, " ---> [Warning] %s\n", warning)
}
_, _ = fmt.Fprintf(b.Stdout, " ---> Running in %s\n", stringid.TruncateID(ctr.ID))
return ctr.ID, nil
}
func hostConfigFromOptions(options *build.ImageBuildOptions) *container.HostConfig {
resources := container.Resources{
CgroupParent: options.CgroupParent,
CPUShares: options.CPUShares,
CPUPeriod: options.CPUPeriod,
CPUQuota: options.CPUQuota,
CpusetCpus: options.CPUSetCPUs,
CpusetMems: options.CPUSetMems,
Memory: options.Memory,
MemorySwap: options.MemorySwap,
Ulimits: options.Ulimits,
}
// We need to make sure no empty string or "default" NetworkMode is
// provided to the daemon as it doesn't support them.
//
// This is in line with what the ContainerCreate API endpoint does.
networkMode := options.NetworkMode
if networkMode == "" || networkMode == network.NetworkDefault {
networkMode = networkSettings.DefaultNetwork
}
hc := &container.HostConfig{
SecurityOpt: options.SecurityOpt,
Isolation: options.Isolation,
ShmSize: options.ShmSize,
Resources: resources,
NetworkMode: container.NetworkMode(networkMode),
// Set a log config to override any default value set on the daemon
LogConfig: defaultLogConfig,
ExtraHosts: options.ExtraHosts,
}
return hc
}
func (b *Builder) getPlatform(state *dispatchState) ocispec.Platform {
// May be nil if not explicitly set in API/dockerfile
out := platforms.DefaultSpec()
if b.platform != nil {
out = *b.platform
}
if state.operatingSystem != "" {
out.OS = state.operatingSystem
}
return out
}

View File

@@ -0,0 +1,88 @@
package dockerfile
import (
"context"
"path/filepath"
"strconv"
"strings"
"github.com/moby/sys/symlink"
"github.com/moby/sys/user"
"github.com/pkg/errors"
)
func parseChownFlag(ctx context.Context, builder *Builder, state *dispatchState, chown, ctrRootPath string, identityMapping user.IdentityMapping) (identity, error) {
var userStr, grpStr string
parts := strings.Split(chown, ":")
if len(parts) > 2 {
return identity{}, errors.New("invalid chown string format: " + chown)
}
if len(parts) == 1 {
// if no group specified, use the user spec as group as well
userStr, grpStr = parts[0], parts[0]
} else {
userStr, grpStr = parts[0], parts[1]
}
passwdPath, err := symlink.FollowSymlinkInScope(filepath.Join(ctrRootPath, "etc", "passwd"), ctrRootPath)
if err != nil {
return identity{}, errors.Wrap(err, "can't resolve /etc/passwd path in container rootfs")
}
groupPath, err := symlink.FollowSymlinkInScope(filepath.Join(ctrRootPath, "etc", "group"), ctrRootPath)
if err != nil {
return identity{}, errors.Wrap(err, "can't resolve /etc/group path in container rootfs")
}
uid, err := lookupUser(userStr, passwdPath)
if err != nil {
return identity{}, errors.Wrap(err, "can't find uid for user "+userStr)
}
gid, err := lookupGroup(grpStr, groupPath)
if err != nil {
return identity{}, errors.Wrap(err, "can't find gid for group "+grpStr)
}
// convert as necessary because of user namespaces
uid, gid, err = identityMapping.ToHost(uid, gid)
if err != nil {
return identity{}, errors.Wrap(err, "unable to convert uid/gid to host mapping")
}
return identity{UID: uid, GID: gid}, nil
}
func lookupUser(userStr, filepath string) (int, error) {
// if the string is actually a uid integer, parse to int and return
// as we don't need to translate with the help of files
uid, err := strconv.Atoi(userStr)
if err == nil {
return uid, nil
}
users, err := user.ParsePasswdFileFilter(filepath, func(u user.User) bool {
return u.Name == userStr
})
if err != nil {
return 0, err
}
if len(users) == 0 {
return 0, errors.New("no such user: " + userStr)
}
return users[0].Uid, nil
}
func lookupGroup(groupStr, filepath string) (int, error) {
// if the string is actually a gid integer, parse to int and return
// as we don't need to translate with the help of files
gid, err := strconv.Atoi(groupStr)
if err == nil {
return gid, nil
}
groups, err := user.ParseGroupFileFilter(filepath, func(g user.Group) bool {
return g.Name == groupStr
})
if err != nil {
return 0, err
}
if len(groups) == 0 {
return 0, errors.New("no such group: " + groupStr)
}
return groups[0].Gid, nil
}

View File

@@ -0,0 +1,163 @@
package dockerfile
import (
"context"
"os"
"path/filepath"
"testing"
"github.com/docker/docker/api/types/build"
"github.com/moby/sys/user"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func TestChownFlagParsing(t *testing.T) {
testFiles := map[string]string{
"passwd": `root:x:0:0::/bin:/bin/false
bin:x:1:1::/bin:/bin/false
wwwwww:x:21:33::/bin:/bin/false
unicorn:x:1001:1002::/bin:/bin/false
`,
"group": `root:x:0:
bin:x:1:
wwwwww:x:33:
unicorn:x:1002:
somegrp:x:5555:
othergrp:x:6666:
`,
}
// test mappings for validating use of maps
idMaps := []user.IDMap{
{
ID: 0,
ParentID: 100000,
Count: 65536,
},
}
remapped := user.IdentityMapping{UIDMaps: idMaps, GIDMaps: idMaps}
unmapped := user.IdentityMapping{}
contextDir := t.TempDir()
if err := os.Mkdir(filepath.Join(contextDir, "etc"), 0o755); err != nil {
t.Fatalf("error creating test directory: %v", err)
}
for filename, content := range testFiles {
createTestTempFile(t, filepath.Join(contextDir, "etc"), filename, content, 0o644)
}
// positive tests
for _, testcase := range []struct {
builder *Builder
name string
chownStr string
idMapping user.IdentityMapping
state *dispatchState
expected identity
}{
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UIDNoMap",
chownStr: "1",
idMapping: unmapped,
state: &dispatchState{},
expected: identity{UID: 1, GID: 1},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UIDGIDNoMap",
chownStr: "0:1",
idMapping: unmapped,
state: &dispatchState{},
expected: identity{UID: 0, GID: 1},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UIDWithMap",
chownStr: "0",
idMapping: remapped,
state: &dispatchState{},
expected: identity{UID: 100000, GID: 100000},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UIDGIDWithMap",
chownStr: "1:33",
idMapping: remapped,
state: &dispatchState{},
expected: identity{UID: 100001, GID: 100033},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UserNoMap",
chownStr: "bin:5555",
idMapping: unmapped,
state: &dispatchState{},
expected: identity{UID: 1, GID: 5555},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "GroupWithMap",
chownStr: "0:unicorn",
idMapping: remapped,
state: &dispatchState{},
expected: identity{UID: 100000, GID: 101002},
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UserOnlyWithMap",
chownStr: "unicorn",
idMapping: remapped,
state: &dispatchState{},
expected: identity{UID: 101001, GID: 101002},
},
} {
t.Run(testcase.name, func(t *testing.T) {
idPair, err := parseChownFlag(context.TODO(), testcase.builder, testcase.state, testcase.chownStr, contextDir, testcase.idMapping)
assert.NilError(t, err, "Failed to parse chown flag: %q", testcase.chownStr)
assert.Check(t, is.DeepEqual(testcase.expected, idPair), "chown flag mapping failure")
})
}
// error tests
for _, testcase := range []struct {
builder *Builder
name string
chownStr string
idMapping user.IdentityMapping
state *dispatchState
descr string
}{
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "BadChownFlagFormat",
chownStr: "bob:1:555",
idMapping: unmapped,
state: &dispatchState{},
descr: "invalid chown string format: bob:1:555",
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "UserNoExist",
chownStr: "bob",
idMapping: unmapped,
state: &dispatchState{},
descr: "can't find uid for user bob: no such user: bob",
},
{
builder: &Builder{options: &build.ImageBuildOptions{Platform: "linux"}},
name: "GroupNoExist",
chownStr: "root:bob",
idMapping: unmapped,
state: &dispatchState{},
descr: "can't find gid for group bob: no such group: bob",
},
} {
t.Run(testcase.name, func(t *testing.T) {
_, err := parseChownFlag(context.TODO(), testcase.builder, testcase.state, testcase.chownStr, contextDir, testcase.idMapping)
assert.Check(t, is.Error(err, testcase.descr), "Expected error string doesn't match")
})
}
}

View File

@@ -0,0 +1,213 @@
package dockerfile
import (
"context"
"fmt"
"os"
"runtime"
"testing"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/build"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/daemon/builder/remotecontext"
"github.com/docker/docker/image"
"github.com/docker/docker/layer"
"github.com/docker/go-connections/nat"
"github.com/moby/go-archive"
"github.com/opencontainers/go-digest"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
"gotest.tools/v3/skip"
)
func TestEmptyDockerfile(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, builder.DefaultDockerfileName, "", 0o777)
readAndCheckDockerfile(t, "emptyDockerfile", contextDir, "", "the Dockerfile (Dockerfile) cannot be empty")
}
func TestSymlinkDockerfile(t *testing.T) {
contextDir := t.TempDir()
createTestSymlink(t, contextDir, builder.DefaultDockerfileName, "/etc/passwd")
// The reason the error is "Cannot locate specified Dockerfile" is because
// in the builder, the symlink is resolved within the context, therefore
// Dockerfile -> /etc/passwd becomes etc/passwd from the context which is
// a nonexistent file.
expectedError := fmt.Sprintf("Cannot locate specified Dockerfile: %s", builder.DefaultDockerfileName)
readAndCheckDockerfile(t, "symlinkDockerfile", contextDir, builder.DefaultDockerfileName, expectedError)
}
func TestDockerfileOutsideTheBuildContext(t *testing.T) {
contextDir := t.TempDir()
expectedError := "path outside the build context: ../../Dockerfile ()"
if runtime.GOOS == "windows" {
expectedError = "failed to resolve scoped path ../../Dockerfile ()"
}
readAndCheckDockerfile(t, "DockerfileOutsideTheBuildContext", contextDir, "../../Dockerfile", expectedError)
}
func TestNonExistingDockerfile(t *testing.T) {
contextDir := t.TempDir()
const expectedError = "Cannot locate specified Dockerfile: Dockerfile"
readAndCheckDockerfile(t, "NonExistingDockerfile", contextDir, "Dockerfile", expectedError)
}
func readAndCheckDockerfile(t *testing.T, testName, contextDir, dockerfilePath, expectedError string) {
if runtime.GOOS != "windows" {
skip.If(t, os.Getuid() != 0, "skipping test that requires root")
}
tarStream, err := archive.Tar(contextDir, archive.Uncompressed)
assert.NilError(t, err)
defer func() {
if err = tarStream.Close(); err != nil {
t.Fatalf("Error when closing tar stream: %s", err)
}
}()
if dockerfilePath == "" { // handled in BuildWithContext
dockerfilePath = builder.DefaultDockerfileName
}
config := backend.BuildConfig{
Options: &build.ImageBuildOptions{Dockerfile: dockerfilePath},
Source: tarStream,
}
_, _, err = remotecontext.Detect(config)
assert.Check(t, is.ErrorContains(err, expectedError))
}
func TestCopyRunConfig(t *testing.T) {
defaultEnv := []string{"foo=1"}
defaultCmd := []string{"old"}
testcases := []struct {
doc string
modifiers []runConfigModifier
expected *container.Config
}{
{
doc: "Set the command",
modifiers: []runConfigModifier{withCmd([]string{"new"})},
expected: &container.Config{
Cmd: []string{"new"},
Env: defaultEnv,
},
},
{
doc: "Set the command to a comment",
modifiers: []runConfigModifier{withCmdComment("comment", runtime.GOOS)},
expected: &container.Config{
Cmd: append(defaultShellForOS(runtime.GOOS), "#(nop) ", "comment"),
Env: defaultEnv,
},
},
{
doc: "Set the command and env",
modifiers: []runConfigModifier{
withCmd([]string{"new"}),
withEnv([]string{"one", "two"}),
},
expected: &container.Config{
Cmd: []string{"new"},
Env: []string{"one", "two"},
},
},
}
for _, testcase := range testcases {
runConfig := &container.Config{
Cmd: defaultCmd,
Env: defaultEnv,
}
runConfigCopy := copyRunConfig(runConfig, testcase.modifiers...)
assert.Check(t, is.DeepEqual(testcase.expected, runConfigCopy), testcase.doc)
// Assert the original was not modified
assert.Check(t, runConfig != runConfigCopy, testcase.doc)
}
}
func fullMutableRunConfig() *container.Config {
return &container.Config{
Cmd: []string{"command", "arg1"},
Env: []string{"env1=foo", "env2=bar"},
ExposedPorts: nat.PortSet{
"1000/tcp": {},
"1001/tcp": {},
},
Volumes: map[string]struct{}{
"one": {},
"two": {},
},
Entrypoint: []string{"entry", "arg1"},
OnBuild: []string{"first", "next"},
Labels: map[string]string{
"label1": "value1",
"label2": "value2",
},
Shell: []string{"shell", "-c"},
}
}
func TestDeepCopyRunConfig(t *testing.T) {
runConfig := fullMutableRunConfig()
ctrCfg := copyRunConfig(runConfig)
assert.Check(t, is.DeepEqual(fullMutableRunConfig(), ctrCfg))
ctrCfg.Cmd[1] = "arg2"
ctrCfg.Env[1] = "env2=new"
ctrCfg.ExposedPorts["10002"] = struct{}{}
ctrCfg.Volumes["three"] = struct{}{}
ctrCfg.Entrypoint[1] = "arg2"
ctrCfg.OnBuild[0] = "start"
ctrCfg.Labels["label3"] = "value3"
ctrCfg.Shell[0] = "sh"
assert.Check(t, is.DeepEqual(fullMutableRunConfig(), runConfig))
}
type MockRWLayer struct{}
func (l *MockRWLayer) Release() error { return nil }
func (l *MockRWLayer) Root() string { return "" }
func (l *MockRWLayer) Commit() (builder.ROLayer, error) {
return &MockROLayer{
diffID: "sha256:1234",
}, nil
}
type MockROLayer struct {
diffID layer.DiffID
}
func (l *MockROLayer) ContentStoreDigest() digest.Digest { return "" }
func (l *MockROLayer) Release() error { return nil }
func (l *MockROLayer) NewRWLayer() (builder.RWLayer, error) { return nil, nil }
func (l *MockROLayer) DiffID() layer.DiffID { return l.diffID }
func getMockBuildBackend() builder.Backend {
return &MockBackend{}
}
func TestExportImage(t *testing.T) {
ds := newDispatchState(NewBuildArgs(map[string]*string{}))
parentImage := &image.Image{
V1Image: image.V1Image{
OS: "linux",
Architecture: "arm64",
Variant: "v8",
},
}
b := &Builder{
imageSources: getMockImageSource(nil, nil, nil),
docker: getMockBuildBackend(),
}
err := b.exportImage(context.TODO(), ds, &MockRWLayer{}, parentImage, &container.Config{})
assert.NilError(t, err)
}

View File

@@ -0,0 +1,120 @@
package dockerfile
import (
"bytes"
"context"
"os"
"path/filepath"
"strings"
"github.com/containerd/platforms"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/api/types/mount"
"github.com/docker/docker/errdefs"
"github.com/docker/docker/pkg/jsonmessage"
"github.com/moby/sys/user"
"golang.org/x/sys/windows"
)
// seTakeOwnershipPrivilege is "SE_TAKE_OWNERSHIP_NAME" in the win32 API.
//
// see https://learn.microsoft.com/en-us/windows/win32/secauthz/privilege-constants
const seTakeOwnershipPrivilege = "SeTakeOwnershipPrivilege"
// Constants for well-known SIDs in the Windows container.
// These are currently undocumented.
// See https://github.com/moby/buildkit/pull/5791#discussion_r1976652227 for more information.
const (
containerAdministratorSidString = "S-1-5-93-2-1" // ContainerAdministrator
containerUserSidString = "S-1-5-93-2-2" // ContainerUser
)
func parseChownFlag(ctx context.Context, builder *Builder, state *dispatchState, chown, ctrRootPath string, identityMapping user.IdentityMapping) (identity, error) {
if builder.options.Platform == "windows" {
return getAccountIdentity(ctx, builder, chown, ctrRootPath, state)
}
uid, gid := identityMapping.RootPair()
return identity{UID: uid, GID: gid}, nil
}
func getAccountIdentity(ctx context.Context, builder *Builder, accountName string, ctrRootPath string, state *dispatchState) (identity, error) {
// If this is potentially a string SID then attempt to convert it to verify
// this, otherwise continue looking for the account.
if strings.HasPrefix(accountName, "S-") || strings.HasPrefix(accountName, "s-") {
sid, err := windows.StringToSid(accountName)
if err == nil {
return identity{SID: sid.String()}, nil
}
}
// Attempt to obtain the SID using the name.
sid, _, accType, err := windows.LookupSID("", accountName)
// If this is a SID that is built-in and hence the same across all systems then use that.
if err == nil && (accType == windows.SidTypeAlias || accType == windows.SidTypeWellKnownGroup) {
return identity{SID: sid.String()}, nil
}
// Check if the account name is one unique to containers.
if strings.EqualFold(accountName, "ContainerAdministrator") {
return identity{SID: containerAdministratorSidString}, nil
} else if strings.EqualFold(accountName, "ContainerUser") {
return identity{SID: containerUserSidString}, nil
}
// All other lookups failed, so therefore determine if the account in
// question exists in the container and if so, obtain its SID.
return lookupNTAccount(ctx, builder, accountName, state)
}
func lookupNTAccount(ctx context.Context, builder *Builder, accountName string, state *dispatchState) (identity, error) {
source, _ := filepath.Split(os.Args[0])
target := "C:\\Docker"
targetExecutable := target + "\\containerutility.exe"
optionsPlatform, err := platforms.Parse(builder.options.Platform)
if err != nil {
return identity{}, errdefs.InvalidParameter(err)
}
runConfig := copyRunConfig(state.runConfig,
withCmdCommentString("internal run to obtain NT account information.", optionsPlatform.OS))
runConfig.Cmd = []string{targetExecutable, "getaccountsid", accountName}
hostConfig := &container.HostConfig{
Mounts: []mount.Mount{
{
Type: mount.TypeBind,
Source: source,
Target: target,
ReadOnly: true,
},
},
}
container, err := builder.containerManager.Create(ctx, runConfig, hostConfig)
if err != nil {
return identity{}, err
}
stdout := new(bytes.Buffer)
stderr := new(bytes.Buffer)
if err := builder.containerManager.Run(ctx, container.ID, stdout, stderr); err != nil {
if err, ok := err.(*statusCodeError); ok {
return identity{}, &jsonmessage.JSONError{
Message: stderr.String(),
Code: err.StatusCode(),
}
}
return identity{}, err
}
accountSid := stdout.String()
return identity{SID: accountSid}, nil
}

View File

@@ -0,0 +1,53 @@
//go:build windows
package dockerfile
import (
"fmt"
"testing"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func TestNormalizeDest(t *testing.T) {
tests := []struct{ current, requested, expected, etext string }{
{``, `D:\`, ``, `Windows does not support destinations not on the system drive (C:)`},
{``, `e:/`, ``, `Windows does not support destinations not on the system drive (C:)`},
{`invalid`, `./c1`, ``, `Current WorkingDir invalid is not platform consistent`},
{`C:`, ``, ``, `Current WorkingDir C: is not platform consistent`},
{`C`, ``, ``, `Current WorkingDir C is not platform consistent`},
{`D:\`, `.`, ``, "Windows does not support relative paths when WORKDIR is not the system drive"},
{``, `D`, `D`, ``},
{``, `./a1`, `.\a1`, ``},
{``, `.\b1`, `.\b1`, ``},
{``, `/`, `\`, ``},
{``, `\`, `\`, ``},
{``, `c:/`, `\`, ``},
{``, `c:\`, `\`, ``},
{``, `.`, `.`, ``},
{`C:\wdd`, `./a1`, `\wdd\a1`, ``},
{`C:\wde`, `.\b1`, `\wde\b1`, ``},
{`C:\wdf`, `/`, `\`, ``},
{`C:\wdg`, `\`, `\`, ``},
{`C:\wdh`, `c:/`, `\`, ``},
{`C:\wdi`, `c:\`, `\`, ``},
{`C:\wdj`, `.`, `\wdj`, ``},
{`C:\wdk`, `foo/bar`, `\wdk\foo\bar`, ``},
{`C:\wdl`, `foo\bar`, `\wdl\foo\bar`, ``},
{`C:\wdm`, `foo/bar/`, `\wdm\foo\bar\`, ``},
{`C:\wdn`, `foo\bar/`, `\wdn\foo\bar\`, ``},
}
for _, testcase := range tests {
msg := fmt.Sprintf("Input: %s, %s", testcase.current, testcase.requested)
actual, err := normalizeDest(testcase.current, testcase.requested)
if testcase.etext == "" {
if !assert.Check(t, err, msg) {
continue
}
assert.Check(t, is.Equal(testcase.expected, actual), msg)
} else {
assert.Check(t, is.ErrorContains(err, testcase.etext))
}
}
}

View File

@@ -0,0 +1,44 @@
package dockerfile
import (
gometrics "github.com/docker/go-metrics"
)
var (
buildsTriggered gometrics.Counter
buildsFailed gometrics.LabeledCounter
)
// Build metrics prometheus messages, these values must be initialized before
// using them. See the example below in the "builds_failed" metric definition.
const (
metricsDockerfileSyntaxError = "dockerfile_syntax_error"
metricsDockerfileEmptyError = "dockerfile_empty_error"
metricsCommandNotSupportedError = "command_not_supported_error"
metricsErrorProcessingCommandsError = "error_processing_commands_error"
metricsBuildTargetNotReachableError = "build_target_not_reachable_error"
metricsMissingOnbuildArgumentsError = "missing_onbuild_arguments_error"
metricsUnknownInstructionError = "unknown_instruction_error"
metricsBuildCanceled = "build_canceled"
)
func init() {
buildMetrics := gometrics.NewNamespace("builder", "", nil)
buildsTriggered = buildMetrics.NewCounter("builds_triggered", "Number of triggered image builds")
buildsFailed = buildMetrics.NewLabeledCounter("builds_failed", "Number of failed image builds", "reason")
for _, r := range []string{
metricsDockerfileSyntaxError,
metricsDockerfileEmptyError,
metricsCommandNotSupportedError,
metricsErrorProcessingCommandsError,
metricsBuildTargetNotReachableError,
metricsMissingOnbuildArgumentsError,
metricsUnknownInstructionError,
metricsBuildCanceled,
} {
buildsFailed.WithValues(r)
}
gometrics.Register(buildMetrics)
}

View File

@@ -0,0 +1,146 @@
package dockerfile
import (
"context"
"encoding/json"
"io"
"runtime"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/api/types/container"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/image"
"github.com/docker/docker/layer"
"github.com/opencontainers/go-digest"
ocispec "github.com/opencontainers/image-spec/specs-go/v1"
)
// MockBackend implements the builder.Backend interface for unit testing
type MockBackend struct {
containerCreateFunc func(config backend.ContainerCreateConfig) (container.CreateResponse, error)
commitFunc func(backend.CommitConfig) (image.ID, error)
getImageFunc func(string) (builder.Image, builder.ROLayer, error)
makeImageCacheFunc func(cacheFrom []string) builder.ImageCache
}
func (m *MockBackend) ContainerAttachRaw(cID string, stdin io.ReadCloser, stdout, stderr io.Writer, stream bool, attached chan struct{}) error {
return nil
}
func (m *MockBackend) ContainerCreateIgnoreImagesArgsEscaped(ctx context.Context, config backend.ContainerCreateConfig) (container.CreateResponse, error) {
if m.containerCreateFunc != nil {
return m.containerCreateFunc(config)
}
return container.CreateResponse{}, nil
}
func (m *MockBackend) ContainerRm(name string, config *backend.ContainerRmConfig) error {
return nil
}
func (m *MockBackend) CommitBuildStep(ctx context.Context, c backend.CommitConfig) (image.ID, error) {
if m.commitFunc != nil {
return m.commitFunc(c)
}
return "", nil
}
func (m *MockBackend) ContainerStart(ctx context.Context, containerID string, checkpoint string, checkpointDir string) error {
return nil
}
func (m *MockBackend) ContainerWait(ctx context.Context, containerID string, condition container.WaitCondition) (<-chan container.StateStatus, error) {
return nil, nil
}
func (m *MockBackend) ContainerCreateWorkdir(containerID string) error {
return nil
}
func (m *MockBackend) CopyOnBuild(containerID string, destPath string, srcRoot string, srcPath string, decompress bool) error {
return nil
}
func (m *MockBackend) GetImageAndReleasableLayer(ctx context.Context, refOrID string, opts backend.GetImageAndLayerOptions) (builder.Image, builder.ROLayer, error) {
if m.getImageFunc != nil {
return m.getImageFunc(refOrID)
}
return &mockImage{id: "theid"}, &mockLayer{}, nil
}
func (m *MockBackend) MakeImageCache(ctx context.Context, cacheFrom []string) (builder.ImageCache, error) {
if m.makeImageCacheFunc != nil {
return m.makeImageCacheFunc(cacheFrom), nil
}
return nil, nil
}
func (m *MockBackend) CreateImage(ctx context.Context, config []byte, parent string, layerDigest digest.Digest) (builder.Image, error) {
return &mockImage{id: "test"}, nil
}
type mockImage struct {
id string
config *container.Config
}
func (i *mockImage) ImageID() string {
return i.id
}
func (i *mockImage) RunConfig() *container.Config {
return i.config
}
func (i *mockImage) OperatingSystem() string {
return runtime.GOOS
}
func (i *mockImage) MarshalJSON() ([]byte, error) {
type rawImage mockImage
return json.Marshal(rawImage(*i)) //nolint:staticcheck
}
type mockImageCache struct {
getCacheFunc func(parentID string, cfg *container.Config) (string, error)
}
func (mic *mockImageCache) GetCache(parentID string, cfg *container.Config, _ ocispec.Platform) (string, error) {
if mic.getCacheFunc != nil {
return mic.getCacheFunc(parentID, cfg)
}
return "", nil
}
type mockLayer struct{}
func (l *mockLayer) ContentStoreDigest() digest.Digest {
return ""
}
func (l *mockLayer) Release() error {
return nil
}
func (l *mockLayer) NewRWLayer() (builder.RWLayer, error) {
return &mockRWLayer{}, nil
}
func (l *mockLayer) DiffID() layer.DiffID {
return "abcdef"
}
type mockRWLayer struct{}
func (l *mockRWLayer) Release() error {
return nil
}
func (l *mockRWLayer) Commit() (builder.ROLayer, error) {
return nil, nil
}
func (l *mockRWLayer) Root() string {
return ""
}

View File

@@ -0,0 +1,29 @@
package dockerfile
import (
"os"
"path/filepath"
"testing"
)
// createTestTempFile creates a temporary file within dir with specific contents and permissions.
// When an error occurs, it terminates the test
func createTestTempFile(t *testing.T, dir, filename, contents string, perm os.FileMode) string {
filePath := filepath.Join(dir, filename)
err := os.WriteFile(filePath, []byte(contents), perm)
if err != nil {
t.Fatalf("Error when creating %s file: %s", filename, err)
}
return filePath
}
// createTestSymlink creates a symlink file within dir which points to oldname
func createTestSymlink(t *testing.T, dir, filename, oldname string) string {
filePath := filepath.Join(dir, filename)
if err := os.Symlink(oldname, filePath); err != nil {
t.Fatalf("Error when creating %s symlink to %s: %s", filename, oldname, err)
}
return filePath
}

View File

@@ -0,0 +1,128 @@
package remotecontext
import (
"io"
"os"
"path/filepath"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/pkg/longpath"
"github.com/docker/docker/pkg/system"
"github.com/docker/docker/pkg/tarsum"
"github.com/moby/go-archive/chrootarchive"
"github.com/moby/go-archive/compression"
"github.com/moby/sys/symlink"
"github.com/pkg/errors"
)
type archiveContext struct {
root string
sums tarsum.FileInfoSums
}
func (c *archiveContext) Close() error {
return os.RemoveAll(c.root)
}
func convertPathError(err error, cleanpath string) error {
switch err := err.(type) {
case *os.PathError:
err.Path = cleanpath
case *system.XattrError:
err.Path = cleanpath
}
return err
}
type modifiableContext interface {
builder.Source
// Remove deletes the entry specified by `path`.
// It is usual for directory entries to delete all its subentries.
Remove(path string) error
}
// FromArchive returns a build source from a tar stream.
//
// It extracts the tar stream to a temporary folder that is deleted as soon as
// the Context is closed.
// As the extraction happens, a tarsum is calculated for every file, and the set of
// all those sums then becomes the source of truth for all operations on this Context.
//
// Closing tarStream has to be done by the caller.
func FromArchive(tarStream io.Reader) (builder.Source, error) {
root, err := longpath.MkdirTemp("", "docker-builder")
if err != nil {
return nil, err
}
// Assume local file system. Since it's coming from a tar file.
tsc := &archiveContext{root: root}
// Make sure we clean-up upon error. In the happy case the caller
// is expected to manage the clean-up
defer func() {
if err != nil {
tsc.Close()
}
}()
decompressedStream, err := compression.DecompressStream(tarStream)
if err != nil {
return nil, err
}
sum, err := tarsum.NewTarSum(decompressedStream, true, tarsum.Version1)
if err != nil {
return nil, err
}
err = chrootarchive.Untar(sum, root, nil)
if err != nil {
return nil, err
}
tsc.sums = sum.GetSums()
return tsc, nil
}
func (c *archiveContext) Root() string {
return c.root
}
func (c *archiveContext) Remove(path string) error {
_, fullpath, err := normalize(path, c.root)
if err != nil {
return err
}
return os.RemoveAll(fullpath)
}
func (c *archiveContext) Hash(path string) (string, error) {
cleanpath, fullpath, err := normalize(path, c.root)
if err != nil {
return "", err
}
rel, err := filepath.Rel(c.root, fullpath)
if err != nil {
return "", convertPathError(err, cleanpath)
}
// Use the checksum of the followed path(not the possible symlink) because
// this is the file that is actually copied.
if tsInfo := c.sums.GetFile(filepath.ToSlash(rel)); tsInfo != nil {
return tsInfo.Sum(), nil
}
// We set sum to path by default for the case where GetFile returns nil.
// The usual case is if relative path is empty.
return path, nil // backwards compat TODO: see if really needed
}
func normalize(path string, root string) (cleanPath, fullPath string, _ error) {
cleanPath = filepath.Clean(string(filepath.Separator) + path)[1:]
fullPath, err := symlink.FollowSymlinkInScope(filepath.Join(root, path), root)
if err != nil {
return "", "", errors.Wrapf(err, "forbidden path outside the build context: %s (%s)", path, cleanPath)
}
return cleanPath, fullPath, nil
}

View File

@@ -0,0 +1,187 @@
package remotecontext
import (
"bufio"
"context"
"fmt"
"io"
"os"
"path/filepath"
"runtime"
"strings"
"github.com/containerd/continuity/driver"
"github.com/containerd/log"
"github.com/docker/docker/api/types/backend"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/daemon/builder/remotecontext/urlutil"
"github.com/docker/docker/errdefs"
"github.com/moby/buildkit/frontend/dockerfile/parser"
"github.com/moby/patternmatcher"
"github.com/moby/patternmatcher/ignorefile"
"github.com/moby/sys/symlink"
"github.com/pkg/errors"
)
// ClientSessionRemote is identifier for client-session context transport
const ClientSessionRemote = "client-session"
// Detect returns a context and dockerfile from remote location or local
// archive.
func Detect(config backend.BuildConfig) (remote builder.Source, dockerfile *parser.Result, _ error) {
remoteURL := config.Options.RemoteContext
switch {
case remoteURL == "":
return newArchiveRemote(config.Source, config.Options.Dockerfile)
case remoteURL == ClientSessionRemote:
return nil, nil, errdefs.InvalidParameter(errors.New("experimental session with v1 builder is no longer supported, use builder version v2 (BuildKit) instead"))
case urlutil.IsGitURL(remoteURL):
return newGitRemote(remoteURL, config.Options.Dockerfile)
case urlutil.IsURL(remoteURL):
return newURLRemote(remoteURL, config.Options.Dockerfile, config.ProgressWriter.ProgressReaderFunc)
default:
return nil, nil, fmt.Errorf("remoteURL (%s) could not be recognized as URL", remoteURL)
}
}
func newArchiveRemote(rc io.ReadCloser, dockerfilePath string) (builder.Source, *parser.Result, error) {
defer rc.Close()
c, err := FromArchive(rc)
if err != nil {
return nil, nil, err
}
return withDockerfileFromContext(c.(modifiableContext), dockerfilePath)
}
func withDockerfileFromContext(c modifiableContext, dockerfilePath string) (builder.Source, *parser.Result, error) {
df, err := openAt(c, dockerfilePath)
if err != nil {
if errors.Is(err, os.ErrNotExist) {
if dockerfilePath == builder.DefaultDockerfileName {
lowercase := strings.ToLower(dockerfilePath)
if _, err := StatAt(c, lowercase); err == nil {
return withDockerfileFromContext(c, lowercase)
}
}
return nil, nil, errors.Errorf("Cannot locate specified Dockerfile: %s", dockerfilePath) // backwards compatible error
}
c.Close()
return nil, nil, err
}
res, err := readAndParseDockerfile(dockerfilePath, df)
if err != nil {
return nil, nil, err
}
df.Close()
if err := removeDockerfile(c, dockerfilePath); err != nil {
c.Close()
return nil, nil, err
}
return c, res, nil
}
func newGitRemote(gitURL string, dockerfilePath string) (builder.Source, *parser.Result, error) {
c, err := MakeGitContext(gitURL) // TODO: change this to NewLazySource
if err != nil {
return nil, nil, err
}
return withDockerfileFromContext(c.(modifiableContext), dockerfilePath)
}
func newURLRemote(url string, dockerfilePath string, progressReader func(in io.ReadCloser) io.ReadCloser) (builder.Source, *parser.Result, error) {
contentType, content, err := downloadRemote(url)
if err != nil {
return nil, nil, err
}
defer content.Close()
switch contentType {
case mimeTypeTextPlain:
res, err := parser.Parse(progressReader(content))
return nil, res, errdefs.InvalidParameter(err)
default:
source, err := FromArchive(progressReader(content))
if err != nil {
return nil, nil, err
}
return withDockerfileFromContext(source.(modifiableContext), dockerfilePath)
}
}
func removeDockerfile(c modifiableContext, filesToRemove ...string) error {
f, err := openAt(c, ".dockerignore")
// Note that a missing .dockerignore file isn't treated as an error
switch {
case os.IsNotExist(err):
return nil
case err != nil:
return err
}
excludes, err := ignorefile.ReadAll(f)
if err != nil {
f.Close()
return errors.Wrap(err, "error reading .dockerignore")
}
f.Close()
filesToRemove = append([]string{".dockerignore"}, filesToRemove...)
for _, fileToRemove := range filesToRemove {
if rm, _ := patternmatcher.MatchesOrParentMatches(fileToRemove, excludes); rm {
if err := c.Remove(fileToRemove); err != nil {
log.G(context.TODO()).Errorf("failed to remove %s: %v", fileToRemove, err)
}
}
}
return nil
}
func readAndParseDockerfile(name string, rc io.Reader) (*parser.Result, error) {
br := bufio.NewReader(rc)
if _, err := br.Peek(1); err != nil {
if err == io.EOF {
return nil, errdefs.InvalidParameter(errors.Errorf("the Dockerfile (%s) cannot be empty", name))
}
return nil, errors.Wrap(err, "unexpected error reading Dockerfile")
}
dockerfile, err := parser.Parse(br)
if err != nil {
return nil, errdefs.InvalidParameter(errors.Wrapf(err, "failed to parse %s", name))
}
return dockerfile, nil
}
func openAt(remote builder.Source, path string) (driver.File, error) {
fullPath, err := FullPath(remote, path)
if err != nil {
return nil, err
}
return os.Open(fullPath)
}
// StatAt is a helper for calling Stat on a path from a source
func StatAt(remote builder.Source, path string) (os.FileInfo, error) {
fullPath, err := FullPath(remote, path)
if err != nil {
return nil, err
}
return os.Stat(fullPath)
}
// FullPath is a helper for getting a full path for a path from a source
func FullPath(remote builder.Source, path string) (string, error) {
remoteRoot := remote.Root()
fullPath, err := symlink.FollowSymlinkInScope(filepath.Join(remoteRoot, path), remoteRoot)
if err != nil {
if runtime.GOOS == "windows" {
return "", fmt.Errorf("failed to resolve scoped path %s (%s): %s. Possible cause is a forbidden path outside the build context", path, fullPath, err)
}
return "", fmt.Errorf("forbidden path outside the build context: %s (%s)", path, fullPath) // backwards compat with old error
}
return fullPath, nil
}

View File

@@ -0,0 +1,116 @@
package remotecontext
import (
"errors"
"log"
"os"
"path/filepath"
"sort"
"testing"
"github.com/docker/docker/daemon/builder"
)
const (
dockerfileContents = "FROM busybox"
dockerignoreFilename = ".dockerignore"
testfileContents = "test"
)
const shouldStayFilename = "should_stay"
func extractFilenames(files []os.DirEntry) []string {
filenames := make([]string, len(files))
for i, file := range files {
filenames[i] = file.Name()
}
return filenames
}
func checkDirectory(t *testing.T, dir string, expectedFiles []string) {
files, err := os.ReadDir(dir)
if err != nil {
t.Fatalf("Could not read directory: %s", err)
}
if len(files) != len(expectedFiles) {
log.Fatalf("Directory should contain exactly %d file(s), got %d", len(expectedFiles), len(files))
}
filenames := extractFilenames(files)
sort.Strings(filenames)
sort.Strings(expectedFiles)
for i, filename := range filenames {
if filename != expectedFiles[i] {
t.Fatalf("File %s should be in the directory, got: %s", expectedFiles[i], filename)
}
}
}
func executeProcess(t *testing.T, contextDir string) {
modifiableCtx := &stubRemote{root: contextDir}
err := removeDockerfile(modifiableCtx, builder.DefaultDockerfileName)
if err != nil {
t.Fatalf("Error when executing Process: %s", err)
}
}
func TestProcessShouldRemoveDockerfileDockerignore(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, shouldStayFilename, testfileContents, 0o777)
createTestTempFile(t, contextDir, dockerignoreFilename, "Dockerfile\n.dockerignore", 0o777)
createTestTempFile(t, contextDir, builder.DefaultDockerfileName, dockerfileContents, 0o777)
executeProcess(t, contextDir)
checkDirectory(t, contextDir, []string{shouldStayFilename})
}
func TestProcessNoDockerignore(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, shouldStayFilename, testfileContents, 0o777)
createTestTempFile(t, contextDir, builder.DefaultDockerfileName, dockerfileContents, 0o777)
executeProcess(t, contextDir)
checkDirectory(t, contextDir, []string{shouldStayFilename, builder.DefaultDockerfileName})
}
func TestProcessShouldLeaveAllFiles(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, shouldStayFilename, testfileContents, 0o777)
createTestTempFile(t, contextDir, builder.DefaultDockerfileName, dockerfileContents, 0o777)
createTestTempFile(t, contextDir, dockerignoreFilename, "input1\ninput2", 0o777)
executeProcess(t, contextDir)
checkDirectory(t, contextDir, []string{shouldStayFilename, builder.DefaultDockerfileName, dockerignoreFilename})
}
// TODO: remove after moving to a separate pkg
type stubRemote struct {
root string
}
func (r *stubRemote) Hash(path string) (string, error) {
return "", errors.New("not implemented")
}
func (r *stubRemote) Root() string {
return r.root
}
func (r *stubRemote) Close() error {
return errors.New("not implemented")
}
func (r *stubRemote) Remove(p string) error {
return os.Remove(filepath.Join(r.root, p))
}

View File

@@ -0,0 +1,45 @@
package remotecontext
import (
"archive/tar"
"crypto/sha256"
"hash"
"os"
"github.com/docker/docker/pkg/tarsum"
"github.com/moby/go-archive"
)
// NewFileHash returns new hash that is used for the builder cache keys
func NewFileHash(path, name string, fi os.FileInfo) (hash.Hash, error) {
var link string
if fi.Mode()&os.ModeSymlink != 0 {
var err error
link, err = os.Readlink(path)
if err != nil {
return nil, err
}
}
hdr, err := archive.FileInfoHeader(name, fi, link)
if err != nil {
return nil, err
}
if err := archive.ReadSecurityXattrToTarHeader(path, hdr); err != nil {
return nil, err
}
tsh := &tarsumHash{hdr: hdr, Hash: sha256.New()}
tsh.Reset() // initialize header
return tsh, nil
}
type tarsumHash struct {
hash.Hash
hdr *tar.Header
}
// Reset resets the Hash to its initial state.
func (tsh *tarsumHash) Reset() {
// comply with hash.Hash and reset to the state hash had before any writes
tsh.Hash.Reset()
tarsum.WriteV1Header(tsh.hdr, tsh.Hash)
}

View File

@@ -0,0 +1,44 @@
package remotecontext
import (
"context"
"os"
"github.com/containerd/log"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/daemon/builder/remotecontext/git"
"github.com/moby/go-archive"
)
// MakeGitContext returns a Context from gitURL that is cloned in a temporary directory.
func MakeGitContext(gitURL string) (builder.Source, error) {
root, err := git.Clone(gitURL, git.WithIsolatedConfig(true))
if err != nil {
return nil, err
}
c, err := archive.Tar(root, archive.Uncompressed)
if err != nil {
return nil, err
}
defer func() {
if err := c.Close(); err != nil {
log.G(context.TODO()).WithFields(log.Fields{
"error": err,
"action": "MakeGitContext",
"module": "builder",
"url": gitURL,
}).Error("error while closing git context")
}
if err := os.RemoveAll(root); err != nil {
log.G(context.TODO()).WithFields(log.Fields{
"error": err,
"action": "MakeGitContext",
"module": "builder",
"url": gitURL,
}).Error("error while removing path and children of root")
}
}()
return FromArchive(c)
}

View File

@@ -0,0 +1,244 @@
package git
import (
"net/http"
"net/url"
"os"
"os/exec"
"path/filepath"
"strings"
"github.com/moby/sys/symlink"
"github.com/pkg/errors"
)
type gitRepo struct {
remote string
ref string
subdir string
isolateConfig bool
}
// CloneOption changes the behaviour of Clone().
type CloneOption func(*gitRepo)
// WithIsolatedConfig disables reading the user or system gitconfig files when
// performing Git operations.
func WithIsolatedConfig(v bool) CloneOption {
return func(gr *gitRepo) {
gr.isolateConfig = v
}
}
// Clone clones a repository into a newly created directory which
// will be under "docker-build-git"
func Clone(remoteURL string, opts ...CloneOption) (string, error) {
repo, err := parseRemoteURL(remoteURL)
if err != nil {
return "", err
}
for _, opt := range opts {
opt(&repo)
}
return repo.clone()
}
func (repo gitRepo) clone() (checkoutDir string, retErr error) {
fetch := fetchArgs(repo.remote, repo.ref)
root, err := os.MkdirTemp("", "docker-build-git")
if err != nil {
return "", err
}
defer func() {
if retErr != nil {
_ = os.RemoveAll(root)
}
}()
if out, err := repo.gitWithinDir(root, "init"); err != nil {
return "", errors.Wrapf(err, "failed to init repo at %s: %s", root, out)
}
// Add origin remote for compatibility with previous implementation that
// used "git clone" and also to make sure local refs are created for branches
if out, err := repo.gitWithinDir(root, "remote", "add", "origin", repo.remote); err != nil {
return "", errors.Wrapf(err, "failed add origin repo at %s: %s", repo.remote, out)
}
if output, err := repo.gitWithinDir(root, fetch...); err != nil {
return "", errors.Wrapf(err, "error fetching: %s", output)
}
checkoutDir, err = repo.checkout(root)
if err != nil {
return "", err
}
cmd := exec.Command("git", "submodule", "update", "--init", "--recursive", "--depth=1")
cmd.Dir = root
output, err := cmd.CombinedOutput()
if err != nil {
return "", errors.Wrapf(err, "error initializing submodules: %s", output)
}
return checkoutDir, nil
}
func parseRemoteURL(remoteURL string) (gitRepo, error) {
repo := gitRepo{}
if !isGitTransport(remoteURL) {
remoteURL = "https://" + remoteURL
}
if strings.HasPrefix(remoteURL, "git@") {
// git@.. is not an URL, so cannot be parsed as URL
var fragment string
repo.remote, fragment, _ = strings.Cut(remoteURL, "#")
repo.ref, repo.subdir = getRefAndSubdir(fragment)
} else {
u, err := url.Parse(remoteURL)
if err != nil {
return repo, err
}
repo.ref, repo.subdir = getRefAndSubdir(u.Fragment)
u.Fragment = ""
repo.remote = u.String()
}
if strings.HasPrefix(repo.ref, "-") {
return gitRepo{}, errors.Errorf("invalid refspec: %s", repo.ref)
}
return repo, nil
}
func getRefAndSubdir(fragment string) (ref string, subdir string) {
ref, subdir, _ = strings.Cut(fragment, ":")
if ref == "" {
ref = "master"
}
return ref, subdir
}
func fetchArgs(remoteURL string, ref string) []string {
args := []string{"fetch"}
if supportsShallowClone(remoteURL) {
args = append(args, "--depth", "1")
}
return append(args, "origin", "--", ref)
}
// Check if a given git URL supports a shallow git clone,
// i.e. it is a non-HTTP server or a smart HTTP server.
func supportsShallowClone(remoteURL string) bool {
if scheme := getScheme(remoteURL); scheme == "http" || scheme == "https" {
// Check if the HTTP server is smart
// Smart servers must correctly respond to a query for the git-upload-pack service
serviceURL := remoteURL + "/info/refs?service=git-upload-pack"
// Try a HEAD request and fallback to a Get request on error
res, err := http.Head(serviceURL) // #nosec G107
if res != nil && res.Body != nil {
defer res.Body.Close()
}
if err != nil || res.StatusCode != http.StatusOK {
res, err = http.Get(serviceURL) // #nosec G107
if err == nil {
res.Body.Close()
}
if err != nil || res.StatusCode != http.StatusOK {
// request failed
return false
}
}
if res.Header.Get("Content-Type") != "application/x-git-upload-pack-advertisement" {
// Fallback, not a smart server
return false
}
return true
}
// Non-HTTP protocols always support shallow clones
return true
}
func (repo gitRepo) checkout(root string) (string, error) {
// Try checking out by ref name first. This will work on branches and sets
// .git/HEAD to the current branch name
if output, err := repo.gitWithinDir(root, "checkout", repo.ref); err != nil {
// If checking out by branch name fails check out the last fetched ref
if _, err2 := repo.gitWithinDir(root, "checkout", "FETCH_HEAD"); err2 != nil {
return "", errors.Wrapf(err, "error checking out %s: %s", repo.ref, output)
}
}
if repo.subdir != "" {
newCtx, err := symlink.FollowSymlinkInScope(filepath.Join(root, repo.subdir), root)
if err != nil {
return "", errors.Wrapf(err, "error setting git context, %q not within git root", repo.subdir)
}
fi, err := os.Stat(newCtx)
if err != nil {
return "", err
}
if !fi.IsDir() {
return "", errors.Errorf("error setting git context, not a directory: %s", newCtx)
}
root = newCtx
}
return root, nil
}
func (repo gitRepo) gitWithinDir(dir string, args ...string) ([]byte, error) {
args = append([]string{"-c", "protocol.file.allow=never"}, args...) // Block sneaky repositories from using repos from the filesystem as submodules.
cmd := exec.Command("git", args...)
cmd.Dir = dir
// Disable unsafe remote protocols.
cmd.Env = append(os.Environ(), "GIT_PROTOCOL_FROM_USER=0")
if repo.isolateConfig {
cmd.Env = append(cmd.Env,
"GIT_CONFIG_NOSYSTEM=1", // Disable reading from system gitconfig.
"HOME=/dev/null", // Disable reading from user gitconfig.
)
}
return cmd.CombinedOutput()
}
// isGitTransport returns true if the provided str is a git transport by inspecting
// the prefix of the string for known protocols used in git.
func isGitTransport(str string) bool {
if strings.HasPrefix(str, "git@") {
return true
}
switch getScheme(str) {
case "git", "http", "https", "ssh":
return true
}
return false
}
// getScheme returns addresses' scheme in lowercase, or an empty
// string in case address is an invalid URL.
func getScheme(address string) string {
u, err := url.Parse(address)
if err != nil {
return ""
}
return u.Scheme
}

View File

@@ -0,0 +1,381 @@
package git
import (
"bytes"
"fmt"
"net/http"
"net/http/cgi"
"net/http/httptest"
"net/url"
"os"
"os/exec"
"path/filepath"
"runtime"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func TestParseRemoteURL(t *testing.T) {
tests := []struct {
doc string
url string
expected gitRepo
}{
{
doc: "git scheme uppercase, no url-fragment",
url: "GIT://github.com/user/repo.git",
expected: gitRepo{
remote: "git://github.com/user/repo.git",
ref: "master",
},
},
{
doc: "git scheme, no url-fragment",
url: "git://github.com/user/repo.git",
expected: gitRepo{
remote: "git://github.com/user/repo.git",
ref: "master",
},
},
{
doc: "git scheme, with url-fragment",
url: "git://github.com/user/repo.git#mybranch:mydir/mysubdir/",
expected: gitRepo{
remote: "git://github.com/user/repo.git",
ref: "mybranch",
subdir: "mydir/mysubdir/",
},
},
{
doc: "https scheme, no url-fragment",
url: "https://github.com/user/repo.git",
expected: gitRepo{
remote: "https://github.com/user/repo.git",
ref: "master",
},
},
{
doc: "https scheme, with url-fragment",
url: "https://github.com/user/repo.git#mybranch:mydir/mysubdir/",
expected: gitRepo{
remote: "https://github.com/user/repo.git",
ref: "mybranch",
subdir: "mydir/mysubdir/",
},
},
{
doc: "git@, no url-fragment",
url: "git@github.com:user/repo.git",
expected: gitRepo{
remote: "git@github.com:user/repo.git",
ref: "master",
},
},
{
doc: "git@, with url-fragment",
url: "git@github.com:user/repo.git#mybranch:mydir/mysubdir/",
expected: gitRepo{
remote: "git@github.com:user/repo.git",
ref: "mybranch",
subdir: "mydir/mysubdir/",
},
},
{
doc: "ssh, no url-fragment",
url: "ssh://github.com/user/repo.git",
expected: gitRepo{
remote: "ssh://github.com/user/repo.git",
ref: "master",
},
},
{
doc: "ssh, with url-fragment",
url: "ssh://github.com/user/repo.git#mybranch:mydir/mysubdir/",
expected: gitRepo{
remote: "ssh://github.com/user/repo.git",
ref: "mybranch",
subdir: "mydir/mysubdir/",
},
},
{
doc: "ssh, with url-fragment and user",
url: "ssh://foo%40barcorp.com@github.com/user/repo.git#mybranch:mydir/mysubdir/",
expected: gitRepo{
remote: "ssh://foo%40barcorp.com@github.com/user/repo.git",
ref: "mybranch",
subdir: "mydir/mysubdir/",
},
},
}
for _, tc := range tests {
t.Run(tc.doc, func(t *testing.T) {
repo, err := parseRemoteURL(tc.url)
assert.NilError(t, err)
assert.Check(t, is.DeepEqual(tc.expected, repo, cmp.AllowUnexported(gitRepo{})))
})
}
}
func TestCloneArgsSmartHttp(t *testing.T) {
mux := http.NewServeMux()
server := httptest.NewServer(mux)
serverURL, _ := url.Parse(server.URL)
serverURL.Path = "/repo.git"
mux.HandleFunc("/repo.git/info/refs", func(w http.ResponseWriter, r *http.Request) {
q := r.URL.Query().Get("service")
w.Header().Set("Content-Type", fmt.Sprintf("application/x-%s-advertisement", q))
})
args := fetchArgs(serverURL.String(), "master")
exp := []string{"fetch", "--depth", "1", "origin", "--", "master"}
assert.Check(t, is.DeepEqual(exp, args))
}
func TestCloneArgsDumbHttp(t *testing.T) {
mux := http.NewServeMux()
server := httptest.NewServer(mux)
serverURL, _ := url.Parse(server.URL)
serverURL.Path = "/repo.git"
mux.HandleFunc("/repo.git/info/refs", func(w http.ResponseWriter, r *http.Request) {
w.Header().Set("Content-Type", "text/plain")
})
args := fetchArgs(serverURL.String(), "master")
exp := []string{"fetch", "origin", "--", "master"}
assert.Check(t, is.DeepEqual(exp, args))
}
func TestCloneArgsGit(t *testing.T) {
args := fetchArgs("git://github.com/docker/docker", "master")
exp := []string{"fetch", "--depth", "1", "origin", "--", "master"}
assert.Check(t, is.DeepEqual(exp, args))
}
func gitGetConfig(name string) string {
b, err := gitRepo{}.gitWithinDir("", "config", "--get", name)
if err != nil {
// since we are interested in empty or non empty string,
// we can safely ignore the err here.
return ""
}
return strings.TrimSpace(string(b))
}
func TestCheckoutGit(t *testing.T) {
root := t.TempDir()
gitpath, err := exec.LookPath("git")
assert.NilError(t, err)
gitversion, _ := exec.Command(gitpath, "version").CombinedOutput()
t.Logf("%s", gitversion) // E.g. "git version 2.30.2"
// Serve all repositories under root using the Smart HTTP protocol so
// they can be cloned. The Dumb HTTP protocol is incompatible with
// shallow cloning but we unconditionally shallow-clone submodules, and
// we explicitly disable the file protocol.
// (Another option would be to use `git daemon` and the Git protocol,
// but that listens on a fixed port number which is a recipe for
// disaster in CI. Funnily enough, `git daemon --port=0` works but there
// is no easy way to discover which port got picked!)
// Associate git-http-backend logs with the current (sub)test.
// Incompatible with parallel subtests.
currentSubtest := t
githttp := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
var logs bytes.Buffer
(&cgi.Handler{
Path: gitpath,
Args: []string{"http-backend"},
Dir: root,
Env: []string{
"GIT_PROJECT_ROOT=" + root,
"GIT_HTTP_EXPORT_ALL=1",
},
Stderr: &logs,
}).ServeHTTP(w, r)
if logs.Len() == 0 {
return
}
for {
line, err := logs.ReadString('\n')
currentSubtest.Log("git-http-backend: " + line)
if err != nil {
break
}
}
})
server := httptest.NewServer(&githttp)
defer server.Close()
eol := "\n"
autocrlf := gitGetConfig("core.autocrlf")
switch autocrlf {
case "true":
eol = "\r\n"
case "false", "input", "":
// accepted values
default:
t.Logf(`unknown core.autocrlf value: "%s"`, autocrlf)
}
must := func(out []byte, err error) {
t.Helper()
if len(out) > 0 {
t.Logf("%s", out)
}
assert.NilError(t, err)
}
gitDir := filepath.Join(root, "repo")
must(gitRepo{}.gitWithinDir(root, "-c", "init.defaultBranch=master", "init", gitDir))
must(gitRepo{}.gitWithinDir(gitDir, "config", "user.email", "test@docker.com"))
must(gitRepo{}.gitWithinDir(gitDir, "config", "user.name", "Docker test"))
assert.NilError(t, os.WriteFile(filepath.Join(gitDir, "Dockerfile"), []byte("FROM scratch"), 0o644))
subDir := filepath.Join(gitDir, "subdir")
assert.NilError(t, os.Mkdir(subDir, 0o755))
assert.NilError(t, os.WriteFile(filepath.Join(subDir, "Dockerfile"), []byte("FROM scratch\nEXPOSE 5000"), 0o644))
if runtime.GOOS != "windows" {
assert.NilError(t, os.Symlink("../subdir", filepath.Join(gitDir, "parentlink")))
assert.NilError(t, os.Symlink("/subdir", filepath.Join(gitDir, "absolutelink")))
}
must(gitRepo{}.gitWithinDir(gitDir, "add", "-A"))
must(gitRepo{}.gitWithinDir(gitDir, "commit", "-am", "First commit"))
must(gitRepo{}.gitWithinDir(gitDir, "checkout", "-b", "test"))
assert.NilError(t, os.WriteFile(filepath.Join(gitDir, "Dockerfile"), []byte("FROM scratch\nEXPOSE 3000"), 0o644))
assert.NilError(t, os.WriteFile(filepath.Join(subDir, "Dockerfile"), []byte("FROM busybox\nEXPOSE 5000"), 0o644))
must(gitRepo{}.gitWithinDir(gitDir, "add", "-A"))
must(gitRepo{}.gitWithinDir(gitDir, "commit", "-am", "Branch commit"))
must(gitRepo{}.gitWithinDir(gitDir, "checkout", "master"))
// set up submodule
subrepoDir := filepath.Join(root, "subrepo")
must(gitRepo{}.gitWithinDir(root, "-c", "init.defaultBranch=master", "init", subrepoDir))
must(gitRepo{}.gitWithinDir(subrepoDir, "config", "user.email", "test@docker.com"))
must(gitRepo{}.gitWithinDir(subrepoDir, "config", "user.name", "Docker test"))
assert.NilError(t, os.WriteFile(filepath.Join(subrepoDir, "subfile"), []byte("subcontents"), 0o644))
must(gitRepo{}.gitWithinDir(subrepoDir, "add", "-A"))
must(gitRepo{}.gitWithinDir(subrepoDir, "commit", "-am", "Subrepo initial"))
must(gitRepo{}.gitWithinDir(gitDir, "submodule", "add", server.URL+"/subrepo", "sub"))
must(gitRepo{}.gitWithinDir(gitDir, "add", "-A"))
must(gitRepo{}.gitWithinDir(gitDir, "commit", "-am", "With submodule"))
type singleCase struct {
frag string
exp string
fail bool
submodule bool
}
cases := []singleCase{
{"", "FROM scratch", false, true},
{"master", "FROM scratch", false, true},
{":subdir", "FROM scratch" + eol + "EXPOSE 5000", false, false},
{":nosubdir", "", true, false}, // missing directory error
{":Dockerfile", "", true, false}, // not a directory error
{"master:nosubdir", "", true, false},
{"master:subdir", "FROM scratch" + eol + "EXPOSE 5000", false, false},
{"master:../subdir", "", true, false},
{"test", "FROM scratch" + eol + "EXPOSE 3000", false, false},
{"test:", "FROM scratch" + eol + "EXPOSE 3000", false, false},
{"test:subdir", "FROM busybox" + eol + "EXPOSE 5000", false, false},
}
if runtime.GOOS != "windows" {
// Windows GIT (2.7.1 x64) does not support parentlink/absolutelink. Sample output below
// git --work-tree .\repo --git-dir .\repo\.git add -A
// error: readlink("absolutelink"): Function not implemented
// error: unable to index file absolutelink
// fatal: adding files failed
cases = append(cases, singleCase{frag: "master:absolutelink", exp: "FROM scratch" + eol + "EXPOSE 5000", fail: false})
cases = append(cases, singleCase{frag: "master:parentlink", exp: "FROM scratch" + eol + "EXPOSE 5000", fail: false})
}
for _, c := range cases {
t.Run(c.frag, func(t *testing.T) {
currentSubtest = t
ref, subdir := getRefAndSubdir(c.frag)
r, err := gitRepo{remote: server.URL + "/repo", ref: ref, subdir: subdir}.clone()
if c.fail {
assert.Check(t, is.ErrorContains(err, ""))
return
}
assert.NilError(t, err)
defer os.RemoveAll(r)
if c.submodule {
b, err := os.ReadFile(filepath.Join(r, "sub/subfile"))
assert.NilError(t, err)
assert.Check(t, is.Equal("subcontents", string(b)))
} else {
_, err := os.Stat(filepath.Join(r, "sub/subfile"))
assert.ErrorContains(t, err, "")
assert.Assert(t, os.IsNotExist(err))
}
b, err := os.ReadFile(filepath.Join(r, "Dockerfile"))
assert.NilError(t, err)
assert.Check(t, is.Equal(c.exp, string(b)))
})
}
}
func TestValidGitTransport(t *testing.T) {
gitUrls := []string{
"git://github.com/docker/docker",
"git@github.com:docker/docker.git",
"git@bitbucket.org:atlassianlabs/atlassian-docker.git",
"https://github.com/docker/docker.git",
"http://github.com/docker/docker.git",
"http://github.com/docker/docker.git#branch",
"http://github.com/docker/docker.git#:dir",
}
incompleteGitUrls := []string{
"github.com/docker/docker",
}
for _, u := range gitUrls {
if !isGitTransport(u) {
t.Fatalf("%q should be detected as valid Git prefix", u)
}
}
for _, u := range incompleteGitUrls {
if isGitTransport(u) {
t.Fatalf("%q should not be detected as valid Git prefix", u)
}
}
}
func TestGitInvalidRef(t *testing.T) {
gitUrls := []string{
"git://github.com/moby/moby#--foo bar",
"git@github.com/moby/moby#--upload-pack=sleep;:",
"git@g.com:a/b.git#-B",
"git@g.com:a/b.git#with space",
}
for _, u := range gitUrls {
_, err := Clone(u)
assert.Assert(t, err != nil)
// On Windows, git has different case for the "invalid refspec" error,
// so we can't use ErrorContains.
assert.Check(t, is.Contains(strings.ToLower(err.Error()), "invalid refspec"))
}
}

View File

@@ -0,0 +1,91 @@
package remotecontext
import (
"encoding/hex"
"os"
"path/filepath"
"github.com/docker/docker/daemon/builder"
"github.com/docker/docker/pkg/pools"
"github.com/pkg/errors"
)
// NewLazySource creates a new LazyContext. LazyContext defines a hashed build
// context based on a root directory. Individual files are hashed first time
// they are asked. It is not safe to call methods of LazyContext concurrently.
func NewLazySource(root string) (builder.Source, error) {
return &lazySource{
root: root,
sums: make(map[string]string),
}, nil
}
type lazySource struct {
root string
sums map[string]string
}
func (c *lazySource) Root() string {
return c.root
}
func (c *lazySource) Close() error {
return nil
}
func (c *lazySource) Hash(path string) (string, error) {
cleanPath, fullPath, err := normalize(path, c.root)
if err != nil {
return "", err
}
relPath, err := filepath.Rel(c.root, fullPath)
if err != nil {
return "", errors.WithStack(convertPathError(err, cleanPath))
}
fi, err := os.Lstat(fullPath)
if err != nil {
// Backwards compatibility: a missing file returns a path as hash.
// This is reached in the case of a broken symlink.
return relPath, nil
}
sum, ok := c.sums[relPath]
if !ok {
sum, err = c.prepareHash(relPath, fi)
if err != nil {
return "", err
}
}
return sum, nil
}
func (c *lazySource) prepareHash(relPath string, fi os.FileInfo) (string, error) {
p := filepath.Join(c.root, relPath)
h, err := NewFileHash(p, relPath, fi)
if err != nil {
return "", errors.Wrapf(err, "failed to create hash for %s", relPath)
}
if fi.Mode().IsRegular() && fi.Size() > 0 {
f, err := os.Open(p)
if err != nil {
return "", errors.Wrapf(err, "failed to open %s", relPath)
}
defer f.Close()
if _, err := pools.Copy(h, f); err != nil {
return "", errors.Wrapf(err, "failed to copy file data for %s", relPath)
}
}
sum := hex.EncodeToString(h.Sum(nil))
c.sums[relPath] = sum
return sum, nil
}
// Rel is an alias for [filepath.Rel].
//
// Deprecated: use [filepath.Rel] instead; this function is no longer used and will be removed in the next release.
func Rel(basepath string, targpath string) (string, error) {
return filepath.Rel(basepath, targpath)
}

View File

@@ -0,0 +1,26 @@
package remotecontext
import (
"mime"
"net/http"
)
// MIME content types.
const (
mimeTypeTextPlain = "text/plain"
mimeTypeOctetStream = "application/octet-stream"
)
// detectContentType returns a best guess representation of the MIME
// content type for the bytes at c. The value detected by
// http.DetectContentType is guaranteed not be nil, defaulting to
// application/octet-stream when a better guess cannot be made. The
// result of this detection is then run through mime.ParseMediaType()
// which separates the actual MIME string from any parameters.
func detectContentType(c []byte) (string, error) {
contentType, _, err := mime.ParseMediaType(http.DetectContentType(c))
if err != nil {
return "", err
}
return contentType, nil
}

View File

@@ -0,0 +1,16 @@
package remotecontext
import (
"testing"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
func TestDetectContentType(t *testing.T) {
input := []byte("That is just a plain text")
contentType, err := detectContentType(input)
assert.NilError(t, err)
assert.Check(t, is.Equal(mimeTypeTextPlain, contentType))
}

View File

@@ -0,0 +1,128 @@
package remotecontext
import (
"bytes"
"fmt"
"io"
"net"
"net/http"
"net/url"
"github.com/docker/docker/errdefs"
"github.com/docker/docker/internal/lazyregexp"
"github.com/docker/docker/pkg/ioutils"
"github.com/pkg/errors"
)
// When downloading remote contexts, limit the amount (in bytes)
// to be read from the response body in order to detect its Content-Type
const maxPreambleLength = 100
const acceptableRemoteMIME = `(?:application/(?:(?:x\-)?tar|octet\-stream|((?:x\-)?(?:gzip|bzip2?|xz)))|(?:text/plain))`
var mimeRe = lazyregexp.New(acceptableRemoteMIME)
// downloadRemote context from a url and returns it, along with the parsed content type
func downloadRemote(remoteURL string) (string, io.ReadCloser, error) {
response, err := GetWithStatusError(remoteURL)
if err != nil {
return "", nil, errors.Wrapf(err, "error downloading remote context %s", remoteURL)
}
contentType, contextReader, err := inspectResponse(
response.Header.Get("Content-Type"),
response.Body,
response.ContentLength)
if err != nil {
response.Body.Close()
return "", nil, errors.Wrapf(err, "error detecting content type for remote %s", remoteURL)
}
return contentType, ioutils.NewReadCloserWrapper(contextReader, response.Body.Close), nil
}
// GetWithStatusError does an http.Get() and returns an error if the
// status code is 4xx or 5xx.
func GetWithStatusError(address string) (*http.Response, error) {
resp, err := http.Get(address) // #nosec G107 -- ignore G107: Potential HTTP request made with variable url
if err != nil {
if uErr, ok := err.(*url.Error); ok {
if dErr, ok := uErr.Err.(*net.DNSError); ok && !dErr.IsTimeout {
return nil, errdefs.NotFound(err)
}
}
return nil, errdefs.System(err)
}
if resp.StatusCode < http.StatusBadRequest {
return resp, nil
}
msg := fmt.Sprintf("failed to GET %s with status %s", address, resp.Status)
body, err := io.ReadAll(resp.Body)
_ = resp.Body.Close()
if err != nil {
return nil, errdefs.System(errors.New(msg + ": error reading body"))
}
msg += ": " + string(bytes.TrimSpace(body))
switch resp.StatusCode {
case http.StatusNotFound:
return nil, errdefs.NotFound(errors.New(msg))
case http.StatusBadRequest:
return nil, errdefs.InvalidParameter(errors.New(msg))
case http.StatusUnauthorized:
return nil, errdefs.Unauthorized(errors.New(msg))
case http.StatusForbidden:
return nil, errdefs.Forbidden(errors.New(msg))
default:
return nil, errdefs.Unknown(errors.New(msg))
}
}
// inspectResponse looks into the http response data at r to determine whether its
// content-type is on the list of acceptable content types for remote build contexts.
// This function returns:
// - a string representation of the detected content-type
// - an io.Reader for the response body
// - an error value which will be non-nil either when something goes wrong while
// reading bytes from r or when the detected content-type is not acceptable.
func inspectResponse(ct string, r io.Reader, clen int64) (string, io.Reader, error) {
plen := clen
if plen <= 0 || plen > maxPreambleLength {
plen = maxPreambleLength
}
preamble := make([]byte, plen)
rlen, err := r.Read(preamble)
if rlen == 0 {
return ct, r, errors.New("empty response")
}
if err != nil && !errors.Is(err, io.EOF) {
return ct, r, err
}
preambleR := bytes.NewReader(preamble[:rlen])
bodyReader := io.MultiReader(preambleR, r)
// Some web servers will use application/octet-stream as the default
// content type for files without an extension (e.g. 'Dockerfile')
// so if we receive this value we better check for text content
contentType := ct
if ct == "" || ct == mimeTypeOctetStream {
contentType, err = detectContentType(preamble)
if err != nil {
return contentType, bodyReader, err
}
}
contentType = selectAcceptableMIME(contentType)
var cterr error
if contentType == "" {
cterr = fmt.Errorf("unsupported Content-Type %q", ct)
contentType = ct
}
return contentType, bodyReader, cterr
}
func selectAcceptableMIME(ct string) string {
return mimeRe.FindString(ct)
}

View File

@@ -0,0 +1,239 @@
package remotecontext
import (
"bytes"
"io"
"net/http"
"net/http/httptest"
"net/url"
"testing"
"github.com/docker/docker/daemon/builder"
"gotest.tools/v3/assert"
is "gotest.tools/v3/assert/cmp"
)
var binaryContext = []byte{0xFD, 0x37, 0x7A, 0x58, 0x5A, 0x00} // xz magic
func TestSelectAcceptableMIME(t *testing.T) {
validMimeStrings := []string{
"application/x-bzip2",
"application/bzip2",
"application/gzip",
"application/x-gzip",
"application/x-xz",
"application/xz",
"application/tar",
"application/x-tar",
"application/octet-stream",
"text/plain",
}
invalidMimeStrings := []string{
"",
"application/octet",
"application/json",
}
for _, m := range invalidMimeStrings {
if selectAcceptableMIME(m) != "" {
t.Fatalf("Should not have accepted %q", m)
}
}
for _, m := range validMimeStrings {
if str := selectAcceptableMIME(m); str == "" {
t.Fatalf("Should have accepted %q", m)
}
}
}
func TestInspectEmptyResponse(t *testing.T) {
ct := "application/octet-stream"
br := io.NopCloser(bytes.NewReader([]byte("")))
contentType, bReader, err := inspectResponse(ct, br, 0)
if err == nil {
t.Fatal("Should have generated an error for an empty response")
}
if contentType != "application/octet-stream" {
t.Fatalf("Content type should be 'application/octet-stream' but is %q", contentType)
}
body, err := io.ReadAll(bReader)
if err != nil {
t.Fatal(err)
}
if len(body) != 0 {
t.Fatal("response body should remain empty")
}
}
func TestInspectResponseBinary(t *testing.T) {
ct := "application/octet-stream"
br := io.NopCloser(bytes.NewReader(binaryContext))
contentType, bReader, err := inspectResponse(ct, br, int64(len(binaryContext)))
if err != nil {
t.Fatal(err)
}
if contentType != "application/octet-stream" {
t.Fatalf("Content type should be 'application/octet-stream' but is %q", contentType)
}
body, err := io.ReadAll(bReader)
if err != nil {
t.Fatal(err)
}
if len(body) != len(binaryContext) {
t.Fatalf("Wrong response size %d, should be == len(binaryContext)", len(body))
}
for i := range body {
if body[i] != binaryContext[i] {
t.Fatalf("Corrupted response body at byte index %d", i)
}
}
}
func TestResponseUnsupportedContentType(t *testing.T) {
content := []byte(dockerfileContents)
ct := "application/json"
br := io.NopCloser(bytes.NewReader(content))
contentType, bReader, err := inspectResponse(ct, br, int64(len(dockerfileContents)))
if err == nil {
t.Fatal("Should have returned an error on content-type 'application/json'")
}
if contentType != ct {
t.Fatalf("Should not have altered content-type: orig: %s, altered: %s", ct, contentType)
}
body, err := io.ReadAll(bReader)
if err != nil {
t.Fatal(err)
}
if string(body) != dockerfileContents {
t.Fatalf("Corrupted response body %s", body)
}
}
func TestInspectResponseTextSimple(t *testing.T) {
content := []byte(dockerfileContents)
ct := "text/plain"
br := io.NopCloser(bytes.NewReader(content))
contentType, bReader, err := inspectResponse(ct, br, int64(len(content)))
if err != nil {
t.Fatal(err)
}
if contentType != "text/plain" {
t.Fatalf("Content type should be 'text/plain' but is %q", contentType)
}
body, err := io.ReadAll(bReader)
if err != nil {
t.Fatal(err)
}
if string(body) != dockerfileContents {
t.Fatalf("Corrupted response body %s", body)
}
}
func TestInspectResponseEmptyContentType(t *testing.T) {
content := []byte(dockerfileContents)
br := io.NopCloser(bytes.NewReader(content))
contentType, bodyReader, err := inspectResponse("", br, int64(len(content)))
if err != nil {
t.Fatal(err)
}
if contentType != "text/plain" {
t.Fatalf("Content type should be 'text/plain' but is %q", contentType)
}
body, err := io.ReadAll(bodyReader)
if err != nil {
t.Fatal(err)
}
if string(body) != dockerfileContents {
t.Fatalf("Corrupted response body %s", body)
}
}
func TestUnknownContentLength(t *testing.T) {
content := []byte(dockerfileContents)
ct := "text/plain"
br := io.NopCloser(bytes.NewReader(content))
contentType, bReader, err := inspectResponse(ct, br, -1)
if err != nil {
t.Fatal(err)
}
if contentType != "text/plain" {
t.Fatalf("Content type should be 'text/plain' but is %q", contentType)
}
body, err := io.ReadAll(bReader)
if err != nil {
t.Fatal(err)
}
if string(body) != dockerfileContents {
t.Fatalf("Corrupted response body %s", body)
}
}
func TestDownloadRemote(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, builder.DefaultDockerfileName, dockerfileContents, 0644)
mux := http.NewServeMux()
server := httptest.NewServer(mux)
serverURL, _ := url.Parse(server.URL)
serverURL.Path = "/" + builder.DefaultDockerfileName
remoteURL := serverURL.String()
mux.Handle("/", http.FileServer(http.Dir(contextDir)))
contentType, content, err := downloadRemote(remoteURL)
assert.NilError(t, err)
assert.Check(t, is.Equal(mimeTypeTextPlain, contentType))
raw, err := io.ReadAll(content)
assert.NilError(t, err)
assert.Check(t, is.Equal(dockerfileContents, string(raw)))
}
func TestGetWithStatusError(t *testing.T) {
testcases := []struct {
err error
statusCode int
expectedErr string
expectedBody string
}{
{
statusCode: 200,
expectedBody: "THE BODY",
},
{
statusCode: 400,
expectedErr: "with status 400 Bad Request: broke",
expectedBody: "broke",
},
}
for _, testcase := range testcases {
ts := httptest.NewServer(
http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
buffer := bytes.NewBufferString(testcase.expectedBody)
w.WriteHeader(testcase.statusCode)
w.Write(buffer.Bytes())
}),
)
defer ts.Close()
response, err := GetWithStatusError(ts.URL)
if testcase.expectedErr == "" {
assert.NilError(t, err)
body, err := readBody(response.Body)
assert.NilError(t, err)
assert.Check(t, is.Contains(string(body), testcase.expectedBody))
} else {
assert.Check(t, is.ErrorContains(err, testcase.expectedErr))
}
}
}
func readBody(b io.ReadCloser) ([]byte, error) {
defer b.Close()
return io.ReadAll(b)
}

View File

@@ -0,0 +1,141 @@
package remotecontext
import (
"os"
"path/filepath"
"testing"
"github.com/docker/docker/daemon/builder"
"github.com/moby/go-archive"
"github.com/moby/sys/reexec"
"github.com/pkg/errors"
"gotest.tools/v3/skip"
)
const (
filename = "test"
contents = "contents test"
)
func TestMain(m *testing.M) {
if reexec.Init() {
return
}
os.Exit(m.Run())
}
func TestCloseRootDirectory(t *testing.T) {
src := makeTestArchiveContext(t, t.TempDir())
err := src.Close()
if err != nil {
t.Fatalf("Error while executing Close: %s", err)
}
_, err = os.Stat(src.Root())
if !errors.Is(err, os.ErrNotExist) {
t.Fatal("Directory should not exist at this point")
}
}
func TestHashFile(t *testing.T) {
contextDir := t.TempDir()
createTestTempFile(t, contextDir, filename, contents, 0o755)
tarSum := makeTestArchiveContext(t, contextDir)
sum, err := tarSum.Hash(filename)
if err != nil {
t.Fatalf("Error when executing Stat: %s", err)
}
if sum == "" {
t.Fatalf("Hash returned empty sum")
}
expected := "55dfeb344351ab27f59aa60ebb0ed12025a2f2f4677bf77d26ea7a671274a9ca"
if actual := sum; expected != actual {
t.Fatalf("invalid checksum. expected %s, got %s", expected, actual)
}
}
func TestHashSubdir(t *testing.T) {
contextDir := t.TempDir()
contextSubdir := filepath.Join(contextDir, "builder-tarsum-test-subdir")
err := os.Mkdir(contextSubdir, 0o755)
if err != nil {
t.Fatalf("Failed to make directory: %s", contextSubdir)
}
testFilename := createTestTempFile(t, contextSubdir, filename, contents, 0o755)
tarSum := makeTestArchiveContext(t, contextDir)
relativePath, err := filepath.Rel(contextDir, testFilename)
if err != nil {
t.Fatalf("Error when getting relative path: %s", err)
}
sum, err := tarSum.Hash(relativePath)
if err != nil {
t.Fatalf("Error when executing Stat: %s", err)
}
if sum == "" {
t.Fatalf("Hash returned empty sum")
}
expected := "74a3326b8e766ce63a8e5232f22e9dd895be647fb3ca7d337e5e0a9b3da8ef28"
if actual := sum; expected != actual {
t.Fatalf("invalid checksum. expected %s, got %s", expected, actual)
}
}
func TestRemoveDirectory(t *testing.T) {
contextDir := t.TempDir()
contextSubdir, err := os.MkdirTemp(contextDir, "builder-tarsum-test-subdir")
if err != nil {
t.Fatalf("Error when creating directory %s with prefix %s: %s", contextDir, "builder-tarsum-test-subdir", err)
}
relativePath, err := filepath.Rel(contextDir, contextSubdir)
if err != nil {
t.Fatalf("Error when getting relative path: %s", err)
}
src := makeTestArchiveContext(t, contextDir)
_, err = os.Stat(filepath.Join(src.Root(), relativePath))
if err != nil {
t.Fatalf("Statting %s shouldn't fail: %+v", relativePath, err)
}
tarSum := src.(modifiableContext)
err = tarSum.Remove(relativePath)
if err != nil {
t.Fatalf("Error when executing Remove: %s", err)
}
_, err = os.Stat(filepath.Join(src.Root(), relativePath))
if !errors.Is(err, os.ErrNotExist) {
t.Fatalf("Directory should not exist at this point: %+v ", err)
}
}
func makeTestArchiveContext(t *testing.T, dir string) builder.Source {
skip.If(t, os.Getuid() != 0, "skipping test that requires root")
tarStream, err := archive.Tar(dir, archive.Uncompressed)
if err != nil {
t.Fatalf("error: %s", err)
}
defer tarStream.Close()
tarSum, err := FromArchive(tarStream)
if err != nil {
t.Fatalf("Error when executing FromArchive: %s", err)
}
return tarSum
}

View File

@@ -0,0 +1,87 @@
// Package urlutil provides helper function to check if a given build-context
// location should be considered a URL or a remote Git repository.
//
// This package is specifically written for use with docker build contexts, and
// should not be used as a general-purpose utility.
package urlutil
import (
"strings"
"github.com/docker/docker/internal/lazyregexp"
)
// urlPathWithFragmentSuffix matches fragments to use as Git reference and build
// context from the Git repository. See IsGitURL for details.
var urlPathWithFragmentSuffix = lazyregexp.New(`\.git(?:#.+)?$`)
// IsURL returns true if the provided str is an HTTP(S) URL by checking if it
// has a http:// or https:// scheme. No validation is performed to verify if the
// URL is well-formed.
func IsURL(str string) bool {
return strings.HasPrefix(str, "https://") || strings.HasPrefix(str, "http://")
}
// IsGitURL returns true if the provided str is a remote git repository "URL".
//
// This function only performs a rudimentary check (no validation is performed
// to ensure the URL is well-formed), and is written specifically for use with
// docker build, with some logic for backward compatibility with older versions
// of docker: do not use this function as a general-purpose utility.
//
// The following patterns are considered to be a Git URL:
//
// - https://(.*).git(?:#.+)?$ git repository URL with optional fragment, as known to be used by GitHub and GitLab.
// - http://(.*).git(?:#.+)?$ same, but non-TLS
// - git://(.*) URLs using git:// scheme
// - git@(.*)
// - github.com/ see description below
//
// The github.com/ prefix is a special case used to treat context-paths
// starting with "github.com/" as a git URL if the given path does not
// exist locally. The "github.com/" prefix is kept for backward compatibility,
// and is a legacy feature.
//
// Going forward, no additional prefixes should be added, and users should
// be encouraged to use explicit URLs (https://github.com/user/repo.git) instead.
//
// Note that IsGitURL does not check if "github.com/" prefixes exist as a local
// path. Code using this function should check if the path exists locally before
// using it as a URL.
//
// # Fragments
//
// Git URLs accept context configuration in their fragment section, separated by
// a colon (`:`). The first part represents the reference to check out, and can
// be either a branch, a tag, or a remote reference. The second part represents
// a subdirectory inside the repository to use as the build context.
//
// For example,the following URL uses a directory named "docker" in the branch
// "container" in the https://github.com/myorg/my-repo.git repository:
//
// https://github.com/myorg/my-repo.git#container:docker
//
// The following table represents all the valid suffixes with their build
// contexts:
//
// | Build Syntax Suffix | Git reference used | Build Context Used |
// |--------------------------------|----------------------|--------------------|
// | my-repo.git | refs/heads/master | / |
// | my-repo.git#mytag | refs/tags/my-tag | / |
// | my-repo.git#mybranch | refs/heads/my-branch | / |
// | my-repo.git#pull/42/head | refs/pull/42/head | / |
// | my-repo.git#:directory | refs/heads/master | /directory |
// | my-repo.git#master:directory | refs/heads/master | /directory |
// | my-repo.git#mytag:directory | refs/tags/my-tag | /directory |
// | my-repo.git#mybranch:directory | refs/heads/my-branch | /directory |
func IsGitURL(str string) bool {
if IsURL(str) && urlPathWithFragmentSuffix.MatchString(str) {
return true
}
for _, prefix := range []string{"git://", "github.com/", "git@"} {
if strings.HasPrefix(str, prefix) {
return true
}
}
return false
}

View File

@@ -0,0 +1,42 @@
package urlutil
import "testing"
var (
gitUrls = []string{
"git://github.com/docker/docker",
"git@github.com:docker/docker.git",
"git@bitbucket.org:atlassianlabs/atlassian-docker.git",
"https://github.com/docker/docker.git",
"http://github.com/docker/docker.git",
"http://github.com/docker/docker.git#branch",
"http://github.com/docker/docker.git#:dir",
}
incompleteGitUrls = []string{
"github.com/docker/docker",
}
invalidGitUrls = []string{
"http://github.com/docker/docker.git:#branch",
"https://github.com/docker/dgit",
}
)
func TestIsGIT(t *testing.T) {
for _, url := range gitUrls {
if !IsGitURL(url) {
t.Fatalf("%q should be detected as valid Git url", url)
}
}
for _, url := range incompleteGitUrls {
if !IsGitURL(url) {
t.Fatalf("%q should be detected as valid Git url", url)
}
}
for _, url := range invalidGitUrls {
if IsGitURL(url) {
t.Fatalf("%q should not be detected as valid Git prefix", url)
}
}
}

View File

@@ -0,0 +1,20 @@
package remotecontext
import (
"os"
"path/filepath"
"testing"
)
// createTestTempFile creates a temporary file within dir with specific contents and permissions.
// When an error occurs, it terminates the test
func createTestTempFile(t *testing.T, dir, filename, contents string, perm os.FileMode) string {
t.Helper()
filePath := filepath.Join(dir, filename)
err := os.WriteFile(filePath, []byte(contents), perm)
if err != nil {
t.Fatalf("Error when creating %s file: %s", filename, err)
}
return filePath
}