metropolis: use new OS image format for updates

This switches the node update implementation to the new OS image format
based on OCI artifacts. Updates are now fetched from an OCI registry.

To update existing clusters, build //metropolis/node:bundle in the new
version, then run the update command of the old version of metroctl with
this bundle. Once a node is updated, it only accepts updates in the
new format. It is possible to rollback if needed by building
//metropolis/node:oci_image in the old version and using the new version
of metroctl.

The node bundle target is no longer referenced anywhere, and will be
removed soon.

Change-Id: I00ac6d0d88e379259cea52c8a106204c5eb73fe7
Reviewed-on: https://review.monogon.dev/c/monogon/+/4123
Tested-by: Jenkins CI
Reviewed-by: Tim Windelschmidt <tim@monogon.tech>
diff --git a/metropolis/node/core/mgmt/update.go b/metropolis/node/core/mgmt/update.go
index de9b876..1e61ef1 100644
--- a/metropolis/node/core/mgmt/update.go
+++ b/metropolis/node/core/mgmt/update.go
@@ -23,7 +23,7 @@
 	if req.ActivationMode == apb.ActivationMode_ACTIVATION_MODE_INVALID {
 		return nil, status.Errorf(codes.InvalidArgument, "activation_mode needs to be explicitly specified")
 	}
