Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

kvserver: don't always refuse to take leases when draining #55624

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ type TestingKnobs struct {
DistSQL ModuleTestingKnobs
SQLEvalContext ModuleTestingKnobs
RegistryLiveness ModuleTestingKnobs
NodeLiveness ModuleTestingKnobs
Server ModuleTestingKnobs
TenantTestingKnobs ModuleTestingKnobs
JobsTestingKnobs ModuleTestingKnobs
Expand Down
152 changes: 0 additions & 152 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1234,158 +1234,6 @@ func TestReplicateAfterRemoveAndSplit(t *testing.T) {
mtc.stores[2].SetReplicaGCQueueActive(true)
}

// Test various mechanism for refreshing pending commands.
func TestRefreshPendingCommands(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// In this scenario, three different mechanisms detect the need to repropose
// commands. Test that each one is sufficient individually. We have this
// redundancy because some mechanisms respond with lower latency than others,
// but each has some scenarios (not currently tested) in which it is
// insufficient on its own. In addition, there is a fourth reproposal
// mechanism (reasonNewLeaderOrConfigChange) which is not relevant to this
// scenario.
//
// We don't test with only reasonNewLeader because that mechanism is less
// robust than refreshing due to snapshot or ticks. In particular, it is
// possible for node 3 to propose the RequestLease command and have that
// command executed by the other nodes but to never see the execution locally
// because it is caught up by applying a snapshot.
testCases := map[string]kvserver.StoreTestingKnobs{
"reasonSnapshotApplied": {
DisableRefreshReasonNewLeader: true,
DisableRefreshReasonTicks: true,
},
"reasonTicks": {
DisableRefreshReasonNewLeader: true,
DisableRefreshReasonSnapshotApplied: true,
},
}
for name, c := range testCases {
t.Run(name, func(t *testing.T) {
sc := kvserver.TestStoreConfig(nil)
sc.TestingKnobs = c
// Disable periodic gossip tasks which can move the range 1 lease
// unexpectedly.
sc.TestingKnobs.DisablePeriodicGossips = true
sc.Clock = nil // manual clock
mtc := &multiTestContext{
storeConfig: &sc,
// This test was written before the multiTestContext started creating
// many system ranges at startup, and hasn't been update to take that
// into account.
startWithSingleRange: true,
}
defer mtc.Stop()
mtc.Start(t, 3)

const rangeID = roachpb.RangeID(1)
mtc.replicateRange(rangeID, 1, 2)

// Put some data in the range so we'll have something to test for.
incArgs := incrementArgs([]byte("a"), 5)
if _, err := kv.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}

// Wait for all nodes to catch up.
mtc.waitForValues(roachpb.Key("a"), []int64{5, 5, 5})

// Stop node 2; while it is down write some more data.
mtc.stopStore(2)

if _, err := kv.SendWrapped(context.Background(), mtc.stores[0].TestSender(), incArgs); err != nil {
t.Fatal(err)
}

// Get the last increment's log index.
repl, err := mtc.stores[0].GetReplica(1)
if err != nil {
t.Fatal(err)
}
index, err := repl.GetLastIndex()
if err != nil {
t.Fatal(err)
}

// Truncate the log at index+1 (log entries < N are removed, so this includes
// the increment).
truncArgs := truncateLogArgs(index+1, rangeID)
if _, err := kv.SendWrapped(context.Background(), mtc.stores[0].TestSender(), truncArgs); err != nil {
t.Fatal(err)
}

// Stop and restart node 0 in order to make sure that any in-flight Raft
// messages have been sent.
mtc.stopStore(0)
mtc.restartStore(0)

////////////////////////////////////////////////////////////////////
// We want store 2 to take the lease later, so we'll drain the other
// stores and expire the lease.
////////////////////////////////////////////////////////////////////

// Disable node liveness heartbeats which can reacquire leases when we're
// trying to expire them. We pause liveness heartbeats here after node 0
// was restarted (which creates a new NodeLiveness).
pauseNodeLivenessHeartbeatLoops(mtc)

// Start draining stores 0 and 1 to prevent them from grabbing any new
// leases.
mtc.advanceClock(context.Background())
var wg sync.WaitGroup
for i := 0; i < 2; i++ {
wg.Add(1)
go func(i int) {
mtc.stores[i].SetDraining(true, nil /* reporter */)
wg.Done()
}(i)
}

// Wait for the stores 0 and 1 to have entered draining mode, and then
// advance the clock. Advancing the clock will leave the liveness records
// of draining nodes in an expired state, so the SetDraining() call above
// will be able to terminate.
draining := false
for !draining {
draining = true
for i := 0; i < 2; i++ {
draining = draining && mtc.stores[i].IsDraining()
}
// Allow this loop to be preempted. Failure to do so can cause a
// deadlock because a non-preemptible loop will prevent GC from
// starting which in turn will cause all other goroutines to be stuck
// as soon as they are called on to assist the GC (this shows up as
// goroutines stuck in "GC assist wait"). With all of the other
// goroutines stuck, nothing will be able to set mtc.stores[i].draining
// to true.
//
// See #18554.
runtime.Gosched()
}
mtc.advanceClock(context.Background())

wg.Wait()

// Restart node 2 and wait for the snapshot to be applied. Note that
// waitForValues reads directly from the engine and thus isn't executing
// a Raft command.
mtc.restartStore(2)
mtc.waitForValues(roachpb.Key("a"), []int64{10, 10, 10})

// Send an increment to the restarted node. If we don't refresh pending
// commands appropriately, the range lease command will not get
// re-proposed when we discover the new leader.
if _, err := kv.SendWrapped(context.Background(), mtc.stores[2].TestSender(), incArgs); err != nil {
t.Fatal(err)
}

mtc.waitForValues(roachpb.Key("a"), []int64{15, 15, 15})
})
}
}

