From 22762a7d1c52f863fdada8b49a6132484e87980f Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 15 Feb 2023 15:30:17 +0100 Subject: [PATCH 1/2] scripts: gceworker: don't anchor absolute paths at repo in get Now one may `./scripts/gceworker.ssh get /home/foo bar`. Previously it would try to copy from `go/src/github.com/cockroachdb/cockroach/home/foo/bar`. Epic: none Release note: None --- scripts/gceworker.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/scripts/gceworker.sh b/scripts/gceworker.sh index d89899df2fc5..e84caa3d957f 100755 --- a/scripts/gceworker.sh +++ b/scripts/gceworker.sh @@ -151,7 +151,13 @@ case "${cmd}" in gcloud "$@" ;; get) - from="${NAME}:go/src/github.com/cockroachdb/cockroach/${1}" + rpath="${1}" + # Check whether we have an absolute path like /foo, ~foo, or ~/foo. + # If not, base the path relative to the CRDB repo. + if [[ "${rpath:0:1}" != / && "${rpath:0:2}" != ~[/a-z] ]]; then + rpath="go/src/github.com/cockroachdb/cockroach/${rpath}" + fi + from="${NAME}:${rpath}" shift gcloud compute scp --recurse "${from}" "$@" ;; From 76afb004ce71fac3c2bb19f33ef8c247e2775211 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 17 Feb 2023 10:38:41 +0000 Subject: [PATCH 2/2] kvserver: allow expired leases to quiesce Previously, a range would only quiesce if it had a valid leaseholder colocated with the Raft leader. However, there's no reason to not quiesce expired leases as well -- in particular, if we enable use of expiration-based leases for all ranges, idle ranges would let their leases expire but wouldn't then be eligible for quiesence. This patch allows ranges with expired leases to quiesce. A replica only quiesces if it believes itself to be the leader and has no unapplied log entries, in which case there's no lease elsewhere. If there's a newer leader then only replicas still on the older term will be quiesced, and these will unquiesce when they hear from the new leader. It additionally allows quiescing in a couple of other cases where the replica has a current lease that it can't use (i.e. `UNUSABLE` and `PROSCRIBED`). Epic: none Release note: None --- pkg/kv/kvserver/replica_raft_quiesce.go | 36 ++++++++++-- pkg/kv/kvserver/replica_test.go | 76 ++++++++++++++++++++++--- 2 files changed, 99 insertions(+), 13 deletions(-) diff --git a/pkg/kv/kvserver/replica_raft_quiesce.go b/pkg/kv/kvserver/replica_raft_quiesce.go index 330db7cc4e8f..0dd07956fcdb 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -14,6 +14,7 @@ import ( "context" "sort" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -197,7 +198,8 @@ type quiescer interface { hasRaftReadyRLocked() bool hasPendingProposalsRLocked() bool hasPendingProposalQuotaRLocked() bool - ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool + leaseStatusAtRLocked(ctx context.Context, now hlc.ClockTimestamp) kvserverpb.LeaseStatus + StoreID() roachpb.StoreID mergeInProgressRLocked() bool isDestroyedRLocked() (DestroyReason, error) } @@ -337,15 +339,37 @@ func shouldReplicaQuiesce( } return nil, nil, false } - // Only quiesce if this replica is the leaseholder as well; - // otherwise the replica which is the valid leaseholder may have - // pending commands which it's waiting on this leader to propose. - if !q.ownsValidLeaseRLocked(ctx, now) { + + // Don't quiesce if there is a current leaseholder elsewhere. Otherwise, the + // leaseholder may have pending commands which it's waiting on this leader to + // propose. + // + // We allow quiescing with an expired lease (both expiration-based and + // epoch-based), since leases are not always eagerly renewed. This replica + // thinks it's the leader, and it checks that there are no unapplied entries, + // so there can't be a new leaseholder if that's still the case. If someone + // else recently acquired leadership then this replica would not be able to + // quiesce those followers, only itself and any stale followers, and it would + // unquiesce once it hears from the new leader. + st := q.leaseStatusAtRLocked(ctx, now) + switch st.State { + // Allow quiescing if the current lease is ours, even if we can't use it. + case kvserverpb.LeaseState_VALID, kvserverpb.LeaseState_UNUSABLE, kvserverpb.LeaseState_PROSCRIBED: + if !st.OwnedBy(q.StoreID()) { + if log.V(4) { + log.Infof(ctx, "not quiescing: not leaseholder") + } + return nil, nil, false + } + // Allow expired leases to quiesce. + case kvserverpb.LeaseState_EXPIRED: + default: if log.V(4) { - log.Infof(ctx, "not quiescing: not leaseholder") + log.Infof(ctx, "not quiescing: lease in state %s", st) } return nil, nil, false } + // We need all of Applied, Commit, LastIndex and Progress.Match indexes to be // equal in order to quiesce. if status.Applied != status.Commit { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 171d2a597f5e..32bcd3a9daeb 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -9924,7 +9924,8 @@ type testQuiescer struct { status *raftSparseStatus lastIndex uint64 raftReady bool - ownsValidLease bool + leaseStatus kvserverpb.LeaseStatus + storeID roachpb.StoreID mergeInProgress bool isDestroyed bool @@ -9965,8 +9966,14 @@ func (q *testQuiescer) hasPendingProposalQuotaRLocked() bool { return q.pendingQuota } -func (q *testQuiescer) ownsValidLeaseRLocked(context.Context, hlc.ClockTimestamp) bool { - return q.ownsValidLease +func (q *testQuiescer) leaseStatusAtRLocked( + ctx context.Context, now hlc.ClockTimestamp, +) kvserverpb.LeaseStatus { + return q.leaseStatus +} + +func (q *testQuiescer) StoreID() roachpb.StoreID { + return q.storeID } func (q *testQuiescer) mergeInProgressRLocked() bool { @@ -9992,6 +9999,7 @@ func TestShouldReplicaQuiesce(t *testing.T) { // true. The transform function is intended to perform one mutation to // this quiescer so that shouldReplicaQuiesce will return false. q := &testQuiescer{ + storeID: 1, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ {NodeID: 1, ReplicaID: 1}, @@ -10017,9 +10025,18 @@ func TestShouldReplicaQuiesce(t *testing.T) { 3: {Match: logIndex}, }, }, - lastIndex: logIndex, - raftReady: false, - ownsValidLease: true, + lastIndex: logIndex, + raftReady: false, + leaseStatus: kvserverpb.LeaseStatus{ + State: kvserverpb.LeaseState_VALID, + Lease: roachpb.Lease{ + Replica: roachpb.ReplicaDescriptor{ + NodeID: 1, + StoreID: 1, + ReplicaID: 1, + }, + }, + }, livenessMap: livenesspb.IsLiveMap{ 1: {IsLive: true}, 2: {IsLive: true}, @@ -10101,7 +10118,52 @@ func TestShouldReplicaQuiesce(t *testing.T) { return q }) test(false, func(q *testQuiescer) *testQuiescer { - q.ownsValidLease = false + q.leaseStatus.State = kvserverpb.LeaseState_ERROR + return q + }) + test(true, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.State = kvserverpb.LeaseState_VALID + return q + }) + test(true, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.State = kvserverpb.LeaseState_UNUSABLE + return q + }) + test(true, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.State = kvserverpb.LeaseState_EXPIRED + return q + }) + test(true, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.State = kvserverpb.LeaseState_PROSCRIBED + return q + }) + test(false, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.State = -99 + return q + }) + test(false, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.Lease.Replica.StoreID = 9 + q.leaseStatus.State = kvserverpb.LeaseState_ERROR + return q + }) + test(false, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.Lease.Replica.StoreID = 9 + q.leaseStatus.State = kvserverpb.LeaseState_VALID + return q + }) + test(false, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.Lease.Replica.StoreID = 9 + q.leaseStatus.State = kvserverpb.LeaseState_UNUSABLE + return q + }) + test(true, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.Lease.Replica.StoreID = 9 + q.leaseStatus.State = kvserverpb.LeaseState_EXPIRED + return q + }) + test(false, func(q *testQuiescer) *testQuiescer { + q.leaseStatus.Lease.Replica.StoreID = 9 + q.leaseStatus.State = kvserverpb.LeaseState_PROSCRIBED return q }) test(false, func(q *testQuiescer) *testQuiescer {