-	if err := s.UpdateService.InstallBundle(ctx, req.BundleUrl, req.ActivationMode == apb.ActivationMode_ACTIVATION_MODE_KEXEC); err != nil {
+	if err := s.UpdateService.InstallImage(ctx, req.OsImage, req.ActivationMode == apb.ActivationMode_ACTIVATION_MODE_KEXEC); err != nil {
 		return nil, status.Errorf(codes.Unavailable, "error installing update: %v", err)
 	}
 	if req.ActivationMode != apb.ActivationMode_ACTIVATION_MODE_NONE {
diff --git a/metropolis/node/core/update/BUILD.bazel b/metropolis/node/core/update/BUILD.bazel
index 6b12a94..8dac43e 100644
--- a/metropolis/node/core/update/BUILD.bazel
+++ b/metropolis/node/core/update/BUILD.bazel
@@ -11,11 +11,16 @@
     deps = [
         "//go/logging",
         "//metropolis/node/core/abloader/spec",
+        "//metropolis/proto/api",
+        "//metropolis/version",
         "//osbase/blockdev",
         "//osbase/build/mkimage/osimage",
         "//osbase/efivarfs",
         "//osbase/gpt",
         "//osbase/kexec",
+        "//osbase/oci/osimage",
+        "//osbase/oci/registry",
+        "//version",
         "@com_github_cenkalti_backoff_v4//:backoff",
         "@org_golang_google_grpc//codes",
         "@org_golang_google_grpc//status",
diff --git a/metropolis/node/core/update/e2e/BUILD.bazel b/metropolis/node/core/update/e2e/BUILD.bazel
index 83716d1..5dcf3f7 100644
--- a/metropolis/node/core/update/e2e/BUILD.bazel
+++ b/metropolis/node/core/update/e2e/BUILD.bazel
@@ -12,12 +12,12 @@
         "//metropolis/node/core/update/e2e/testos:kernel_efi_x",
         "//metropolis/node/core/abloader",
         # For the two update tests
-        "//metropolis/node/core/update/e2e/testos:testos_bundle_y",
-        "//metropolis/node/core/update/e2e/testos:testos_bundle_z",
+        "//metropolis/node/core/update/e2e/testos:testos_image_y",
+        "//metropolis/node/core/update/e2e/testos:testos_image_z",
     ],
     x_defs = {
-        "xBundleYPath": "$(rlocationpath //metropolis/node/core/update/e2e/testos:testos_bundle_y )",
-        "xBundleZPath": "$(rlocationpath //metropolis/node/core/update/e2e/testos:testos_bundle_z )",
+        "xImageYPath": "$(rlocationpath //metropolis/node/core/update/e2e/testos:testos_image_y )",
+        "xImageZPath": "$(rlocationpath //metropolis/node/core/update/e2e/testos:testos_image_z )",
         "xOvmfVarsPath": "$(rlocationpath //third_party/edk2:OVMF_VARS.fd )",
         "xOvmfCodePath": "$(rlocationpath //third_party/edk2:OVMF_CODE.fd )",
         "xBootPath": "$(rlocationpath //metropolis/node/core/update/e2e/testos:kernel_efi_x )",
@@ -27,6 +27,8 @@
     deps = [
         "//osbase/blockdev",
         "//osbase/build/mkimage/osimage",
+        "//osbase/oci",
+        "//osbase/oci/registry",
         "//osbase/structfs",
         "@io_bazel_rules_go//go/runfiles",
     ],
diff --git a/metropolis/node/core/update/e2e/e2e_test.go b/metropolis/node/core/update/e2e/e2e_test.go
index 4a1c6a2..ec51281 100644
--- a/metropolis/node/core/update/e2e/e2e_test.go
+++ b/metropolis/node/core/update/e2e/e2e_test.go
@@ -15,7 +15,6 @@
 	"path/filepath"
 	"regexp"
 	"strings"
-	"sync"
 	"testing"
 	"time"
 
@@ -23,6 +22,8 @@
 
 	"source.monogon.dev/osbase/blockdev"
 	"source.monogon.dev/osbase/build/mkimage/osimage"
+	"source.monogon.dev/osbase/oci"
+	"source.monogon.dev/osbase/oci/registry"
 	"source.monogon.dev/osbase/structfs"
 )
 
@@ -30,8 +31,8 @@
 	// These are filled by bazel at linking time with the canonical path of
 	// their corresponding file. Inside the init function we resolve it
 	// with the rules_go runfiles package to the real path.
-	xBundleYPath  string
-	xBundleZPath  string
+	xImageYPath   string
+	xImageZPath   string
 	xOvmfVarsPath string
 	xOvmfCodePath string
 	xBootPath     string
@@ -42,7 +43,7 @@
 func init() {
 	var err error
 	for _, path := range []*string{
-		&xBundleYPath, &xBundleZPath, &xOvmfVarsPath,
+		&xImageYPath, &xImageZPath, &xOvmfVarsPath,
 		&xOvmfCodePath, &xBootPath, &xSystemXPath,
 		&xAbloaderPath,
 	} {
@@ -53,8 +54,6 @@
 	}
 }
 
-const Mi = 1024 * 1024
-
 var variantRegexp = regexp.MustCompile(`TESTOS_VARIANT=([A-Z])`)
 
 func stdoutHandler(t *testing.T, cmd *exec.Cmd, cancel context.CancelFunc, testosStarted chan string) {
@@ -68,11 +67,11 @@
 			if strings.HasPrefix(s.Text(), "[") {
 				continue
 			}
-			errIdx := strings.Index(s.Text(), "Error installing new bundle")
+			errIdx := strings.Index(s.Text(), "Error installing new image")
 			if errIdx != -1 {
 				cancel()
 			}
-			t.Log("vm: " + s.Text())
+			fmt.Printf("vm: %q\n", s.Text())
 			if m := variantRegexp.FindStringSubmatch(s.Text()); len(m) == 2 {
 				select {
 				case testosStarted <- m[1]:
@@ -94,7 +93,7 @@
 			if strings.HasPrefix(s.Text(), "[") {
 				continue
 			}
-			t.Log("qemu: " + s.Text())
+			fmt.Printf("qemu: %q\n", s.Text())
 		}
 	}()
 }
@@ -132,61 +131,34 @@
 	}
 }
 
