cloud/bmaas: implement exponential backoffs

This lets work fail with a proper exponential backoff. This is important
not just to not hammer external systems, but also to not end up with
exteremely long and verbose historical logs for repeatedly failing
processes.

To implement these, we have to slightly alter our model: instead of
always persisting backoffs for a given machine, we only persist them as
long as the last work for a given process has failed, deleting pertinent
backoffs (if any) on success. Then, the existence of a backoff item is
used to calculate the value of the next backoff.

The change also introduces and explicit period, in seconds, to the
backoff item. It is currently implemented as a nullable field, but a
future migration/update might make them non-nullable (and delete any
straggling backoffs that still don't have the period set).

Change-Id: I958fcd957dae1156349224f07fb8d4836955d375
Reviewed-on: https://review.monogon.dev/c/monogon/+/1565
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/cloud/bmaas/bmdb/BUILD.bazel b/cloud/bmaas/bmdb/BUILD.bazel
index 937f4c0..761baef 100644
--- a/cloud/bmaas/bmdb/BUILD.bazel
+++ b/cloud/bmaas/bmdb/BUILD.bazel
@@ -23,6 +23,7 @@
 go_test(
     name = "bmdb_test",
     srcs = [
+        "backoff_test.go",
         "migrations_test.go",
         "queries_test.go",
         "reflection_test.go",
@@ -38,6 +39,7 @@
         "//cloud/bmaas/bmdb/reflection",
         "//cloud/bmaas/server/api",
         "//cloud/lib/component",
+        "@com_github_google_go_cmp//cmp",
         "@com_github_google_uuid//:uuid",
         "@org_golang_google_protobuf//proto",
     ],
diff --git a/cloud/bmaas/bmdb/backoff_test.go b/cloud/bmaas/bmdb/backoff_test.go
new file mode 100644
index 0000000..9d22f2a
--- /dev/null
+++ b/cloud/bmaas/bmdb/backoff_test.go
@@ -0,0 +1,61 @@
+package bmdb
+
+import (
+	"testing"
+	"time"
+
+	"github.com/google/go-cmp/cmp"
+)
+
+// TestBackoffMath exercises the rules of Backoff.
+func TestBackoffMath(t *testing.T) {
+	for _, te := range []struct {
+		name     string
+		b        *Backoff
+		existing *existingBackoff
+		wantSecs []int64
+	}{
+		{"NoBackoffSet", nil, nil, []int64{1, 1, 1}},
+		{"EmptyBackoff", &Backoff{}, nil, []int64{1, 1, 1}},
+		{"SimpleBackoff", &Backoff{Initial: time.Minute}, nil, []int64{60, 60, 60}},
+		{"ExponentialWithMax",
+			&Backoff{Initial: time.Minute, Exponent: 1.1, Maximum: time.Minute * 2},
+			nil,
+			[]int64{60, 66, 73, 81, 90, 99, 109, 120, 120},
+		},
+
+		{"SimpleOverridePrevious",
+			&Backoff{Initial: time.Minute},
+			&existingBackoff{lastInterval: time.Second * 2},
+			[]int64{60, 60, 60},
+		},
+		{"ExponentialOverridePrevious",
+			&Backoff{Initial: time.Minute, Exponent: 2.0, Maximum: time.Minute * 2},
+			&existingBackoff{lastInterval: time.Second * 2},
+			[]int64{4, 8, 16, 32, 64, 120, 120},
+		},
+
+		{"ContinueExisting", nil, &existingBackoff{lastInterval: time.Minute}, []int64{60, 60, 60}},
+		{"ContinueExistingInvalid1", nil, &existingBackoff{lastInterval: 0}, []int64{1, 1, 1}},
+		{"ContinueExistingInvalid2", nil, &existingBackoff{lastInterval: time.Millisecond}, []int64{1, 1, 1}},
+
+		{"InvalidBackoff1", &Backoff{Exponent: 0.2}, nil, []int64{1, 1, 1}},
+		{"InvalidBackoff2", &Backoff{Maximum: time.Millisecond, Initial: time.Millisecond}, nil, []int64{1, 1, 1}},
+	} {
+		t.Run(te.name, func(t *testing.T) {
+			existing := te.existing
+
+			gotSecs := make([]int64, len(te.wantSecs))
+			for j := 0; j < len(te.wantSecs); j++ {
+				gotSecs[j] = te.b.next(existing)
+				existing = &existingBackoff{
+					lastInterval: time.Duration(gotSecs[j]) * time.Second,
+				}
+			}
+
+			if diff := cmp.Diff(te.wantSecs, gotSecs); diff != "" {
+				t.Errorf("Difference: %s", diff)
+			}
+		})
+	}
+}
diff --git a/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.down.sql b/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.down.sql
new file mode 100644
index 0000000..e0cea5f
--- /dev/null
+++ b/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.down.sql
@@ -0,0 +1,2 @@
+ALTER TABLE work_backoff
+DROP COLUMN last_interval_seconds;
\ No newline at end of file
diff --git a/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.up.sql b/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.up.sql
new file mode 100644
index 0000000..94fd8a5
--- /dev/null
+++ b/cloud/bmaas/bmdb/model/migrations/1681826233_exponential_backoff.up.sql
@@ -0,0 +1,5 @@
+-- Add interval, in seconds. This is used to calculate subsequent backoff values
+-- for exponential backoffs. A future migration should make this field
+-- non-nullable.
+ALTER TABLE work_backoff
+ADD COLUMN last_interval_seconds BIGINT NULL;
\ No newline at end of file
diff --git a/cloud/bmaas/bmdb/model/queries_base.sql b/cloud/bmaas/bmdb/model/queries_base.sql
index 4119f94..e150fbb 100644
--- a/cloud/bmaas/bmdb/model/queries_base.sql
+++ b/cloud/bmaas/bmdb/model/queries_base.sql
@@ -54,14 +54,28 @@
 -- name: WorkBackoffInsert :exec
 -- Upsert a backoff for a given machine/process.
 INSERT INTO work_backoff (
-    machine_id, process, cause, until
+    machine_id, process, cause, until, last_interval_seconds
 ) VALUES (
-    $1, $2, $3, now() + (sqlc.arg(seconds)::int * interval '1 second')
+    $1, $2, $3,
+    now() + (sqlc.arg(seconds)::int * interval '1 second'),
+    sqlc.arg(seconds)::bigint
 ) ON CONFLICT (machine_id, process) DO UPDATE SET
     cause = $3,
-    until = now() + (sqlc.arg(seconds)::int * interval '1 second')
+    until = now() + (sqlc.arg(seconds)::int * interval '1 second'),
+    last_interval_seconds = sqlc.arg(seconds)::bigint
 ;
 
+-- name: WorkBackoffDelete :exec
+DELETE FROM work_backoff
+WHERE machine_id = $1
+  AND process = $2;
+
+-- name: WorkBackoffOf :many
+SELECT *
+FROM work_backoff
+WHERE machine_id = $1
+  AND process = $2;
+
 -- name: ListHistoryOf :many
 -- Retrieve full audit history of a machine.
 SELECT *
diff --git a/cloud/bmaas/bmdb/reflection_test.go b/cloud/bmaas/bmdb/reflection_test.go
index 3d23efc..6b8b506 100644
--- a/cloud/bmaas/bmdb/reflection_test.go
+++ b/cloud/bmaas/bmdb/reflection_test.go
@@ -64,8 +64,10 @@
 	if err != nil {
 		t.Fatal(err)
 	}
-	to := time.Hour
-	w.Fail(ctx, &to, "failure test")
+	backoff := Backoff{
+		Initial: time.Hour,
+	}
+	w.Fail(ctx, &backoff, "failure test")
 
 	// On another machine, create a failure with a 1 second backoff.
 	w, err = sess.Work(ctx, model.ProcessUnitTest1, func(q *model.Queries) ([]uuid.UUID, error) {
@@ -74,11 +76,13 @@
 	if err != nil {
 		t.Fatal(err)
 	}
-	to = time.Second
-	w.Fail(ctx, &to, "failure test")
+	backoff = Backoff{
+		Initial: time.Second,
+	}
+	w.Fail(ctx, &backoff, "failure test")
 	// Later on in the test we must wait for this backoff to actually elapse. Start
 	// counting now.
-	elapsed := time.NewTicker(to * 1)
+	elapsed := time.NewTicker(time.Second * 1)
 	defer elapsed.Stop()
 
 	// On another machine, create work and don't finish it yet.
diff --git a/cloud/bmaas/bmdb/sessions.go b/cloud/bmaas/bmdb/sessions.go
index 6058b92..144e6da 100644
--- a/cloud/bmaas/bmdb/sessions.go
+++ b/cloud/bmaas/bmdb/sessions.go
@@ -208,6 +208,7 @@
 // The returned Work object is _not_ goroutine safe.
 func (s *Session) Work(ctx context.Context, process model.Process, fn func(q *model.Queries) ([]uuid.UUID, error)) (*Work, error) {
 	var mid *uuid.UUID
+	var exisingingBackoff *existingBackoff
 	err := s.Transact(ctx, func(q *model.Queries) error {
 		mids, err := fn(q)
 		if err != nil {
@@ -237,6 +238,25 @@
 		if err != nil {
 			return fmt.Errorf("could not insert history event: %w", err)
 		}
+		backoffs, err := q.WorkBackoffOf(ctx, model.WorkBackoffOfParams{
+			MachineID: mids[0],
+			Process:   process,
+		})
+		if err != nil {
+			return fmt.Errorf("could not get backoffs: %w", err)
+		}
+		if len(backoffs) > 0 {
+			// If the backoff exists but the last interval is null (e.g. is from a previous
+			// version of the schema when backoffs had no interval data) pretend it doesn't
+			// exist. Then the backoff mechanism can restart from a clean slate and populate
+			// a new, full backoff row.
+			if backoff := backoffs[0]; backoff.LastIntervalSeconds.Valid {
+				klog.Infof("Existing backoff: %d seconds", backoff.LastIntervalSeconds.Int64)
+				exisingingBackoff = &existingBackoff{
+					lastInterval: time.Second * time.Duration(backoff.LastIntervalSeconds.Int64),
+				}
+			}
+		}
 		return nil
 	})
 	if err != nil {
@@ -247,9 +267,187 @@
 		Machine: *mid,
 		s:       s,
 		process: process,
+		backoff: exisingingBackoff,
 	}, nil
 }
 
+// existingBackoff contains backoff information retrieved from a work item that
+// has previously failed with a backoff.
+type existingBackoff struct {
+	// lastInterval is the last interval as stored in the backoff table.
+	lastInterval time.Duration
+}
+
+// Backoff describes the configuration of backoff for a failed work item. It can
+// be passed to Work.Fail to cause an item to not be processed again (to be 'in
+// backoff') for a given period of time. Exponential backoff can be configured so
+// that subsequent failures of a process will have exponentially increasing
+// backoff periods, up to some maximum length.
+//
+// The underlying unit of backoff period length in the database is one second.
+// What that means is that all effective calculated backoff periods must be an
+// integer number of seconds. This is performed by always rounding up this period
+// to the nearest second. A side effect of this is that with exponential backoff,
+// non-integer exponents will be less precisely applied for small backoff values,
+// e.g. an exponent of 1.1 with initial backoff of 1s will generate the following
+// sequence of backoff periods:
+//
+// 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 13, 15, 17
+//
+// Which corresponds to the following approximate multipliers in between periods:
+//
+// 2.00, 1.50, 1.33, 1.25, 1.20, 1.17, 1.14, 1.12, 1.11, 1.10, 1.18, 1.15, 1.13
+//
+// Thus, the exponent value should be treated more as a limit that the sequence
+// of periods will approach than a hard rule for calculating the periods.
+// However, if the exponent is larger than 1 (i.e. any time exponential backoff
+// is requested), this guarantees that the backoff won't get 'stuck' on a
+// repeated period value due to a rounding error.
+//
+// A zero backoff structure is valid and represents a non-exponential backoff of
+// one second.
+//
+// A partially filled structure is also valid. See the field comments for more
+// information about how fields are capped if not set. The described behaviour
+// allows for two useful shorthands:
+//
+//  1. If only Initial is set, then the backoff is non-exponential and will always
+//     be of value Initial (or whatever the previous period already persisted the
+//     database).
+//  2. If only Maximum and Exponent are set, the backoff will be exponential,
+//     starting at one second, and exponentially increasing to Maximum.
+//
+// It is recommended to construct Backoff structures as const values and treat
+// them as read-only 'descriptors', one per work kind / process.
+//
+// One feature currently missing from the Backoff implementation is jitter. This
+// might be introduced in the future if deemed necessary.
+type Backoff struct {
+	// Initial backoff period, used for the backoff if this item failed for the first
+	// time (i.e. has not had a Finish call in between two Fail calls).
+	//
+	// Subsequent calls will ignore this field if the backoff is exponential. If
+	// non-exponential, the initial time will always override whatever was previously
+	// persisted in the database, i.e. the backoff will always be of value 'Initial'.
+	//
+	// Cannot be lower than one second. If it is, it will be capped to it.
+	Initial time.Duration `u:"initial"`
+
+	// Maximum time for backoff. If the calculation of the next back off period
+	// (based on the Exponent and last backoff value) exceeds this maximum, it will
+	// be capped to it.
+	//
+	// Maximum is not persisted in the database. Instead, it is always read from this
+	// structure.
+	//
+	// Cannot be lower than Initial. If it is, it will be capped to it.
+	Maximum time.Duration `u:"maximum"`
+
+	// Exponent used for next backoff calculation. Any time a work item fails
+	// directly after another failure, the previous backoff period will be multiplied
+	// by the exponent to yield the new backoff period. The new period will then be
+	// capped to Maximum.
+	//
+	// Exponent is not persisted in the database. Instead, it is always read from
+	// this structure.
+	//
+	// Cannot be lower than 1.0. If it is, it will be capped to it.
+	Exponent float64 `u:"exponent"`
+}
+
+// normalized copies the given backoff and returns a 'normalized' version of it,
+// with the 'when zero/unset' rules described in the Backoff documentation
+// strings.
+func (b *Backoff) normalized() *Backoff {
+	c := *b
+
+	if c.Exponent < 1.0 {
+		c.Exponent = 1.0
+	}
+	if c.Initial < time.Second {
+		c.Initial = time.Second
+	}
+	if c.Maximum < c.Initial {
+		c.Maximum = c.Initial
+	}
+	return &c
+}
+
+func (b *Backoff) simple() bool {
+	// Non-normalized simple backoffs will have a zero exponent.
+	if b.Exponent == 0.0 {
+		return true
+	}
+	// Normalized simple backoffs will have a 1.0 exponent.
+	if b.Exponent == 1.0 {
+		return true
+	}
+	return false
+}
+
+// next calculates the backoff period based on a backoff descriptor and previous
+// existing backoff information. Both or either can be nil.
+func (b *Backoff) next(e *existingBackoff) int64 {
+	second := time.Second.Nanoseconds()
+
+	// Minimum interval is one second. Start with that.
+	last := second
+	// Then, if we have a previous interval, and it's greater than a second, use that
+	// as the last interval.
+	if e != nil {
+		if previous := e.lastInterval.Nanoseconds(); previous > second {
+			last = previous
+		}
+	}
+
+	// If no backoff is configured, go with either the minimum of one second, or
+	// whatever the last previous interval was.
+	if b == nil {
+		return last / second
+	}
+
+	// Make a copy of the backoff descriptor, normalizing as necessary.
+	c := b.normalized()
+
+	// Simple backoffs always return Initial.
+	if b.simple() {
+		return c.Initial.Nanoseconds() / second
+	}
+
+	// If there is no existing backoff, return the initial backoff value directly.
+	if e == nil {
+		return c.Initial.Nanoseconds() / second
+	}
+
+	// Start out with the persisted interval.
+	next := last
+	// If by any chance we persisted an interval less than one second, clamp it.
+	if next < second {
+		next = second
+	}
+
+	// Multiply by exponent from descriptor.
+	next = int64(float64(next) * c.Exponent)
+
+	// Handle overflows. If multiplying by a positive number resulted in a lower
+	// value than what we started with, it means we overflowed and wrapped around. If
+	// so, clamp to maximum.
+	if next < last {
+		next = c.Maximum.Nanoseconds()
+	}
+
+	// Clamp to maximum.
+	if next > c.Maximum.Nanoseconds() {
+		next = c.Maximum.Nanoseconds()
+	}
+	// Round up to the nearest second.
+	if next%second == 0 {
+		return next / second
+	} else {
+		return next/second + 1
+	}
+}
+
 // Work being performed on a machine.
 type Work struct {
 	// Machine that this work is being performed on, as retrieved by the retrieval
@@ -261,6 +459,8 @@
 	done bool
 	// process that this work performs.
 	process model.Process
+
+	backoff *existingBackoff
 }
 
 // Cancel the Work started on a machine. If the work has already been finished
@@ -318,6 +518,13 @@
 		if err != nil {
 			return err
 		}
+		err = q.WorkBackoffDelete(ctx, model.WorkBackoffDeleteParams{
+			MachineID: w.Machine,
+			Process:   w.process,
+		})
+		if err != nil {
+			return err
+		}
 		err = q.WorkHistoryInsert(ctx, model.WorkHistoryInsertParams{
 			MachineID: w.Machine,
 			Process:   w.process,
@@ -330,12 +537,28 @@
 	})
 }
 
-// Fail work and introduce backoff for a given duration (if given backoff is
-// non-nil). As long as that backoff is active, no further work for this
-// machine/process will be started. The given cause is an operator-readable
+// Fail work and introduce backoff. The given cause is an operator-readable
 // string that will be persisted alongside the backoff and the work history/audit
 // table.
-func (w *Work) Fail(ctx context.Context, backoff *time.Duration, cause string) error {
+//
+// The backoff describes a period during which the same process will not be
+// retried on this machine until its expiration.
+//
+// The given backoff is a structure which describes both the initial backoff
+// period if the work failed for the first time, and a mechanism to exponentially
+// increase the backoff period if that work failed repeatedly. The work is
+// defined to have failed repeatedly if it only resulted in Cancel/Fail calls
+// without any Finish calls in the meantime.
+//
+// Only the last backoff period is persisted in the database. The exponential
+// backoff behaviour (including its maximum time) is always calculated based on
+// the given backoff structure.
+//
+// If nil, the backoff defaults to a non-exponential, one second backoff. This is
+// the minimum designed to keep the system chugging along without repeatedly
+// trying a failed job in a loop. However, the backoff should generally be set to
+// some well engineered value to prevent spurious retries.
+func (w *Work) Fail(ctx context.Context, backoff *Backoff, cause string) error {
 	if w.done {
 		return fmt.Errorf("already finished")
 	}
@@ -362,17 +585,16 @@
 		if err != nil {
 			return err
 		}
-		if backoff != nil && backoff.Seconds() >= 1.0 {
-			seconds := int64(backoff.Seconds())
-			klog.Infof("Adding backoff for %q on machine %q (%d seconds)", w.process, w.Machine, seconds)
-			return q.WorkBackoffInsert(ctx, model.WorkBackoffInsertParams{
-				MachineID: w.Machine,
-				Process:   w.process,
-				Seconds:   seconds,
-				Cause:     cause,
-			})
-		} else {
-			return nil
+		if backoff == nil {
+			klog.Warningf("Nil backoff for %q on machine %q: defaulting to one second non-exponential.", w.process, w.Machine)
 		}
+		seconds := backoff.next(w.backoff)
+		klog.Infof("Adding backoff for %q on machine %q: %d seconds", w.process, w.Machine, seconds)
+		return q.WorkBackoffInsert(ctx, model.WorkBackoffInsertParams{
+			MachineID: w.Machine,
+			Process:   w.process,
+			Cause:     cause,
+			Seconds:   seconds,
+		})
 	})
 }
diff --git a/cloud/bmaas/bmdb/sessions_test.go b/cloud/bmaas/bmdb/sessions_test.go
index f3977d5..e697150 100644
--- a/cloud/bmaas/bmdb/sessions_test.go
+++ b/cloud/bmaas/bmdb/sessions_test.go
@@ -160,6 +160,7 @@
 	}
 }
 
+// TestWorkBackoff exercises the backoff functionality within the BMDB.
 func TestWorkBackoff(t *testing.T) {
 	b := dut()
 	conn, err := b.Open(true)
@@ -192,70 +193,123 @@
 		t.Fatalf("Creating machine failed: %v", err)
 	}
 
-	// Work on machine, but fail it with a backoff.
-	work, err := session.Work(ctx, model.ProcessShepherdAccess, func(q *model.Queries) ([]uuid.UUID, error) {
-		machines, err := q.GetMachinesForAgentStart(ctx, 1)
-		if err != nil {
-			return nil, err
+	waitMachine := func(nsec int64) *Work {
+		t.Helper()
+
+		deadline := time.Now().Add(time.Duration(nsec) * 2 * time.Second)
+		for {
+			if time.Now().After(deadline) {
+				t.Fatalf("Deadline expired")
+			}
+			work, err := session.Work(ctx, model.ProcessShepherdAccess, func(q *model.Queries) ([]uuid.UUID, error) {
+				machines, err := q.GetMachinesForAgentStart(ctx, 1)
+				if err != nil {
+					return nil, err
+				}
+				if len(machines) < 1 {
+					return nil, ErrNothingToDo
+				}
+				return []uuid.UUID{machines[0].MachineID}, nil
+			})
+			if err == nil {
+				return work
+			}
+			if !errors.Is(err, ErrNothingToDo) {
+				t.Fatalf("Unexpected work error: %v", err)
+			}
+			time.Sleep(100 * time.Millisecond)
 		}
-		if len(machines) < 1 {
-			return nil, ErrNothingToDo
-		}
-		return []uuid.UUID{machines[0].MachineID}, nil
-	})
-	if err != nil {
-		t.Fatalf("Starting work failed: %v", err)
 	}
-	backoff := time.Hour
+
+	// Work on machine, but fail it with a backoff.
+	work := waitMachine(1)
+	backoff := Backoff{
+		Initial:  time.Second,
+		Maximum:  5 * time.Second,
+		Exponent: 2,
+	}
 	if err := work.Fail(ctx, &backoff, "test"); err != nil {
 		t.Fatalf("Failing work failed: %v", err)
 	}
 
-	// The machine shouldn't be returned now.
-	err = session.Transact(ctx, func(q *model.Queries) error {
-		machines, err := q.GetMachinesForAgentStart(ctx, 1)
-		if err != nil {
-			return err
-		}
-		if len(machines) > 0 {
-			t.Errorf("Expected no machines ready for agent start.")
-		}
-		return nil
-	})
-	if err != nil {
-		t.Errorf("Failed to retrieve machines for agent start: %v", err)
-	}
+	expect := func(count int) {
+		t.Helper()
 
-	// Instead of waiting for the backoff to expire, set it again, but this time
-	// make it immediate. This works because the backoff query acts as an upsert.
-	err = session.Transact(ctx, func(q *model.Queries) error {
-		return q.WorkBackoffInsert(ctx, model.WorkBackoffInsertParams{
-			MachineID: machine.MachineID,
-			Process:   model.ProcessShepherdAccess,
-			Seconds:   0,
+		var machines []model.MachineProvided
+		var err error
+		err = session.Transact(ctx, func(q *model.Queries) error {
+			machines, err = q.GetMachinesForAgentStart(ctx, 1)
+			if err != nil {
+				return err
+			}
+			return nil
 		})
-	})
-	if err != nil {
-		t.Errorf("Failed to update backoff: %v", err)
+		if err != nil {
+			t.Errorf("Failed to retrieve machines for agent start: %v", err)
+		}
+		if want, got := count, len(machines); want != got {
+			t.Errorf("Expected %d machines, got %d", want, got)
+		}
 	}
 
-	// Just in case.
-	time.Sleep(100 * time.Millisecond)
+	// The machine shouldn't be returned now.
+	expect(0)
+
+	// Wait for the backoff to expire.
+	time.Sleep(1100 * time.Millisecond)
 
 	// The machine should now be returned again.
-	err = session.Transact(ctx, func(q *model.Queries) error {
-		machines, err := q.GetMachinesForAgentStart(ctx, 1)
-		if err != nil {
+	expect(1)
+
+	// Prepare helper for checking exponential backoffs.
+	failAndCheck := func(nsec int64) {
+		t.Helper()
+		work := waitMachine(nsec)
+		if err := work.Fail(ctx, &backoff, "test"); err != nil {
+			t.Fatalf("Failing work failed: %v", err)
+		}
+
+		var backoffs []model.WorkBackoff
+		err = session.Transact(ctx, func(q *model.Queries) error {
+			var err error
+			backoffs, err = q.WorkBackoffOf(ctx, model.WorkBackoffOfParams{
+				MachineID: machine.MachineID,
+				Process:   model.ProcessShepherdAccess,
+			})
 			return err
+		})
+		if err != nil {
+			t.Errorf("Failed to retrieve machines for agent start: %v", err)
 		}
-		if len(machines) != 1 {
-			t.Errorf("Expected exactly one machine ready for agent start.")
+		if len(backoffs) < 1 {
+			t.Errorf("No backoff")
+		} else {
+			backoff := backoffs[0]
+			if want, got := nsec, backoff.LastIntervalSeconds.Int64; want != got {
+				t.Fatalf("Wanted backoff of %d seconds, got %d", want, got)
+			}
 		}
+	}
+
+	// Exercise exponential backoff functionality.
+	failAndCheck(2)
+	failAndCheck(4)
+	failAndCheck(5)
+	failAndCheck(5)
+
+	// If the job now succeeds, subsequent failures should start from 1 again.
+	work = waitMachine(5)
+	err = work.Finish(ctx, func(q *model.Queries) error {
+		// Not setting any tags that would cause subsequent queries to not return the
+		// machine anymore.
 		return nil
 	})
 	if err != nil {
-		t.Errorf("Failed to retrieve machines for agent start: %v", err)
+		t.Fatalf("Could not finish work: %v", err)
 	}
+
+	failAndCheck(1)
+	failAndCheck(2)
 }
 
 // TestAgentStartWorkflow exercises the agent start workflow within the BMDB.