Feat: send additional build contexts for remote builds

Fixed the --build-context flag to properly send files for remote builds. Previously
only the main context was sent over as a tar while additional contexts were passed as
local paths and this would cause builds to fail since the files wouldn't exist.

New changes modifies the Build API to use multipart HTTP requests allowing multiple
build contexts to be used. Each additional context is packaged and
transferred based on its type:
- Local Directories: Sent as tar archives
- Git Repositories: link sent to the server where its then cloned
- Container Images: Image reference sent to the server, it then pulls the image there
- URLs/archives: URL sent to the server, which handles the download

Fixes: #23433

Signed-off-by: Joshua Arrevillaga <2004jarrevillaga@gmail.com>
This commit is contained in:
Joshua Arrevillaga
2025-07-03 14:56:21 -04:00
parent 7efa0b829a
commit 73f3e98947
5 changed files with 719 additions and 26 deletions

View File

@ -8,6 +8,7 @@ import (
"errors" "errors"
"fmt" "fmt"
"io" "io"
"mime"
"net/http" "net/http"
"os" "os"
"path/filepath" "path/filepath"
@ -30,6 +31,7 @@ import (
"github.com/containers/podman/v5/pkg/rootless" "github.com/containers/podman/v5/pkg/rootless"
"github.com/containers/podman/v5/pkg/util" "github.com/containers/podman/v5/pkg/util"
"github.com/containers/storage/pkg/archive" "github.com/containers/storage/pkg/archive"
"github.com/containers/storage/pkg/chrootarchive"
"github.com/containers/storage/pkg/fileutils" "github.com/containers/storage/pkg/fileutils"
"github.com/docker/docker/pkg/jsonmessage" "github.com/docker/docker/pkg/jsonmessage"
"github.com/opencontainers/runtime-spec/specs-go" "github.com/opencontainers/runtime-spec/specs-go"
@ -44,13 +46,22 @@ func genSpaceErr(err error) error {
} }
func BuildImage(w http.ResponseWriter, r *http.Request) { func BuildImage(w http.ResponseWriter, r *http.Request) {
multipart := false
if hdr, found := r.Header["Content-Type"]; found && len(hdr) > 0 { if hdr, found := r.Header["Content-Type"]; found && len(hdr) > 0 {
contentType := hdr[0] contentType, _, err := mime.ParseMediaType(hdr[0])
if err != nil {
utils.BadRequest(w, "Content-Type", hdr[0], fmt.Errorf("failed to parse content type: %w", err))
return
}
switch contentType { switch contentType {
case "application/tar": case "application/tar":
logrus.Infof("tar file content type is %s, should use \"application/x-tar\" content type", contentType) logrus.Infof("tar file content type is %s, should use \"application/x-tar\" content type", contentType)
case "application/x-tar": case "application/x-tar":
break break
case "multipart/form-data":
logrus.Infof("Received %s", hdr[0])
multipart = true
default: default:
if utils.IsLibpodRequest(r) { if utils.IsLibpodRequest(r) {
utils.BadRequest(w, "Content-Type", hdr[0], utils.BadRequest(w, "Content-Type", hdr[0],
@ -81,7 +92,7 @@ func BuildImage(w http.ResponseWriter, r *http.Request) {
} }
}() }()
contextDirectory, err := extractTarFile(anchorDir, r) contextDirectory, additionalBuildContexts, err := handleBuildContexts(anchorDir, r, multipart)
if err != nil { if err != nil {
utils.InternalServerError(w, genSpaceErr(err)) utils.InternalServerError(w, genSpaceErr(err))
return return
@ -440,14 +451,6 @@ func BuildImage(w http.ResponseWriter, r *http.Request) {
additionalTags = append(additionalTags, possiblyNormalizedTag) additionalTags = append(additionalTags, possiblyNormalizedTag)
} }
var additionalBuildContexts = map[string]*buildahDefine.AdditionalBuildContext{}
if _, found := r.URL.Query()["additionalbuildcontexts"]; found {
if err := json.Unmarshal([]byte(query.AdditionalBuildContexts), &additionalBuildContexts); err != nil {
utils.BadRequest(w, "additionalbuildcontexts", query.AdditionalBuildContexts, err)
return
}
}
var idMappingOptions buildahDefine.IDMappingOptions var idMappingOptions buildahDefine.IDMappingOptions
if _, found := r.URL.Query()["idmappingoptions"]; found { if _, found := r.URL.Query()["idmappingoptions"]; found {
if err := json.Unmarshal([]byte(query.IDMappingOptions), &idMappingOptions); err != nil { if err := json.Unmarshal([]byte(query.IDMappingOptions), &idMappingOptions); err != nil {
@ -920,6 +923,149 @@ func BuildImage(w http.ResponseWriter, r *http.Request) {
} }
} }
func handleBuildContexts(anchorDir string, r *http.Request, multipart bool) (contextDir string, additionalContexts map[string]*buildahDefine.AdditionalBuildContext, err error) {
additionalContexts = make(map[string]*buildahDefine.AdditionalBuildContext)
query := r.URL.Query()
for _, url := range query["additionalbuildcontexts"] {
name, value, found := strings.Cut(url, "=")
if !found {
return "", nil, fmt.Errorf("invalid additional build context format: %q", url)
}
logrus.Debugf("name: %q, context: %q", name, value)
switch {
case strings.HasPrefix(value, "url:"):
value = strings.TrimPrefix(value, "url:")
tempDir, subdir, err := buildahDefine.TempDirForURL(anchorDir, "buildah", value)
if err != nil {
return "", nil, fmt.Errorf("downloading URL %q: %w", name, err)
}
contextPath := filepath.Join(tempDir, subdir)
additionalContexts[name] = &buildahDefine.AdditionalBuildContext{
IsURL: true,
IsImage: false,
Value: contextPath,
DownloadedCache: contextPath,
}
logrus.Debugf("Downloaded URL context %q to %q", name, contextPath)
case strings.HasPrefix(value, "image:"):
value = strings.TrimPrefix(value, "image:")
additionalContexts[name] = &buildahDefine.AdditionalBuildContext{
IsURL: false,
IsImage: true,
Value: value,
}
logrus.Debugf("Using image context %q: %q", name, value)
}
}
// If we have a multipart we use the operations, if not default extraction for main context
if multipart {
logrus.Debug("Multipart is needed")
reader, err := r.MultipartReader()
if err != nil {
return "", nil, fmt.Errorf("failed to create multipart reader: %w", err)
}
for {
part, err := reader.NextPart()
if err == io.EOF {
break
}
if err != nil {
return "", nil, fmt.Errorf("failed to read multipart: %w", err)
}
fieldName := part.FormName()
switch {
case fieldName == "MainContext":
mainDir, err := extractTarFile(anchorDir, part)
if err != nil {
part.Close()
return "", nil, fmt.Errorf("extracting main context in multipart: %w", err)
}
if mainDir == "" {
part.Close()
return "", nil, fmt.Errorf("main context directory is empty")
}
contextDir = mainDir
part.Close()
case strings.HasPrefix(fieldName, "build-context-"):
contextName := strings.TrimPrefix(fieldName, "build-context-")
// Create temp directory directly under anchorDir
additionalAnchor, err := os.MkdirTemp(anchorDir, contextName+"-*")
if err != nil {
part.Close()
return "", nil, fmt.Errorf("creating temp directory for additional context %q: %w", contextName, err)
}
if err := chrootarchive.Untar(part, additionalAnchor, nil); err != nil {
part.Close()
return "", nil, fmt.Errorf("extracting additional context %q: %w", contextName, err)
}
var latestModTime time.Time
fileCount := 0
walkErr := filepath.Walk(additionalAnchor, func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
// Skip the root directory itself since it's always going to have the latest timestamp
if path == additionalAnchor {
return nil
}
if !info.IsDir() {
fileCount++
}
// Use any extracted content timestamp (files or subdirectories)
if info.ModTime().After(latestModTime) {
latestModTime = info.ModTime()
}
return nil
})
if walkErr != nil {
part.Close()
return "", nil, fmt.Errorf("error walking additional context: %w", walkErr)
}
// If we found any files, set the timestamp on the additional context directory
// to the latest modified time found in the files.
if !latestModTime.IsZero() {
if err := os.Chtimes(additionalAnchor, latestModTime, latestModTime); err != nil {
logrus.Warnf("Failed to set timestamp on additional context directory: %v", err)
}
}
additionalContexts[contextName] = &buildahDefine.AdditionalBuildContext{
IsURL: false,
IsImage: false,
Value: additionalAnchor,
}
part.Close()
default:
logrus.Debugf("Ignoring unknown multipart field: %s", fieldName)
part.Close()
}
}
} else {
logrus.Debug("No multipart needed")
contextDir, err = extractTarFile(anchorDir, r.Body)
if err != nil {
return "", nil, err
}
}
return contextDir, additionalContexts, nil
}
func parseNetworkConfigurationPolicy(network string) buildah.NetworkConfigurationPolicy { func parseNetworkConfigurationPolicy(network string) buildah.NetworkConfigurationPolicy {
if val, err := strconv.Atoi(network); err == nil { if val, err := strconv.Atoi(network); err == nil {
return buildah.NetworkConfigurationPolicy(val) return buildah.NetworkConfigurationPolicy(val)
@ -943,13 +1089,13 @@ func parseLibPodIsolation(isolation string) (buildah.Isolation, error) {
return parse.IsolationOption(isolation) return parse.IsolationOption(isolation)
} }
func extractTarFile(anchorDir string, r *http.Request) (string, error) { func extractTarFile(anchorDir string, r io.ReadCloser) (string, error) {
buildDir := filepath.Join(anchorDir, "build") buildDir := filepath.Join(anchorDir, "build")
err := os.Mkdir(buildDir, 0o700) err := os.Mkdir(buildDir, 0o700)
if err != nil { if err != nil {
return "", err return "", err
} }
err = archive.Untar(r.Body, buildDir, nil) err = archive.Untar(r, buildDir, nil)
return buildDir, err return buildDir, err
} }

View File

@ -1491,7 +1491,7 @@ func (s *APIServer) registerImagesHandlers(r *mux.Router) error {
// name: Content-Type // name: Content-Type
// type: string // type: string
// default: application/x-tar // default: application/x-tar
// enum: ["application/x-tar"] // enum: ["application/x-tar", "multipart/form-data"]
// - in: header // - in: header
// name: X-Registry-Config // name: X-Registry-Config
// type: string // type: string
@ -1516,6 +1516,28 @@ func (s *APIServer) registerImagesHandlers(r *mux.Router) error {
// and build for all of the platforms that are available. Stages that use *scratch* as a starting point can not be inspected, // and build for all of the platforms that are available. Stages that use *scratch* as a starting point can not be inspected,
// so at least one non-*scratch* stage must be present for detection to work usefully. // so at least one non-*scratch* stage must be present for detection to work usefully.
// - in: query // - in: query
// name: additionalbuildcontexts
// type: array
// items:
// type: string
// default: []
// description: |
// Additional build contexts for builds that require more than one context.
// Each additional context must be specified as a key-value pair in the format "name=value".
//
// The value can be specified in two formats:
// - URL context: Use the prefix "url:" followed by a URL to a tar archive
// Example: "mycontext=url:https://example.com/context.tar"
// - Image context: Use the prefix "image:" followed by an image reference
// Example: "mycontext=image:alpine:latest" or "mycontext=image:docker.io/library/ubuntu:22.04"
//
// Local contexts are provided via multipart/form-data upload. When using multipart/form-data,
// include additional build contexts as separate form fields with names prefixed by "build-context-".
// For example, a local context named "mycontext" should be uploaded as a tar file in a field
// named "build-context-mycontext".
//
// (As of version 5.6.0)
// - in: query
// name: extrahosts // name: extrahosts
// type: string // type: string
// default: // default:

View File

@ -8,6 +8,7 @@ import (
"fmt" "fmt"
"io" "io"
"io/fs" "io/fs"
"mime/multipart"
"net/http" "net/http"
"net/url" "net/url"
"os" "os"
@ -16,6 +17,7 @@ import (
"strconv" "strconv"
"strings" "strings"
"github.com/blang/semver/v4"
"github.com/containers/buildah/define" "github.com/containers/buildah/define"
imageTypes "github.com/containers/image/v5/types" imageTypes "github.com/containers/image/v5/types"
ldefine "github.com/containers/podman/v5/libpod/define" ldefine "github.com/containers/podman/v5/libpod/define"
@ -24,6 +26,7 @@ import (
"github.com/containers/podman/v5/pkg/domain/entities/types" "github.com/containers/podman/v5/pkg/domain/entities/types"
"github.com/containers/podman/v5/pkg/specgen" "github.com/containers/podman/v5/pkg/specgen"
"github.com/containers/podman/v5/pkg/util" "github.com/containers/podman/v5/pkg/util"
"github.com/containers/storage/pkg/archive"
"github.com/containers/storage/pkg/fileutils" "github.com/containers/storage/pkg/fileutils"
"github.com/containers/storage/pkg/ioutils" "github.com/containers/storage/pkg/ioutils"
"github.com/containers/storage/pkg/regexp" "github.com/containers/storage/pkg/regexp"
@ -126,17 +129,7 @@ func Build(ctx context.Context, containerFiles []string, options types.BuildOpti
for _, tag := range options.AdditionalTags { for _, tag := range options.AdditionalTags {
params.Add("t", tag) params.Add("t", tag)
} }
if additionalBuildContexts := options.AdditionalBuildContexts; len(additionalBuildContexts) > 0 {
// TODO: Additional build contexts should be packaged and sent as tar files
// For the time being we make our best to make them accessible on remote
// machines too (i.e. on macOS and Windows).
convertAdditionalBuildContexts(additionalBuildContexts)
additionalBuildContextMap, err := jsoniter.Marshal(additionalBuildContexts)
if err != nil {
return nil, err
}
params.Set("additionalbuildcontexts", string(additionalBuildContextMap))
}
if options.IDMappingOptions != nil { if options.IDMappingOptions != nil {
idmappingsOptions, err := jsoniter.Marshal(options.IDMappingOptions) idmappingsOptions, err := jsoniter.Marshal(options.IDMappingOptions)
if err != nil { if err != nil {
@ -643,11 +636,147 @@ func Build(ctx context.Context, containerFiles []string, options types.BuildOpti
} }
}() }()
var requestBody io.Reader
var contentType string
// If there are additional build contexts, we need to handle them based on the server version
// podman version >= 5.6.0 supports multipart/form-data for additional build contexts that
// are local directories or archives. URLs and images are still sent as query parameters.
if len(options.AdditionalBuildContexts) > 0 {
serverVersion := bindings.ServiceVersion(ctx)
// Extract just the version numbers (remove -dev, -rc, etc)
versionStr := serverVersion.String()
if idx := strings.Index(versionStr, "-"); idx > 0 {
versionStr = versionStr[:idx]
}
serverVer, err := semver.ParseTolerant(versionStr)
if err != nil {
return nil, fmt.Errorf("parsing server version %q: %w", serverVersion, err)
}
minMultipartVersion, _ := semver.ParseTolerant("5.6.0")
if serverVer.GTE(minMultipartVersion) {
imageContexts := make(map[string]string)
urlContexts := make(map[string]string)
localContexts := make(map[string]*define.AdditionalBuildContext)
for name, context := range options.AdditionalBuildContexts {
switch {
case context.IsImage:
imageContexts[name] = context.Value
case context.IsURL:
urlContexts[name] = context.Value
default:
localContexts[name] = context
}
}
logrus.Debugf("URL Contexts: %v", urlContexts)
for name, url := range urlContexts {
params.Add("additionalbuildcontexts", fmt.Sprintf("%s=url:%s", name, url))
}
logrus.Debugf("Image Contexts: %v", imageContexts)
for name, imageRef := range imageContexts {
params.Add("additionalbuildcontexts", fmt.Sprintf("%s=image:%s", name, imageRef))
}
if len(localContexts) > 0 {
// Multipart request structure:
// - "MainContext": The main build context as a tar file
// - "build-context-<name>": Each additional local context as a tar file
logrus.Debugf("Using additional local build contexts: %v", localContexts)
pr, pw := io.Pipe()
writer := multipart.NewWriter(pw)
contentType = writer.FormDataContentType()
requestBody = pr
if headers == nil {
headers = make(http.Header)
}
headers.Set("Content-Type", contentType)
go func() {
defer pw.Close()
defer writer.Close()
mainContext, err := writer.CreateFormFile("MainContext", "MainContext.tar")
if err != nil {
pw.CloseWithError(fmt.Errorf("creating form file for main context: %w", err))
return
}
if _, err := io.Copy(mainContext, tarfile); err != nil {
pw.CloseWithError(fmt.Errorf("copying main context: %w", err))
return
}
for name, context := range localContexts {
logrus.Debugf("Processing additional local context: %s", name)
part, err := writer.CreateFormFile(fmt.Sprintf("build-context-%s", name), name)
if err != nil {
pw.CloseWithError(fmt.Errorf("creating form file for context %q: %w", name, err))
return
}
// Context is already a tar
if archive.IsArchivePath(context.Value) {
file, err := os.Open(context.Value)
if err != nil {
pw.CloseWithError(fmt.Errorf("opening archive %q: %w", name, err))
return
}
if _, err := io.Copy(part, file); err != nil {
file.Close()
pw.CloseWithError(fmt.Errorf("copying context %q: %w", name, err))
return
}
file.Close()
} else {
tarContent, err := nTar(nil, context.Value)
if err != nil {
pw.CloseWithError(fmt.Errorf("creating tar content %q: %w", name, err))
return
}
if _, err = io.Copy(part, tarContent); err != nil {
pw.CloseWithError(fmt.Errorf("copying tar content %q: %w", name, err))
return
}
if err := tarContent.Close(); err != nil {
logrus.Errorf("Error closing tar content for context %q: %v\n", name, err)
}
}
}
}()
logrus.Debugf("Multipart body is created with content type: %s", contentType)
} else {
requestBody = tarfile
logrus.Debugf("Using main build context: %q", options.ContextDirectory)
}
} else {
convertAdditionalBuildContexts(options.AdditionalBuildContexts)
additionalBuildContextMap, err := jsoniter.Marshal(options.AdditionalBuildContexts)
if err != nil {
return nil, err
}
params.Set("additionalbuildcontexts", string(additionalBuildContextMap))
requestBody = tarfile
logrus.Debugf("Using main build context: %q", options.ContextDirectory)
}
} else {
requestBody = tarfile
logrus.Debugf("Using main build context: %q", options.ContextDirectory)
}
conn, err := bindings.GetClient(ctx) conn, err := bindings.GetClient(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }
response, err := conn.DoRequest(ctx, tarfile, http.MethodPost, "/build", params, headers) response, err := conn.DoRequest(ctx, requestBody, http.MethodPost, "/build", params, headers)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -250,7 +250,6 @@ var _ = Describe("run basic podman commands", func() {
It("podman build contexts", func() { It("podman build contexts", func() {
skipIfVmtype(define.HyperVVirt, "FIXME: #23429 - Error running podman build with option --build-context on Hyper-V") skipIfVmtype(define.HyperVVirt, "FIXME: #23429 - Error running podman build with option --build-context on Hyper-V")
skipIfVmtype(define.QemuVirt, "FIXME: #23433 - Additional build contexts should be sent as additional tar files")
name := randomString() name := randomString()
i := new(initMachine) i := new(initMachine)
session, err := mb.setName(name).setCmd(i.withImage(mb.imagePath).withNow()).run() session, err := mb.setName(name).setCmd(i.withImage(mb.imagePath).withNow()).run()
@ -273,6 +272,15 @@ var _ = Describe("run basic podman commands", func() {
bm := basicMachine{} bm := basicMachine{}
build, err := mb.setCmd(bm.withPodmanCommand([]string{"build", "-t", name, "--build-context", "test-context=" + additionalContextDir, mainContextDir})).run() build, err := mb.setCmd(bm.withPodmanCommand([]string{"build", "-t", name, "--build-context", "test-context=" + additionalContextDir, mainContextDir})).run()
if build != nil && build.ExitCode() != 0 {
output := build.outputToString() + build.errorToString()
if strings.Contains(output, "multipart/form-data") &&
strings.Contains(output, "not supported") {
Skip("Build contexts with multipart/form-data are not supported on this version")
}
}
Expect(err).ToNot(HaveOccurred()) Expect(err).ToNot(HaveOccurred())
Expect(build).To(Exit(0)) Expect(build).To(Exit(0))
Expect(build.outputToString()).To(ContainSubstring("COMMIT")) Expect(build.outputToString()).To(ContainSubstring("COMMIT"))

View File

@ -972,4 +972,392 @@ RUN ls /dev/test1`, CITEST_IMAGE)
session.WaitWithDefaultTimeout() session.WaitWithDefaultTimeout()
Expect(session).Should(ExitWithError(1, `building at STEP "RUN --mount=type=cache,target=/test,z cat /test/world": while running runtime: exit status 1`)) Expect(session).Should(ExitWithError(1, `building at STEP "RUN --mount=type=cache,target=/test,z cat /test/world": while running runtime: exit status 1`))
}) })
It("podman build --build-context: local source", func() {
podmanTest.RestartRemoteService()
localCtx1 := filepath.Join(podmanTest.TempDir, "context1")
localCtx2 := filepath.Join(podmanTest.TempDir, "context2")
Expect(os.MkdirAll(localCtx1, 0755)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx1, "file1.txt"), []byte("Content from context1"), 0644)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx1, "config.json"), []byte(`{"source": "context1"}`), 0644)).To(Succeed())
Expect(os.MkdirAll(filepath.Join(localCtx2, "subdir"), 0755)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx2, "file2.txt"), []byte("Content from context2"), 0644)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx2, "subdir", "nested.txt"), []byte("Nested content"), 0644)).To(Succeed())
containerfile := `FROM quay.io/libpod/alpine:latest
COPY --from=localctx1 /file1.txt /from-context1.txt
COPY --from=localctx1 /config.json /config1.json`
containerfilePath := filepath.Join(podmanTest.TempDir, "Containerfile1")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session := podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-local-single",
"--build-context", fmt.Sprintf("localctx1=%s", localCtx1),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-local-single", "cat", "/from-context1.txt"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(Equal("Content from context1"))
containerfile = `FROM quay.io/libpod/alpine:latest
COPY --from=ctx1 /file1.txt /file1.txt
COPY --from=ctx2 /file2.txt /file2.txt
COPY --from=ctx2 /subdir/nested.txt /nested.txt`
containerfilePath = filepath.Join(podmanTest.TempDir, "Containerfile2")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-local-multi",
"--build-context", fmt.Sprintf("ctx1=%s", localCtx1),
"--build-context", fmt.Sprintf("ctx2=%s", localCtx2),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-local-multi", "cat", "/nested.txt"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(Equal("Nested content"))
mainFile := filepath.Join(podmanTest.TempDir, "main.txt")
Expect(os.WriteFile(mainFile, []byte("From main context"), 0644)).To(Succeed())
containerfile = `FROM quay.io/libpod/alpine:latest
COPY main.txt /main.txt
COPY --from=additional /file1.txt /additional.txt`
containerfilePath = filepath.Join(podmanTest.TempDir, "Containerfile3")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-local-mixed",
"--build-context", fmt.Sprintf("additional=%s", localCtx1),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-local-mixed", "cat", "/main.txt"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(Equal("From main context"))
session = podmanTest.Podman([]string{"rmi", "-f", "test-local-single", "test-local-multi", "test-local-mixed"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
})
It("podman build --build-context: URL source", func() {
podmanTest.RestartRemoteService()
testRepoURL := "https://github.com/containers/PodmanHello.git"
testArchiveURL := "https://github.com/containers/PodmanHello/archive/refs/heads/main.tar.gz"
containerfile := `FROM quay.io/libpod/alpine:latest
COPY --from=urlctx . /url-context/`
containerfilePath := filepath.Join(podmanTest.TempDir, "ContainerfileURL1")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session := podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-url-single",
"--build-context", fmt.Sprintf("urlctx=%s", testRepoURL),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-url-single", "ls", "/url-context/"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
output := session.OutputToString()
Expect(output).To(ContainSubstring("LICENSE"))
Expect(output).To(ContainSubstring("README.md"))
containerfile = `FROM quay.io/libpod/alpine:latest
COPY --from=archive . /from-archive/`
containerfilePath = filepath.Join(podmanTest.TempDir, "ContainerfileURL2")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-archive",
"--build-context", fmt.Sprintf("archive=%s", testArchiveURL),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-archive", "ls", "/from-archive/"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
output = session.OutputToString()
Expect(output).To(ContainSubstring("PodmanHello-main"))
session = podmanTest.Podman([]string{"run", "--rm", "test-archive", "ls", "/from-archive/PodmanHello-main/"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
output = session.OutputToString()
Expect(output).To(ContainSubstring("LICENSE"))
Expect(output).To(ContainSubstring("README.md"))
localCtx := filepath.Join(podmanTest.TempDir, "localcontext")
Expect(os.MkdirAll(localCtx, 0755)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx, "local.txt"), []byte("Local content"), 0644)).To(Succeed())
containerfile = `FROM quay.io/libpod/alpine:latest
COPY --from=urlrepo . /from-url/
COPY --from=localctx /local.txt /local.txt
RUN echo "Combined URL and local contexts" > /combined.txt`
containerfilePath = filepath.Join(podmanTest.TempDir, "ContainerfileURL3")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-url-mixed",
"--build-context", fmt.Sprintf("urlrepo=%s", testRepoURL),
"--build-context", fmt.Sprintf("localctx=%s", localCtx),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-url-mixed", "cat", "/local.txt"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(Equal("Local content"))
mainFile := filepath.Join(podmanTest.TempDir, "main-url-test.txt")
Expect(os.WriteFile(mainFile, []byte("Main context for URL test"), 0644)).To(Succeed())
containerfile = `FROM quay.io/libpod/alpine:latest
COPY main-url-test.txt /main.txt
COPY --from=gitrepo . /git-repo/`
containerfilePath = filepath.Join(podmanTest.TempDir, "ContainerfileURL5")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-url-main",
"--build-context", fmt.Sprintf("gitrepo=%s", testRepoURL),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-url-main", "cat", "/main.txt"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(Equal("Main context for URL test"))
session = podmanTest.Podman([]string{"rmi", "-f", "test-url-single", "test-archive", "test-url-mixed", "test-url-main"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
})
It("podman build --build-context: Image source", func() {
podmanTest.RestartRemoteService()
alpineImage := "quay.io/libpod/alpine:latest"
busyboxImage := "quay.io/libpod/busybox:latest"
containerfile := `FROM quay.io/libpod/busybox:latest AS source
FROM quay.io/libpod/alpine:latest
COPY --from=source /bin/busybox /busybox-from-stage`
containerfilePath := filepath.Join(podmanTest.TempDir, "ContainerfileMultiStage")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session := podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-multi-stage",
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
testCases := []struct {
name string
prefix string
image string
contextName string
containerfile string
verifyCmd []string
}{
{
name: "docker-image-prefix",
prefix: "docker-image://",
image: alpineImage,
contextName: "dockerimg",
containerfile: `FROM quay.io/libpod/alpine:latest
COPY --from=dockerimg /etc/alpine-release /alpine-version.txt`,
verifyCmd: []string{"cat", "/alpine-version.txt"},
},
{
name: "container-image-prefix",
prefix: "container-image://",
image: busyboxImage,
contextName: "containerimg",
containerfile: `FROM quay.io/libpod/alpine:latest
COPY --from=containerimg /bin/busybox /busybox-binary`,
verifyCmd: []string{"/busybox-binary", "--help"},
},
{
name: "docker-prefix",
prefix: "docker://",
image: alpineImage,
contextName: "dockershort",
containerfile: `FROM quay.io/libpod/alpine:latest
COPY --from=dockershort /etc/os-release /os-release.txt`,
verifyCmd: []string{"cat", "/os-release.txt"},
},
}
for _, tc := range testCases {
containerfilePath = filepath.Join(podmanTest.TempDir, fmt.Sprintf("Containerfile_%s", tc.name))
Expect(os.WriteFile(containerfilePath, []byte(tc.containerfile), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", fmt.Sprintf("test-%s", tc.name),
"--build-context", fmt.Sprintf("%s=%s%s", tc.contextName, tc.prefix, tc.image),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman(append([]string{"run", "--rm", fmt.Sprintf("test-%s", tc.name)}, tc.verifyCmd...))
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
if tc.name == "container-image-prefix" {
Expect(session.OutputToString()).To(ContainSubstring("BusyBox"))
}
}
session = podmanTest.Podman([]string{"rmi", "-f", "test-multi-stage"})
session.WaitWithDefaultTimeout()
for _, tc := range testCases {
session = podmanTest.Podman([]string{"rmi", "-f", fmt.Sprintf("test-%s", tc.name)})
session.WaitWithDefaultTimeout()
}
})
It("podman build --build-context: Mixed source", func() {
podmanTest.RestartRemoteService()
localCtx := filepath.Join(podmanTest.TempDir, "local-mixed")
Expect(os.MkdirAll(localCtx, 0755)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx, "local-config.json"), []byte(`{"context": "local", "version": "1.0"}`), 0644)).To(Succeed())
Expect(os.WriteFile(filepath.Join(localCtx, "app.conf"), []byte("# Local app configuration\nmode=production\nport=8080"), 0644)).To(Succeed())
urlContext := "https://github.com/containers/PodmanHello.git"
alpineImage := "quay.io/libpod/alpine:latest"
busyboxImage := "quay.io/libpod/busybox:latest"
mainFile := filepath.Join(podmanTest.TempDir, "VERSION")
Expect(os.WriteFile(mainFile, []byte("v1.0.0-mixed"), 0644)).To(Succeed())
containerfile := `FROM quay.io/libpod/alpine:latest
# From main build context
COPY VERSION /app/VERSION
# From local directory context
COPY --from=localdir /local-config.json /app/config/local-config.json
COPY --from=localdir /app.conf /app/config/app.conf
# From URL/Git context
COPY --from=gitrepo /LICENSE /app/licenses/podman-hello-LICENSE
COPY --from=gitrepo /README.md /app/docs/podman-hello-README.md
# From image contexts with different prefixes
COPY --from=alpineimg /etc/alpine-release /app/base-images/alpine-version
COPY --from=busyboximg /bin/busybox /app/tools/busybox
# Create a summary file
RUN echo "Build with all context types completed" > /app/build-summary.txt && \
chmod +x /app/tools/busybox
WORKDIR /app`
containerfilePath := filepath.Join(podmanTest.TempDir, "ContainerfileMixed")
Expect(os.WriteFile(containerfilePath, []byte(containerfile), 0644)).To(Succeed())
session := podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-all-contexts",
"--build-context", fmt.Sprintf("localdir=%s", localCtx),
"--build-context", fmt.Sprintf("gitrepo=%s", urlContext),
"--build-context", fmt.Sprintf("alpineimg=docker-image://%s", alpineImage),
"--build-context", fmt.Sprintf("busyboximg=container-image://%s", busyboxImage),
"-f", containerfilePath, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
verifyTests := []struct {
cmd []string
expected string
}{
{[]string{"cat", "/app/VERSION"}, "v1.0.0-mixed"},
{[]string{"cat", "/app/config/local-config.json"}, `"context": "local"`},
{[]string{"cat", "/app/config/app.conf"}, "port=8080"},
{[]string{"test", "-f", "/app/licenses/podman-hello-LICENSE"}, ""},
{[]string{"test", "-f", "/app/docs/podman-hello-README.md"}, ""},
{[]string{"cat", "/app/base-images/alpine-version"}, "3."},
{[]string{"/app/tools/busybox", "--help"}, "BusyBox"},
{[]string{"cat", "/app/build-summary.txt"}, "Build with all context types completed"},
}
for _, test := range verifyTests {
session = podmanTest.Podman(append([]string{"run", "--rm", "test-all-contexts"}, test.cmd...))
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
if test.expected != "" {
Expect(session.OutputToString()).To(ContainSubstring(test.expected))
}
}
session = podmanTest.Podman([]string{
"run", "--rm", "test-all-contexts",
"/app/tools/busybox", "grep", "port", "/app/config/app.conf",
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
Expect(session.OutputToString()).To(ContainSubstring("port=8080"))
containerfile2 := `FROM quay.io/libpod/alpine:latest
COPY --from=img1 /etc/os-release /prefix-test/docker-prefix.txt
COPY --from=img2 /etc/alpine-release /prefix-test/container-prefix.txt`
containerfilePath2 := filepath.Join(podmanTest.TempDir, "ContainerfileMixed2")
Expect(os.WriteFile(containerfilePath2, []byte(containerfile2), 0644)).To(Succeed())
session = podmanTest.Podman([]string{
"build", "--pull-never", "-t", "test-prefix-mix",
"--build-context", fmt.Sprintf("img1=docker://%s", alpineImage),
"--build-context", fmt.Sprintf("img2=container-image://%s", alpineImage),
"-f", containerfilePath2, podmanTest.TempDir,
})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
session = podmanTest.Podman([]string{"run", "--rm", "test-prefix-mix", "ls", "/prefix-test/"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
output := session.OutputToString()
Expect(output).To(ContainSubstring("docker-prefix.txt"))
Expect(output).To(ContainSubstring("container-prefix.txt"))
session = podmanTest.Podman([]string{"rmi", "-f", "test-all-contexts", "test-prefix-mix"})
session.WaitWithDefaultTimeout()
Expect(session).Should(ExitCleanly())
})
}) })