treewide: port everything to blockdev

This gets rid of most ad-hoc block device code, using blockdev for
everything. It also gets rid of diskfs for everything but tests. This
enables Metropolis to be installed on non-512-byte block sizes.

Change-Id: I644b5b68bb7bed8106585df3179674789031687a
Reviewed-on: https://review.monogon.dev/c/monogon/+/1873
Tested-by: Jenkins CI
Reviewed-by: Serge Bazanski <serge@monogon.tech>
diff --git a/cloud/agent/BUILD.bazel b/cloud/agent/BUILD.bazel
index e8b4ffd..793f15b 100644
--- a/cloud/agent/BUILD.bazel
+++ b/cloud/agent/BUILD.bazel
@@ -18,6 +18,7 @@
         "//metropolis/node/build/mkimage/osimage",
         "//metropolis/node/core/devmgr",
         "//metropolis/node/core/network",
+        "//metropolis/pkg/blockdev",
         "//metropolis/pkg/bootparam",
         "//metropolis/pkg/efivarfs",
         "//metropolis/pkg/logtree",
diff --git a/cloud/agent/install.go b/cloud/agent/install.go
index a1aec0d..3468cc9 100644
--- a/cloud/agent/install.go
+++ b/cloud/agent/install.go
@@ -5,6 +5,7 @@
 	"bytes"
 	"errors"
 	"fmt"
+	"io/fs"
 	"net/http"
 	"path/filepath"
 
@@ -13,11 +14,27 @@
 
 	bpb "source.monogon.dev/cloud/bmaas/server/api"
 	"source.monogon.dev/metropolis/node/build/mkimage/osimage"
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/efivarfs"
 	"source.monogon.dev/metropolis/pkg/logtree"
 	npb "source.monogon.dev/net/proto"
 )
 
+// FileSizedReader is a small adapter from fs.File to fs.SizedReader
+// Panics on Stat() failure, so should only be used with sources where Stat()
+// cannot fail.
+type FileSizedReader struct {
+	fs.File
+}
+
+func (f FileSizedReader) Size() int64 {
+	stat, err := f.Stat()
+	if err != nil {
+		panic(err)
+	}
+	return stat.Size()
+}
+
 // install dispatches OSInstallationRequests to the appropriate installer
 // method
 func install(req *bpb.OSInstallationRequest, netConfig *npb.Net, l logtree.LeveledLogger, isEFIBoot bool) error {
@@ -104,6 +121,11 @@
 		return fmt.Errorf("failed marshaling: %w", err)
 	}
 
+	rootDev, err := blockdev.Open(filepath.Join("/dev", req.RootDevice))
+	if err != nil {
+		return fmt.Errorf("failed to open root device: %w", err)
+	}
+
 	installParams := osimage.Params{
 		PartitionSize: osimage.PartitionSizeInfo{
 			ESP:    128,
@@ -111,9 +133,9 @@
 			Data:   128,
 		},
 		SystemImage:    systemImage,
-		EFIPayload:     efiPayload,
+		EFIPayload:     FileSizedReader{efiPayload},
 		NodeParameters: bytes.NewReader(nodeParamsRaw),
-		OutputPath:     filepath.Join("/dev", req.RootDevice),
+		Output:         rootDev,
 	}
 
 	be, err := osimage.Create(&installParams)
diff --git a/metropolis/cli/metroctl/BUILD.bazel b/metropolis/cli/metroctl/BUILD.bazel
index f013c42..111c6e6 100644
--- a/metropolis/cli/metroctl/BUILD.bazel
+++ b/metropolis/cli/metroctl/BUILD.bazel
@@ -46,6 +46,8 @@
         "//metropolis/cli/pkg/datafile",
         "//metropolis/node/core/identity",
         "//metropolis/node/core/rpc",
+        "//metropolis/pkg/blkio",
+        "//metropolis/pkg/fat32",
         "//metropolis/pkg/logtree",
         "//metropolis/proto/api",
         "//metropolis/proto/common",
diff --git a/metropolis/cli/metroctl/cmd_install.go b/metropolis/cli/metroctl/cmd_install.go
index 9ad7953..39790f1 100644
--- a/metropolis/cli/metroctl/cmd_install.go
+++ b/metropolis/cli/metroctl/cmd_install.go
@@ -15,6 +15,8 @@
 	"source.monogon.dev/metropolis/cli/metroctl/core"
 	clicontext "source.monogon.dev/metropolis/cli/pkg/context"
 	"source.monogon.dev/metropolis/cli/pkg/datafile"
+	"source.monogon.dev/metropolis/pkg/blkio"
+	"source.monogon.dev/metropolis/pkg/fat32"
 	"source.monogon.dev/metropolis/proto/api"
 	cpb "source.monogon.dev/metropolis/proto/common"
 )
@@ -49,30 +51,21 @@
 	size   uint64
 }
 