// Test that when a Raft group is not able to establish a quorum, its Raft log
// does not grow without bound. It tests two different scenarios where this used
// to be possible (see #27772):
Expand Down
20 changes: 18 additions & 2 deletions pkg/kv/kvserver/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/opentracing/opentracing-go"
"go.etcd.io/etcd/raft"
)

var leaseStatusLogLimiter = log.Every(5 * time.Second)
Expand Down Expand Up @@ -583,6 +584,15 @@ func (r *Replica) leaseStatus(
return status
}

// currentLeaseStatus returns the status of the current lease for a current
// timestamp.
func (r *Replica) currentLeaseStatus(ctx context.Context) kvserverpb.LeaseStatus {
timestamp := r.store.Clock().Now()
r.mu.RLock()
defer r.mu.RUnlock()
return r.leaseStatus(ctx, *r.mu.state.Lease, timestamp, r.mu.minLeaseProposedTS)
}

// requiresExpiringLeaseRLocked returns whether this range uses an
// expiration-based lease; false if epoch-based. Ranges located before or
// including the node liveness table must use expiration leases to avoid
Expand Down Expand Up @@ -616,8 +626,14 @@ func (r *Replica) requestLeaseLocked(
return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(
newNotLeaseHolderError(&transferLease, r.store.StoreID(), r.mu.state.Desc)))
}
if r.store.IsDraining() {
// We've retired from active duty.
// If we're draining, we'd rather not take any new leases (since we're also
// trying to move leases away elsewhere). But if we're the leader, we don't
// really have a choice and we take the lease - there might not be any other
// replica available to take this lease (perhaps they're all draining).
if r.store.IsDraining() && (r.raftBasicStatusRLocked().RaftState != raft.StateLeader) {
// TODO(andrei): If we start refusing to take leases on followers elsewhere,
// this code can go away.
log.VEventf(ctx, 2, "refusing to take the lease because we're draining")
return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(
newNotLeaseHolderError(nil, r.store.StoreID(), r.mu.state.Desc)))
}
Expand Down
120 changes: 96 additions & 24 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1428,36 +1428,108 @@ func TestReplicaLeaseRejectUnknownRaftNodeID(t *testing.T) {
}
}

// TestReplicaDrainLease makes sure that no new leases are granted when
// the Store is draining.
// Test that draining nodes only take the lease if they're the leader.
func TestReplicaDrainLease(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
tc := testContext{}
stopper := stop.NewStopper()
defer stopper.Stop(context.Background())
tc.Start(t, stopper)

// Acquire initial lease.
ctx := context.Background()
status, pErr := tc.repl.redirectOnOrAcquireLease(ctx)
if pErr != nil {
t.Fatal(pErr)
clusterArgs := base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
NodeLiveness: NodeLivenessTestingKnobs{
// This test waits for an epoch-based lease to expire, so we're setting the
// liveness duration as low as possible while still keeping the test stable.
LivenessDuration: 2000 * time.Millisecond,
RenewalDuration: 1000 * time.Millisecond,
},
Store: &StoreTestingKnobs{
// We eliminate clock offsets in order to eliminate the stasis period of
// leases. Otherwise we'd need to make leases longer.
MaxOffset: time.Nanosecond,
},
},
},
}
tc := serverutils.StartNewTestCluster(t, 2, clusterArgs)
defer tc.Stopper().Stop(ctx)
rngKey := tc.ScratchRange(t)
tc.AddReplicasOrFatal(t, rngKey, tc.Target(1))

tc.store.SetDraining(true, nil /* reporter */)
tc.repl.mu.Lock()
pErr = <-tc.repl.requestLeaseLocked(ctx, status).C()
tc.repl.mu.Unlock()
_, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError)
if !ok {
t.Fatalf("expected NotLeaseHolderError, not %v", pErr)
}
tc.store.SetDraining(false, nil /* reporter */)
// Newly undrained, leases work again.
if _, pErr := tc.repl.redirectOnOrAcquireLease(ctx); pErr != nil {
t.Fatal(pErr)
}
s1 := tc.Server(0)
s2 := tc.Server(1)
store1, err := s1.GetStores().(*Stores).GetStore(s1.GetFirstStoreID())
require.NoError(t, err)
store2, err := s2.GetStores().(*Stores).GetStore(s2.GetFirstStoreID())
require.NoError(t, err)

