core/internal/kubernetes: refactor reconciler, move to subpackage

This makes the reconciler a bit more generic, and thus allows for
writing some basic tests (of the reconciler logic and of the declared
resources).

We also start the cleanup of //core/internal/kubernetes by moving the
reconciler into a separate subpackage. This creates two sketchy
cross-package references that we'll need to fix in the future once we
continue the cleanup and modularization of the Kubernetes package.

Test Plan: the reconciler is now tested with unit tests!

X-Origin-Diff: phab/D552
GitOrigin-RevId: b43643065c8174402922c62e80cd9c87fdce2f13
diff --git a/core/internal/kubernetes/BUILD.bazel b/core/internal/kubernetes/BUILD.bazel
index 36866e7..6778845 100644
--- a/core/internal/kubernetes/BUILD.bazel
+++ b/core/internal/kubernetes/BUILD.bazel
@@ -9,7 +9,6 @@
         "csi.go",
         "kubelet.go",
         "provisioner.go",
-        "reconcile.go",
         "scheduler.go",
         "service.go",
     ],
@@ -19,6 +18,7 @@
         "//core/api/api:go_default_library",
         "//core/internal/common/supervisor:go_default_library",
         "//core/internal/consensus:go_default_library",
+        "//core/internal/kubernetes/reconciler:go_default_library",
         "//core/internal/storage:go_default_library",
         "//core/pkg/fileargs:go_default_library",
         "//core/pkg/fsquota:go_default_library",
@@ -27,8 +27,6 @@
         "@io_bazel_rules_go//proto/wkt:wrappers_go_proto",
         "@io_etcd_go_etcd//clientv3:go_default_library",
         "@io_k8s_api//core/v1:go_default_library",
-        "@io_k8s_api//policy/v1beta1:go_default_library",
-        "@io_k8s_api//rbac/v1:go_default_library",
         "@io_k8s_api//storage/v1:go_default_library",
         "@io_k8s_apimachinery//pkg/api/errors:go_default_library",
         "@io_k8s_apimachinery//pkg/apis/meta/v1:go_default_library",
diff --git a/core/internal/kubernetes/kubelet.go b/core/internal/kubernetes/kubelet.go
index 502b14d..3b0d966 100644
--- a/core/internal/kubernetes/kubelet.go
+++ b/core/internal/kubernetes/kubelet.go
@@ -28,12 +28,13 @@
 	"os"
 	"os/exec"
 
+	"git.monogon.dev/source/nexantic.git/core/internal/common/supervisor"
+	"git.monogon.dev/source/nexantic.git/core/internal/kubernetes/reconciler"
+	"git.monogon.dev/source/nexantic.git/core/pkg/fileargs"
+
 	"go.etcd.io/etcd/clientv3"
 	v1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	kubeletconfig "k8s.io/kubelet/config/v1beta1"
-
-	"git.monogon.dev/source/nexantic.git/core/internal/common/supervisor"
-	"git.monogon.dev/source/nexantic.git/core/pkg/fileargs"
 )
 
 type KubeletSpec struct {
@@ -103,11 +104,12 @@
 					ClientCAFile: "/data/kubernetes/ca.crt",
 				},
 			},
+			// TODO(q3k): move reconciler.False to a generic package, fix the following references.
 			ClusterDomain:                "cluster.local", // cluster.local is hardcoded in the certificate too currently
-			EnableControllerAttachDetach: False(),
+			EnableControllerAttachDetach: reconciler.False(),
 			HairpinMode:                  "none",
-			MakeIPTablesUtilChains:       False(), // We don't have iptables
-			FailSwapOn:                   False(), // Our kernel doesn't have swap enabled which breaks Kubelet's detection
+			MakeIPTablesUtilChains:       reconciler.False(), // We don't have iptables
+			FailSwapOn:                   reconciler.False(), // Our kernel doesn't have swap enabled which breaks Kubelet's detection
 			KubeReserved: map[string]string{
 				"cpu":    "200m",
 				"memory": "300Mi",
diff --git a/core/internal/kubernetes/provisioner.go b/core/internal/kubernetes/provisioner.go
index 9a3d936..c227fdf 100644
--- a/core/internal/kubernetes/provisioner.go
+++ b/core/internal/kubernetes/provisioner.go
@@ -47,6 +47,7 @@
 	"k8s.io/client-go/util/workqueue"
 )
 
+// ONCHANGE(//core/internal/kubernetes/reconciler:resources_csi.go): needs to match csiProvisionerName declared.
 const csiProvisionerName = "com.nexantic.smalltown.vfs"
 
 // csiProvisioner is responsible for the provisioning and deprovisioning of CSI-based container volumes. It runs on all
