From 4ac90bb78285f6c1700996d50698f4e6b14e38de Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Mon, 12 Nov 2018 14:17:03 +0100 Subject: [PATCH] storage: refactor log truncation index computation This makes it a lot easier to log descriptive debug messages indicating how a truncation decision was arrived at, and in particular allows pointing the finger at truncations that lead to Raft snapshots, which is relevant in the context of #32046. Release note: None --- pkg/storage/raft_log_queue.go | 241 ++++++++++++++++++++--------- pkg/storage/raft_log_queue_test.go | 164 ++++++++++++++------ 2 files changed, 283 insertions(+), 122 deletions(-) diff --git a/pkg/storage/raft_log_queue.go b/pkg/storage/raft_log_queue.go index 7e2622c068ba..90c51047e3ce 100644 --- a/pkg/storage/raft_log_queue.go +++ b/pkg/storage/raft_log_queue.go @@ -16,11 +16,11 @@ package storage import ( "context" + "fmt" "sort" + "strings" "time" - "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" - "github.com/pkg/errors" "go.etcd.io/etcd/raft" @@ -28,8 +28,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) const ( @@ -55,12 +58,15 @@ const ( type raftLogQueue struct { *baseQueue db *client.DB + + logSnapshots util.EveryN } // newRaftLogQueue returns a new instance of raftLogQueue. func newRaftLogQueue(store *Store, db *client.DB, gossip *gossip.Gossip) *raftLogQueue { rlq := &raftLogQueue{ - db: db, + db: db, + logSnapshots: util.Every(10 * time.Second), } rlq.baseQueue = newBaseQueue( "raftlog", rlq, store, gossip, @@ -79,29 +85,11 @@ func newRaftLogQueue(store *Store, db *client.DB, gossip *gossip.Gossip) *raftLo return rlq } -func shouldTruncate(truncatableIndexes uint64, raftLogSize int64) bool { - return truncatableIndexes >= RaftLogQueueStaleThreshold || - (truncatableIndexes > 0 && raftLogSize >= RaftLogQueueStaleSize) -} - -// getTruncatableIndexes returns the number of truncatable indexes, the oldest -// index that cannot be truncated, and the current Raft log size. See -// computeTruncatableIndex. -func getTruncatableIndexes(ctx context.Context, r *Replica) (uint64, uint64, int64, error) { +// newTruncateDecision returns a truncateDecision for the given Replica if no +// error occurs. If no truncation can be carried out, a zero decision is +// returned. +func newTruncateDecision(ctx context.Context, r *Replica) (*truncateDecision, error) { rangeID := r.RangeID - raftStatus := r.RaftStatus() - if raftStatus == nil { - if log.V(6) { - log.Infof(ctx, "the raft group doesn't exist for r%d", rangeID) - } - return 0, 0, 0, nil - } - - // Is this the raft leader? We only perform log truncation on the raft leader - // which has the up to date info on followers. - if raftStatus.RaftState != raft.StateLeader { - return 0, 0, 0, nil - } r.mu.Lock() raftLogSize := r.mu.raftLogSize @@ -119,21 +107,129 @@ func getTruncatableIndexes(ctx context.Context, r *Replica) (uint64, uint64, int if targetSize > r.store.cfg.RaftLogTruncationThreshold { targetSize = r.store.cfg.RaftLogTruncationThreshold } + raftStatus := r.raftStatusRLocked() + firstIndex, err := r.raftFirstIndexLocked() pendingSnapshotIndex := r.mu.pendingSnapshotIndex lastIndex := r.mu.lastIndex r.mu.Unlock() + if err != nil { - return 0, 0, 0, errors.Errorf("error retrieving first index for r%d: %s", rangeID, err) + return nil, errors.Errorf("error retrieving first index for r%d: %s", rangeID, err) + } + + if raftStatus == nil { + if log.V(6) { + log.Infof(ctx, "the raft group doesn't exist for r%d", rangeID) + } + return &truncateDecision{}, nil + } + + // Is this the raft leader? We only perform log truncation on the raft leader + // which has the up to date info on followers. + if raftStatus.RaftState != raft.StateLeader { + return &truncateDecision{}, nil + } + + input := truncateDecisionInput{ + RaftStatus: raftStatus, + LogSize: raftLogSize, + MaxLogSize: targetSize, + FirstIndex: firstIndex, + LastIndex: lastIndex, + PendingPreemptiveSnapshotIndex: pendingSnapshotIndex, + } + + decision := computeTruncateDecision(input) + return &decision, nil +} + +const ( + truncatableIndexChosenViaQuorumIndex = "quorum" + truncatableIndexChosenViaFollowers = "followers" + truncatableIndexChosenViaPendingSnap = "pending snapshot" + truncatableIndexChosenViaFirstIndex = "first index" + truncatableIndexChosenViaLastIndex = "last index" +) + +type truncateDecisionInput struct { + RaftStatus *raft.Status // never nil + LogSize, MaxLogSize int64 + FirstIndex, LastIndex uint64 + PendingPreemptiveSnapshotIndex uint64 +} + +func (input truncateDecisionInput) LogTooLarge() bool { + return input.LogSize > input.MaxLogSize +} + +type truncateDecision struct { + Input truncateDecisionInput + QuorumIndex uint64 // largest index known to be present on quorum + + NewFirstIndex uint64 // first index of the resulting log after truncation + ChosenVia string +} + +func (td *truncateDecision) raftSnapshotsForIndex(index uint64) int { + var n int + for _, p := range td.Input.RaftStatus.Progress { + if p.Match < index { + n++ + } } + if td.Input.PendingPreemptiveSnapshotIndex != 0 && td.Input.PendingPreemptiveSnapshotIndex < index { + n++ + } + + return n +} - truncatableIndex := computeTruncatableIndex( - raftStatus, raftLogSize, targetSize, firstIndex, lastIndex, pendingSnapshotIndex) - // Return the number of truncatable indexes. - return truncatableIndex - firstIndex, truncatableIndex, raftLogSize, nil +func (td *truncateDecision) NumNewRaftSnapshots() int { + return td.raftSnapshotsForIndex(td.NewFirstIndex) - td.raftSnapshotsForIndex(td.Input.FirstIndex) } -// computeTruncatableIndex returns the oldest index that cannot be +func (td *truncateDecision) String() string { + var buf strings.Builder + _, _ = fmt.Fprintf( + &buf, + "truncate %d entries to first index %d (chosen via: %s)", + td.NumTruncatableIndexes(), td.NewFirstIndex, td.ChosenVia, + ) + if td.Input.LogTooLarge() { + _, _ = fmt.Fprintf( + &buf, + "; log too large (%s > %s)", + humanizeutil.IBytes(td.Input.LogSize), + humanizeutil.IBytes(td.Input.MaxLogSize), + ) + } + if n := td.NumNewRaftSnapshots(); n > 0 { + _, _ = fmt.Fprintf(&buf, "; implies %d Raft snapshot%s", n, util.Pluralize(int64(n))) + } + + return buf.String() +} + +func (td *truncateDecision) NumTruncatableIndexes() int { + if td.NewFirstIndex < td.Input.FirstIndex { + log.Fatalf( + context.Background(), + "invalid truncate decision: first index would move from %d to %d", + td.Input.FirstIndex, + td.NewFirstIndex, + ) + } + return int(td.NewFirstIndex - td.Input.FirstIndex) +} + +func (td *truncateDecision) ShouldTruncate() bool { + n := td.NumTruncatableIndexes() + return n >= RaftLogQueueStaleThreshold || + (n > 0 && td.Input.LogSize >= RaftLogQueueStaleSize) +} + +// computeTruncateDecision returns the oldest index that cannot be // truncated. If there is a behind node, we want to keep old raft logs so it // can catch up without having to send a full snapshot. However, if a node down // is down long enough, sending a snapshot is more efficient and we should @@ -150,53 +246,57 @@ func getTruncatableIndexes(ctx context.Context, r *Replica) (uint64, uint64, int // the behind node will be caught up to a point behind the current first index // and thus require another snapshot, likely entering a never ending loop of // snapshots. See #8629. -func computeTruncatableIndex( - raftStatus *raft.Status, - raftLogSize int64, - targetSize int64, - firstIndex uint64, - lastIndex uint64, - pendingSnapshotIndex uint64, -) uint64 { - quorumIndex := getQuorumIndex(raftStatus, pendingSnapshotIndex) - truncatableIndex := quorumIndex - - if raftLogSize <= targetSize { +func computeTruncateDecision(input truncateDecisionInput) truncateDecision { + decision := truncateDecision{Input: input} + decision.QuorumIndex = getQuorumIndex(input.RaftStatus, input.PendingPreemptiveSnapshotIndex) + + decision.NewFirstIndex = decision.QuorumIndex + decision.ChosenVia = truncatableIndexChosenViaQuorumIndex + + if !input.LogTooLarge() { // Only truncate to one of the follower indexes if the raft log is less // than the target size. If the raft log is greater than the target size we // always truncate to the quorum commit index. - for _, progress := range raftStatus.Progress { + for _, progress := range input.RaftStatus.Progress { index := progress.Match - if truncatableIndex > index { - truncatableIndex = index + if decision.NewFirstIndex > index { + decision.NewFirstIndex = index + decision.ChosenVia = truncatableIndexChosenViaFollowers } } // The pending snapshot index acts as a placeholder for a replica that is // about to be added to the range. We don't want to truncate the log in a // way that will require that new replica to be caught up via a Raft // snapshot. - if pendingSnapshotIndex > 0 && truncatableIndex > pendingSnapshotIndex { - truncatableIndex = pendingSnapshotIndex + if input.PendingPreemptiveSnapshotIndex > 0 && decision.NewFirstIndex > input.PendingPreemptiveSnapshotIndex { + decision.NewFirstIndex = input.PendingPreemptiveSnapshotIndex + decision.ChosenVia = truncatableIndexChosenViaPendingSnap } } - if truncatableIndex < firstIndex { - truncatableIndex = firstIndex - } - // Never truncate past the quorum commit index (this can only occur if - // firstIndex > quorumIndex). - if truncatableIndex > quorumIndex { - truncatableIndex = quorumIndex + // Advance to the first index, but never truncate past the quorum commit + // index. + if decision.NewFirstIndex < input.FirstIndex && input.FirstIndex <= decision.QuorumIndex { + decision.NewFirstIndex = input.FirstIndex + decision.ChosenVia = truncatableIndexChosenViaFirstIndex } // Never truncate past the last index. Naively, you would expect lastIndex to // never be smaller than quorumIndex, but RaftStatus.Progress.Match is // updated on the leader when a command is proposed and in a single replica // Raft group this also means that RaftStatus.Commit is updated at propose // time. - if truncatableIndex > lastIndex { - truncatableIndex = lastIndex + if decision.NewFirstIndex > input.LastIndex { + decision.NewFirstIndex = input.LastIndex + decision.ChosenVia = truncatableIndexChosenViaLastIndex + } + + // If new first index dropped below first index, make them equal (resulting + // in a no-op). + if decision.NewFirstIndex < decision.Input.FirstIndex { + decision.NewFirstIndex = decision.Input.FirstIndex + decision.ChosenVia = truncatableIndexChosenViaFirstIndex } - return truncatableIndex + return decision } // getQuorumIndex returns the index which a quorum of the nodes have @@ -229,45 +329,40 @@ func getQuorumIndex(raftStatus *raft.Status, pendingSnapshotIndex uint64) uint64 func (rlq *raftLogQueue) shouldQueue( ctx context.Context, now hlc.Timestamp, r *Replica, _ *config.SystemConfig, ) (shouldQ bool, priority float64) { - truncatableIndexes, _, raftLogSize, err := getTruncatableIndexes(ctx, r) + decision, err := newTruncateDecision(ctx, r) if err != nil { log.Warning(ctx, err) return false, 0 } - - return shouldTruncate(truncatableIndexes, raftLogSize), float64(raftLogSize) + return decision.ShouldTruncate(), float64(decision.Input.LogSize) } // process truncates the raft log of the range if the replica is the raft // leader and if the total number of the range's raft log's stale entries // exceeds RaftLogQueueStaleThreshold. func (rlq *raftLogQueue) process(ctx context.Context, r *Replica, _ *config.SystemConfig) error { - truncatableIndexes, oldestIndex, raftLogSize, err := getTruncatableIndexes(ctx, r) + decision, err := newTruncateDecision(ctx, r) if err != nil { return err } // Can and should the raft logs be truncated? - if shouldTruncate(truncatableIndexes, raftLogSize) { - r.mu.Lock() - raftLogSize := r.mu.raftLogSize - lastIndex := r.mu.lastIndex - r.mu.Unlock() - - if log.V(1) { - log.Infof(ctx, "truncating raft log entries [%d-%d], resulting in log [%d,%d], reclaiming ~%s", - oldestIndex-truncatableIndexes, oldestIndex-1, oldestIndex, lastIndex, humanizeutil.IBytes(raftLogSize)) + if decision.ShouldTruncate() { + if n := decision.NumNewRaftSnapshots(); log.V(1) || n > 0 && rlq.logSnapshots.ShouldProcess(timeutil.Now()) { + log.Info(ctx, decision) + } else { + log.VEvent(ctx, 1, decision.String()) } b := &client.Batch{} b.AddRawRequest(&roachpb.TruncateLogRequest{ RequestHeader: roachpb.RequestHeader{Key: r.Desc().StartKey.AsRawKey()}, - Index: oldestIndex, + Index: decision.NewFirstIndex, RangeID: r.RangeID, }) if err := rlq.db.Run(ctx, b); err != nil { return err } - r.store.metrics.RaftLogTruncated.Inc(int64(truncatableIndexes)) + r.store.metrics.RaftLogTruncated.Inc(int64(decision.NumTruncatableIndexes())) } return nil } diff --git a/pkg/storage/raft_log_queue_test.go b/pkg/storage/raft_log_queue_test.go index 5c7750e094a1..5bfcb54539b8 100644 --- a/pkg/storage/raft_log_queue_test.go +++ b/pkg/storage/raft_log_queue_test.go @@ -18,19 +18,17 @@ import ( "context" "fmt" "math" - "reflect" "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/storage/engine" - "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" - "github.com/pkg/errors" "go.etcd.io/etcd/raft" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -52,7 +50,11 @@ func TestShouldTruncate(t *testing.T) { } for _, c := range testCases { t.Run("", func(t *testing.T) { - v := shouldTruncate(c.truncatableIndexes, c.raftLogSize) + var d truncateDecision + d.Input.LogSize = c.raftLogSize + d.Input.FirstIndex = 123 + d.NewFirstIndex = d.Input.FirstIndex + c.truncatableIndexes + v := d.ShouldTruncate() if c.expected != v { t.Fatalf("expected %v, but found %v", c.expected, v) } @@ -96,7 +98,7 @@ func TestGetQuorumIndex(t *testing.T) { } } -func TestComputeTruncatableIndex(t *testing.T) { +func TestComputeTruncateDecision(t *testing.T) { defer leaktest.AfterTest(t)() const targetSize = 1000 @@ -107,27 +109,90 @@ func TestComputeTruncatableIndex(t *testing.T) { firstIndex uint64 lastIndex uint64 pendingSnapshot uint64 - expected uint64 + exp string }{ - {[]uint64{1, 2}, 100, 1, 1, 0, 1}, - {[]uint64{1, 5, 5}, 100, 1, 1, 0, 1}, - {[]uint64{1, 5, 5}, 100, 2, 2, 0, 2}, - {[]uint64{5, 5, 5}, 100, 2, 5, 0, 5}, - {[]uint64{5, 5, 5}, 100, 2, 2, 1, 2}, - {[]uint64{5, 5, 5}, 100, 2, 3, 3, 3}, - {[]uint64{1, 2, 3, 4}, 100, 1, 1, 0, 1}, - {[]uint64{1, 2, 3, 4}, 100, 2, 2, 0, 2}, - // If over targetSize, should truncate to quorum committed index. - {[]uint64{1, 3, 3, 4}, 2000, 1, 3, 0, 3}, - {[]uint64{1, 3, 3, 4}, 2000, 2, 3, 0, 3}, - {[]uint64{1, 3, 3, 4}, 2000, 3, 3, 0, 3}, + { + // Nothing to truncate. + []uint64{1, 2}, 100, 1, 1, 0, + "truncate 0 entries to first index 1 (chosen via: quorum)"}, + { + // Nothing to truncate on this replica, though a quorum elsewhere has more progress. + // NB this couldn't happen if we're truly the Raft leader, unless we appended to our + // own log asynchronously. + []uint64{1, 5, 5}, 100, 1, 1, 0, + "truncate 0 entries to first index 1 (chosen via: followers)", + }, + { + // We're not truncating anything, but one follower is already cut off. There's no pending + // snapshot so we shouldn't be causing any additional snapshots. + []uint64{1, 5, 5}, 100, 2, 2, 0, + "truncate 0 entries to first index 2 (chosen via: first index)", + }, + { + // The happy case. + []uint64{5, 5, 5}, 100, 2, 5, 0, + "truncate 3 entries to first index 5 (chosen via: quorum)", + }, + { + // No truncation, but the outstanding snapshot is made obsolete by the truncation. However + // it was already obsolete before. (This example is also not one you could manufacture in + // a real system). + []uint64{5, 5, 5}, 100, 2, 2, 1, + "truncate 0 entries to first index 2 (chosen via: first index)", + }, + { + // Respecting the pending snapshot. + []uint64{5, 5, 5}, 100, 2, 5, 3, + "truncate 1 entries to first index 3 (chosen via: pending snapshot)", + }, + { + // Log is below target size, so respecting the slowest follower. + []uint64{1, 2, 3, 4}, 100, 1, 5, 0, + "truncate 0 entries to first index 1 (chosen via: followers)", + }, + { + // Truncating since local log starts at 2. One follower is already cut off without a pending + // snapshot. + []uint64{1, 2, 3, 4}, 100, 2, 2, 0, + "truncate 0 entries to first index 2 (chosen via: first index)", + }, + // If over targetSize, should truncate to quorum committed index. Minority will need snapshots. + { + []uint64{1, 3, 3, 4}, 2000, 1, 3, 0, + "truncate 2 entries to first index 3 (chosen via: quorum); log too large (2.0 KiB > 1000 B); implies 1 Raft snapshot", + }, + { + []uint64{100, 100}, 2000, 1, 100, 50, + "truncate 99 entries to first index 100 (chosen via: quorum); log too large (2.0 KiB > 1000 B); implies 1 Raft snapshot", + }, + { + []uint64{1, 3, 3, 4}, 2000, 2, 3, 0, + "truncate 1 entries to first index 3 (chosen via: quorum); log too large (2.0 KiB > 1000 B)", + }, + { + []uint64{1, 3, 3, 4}, 2000, 3, 3, 0, + "truncate 0 entries to first index 3 (chosen via: quorum); log too large (2.0 KiB > 1000 B)", + }, // The pending snapshot index affects the quorum commit index. - {[]uint64{4}, 2000, 1, 1, 1, 1}, + { + []uint64{4}, 2000, 1, 7, 1, + "truncate 0 entries to first index 1 (chosen via: quorum); log too large (2.0 KiB > 1000 B)", + }, // Never truncate past the quorum commit index. - {[]uint64{3, 3, 6}, 100, 4, 4, 0, 3}, + { + []uint64{3, 3, 6}, 100, 2, 7, 0, + "truncate 1 entries to first index 3 (chosen via: quorum)", + }, // Never truncate past the last index. - {[]uint64{5}, 100, 1, 3, 0, 3}, - } + { + []uint64{5}, 100, 1, 3, 0, + "truncate 2 entries to first index 3 (chosen via: last index)", + }, + // Never truncate "before the first index". + { + []uint64{5}, 100, 2, 3, 1, + "truncate 0 entries to first index 2 (chosen via: first index)", + }} for i, c := range testCases { status := &raft.Status{ Progress: make(map[uint64]raft.Progress), @@ -135,10 +200,16 @@ func TestComputeTruncatableIndex(t *testing.T) { for j, v := range c.progress { status.Progress[uint64(j)] = raft.Progress{Match: v} } - out := computeTruncatableIndex(status, c.raftLogSize, targetSize, - c.firstIndex, c.lastIndex, c.pendingSnapshot) - if !reflect.DeepEqual(c.expected, out) { - t.Errorf("%d: computeTruncatableIndex(...) expected %d, but got %d", i, c.expected, out) + decision := computeTruncateDecision(truncateDecisionInput{ + RaftStatus: status, + LogSize: c.raftLogSize, + MaxLogSize: targetSize, + FirstIndex: c.firstIndex, + LastIndex: c.lastIndex, + PendingPreemptiveSnapshotIndex: c.pendingSnapshot, + }) + if act, exp := decision.String(), c.exp; act != exp { + t.Errorf("%d: got:\n%s\nwanted:\n%s", i, act, exp) } } } @@ -165,9 +236,9 @@ func verifyLogSizeInSync(t *testing.T, r *Replica) { } } -// TestGetTruncatableIndexes verifies that old raft log entries are correctly +// TestNewTruncateDecision verifies that old raft log entries are correctly // removed. -func TestGetTruncatableIndexes(t *testing.T) { +func TestNewTruncateDecision(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) @@ -179,18 +250,12 @@ func TestGetTruncatableIndexes(t *testing.T) { t.Fatal(err) } - getIndexes := func() (uint64, uint64, uint64, error) { - r.mu.Lock() - firstIndex, err := r.raftFirstIndexLocked() - r.mu.Unlock() - if err != nil { - return 0, 0, 0, err - } - truncatableIndexes, oldestIndex, _, err := getTruncatableIndexes(context.Background(), r) + getIndexes := func() (uint64, int, uint64, error) { + d, err := newTruncateDecision(context.Background(), r) if err != nil { return 0, 0, 0, err } - return firstIndex, truncatableIndexes, oldestIndex, nil + return d.Input.FirstIndex, d.NumTruncatableIndexes(), d.NewFirstIndex, nil } aFirst, aTruncatable, aOldest, err := getIndexes() @@ -215,13 +280,13 @@ func TestGetTruncatableIndexes(t *testing.T) { t.Fatal(err) } if aFirst != bFirst { - t.Errorf("expected firstIndex to not change, instead it changed from %d -> %d", aFirst, bFirst) + t.Fatalf("expected firstIndex to not change, instead it changed from %d -> %d", aFirst, bFirst) } if aTruncatable >= bTruncatable { - t.Errorf("expected truncatableIndexes to increase, instead it changed from %d -> %d", aTruncatable, bTruncatable) + t.Fatalf("expected truncatableIndexes to increase, instead it changed from %d -> %d", aTruncatable, bTruncatable) } if aOldest >= bOldest { - t.Errorf("expected oldestIndex to increase, instead it changed from %d -> %d", aOldest, bOldest) + t.Fatalf("expected oldestIndex to increase, instead it changed from %d -> %d", aOldest, bOldest) } // Enable the raft log scanner and and force a truncation. @@ -231,10 +296,11 @@ func TestGetTruncatableIndexes(t *testing.T) { // There can be a delay from when the truncation command is issued and the // indexes updating. - var cFirst, cTruncatable, cOldest uint64 + var cFirst, cOldest uint64 + var numTruncatable int testutils.SucceedsSoon(t, func() error { var err error - cFirst, cTruncatable, cOldest, err = getIndexes() + cFirst, numTruncatable, cOldest, err = getIndexes() if err != nil { t.Fatal(err) } @@ -243,8 +309,8 @@ func TestGetTruncatableIndexes(t *testing.T) { } return nil }) - if bTruncatable < cTruncatable { - t.Errorf("expected truncatableIndexes to decrease, instead it changed from %d -> %d", bTruncatable, cTruncatable) + if bTruncatable < numTruncatable { + t.Errorf("expected numTruncatable to decrease, instead it changed from %d -> %d", bTruncatable, numTruncatable) } if bOldest >= cOldest { t.Errorf("expected oldestIndex to increase, instead it changed from %d -> %d", bOldest, cOldest) @@ -261,7 +327,7 @@ func TestGetTruncatableIndexes(t *testing.T) { // Unlike the last iteration, where we expect a truncation and can wait on // it with succeedsSoon, we can't do that here. This check is fragile in // that the truncation triggered here may lose the race against the call to - // GetFirstIndex or getTruncatableIndexes, giving a false negative. Fixing + // GetFirstIndex or newTruncateDecision, giving a false negative. Fixing // this requires additional instrumentation of the queues, which was deemed // to require too much work at the time of this writing. dFirst, dTruncatable, dOldest, err := getIndexes() @@ -271,8 +337,8 @@ func TestGetTruncatableIndexes(t *testing.T) { if cFirst != dFirst { t.Errorf("truncation should not have occurred, but firstIndex changed from %d -> %d", cFirst, dFirst) } - if cTruncatable != dTruncatable { - t.Errorf("truncation should not have occurred, but truncatableIndexes changed from %d -> %d", cTruncatable, dTruncatable) + if numTruncatable != dTruncatable { + t.Errorf("truncation should not have occurred, but truncatableIndexes changed from %d -> %d", numTruncatable, dTruncatable) } if cOldest != dOldest { t.Errorf("truncation should not have occurred, but oldestIndex changed from %d -> %d", cOldest, dOldest)