-type bundleServing struct {
-	t              *testing.T
-	bundlePaths    map[string]string
-	bundleFilePath string
-	// Protects bundleFilePath above
-	m sync.Mutex
-}
-
-func (b *bundleServing) setNextBundle(variant string) {
-	b.m.Lock()
-	defer b.m.Unlock()
-	p, ok := b.bundlePaths[variant]
-	if !ok {
-		b.t.Fatalf("no bundle for variant %s available", variant)
-	}
-	b.bundleFilePath = p
-}
-
-// setup sets up an an HTTP server for serving bundles which can be controlled
-// through the returned bundleServing struct as well as the initial boot disk
-// and EFI variable storage. It also returns the required QEMU arguments to
-// boot the initial TestOS.
-func setup(t *testing.T) (*bundleServing, []string) {
+// setup sets up a registry server as well as the initial boot disk
+// and EFI variable storage. It also returns the required QEMU arguments.
+func setup(t *testing.T) []string {
 	t.Helper()
-	blobAddr := net.TCPAddr{
+	registryAddr := net.TCPAddr{
 		IP:   net.IPv4(10, 42, 0, 5),
 		Port: 80,
 	}
 
-	b := bundleServing{
-		t:           t,
-		bundlePaths: make(map[string]string),
-	}
-
-	m := http.NewServeMux()
-	b.bundlePaths["Y"] = xBundleYPath
-	b.bundlePaths["Z"] = xBundleZPath
-	m.HandleFunc("/bundle.bin", func(w http.ResponseWriter, req *http.Request) {
-		b.m.Lock()
-		bundleFilePath := b.bundleFilePath
-		b.m.Unlock()
-		if bundleFilePath == "" {
-			w.WriteHeader(http.StatusBadRequest)
-			w.Write([]byte("No next bundle set in the test harness"))
-			return
-		}
-		http.ServeFile(w, req, bundleFilePath)
-	})
-	blobLis, err := net.Listen("tcp", "127.0.0.1:0")
+	imageY, err := oci.ReadLayout(xImageYPath)
 	if err != nil {
 		t.Fatal(err)
 	}
-	t.Cleanup(func() { blobLis.Close() })
-	blobListenAddr := blobLis.Addr().(*net.TCPAddr)
-	go http.Serve(blobLis, m)
+	imageZ, err := oci.ReadLayout(xImageZPath)
+	if err != nil {
+		t.Fatal(err)
+	}
+
+	registryServer := registry.NewServer()
+	registryServer.AddImage("testos", "y", imageY)
+	registryServer.AddImage("testos", "z", imageZ)
+	registryLis, err := net.Listen("tcp", "127.0.0.1:0")
+	if err != nil {
+		t.Fatal(err)
+	}
+	t.Cleanup(func() { registryLis.Close() })
+	registryListenAddr := registryLis.Addr().(*net.TCPAddr)
+	go http.Serve(registryLis, registryServer)
 
 	rootDevPath := filepath.Join(t.TempDir(), "root.img")
 	// Make a 512 bytes * 2Mi = 1Gi file-backed block device
@@ -225,7 +197,7 @@
 		t.Fatalf("unable to generate starting point image: %v", err)
 	}
 
-	blobGuestFwd := fmt.Sprintf("guestfwd=tcp:%s-tcp:127.0.0.1:%d", blobAddr.String(), blobListenAddr.Port)
+	registryGuestFwd := fmt.Sprintf("guestfwd=tcp:%s-tcp:127.0.0.1:%d", registryAddr.String(), registryListenAddr.Port)
 
 	ovmfVars, err := os.CreateTemp("", "ab-ovmf-vars")
 	if err != nil {
@@ -248,33 +220,33 @@
 		"-drive", "if=pflash,format=raw,readonly=on,file=" + xOvmfCodePath,
 		"-drive", "if=pflash,format=raw,file=" + ovmfVars.Name(),
 		"-drive", "if=virtio,format=raw,cache=unsafe,file=" + rootDevPath,
-		"-netdev", fmt.Sprintf("user,id=net0,net=10.42.0.0/24,dhcpstart=10.42.0.10,%s", blobGuestFwd),
+		"-netdev", fmt.Sprintf("user,id=net0,net=10.42.0.0/24,dhcpstart=10.42.0.10,%s", registryGuestFwd),
 		"-device", "virtio-net-pci,netdev=net0,mac=22:d5:8e:76:1d:07",
 		"-device", "virtio-rng-pci",
 		"-serial", "stdio",
 		"-no-reboot",
+		"-fw_cfg", "name=opt/testos_y_digest,string=" + imageY.ManifestDigest,
+		"-fw_cfg", "name=opt/testos_z_digest,string=" + imageZ.ManifestDigest,
 	}
-	return &b, qemuArgs
+	return qemuArgs
 }
 
 func TestABUpdateSequenceReboot(t *testing.T) {
-	bsrv, qemuArgs := setup(t)
+	qemuArgs := setup(t)
 
-	t.Log("Launching X image to install Y")
-	bsrv.setNextBundle("Y")
+	fmt.Println("Launching X image to install Y")
 	runAndCheckVariant(t, "X", qemuArgs)
 
-	t.Log("Launching Y on slot B to install Z on slot A")
-	bsrv.setNextBundle("Z")
+	fmt.Println("Launching Y on slot B to install Z on slot A")
 	runAndCheckVariant(t, "Y", qemuArgs)
 
-	t.Log("Launching Z on slot A")
+	fmt.Println("Launching Z on slot A")
 	runAndCheckVariant(t, "Z", qemuArgs)
 }
 
 func TestABUpdateSequenceKexec(t *testing.T) {
-	bsrv, qemuArgs := setup(t)
-	qemuArgs = append(qemuArgs, "-fw_cfg", "name=use_kexec,string=1")
+	qemuArgs := setup(t)
+	qemuArgs = append(qemuArgs, "-fw_cfg", "name=opt/use_kexec,string=1")
 
 	ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
 	defer cancel()
@@ -314,8 +286,7 @@
 					return
 				}
 			}