diff --git a/core/internal/kubernetes/reconcile.go b/core/internal/kubernetes/reconcile.go
deleted file mode 100644
index b000883..0000000
--- a/core/internal/kubernetes/reconcile.go
+++ /dev/null
@@ -1,422 +0,0 @@
-// Copyright 2020 The Monogon Project Authors.
-//
-// SPDX-License-Identifier: Apache-2.0
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-// The reconciler ensures that a base set of K8s resources is always available in the cluster. These are necessary to
-// ensure correct out-of-the-box functionality. All resources containing the smalltown.com/builtin=true label are assumed
-// to be managed by the reconciler.
-// It currently does not revert modifications made by admins, it is  planned to create an admission plugin prohibiting
-// such modifications to resources with the smalltown.com/builtin label to deal with that problem. This would also solve a
-// potential issue where you could delete resources just by adding the smalltown.com/builtin=true label.
-package kubernetes
-
-import (
-	"context"
-	"time"
-
-	storagev1 "k8s.io/api/storage/v1"
-
-	"go.uber.org/zap"
-	corev1 "k8s.io/api/core/v1"
-	"k8s.io/api/policy/v1beta1"
-	rbacv1 "k8s.io/api/rbac/v1"
-	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
-	"k8s.io/client-go/kubernetes"
-
-	"git.monogon.dev/source/nexantic.git/core/internal/common/supervisor"
-)
-
-const builtinRBACPrefix = "smalltown:"
-
-// Sad workaround for all the pointer booleans in K8s specs
-func True() *bool {
-	val := true
-	return &val
-}
-func False() *bool {
-	val := false
-	return &val
-}
-
-func rbac(name string) string {
-	return builtinRBACPrefix + name
-}
-
-// Extended from https://github.com/kubernetes/kubernetes/blob/master/cluster/gce/addons/podsecuritypolicies/unprivileged-addon.yaml
-var builtinPSPs = []*v1beta1.PodSecurityPolicy{
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: "default",
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-			Annotations: map[string]string{
-				"kubernetes.io/description": "This default PSP allows the creation of pods using features that are" +
-					" generally considered safe against any sort of escape.",
-			},
-		},
-		Spec: v1beta1.PodSecurityPolicySpec{
-			AllowPrivilegeEscalation: True(),
-			AllowedCapabilities: []corev1.Capability{ // runc's default list of allowed capabilities
-				"SETPCAP",
-				"MKNOD",
-				"AUDIT_WRITE",
-				"CHOWN",
-				"NET_RAW",
-				"DAC_OVERRIDE",
-				"FOWNER",
-				"FSETID",
-				"KILL",
-				"SETGID",
-				"SETUID",
-				"NET_BIND_SERVICE",
-				"SYS_CHROOT",
-				"SETFCAP",
-			},
-			HostNetwork: false,
-			HostIPC:     false,
-			HostPID:     false,
-			FSGroup: v1beta1.FSGroupStrategyOptions{
-				Rule: v1beta1.FSGroupStrategyRunAsAny,
-			},
-			RunAsUser: v1beta1.RunAsUserStrategyOptions{
-				Rule: v1beta1.RunAsUserStrategyRunAsAny,
-			},
-			SELinux: v1beta1.SELinuxStrategyOptions{
-				Rule: v1beta1.SELinuxStrategyRunAsAny,
-			},
-			SupplementalGroups: v1beta1.SupplementalGroupsStrategyOptions{
-				Rule: v1beta1.SupplementalGroupsStrategyRunAsAny,
-			},
-			Volumes: []v1beta1.FSType{ // Volumes considered safe to use
-				v1beta1.ConfigMap,
-				v1beta1.EmptyDir,
-				v1beta1.Projected,
-				v1beta1.Secret,
-				v1beta1.DownwardAPI,
-				v1beta1.PersistentVolumeClaim,
-			},
-		},
-	},
-}
-
-var builtinClusterRoles = []*rbacv1.ClusterRole{
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: rbac("psp-default"),
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-			Annotations: map[string]string{
-				"kubernetes.io/description": "This role grants access to the \"default\" PSP.",
-			},
-		},
-		Rules: []rbacv1.PolicyRule{
-			{
-				APIGroups:     []string{"policy"},
-				Resources:     []string{"podsecuritypolicies"},
-				ResourceNames: []string{"default"},
-				Verbs:         []string{"use"},
-			},
-		},
-	},
-}
-
-var builtinClusterRoleBindings = []*rbacv1.ClusterRoleBinding{
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: rbac("default-psp-for-sa"),
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-			Annotations: map[string]string{
-				"kubernetes.io/description": "This binding grants every service account access to the \"default\" PSP. " +
-					"Creation of Pods is still restricted by other RBAC roles. Otherwise no pods (unprivileged or not) " +
-					"can be created.",
-			},
-		},
-		RoleRef: rbacv1.RoleRef{
-			APIGroup: rbacv1.GroupName,
-			Kind:     "ClusterRole",
-			Name:     rbac("psp-default"),
-		},
-		Subjects: []rbacv1.Subject{
-			{
-				APIGroup: rbacv1.GroupName,
-				Kind:     "Group",
-				Name:     "system:serviceaccounts",
-			},
-		},
-	},
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: rbac("apiserver-kubelet-client"),
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-			Annotations: map[string]string{
-				"kubernetes.io/description": "This binding grants the apiserver access to the kubelets. This enables " +
-					"lots of built-in functionality like reading logs or forwarding ports via the API.",
-			},
-		},
-		RoleRef: rbacv1.RoleRef{
-			APIGroup: rbacv1.GroupName,
-			Kind:     "ClusterRole",
-			Name:     "system:kubelet-api-admin",
-		},
-		Subjects: []rbacv1.Subject{
-			{
-				APIGroup: rbacv1.GroupName,
-				Kind:     "User",
-				Name:     "smalltown:apiserver-kubelet-client",
-			},
-		},
-	},
-}
-
-var reclaimPolicyDelete = corev1.PersistentVolumeReclaimDelete
-var waitForConsumerBinding = storagev1.VolumeBindingWaitForFirstConsumer
-
-var builtinStorageClasses = []*storagev1.StorageClass{
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: "local",
-			Annotations: map[string]string{
-				"storageclass.kubernetes.io/is-default-class": "true",
-			},
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-		},
-		AllowVolumeExpansion: True(),
-		Provisioner:          csiProvisionerName,
-		ReclaimPolicy:        &reclaimPolicyDelete,
-		VolumeBindingMode:    &waitForConsumerBinding,
-	},
-}
-
-var builtinCSIDrivers = []*storagev1.CSIDriver{
-	{
-		ObjectMeta: metav1.ObjectMeta{
-			Name: csiProvisionerName,
-			Labels: map[string]string{
-				"smalltown.com/builtin": "true",
-			},
-		},
-		Spec: storagev1.CSIDriverSpec{
-			AttachRequired:       False(),
-			PodInfoOnMount:       False(),
-			VolumeLifecycleModes: []storagev1.VolumeLifecycleMode{storagev1.VolumeLifecyclePersistent},
-		},
-	},
-}
-
-type reconciler func(context.Context, kubernetes.Interface) error
-
-func runReconciler(clientSet kubernetes.Interface) supervisor.Runnable {
-	return func(ctx context.Context) error {
-		log := supervisor.Logger(ctx)
-		reconcilers := map[string]reconciler{
-			"psps":                reconcilePSPs,
-			"clusterroles":        reconcileClusterRoles,
-			"clusterrolebindings": reconcileClusterRoleBindings,
-			"storageclasses":      reconcileSCs,
-			"csidrivers":          reconcileCSIDrivers,
-		}
-		t := time.NewTicker(10 * time.Second)
-		reconcile := func() {
-			for name, reconciler := range reconcilers {
-				if err := reconciler(ctx, clientSet); err != nil {
-					log.Warn("Failed to reconcile built-in resources", zap.String("kind", name), zap.Error(err))
-				}
-			}
-		}
-		supervisor.Signal(ctx, supervisor.SignalHealthy)
-		reconcile()
-		for {
-			select {
-			case <-t.C:
-				reconcile()
-			case <-ctx.Done():
-				return nil
-			}
-		}
-	}
-}
-
-func reconcilePSPs(ctx context.Context, clientSet kubernetes.Interface) error {
-	pspClient := clientSet.PolicyV1beta1().PodSecurityPolicies()
-	availablePSPs, err := pspClient.List(ctx, metav1.ListOptions{
-		LabelSelector: "smalltown.com/builtin=true",
-	})
-	if err != nil {
-		return err
-	}
-	availablePSPMap := make(map[string]struct{})
-	for _, psp := range availablePSPs.Items {
-		availablePSPMap[psp.Name] = struct{}{}
-	}
-	expectedPSPMap := make(map[string]*v1beta1.PodSecurityPolicy)
-	for _, psp := range builtinPSPs {
-		expectedPSPMap[psp.Name] = psp
-	}
-	for pspName, psp := range expectedPSPMap {
-		if _, ok := availablePSPMap[pspName]; !ok {
-			if _, err := pspClient.Create(ctx, psp, metav1.CreateOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	for pspName, _ := range availablePSPMap {
-		if _, ok := expectedPSPMap[pspName]; !ok {
-			if err := pspClient.Delete(ctx, pspName, metav1.DeleteOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	return nil
-}
-
-func reconcileClusterRoles(ctx context.Context, clientSet kubernetes.Interface) error {
-	crClient := clientSet.RbacV1().ClusterRoles()
-	availableCRs, err := crClient.List(ctx, metav1.ListOptions{
-		LabelSelector: "smalltown.com/builtin=true",
-	})
-	if err != nil {
-		return err
-	}
-	availableCRMap := make(map[string]struct{})
-	for _, cr := range availableCRs.Items {
-		availableCRMap[cr.Name] = struct{}{}
-	}
-	expectedCRMap := make(map[string]*rbacv1.ClusterRole)
-	for _, cr := range builtinClusterRoles {
-		expectedCRMap[cr.Name] = cr
-	}
-	for crName, psp := range expectedCRMap {
-		if _, ok := availableCRMap[crName]; !ok {
-			if _, err := crClient.Create(ctx, psp, metav1.CreateOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	for crName, _ := range availableCRMap {
-		if _, ok := expectedCRMap[crName]; !ok {
-			if err := crClient.Delete(ctx, crName, metav1.DeleteOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	return nil
-}
-
-func reconcileClusterRoleBindings(ctx context.Context, clientset kubernetes.Interface) error {
-	crbClient := clientset.RbacV1().ClusterRoleBindings()
-	availableCRBs, err := crbClient.List(ctx, metav1.ListOptions{
-		LabelSelector: "smalltown.com/builtin=true",
-	})
-	if err != nil {
-		return err
-	}
-	availableCRBMap := make(map[string]struct{})
-	for _, crb := range availableCRBs.Items {
-		availableCRBMap[crb.Name] = struct{}{}
-	}
-	expectedCRBMap := make(map[string]*rbacv1.ClusterRoleBinding)
-	for _, crb := range builtinClusterRoleBindings {
-		expectedCRBMap[crb.Name] = crb
-	}
-	for crbName, psp := range expectedCRBMap {
-		if _, ok := availableCRBMap[crbName]; !ok {
-			if _, err := crbClient.Create(ctx, psp, metav1.CreateOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	for crbName, _ := range availableCRBMap {
-		if _, ok := expectedCRBMap[crbName]; !ok {
-			if err := crbClient.Delete(ctx, crbName, metav1.DeleteOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	return nil
-}
-
-func reconcileSCs(ctx context.Context, clientSet kubernetes.Interface) error {
-	scsClient := clientSet.StorageV1().StorageClasses()
-	availableSCs, err := scsClient.List(ctx, metav1.ListOptions{
-		LabelSelector: "smalltown.com/builtin=true",
-	})
-	if err != nil {
-		return err
-	}
-	availableSCMap := make(map[string]struct{})
-	for _, sc := range availableSCs.Items {
-		availableSCMap[sc.Name] = struct{}{}
-	}
-	expectedSCMap := make(map[string]*storagev1.StorageClass)
-	for _, sc := range builtinStorageClasses {
-		expectedSCMap[sc.Name] = sc
-	}
-	for scName, sc := range expectedSCMap {
-		if _, ok := availableSCMap[scName]; !ok {
-			if _, err := scsClient.Create(ctx, sc, metav1.CreateOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	for pspName, _ := range availableSCMap {
-		if _, ok := expectedSCMap[pspName]; !ok {
-			if err := scsClient.Delete(ctx, pspName, metav1.DeleteOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	return nil
-}
-
-func reconcileCSIDrivers(ctx context.Context, clientSet kubernetes.Interface) error {
-	drvClient := clientSet.StorageV1().CSIDrivers()
-	availableDrvs, err := drvClient.List(ctx, metav1.ListOptions{
-		LabelSelector: "smalltown.com/builtin=true",
-	})
-	if err != nil {
-		return err
-	}
-	availableDrvMap := make(map[string]struct{})
-	for _, drv := range availableDrvs.Items {
-		availableDrvMap[drv.Name] = struct{}{}
-	}
-	expectedDrvMap := make(map[string]*storagev1.CSIDriver)
-	for _, drv := range builtinCSIDrivers {
-		expectedDrvMap[drv.Name] = drv
-	}
-	for drvName, drv := range expectedDrvMap {
-		if _, ok := availableDrvMap[drvName]; !ok {
-			if _, err := drvClient.Create(ctx, drv, metav1.CreateOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	for pspName, _ := range availableDrvMap {
-		if _, ok := expectedDrvMap[pspName]; !ok {
-			if err := drvClient.Delete(ctx, pspName, metav1.DeleteOptions{}); err != nil {
-				return err
-			}
-		}
-	}
-	return nil
-}
diff --git a/core/internal/kubernetes/reconciler/BUILD.bazel b/core/internal/kubernetes/reconciler/BUILD.bazel
new file mode 100644
index 0000000..1e82abe
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/BUILD.bazel
@@ -0,0 +1,36 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+    name = "go_default_library",
+    srcs = [
+        "reconciler.go",
+        "resources_csi.go",
+        "resources_podsecuritypolicy.go",
+        "resources_rbac.go",
+        "resources_storageclass.go",
+    ],
+    importpath = "git.monogon.dev/source/nexantic.git/core/internal/kubernetes/reconciler",
+    visibility = ["//core:__subpackages__"],
+    deps = [
+        "//core/internal/common/supervisor:go_default_library",
+        "@io_k8s_api//core/v1:go_default_library",
+        "@io_k8s_api//policy/v1beta1:go_default_library",
+        "@io_k8s_api//rbac/v1:go_default_library",
+        "@io_k8s_api//storage/v1:go_default_library",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:go_default_library",
+        "@io_k8s_client_go//kubernetes:go_default_library",
+        "@org_uber_go_zap//:go_default_library",
+    ],
+)
+
+go_test(
+    name = "go_default_test",
+    srcs = ["reconciler_test.go"],
+    embed = [":go_default_library"],
+    deps = [
+        "@io_k8s_api//policy/v1beta1:go_default_library",
+        "@io_k8s_api//rbac/v1:go_default_library",
+        "@io_k8s_api//storage/v1:go_default_library",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:go_default_library",
+    ],
+)
diff --git a/core/internal/kubernetes/reconciler/reconciler.go b/core/internal/kubernetes/reconciler/reconciler.go
new file mode 100644
index 0000000..a8b6272
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/reconciler.go
@@ -0,0 +1,164 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// The reconciler ensures that a base set of K8s resources is always available in the cluster. These are necessary to
+// ensure correct out-of-the-box functionality. All resources containing the smalltown.com/builtin=true label are assumed
+// to be managed by the reconciler.
+// It currently does not revert modifications made by admins, it is  planned to create an admission plugin prohibiting
+// such modifications to resources with the smalltown.com/builtin label to deal with that problem. This would also solve a
+// potential issue where you could delete resources just by adding the smalltown.com/builtin=true label.
+package reconciler
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"git.monogon.dev/source/nexantic.git/core/internal/common/supervisor"
+
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+
+	"go.uber.org/zap"
+	"k8s.io/client-go/kubernetes"
+)
+
+// Sad workaround for all the pointer booleans in K8s specs
+func True() *bool {
+	val := true
+	return &val
+}
+func False() *bool {
+	val := false
+	return &val
+}
+
+const (
+	// BuiltinLabelKey is used as a k8s label to mark built-in objects (ie., managed by the reconciler)
+	BuiltinLabelKey = "smalltown.com/builtin"
+	// BuiltinLabelValue is used as a k8s label value, under the BuiltinLabelKey key.
+	BuiltinLabelValue = "true"
+	// BuiltinRBACPrefix is used to prefix all built-in objects that are part of the rbac/v1 API (eg.
+	// {Cluster,}Role{Binding,} objects). This corresponds to the colon-separated 'namespaces' notation used by
+	// Kubernetes system (system:) objects.
+	BuiltinRBACPrefix = "smalltown:"
+)
+
+// builtinLabels makes a kubernetes-compatible label dictionary (key->value) that is used to mark objects that are
+// built-in into Smalltown (ie., managed by the reconciler). These are then subsequently retrieved by listBuiltins.
+// The extra argument specifies what other labels are to be merged into the the labels dictionary, for convenience. If
+// nil or empty, no extra labels will be applied.
+func builtinLabels(extra map[string]string) map[string]string {
+	l := map[string]string{
+		BuiltinLabelKey: BuiltinLabelValue,
+	}
+	if extra != nil {
+		for k, v := range extra {
+			l[k] = v
+		}
+	}
+	return l
+}
+
+// listBuiltins returns a k8s client ListOptions structure that allows to retrieve all objects that are built-in into
+// Smalltown currently present in the API server (ie., ones that are to be managed by the reconciler). These are created
+// by applying builtinLabels to their metadata labels.
+var listBuiltins = meta.ListOptions{
+	LabelSelector: fmt.Sprintf("%s=%s", BuiltinLabelKey, BuiltinLabelValue),
+}
+
+// builtinRBACName returns a name that is compatible with colon-delimited 'namespaced' objects, a la system:*.
+// These names are to be used by all builtins created as part of the rbac/v1 Kubernetes API.
+func builtinRBACName(name string) string {
+	return BuiltinRBACPrefix + name
+}
+
+// resource is a type of resource to be managed by the reconciler. All builti-ins/reconciled objects must implement
+// this interface to be managed correctly by the reconciler.
+type resource interface {
+	// List returns a list of names of objects current present on the target (ie. k8s API server).
+	List(ctx context.Context) ([]string, error)
+	// Create creates an object on the target. The el interface{} argument is the black box object returned by the
+	// Expected() call.
+	Create(ctx context.Context, el interface{}) error
+	// Delete delete an object, by name, from the target.
+	Delete(ctx context.Context, name string) error
+	// Expected returns a map of all objects expected to be present on the target. The keys are names (which must
+	// correspond to the names returned by List() and used by Delete(), and the values are blackboxes that will then
+	// be passed to the Create() call if their corresponding key (name) does not exist on the target.
+	Expected() map[string]interface{}
+}
+
+func allResources(clientSet kubernetes.Interface) map[string]resource {
+	return map[string]resource{
+		"psps":                resourcePodSecurityPolicies{clientSet},
+		"clusterroles":        resourceClusterRoles{clientSet},
+		"clusterrolebindings": resourceClusterRoleBindings{clientSet},
+		"storageclasses":      resourceStorageClasses{clientSet},
+		"csidrivers":          resourceCSIDrivers{clientSet},
+	}
+}
+
+func Run(clientSet kubernetes.Interface) supervisor.Runnable {
+	return func(ctx context.Context) error {
+		log := supervisor.Logger(ctx)
+		resources := allResources(clientSet)
+		t := time.NewTicker(10 * time.Second)
+		reconcileAll := func() {
+			for name, resource := range resources {
+				if err := reconcile(ctx, resource); err != nil {
+					log.Warn("Failed to reconcile built-in resources", zap.String("kind", name), zap.Error(err))
+				}
+			}
+		}
+		supervisor.Signal(ctx, supervisor.SignalHealthy)
+		reconcileAll()
+		for {
+			select {
+			case <-t.C:
+				reconcileAll()
+			case <-ctx.Done():
+				return nil
+			}
+		}
+	}
+}
+
+func reconcile(ctx context.Context, r resource) error {
+	present, err := r.List(ctx)
+	if err != nil {
+		return err
+	}
+	presentSet := make(map[string]bool)
+	for _, el := range present {
+		presentSet[el] = true
+	}
+	expectedMap := r.Expected()
+	for name, el := range expectedMap {
+		if !presentSet[name] {
+			if err := r.Create(ctx, el); err != nil {
+				return err
+			}
+		}
+	}
+	for name, _ := range presentSet {
+		if _, ok := expectedMap[name]; !ok {
+			if err := r.Delete(ctx, name); err != nil {
+				return err
+			}
+		}
+	}
+	return nil
+}
diff --git a/core/internal/kubernetes/reconciler/reconciler_test.go b/core/internal/kubernetes/reconciler/reconciler_test.go
new file mode 100644
index 0000000..5d78d82
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/reconciler_test.go
@@ -0,0 +1,181 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package reconciler
+
+import (
+	"context"
+	"fmt"
+	"testing"
+
+	policy "k8s.io/api/policy/v1beta1"
+	rbac "k8s.io/api/rbac/v1"
+	storage "k8s.io/api/storage/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+)
+
+// kubernetesMeta unwraps an interface{} that might contain a Kubernetes resource of type that is managed by the
+// reconciler. Any time a new Kubernetes type is managed by the reconciler, the following switch should be extended
+// to cover that type.
+func kubernetesMeta(v interface{}) *meta.ObjectMeta {
+	switch v2 := v.(type) {
+	case *rbac.ClusterRole:
+		return &v2.ObjectMeta
+	case *rbac.ClusterRoleBinding:
+		return &v2.ObjectMeta
+	case *storage.CSIDriver:
+		return &v2.ObjectMeta
+	case *storage.StorageClass:
+		return &v2.ObjectMeta
+	case *policy.PodSecurityPolicy:
+		return &v2.ObjectMeta
+	}
+	return nil
+}
+
+// TestExpectedNamedCorrectly ensures that all the Expected objects of all resource types have a correspondence between
+// their returned key and inner name. This contract must be met in order for the reconciler to not create runaway
+// resources. This assumes all managed resources are Kubernetes resources.
+func TestExpectedNamedCorrectly(t *testing.T) {
+	for reconciler, r := range allResources(nil) {
+		for outer, v := range r.Expected() {
+			meta := kubernetesMeta(v)
+			if meta == nil {
+				t.Errorf("reconciler %q, object %q: could not decode kubernetes metadata", reconciler, outer)
+				continue
+			}
+			if inner := meta.Name; outer != inner {
+				t.Errorf("reconciler %q, object %q: inner name mismatch (%q)", reconciler, outer, inner)
+				continue
+			}
+		}
+	}
+}
+
+// TestExpectedLabeledCorrectly ensures that all the Expected objects of all resource types have a Kubernetes metadata
+// label that signifies it's a builtin object, to be retrieved afterwards. This contract must be met in order for the
+// reconciler to not keep overwriting objects (and possibly failing), when a newly created object is not then
+// retrievable using a selector corresponding to this label. This assumes all managed resources are Kubernetes objects.
+func TestExpectedLabeledCorrectly(t *testing.T) {
+	for reconciler, r := range allResources(nil) {
+		for outer, v := range r.Expected() {
+			meta := kubernetesMeta(v)
+			if meta == nil {
+				t.Errorf("reconciler %q, object %q: could not decode kubernetes metadata", reconciler, outer)
+				continue
+			}
+			if data := meta.Labels[BuiltinLabelKey]; data != BuiltinLabelValue {
+				t.Errorf("reconciler %q, object %q: %q=%q, wanted =%q", reconciler, outer, BuiltinLabelKey, data, BuiltinLabelValue)
+				continue
+			}
+		}
+	}
+}
+
+// testResource is a resource type used for testing. The inner type is a string that is equal to its name (key).
+// It simulates a target (ie. k8s apiserver mock) that always acts nominally (all resources are created, deleted as
+// requested, and the state is consistent with requests).
+type testResource struct {
+	// current is the simulated state of resources in the target.
+	current map[string]string
+	// expected is what this type will report as the Expected() resources.
+	expected map[string]string
+}
+
+func (r *testResource) List(ctx context.Context) ([]string, error) {
+	var keys []string
+	for k, _ := range r.current {
+		keys = append(keys, k)
+	}
+	return keys, nil
+}
+
+func (r *testResource) Create(ctx context.Context, el interface{}) error {
+	r.current[el.(string)] = el.(string)
+	return nil
+}
+
+func (r *testResource) Delete(ctx context.Context, name string) error {
+	delete(r.current, name)
+	return nil
+}
+
+func (r *testResource) Expected() map[string]interface{} {
+	exp := make(map[string]interface{})
+	for k, v := range r.expected {
+		exp[k] = v
+	}
+	return exp
+}
+
+// newTestResource creates a test resource with a list of expected resource strings.
+func newTestResource(want ...string) *testResource {
+	expected := make(map[string]string)
+	for _, w := range want {
+		expected[w] = w
+	}
+	return &testResource{
+		current:  make(map[string]string),
+		expected: expected,
+	}
+}
+
+// currentDiff returns a human-readable string showing the different between the current state and the given resource
+// strings. If no difference is present, the returned string is empty.
+func (r *testResource) currentDiff(want ...string) string {
+	expected := make(map[string]string)
+	for _, w := range want {
+		if _, ok := r.current[w]; !ok {
+			return fmt.Sprintf("%q missing in current", w)
+		}
+		expected[w] = w
+	}
+	for _, g := range r.current {
+		if _, ok := expected[g]; !ok {
+			return fmt.Sprintf("%q spurious in current", g)
+		}
+	}
+	return ""
+}
+
+// TestBasicReconciliation ensures that the reconcile function does manipulate a target state based on a set of
+// expected resources.
+func TestBasicReconciliation(t *testing.T) {
+	ctx := context.Background()
+	r := newTestResource("foo", "bar", "baz")
+
+	// nothing should have happened yet (testing the test)
+	if diff := r.currentDiff(); diff != "" {
+		t.Fatalf("wrong state after creation: %s", diff)
+	}
+
+	if err := reconcile(ctx, r); err != nil {
+		t.Fatalf("reconcile: %v", err)
+	}
+	// everything requested should have been created
+	if diff := r.currentDiff("foo", "bar", "baz"); diff != "" {
+		t.Fatalf("wrong state after reconciliation: %s", diff)
+	}
+
+	delete(r.expected, "foo")
+	if err := reconcile(ctx, r); err != nil {
+		t.Fatalf("reconcile: %v", err)
+	}
+	// foo should not be missing
+	if diff := r.currentDiff("bar", "baz"); diff != "" {
+		t.Fatalf("wrong state after deleting foo: %s", diff)
+	}
+}
diff --git a/core/internal/kubernetes/reconciler/resources_csi.go b/core/internal/kubernetes/reconciler/resources_csi.go
new file mode 100644
index 0000000..73a92d2
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/resources_csi.go
@@ -0,0 +1,71 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package reconciler
+
+import (
+	"context"
+
+	storage "k8s.io/api/storage/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/client-go/kubernetes"
+)
+
+// TODO(q3k): this is duplicated with //core/internal/kubernetes:provisioner.go; integrate this once provisioner.go
+// gets moved into a subpackage.
+// ONCHANGE(//core/internal/kubernetes:provisioner.go): needs to match csiProvisionerName declared.
+const csiProvisionerName = "com.nexantic.smalltown.vfs"
+
+type resourceCSIDrivers struct {
+	kubernetes.Interface
+}
+
+func (r resourceCSIDrivers) List(ctx context.Context) ([]string, error) {
+	res, err := r.StorageV1().CSIDrivers().List(ctx, listBuiltins)
+	if err != nil {
+		return nil, err
+	}
+	objs := make([]string, len(res.Items))
+	for i, el := range res.Items {
+		objs[i] = el.ObjectMeta.Name
+	}
+	return objs, nil
+}
+
+func (r resourceCSIDrivers) Create(ctx context.Context, el interface{}) error {
+	_, err := r.StorageV1().CSIDrivers().Create(ctx, el.(*storage.CSIDriver), meta.CreateOptions{})
+	return err
+}
+
+func (r resourceCSIDrivers) Delete(ctx context.Context, name string) error {
+	return r.StorageV1().CSIDrivers().Delete(ctx, name, meta.DeleteOptions{})
+}
+
+func (r resourceCSIDrivers) Expected() map[string]interface{} {
+	return map[string]interface{}{
+		csiProvisionerName: &storage.CSIDriver{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   csiProvisionerName,
+				Labels: builtinLabels(nil),
+			},
+			Spec: storage.CSIDriverSpec{
+				AttachRequired:       False(),
+				PodInfoOnMount:       False(),
+				VolumeLifecycleModes: []storage.VolumeLifecycleMode{storage.VolumeLifecyclePersistent},
+			},
+		},
+	}
+}
diff --git a/core/internal/kubernetes/reconciler/resources_podsecuritypolicy.go b/core/internal/kubernetes/reconciler/resources_podsecuritypolicy.go
new file mode 100644
index 0000000..507089f
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/resources_podsecuritypolicy.go
@@ -0,0 +1,108 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package reconciler
+
+import (
+	"context"
+
+	core "k8s.io/api/core/v1"
+	policy "k8s.io/api/policy/v1beta1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/client-go/kubernetes"
+)
+
+type resourcePodSecurityPolicies struct {
+	kubernetes.Interface
+}
+
+func (r resourcePodSecurityPolicies) List(ctx context.Context) ([]string, error) {
+	res, err := r.PolicyV1beta1().PodSecurityPolicies().List(ctx, listBuiltins)
+	if err != nil {
+		return nil, err
+	}
+	objs := make([]string, len(res.Items))
+	for i, el := range res.Items {
+		objs[i] = el.ObjectMeta.Name
+	}
+	return objs, nil
+}
+
+func (r resourcePodSecurityPolicies) Create(ctx context.Context, el interface{}) error {
+	_, err := r.PolicyV1beta1().PodSecurityPolicies().Create(ctx, el.(*policy.PodSecurityPolicy), meta.CreateOptions{})
+	return err
+}
+
+func (r resourcePodSecurityPolicies) Delete(ctx context.Context, name string) error {
+	return r.PolicyV1beta1().PodSecurityPolicies().Delete(ctx, name, meta.DeleteOptions{})
+}
+
+func (r resourcePodSecurityPolicies) Expected() map[string]interface{} {
+	return map[string]interface{}{
+		"default": &policy.PodSecurityPolicy{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   "default",
+				Labels: builtinLabels(nil),
+				Annotations: map[string]string{
+					"kubernetes.io/description": "This default PSP allows the creation of pods using features that are" +
+						" generally considered safe against any sort of escape.",
+				},
+			},
+			Spec: policy.PodSecurityPolicySpec{
+				AllowPrivilegeEscalation: True(),
+				AllowedCapabilities: []core.Capability{ // runc's default list of allowed capabilities
+					"SETPCAP",
+					"MKNOD",
+					"AUDIT_WRITE",
+					"CHOWN",
+					"NET_RAW",
+					"DAC_OVERRIDE",
+					"FOWNER",
+					"FSETID",
+					"KILL",
+					"SETGID",
+					"SETUID",
+					"NET_BIND_SERVICE",
+					"SYS_CHROOT",
+					"SETFCAP",
+				},
+				HostNetwork: false,
+				HostIPC:     false,
+				HostPID:     false,
+				FSGroup: policy.FSGroupStrategyOptions{
+					Rule: policy.FSGroupStrategyRunAsAny,
+				},
+				RunAsUser: policy.RunAsUserStrategyOptions{
+					Rule: policy.RunAsUserStrategyRunAsAny,
+				},
+				SELinux: policy.SELinuxStrategyOptions{
+					Rule: policy.SELinuxStrategyRunAsAny,
+				},
+				SupplementalGroups: policy.SupplementalGroupsStrategyOptions{
+					Rule: policy.SupplementalGroupsStrategyRunAsAny,
+				},
+				Volumes: []policy.FSType{ // Volumes considered safe to use
+					policy.ConfigMap,
+					policy.EmptyDir,
+					policy.Projected,
+					policy.Secret,
+					policy.DownwardAPI,
+					policy.PersistentVolumeClaim,
+				},
+			},
+		},
+	}
+}
diff --git a/core/internal/kubernetes/reconciler/resources_rbac.go b/core/internal/kubernetes/reconciler/resources_rbac.go
new file mode 100644
index 0000000..40ca879
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/resources_rbac.go
@@ -0,0 +1,154 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package reconciler
+
+import (
+	"context"
+
+	rbac "k8s.io/api/rbac/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/client-go/kubernetes"
+)
+
+var (
+	clusterRolePSPDefault                    = builtinRBACName("psp-default")
+	clusterRoleBindingDefaultPSP             = builtinRBACName("default-psp-for-sa")
+	clusterRoleBindingAPIServerKubeletClient = builtinRBACName("apiserver-kubelet-client")
+)
+
+type resourceClusterRoles struct {
+	kubernetes.Interface
+}
+
+func (r resourceClusterRoles) List(ctx context.Context) ([]string, error) {
+	res, err := r.RbacV1().ClusterRoles().List(ctx, listBuiltins)
+	if err != nil {
+		return nil, err
+	}
+	objs := make([]string, len(res.Items))
+	for i, el := range res.Items {
+		objs[i] = el.ObjectMeta.Name
+	}
+	return objs, nil
+}
+
+func (r resourceClusterRoles) Create(ctx context.Context, el interface{}) error {
+	_, err := r.RbacV1().ClusterRoles().Create(ctx, el.(*rbac.ClusterRole), meta.CreateOptions{})
+	return err
+}
+
+func (r resourceClusterRoles) Delete(ctx context.Context, name string) error {
+	return r.RbacV1().ClusterRoles().Delete(ctx, name, meta.DeleteOptions{})
+}
+
+func (r resourceClusterRoles) Expected() map[string]interface{} {
+	return map[string]interface{}{
+		clusterRolePSPDefault: &rbac.ClusterRole{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   clusterRolePSPDefault,
+				Labels: builtinLabels(nil),
+				Annotations: map[string]string{
+					"kubernetes.io/description": "This role grants access to the \"default\" PSP.",
+				},
+			},
+			Rules: []rbac.PolicyRule{
+				{
+					APIGroups:     []string{"policy"},
+					Resources:     []string{"podsecuritypolicies"},
+					ResourceNames: []string{"default"},
+					Verbs:         []string{"use"},
+				},
+			},
+		},
+	}
+}
+
+type resourceClusterRoleBindings struct {
+	kubernetes.Interface
+}
+
+func (r resourceClusterRoleBindings) List(ctx context.Context) ([]string, error) {
+	res, err := r.RbacV1().ClusterRoleBindings().List(ctx, listBuiltins)
+	if err != nil {
+		return nil, err
+	}
+	objs := make([]string, len(res.Items))
+	for i, el := range res.Items {
+		objs[i] = el.ObjectMeta.Name
+	}
+	return objs, nil
+}
+
+func (r resourceClusterRoleBindings) Create(ctx context.Context, el interface{}) error {
+	_, err := r.RbacV1().ClusterRoleBindings().Create(ctx, el.(*rbac.ClusterRoleBinding), meta.CreateOptions{})
+	return err
+}
+
+func (r resourceClusterRoleBindings) Delete(ctx context.Context, name string) error {
+	return r.RbacV1().ClusterRoleBindings().Delete(ctx, name, meta.DeleteOptions{})
+}
+
+func (r resourceClusterRoleBindings) Expected() map[string]interface{} {
+	return map[string]interface{}{
+		clusterRoleBindingDefaultPSP: &rbac.ClusterRoleBinding{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   clusterRoleBindingDefaultPSP,
+				Labels: builtinLabels(nil),
+				Annotations: map[string]string{
+					"kubernetes.io/description": "This binding grants every service account access to the \"default\" PSP. " +
+						"Creation of Pods is still restricted by other RBAC roles. Otherwise no pods (unprivileged or not) " +
+						"can be created.",
+				},
+			},
+			RoleRef: rbac.RoleRef{
+				APIGroup: rbac.GroupName,
+				Kind:     "ClusterRole",
+				Name:     clusterRolePSPDefault,
+			},
+			Subjects: []rbac.Subject{
+				{
+					APIGroup: rbac.GroupName,
+					Kind:     "Group",
+					Name:     "system:serviceaccounts",
+				},
+			},
+		},
+		clusterRoleBindingAPIServerKubeletClient: &rbac.ClusterRoleBinding{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   clusterRoleBindingAPIServerKubeletClient,
+				Labels: builtinLabels(nil),
+				Annotations: map[string]string{
+					"kubernetes.io/description": "This binding grants the apiserver access to the kubelets. This enables " +
+						"lots of built-in functionality like reading logs or forwarding ports via the API.",
+				},
+			},
+			RoleRef: rbac.RoleRef{
+				APIGroup: rbac.GroupName,
+				Kind:     "ClusterRole",
+				Name:     "system:kubelet-api-admin",
+			},
+			Subjects: []rbac.Subject{
+				{
+					APIGroup: rbac.GroupName,
+					Kind:     "User",
+					// TODO(q3k): describe this name's contract, or unify with whatever creates this.
+					Name: "smalltown:apiserver-kubelet-client",
+				},
+			},
+		},
+	}
+}
diff --git a/core/internal/kubernetes/reconciler/resources_storageclass.go b/core/internal/kubernetes/reconciler/resources_storageclass.go
new file mode 100644
index 0000000..72476ec
--- /dev/null
+++ b/core/internal/kubernetes/reconciler/resources_storageclass.go
@@ -0,0 +1,72 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package reconciler
+
+import (
+	"context"
+
+	core "k8s.io/api/core/v1"
+	storage "k8s.io/api/storage/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/client-go/kubernetes"
+)
+
+var reclaimPolicyDelete = core.PersistentVolumeReclaimDelete
+var waitForConsumerBinding = storage.VolumeBindingWaitForFirstConsumer
+
+type resourceStorageClasses struct {
+	kubernetes.Interface
+}
+
+func (r resourceStorageClasses) List(ctx context.Context) ([]string, error) {
+	res, err := r.StorageV1().StorageClasses().List(ctx, listBuiltins)
+	if err != nil {
+		return nil, err
+	}
+	objs := make([]string, len(res.Items))
+	for i, el := range res.Items {
+		objs[i] = el.ObjectMeta.Name
+	}
+	return objs, nil
+}
+
+func (r resourceStorageClasses) Create(ctx context.Context, el interface{}) error {
+	_, err := r.StorageV1().StorageClasses().Create(ctx, el.(*storage.StorageClass), meta.CreateOptions{})
+	return err
+}
+
+func (r resourceStorageClasses) Delete(ctx context.Context, name string) error {
+	return r.StorageV1().StorageClasses().Delete(ctx, name, meta.DeleteOptions{})
+}
+
+func (r resourceStorageClasses) Expected() map[string]interface{} {
+	return map[string]interface{}{
+		"local": &storage.StorageClass{
+			ObjectMeta: meta.ObjectMeta{
+				Name:   "local",
+				Labels: builtinLabels(nil),
+				Annotations: map[string]string{
+					"storageclass.kubernetes.io/is-default-class": "true",
+				},
+			},
+			AllowVolumeExpansion: True(),
+			Provisioner:          csiProvisionerName,
+			ReclaimPolicy:        &reclaimPolicyDelete,
+			VolumeBindingMode:    &waitForConsumerBinding,
+		},
+	}
+}
diff --git a/core/internal/kubernetes/service.go b/core/internal/kubernetes/service.go
index 7ef6b1d..f95f03e 100644
--- a/core/internal/kubernetes/service.go
+++ b/core/internal/kubernetes/service.go
@@ -35,6 +35,7 @@
 	schema "git.monogon.dev/source/nexantic.git/core/generated/api"
 	"git.monogon.dev/source/nexantic.git/core/internal/common/supervisor"
 	"git.monogon.dev/source/nexantic.git/core/internal/consensus"
+	"git.monogon.dev/source/nexantic.git/core/internal/kubernetes/reconciler"
 	"git.monogon.dev/source/nexantic.git/core/internal/storage"
 	"git.monogon.dev/source/nexantic.git/core/pkg/logbuffer"
 )
@@ -177,7 +178,7 @@
 		if err := supervisor.Run(ctx, "kubelet", runKubelet(&KubeletSpec{}, s.kubeletLogs)); err != nil {
 			return err
 		}
-		if err := supervisor.Run(ctx, "reconciler", runReconciler(clientSet)); err != nil {
+		if err := supervisor.Run(ctx, "reconciler", reconciler.Run(clientSet)); err != nil {
 			return err
 		}
 		if err := supervisor.Run(ctx, "csi-plugin", runCSIPlugin(s.storageService)); err != nil {