From 892d5b4e9804f28140e21c3c6a2edd66cec5c7b7 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 8 Aug 2022 14:46:29 +0200 Subject: [PATCH] kvserver: simplify and track entire set of gossiped IOThresholds This commit makes the following changes: - track *all* IOThresholds in the store's map, not just the ones for overloaded stores. - improve the container for these IOThresholds to be easier to work with. - Rather than "hard-coding" a value of "1.0" to mean overloaded, use (and plumb) the value of the cluster setting. "1.0" is the value at which I/O admission control chooses to engage; but the cluster setting is usually smaller and determines when to consider followers on a remote store pausable. The API now reflects that and avoids this kind of confusion. - Rename all uses of the container away from "overload" towards "IOThreshold". - add a Sequence() method that is bumped whenever the set of Stores whose IOThreshold score indicates pausability changes. I originally started to work on this to address #84465, but realized that we couldn't "just" leave the set of paused followers untouched absent sequence changes. This is because the set of paused followers has additional inputs, most importantly the set of live followers. This set is per-Replica and subject to change, so we can't be too sure the outcome would be the same, and we do want to be reactive to followers becoming nonresponsive by, if necessary, unpausing followers. I think we will have to address #84465 by reducing the frequency at which the paused stores are revisited, but adding an eager pass whenever the sequence is bumped. Additionally, for #84252, we are likely also going to be able to rely on the sequence number to trigger unquiescing of ranges that were previously quiesced in the presence of a paused follower. Regardless of these future possible uses, this is a nice conceptual clean-up and a good last PR to land for pausing in the 22.2 cycle (unless we find time to add quiescence in presence of paused followers, in which case that would be worthy follow-up). I verified that with this commit, the [roachtest] still works and effectively avoids I/O admission control activation a large percentage of the time at a setting of 0.8. This gives good confidence - at least for this exact test - that with 0.5 we'd probably never see admission control throttle foreground writes. However, the test is fairly specific since it severely constrains n3's disk throughput, so 0.8 might be perfectly appropriate in practice still. We'll need some more experience to tell. [roachtest]: https://github.com/cockroachdb/cockroach/pull/81516 Touches https://github.com/cockroachdb/cockroach/issues/84465. Touches https://github.com/cockroachdb/cockroach/issues/84252. Release note: None --- pkg/kv/kvserver/replica.go | 2 +- pkg/kv/kvserver/replica_command.go | 7 +- pkg/kv/kvserver/replica_raft.go | 7 +- pkg/kv/kvserver/replica_raft_overload.go | 144 +++++++++++++++--- pkg/kv/kvserver/replica_raft_overload_test.go | 31 +++- pkg/kv/kvserver/replica_test.go | 4 +- pkg/kv/kvserver/store.go | 21 +-- pkg/kv/kvserver/store_raft.go | 32 ++-- pkg/kv/kvserver/testdata/io_threshold_map.txt | 6 + .../admission/admissionpb/io_threshold.go | 10 +- 10 files changed, 196 insertions(+), 68 deletions(-) create mode 100644 pkg/kv/kvserver/testdata/io_threshold_map.txt diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 2862b5ace9c8..1ce438c56de5 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -636,7 +636,7 @@ type Replica struct { // Followers to which replication traffic is currently dropped. // // Never mutated in place (always replaced wholesale), so can be leaked - // outside of the surrounding mutex. + // outside the surrounding mutex. pausedFollowers map[roachpb.ReplicaID]struct{} } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index f48c2423dd6b..9adc86ffc2c0 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2601,12 +2601,15 @@ func (r *Replica) sendSnapshot( return err } - if ioThresh := r.store.ioOverloadedStores.Load()[recipient.StoreID]; ioThresh != nil && destPaused { + if destPaused { // If the destination is paused, be more hesitant to send snapshots. The destination being // paused implies that we have recently checked that it's not required for quorum, and that // we wish to conserve I/O on that store, which sending a snapshot counteracts. So hold back on // the snapshot as well. - return errors.Errorf("skipping snapshot; %s is overloaded: %s", recipient, ioThresh) + return errors.Errorf( + "skipping snapshot; %s is overloaded: %s", + recipient, r.store.ioThresholds.Current().IOThreshold(recipient.StoreID), + ) } // Check follower snapshots cluster setting. diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index e3675b20ce92..6aa1d0f75b9d 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -1164,9 +1163,7 @@ func maybeFatalOnRaftReadyErr(ctx context.Context, expl string, err error) (remo // tick the Raft group, returning true if the raft group exists and should // be queued for Ready processing; false otherwise. func (r *Replica) tick( - ctx context.Context, - livenessMap liveness.IsLiveMap, - ioOverloadMap map[roachpb.StoreID]*admissionpb.IOThreshold, + ctx context.Context, livenessMap liveness.IsLiveMap, ioThresholdMap *ioThresholdMap, ) (bool, error) { r.unreachablesMu.Lock() remotes := r.unreachablesMu.remotes @@ -1191,7 +1188,7 @@ func (r *Replica) tick( return false, nil } - r.updatePausedFollowersLocked(ctx, ioOverloadMap) + r.updatePausedFollowersLocked(ctx, ioThresholdMap) now := r.store.Clock().NowAsClockTimestamp() if r.maybeQuiesceRaftMuLockedReplicaMuLocked(ctx, now, livenessMap) { diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index 003ede45322e..18b1b04ef729 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -14,12 +14,13 @@ import ( "context" "math/rand" "sort" - "sync/atomic" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" "go.etcd.io/etcd/raft/v3/tracker" ) @@ -27,9 +28,9 @@ var pauseReplicationIOThreshold = settings.RegisterFloatSetting( settings.SystemOnly, "admission.kv.pause_replication_io_threshold", "pause replication to non-essential followers when their I/O admission control score exceeds the given threshold (zero to disable)", - // TODO(tbg): set a nonzero default. + // TODO(tbg): set a sub-one default. // See: https://github.com/cockroachdb/cockroach/issues/83920 - 0.0, + 10000, func(v float64) error { if v == 0 { return nil @@ -42,12 +43,16 @@ var pauseReplicationIOThreshold = settings.RegisterFloatSetting( }, ) +type ioThresholdMapI interface { + // AbovePauseThreshold returns true if the store's score exceeds the threshold + // set for trying to pause replication traffic to followers on it. + AbovePauseThreshold(_ roachpb.StoreID) bool +} + type computeExpendableOverloadedFollowersInput struct { - self roachpb.ReplicaID - replDescs roachpb.ReplicaSet - // TODO(tbg): all entries are overloaded, so consdier removing the IOThreshold here - // because it's confusing. - ioOverloadMap map[roachpb.StoreID]*admissionpb.IOThreshold + self roachpb.ReplicaID + replDescs roachpb.ReplicaSet + ioOverloadMap ioThresholdMapI // getProgressMap returns Raft's view of the progress map. This is only called // when needed, and at most once. getProgressMap func(context.Context) map[uint64]tracker.Progress @@ -86,7 +91,7 @@ const ( // overload), this method does very little work. // // If at least one follower is (close to being) overloaded, we determine the -// maximum set of such followers that we can afford not replicating to without +// maximum set of such followers that we can afford not to replicate to without // losing quorum by successively reducing the set of overloaded followers by one // randomly selected overloaded voter. The randomness makes it more likely that // when there are multiple overloaded stores in the system that cannot be @@ -108,7 +113,7 @@ func computeExpendableOverloadedFollowers( var prs map[uint64]tracker.Progress for _, replDesc := range d.replDescs.AsProto() { - if _, overloaded := d.ioOverloadMap[replDesc.StoreID]; !overloaded || replDesc.ReplicaID == d.self { + if pausable := d.ioOverloadMap.AbovePauseThreshold(replDesc.StoreID); !pausable || replDesc.ReplicaID == d.self { continue } // There's at least one overloaded follower, so initialize @@ -193,26 +198,117 @@ func computeExpendableOverloadedFollowers( return liveOverloadedVoterCandidates, nonLive } -type overloadedStoresMap atomic.Value // map[roachpb.StoreID]*admissionpb.IOThreshold +type ioThresholdMap struct { + threshold float64 // threshold at which the score indicates pausability + seq int // bumped on creation if pausable set changed + m map[roachpb.StoreID]*admissionpb.IOThreshold +} + +func (osm ioThresholdMap) String() string { + return redact.StringWithoutMarkers(osm) +} -func (osm *overloadedStoresMap) Load() map[roachpb.StoreID]*admissionpb.IOThreshold { - v, _ := (*atomic.Value)(osm).Load().(map[roachpb.StoreID]*admissionpb.IOThreshold) - return v +var _ redact.SafeFormatter = (*ioThresholdMap)(nil) + +func (osm ioThresholdMap) SafeFormat(s redact.SafePrinter, verb rune) { + var sl []roachpb.StoreID + for id := range osm.m { + sl = append(sl, id) + } + sort.Slice(sl, func(i, j int) bool { + a, _ := osm.m[sl[i]].Score() + b, _ := osm.m[sl[j]].Score() + return a < b + }) + for i, id := range sl { + if i > 0 { + s.SafeString(", ") + } + s.Printf("s%d: %s", id, osm.m[id]) + } + if len(sl) > 0 { + s.Printf(" [pausable-threshold=%.2f]", osm.threshold) + } +} + +var _ ioThresholdMapI = (*ioThresholdMap)(nil) + +// AbovePauseThreshold implements ioThresholdMapI. +func (osm *ioThresholdMap) AbovePauseThreshold(id roachpb.StoreID) bool { + sc, _ := osm.m[id].Score() + return sc > osm.threshold +} + +func (osm *ioThresholdMap) NumAbovePauseThreshold() int { + var n int + for id := range osm.m { + if osm.AbovePauseThreshold(id) { + n++ + } + } + return n } -func (osm *overloadedStoresMap) Swap( - m map[roachpb.StoreID]*admissionpb.IOThreshold, -) map[roachpb.StoreID]*admissionpb.IOThreshold { - v, _ := (*atomic.Value)(osm).Swap(m).(map[roachpb.StoreID]*admissionpb.IOThreshold) - return v +func (osm *ioThresholdMap) IOThreshold(id roachpb.StoreID) *admissionpb.IOThreshold { + return osm.m[id] +} + +// Sequence allows distinguishing sets of overloaded stores. Whenever an +// ioThresholdMap is created, it inherits the sequence of its predecessor, +// incrementing only when the set of pausable stores has changed in the +// transition. +func (osm *ioThresholdMap) Sequence() int { + return osm.seq +} + +type ioThresholds struct { + mu struct { + syncutil.Mutex + inner *ioThresholdMap // always replaced wholesale, so can leak out of mu + } +} + +func (osm *ioThresholds) Current() *ioThresholdMap { + osm.mu.Lock() + defer osm.mu.Unlock() + return osm.mu.inner +} + +// Replace replaces the stored view of stores for which we track IOThresholds. +// If the set of overloaded stores (i.e. with a score of >= seqThreshold) +// changes in the process, the updated view will have an incremented Sequence(). +func (osm *ioThresholds) Replace( + m map[roachpb.StoreID]*admissionpb.IOThreshold, seqThreshold float64, +) (prev, cur *ioThresholdMap) { + osm.mu.Lock() + defer osm.mu.Unlock() + last := osm.mu.inner + if last == nil { + last = &ioThresholdMap{} + } + next := &ioThresholdMap{threshold: seqThreshold, seq: last.seq, m: m} + var delta int + for id := range last.m { + if last.AbovePauseThreshold(id) != next.AbovePauseThreshold(id) { + delta = 1 + break + } + } + for id := range next.m { + if last.AbovePauseThreshold(id) != next.AbovePauseThreshold(id) { + delta = 1 + break + } + } + next.seq += delta + osm.mu.inner = next + return last, next } -func (r *Replica) updatePausedFollowersLocked( - ctx context.Context, ioOverloadMap map[roachpb.StoreID]*admissionpb.IOThreshold, -) { +func (r *Replica) updatePausedFollowersLocked(ctx context.Context, ioThresholdMap *ioThresholdMap) { r.mu.pausedFollowers = nil - if len(ioOverloadMap) == 0 { + if ioThresholdMap.NumAbovePauseThreshold() == 0 { return } @@ -254,7 +350,7 @@ func (r *Replica) updatePausedFollowersLocked( d := computeExpendableOverloadedFollowersInput{ self: r.replicaID, replDescs: r.descRLocked().Replicas(), - ioOverloadMap: ioOverloadMap, + ioOverloadMap: ioThresholdMap, getProgressMap: func(_ context.Context) map[uint64]tracker.Progress { prs := r.mu.internalRaftGroup.Status().Progress updateRaftProgressFromActivity(ctx, prs, r.descRLocked().Replicas().AsProto(), func(id roachpb.ReplicaID) bool { diff --git a/pkg/kv/kvserver/replica_raft_overload_test.go b/pkg/kv/kvserver/replica_raft_overload_test.go index 842d08cd268e..8ded4543e94a 100644 --- a/pkg/kv/kvserver/replica_raft_overload_test.go +++ b/pkg/kv/kvserver/replica_raft_overload_test.go @@ -21,11 +21,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/v3/tracker" ) @@ -47,7 +49,7 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) var seed uint64 var replDescs roachpb.ReplicaSet var self roachpb.ReplicaID - ioOverloadMap := map[roachpb.StoreID]*admissionpb.IOThreshold{} + ioOverloadMap := &ioThresholdMap{threshold: 1.0, m: map[roachpb.StoreID]*admissionpb.IOThreshold{}} snapshotMap := map[roachpb.ReplicaID]struct{}{} downMap := map[roachpb.ReplicaID]struct{}{} match := map[roachpb.ReplicaID]uint64{} @@ -87,7 +89,7 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) } replDescs.AddReplica(replDesc) case "overloaded": - ioOverloadMap[roachpb.StoreID(id)] = &admissionpb.IOThreshold{ + ioOverloadMap.m[roachpb.StoreID(id)] = &admissionpb.IOThreshold{ L0NumSubLevels: 1000, L0NumSubLevelsThreshold: 20, L0NumFiles: 1, @@ -156,3 +158,28 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) }) }) } + +func TestIoThresholdMap_SafeFormat(t *testing.T) { + defer leaktest.AfterTest(t)() + m := ioThresholdMap{threshold: 0.8, seq: 1, m: map[roachpb.StoreID]*admissionpb.IOThreshold{ + 1: { // score 0.7 + L0NumSubLevels: 100, + L0NumSubLevelsThreshold: 1000, + L0NumFiles: 700, + L0NumFilesThreshold: 1000, + }, + 7: { // score 0.9 + L0NumSubLevels: 90, + L0NumSubLevelsThreshold: 100, + L0NumFiles: 100, + L0NumFilesThreshold: 1000, + }, + 9: { // score 1.1 + L0NumSubLevels: 110, + L0NumSubLevelsThreshold: 100, + L0NumFiles: 100, + L0NumFilesThreshold: 1000, + }, + }} + echotest.Require(t, string(redact.Sprint(m)), testutils.TestDataPath(t, "io_threshold_map.txt")) +} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 73bc379b1038..574db8d309fb 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -56,6 +56,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -8117,7 +8118,8 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { r.mu.Unlock() // Tick raft. - if _, err := r.tick(ctx, nil, nil); err != nil { + iot := ioThresholdMap{m: map[roachpb.StoreID]*admissionpb.IOThreshold{}} + if _, err := r.tick(ctx, nil, &iot); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index aa72c7f8c7af..cb5769755027 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -932,10 +932,10 @@ type Store struct { // liveness. It is updated periodically in raftTickLoop() // and reactively in nodeIsLiveCallback() on liveness updates. livenessMap atomic.Value - // ioOverloadedStores is analogous to livenessMap, but stores a - // map[StoreID]*IOThreshold. It is gossip-backed but is not updated + // ioThresholds is analogous to livenessMap, but stores the *IOThresholds for + // the stores in the cluster . It is gossip-backed but is not updated // reactively, i.e. will refresh on each tick loop iteration only. - ioOverloadedStores overloadedStoresMap + ioThresholds *ioThresholds // cachedCapacity caches information on store capacity to prevent // expensive recomputations in case leases or replicas are rapidly @@ -1177,13 +1177,16 @@ func NewStore( if !cfg.Valid() { log.Fatalf(ctx, "invalid store configuration: %+v", &cfg) } + iot := ioThresholds{} + iot.Replace(nil, 1.0) // init as empty s := &Store{ - cfg: cfg, - db: cfg.DB, // TODO(tschottdorf): remove redundancy. - engine: eng, - nodeDesc: nodeDesc, - metrics: newStoreMetrics(cfg.HistogramWindowInterval), - ctSender: cfg.ClosedTimestampSender, + cfg: cfg, + db: cfg.DB, // TODO(tschottdorf): remove redundancy. + engine: eng, + nodeDesc: nodeDesc, + metrics: newStoreMetrics(cfg.HistogramWindowInterval), + ctSender: cfg.ClosedTimestampSender, + ioThresholds: &iot, } s.ioThreshold.t = &admissionpb.IOThreshold{} if cfg.RPCContext != nil { diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 6cc38f1bdc1f..284e60d8c197 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -646,12 +646,12 @@ func (s *Store) processTick(_ context.Context, rangeID roachpb.RangeID) bool { return false } livenessMap, _ := s.livenessMap.Load().(liveness.IsLiveMap) - storeOverloadMap := s.ioOverloadedStores.Load() + ioThresholds := s.ioThresholds.Current() start := timeutil.Now() ctx := r.raftCtx - exists, err := r.tick(ctx, livenessMap, storeOverloadMap) + exists, err := r.tick(ctx, livenessMap, ioThresholds) if err != nil { log.Errorf(ctx, "%v", err) } @@ -741,7 +741,7 @@ func (s *Store) raftTickLoop(ctx context.Context) { if s.cfg.NodeLiveness != nil { s.updateLivenessMap() } - s.updateStoreOverloadMap() + s.updateIOThresholdMap() s.unquiescedReplicas.Lock() // Why do we bother to ever queue a Replica on the Raft scheduler for @@ -763,25 +763,19 @@ func (s *Store) raftTickLoop(ctx context.Context) { } } -var shouldLogStoreOverloadMap = log.Every(10 * time.Second) - -func (s *Store) updateStoreOverloadMap() { - storeOverloadMap := map[roachpb.StoreID]*admissionpb.IOThreshold{} - overloadThresh := pauseReplicationIOThreshold.Get(&s.cfg.Settings.SV) +func (s *Store) updateIOThresholdMap() { + ioThresholdMap := map[roachpb.StoreID]*admissionpb.IOThreshold{} for _, sd := range s.allocator.StorePool.GetStores() { - if score, _ := sd.Capacity.IOThreshold.Score(); overloadThresh != 0 && score > overloadThresh { - ioThreshold := sd.Capacity.IOThreshold // need a copy - storeOverloadMap[sd.StoreID] = &ioThreshold - } + ioThreshold := sd.Capacity.IOThreshold // need a copy + ioThresholdMap[sd.StoreID] = &ioThreshold } - old := s.ioOverloadedStores.Swap(storeOverloadMap) - // Consider logging if we're going from seeing overloaded stores to seeing no overloaded stores, or when - // there are still overloaded stores and we haven't logged for a while. - shouldLog := log.V(1) || - (len(old) > 0 && len(storeOverloadMap) == 0) || - (len(storeOverloadMap) > 0 && shouldLogStoreOverloadMap.ShouldLog()) + threshold := pauseReplicationIOThreshold.Get(&s.cfg.Settings.SV) + old, cur := s.ioThresholds.Replace(ioThresholdMap, threshold) + // Log whenever the set of overloaded stores changes. + shouldLog := log.V(1) || old.seq != cur.seq if shouldLog { - log.Infof(s.AnnotateCtx(context.Background()), "IO overloaded stores [threshold %.2f]: %+v (before: %+v)", overloadThresh, storeOverloadMap, old) + log.Infof( + s.AnnotateCtx(context.Background()), "pausable stores: %+v", cur) } } diff --git a/pkg/kv/kvserver/testdata/io_threshold_map.txt b/pkg/kv/kvserver/testdata/io_threshold_map.txt new file mode 100644 index 000000000000..34b113eddba7 --- /dev/null +++ b/pkg/kv/kvserver/testdata/io_threshold_map.txt @@ -0,0 +1,6 @@ +# Note that the [L0-overload] triggers on a threshold of 1.0; IO overload +# is different from being considered pausable. We generally want followers +# to be paused before they hit IO overload. +echo +---- +s1: 0.700, s7: 0.900, s9: 1.100[L0-overload] [pausable-threshold=0.80] diff --git a/pkg/util/admission/admissionpb/io_threshold.go b/pkg/util/admission/admissionpb/io_threshold.go index 2ed07df22064..ab088938aaca 100644 --- a/pkg/util/admission/admissionpb/io_threshold.go +++ b/pkg/util/admission/admissionpb/io_threshold.go @@ -31,8 +31,8 @@ import ( // to compactions falling behind (though that may change if we increase the // max number of compactions). And we will need to incorporate overload due to // disk bandwidth bottleneck. -func (iot IOThreshold) Score() (float64, bool) { - if iot == (IOThreshold{}) { +func (iot *IOThreshold) Score() (float64, bool) { + if iot == nil { return 0, false } f := math.Max( @@ -43,8 +43,8 @@ func (iot IOThreshold) Score() (float64, bool) { } // SafeFormat implements redact.SafeFormatter. -func (iot IOThreshold) SafeFormat(s interfaces.SafePrinter, _ rune) { - if iot == (IOThreshold{}) { +func (iot *IOThreshold) SafeFormat(s interfaces.SafePrinter, _ rune) { + if iot == nil { s.Printf("N/A") return } @@ -55,6 +55,6 @@ func (iot IOThreshold) SafeFormat(s interfaces.SafePrinter, _ rune) { } } -func (iot IOThreshold) String() string { +func (iot *IOThreshold) String() string { return redact.StringWithoutMarkers(iot) }