-			bsrv.setNextBundle(expectedVariant)
-			t.Logf("Got %s, installing %s", variant, expectedVariant)
+			fmt.Printf("Got %s, installing %s\n", variant, expectedVariant)
 		case err := <-procExit:
 			t.Fatalf("QEMU exited unexpectedly: %v", err)
 		case <-ctx.Done():
diff --git a/metropolis/node/core/update/e2e/testos/BUILD.bazel b/metropolis/node/core/update/e2e/testos/BUILD.bazel
index c0f6aed..5301b57 100644
--- a/metropolis/node/core/update/e2e/testos/BUILD.bazel
+++ b/metropolis/node/core/update/e2e/testos/BUILD.bazel
@@ -15,6 +15,7 @@
     deps = [
         "//metropolis/node/core/network",
         "//metropolis/node/core/update",
+        "//metropolis/proto/api",
         "//osbase/blockdev",
         "//osbase/bringup",
         "//osbase/build/mkimage/osimage",
diff --git a/metropolis/node/core/update/e2e/testos/main.go b/metropolis/node/core/update/e2e/testos/main.go
index 39c019d..2cafebc 100644
--- a/metropolis/node/core/update/e2e/testos/main.go
+++ b/metropolis/node/core/update/e2e/testos/main.go
@@ -18,6 +18,8 @@
 	"source.monogon.dev/osbase/build/mkimage/osimage"
 	"source.monogon.dev/osbase/gpt"
 	"source.monogon.dev/osbase/supervisor"
+
+	apb "source.monogon.dev/metropolis/proto/api"
 )
 
 var Variant = "U"
@@ -68,16 +70,35 @@
 	if err := updateSvc.MarkBootSuccessful(); err != nil {
 		supervisor.Logger(ctx).Errorf("error marking boot successful: %w", err)
 	}
-	_, err = os.Stat("/sys/firmware/qemu_fw_cfg/by_name/use_kexec/raw")
+
+	_, err = os.Stat("/sys/firmware/qemu_fw_cfg/by_name/opt/use_kexec/raw")
 	useKexec := err == nil
 	supervisor.Logger(ctx).Infof("Kexec: %v", useKexec)
