Skip to content

Commit

Permalink
Merge pull request #15830 from tamird/storage-fix-nullable
Browse files Browse the repository at this point in the history
storage: restore nullability to avoid upgrade woes
  • Loading branch information
tamird authored May 10, 2017
2 parents a7e6d87 + d0bd273 commit 0ba0af9
Show file tree
Hide file tree
Showing 11 changed files with 106 additions and 83 deletions.
2 changes: 1 addition & 1 deletion pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -441,7 +441,7 @@ func TestRangesResponse(t *testing.T) {
if len(ri.State.Desc.Replicas) != 1 || ri.State.Desc.Replicas[0] != expReplica {
t.Errorf("unexpected replica list %+v", ri.State.Desc.Replicas)
}
if ri.State.Lease == (roachpb.Lease{}) {
if ri.State.Lease == nil || *ri.State.Lease == (roachpb.Lease{}) {
t.Error("expected a nontrivial Lease")
}
if ri.State.LastIndex == 0 {
Expand Down
24 changes: 12 additions & 12 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -857,9 +857,9 @@ func (r *Replica) getLease() (roachpb.Lease, *roachpb.Lease) {

func (r *Replica) getLeaseRLocked() (roachpb.Lease, *roachpb.Lease) {
if nextLease, ok := r.mu.pendingLeaseRequest.RequestPending(); ok {
return r.mu.state.Lease, &nextLease
return *r.mu.state.Lease, &nextLease
}
return r.mu.state.Lease, nil
return *r.mu.state.Lease, nil
}

// ownsValidLease returns whether this replica is the current valid
Expand All @@ -870,7 +870,7 @@ func (r *Replica) ownsValidLease(ts hlc.Timestamp) bool {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.state.Lease.OwnedBy(r.store.StoreID()) &&
r.leaseStatus(r.mu.state.Lease, ts, r.mu.minLeaseProposedTS).state == leaseValid
r.leaseStatus(*r.mu.state.Lease, ts, r.mu.minLeaseProposedTS).state == leaseValid
}

// IsLeaseValid returns true if the replica's lease is owned by this
Expand Down Expand Up @@ -926,7 +926,7 @@ func (r *Replica) leaseGoodToGo(ctx context.Context) (LeaseStatus, bool) {
return LeaseStatus{}, false
}

status := r.leaseStatus(r.mu.state.Lease, timestamp, r.mu.minLeaseProposedTS)
status := r.leaseStatus(*r.mu.state.Lease, timestamp, r.mu.minLeaseProposedTS)
switch status.state {
case leaseValid:
if status.lease.OwnedBy(r.store.StoreID()) {
Expand Down Expand Up @@ -969,7 +969,7 @@ func (r *Replica) redirectOnOrAcquireLease(ctx context.Context) (LeaseStatus, *r
r.mu.Lock()
defer r.mu.Unlock()

status = r.leaseStatus(r.mu.state.Lease, timestamp, r.mu.minLeaseProposedTS)
status = r.leaseStatus(*r.mu.state.Lease, timestamp, r.mu.minLeaseProposedTS)
switch status.state {
case leaseError:
// Lease state couldn't be determined.
Expand Down Expand Up @@ -1380,7 +1380,7 @@ func (r *Replica) maybeInitializeRaftGroup(ctx context.Context) {
// will only campaign if it's been idle for >= election timeout,
// so there's most likely been no traffic to the range.
shouldCampaignOnCreation := r.mu.state.Lease.OwnedBy(r.store.StoreID()) ||
r.leaseStatus(r.mu.state.Lease, r.store.Clock().Now(), r.mu.minLeaseProposedTS).state != leaseValid
r.leaseStatus(*r.mu.state.Lease, r.store.Clock().Now(), r.mu.minLeaseProposedTS).state != leaseValid
if err := r.withRaftGroupLocked(shouldCampaignOnCreation, func(raftGroup *raft.RawNode) (bool, error) {
return true, nil
}); err != nil {
Expand Down Expand Up @@ -3068,7 +3068,7 @@ func (r *Replica) maybeQuiesceLocked() bool {
// 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 l := r.mu.state.Lease; !l.OwnedBy(r.store.StoreID()) &&
if l := *r.mu.state.Lease; !l.OwnedBy(r.store.StoreID()) &&
r.isLeaseValidRLocked(l, r.store.Clock().Now()) {
if log.V(4) {
log.Infof(ctx, "not quiescing: not leaseholder")
Expand Down Expand Up @@ -3493,7 +3493,7 @@ func (r *Replica) processRaftCommand(
if idKey != "" {
isLeaseRequest = raftCmd.ReplicatedEvalResult.IsLeaseRequest
if isLeaseRequest {
requestedLease = raftCmd.ReplicatedEvalResult.State.Lease
requestedLease = *raftCmd.ReplicatedEvalResult.State.Lease
}
ts = raftCmd.ReplicatedEvalResult.Timestamp
rSpan = roachpb.RSpan{
Expand All @@ -3514,7 +3514,7 @@ func (r *Replica) processRaftCommand(
// proposer_lease field in the proto.
verifyLease := func() error {
// Handle the case of pre-epoch-based-leases command.
return raftCmd.ProposerLease.Equivalent(r.mu.state.Lease)
return raftCmd.ProposerLease.Equivalent(*r.mu.state.Lease)
}

// TODO(tschottdorf): consider the Trace situation here.
Expand Down Expand Up @@ -3543,7 +3543,7 @@ func (r *Replica) processRaftCommand(
if !isLeaseRequest {
// We return a NotLeaseHolderError so that the DistSender retries.
nlhe := newNotLeaseHolderError(
&r.mu.state.Lease, raftCmd.ProposerReplica.StoreID, r.mu.state.Desc)
r.mu.state.Lease, raftCmd.ProposerReplica.StoreID, r.mu.state.Desc)
nlhe.CustomMsg = fmt.Sprintf(
"stale proposal: command was proposed under lease %s but is being applied "+
"under lease: %s", raftCmd.ProposerLease, r.mu.state.Lease)
Expand All @@ -3553,7 +3553,7 @@ func (r *Replica) processRaftCommand(
// replica.RedirectOnOrAcquireLease() understands. Note that these
// requests don't go through the DistSender.
forcedErr = roachpb.NewError(&roachpb.LeaseRejectedError{
Existing: r.mu.state.Lease,
Existing: *r.mu.state.Lease,
Requested: requestedLease,
})
}
Expand Down Expand Up @@ -4674,7 +4674,7 @@ func (r *Replica) Metrics(
) ReplicaMetrics {
r.mu.RLock()
raftStatus := r.raftStatusRLocked()
status := r.leaseStatus(r.mu.state.Lease, now, r.mu.minLeaseProposedTS)
status := r.leaseStatus(*r.mu.state.Lease, now, r.mu.minLeaseProposedTS)
quiescent := r.mu.quiescent || r.mu.internalRaftGroup == nil
desc := r.mu.state.Desc
selfBehindCount := r.getEstimatedBehindCountRLocked(raftStatus)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -2057,7 +2057,7 @@ func evalNewLease(
// TODO(tschottdorf): Maybe we shouldn't do this at all. Need to think
// through potential consequences.
pd.Replicated.BlockReads = !isExtension
pd.Replicated.State.Lease = lease
pd.Replicated.State.Lease = &lease
pd.Local.leaseMetricsResult = new(leaseMetricsType)
if isTransfer {
*pd.Local.leaseMetricsResult = leaseTransferSuccess
Expand Down
20 changes: 10 additions & 10 deletions pkg/storage/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@ import (
"reflect"
"time"

"github.com/coreos/etcd/raft"
"github.com/kr/pretty"
"github.com/pkg/errors"
"golang.org/x/net/context"

"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -29,9 +32,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/coreos/etcd/raft"
"github.com/kr/pretty"
"github.com/pkg/errors"
)

// leaseMetricsType is used to distinguish between various lease
Expand Down Expand Up @@ -206,12 +206,12 @@ func (p *EvalResult) MergeAndDestroy(q EvalResult) error {
}
q.Replicated.State.Desc = nil

if p.Replicated.State.Lease == (roachpb.Lease{}) {
if p.Replicated.State.Lease == nil {
p.Replicated.State.Lease = q.Replicated.State.Lease
} else if q.Replicated.State.Lease != (roachpb.Lease{}) {
} else if q.Replicated.State.Lease != nil {
return errors.New("conflicting Lease")
}
q.Replicated.State.Lease = roachpb.Lease{}
q.Replicated.State.Lease = nil

if p.Replicated.State.TruncatedState == nil {
p.Replicated.State.TruncatedState = q.Replicated.State.TruncatedState
Expand Down Expand Up @@ -615,16 +615,16 @@ func (r *Replica) handleReplicatedEvalResult(
rResult.ChangeReplicas = nil
}

if newLease := rResult.State.Lease; newLease != (roachpb.Lease{}) {
rResult.State.Lease = roachpb.Lease{} // for assertion
if newLease := rResult.State.Lease; newLease != nil {
rResult.State.Lease = nil // for assertion

r.mu.Lock()
replicaID := r.mu.replicaID
prevLease := r.mu.state.Lease
prevLease := *r.mu.state.Lease
r.mu.state.Lease = newLease
r.mu.Unlock()

r.leasePostApply(ctx, newLease, replicaID, prevLease)
r.leasePostApply(ctx, *newLease, replicaID, prevLease)
}

if newTruncState := rResult.State.TruncatedState; newTruncState != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -446,7 +446,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID
r.mu.Lock()
defer r.mu.Unlock()

status := r.leaseStatus(r.mu.state.Lease, r.store.Clock().Now(), r.mu.minLeaseProposedTS)
status := r.leaseStatus(*r.mu.state.Lease, r.store.Clock().Now(), r.mu.minLeaseProposedTS)
if status.lease.OwnedBy(target) {
// The target is already the lease holder. Nothing to do.
return nil, nil, nil
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/replica_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func (rsl replicaStateLoader) load(
if err != nil {
return storagebase.ReplicaState{}, err
}
s.Lease = lease
s.Lease = &lease

if s.GCThreshold, err = rsl.loadGCThreshold(ctx, reader); err != nil {
return storagebase.ReplicaState{}, err
Expand Down Expand Up @@ -109,7 +109,7 @@ func (rsl replicaStateLoader) save(
ctx context.Context, eng engine.ReadWriter, state storagebase.ReplicaState,
) (enginepb.MVCCStats, error) {
ms := &state.Stats
if err := rsl.setLease(ctx, eng, ms, state.Lease); err != nil {
if err := rsl.setLease(ctx, eng, ms, *state.Lease); err != nil {
return enginepb.MVCCStats{}, err
}
if err := rsl.setAppliedIndex(
Expand Down Expand Up @@ -515,7 +515,7 @@ func writeInitialState(
RangeID: desc.RangeID,
}
s.Stats = ms
s.Lease = lease
s.Lease = &lease

if existingLease, err := rsl.loadLease(ctx, eng); err != nil {
return enginepb.MVCCStats{}, errors.Wrap(err, "error reading lease")
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -710,7 +710,7 @@ func TestReplicaRangeBoundsChecking(t *testing.T) {
func hasLease(repl *Replica, timestamp hlc.Timestamp) (owned bool, expired bool) {
repl.mu.Lock()
defer repl.mu.Unlock()
status := repl.leaseStatus(repl.mu.state.Lease, timestamp, repl.mu.minLeaseProposedTS)
status := repl.leaseStatus(*repl.mu.state.Lease, timestamp, repl.mu.minLeaseProposedTS)
return repl.mu.state.Lease.OwnedBy(repl.store.StoreID()), status.state != leaseValid
}

Expand Down Expand Up @@ -1837,7 +1837,7 @@ func TestLeaseConcurrent(t *testing.T) {
for i := 0; i < num; i++ {
if err := stopper.RunAsyncTask(context.Background(), func(ctx context.Context) {
tc.repl.mu.Lock()
status := tc.repl.leaseStatus(tc.repl.mu.state.Lease, ts, hlc.Timestamp{})
status := tc.repl.leaseStatus(*tc.repl.mu.state.Lease, ts, hlc.Timestamp{})
leaseCh := tc.repl.requestLeaseLocked(ctx, status)
tc.repl.mu.Unlock()
wg.Done()
Expand Down Expand Up @@ -6559,7 +6559,7 @@ func TestReplicaIDChangePending(t *testing.T) {
// Stop the command from being proposed to the raft group and being removed.
repl.mu.Lock()
repl.mu.submitProposalFn = func(p *ProposalData) error { return nil }
lease := repl.mu.state.Lease
lease := *repl.mu.state.Lease
repl.mu.Unlock()

// Add a command to the pending list.
Expand Down
111 changes: 63 additions & 48 deletions pkg/storage/storagebase/state.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 0ba0af9

Please sign in to comment.