-func external(name, datafilePath string, flag *string) *externalFile {
+func external(name, datafilePath string, flag *string) fat32.SizedReader {
 	if flag == nil || *flag == "" {
 		df, err := datafile.Get(datafilePath)
 		if err != nil {
 			log.Fatalf("No %s specified", name)
 		}
-		return &externalFile{
-			reader: bytes.NewReader(df),
-			size:   uint64(len(df)),
-		}
+		return bytes.NewReader(df)
 	}
 
-	f, err := os.Open(*bundlePath)
+	f, err := blkio.NewFileReader(*bundlePath)
 	if err != nil {
 		log.Fatalf("Failed to open specified %s: %v", name, err)
 	}
-	st, err := f.Stat()
-	if err != nil {
-		log.Fatalf("Failed to stat specified %s: %v", name, err)
-	}
-	return &externalFile{
-		reader: f,
-		size:   uint64(st.Size()),
-	}
+
+	return f
 }
 
 func doGenUSB(cmd *cobra.Command, args []string) {
@@ -154,12 +147,10 @@
 	}
 
 	installerImageArgs := core.MakeInstallerImageArgs{
-		TargetPath:    args[0],
-		Installer:     installer.reader,
-		InstallerSize: installer.size,
-		NodeParams:    params,
-		Bundle:        bundle.reader,
-		BundleSize:    bundle.size,
+		TargetPath: args[0],
+		Installer:  installer,
+		NodeParams: params,
+		Bundle:     bundle,
 	}
 
 	log.Printf("Generating installer image (this can take a while, see issues/92).")
diff --git a/metropolis/cli/metroctl/core/BUILD.bazel b/metropolis/cli/metroctl/core/BUILD.bazel
index 133ff2e..7c1a0f4 100644
--- a/metropolis/cli/metroctl/core/BUILD.bazel
+++ b/metropolis/cli/metroctl/core/BUILD.bazel
@@ -14,11 +14,10 @@
         "//metropolis/node",
         "//metropolis/node/core/rpc",
         "//metropolis/node/core/rpc/resolver",
+        "//metropolis/pkg/blockdev",
+        "//metropolis/pkg/fat32",
+        "//metropolis/pkg/gpt",
         "//metropolis/proto/api",
-        "@com_github_diskfs_go_diskfs//:go-diskfs",
-        "@com_github_diskfs_go_diskfs//disk",
-        "@com_github_diskfs_go_diskfs//filesystem",
-        "@com_github_diskfs_go_diskfs//partition/gpt",
         "@io_k8s_client_go//pkg/apis/clientauthentication/v1:clientauthentication",
         "@io_k8s_client_go//tools/clientcmd",
         "@io_k8s_client_go//tools/clientcmd/api",
diff --git a/metropolis/cli/metroctl/core/install.go b/metropolis/cli/metroctl/core/install.go
index c20e38d..31b7328 100644
--- a/metropolis/cli/metroctl/core/install.go
+++ b/metropolis/cli/metroctl/core/install.go
@@ -1,168 +1,100 @@
 package core
 
 import (
+	"bytes"
 	"errors"
 	"fmt"
-	"io"
+	"math"
 	"os"
 
-	"github.com/diskfs/go-diskfs"
-	"github.com/diskfs/go-diskfs/disk"
-	"github.com/diskfs/go-diskfs/filesystem"
-	"github.com/diskfs/go-diskfs/partition/gpt"
 	"google.golang.org/protobuf/proto"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
+	"source.monogon.dev/metropolis/pkg/fat32"
+	"source.monogon.dev/metropolis/pkg/gpt"
 	"source.monogon.dev/metropolis/proto/api"
 )
 
-func mibToSectors(size uint64, logicalBlockSize int64) uint64 {
-	return (size * 1024 * 1024) / uint64(logicalBlockSize)
-}
-
-// Mask for aligning values to 1MiB boundaries. Go complains if you shift
-// 1 bits out of the value in a constant so the construction is a bit
-// convoluted.
-const align1MiBMask = (1<<44 - 1) << 20
-
-const MiB = 1024 * 1024
-
 type MakeInstallerImageArgs struct {
 	// Path to either a file or a disk which will contain the installer data.
 	TargetPath string
 
 	// Reader for the installer EFI executable. Mandatory.
-	Installer     io.Reader
-	InstallerSize uint64
+	Installer fat32.SizedReader
 
 	// Optional NodeParameters to be embedded for use by the installer.
 	NodeParams *api.NodeParameters
 
 	// Optional Reader for a Metropolis bundle for use by the installer.
-	Bundle     io.Reader
-	BundleSize uint64
+	Bundle fat32.SizedReader
 }
 
-// MakeInstallerImage generates an installer disk image containing a GPT
+// MakeInstallerImage generates an installer disk image containing a Table
 // partition table and a single FAT32 partition with an installer and optionally
 // with a bundle and/or Node Parameters.
 func MakeInstallerImage(args MakeInstallerImageArgs) error {
 	if args.Installer == nil {
 		return errors.New("Installer is mandatory")
 	}
-	if args.InstallerSize == 0 {
-		return errors.New("InstallerSize needs to be valid (>0)")
-	}
-	if args.Bundle != nil && args.BundleSize == 0 {
-		return errors.New("if a Bundle is passed BundleSize needs to be valid (>0)")
-	}
 
-	var err error
-	var nodeParamsRaw []byte
-	if args.NodeParams != nil {
-		nodeParamsRaw, err = proto.Marshal(args.NodeParams)
-		if err != nil {
-			return fmt.Errorf("failed to marshal node params: %w", err)
-		}
-	}
+	espRoot := fat32.Inode{Attrs: fat32.AttrDirectory}
 
-	var img *disk.Disk
-	// The following section is a bit ugly, it would technically be nicer to
-	// just pack all clusters of the FAT32 together, figure out how many were
-	// needed at the end and truncate the partition there. But that would
-	// require writing a new FAT32 writer, the effort to do that is in no way
-	// proportional to its advantages. So let's just do some conservative
-	// calculations on how much space we need and call it a day.
-
-	// ~4MiB FAT32 headers, 1MiB alignment overhead (bitmask drops up to 1MiB),
-	// 5% filesystem overhead
-	partitionSizeBytes := (uint64(float32(5*MiB+args.BundleSize+args.InstallerSize+uint64(len(nodeParamsRaw))) * 1.05)) & align1MiBMask
-	// FAT32 has a minimum partition size of 32MiB, so clamp the lower partition
-	// size to a notch more than that.
-	minimumSize := uint64(33 * MiB)
-	if partitionSizeBytes < minimumSize {
-		partitionSizeBytes = minimumSize
-	}
-	// If creating an image, create it with minimal size, i.e. 1MiB at each
-	// end for partitioning metadata and alignment.
-	// 1MiB alignment is used as that will essentially guarantee that any
-	// partition is aligned to whatever internal block size is used by the
-	// storage device. Especially flash-based storage likes to use much bigger
-	// blocks than advertised as sectors which can degrade performance when
-	// partitions are misaligned.
-	calculatedImageBytes := 2*MiB + partitionSizeBytes
-
-	if _, err = os.Stat(args.TargetPath); os.IsNotExist(err) {
-		img, err = diskfs.Create(args.TargetPath, int64(calculatedImageBytes), diskfs.Raw)
-	} else {
-		img, err = diskfs.Open(args.TargetPath)
-	}
-	if err != nil {
-		return fmt.Errorf("failed to create/open target: %w", err)
-	}
-	defer img.File.Close()
-	// This has an edge case where the data would technically fit but our 5%
-	// overhead are too conservative. But it is very rare and I don't really
-	// trust diskfs to generate good errors when it overflows so we'll just
-	// reject early.
-	if uint64(img.Size) < calculatedImageBytes {
-		return errors.New("target too small, data won't fit")
-	}
-
-	gptTable := &gpt.Table{
-		LogicalSectorSize:  int(img.LogicalBlocksize),
-		PhysicalSectorSize: int(img.PhysicalBlocksize),
-		ProtectiveMBR:      true,
-		Partitions: []*gpt.Partition{
-			{
-				Type:  gpt.EFISystemPartition,
-				Name:  "MetropolisInstaller",
-				Start: mibToSectors(1, img.LogicalBlocksize),
-				Size:  partitionSizeBytes,
-			},
-		},
-	}
-	if err := img.Partition(gptTable); err != nil {
-		return fmt.Errorf("failed to partition target: %w", err)
-	}
-	fs, err := img.CreateFilesystem(disk.FilesystemSpec{Partition: 1, FSType: filesystem.TypeFat32, VolumeLabel: "METRO_INST"})
-	if err != nil {
-		return fmt.Errorf("failed to create target filesystem: %w", err)
-	}
-
-	// Create EFI partition structure.
-	for _, dir := range []string{"/EFI", "/EFI/BOOT", "/metropolis-installer"} {
-		if err := fs.Mkdir(dir); err != nil {
-			panic(err)
-		}
-	}
 	// This needs to be a "Removable Media" according to the UEFI Specification
 	// V2.9 Section 3.5.1.1. This file is booted by any compliant UEFI firmware
 	// in absence of another bootable boot entry.
-	installerFile, err := fs.OpenFile("/EFI/BOOT/BOOTx64.EFI", os.O_CREATE|os.O_RDWR)
-	if err != nil {
-		panic(err)
+	if err := espRoot.PlaceFile("EFI/BOOT/BOOTx64.EFI", args.Installer); err != nil {
+		return err
 	}
-	if _, err := io.Copy(installerFile, args.Installer); err != nil {
-		return fmt.Errorf("failed to copy installer file: %w", err)
-	}
+
 	if args.NodeParams != nil {
-		nodeParamsFile, err := fs.OpenFile("/metropolis-installer/nodeparams.pb", os.O_CREATE|os.O_RDWR)
+		nodeParamsRaw, err := proto.Marshal(args.NodeParams)
 		if err != nil {
-			panic(err)
+			return fmt.Errorf("failed to marshal node params: %w", err)
 		}
-		_, err = nodeParamsFile.Write(nodeParamsRaw)
-		if err != nil {
-			return fmt.Errorf("failed to write node params: %w", err)
+		if err := espRoot.PlaceFile("metropolis-installer/nodeparams.pb", bytes.NewReader(nodeParamsRaw)); err != nil {
+			return err
 		}
 	}
 	if args.Bundle != nil {
-		bundleFile, err := fs.OpenFile("/metropolis-installer/bundle.bin", os.O_CREATE|os.O_RDWR)
+		if err := espRoot.PlaceFile("metropolis-installer/bundle.bin", args.Bundle); err != nil {
+			return err
+		}
+	}
+	var targetDev blockdev.BlockDev
+	var err error
+	targetDev, err = blockdev.Open(args.TargetPath)
+	if err != nil {
+		if errors.Is(err, os.ErrNotExist) {
+			targetDev, err = blockdev.CreateFile(args.TargetPath, 512, 1024*1024+4096)
+		}
 		if err != nil {
-			panic(err)
+			return fmt.Errorf("unable to open target device: %w", err)
 		}
-		if _, err := io.Copy(bundleFile, args.Bundle); err != nil {
-			return fmt.Errorf("failed to copy bundle: %w", err)
-		}
+	}
+	partTable, err := gpt.New(targetDev)
+	if err != nil {
+		return fmt.Errorf("target device has invalid geometry: %w", err)
+	}
+	esp := gpt.Partition{
+		Type: gpt.PartitionTypeEFISystem,
+		Name: "MetropolisInstaller",
+	}
+	fatOpts := fat32.Options{Label: "METRO_INST"}
+	// TODO(#254): Build and use dynamically-grown block devices
+	var espSize int64 = 512 * 1024 * 1024
+	if err := partTable.AddPartition(&esp, espSize); err != nil {
+		return fmt.Errorf("unable to create partition layout: %w", err)
+	}
+	if esp.BlockSize() > math.MaxUint16 {
+		return fmt.Errorf("block size (%d) too large for FAT32", esp.BlockSize())
+	}
+	fatOpts.BlockSize = uint16(esp.BlockSize())
+	fatOpts.BlockCount = uint32(esp.BlockCount())
+	if err := fat32.WriteFS(blockdev.NewRWS(esp), espRoot, fatOpts); err != nil {
+		return fmt.Errorf("failed to write FAT32: %w", err)
+	}
+	if err := partTable.Write(); err != nil {
+		return fmt.Errorf("unable to write partition table: %w", err)
 	}
 	return nil
 }
diff --git a/metropolis/installer/BUILD.bazel b/metropolis/installer/BUILD.bazel
index f3b6d1e..17fd8c7 100644
--- a/metropolis/installer/BUILD.bazel
+++ b/metropolis/installer/BUILD.bazel
@@ -13,6 +13,7 @@
     visibility = ["//visibility:private"],
     deps = [
         "//metropolis/node/build/mkimage/osimage",
+        "//metropolis/pkg/blockdev",
         "//metropolis/pkg/efivarfs",
         "//metropolis/pkg/sysfs",
         "@org_golang_x_sys//unix",
diff --git a/metropolis/installer/main.go b/metropolis/installer/main.go
index d3c638a..6d3fc6d 100644
--- a/metropolis/installer/main.go
+++ b/metropolis/installer/main.go
@@ -23,7 +23,7 @@
 	"archive/zip"
 	"errors"
 	"fmt"
-	"io"
+	"io/fs"
 	"os"
 	"path/filepath"
 	"strings"
@@ -33,6 +33,7 @@
 	"golang.org/x/sys/unix"
 
 	"source.monogon.dev/metropolis/node/build/mkimage/osimage"
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/efivarfs"
 	"source.monogon.dev/metropolis/pkg/sysfs"
 )
@@ -113,16 +114,13 @@
 
 		// Skip devices of insufficient size.
 		devPath := filepath.Join("/dev", devInfo.Name())
-		dev, err := os.Open(devPath)
+		dev, err := blockdev.Open(devPath)
 		if err != nil {
 			return nil, fmt.Errorf("couldn't open a block device at %q: %w", devPath, err)
 		}
-		size, err := unix.IoctlGetInt(int(dev.Fd()), unix.BLKGETSIZE64)
+		devSize := uint64(dev.BlockCount() * dev.BlockSize())
 		dev.Close()
-		if err != nil {
-			return nil, fmt.Errorf("couldn't probe the size of %q: %w", devPath, err)
-		}
-		if uint64(size) < minSize {
+		if devSize < minSize {
 			continue
 		}
 
@@ -131,51 +129,19 @@
 	return suitable, nil
 }
 
-// rereadPartitionTable causes the kernel to read the partition table present
-// in the block device at blkdevPath. It may return an error.
-func rereadPartitionTable(blkdevPath string) error {
-	dev, err := os.Open(blkdevPath)
-	if err != nil {
-		return fmt.Errorf("couldn't open the block device at %q: %w", blkdevPath, err)
-	}
-	defer dev.Close()
-	ret, err := unix.IoctlRetInt(int(dev.Fd()), unix.BLKRRPART)
-	if err != nil {
-		return fmt.Errorf("while doing an ioctl: %w", err)
-	}
-	if syscall.Errno(ret) == unix.EINVAL {
-		return fmt.Errorf("got an EINVAL from BLKRRPART ioctl")
-	}
-	return nil
+// FileSizedReader is a small adapter from fs.File to fs.SizedReader
+// Panics on Stat() failure, so should only be used with sources where Stat()
+// cannot fail.
+type FileSizedReader struct {
+	fs.File
 }
 
-// initializeSystemPartition writes image contents to the node's system
-// partition using the block device abstraction layer as opposed to slower
-// go-diskfs. tgtBlkdev must contain a path pointing to the block device
-// associated with the system partition. It may return an error.
-func initializeSystemPartition(image io.Reader, tgtBlkdev string) error {
-	// Check that tgtBlkdev points at an actual block device.
-	info, err := os.Stat(tgtBlkdev)
+func (f FileSizedReader) Size() int64 {
+	stat, err := f.Stat()
 	if err != nil {
-		return fmt.Errorf("couldn't stat the system partition at %q: %w", tgtBlkdev, err)
+		panic(err)
 	}
-	if info.Mode()&os.ModeDevice == 0 {
-		return fmt.Errorf("system partition path %q doesn't point to a block device", tgtBlkdev)
-	}
-
-	// Get the system partition's file descriptor.
-	sys, err := os.OpenFile(tgtBlkdev, os.O_WRONLY, 0600)
-	if err != nil {
-		return fmt.Errorf("couldn't open the system partition at %q: %w", tgtBlkdev, err)
-	}
-	defer sys.Close()
-	// Copy the system partition contents. Use a bigger buffer to optimize disk
-	// writes.
-	buf := make([]byte, mib)
-	if _, err := io.CopyBuffer(sys, image, buf); err != nil {
-		return fmt.Errorf("couldn't copy partition contents: %w", err)
-	}
-	return nil
+	return stat.Size()
 }
 
 func main() {
@@ -260,15 +226,9 @@
 			// whenever it's writing to block devices, such as now.
 			Data: 128,
 		},
-		// Due to a bug in go-diskfs causing slow writes, SystemImage is explicitly
-		// marked unused here, as system partition contents will be written using
-		// a workaround below instead.
-		// TODO(mateusz@monogon.tech): Address that bug either by patching go-diskfs
-		// or rewriting osimage.
-		SystemImage: nil,
-
-		EFIPayload:     efiPayload,
-		NodeParameters: nodeParameters,
+		SystemImage:    systemImage,
+		EFIPayload:     FileSizedReader{efiPayload},
+		NodeParameters: FileSizedReader{nodeParameters},
 	}
 	// Calculate the minimum target size based on the installation parameters.
 	minSize := uint64((installParams.PartitionSize.ESP +
@@ -287,7 +247,12 @@
 	tgtBlkdevName := blkDevs[0]
 	// Update the osimage parameters with a path pointing at the target device.
 	tgtBlkdevPath := filepath.Join("/dev", tgtBlkdevName)
-	installParams.OutputPath = tgtBlkdevPath
+
+	tgtBlockDev, err := blockdev.Open(tgtBlkdevPath)
+	if err != nil {
+		panicf("error opening target device: %v", err)
+	}
+	installParams.Output = tgtBlockDev
 
 	// Use osimage to partition the target block device and set up its ESP.
 	// Create will return an EFI boot entry on success.
@@ -296,23 +261,6 @@
 	if err != nil {
 		panicf("While installing: %v", err)
 	}
-	// The target device's partition table has just been updated. Re-read it to
-	// make the node system partition reachable through /dev.
-	if err := rereadPartitionTable(tgtBlkdevPath); err != nil {
-		panicf("While re-reading the partition table of %q: %v", tgtBlkdevPath, err)
-	}
-	// Look up the node's system partition path to be later used in the
-	// initialization step. It's always the second partition, right after
-	// the ESP.
-	sysBlkdevName, err := sysfs.PartitionBlockDevice(tgtBlkdevName, 2)
-	if err != nil {
-		panicf("While looking up the system partition: %v", err)
-	}
-	sysBlkdevPath := filepath.Join("/dev", sysBlkdevName)
-	// Copy the system partition contents.
-	if err := initializeSystemPartition(systemImage, sysBlkdevPath); err != nil {
-		panicf("While initializing the system partition at %q: %v", sysBlkdevPath, err)
-	}
 
 	// Create an EFI boot entry for Metropolis.
 	en, err := efivarfs.AddBootEntry(be)
@@ -325,6 +273,7 @@
 	}
 
 	// Reboot.
+	tgtBlockDev.Close()
 	unix.Sync()
 	logf("Installation completed. Rebooting.")
 	unix.Reboot(unix.LINUX_REBOOT_CMD_RESTART)
diff --git a/metropolis/installer/test/main.go b/metropolis/installer/test/main.go
index 7954c1d..3636658 100644
--- a/metropolis/installer/test/main.go
+++ b/metropolis/installer/test/main.go
@@ -126,12 +126,10 @@
 	installer := datafile.MustGet("metropolis/installer/test/kernel.efi")
 	bundle := datafile.MustGet("metropolis/installer/test/testos/testos_bundle.zip")
 	iargs := mctl.MakeInstallerImageArgs{
-		Installer:     bytes.NewBuffer(installer),
-		InstallerSize: uint64(len(installer)),
-		TargetPath:    installerImage,
-		NodeParams:    &api.NodeParameters{},
-		Bundle:        bytes.NewBuffer(bundle),
-		BundleSize:    uint64(len(bundle)),
+		Installer:  bytes.NewReader(installer),
+		TargetPath: installerImage,
+		NodeParams: &api.NodeParameters{},
+		Bundle:     bytes.NewReader(bundle),
 	}
 	if err := mctl.MakeInstallerImage(iargs); err != nil {
 		log.Fatalf("Couldn't create the installer image at %q: %v", installerImage, err)
@@ -217,7 +215,10 @@
 	defer ctxC()
 
 	// Prepare the block device image the installer will install to.
-	storagePath, err := getStorage(4096 + 128 + 128 + 1)
+	// Needs enough storage for a 4096 MiB system partition, a 128 MiB ESP and
+	// a 128MiB data partition. In addition at the start and end we need 1MiB
+	// for GPT headers and alignment.
+	storagePath, err := getStorage(4096 + 128 + 128 + 2)
 	defer os.Remove(storagePath)
 	if err != nil {
 		t.Errorf(err.Error())
@@ -246,18 +247,18 @@
 	// Check that the first partition is likely to be a valid ESP.
 	pi := ti.GetPartitions()
 	esp := (pi[0]).(*gpt.Partition)
-	if esp.Name != osimage.ESPVolumeLabel || esp.Start == 0 || esp.End == 0 {
+	if esp.Name != osimage.ESPLabel || esp.Start == 0 || esp.End == 0 {
 		t.Error("The node's ESP GPT entry looks off.")
 	}
 	// Verify the system partition's GPT entry.
 	system := (pi[1]).(*gpt.Partition)
-	if system.Name != osimage.SystemVolumeLabel || system.Start == 0 || system.End == 0 {
+	if system.Name != osimage.SystemLabel || system.Start == 0 || system.End == 0 {
 		t.Error("The node's system partition GPT entry looks off.")
 	}
 	// Verify the data partition's GPT entry.
 	data := (pi[2]).(*gpt.Partition)
-	if data.Name != osimage.DataVolumeLabel || data.Start == 0 || data.End == 0 {
-		t.Errorf("The node's data partition GPT entry looks off.")
+	if data.Name != osimage.DataLabel || data.Start == 0 || data.End == 0 {
+		t.Errorf("The node's data partition GPT entry looks off: %+v", data)
 	}
 	// Verify that there are no more partitions.
 	fourth := (pi[3]).(*gpt.Partition)
diff --git a/metropolis/installer/test/testos/BUILD.bazel b/metropolis/installer/test/testos/BUILD.bazel
index 6eded96..8d5f267 100644
--- a/metropolis/installer/test/testos/BUILD.bazel
+++ b/metropolis/installer/test/testos/BUILD.bazel
@@ -17,7 +17,7 @@
 
 efi_unified_kernel_image(
     name = "kernel_efi",
-    cmdline = "loglevel=0 console=ttyS0 init=/init",
+    cmdline = "quiet console=ttyS0 init=/init",
     kernel = "//third_party/linux",
     verity = ":verity_rootfs",
 )
diff --git a/metropolis/node/build/mkimage/BUILD.bazel b/metropolis/node/build/mkimage/BUILD.bazel
index 2bf8229..2c7d699 100644
--- a/metropolis/node/build/mkimage/BUILD.bazel
+++ b/metropolis/node/build/mkimage/BUILD.bazel
@@ -5,7 +5,11 @@
     srcs = ["main.go"],
     importpath = "source.monogon.dev/metropolis/node/build/mkimage",
     visibility = ["//visibility:private"],
-    deps = ["//metropolis/node/build/mkimage/osimage"],
+    deps = [
+        "//metropolis/node/build/mkimage/osimage",
+        "//metropolis/pkg/blkio",
+        "//metropolis/pkg/blockdev",
+    ],
 )
 
 go_binary(
diff --git a/metropolis/node/build/mkimage/main.go b/metropolis/node/build/mkimage/main.go
index b885618..077348e 100644
--- a/metropolis/node/build/mkimage/main.go
+++ b/metropolis/node/build/mkimage/main.go
@@ -32,6 +32,8 @@
 	"os"
 
 	"source.monogon.dev/metropolis/node/build/mkimage/osimage"
+	"source.monogon.dev/metropolis/pkg/blkio"
+	"source.monogon.dev/metropolis/pkg/blockdev"
 )
 
 func main() {
@@ -40,26 +42,27 @@
 		efiPayload  string
 		systemImage string
 		nodeParams  string
+		outputPath  string
+		diskUUID    string
 		cfg         osimage.Params
 	)
 	flag.StringVar(&efiPayload, "efi", "", "Path to the UEFI payload used")
 	flag.StringVar(&systemImage, "system", "", "Path to the system partition image used")
 	flag.StringVar(&nodeParams, "node_parameters", "", "Path to Node Parameters to be written to the ESP (default: don't write Node Parameters)")
-	flag.StringVar(&cfg.OutputPath, "out", "", "Path to the resulting disk image or block device")
-	flag.Uint64Var(&cfg.PartitionSize.Data, "data_partition_size", 2048, "Override the data partition size (default 2048 MiB). Used only when generating image files.")
-	flag.Uint64Var(&cfg.PartitionSize.ESP, "esp_partition_size", 128, "Override the ESP partition size (default: 128MiB)")
-	flag.Uint64Var(&cfg.PartitionSize.System, "system_partition_size", 1024, "Override the System partition size (default: 1024MiB)")
-	flag.StringVar(&cfg.DiskGUID, "GUID", "", "Disk GUID marked in the resulting image's partition table (default: randomly generated)")
+	flag.StringVar(&outputPath, "out", "", "Path to the resulting disk image or block device")
+	flag.Int64Var(&cfg.PartitionSize.Data, "data_partition_size", 2048, "Override the data partition size (default 2048 MiB). Used only when generating image files.")
+	flag.Int64Var(&cfg.PartitionSize.ESP, "esp_partition_size", 128, "Override the ESP partition size (default: 128MiB)")
+	flag.Int64Var(&cfg.PartitionSize.System, "system_partition_size", 1024, "Override the System partition size (default: 1024MiB)")
+	flag.StringVar(&diskUUID, "GUID", "", "Disk GUID marked in the resulting image's partition table (default: randomly generated)")
 	flag.Parse()
 
 	// Open the input files for osimage.Create, fill in reader objects and
 	// metadata in osimage.Params.
 	// Start with the EFI Payload the OS will boot from.
-	p, err := os.Open(efiPayload)
+	p, err := blkio.NewFileReader(efiPayload)
 	if err != nil {
 		log.Fatalf("while opening the EFI payload at %q: %v", efiPayload, err)
 	}
-	defer p.Close()
 	cfg.EFIPayload = p
 
 	// Attempt to open the system image if its path is set. In case the path
@@ -76,14 +79,19 @@
 
 	// Attempt to open the node parameters file if its path is set.
 	if nodeParams != "" {
-		np, err := os.Open(nodeParams)
+		np, err := blkio.NewFileReader(nodeParams)
 		if err != nil {
 			log.Fatalf("while opening node parameters at %q: %v", nodeParams, err)
 		}
-		defer np.Close()
 		cfg.NodeParameters = np
 	}
 
+	// TODO(#254): Build and use dynamically-grown block devices
+	cfg.Output, err = blockdev.CreateFile(outputPath, 512, 10*1024*1024)
+	if err != nil {
+		panic(err)
+	}
+
 	// Write the parametrized OS image.
 	if _, err := osimage.Create(&cfg); err != nil {
 		log.Fatalf("while creating a Metropolis OS image: %v", err)
diff --git a/metropolis/node/build/mkimage/osimage/BUILD.bazel b/metropolis/node/build/mkimage/osimage/BUILD.bazel
index 35bfb9c..cfbf736 100644
--- a/metropolis/node/build/mkimage/osimage/BUILD.bazel
+++ b/metropolis/node/build/mkimage/osimage/BUILD.bazel
@@ -6,11 +6,10 @@
     importpath = "source.monogon.dev/metropolis/node/build/mkimage/osimage",
     visibility = ["//visibility:public"],
     deps = [
+        "//metropolis/pkg/blockdev",
         "//metropolis/pkg/efivarfs",
-        "@com_github_diskfs_go_diskfs//:go-diskfs",
-        "@com_github_diskfs_go_diskfs//disk",
-        "@com_github_diskfs_go_diskfs//filesystem",
-        "@com_github_diskfs_go_diskfs//partition/gpt",
+        "//metropolis/pkg/fat32",
+        "//metropolis/pkg/gpt",
         "@com_github_google_uuid//:uuid",
     ],
 )
diff --git a/metropolis/node/build/mkimage/osimage/osimage.go b/metropolis/node/build/mkimage/osimage/osimage.go
index 97fa7c0..7098439 100644
--- a/metropolis/node/build/mkimage/osimage/osimage.go
+++ b/metropolis/node/build/mkimage/osimage/osimage.go
@@ -21,379 +21,157 @@
 import (
 	"fmt"
 	"io"
-	"os"
+	"strings"
 
-	diskfs "github.com/diskfs/go-diskfs"
-	"github.com/diskfs/go-diskfs/disk"
-	"github.com/diskfs/go-diskfs/filesystem"
-	"github.com/diskfs/go-diskfs/partition/gpt"
 	"github.com/google/uuid"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/efivarfs"
+	"source.monogon.dev/metropolis/pkg/fat32"
+	"source.monogon.dev/metropolis/pkg/gpt"
+)
+
+var (
+	SystemAType = uuid.MustParse("ee96054b-f6d0-4267-aaaa-724b2afea74c")
+	SystemBType = uuid.MustParse("ee96054b-f6d0-4267-bbbb-724b2afea74c")
+
+	DataType = uuid.MustParse("9eeec464-6885-414a-b278-4305c51f7966")
 )
 
 const (
-	systemPartitionType = gpt.Type("ee96055b-f6d0-4267-8bbb-724b2afea74c")
-	SystemVolumeLabel   = "METROPOLIS-SYSTEM"
-
-	dataPartitionType = gpt.Type("9eeec464-6885-414a-b278-4305c51f7966")
-	DataVolumeLabel   = "METROPOLIS-NODE-DATA"
-
-	ESPVolumeLabel = "ESP"
+	SystemLabel = "METROPOLIS-SYSTEM"
+	DataLabel   = "METROPOLIS-NODE-DATA"
+	ESPLabel    = "ESP"
 
 	EFIPayloadPath = "/EFI/BOOT/BOOTx64.EFI"
-	nodeParamsPath = "/metropolis/parameters.pb"
-
-	mib = 1024 * 1024
+	nodeParamsPath = "metropolis/parameters.pb"
 )
 
-// put creates a file on the target filesystem fs and fills it with
-// contents read from an io.Reader object src.
-func put(fs filesystem.FileSystem, dst string, src io.Reader) error {
-	target, err := fs.OpenFile(dst, os.O_CREATE|os.O_RDWR)
-	if err != nil {
-		return fmt.Errorf("while opening %q: %w", dst, err)
-	}
-
-	// If this is streamed (e.g. using io.Copy) it exposes a bug in diskfs, so
-	// do it in one go.
-	// TODO(mateusz@monogon.tech): Investigate the bug.
-	data, err := io.ReadAll(src)
-	if err != nil {
-		return fmt.Errorf("while reading %q: %w", src, err)
-	}
-	if _, err := target.Write(data); err != nil {
-		return fmt.Errorf("while writing to %q: %w", dst, err)
-	}
-	return nil
-}
-
-// initializeESP creates an ESP filesystem in a partition specified by
-// index. It then creates the EFI executable and copies into it contents
-// of the reader object exec, which must not be nil. The node parameters
-// file is optionally created if params is not nil. initializeESP may return
-// an error.
-func initializeESP(image *disk.Disk, index int, exec, params io.Reader) error {
-	// Create a FAT ESP filesystem inside a partition pointed to by
-	// index.
-	spec := disk.FilesystemSpec{
-		Partition:   index,
-		FSType:      filesystem.TypeFat32,
-		VolumeLabel: ESPVolumeLabel,
-	}
-	fs, err := image.CreateFilesystem(spec)
-	if err != nil {
-		return fmt.Errorf("while creating an ESP filesystem: %w", err)
-	}
-
-	// Create the EFI partition structure.
-	for _, dir := range []string{"/EFI", "/EFI/BOOT", "/metropolis"} {
-		if err := fs.Mkdir(dir); err != nil {
-			return fmt.Errorf("while creating %q: %w", dir, err)
-		}
-	}
-
-	// Copy the EFI payload to the newly created filesystem.
-	if exec == nil {
-		return fmt.Errorf("exec must not be nil")
-	}
-	if err := put(fs, EFIPayloadPath, exec); err != nil {
-		return fmt.Errorf("while writing an EFI payload: %w", err)
-	}
-
-	if params != nil {
-		// Copy Node Parameters into the ESP.
-		if err := put(fs, nodeParamsPath, params); err != nil {
-			return fmt.Errorf("while writing node parameters: %w", err)
-		}
-	}
-	return nil
-}
-
-// zeroSrc is a source of null bytes implementing io.Reader. It acts as a
-// helper data type.
-type zeroSrc struct{}
-
-// Read implements io.Reader for zeroSrc. It fills b with zero bytes. The
-// returned error is always nil.
-func (_ zeroSrc) Read(b []byte) (n int, err error) {
-	for i := range b {
-		b[i] = 0
-	}
-	return len(b), nil
-}
-
-// initializeSystemPartition copies system partition contents into a partition
-// at index. The remaining partition space is zero-padded. This function may
-// return an error.
-func initializeSystemPartition(image *disk.Disk, index int, contents io.Reader) error {
-	// Check the parameters.
-	if contents == nil {
-		return fmt.Errorf("system partition contents parameter must not be nil")
-	}
-	if index <= 0 {
-		return fmt.Errorf("partition index must be greater than zero")
-	}
-
-	// Get the system partition's size.
-	table, err := image.GetPartitionTable()
-	if err != nil {
-		return fmt.Errorf("while accessing a go-diskfs partition table: %w", err)
-	}
-	partitions := table.GetPartitions()
-	if index > len(partitions) {
-		return fmt.Errorf("partition index out of bounds")
-	}
-	size := partitions[index-1].GetSize()
-
-	// Copy the contents of the Metropolis system image into the system partition
-	// at partitionIndex. Zero-pad the remaining space.
-	var zero zeroSrc
-	sys := io.LimitReader(io.MultiReader(contents, zero), size)
-	if _, err := image.WritePartitionContents(index, sys); err != nil {
-		return fmt.Errorf("while copying the system partition: %w", err)
-	}
-	return nil
-}
-
-// mibToSectors converts a size expressed in mebibytes to a number of
-// sectors needed to store data of that size. sectorSize parameter
-// specifies the size of a logical sector.
-func mibToSectors(size, sectorSize uint64) uint64 {
-	return (size * mib) / sectorSize
-}
-
 // PartitionSizeInfo contains parameters used during partition table
 // initialization and, in case of image files, space allocation.
 type PartitionSizeInfo struct {
 	// Size of the EFI System Partition (ESP), in mebibytes. The size must
 	// not be zero.
-	ESP uint64
+	ESP int64
 	// Size of the Metropolis system partition, in mebibytes. The partition
 	// won't be created if the size is zero.
-	System uint64
+	System int64
 	// Size of the Metropolis data partition, in mebibytes. The partition
 	// won't be created if the size is zero. If the image is output to a
 	// block device, the partition will be extended to fill the remaining
 	// space.
-	Data uint64
-}
-
-// partitionList stores partition definitions in an ascending order.
-type partitionList []*gpt.Partition
-
-// appendPartition puts a new partition at the end of a partitionList,
-// automatically calculating its start and end markers based on data in
-// the list and the argument psize. A partition type and a name are
-// assigned to the partition. The containing image is used to calculate
-// sector addresses based on its logical block size.
-func (pl *partitionList) appendPartition(image *disk.Disk, ptype gpt.Type, pname string, psize uint64) {
-	// Calculate the start and end marker.
-	var pstart, pend uint64
-	if len(*pl) != 0 {
-		pstart = (*pl)[len(*pl)-1].End + 1
-	} else {
-		pstart = mibToSectors(1, uint64(image.LogicalBlocksize))
-	}
-	pend = pstart + mibToSectors(psize, uint64(image.LogicalBlocksize)) - 1
-
-	// Put the new partition at the end of the list.
-	*pl = append(*pl, &gpt.Partition{
-		Type:  ptype,
-		Name:  pname,
-		Start: pstart,
-		End:   pend,
-	})
-}
-
-// extendLastPartition moves the end marker of the last partition in a
-// partitionList to the end of image, assigning all of the remaining free
-// space to it. It may return an error.
-func (pl *partitionList) extendLastPartition(image *disk.Disk) error {
-	if len(*pl) == 0 {
-		return fmt.Errorf("no partitions defined")
-	}
-	if image.Size == 0 {
-		return fmt.Errorf("the image size mustn't be zero")
-	}
-	if image.LogicalBlocksize == 0 {
-		return fmt.Errorf("the image's logical block size mustn't be zero")
-	}
-
-	// The last 33 blocks are occupied by the Secondary GPT.
-	(*pl)[len(*pl)-1].End = uint64(image.Size/image.LogicalBlocksize) - 33
-	return nil
-}
-
-// initializePartitionTable applies a Metropolis-compatible GPT partition
-// table to an image. Logical and physical sector sizes are based on
-// block sizes exposed by Disk. Partition extents are defined by the size
-// argument. A diskGUID is associated with the partition table. In an event
-// the table couldn't have been applied, the function will return an error.
-func initializePartitionTable(image *disk.Disk, size *PartitionSizeInfo, diskGUID string) error {
-	// Start with preparing a partition list.
-	var pl partitionList
-	// Create the ESP.
-	if size.ESP == 0 {
-		return fmt.Errorf("ESP size mustn't be zero")
-	}
-	pl.appendPartition(image, gpt.EFISystemPartition, ESPVolumeLabel, size.ESP)
-	// Create the system partition only if its size is specified.
-	if size.System != 0 {
-		pl.appendPartition(image, systemPartitionType, SystemVolumeLabel, size.System)
-	}
-	// Create the data partition only if its size is specified.
-	if size.Data != 0 {
-		// Don't specify the last partition's length, as it will be extended
-		// to fit the image size anyway. size.Data will still be used in the
-		// space allocation step.
-		pl.appendPartition(image, dataPartitionType, DataVolumeLabel, 0)
-		if err := pl.extendLastPartition(image); err != nil {
-			return fmt.Errorf("while extending the last partition: %w", err)
-		}
-	}
-
-	// Build and apply the partition table.
-	table := &gpt.Table{
-		LogicalSectorSize:  int(image.LogicalBlocksize),
-		PhysicalSectorSize: int(image.PhysicalBlocksize),
-		ProtectiveMBR:      true,
-		GUID:               diskGUID,
-		Partitions:         pl,
-	}
-	if err := image.Partition(table); err != nil {
-		// Return the error unwrapped.
-		return err
-	}
-	return nil
+	Data int64
 }
 
 // Params contains parameters used by Create to build a Metropolis OS
 // image.
 type Params struct {
-	// OutputPath is the path an OS image will be written to. If the path
-	// points to an existing block device, the data partition will be
-	// extended to fill it entirely. Otherwise, a regular image file will
-	// be created at OutputPath. The path must not point to an existing
-	// regular file.
-	OutputPath string
+	// Output is the block device to which the OS image is written.
+	Output blockdev.BlockDev
 	// EFIPayload provides contents of the EFI payload file. It must not be
 	// nil.
-	EFIPayload io.Reader
+	EFIPayload fat32.SizedReader
 	// SystemImage provides contents of the Metropolis system partition.
 	// If nil, no contents will be copied into the partition.
 	SystemImage io.Reader
 	// NodeParameters provides contents of the node parameters file. If nil,
 	// the node parameters file won't be created in the target ESP
 	// filesystem.
-	NodeParameters io.Reader
-	// DiskGUID is a unique identifier of the image and a part of GPT
+	NodeParameters fat32.SizedReader
+	// DiskGUID is a unique identifier of the image and a part of Table
 	// header. It's optional and can be left blank if the identifier is
 	// to be randomly generated. Setting it to a predetermined value can
 	// help in implementing reproducible builds.
-	DiskGUID string
+	DiskGUID uuid.UUID
 	// PartitionSize specifies a size for the ESP, Metropolis System and
 	// Metropolis data partition.
 	PartitionSize PartitionSizeInfo
 }
 
-// Create writes a Metropolis OS image to either a newly created regular
-// file or a block device. The image is parametrized by the params
-// argument. In case a regular file already exists at params.OutputPath,
-// the function will fail. It returns nil on success or an error, if one
-// did occur.
+const Mi = 1024 * 1024
+
+// Create writes a Metropolis OS image to a block device.
 func Create(params *Params) (*efivarfs.LoadOption, error) {
-	// Validate each parameter before use.
-	if params.OutputPath == "" {
-		return nil, fmt.Errorf("image output path must be set")
+	// Discard the entire device, we're going to write new data over it.
+	// Ignore errors, this is only advisory.
+	params.Output.Discard(0, params.Output.BlockCount())
+
+	tbl, err := gpt.New(params.Output)
+	if err != nil {
+		return nil, fmt.Errorf("invalid block device: %w", err)
+	}
+	tbl.ID = params.DiskGUID
+	esp := gpt.Partition{
+		Type: gpt.PartitionTypeEFISystem,
+		Name: ESPLabel,
+	}
+	if err := tbl.AddPartition(&esp, params.PartitionSize.ESP*Mi); err != nil {
+		return nil, fmt.Errorf("failed to allocate ESP: %w", err)
 	}
 
-	// Learn whether we're creating a new image or writing to an existing
-	// block device by stat-ing the output path parameter.
-	outInfo, err := os.Stat(params.OutputPath)
-	if err != nil && !os.IsNotExist(err) {
+	rootInode := fat32.Inode{
+		Attrs: fat32.AttrDirectory,
+	}
+	if err := rootInode.PlaceFile(strings.TrimPrefix(EFIPayloadPath, "/"), params.EFIPayload); err != nil {
 		return nil, err
 	}
-
-	// Calculate the image size (bytes) by summing up partition sizes
-	// (mebibytes).
-	minSize := (int64(params.PartitionSize.ESP) +
-		int64(params.PartitionSize.System) +
-		int64(params.PartitionSize.Data) + 1) * mib
-	var diskImg *disk.Disk
-	if !os.IsNotExist(err) && outInfo.Mode()&os.ModeDevice != 0 {
-		// Open the block device. The data partition size parameter won't
-		// matter in this case, as said partition will be extended till the
-		// end of device.
-		diskImg, err = diskfs.Open(params.OutputPath)
-		if err != nil {
-			return nil, fmt.Errorf("couldn't open the block device at %q: %w", params.OutputPath, err)
-		}
-		// Make sure there's at least minSize space available on the block
-		// device.
-		if minSize > diskImg.Size {
-			return nil, fmt.Errorf("not enough space available on the block device at %q", params.OutputPath)
-		}
-	} else {
-		// Attempt to create an image file at params.OutputPath. diskfs.Create
-		// will abort in case a file already exists at the given path.
-		// Calculate the image size expressed in bytes by summing up declared
-		// partition lengths.
-		diskImg, err = diskfs.Create(params.OutputPath, minSize, diskfs.Raw)
-		if err != nil {
-			return nil, fmt.Errorf("couldn't create a disk image at %q: %w", params.OutputPath, err)
+	if params.NodeParameters != nil {
+		if err := rootInode.PlaceFile(nodeParamsPath, params.NodeParameters); err != nil {
+			return nil, err
 		}
 	}
-
-	// Go through the initialization steps, starting with applying a
-	// partition table according to params.
-	if err := initializePartitionTable(diskImg, &params.PartitionSize, params.DiskGUID); err != nil {
-		return nil, fmt.Errorf("failed to initialize the partition table: %w", err)
+	if err := fat32.WriteFS(blockdev.NewRWS(esp), rootInode, fat32.Options{
+		BlockSize:  uint16(esp.BlockSize()),
+		BlockCount: uint32(esp.BlockCount()),
+		Label:      "MNGN_BOOT",
+	}); err != nil {
+		return nil, fmt.Errorf("failed to write FAT32: %w", err)
 	}
-	// The system partition will be created only if its specified size isn't
-	// equal to zero, making the initialization step optional as well. In
-	// addition, params.SystemImage must be set.
+
+	// Create the system partition only if its size is specified.
 	if params.PartitionSize.System != 0 && params.SystemImage != nil {
-		if err := initializeSystemPartition(diskImg, 2, params.SystemImage); err != nil {
-			return nil, fmt.Errorf("failed to initialize the system partition: %w", err)
+		systemPartitionA := gpt.Partition{
+			Type: SystemAType,
+			Name: SystemLabel,
+		}
+		if err := tbl.AddPartition(&systemPartitionA, params.PartitionSize.System*Mi); err != nil {
+			return nil, fmt.Errorf("failed to allocate system partition A: %w", err)
+		}
+		if _, err := io.Copy(blockdev.NewRWS(systemPartitionA), params.SystemImage); err != nil {
+			return nil, fmt.Errorf("failed to write system partition A: %w", err)
 		}
 	} else if params.PartitionSize.System == 0 && params.SystemImage != nil {
 		// Safeguard against contradicting parameters.
 		return nil, fmt.Errorf("the system image parameter was passed while the associated partition size is zero")
 	}
-	// Attempt to initialize the ESP unconditionally, as it's the only
-	// partition guaranteed to be created regardless of params.PartitionSize.
-	if err := initializeESP(diskImg, 1, params.EFIPayload, params.NodeParameters); err != nil {
-		return nil, fmt.Errorf("failed to initialize the ESP: %w", err)
+	// Create the data partition only if its size is specified.
+	if params.PartitionSize.Data != 0 {
+		dataPartition := gpt.Partition{
+			Type: DataType,
+			Name: DataLabel,
+		}
+		if err := tbl.AddPartition(&dataPartition, -1); err != nil {
+			return nil, fmt.Errorf("failed to allocate data partition: %w", err)
+		}
 	}
-	// The data partition, even if created, is always left uninitialized.
 
-	// Build an EFI boot entry pointing to the image's ESP. go-diskfs won't let
-	// you do that after you close the image.
-	t, err := diskImg.GetPartitionTable()
-	p := t.GetPartitions()
-	esp := (p[0]).(*gpt.Partition)
-	guid, err := uuid.Parse(esp.GUID)
-	if err != nil {
-		return nil, fmt.Errorf("couldn't parse the GPT GUID: %w", err)
+	if err := tbl.Write(); err != nil {
+		return nil, fmt.Errorf("failed to write Table: %w", err)
 	}
-	be := efivarfs.LoadOption{
-		Description: "Metropolis",
+
+	// Build an EFI boot entry pointing to the image's ESP.
+	return &efivarfs.LoadOption{
+		Description: "Metropolis Slot A",
 		FilePath: efivarfs.DevicePath{
 			&efivarfs.HardDrivePath{
 				PartitionNumber:     1,
-				PartitionStartBlock: esp.Start,
-				PartitionSizeBlocks: esp.End - esp.Start + 1,
+				PartitionStartBlock: esp.FirstBlock,
+				PartitionSizeBlocks: esp.SizeBlocks(),
 				PartitionMatch: efivarfs.PartitionGPT{
-					PartitionUUID: guid,
+					PartitionUUID: esp.ID,
 				},
 			},
 			efivarfs.FilePath(EFIPayloadPath),
 		},
-	}
-	// Close the image and return the EFI boot entry.
-	if err := diskImg.File.Close(); err != nil {
-		return nil, fmt.Errorf("failed to finalize image: %w", err)
-	}
-	return &be, nil
+	}, nil
 }
diff --git a/metropolis/node/core/localstorage/crypt/BUILD.bazel b/metropolis/node/core/localstorage/crypt/BUILD.bazel
index b330bf1..44188d1 100644
--- a/metropolis/node/core/localstorage/crypt/BUILD.bazel
+++ b/metropolis/node/core/localstorage/crypt/BUILD.bazel
@@ -13,6 +13,7 @@
     importpath = "source.monogon.dev/metropolis/node/core/localstorage/crypt",
     visibility = ["//metropolis/node/core/localstorage:__subpackages__"],
     deps = [
+        "//metropolis/pkg/blockdev",
         "//metropolis/pkg/devicemapper",
         "//metropolis/pkg/efivarfs",
         "//metropolis/pkg/gpt",
diff --git a/metropolis/node/core/localstorage/crypt/blockdev.go b/metropolis/node/core/localstorage/crypt/blockdev.go
index 353164a..0dadb6d 100644
--- a/metropolis/node/core/localstorage/crypt/blockdev.go
+++ b/metropolis/node/core/localstorage/crypt/blockdev.go
@@ -23,11 +23,11 @@
 	"path/filepath"
 	"strconv"
 	"strings"
-	"unsafe"
 
 	"github.com/google/uuid"
 	"golang.org/x/sys/unix"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/efivarfs"
 	"source.monogon.dev/metropolis/pkg/gpt"
 	"source.monogon.dev/metropolis/pkg/supervisor"
@@ -91,14 +91,15 @@
 		return nil
 	}
 
-	table, err := data.readPartitionTable()
+	blkdev, err := blockdev.Open(fmt.Sprintf("/dev/%v", data["DEVNAME"]))
 	if err != nil {
-		return fmt.Errorf("when reading disk info: %w", err)
+		return fmt.Errorf("failed to open block device: %w", err)
 	}
+	defer blkdev.Close()
 
-	// Not a normal block device or not a gpt table.
-	if table == nil {
-		return nil
+	table, err := gpt.Read(blkdev)
+	if err != nil {
+		return nil // Probably just not a GPT-partitioned disk
 	}
 
 	skipDisk := false
@@ -149,7 +150,6 @@
 		return err
 	}
 
-	// TODO(tim): Is this safe? Are we actually using the partition number for the slice index?
 	part := table.Partitions[pi.partNumber-1]
 
 	nodePath := nodePathForPartitionType(part.Type)
@@ -251,35 +251,3 @@
 
 	return
 }
-
-// readPartitionTable tries to read a GPT partition table based on the blockUEvent
-// data. It returns nil when either the block device is not a regular device
-// or it fails to parse the GPT table.
-func (b blockUEvent) readPartitionTable() (*gpt.Table, error) {
-	// TODO(lorenz): This extraction code is all a bit hairy, will get
-	// replaced by blockdev shortly.
-	blkdev, err := os.Open(fmt.Sprintf("/dev/%v", b["DEVNAME"]))
-	if err != nil {
-		return nil, fmt.Errorf("failed to open block device: %w", err)
-	}
-	defer blkdev.Close()
-
-	blockSize, err := unix.IoctlGetUint32(int(blkdev.Fd()), unix.BLKSSZGET)
-	if err != nil {
-		return nil, nil // This is not a regular block device
-	}
-
-	var sizeBytes uint64
-	_, _, err = unix.Syscall(unix.SYS_IOCTL, blkdev.Fd(), unix.BLKGETSIZE64, uintptr(unsafe.Pointer(&sizeBytes)))
-	if err != unix.Errno(0) {
-		return nil, fmt.Errorf("failed to get device size: %w", err)
-	}
-
-	blkdev.Seek(int64(blockSize), 0)
-	table, err := gpt.Read(blkdev, int64(blockSize), int64(sizeBytes)/int64(blockSize))
-	if err != nil {
-		return nil, nil // Probably just not a GPT-partitioned disk
-	}
-
-	return table, nil
-}
diff --git a/metropolis/node/core/localstorage/crypt/crypt.go b/metropolis/node/core/localstorage/crypt/crypt.go
index af7451b..0336832 100644
--- a/metropolis/node/core/localstorage/crypt/crypt.go
+++ b/metropolis/node/core/localstorage/crypt/crypt.go
@@ -36,13 +36,9 @@
 package crypt
 
 import (
-	"errors"
 	"fmt"
-	"os"
-	"syscall"
-	"unsafe"
 
-	"golang.org/x/sys/unix"
+	"source.monogon.dev/metropolis/pkg/blockdev"
 )
 
 // Mode of block device encryption and/or authentication, if any. See the
@@ -93,37 +89,6 @@
 	panic("invalid mode " + m)
 }
 
-// getSizeBytes returns the size of a block device in bytes.
-func getSizeBytes(path string) (uint64, error) {
-	blkdev, err := os.Open(path)
-	if err != nil {
-		return 0, fmt.Errorf("failed to open block device: %w", err)
-	}
-	defer blkdev.Close()
-
-	var sizeBytes uint64
-	_, _, err = unix.Syscall(unix.SYS_IOCTL, blkdev.Fd(), unix.BLKGETSIZE64, uintptr(unsafe.Pointer(&sizeBytes)))
-	if err != unix.Errno(0) {
-		return 0, fmt.Errorf("failed to get device size: %w", err)
-	}
-	return sizeBytes, nil
-}
-
-// getBlockSize returns the size of a block device's sector in bytes.
-func getBlockSize(path string) (uint32, error) {
-	blkdev, err := os.Open(path)
-	if err != nil {
-		return 0, fmt.Errorf("failed to open block device: %w", err)
-	}
-	defer blkdev.Close()
-
-	blockSize, err := unix.IoctlGetUint32(int(blkdev.Fd()), unix.BLKSSZGET)
-	if err != nil {
-		return 0, fmt.Errorf("BLKSSZGET: %w", err)
-	}
-	return blockSize, nil
-}
-
 // Map sets up an underlying block device (at path 'underlying') for access.
 // Depending on the given mode, authentication/integrity device-mapper targets
 // will be set up, and the top-level new block device path will be returned.
@@ -236,30 +201,14 @@
 
 	// Zero out device if authentication is enabled.
 	if mode.authenticated() {
-		blockSize, err := getBlockSize(device)
+		blkdev, err := blockdev.Open(device)
 		if err != nil {
 			return "", err
 		}
-
-		blkdev, err := os.OpenFile(device, unix.O_DIRECT|os.O_WRONLY, 0000)
+		err = blkdev.Zero(0, blkdev.BlockCount()*blkdev.BlockSize())
+		blkdev.Close()
 		if err != nil {
-			return "", fmt.Errorf("failed to open new device for zeroing: %w", err)
-		}
-
-		// Use a multiple of the block size to make the initial zeroing faster.
-		zeroedBuf := make([]byte, blockSize*256)
-		for {
-			_, err := blkdev.Write(zeroedBuf)
-			if errors.Is(err, syscall.ENOSPC) {
-				break
-			}
-			if err != nil {
-				blkdev.Close()
-				return "", fmt.Errorf("failed to zero-initalize new device: %w", err)
-			}
-		}
-		if err := blkdev.Close(); err != nil {
-			return "", fmt.Errorf("failed to close initialized device: %w", err)
+			return "", fmt.Errorf("failed to zero-initalize new device: %w", err)
 		}
 	}
 
diff --git a/metropolis/node/core/localstorage/crypt/crypt_encryption.go b/metropolis/node/core/localstorage/crypt/crypt_encryption.go
index 4fd6061..c5f246c 100644
--- a/metropolis/node/core/localstorage/crypt/crypt_encryption.go
+++ b/metropolis/node/core/localstorage/crypt/crypt_encryption.go
@@ -7,6 +7,7 @@
 
 	"golang.org/x/sys/unix"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/devicemapper"
 )
 
@@ -19,14 +20,11 @@
 }
 
 func mapEncryption(name, underlying string, encryptionKey []byte, authenticated bool) (string, error) {
-	sizeBytes, err := getSizeBytes(underlying)
+	blkdev, err := blockdev.Open(underlying)
 	if err != nil {
-		return "", fmt.Errorf("getting size of block device failed: %w", err)
+		return "", fmt.Errorf("opening underlying block device failed: %w", err)
 	}
-	blockSize, err := getBlockSize(underlying)
-	if err != nil {
-		return "", fmt.Errorf("getting block size failed: %w", err)
-	}
+	defer blkdev.Close()
 
 	optParams := []string{
 		"no_read_workqueue", "no_write_workqueue",
@@ -49,7 +47,7 @@
 
 	cryptDev, err := devicemapper.CreateActiveDevice(encryptionDMName(name), false, []devicemapper.Target{
 		{
-			Length:     sizeBytes / uint64(blockSize),
+			Length:     uint64(blkdev.BlockCount()),
 			Type:       "crypt",
 			Parameters: params,
 		},
diff --git a/metropolis/node/core/localstorage/crypt/crypt_integrity.go b/metropolis/node/core/localstorage/crypt/crypt_integrity.go
index 4130aef..7276a3e 100644
--- a/metropolis/node/core/localstorage/crypt/crypt_integrity.go
+++ b/metropolis/node/core/localstorage/crypt/crypt_integrity.go
@@ -7,6 +7,7 @@
 
 	"golang.org/x/sys/unix"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/devicemapper"
 )
 
@@ -25,20 +26,19 @@
 // This is described in further detail in
 // https://docs.kernel.org/admin-guide/device-mapper/dm-integrity.html.
 func readIntegrityDataSectors(path string) (uint64, error) {
-	integrityPartition, err := os.Open(path)
+	integrityPartition, err := blockdev.Open(path)
 	if err != nil {
 		return 0, err
 	}
 	defer integrityPartition.Close()
+
+	firstBlock := make([]byte, integrityPartition.BlockSize())
+	if _, err = integrityPartition.ReadAt(firstBlock, 0); err != nil {
+		return 0, err
+	}
 	// Based on structure defined in
 	//   https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/drivers/md/dm-integrity.c#n59
-	if _, err := integrityPartition.Seek(16, 0); err != nil {
-		return 0, err
-	}
-	var providedDataSectors uint64
-	if err := binary.Read(integrityPartition, binary.LittleEndian, &providedDataSectors); err != nil {
-		return 0, err
-	}
+	providedDataSectors := binary.LittleEndian.Uint64(firstBlock[16:24])
 
 	// Let's perform some simple checks on the read value to make sure the returned
 	// data isn't corrupted or has been tampered with.
@@ -47,17 +47,8 @@
 		return 0, fmt.Errorf("invalid data sector count of zero")
 	}
 
-	underlyingSizeBytes, err := getSizeBytes(path)
-	if err != nil {
-		return 0, fmt.Errorf("getting underlying block device size failed: %w", err)
-	}
-	underlyingBlockSize, err := getBlockSize(path)
-	if err != nil {
-		return 0, fmt.Errorf("getting underlying block device block size failed: %w", err)
-	}
-	underlyingSectors := underlyingSizeBytes / uint64(underlyingBlockSize)
-	if providedDataSectors > underlyingSectors {
-		return 0, fmt.Errorf("device claims %d data sectors but underlying device only has %d", providedDataSectors, underlyingSectors)
+	if providedDataSectors > uint64(integrityPartition.BlockCount()) {
+		return 0, fmt.Errorf("device claims %d data sectors but underlying device only has %d", providedDataSectors, integrityPartition.BlockCount())
 	}
 	return providedDataSectors, nil
 }
diff --git a/metropolis/node/core/localstorage/directory_root.go b/metropolis/node/core/localstorage/directory_root.go
index 44af268..c385f70 100644
--- a/metropolis/node/core/localstorage/directory_root.go
+++ b/metropolis/node/core/localstorage/directory_root.go
@@ -62,6 +62,7 @@
 		r.Ephemeral.Consensus,
 		r.Ephemeral.Containerd, r.Ephemeral.Containerd.Tmp, r.Ephemeral.Containerd.RunSC, r.Ephemeral.Containerd.IPAM,
 		r.Ephemeral.FlexvolumePlugins,
+		r.ESP.Metropolis,
 	} {
 		err := d.MkdirAll(0700)
 		if err != nil {
diff --git a/metropolis/pkg/blkio/BUILD.bazel b/metropolis/pkg/blkio/BUILD.bazel
new file mode 100644
index 0000000..8b071ec
--- /dev/null
+++ b/metropolis/pkg/blkio/BUILD.bazel
@@ -0,0 +1,8 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+    name = "blkio",
+    srcs = ["blkio.go"],
+    importpath = "source.monogon.dev/metropolis/pkg/blkio",
+    visibility = ["//visibility:public"],
+)
diff --git a/metropolis/pkg/blkio/blkio.go b/metropolis/pkg/blkio/blkio.go
new file mode 100644
index 0000000..d0b7174
--- /dev/null
+++ b/metropolis/pkg/blkio/blkio.go
@@ -0,0 +1,83 @@
+package blkio
+
+import (
+	"fmt"
+	"io"
+	"os"
+)
+
+type ReaderWithSize struct {
+	io.Reader
+	size int64
+}
+
+// SizedReader is an io.Reader with a known size
+type SizedReader interface {
+	io.Reader
+	Size() int64
+}
+
+// NewSizedReader returns a SizedReader given a reader and a size.
+// The returned SizedReader is a ReaderWithSize.
+func NewSizedReader(r io.Reader, size int64) SizedReader {
+	return &ReaderWithSize{r, size}
+}
+
+func (r *ReaderWithSize) Size() int64 {
+	return r.size
+}
+
+// LazyFileReader implements a SizedReader which opens a file on first read
+// and closes it again after the reader has reached EOF.
+type LazyFileReader struct {
+	name string
+	size int64
+	f    *os.File
+	done bool
+}
+
+func (r *LazyFileReader) init() error {
+	f, err := os.Open(r.name)
+	if err != nil {
+		return fmt.Errorf("failed to open file for reading: %w", err)
+	}
+	r.f = f
+	return nil
+}
+
+func (r *LazyFileReader) Size() int64 {
+	return r.size
+}
+
+func (r *LazyFileReader) Read(b []byte) (n int, err error) {
+	if r.done {
+		return 0, io.EOF
+	}
+	if r.f == nil {
+		if err = r.init(); err != nil {
+			return
+		}
+	}
+	n, err = r.f.Read(b)
+	if err == io.EOF {
+		r.done = true
+		r.f.Close()
+	}
+	return
+}
+
+func (r *LazyFileReader) Close() {
+	r.done = true
+	r.f.Close()
+}
+
+func NewFileReader(name string) (*LazyFileReader, error) {
+	info, err := os.Stat(name)
+	if err != nil {
+		return nil, fmt.Errorf("failed to stat: %w", err)
+	}
+	return &LazyFileReader{
+		size: info.Size(),
+		name: name,
+	}, nil
+}
diff --git a/metropolis/pkg/gpt/BUILD.bazel b/metropolis/pkg/gpt/BUILD.bazel
index bc844b7..2bb0b4a 100644
--- a/metropolis/pkg/gpt/BUILD.bazel
+++ b/metropolis/pkg/gpt/BUILD.bazel
@@ -10,6 +10,7 @@
     importpath = "source.monogon.dev/metropolis/pkg/gpt",
     visibility = ["//visibility:public"],
     deps = [
+        "//metropolis/pkg/blockdev",
         "//metropolis/pkg/msguid",
         "@com_github_google_uuid//:uuid",
     ],
@@ -25,8 +26,8 @@
     ],
     embed = [":gpt"],
     deps = [
+        "//metropolis/pkg/blockdev",
         "@com_github_google_uuid//:uuid",
-        "@org_golang_x_sys//unix",
     ],
 )
 
diff --git a/metropolis/pkg/gpt/gpt.go b/metropolis/pkg/gpt/gpt.go
index 81c50f7..f5c2a907 100644
--- a/metropolis/pkg/gpt/gpt.go
+++ b/metropolis/pkg/gpt/gpt.go
@@ -10,13 +10,13 @@
 	"errors"
 	"fmt"
 	"hash/crc32"
-	"io"
 	"sort"
 	"strings"
 	"unicode/utf16"
 
 	"github.com/google/uuid"
 
+	"source.monogon.dev/metropolis/pkg/blockdev"
 	"source.monogon.dev/metropolis/pkg/msguid"
 )
 
@@ -58,8 +58,6 @@
 	PartitionTypeEFISystem = uuid.MustParse("C12A7328-F81F-11D2-BA4B-00A0C93EC93B")
 )
 
-var zeroUUID = [16]byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}
-
 // Attribute is a bitfield of attributes set on a partition. Bits 0 to 47 are
 // reserved for UEFI specification use and all current assignments are in the
 // following const block. Bits 48 to 64 are available for per-Type use by
@@ -112,6 +110,8 @@
 	LastBlock uint64
 	// Bitset of attributes of this partition.
 	Attributes Attribute
+
+	*blockdev.Section
 }
 
 // SizeBlocks returns the size of the partition in blocks
@@ -125,7 +125,16 @@
 	if p == nil {
 		return true
 	}
-	return p.Type == zeroUUID
+	return p.Type == uuid.Nil
+}
+
+// New returns an empty table on the given block device.
+// It does not read any existing GPT on the disk (use Read for that), nor does
+// it write anything until Write is called.
+func New(b blockdev.BlockDev) (*Table, error) {
+	return &Table{
+		b: b,
+	}, nil
 }
 
 type Table struct {
@@ -142,17 +151,13 @@
 	// with BIOS booting is not required. Only useful on x86 systems.
 	BootCode []byte
 
-	// BlockSize contains the logical block size of the block device. It must
-	// be a power of two equal to or larger than 512.
-	BlockSize int64
-	// BlockCount contains the number of logical blocks on the block device.
-	// BlockCount times BlockSize is the size of the block device in bytes.
-	BlockCount int64
-
 	// Partitions contains the list of partitions in this table. This is
 	// artificially limited to 128 partitions.
 	Partitions []*Partition
+
+	b blockdev.BlockDev
 }
+
 type addOptions struct {
 	preferEnd        bool
 	keepEmptyEntries bool
@@ -195,23 +200,23 @@
 // By default, AddPartition aligns FirstBlock to 1MiB boundaries, but this can
 // be overridden using WithAlignment.
 func (g *Table) AddPartition(p *Partition, size int64, options ...AddOption) error {
-	if g.BlockSize < 512 {
-		return errors.New("block size is smaller than 512 bytes, this is unsupported")
-	}
+	blockSize := g.b.BlockSize()
 	var opts addOptions
 	// Align to 1MiB or the block size, whichever is bigger
 	opts.alignment = 1 * 1024 * 1024
-	if g.BlockSize > opts.alignment {
-		opts.alignment = g.BlockSize
+	if blockSize > opts.alignment {
+		opts.alignment = blockSize
 	}
 	for _, o := range options {
 		o(&opts)
 	}
-	if opts.alignment%g.BlockSize != 0 {
-		return fmt.Errorf("requested alignment (%d bytes) is not an integer multiple of the block size (%d), unable to align", opts.alignment, g.BlockSize)
+	if opts.alignment%blockSize != 0 {
+		return fmt.Errorf("requested alignment (%d bytes) is not an integer multiple of the block size (%d), unable to align", opts.alignment, blockSize)
 	}
-	// Number of blocks the partition should occupy, rounded up.
-	blocks := (size + g.BlockSize - 1) / g.BlockSize
+	if p.ID == uuid.Nil {
+		p.ID = uuid.New()
+	}
+
 	fs, _, err := g.GetFreeSpaces()
 	if err != nil {
 		return fmt.Errorf("unable to determine free space: %v", err)
@@ -222,13 +227,28 @@
 			fs[i], fs[j] = fs[j], fs[i]
 		}
 	}
+	// Number of blocks the partition should occupy, rounded up.
+	blocks := (size + blockSize - 1) / blockSize
+	if size == -1 {
+		var largestFreeSpace int64
+		for _, freeInt := range fs {
+			intSz := freeInt[1] - freeInt[0]
+			if intSz > largestFreeSpace {
+				largestFreeSpace = intSz
+			}
+		}
+		blocks = largestFreeSpace
+	}
 	var maxFreeBlocks int64
 	for _, freeInt := range fs {
 		start := freeInt[0]
 		end := freeInt[1]
 		freeBlocks := end - start
 		// Align start properly
-		paddingBlocks := (-start) % (opts.alignment / g.BlockSize)
+		alignTo := (opts.alignment / blockSize)
+		// Go doesn't implement the euclidean modulus, thus this construction
+		// is necessary.
+		paddingBlocks := ((alignTo - start) % alignTo) % alignTo
 		freeBlocks -= paddingBlocks
 		start += paddingBlocks
 		if maxFreeBlocks < freeBlocks {
@@ -237,13 +257,13 @@
 		if freeBlocks >= blocks {
 			if !opts.preferEnd {
 				p.FirstBlock = uint64(start)
-				p.LastBlock = uint64(start + blocks)
+				p.LastBlock = uint64(start + blocks - 1)
 			} else {
 				// Realign FirstBlock. This will always succeed as
 				// there is enough space to align to the start.
-				moveLeft := (end - blocks - 1) % (opts.alignment / g.BlockSize)
+				moveLeft := (end - blocks - 1) % (opts.alignment / blockSize)
 				p.FirstBlock = uint64(end - (blocks + 1 + moveLeft))
-				p.LastBlock = uint64(end - (1 + moveLeft))
+				p.LastBlock = uint64(end - (2 + moveLeft))
 			}
 			newPartPos := -1
 			if !opts.keepEmptyEntries {
@@ -259,6 +279,7 @@
 			} else {
 				g.Partitions[newPartPos] = p
 			}
+			p.Section = blockdev.NewSection(g.b, int64(p.FirstBlock), int64(p.LastBlock)+1)
 			return nil
 		}
 	}
@@ -269,15 +290,17 @@
 // FirstUsableBlock returns the first usable (i.e. a partition can start there)
 // block.
 func (g *Table) FirstUsableBlock() int64 {
-	partitionEntryBlocks := (16384 + g.BlockSize - 1) / g.BlockSize
+	blockSize := g.b.BlockSize()
+	partitionEntryBlocks := (16384 + blockSize - 1) / blockSize
 	return 2 + partitionEntryBlocks
 }
 
 // LastUsableBlock returns the last usable (i.e. a partition can end there)
 // block. This block is inclusive.
 func (g *Table) LastUsableBlock() int64 {
-	partitionEntryBlocks := (16384 + g.BlockSize - 1) / g.BlockSize
-	return g.BlockCount - (2 + partitionEntryBlocks)
+	blockSize := g.b.BlockSize()
+	partitionEntryBlocks := (16384 + blockSize - 1) / blockSize
+	return g.b.BlockCount() - (2 + partitionEntryBlocks)
 }
 
 // GetFreeSpaces returns a slice of tuples, each containing a half-closed
@@ -299,6 +322,8 @@
 	// of its cyclomatic complexity and O(n*log n) is tiny for even very big
 	// partition tables.
 
+	blockCount := g.b.BlockCount()
+
 	// startBlocks contains the start blocks (inclusive) of all occupied
 	// intervals.
 	var startBlocks []int64
@@ -312,7 +337,7 @@
 
 	// Reserve the alternate GPT interval (needs +1 for exclusive interval)
 	startBlocks = append(startBlocks, g.LastUsableBlock()+1)
-	endBlocks = append(endBlocks, g.BlockCount)
+	endBlocks = append(endBlocks, blockCount)
 
 	for i, part := range g.Partitions {
 		if part.IsUnused() {
@@ -324,7 +349,7 @@
 		if part.FirstBlock > part.LastBlock {
 			return nil, false, fmt.Errorf("partition %d has a LastBlock smaller than its FirstBlock, its interval is [%d, %d]", i, part.FirstBlock, part.LastBlock)
 		}
-		if part.FirstBlock >= uint64(g.BlockCount) || part.LastBlock >= uint64(g.BlockCount) {
+		if part.FirstBlock >= uint64(blockCount) || part.LastBlock >= uint64(blockCount) {
 			return nil, false, fmt.Errorf("partition %d exceeds the block count of the block device", i)
 		}
 		startBlocks = append(startBlocks, int64(part.FirstBlock))
@@ -398,14 +423,14 @@
 	return 3 + (2 * partitionEntryBlocks)
 }
 
-// Write writes a list of GPT partitions with a protective MBR to the given
-// WriteSeeker. It must have a defined end, i.e. w.Seek(-x, io.SeekEnd) must
-// seek to x bytes before the end of the disk. If gpt.ID or any of the
-// partition IDs are the all-zero UUID, a new random one is generated and
-// written back. If the output is supposed to be reproducible, generate the
-// UUIDs beforehand.
-func Write(w io.WriteSeeker, gpt *Table) error {
-	if gpt.BlockSize < 512 {
+// Write writes the two GPTs, first the alternate, then the primary to the
+// block device. If gpt.ID or any of the partition IDs are the all-zero UUID,
+// new random ones are generated and written back. If the output is supposed
+// to be reproducible, generate the UUIDs beforehand.
+func (gpt *Table) Write() error {
+	blockSize := gpt.b.BlockSize()
+	blockCount := gpt.b.BlockCount()
+	if blockSize < 512 {
 		return errors.New("block size is smaller than 512 bytes, this is unsupported")
 	}
 	// Layout looks as follows:
@@ -414,30 +439,30 @@
 	// Block 2-(16384 bytes): GPT partition entries
 	// Block (16384 bytes)-n: GPT partition entries alternate copy
 	// Block n: GPT Header alternate copy
-	if len(gpt.Partitions) > 128 {
-		return errors.New("Bigger-than default GPTs (>128 partitions) are unimplemented")
+	partitionEntryCount := 128
+	if len(gpt.Partitions) > partitionEntryCount {
+		return errors.New("bigger-than default GPTs (>128 partitions) are unimplemented")
 	}
 
-	partitionEntryBlocks := (16384 + gpt.BlockSize - 1) / gpt.BlockSize
-	if gpt.BlockCount < 3+(2*partitionEntryBlocks) {
+	partitionEntryBlocks := (16384 + blockSize - 1) / blockSize
+	if blockCount < 3+(2*partitionEntryBlocks) {
 		return errors.New("not enough blocks to write GPT")
 	}
 
-	if gpt.ID == zeroUUID {
+	if gpt.ID == uuid.Nil {
 		gpt.ID = uuid.New()
 	}
 
 	partSize := binary.Size(partition{})
-	slotCount := 128
-
 	var partitionEntriesData bytes.Buffer
-	for i := 0; i < slotCount; i++ {
+	for i := 0; i < partitionEntryCount; i++ {
 		if len(gpt.Partitions) <= i || gpt.Partitions[i] == nil {
+			// Write an empty entry
 			partitionEntriesData.Write(make([]byte, partSize))
 			continue
 		}
 		p := gpt.Partitions[i]
-		if p.ID == zeroUUID {
+		if p.ID == uuid.Nil {
 			p.ID = uuid.New()
 		}
 		rawP := partition{
@@ -459,11 +484,11 @@
 		HeaderSize: uint32(binary.Size(&header{})),
 		ID:         msguid.From(gpt.ID),
 
-		PartitionEntryCount: uint32(slotCount),
+		PartitionEntryCount: uint32(partitionEntryCount),
 		PartitionEntrySize:  uint32(partSize),
 
 		FirstUsableBlock: uint64(2 + partitionEntryBlocks),
-		LastUsableBlock:  uint64(gpt.BlockCount - (2 + partitionEntryBlocks)),
+		LastUsableBlock:  uint64(blockCount - (2 + partitionEntryBlocks)),
 	}
 	hdr.PartitionEntriesCRC32 = crc32.ChecksumIEEE(partitionEntriesData.Bytes())
 
@@ -477,38 +502,36 @@
 	// this problem.
 
 	// Alternate header
-	if _, err := w.Seek((gpt.LastUsableBlock()+1)*gpt.BlockSize, io.SeekStart); err != nil {
-		return fmt.Errorf("failed to seek to end of block device: %w", err)
-	}
-	hdr.HeaderBlock = uint64(gpt.BlockCount - 1)
+	hdr.HeaderBlock = uint64(blockCount - 1)
 	hdr.AlternateHeaderBlock = 1
-	hdr.PartitionEntriesStartBlock = uint64(gpt.BlockCount - (1 + partitionEntryBlocks))
+	hdr.PartitionEntriesStartBlock = uint64(blockCount - (1 + partitionEntryBlocks))
 
 	hdrChecksum.Reset()
 	hdr.HeaderCRC32 = 0
 	binary.Write(hdrChecksum, binary.LittleEndian, &hdr)
 	hdr.HeaderCRC32 = hdrChecksum.Sum32()
 
-	if _, err := w.Write(partitionEntriesData.Bytes()); err != nil {
+	for partitionEntriesData.Len()%int(blockSize) != 0 {
+		partitionEntriesData.WriteByte(0x00)
+	}
+	if _, err := gpt.b.WriteAt(partitionEntriesData.Bytes(), int64(hdr.PartitionEntriesStartBlock)*blockSize); err != nil {
 		return fmt.Errorf("failed to write alternate partition entries: %w", err)
 	}
-	if _, err := w.Seek((gpt.BlockCount-1)*gpt.BlockSize, io.SeekStart); err != nil {
-		return fmt.Errorf("failed to seek to end of block device: %w", err)
-	}
 
-	if err := binary.Write(w, binary.LittleEndian, &hdr); err != nil {
-		return fmt.Errorf("failed to write alternate header: %w", err)
+	var hdrRaw bytes.Buffer
+	if err := binary.Write(&hdrRaw, binary.LittleEndian, &hdr); err != nil {
+		return fmt.Errorf("failed to encode alternate header: %w", err)
 	}
-	if _, err := w.Write(make([]byte, gpt.BlockSize-int64(binary.Size(hdr)))); err != nil {
-		return fmt.Errorf("failed to write padding: %v", err)
+	for hdrRaw.Len()%int(blockSize) != 0 {
+		hdrRaw.WriteByte(0x00)
+	}
+	if _, err := gpt.b.WriteAt(hdrRaw.Bytes(), (blockCount-1)*blockSize); err != nil {
+		return fmt.Errorf("failed to write alternate header: %v", err)
 	}
 
 	// Primary header
-	if _, err := w.Seek(0, io.SeekStart); err != nil {
-		return fmt.Errorf("failed to seek to end of block device: %w", err)
-	}
 	hdr.HeaderBlock = 1
-	hdr.AlternateHeaderBlock = uint64(gpt.BlockCount - 1)
+	hdr.AlternateHeaderBlock = uint64(blockCount - 1)
 	hdr.PartitionEntriesStartBlock = 2
 
 	hdrChecksum.Reset()
@@ -516,33 +539,39 @@
 	binary.Write(hdrChecksum, binary.LittleEndian, &hdr)
 	hdr.HeaderCRC32 = hdrChecksum.Sum32()
 
-	if err := makeProtectiveMBR(w, gpt.BlockCount, gpt.BootCode); err != nil {
-		return fmt.Errorf("failed to write first block: %w", err)
+	hdrRaw.Reset()
+
+	if err := makeProtectiveMBR(&hdrRaw, blockCount, gpt.BootCode); err != nil {
+		return fmt.Errorf("failed creating protective MBR: %w", err)
+	}
+	for hdrRaw.Len()%int(blockSize) != 0 {
+		hdrRaw.WriteByte(0x00)
+	}
+	if err := binary.Write(&hdrRaw, binary.LittleEndian, &hdr); err != nil {
+		panic(err)
+	}
+	for hdrRaw.Len()%int(blockSize) != 0 {
+		hdrRaw.WriteByte(0x00)
+	}
+	hdrRaw.Write(partitionEntriesData.Bytes())
+	for hdrRaw.Len()%int(blockSize) != 0 {
+		hdrRaw.WriteByte(0x00)
 	}
 
-	if err := binary.Write(w, binary.LittleEndian, &hdr); err != nil {
-		return fmt.Errorf("failed to write primary header: %w", err)
-	}
-	if _, err := w.Write(make([]byte, gpt.BlockSize-int64(binary.Size(hdr)))); err != nil {
-		return fmt.Errorf("failed to write padding: %v", err)
-	}
-	if _, err := w.Write(partitionEntriesData.Bytes()); err != nil {
-		return fmt.Errorf("failed to write primary partition entries: %w", err)
+	if _, err := gpt.b.WriteAt(hdrRaw.Bytes(), 0); err != nil {
+		return fmt.Errorf("failed to write primary GPT: %w", err)
 	}
 	return nil
 }
 
-// Read reads a Table from a block device given its block size and count.
-func Read(r io.ReadSeeker, blockSize int64, blockCount int64) (*Table, error) {
-	if Overhead(blockSize) > blockCount {
+// Read reads a Table from a block device.
+func Read(r blockdev.BlockDev) (*Table, error) {
+	if Overhead(r.BlockSize()) > r.BlockCount() {
 		return nil, errors.New("disk cannot contain a GPT as the block count is too small to store one")
 	}
-	if _, err := r.Seek(0, io.SeekStart); err != nil {
-		return nil, fmt.Errorf("failed to seek to block 0: %w", err)
-	}
-	zeroBlock := make([]byte, blockSize)
-	if _, err := io.ReadFull(r, zeroBlock); err != nil {
-		return nil, fmt.Errorf("failed to read first two blocks: %w", err)
+	zeroBlock := make([]byte, r.BlockSize())
+	if _, err := r.ReadAt(zeroBlock, 0); err != nil {
+		return nil, fmt.Errorf("failed to read first block: %w", err)
 	}
 
 	var m mbr
@@ -594,9 +623,9 @@
 		bootCode = bytes.TrimRight(m.BootCode[:], "\x00")
 	}
 	// Read the primary GPT. If it is damaged and/or broken, read the alternate.
-	primaryGPT, err := readSingleGPT(r, blockSize, blockCount, 1)
+	primaryGPT, err := readSingleGPT(r, 1)
 	if err != nil {
-		alternateGPT, err2 := readSingleGPT(r, blockSize, blockCount, blockCount-1)
+		alternateGPT, err2 := readSingleGPT(r, r.BlockCount()-1)
 		if err2 != nil {
 			return nil, fmt.Errorf("failed to read both GPTs: primary GPT (%v), secondary GPT (%v)", err, err2)
 		}
@@ -607,12 +636,9 @@
 	return primaryGPT, nil
 }
 
-func readSingleGPT(r io.ReadSeeker, blockSize int64, blockCount int64, headerBlockPos int64) (*Table, error) {
-	if _, err := r.Seek(blockSize*headerBlockPos, io.SeekStart); err != nil {
-		return nil, fmt.Errorf("failed to seek to block %d: %w", headerBlockPos, err)
-	}
-	hdrBlock := make([]byte, blockSize)
-	if _, err := io.ReadFull(r, hdrBlock); err != nil {
+func readSingleGPT(r blockdev.BlockDev, headerBlockPos int64) (*Table, error) {
+	hdrBlock := make([]byte, r.BlockSize())
+	if _, err := r.ReadAt(hdrBlock, r.BlockSize()*headerBlockPos); err != nil {
 		return nil, fmt.Errorf("failed to read GPT header block: %w", err)
 	}
 	hdrBlockReader := bytes.NewReader(hdrBlock)
@@ -626,7 +652,7 @@
 	if hdr.HeaderSize < uint32(binary.Size(hdr)) {
 		return nil, fmt.Errorf("GPT header size is too small, likely corrupted")
 	}
-	if int64(hdr.HeaderSize) > blockSize {
+	if int64(hdr.HeaderSize) > r.BlockSize() {
 		return nil, fmt.Errorf("GPT header size is bigger than block size, likely corrupted")
 	}
 	// Use reserved bytes to hash, but do not expose them to the user.
@@ -650,7 +676,7 @@
 	if hdr.PartitionEntrySize < uint32(binary.Size(partition{})) {
 		return nil, errors.New("partition entry size too small")
 	}
-	if hdr.PartitionEntriesStartBlock > uint64(blockCount) {
+	if hdr.PartitionEntriesStartBlock > uint64(r.BlockCount()) {
 		return nil, errors.New("partition entry start block is out of range")
 	}
 	// Sanity-check total size of the partition entry area. Otherwise, this is a
@@ -661,10 +687,7 @@
 		return nil, errors.New("partition entry area bigger than 4MiB, refusing to read")
 	}
 	partitionEntryData := make([]byte, hdr.PartitionEntrySize*hdr.PartitionEntryCount)
-	if _, err := r.Seek(blockSize*int64(hdr.PartitionEntriesStartBlock), io.SeekStart); err != nil {
-		return nil, fmt.Errorf("failed to seek to partition entry start block: %w", err)
-	}
-	if _, err := io.ReadFull(r, partitionEntryData); err != nil {
+	if _, err := r.ReadAt(partitionEntryData, r.BlockSize()*int64(hdr.PartitionEntriesStartBlock)); err != nil {
 		return nil, fmt.Errorf("failed to read partition entries: %w", err)
 	}
 	if crc32.ChecksumIEEE(partitionEntryData) != hdr.PartitionEntriesCRC32 {
@@ -672,8 +695,6 @@
 	}
 	var g Table
 	g.ID = msguid.To(hdr.ID)
-	g.BlockSize = blockSize
-	g.BlockCount = blockCount
 	for i := uint32(0); i < hdr.PartitionEntryCount; i++ {
 		entryReader := bytes.NewReader(partitionEntryData[i*hdr.PartitionEntrySize : (i+1)*hdr.PartitionEntrySize])
 		var part partition
@@ -682,7 +703,7 @@
 		}
 		// If the partition type is the all-zero UUID, this slot counts as
 		// unused.
-		if part.Type == zeroUUID {
+		if part.Type == uuid.Nil {
 			g.Partitions = append(g.Partitions, nil)
 			continue
 		}
@@ -705,5 +726,6 @@
 		}
 	}
 	g.Partitions = g.Partitions[:maxValidPartition+1]
+	g.b = r
 	return &g, nil
 }
diff --git a/metropolis/pkg/gpt/gpt_test.go b/metropolis/pkg/gpt/gpt_test.go
index 924d4f7..42b9df5 100644
--- a/metropolis/pkg/gpt/gpt_test.go
+++ b/metropolis/pkg/gpt/gpt_test.go
@@ -8,6 +8,8 @@
 	"testing"
 
 	"github.com/google/uuid"
+
+	"source.monogon.dev/metropolis/pkg/blockdev"
 )
 
 func TestFreeSpaces(t *testing.T) {
@@ -50,11 +52,12 @@
 	// debug.
 	for _, c := range cases {
 		t.Run(c.name, func(t *testing.T) {
-			g := Table{
-				BlockSize:  512,
-				BlockCount: 2048, // 0.5KiB * 2048 = 1MiB
-				Partitions: c.parts,
+			d := blockdev.MustNewMemory(512, 2048) // 1MiB
+			g, err := New(d)
+			if err != nil {
+				panic(err)
 			}
+			g.Partitions = c.parts
 			fs, overlap, err := g.GetFreeSpaces()
 			if err != nil {
 				t.Fatal(err)
@@ -78,35 +81,32 @@
 	if os.Getenv("IN_KTEST") == "true" {
 		t.Skip("In ktest")
 	}
+	d := blockdev.MustNewMemory(512, 2048) // 1 MiB
+
 	g := Table{
-		ID:         uuid.NewSHA1(zeroUUID, []byte("test")),
-		BlockSize:  512,
-		BlockCount: 2048,
-		BootCode:   []byte("just some test code"),
+		ID:       uuid.NewSHA1(uuid.Nil, []byte("test")),
+		BootCode: []byte("just some test code"),
 		Partitions: []*Partition{
 			nil,
 			// This emoji is very complex and exercises UTF16 surrogate encoding
 			// and composing.
-			{Name: "Test 🏃‍♂️", FirstBlock: 10, LastBlock: 19, Type: PartitionTypeEFISystem, ID: uuid.NewSHA1(zeroUUID, []byte("test1")), Attributes: AttrNoBlockIOProto},
+			{Name: "Test 🏃‍♂️", FirstBlock: 10, LastBlock: 19, Type: PartitionTypeEFISystem, ID: uuid.NewSHA1(uuid.Nil, []byte("test1")), Attributes: AttrNoBlockIOProto},
 			nil,
-			{Name: "Test2", FirstBlock: 20, LastBlock: 49, Type: PartitionTypeEFISystem, ID: uuid.NewSHA1(zeroUUID, []byte("test2")), Attributes: 0},
+			{Name: "Test2", FirstBlock: 20, LastBlock: 49, Type: PartitionTypeEFISystem, ID: uuid.NewSHA1(uuid.Nil, []byte("test2")), Attributes: 0},
 		},
+		b: d,
 	}
-	f, err := os.CreateTemp("", "")
-	if err != nil {
-		t.Fatalf("Failed to create temporary file: %v", err)
+	if err := g.Write(); err != nil {
+		t.Fatalf("Error while writing Table: %v", err)
 	}
-	defer os.Remove(f.Name())
-	if err := Write(f, &g); err != nil {
-		t.Fatalf("Error while wrinting Table: %v", err)
-	}
-	f.Seek(0, io.SeekStart)
+
 	originalHash := sha256.New()
-	if _, err := io.Copy(originalHash, f); err != nil {
+	sr1 := io.NewSectionReader(d, 0, d.BlockSize()*d.BlockCount())
+	if _, err := io.CopyBuffer(originalHash, sr1, make([]byte, d.OptimalBlockSize())); err != nil {
 		panic(err)
 	}
 
-	g2, err := Read(f, 512, 2048)
+	g2, err := Read(d)
 	if err != nil {
 		t.Fatalf("Failed to read back GPT: %v", err)
 	}
@@ -114,23 +114,20 @@
 		t.Errorf("Disk UUID changed when reading back: %v", err)
 	}
 	// Destroy primary GPT
-	f.Seek(1*g.BlockSize, io.SeekStart)
-	f.Write(make([]byte, 512))
-	f.Seek(0, io.SeekStart)
-	g3, err := Read(f, 512, 2048)
+	d.Zero(1*d.BlockSize(), 5*d.BlockSize())
+	g3, err := Read(d)
 	if err != nil {
 		t.Fatalf("Failed to read back GPT with primary GPT destroyed: %v", err)
 	}
 	if g3.ID != g.ID {
 		t.Errorf("Disk UUID changed when reading back: %v", err)
 	}
-	f.Seek(0, io.SeekStart)
-	if err := Write(f, g3); err != nil {
+	if err := g3.Write(); err != nil {
 		t.Fatalf("Error while writing back GPT: %v", err)
 	}
-	f.Seek(0, io.SeekStart)
 	rewrittenHash := sha256.New()
-	if _, err := io.Copy(rewrittenHash, f); err != nil {
+	sr2 := io.NewSectionReader(d, 0, d.BlockSize()*d.BlockCount())
+	if _, err := io.CopyBuffer(rewrittenHash, sr2, make([]byte, d.OptimalBlockSize())); err != nil {
 		panic(err)
 	}
 	if !bytes.Equal(originalHash.Sum([]byte{}), rewrittenHash.Sum([]byte{})) {
diff --git a/metropolis/pkg/gpt/linux_test.go b/metropolis/pkg/gpt/linux_test.go
index 19ac519..7f7f6de 100644
--- a/metropolis/pkg/gpt/linux_test.go
+++ b/metropolis/pkg/gpt/linux_test.go
@@ -5,7 +5,8 @@
 	"testing"
 
 	"github.com/google/uuid"
-	"golang.org/x/sys/unix"
+
+	"source.monogon.dev/metropolis/pkg/blockdev"
 )
 
 var testUUID = uuid.MustParse("85c0b60f-caf9-40dd-86fa-f8797e26238d")
@@ -14,15 +15,13 @@
 	if os.Getenv("IN_KTEST") != "true" {
 		t.Skip("Not in ktest")
 	}
-	ram0, err := os.OpenFile("/dev/ram0", os.O_RDWR, 0)
+	ram0, err := blockdev.Open("/dev/ram0")
 	if err != nil {
 		panic(err)
 	}
 	g := Table{
-		ID:         uuid.NewSHA1(testUUID, []byte("test")),
-		BlockSize:  512,
-		BlockCount: 2048,
-		BootCode:   []byte("just some test code"),
+		ID:       uuid.NewSHA1(testUUID, []byte("test")),
+		BootCode: []byte("just some test code"),
 		Partitions: []*Partition{
 			nil,
 			// This emoji is very complex and exercises UTF16 surrogate encoding
@@ -31,13 +30,13 @@
 			nil,
 			{Name: "Test2", FirstBlock: 20, LastBlock: 49, Type: PartitionTypeEFISystem, ID: uuid.NewSHA1(testUUID, []byte("test2")), Attributes: 0},
 		},
+		b: ram0,
 	}
-	if err := Write(ram0, &g); err != nil {
+	if err := g.Write(); err != nil {
 		t.Fatalf("Failed to write GPT: %v", err)
 	}
-
-	if err := unix.IoctlSetInt(int(ram0.Fd()), unix.BLKRRPART, 0); err != nil {
-		t.Fatalf("Failed to reread partition table: %v", err)
+	if err := ram0.RefreshPartitionTable(); err != nil {
+		t.Fatalf("Failed to refresh partition table: %v", err)
 	}
 	if _, err := os.Stat("/sys/block/ram0/ram0p2"); err != nil {
 		t.Errorf("Expected ram0p2 to exist, got %v", err)