Skip to content

Commit

Permalink
kv: Replace multiTestContext with TestCluster in consistency_queue_test
Browse files Browse the repository at this point in the history
Makes progress on cockroachdb#8299

multiTestContext is legacy construct that is deprecated in favor of running tests via TestCluster. This is one PR out of many to remove the usage of multiTestContext in the consistency_queue test cases.

Release note : None
  • Loading branch information
lunevalex committed Jun 23, 2020
1 parent 264ab25 commit 7eeb6db
Show file tree
Hide file tree
Showing 3 changed files with 195 additions and 123 deletions.
57 changes: 43 additions & 14 deletions pkg/kv/kvserver/consistency_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,15 @@ type consistencyQueue struct {
replicaCountFn func() int
}

// A data wrapper to allow for the shouldQueue method to be easier to test
type consistencyShouldQueueData struct {
desc *roachpb.RangeDescriptor
getQueueLastProcessed func(ctx context.Context) (hlc.Timestamp, error)
isNodeLive func(nodeID roachpb.NodeID) (bool, error)
disableLastProcessedCheck bool
interval time.Duration
}

// newConsistencyQueue returns a new instance of consistencyQueue.
func newConsistencyQueue(store *Store, gossip *gossip.Gossip) *consistencyQueue {
q := &consistencyQueue{
Expand Down Expand Up @@ -77,30 +86,50 @@ func newConsistencyQueue(store *Store, gossip *gossip.Gossip) *consistencyQueue
func (q *consistencyQueue) shouldQueue(
ctx context.Context, now hlc.Timestamp, repl *Replica, _ *config.SystemConfig,
) (bool, float64) {
interval := q.interval()
if interval <= 0 {
return consistencyQueueShouldQueueImpl(ctx, now,
consistencyShouldQueueData{
desc: repl.Desc(),
getQueueLastProcessed: func(ctx context.Context) (hlc.Timestamp, error) {
return repl.getQueueLastProcessed(ctx, q.name)
},
isNodeLive: func(nodeID roachpb.NodeID) (bool, error) {
if repl.store.cfg.NodeLiveness != nil {
return repl.store.cfg.NodeLiveness.IsLive(nodeID)
}
// Some tests run without a NodeLiveness configured.
return true, nil
},
disableLastProcessedCheck: repl.store.cfg.TestingKnobs.DisableLastProcessedCheck,
interval: q.interval(),
})
}

// ConsistencyQueueShouldQueueImpl is exposed for testability without having
// to setup a fully fledged replica.
func consistencyQueueShouldQueueImpl(
ctx context.Context, now hlc.Timestamp, data consistencyShouldQueueData,
) (bool, float64) {
if data.interval <= 0 {
return false, 0
}

shouldQ, priority := true, float64(0)
if !repl.store.cfg.TestingKnobs.DisableLastProcessedCheck {
lpTS, err := repl.getQueueLastProcessed(ctx, q.name)
if !data.disableLastProcessedCheck {
lpTS, err := data.getQueueLastProcessed(ctx)
if err != nil {
return false, 0
}
if shouldQ, priority = shouldQueueAgain(now, lpTS, interval); !shouldQ {
if shouldQ, priority = shouldQueueAgain(now, lpTS, data.interval); !shouldQ {
return false, 0
}
}
// Check if all replicas are live. Some tests run without a NodeLiveness configured.
if repl.store.cfg.NodeLiveness != nil {
for _, rep := range repl.Desc().Replicas().All() {
if live, err := repl.store.cfg.NodeLiveness.IsLive(rep.NodeID); err != nil {
log.VErrEventf(ctx, 3, "node %d liveness failed: %s", rep.NodeID, err)
return false, 0
} else if !live {
return false, 0
}
// Check if all replicas are live.
for _, rep := range data.desc.Replicas().All() {
if live, err := data.isNodeLive(rep.NodeID); err != nil {
log.VErrEventf(ctx, 3, "node %d liveness failed: %s", rep.NodeID, err)
return false, 0
} else if !live {
return false, 0
}
}
return true, priority
Expand Down
Loading

0 comments on commit 7eeb6db

Please sign in to comment.