-	if Variant != "Z" {
-		if err := updateSvc.InstallBundle(ctx, "http://10.42.0.5:80/bundle.bin", useKexec); err != nil {
-			supervisor.Logger(ctx).Errorf("Error installing new bundle: %v", err)
+
+	nextVariantMap := map[string]string{
+		"X": "y",
+		"Y": "z",
+	}
+	nextVariant := nextVariantMap[Variant]
+
+	if nextVariant != "" {
+		nextDigest, err := os.ReadFile(fmt.Sprintf("/sys/firmware/qemu_fw_cfg/by_name/opt/testos_%s_digest/raw", nextVariant))
+		if err != nil {
+			return fmt.Errorf("unable to read next digest: %w", err)
+		}
+		imageRef := &apb.OSImageRef{
+			Scheme:     "http",
+			Host:       "10.42.0.5:80",
+			Repository: "testos",
+			Tag:        nextVariant,
+			Digest:     string(nextDigest),
+		}
+		if err := updateSvc.InstallImage(ctx, imageRef, useKexec); err != nil {
+			supervisor.Logger(ctx).Errorf("Error installing new image: %v", err)
 		}
 	}
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
-	supervisor.Logger(ctx).Info("Installed bundle successfully, powering off")
+	supervisor.Logger(ctx).Info("Installed image successfully, powering off")
 	unix.Sync()
 	time.Sleep(1 * time.Second)
 	if useKexec && Variant != "Z" {
diff --git a/metropolis/node/core/update/e2e/testos/testos.bzl b/metropolis/node/core/update/e2e/testos/testos.bzl
index a38b68d..8822e50 100644
--- a/metropolis/node/core/update/e2e/testos/testos.bzl
+++ b/metropolis/node/core/update/e2e/testos/testos.bzl
@@ -1,7 +1,6 @@
 load("@io_bazel_rules_go//go:def.bzl", "go_binary")
-load("@rules_pkg//:mappings.bzl", "pkg_files")
-load("@rules_pkg//:pkg.bzl", "pkg_zip")
 load("//osbase/build/mkerofs:def.bzl", "erofs_image")
+load("//osbase/build/mkoci:def.bzl", "oci_os_image")
 load("//osbase/build/mkpayload:def.bzl", "efi_unified_kernel_image")
 load("//osbase/build/mkverity:def.bzl", "verity_image")
 
@@ -34,25 +33,12 @@
         visibility = ["//metropolis/node/core/update/e2e:__pkg__"],
     )
 
-    # An intermediary "bundle" format until we finalize the actual bundle format. This is NOT stable until migrated
-    # to the actual bundle format.
-    # TODO(lorenz): Replace this
-    pkg_files(
-        name = "testos_bundle_files_" + variant,
-        srcs = [
-            ":kernel_efi_" + variant,
-            ":verity_rootfs_" + variant,
-        ],
-        renames = {
-            ":kernel_efi_" + variant: "kernel_efi.efi",
-            ":verity_rootfs_" + variant: "verity_rootfs.img",
+    oci_os_image(
+        name = "testos_image_" + variant,
+        srcs = {
+            "system": ":verity_rootfs_" + variant,
+            "kernel.efi": ":kernel_efi_" + variant,
         },
-    )
-    pkg_zip(
-        name = "testos_bundle_" + variant,
-        srcs = [
-            ":testos_bundle_files_" + variant,
-        ],
         visibility = ["//metropolis/node/core/update/e2e:__pkg__"],
     )
 
diff --git a/metropolis/node/core/update/update.go b/metropolis/node/core/update/update.go
index 0dbc485..48f778d 100644
--- a/metropolis/node/core/update/update.go
+++ b/metropolis/node/core/update/update.go
@@ -4,7 +4,6 @@
 package update
 
 import (
-	"archive/zip"
 	"bytes"
 	"context"
 	"crypto/sha256"
@@ -13,12 +12,12 @@
 	"errors"
 	"fmt"
 	"io"
-	"net/http"
 	"os"
 	"path/filepath"
 	"regexp"
 	"strconv"
 	"strings"
+	"time"
 
 	"github.com/cenkalti/backoff/v4"
 	"golang.org/x/sys/unix"
@@ -27,12 +26,18 @@
 	"google.golang.org/protobuf/proto"
 
 	"source.monogon.dev/go/logging"
-	abloaderpb "source.monogon.dev/metropolis/node/core/abloader/spec"
+	mversion "source.monogon.dev/metropolis/version"
 	"source.monogon.dev/osbase/blockdev"
 	"source.monogon.dev/osbase/build/mkimage/osimage"
 	"source.monogon.dev/osbase/efivarfs"
 	"source.monogon.dev/osbase/gpt"
 	"source.monogon.dev/osbase/kexec"
+	ociosimage "source.monogon.dev/osbase/oci/osimage"
+	"source.monogon.dev/osbase/oci/registry"
+	"source.monogon.dev/version"
+
+	abloaderpb "source.monogon.dev/metropolis/node/core/abloader/spec"
+	apb "source.monogon.dev/metropolis/proto/api"
 )
 
 // Service contains data and functionality to perform A/B updates on a
@@ -255,41 +260,55 @@
 	return nil
 }
 
-// InstallBundle installs the bundle at the given HTTP(S) URL into the currently
-// inactive slot and sets that slot to boot next. If it doesn't return an error,
-// a reboot boots into the new slot.
-func (s *Service) InstallBundle(ctx context.Context, bundleURL string, withKexec bool) error {
+// InstallImage fetches the given image, installs it into the currently inactive
+// slot and sets that slot to boot next. If it doesn't return an error, a reboot
+// boots into the new slot.
+func (s *Service) InstallImage(ctx context.Context, imageRef *apb.OSImageRef, withKexec bool) error {
+	if imageRef == nil {
+		return fmt.Errorf("missing OS image in OS installation request")
+	}
+	if imageRef.Digest == "" {
+		return fmt.Errorf("missing digest in OS installation request")
+	}
 	if s.ESPPath == "" {
 		return errors.New("no ESP information provided to update service, cannot continue")
 	}
-	// Download into a buffer as ZIP files cannot efficiently be read from
-	// HTTP in Go as the ReaderAt has no way of indicating continuous sections,
-	// thus a ton of small range requests would need to be used, causing
-	// a huge latency penalty as well as costing a lot of money on typical
-	// object storages. This should go away when we switch to a better bundle
-	// format which can be streamed.
-	var bundleRaw bytes.Buffer
-	b := backoff.NewExponentialBackOff()
-	err := backoff.Retry(func() error {
-		return s.tryDownloadBundle(ctx, bundleURL, &bundleRaw)
-	}, backoff.WithContext(b, ctx))
-	if err != nil {
-		return fmt.Errorf("error downloading Metropolis bundle: %w", err)
+
+	downloadCtx, cancel := context.WithTimeout(ctx, 15*time.Minute)
+	defer cancel()
+
+	client := &registry.Client{
+		GetBackOff: func() backoff.BackOff {
+			return backoff.NewExponentialBackOff()
+		},
+		RetryNotify: func(err error, d time.Duration) {
+			s.Logger.Warningf("Error while fetching OS image, retrying in %v: %v", d, err)
+		},
+		UserAgent:  "MonogonOS/" + strings.TrimPrefix(version.Semver(mversion.Version), "v"),
+		Scheme:     imageRef.Scheme,
+		Host:       imageRef.Host,
+		Repository: imageRef.Repository,
 	}
-	bundle, err := zip.NewReader(bytes.NewReader(bundleRaw.Bytes()), int64(bundleRaw.Len()))
+
+	image, err := client.Read(downloadCtx, imageRef.Tag, imageRef.Digest)
 	if err != nil {
-		return fmt.Errorf("failed to open node bundle: %w", err)
+		return fmt.Errorf("failed to fetch OS image: %w", err)
 	}
-	efiPayload, err := bundle.Open("kernel_efi.efi")
+
+	osImage, err := ociosimage.Read(image)
 	if err != nil {
-		return fmt.Errorf("invalid bundle: %w", err)
+		return fmt.Errorf("failed to fetch OS image: %w", err)
 	}
-	defer efiPayload.Close()
-	systemImage, err := bundle.Open("verity_rootfs.img")
+
+	efiPayload, err := osImage.Payload("kernel.efi")
 	if err != nil {
-		return fmt.Errorf("invalid bundle: %w", err)
+		return fmt.Errorf("cannot open EFI payload in OS image: %w", err)
 	}
-	defer systemImage.Close()
+	systemImage, err := osImage.Payload("system")
+	if err != nil {
+		return fmt.Errorf("cannot open system image in OS image: %w", err)
+	}
+
 	activeSlot := s.CurrentlyRunningSlot()
 	if activeSlot == SlotInvalid {
 		return errors.New("unable to determine active slot, cannot continue")
@@ -300,8 +319,18 @@
 	if err != nil {
 		return status.Errorf(codes.Internal, "Inactive system slot unavailable: %v", err)
 	}
-	defer systemPart.Close()
-	if _, err := io.Copy(blockdev.NewRWS(systemPart), systemImage); err != nil {
+	systemImageContent, err := systemImage.Open()
+	if err != nil {
+		systemPart.Close()
+		return fmt.Errorf("failed to open system image: %w", err)
+	}
+	_, err = io.Copy(blockdev.NewRWS(systemPart), systemImageContent)
+	systemImageContent.Close()
+	closeErr := systemPart.Close()
+	if err == nil {
+		err = closeErr
+	}
+	if err != nil {
 		return status.Errorf(codes.Unavailable, "Failed to copy system image: %v", err)
 	}
 
@@ -310,7 +339,13 @@
 		return fmt.Errorf("failed to open boot file: %w", err)
 	}
 	defer bootFile.Close()
-	if _, err := io.Copy(bootFile, efiPayload); err != nil {
+	efiPayloadContent, err := efiPayload.Open()
+	if err != nil {
+		return fmt.Errorf("failed to open EFI payload: %w", err)
+	}
+	_, err = io.Copy(bootFile, efiPayloadContent)
+	efiPayloadContent.Close()
+	if err != nil {
 		return fmt.Errorf("failed to write boot file: %w", err)
 	}
 
@@ -331,38 +366,6 @@
 	return nil
 }
 
-func (*Service) tryDownloadBundle(ctx context.Context, bundleURL string, bundleRaw *bytes.Buffer) error {
-	bundleReq, err := http.NewRequestWithContext(ctx, "GET", bundleURL, nil)
-	if err != nil {
-		return fmt.Errorf("failed to create request: %w", err)
-	}
-	bundleRes, err := http.DefaultClient.Do(bundleReq)
-	if err != nil {
-		return fmt.Errorf("HTTP request failed: %w", err)
-	}
-	defer bundleRes.Body.Close()
-	switch bundleRes.StatusCode {
-	case http.StatusTooEarly, http.StatusTooManyRequests,
-		http.StatusInternalServerError, http.StatusBadGateway,
-		http.StatusServiceUnavailable, http.StatusGatewayTimeout:
-		return fmt.Errorf("HTTP error %d", bundleRes.StatusCode)
-	default:
-		// Non-standard code range used for proxy-related issue by various
-		// vendors. Treat as non-permanent error.
-		if bundleRes.StatusCode >= 520 && bundleRes.StatusCode < 599 {
-			return fmt.Errorf("HTTP error %d", bundleRes.StatusCode)
-		}
-		if bundleRes.StatusCode != 200 {
-			return backoff.Permanent(fmt.Errorf("HTTP error %d", bundleRes.StatusCode))
-		}
-	}
-	if _, err := bundleRaw.ReadFrom(bundleRes.Body); err != nil {
-		bundleRaw.Reset()
-		return err
-	}
-	return nil
-}
-
 // newMemfile creates a new file which is not located on a specific filesystem,
 // but is instead backed by anonymous memory.
 func newMemfile(name string, flags int) (*os.File, error) {