rd := tc.LookupRangeOrFatal(t, rngKey)
r1, err := store1.GetReplica(rd.RangeID)
require.NoError(t, err)
status := r1.currentLeaseStatus(ctx)
require.True(t, status.Lease.OwnedBy(store1.StoreID()), "someone else got the lease: %s", status)
// We expect the lease to be valid, but don't check that because, under race, it might have
// expired already.

// Stop n1's heartbeats and wait for the lease to expire.

log.Infof(ctx, "test: suspending heartbeats for n1")
cleanup := s1.NodeLiveness().(*NodeLiveness).PauseAllHeartbeatsForTest()
defer cleanup()

require.NoError(t, err)
testutils.SucceedsSoon(t, func() error {
status := r1.currentLeaseStatus(ctx)
require.True(t, status.Lease.OwnedBy(store1.StoreID()), "someone else got the lease: %s", status)
if status.State == kvserverpb.LeaseState_VALID {
return errors.New("lease still valid")
}
// We need to wait for the stasis state to pass too; during stasis other
// replicas can't take the lease.
if status.State == kvserverpb.LeaseState_STASIS {
return errors.New("lease still in stasis")
}
return nil
})

require.Equal(t, r1.RaftStatus().Lead, uint64(r1.ReplicaID()),
"expected leadership to still be on the first replica")

// Mark the stores as draining. We'll then start checking how acquiring leases
// behaves while draining.
store1.draining.Store(true)
store2.draining.Store(true)

r2, err := store2.GetReplica(rd.RangeID)
require.NoError(t, err)
// Check that a draining replica that's not the leader does NOT take the
// lease.
_, pErr := r2.redirectOnOrAcquireLease(ctx)
require.NotNil(t, pErr)
require.IsType(t, &roachpb.NotLeaseHolderError{}, pErr.GetDetail())

// Now transfer the leadership from r1 to r2 and check that r1 can now acquire
// the lease.

// Initiate the leadership transfer.
r1.mu.Lock()
r1.mu.internalRaftGroup.TransferLeader(uint64(r2.ReplicaID()))
r1.mu.Unlock()
// Run the range through the Raft scheduler, otherwise the leadership messages
// doesn't get sent because the range is quiesced.
store1.EnqueueRaftUpdateCheck(r1.RangeID)

// Wait for the leadership transfer to happen.
testutils.SucceedsSoon(t, func() error {
if r2.RaftStatus().SoftState.RaftState != raft.StateLeader {
return errors.Newf("r1 not yet leader")
}
return nil
})

// Check that r2 can now acquire the lease.
_, pErr = r2.redirectOnOrAcquireLease(ctx)
require.NoError(t, pErr.GoError())
}

// TestReplicaGossipFirstRange verifies that the first range gossips its
Expand Down
17 changes: 17 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package kvserver
import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
Expand Down Expand Up @@ -250,3 +251,19 @@ type StoreTestingKnobs struct {

// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
func (*StoreTestingKnobs) ModuleTestingKnobs() {}

// NodeLivenessTestingKnobs allows tests to override some node liveness
// controls. When set, fields ultimately affect the NodeLivenessOptions used by
// the cluster.
type NodeLivenessTestingKnobs struct {
// LivenessDuration overrides a liveness record's life time.
LivenessDuration time.Duration
// RenewalDuration specifies how long before the expiration a record is
// heartbeated. If LivenessDuration is set, this should probably be set too.
RenewalDuration time.Duration
}

var _ base.ModuleTestingKnobs = NodeLivenessTestingKnobs{}

// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface.
func (NodeLivenessTestingKnobs) ModuleTestingKnobs() {}
9 changes: 9 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,15 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
db := kv.NewDBWithContext(cfg.AmbientCtx, tcsFactory, clock, dbCtx)

nlActive, nlRenewal := cfg.NodeLivenessDurations()
if knobs := cfg.TestingKnobs.NodeLiveness; knobs != nil {
nlKnobs := knobs.(kvserver.NodeLivenessTestingKnobs)
if duration := nlKnobs.LivenessDuration; duration != 0 {
nlActive = duration
}
if duration := nlKnobs.RenewalDuration; duration != 0 {
nlRenewal = duration
}
}

nodeLiveness := kvserver.NewNodeLiveness(kvserver.NodeLivenessOptions{
AmbientCtx: cfg.AmbientCtx,
Expand Down
5 changes: 5 additions & 0 deletions pkg/testutils/serverutils/test_cluster_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,11 @@ type TestClusterInterface interface {
// ReplicationMode returns the ReplicationMode that the test cluster was
// configured with.
ReplicationMode() base.TestClusterReplicationMode

// ScratchRange returns the start key of a span of keyspace suitable for use
// as kv scratch space (it doesn't overlap system spans or SQL tables). The
// range is lazily split off on the first call to ScratchRange.
ScratchRange(t testing.TB) roachpb.Key
}

// TestClusterFactory encompasses the actual implementation of the shim
Expand Down