Skip to content

Commit

Permalink
kvserver: add assertions for invariants around up liveness records
Browse files Browse the repository at this point in the history
Now that we have #53842, we maintain the invariant that there always
exists a liveness record for any given node. We can now simplify our
handling of liveness records internally: where previously we had code to
handle the possibility of empty liveness records (we created a new one
on the fly), we can change them to assertions to verify that's no longer
possible.

When retrieving the liveness record from our in-memory cache,
it's possible for us to not find anything due to gossip delays. Instead
of simply giving up then, now we can read the records directly from KV
(and update our caches while in the area). This PR introduces this
mechanism through usage of `getLivenessRecordFromKV`.

Finally, as a bonus, this PR also surfaces a better error when trying to
decommission non-existent nodes. We're able to do this because now we
can always assume that a missing liveness record, as seen in the
decommission codepath, implies that the user is trying to decommission a
non-existent node.

---

We don't intend to backport this to 20.2 due to the hazard described in
\#54216. We want this PR to bake on master and (possibly) trip up the
assertions added above if we've missed anything. They're the only ones
checking for the invariant we've introduced around liveness records.
That invariant will be depended on for long running migrations, so
better to shake things out early.

Release note: None
  • Loading branch information
irfansharif committed Sep 24, 2020
1 parent a688bb7 commit d66a0f0
Show file tree
Hide file tree
Showing 14 changed files with 266 additions and 266 deletions.
11 changes: 10 additions & 1 deletion pkg/cli/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,11 @@ import (
"os"
"reflect"
"strconv"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
Expand Down Expand Up @@ -337,7 +339,14 @@ func runDecommissionNode(cmd *cobra.Command, args []string) error {
}

c := serverpb.NewAdminClient(conn)
return runDecommissionNodeImpl(ctx, c, nodeCtx.nodeDecommissionWait, nodeIDs)
if err := runDecommissionNodeImpl(ctx, c, nodeCtx.nodeDecommissionWait, nodeIDs); err != nil {
// Are we trying to recommission/decommision a node that does not exist?
if strings.Contains(err.Error(), kvserver.ErrMissingLivenessRecord.Error()) {
return fmt.Errorf("node does not exist")
}
return err
}
return nil
}

func handleNodeDecommissionSelf(
Expand Down
4 changes: 2 additions & 2 deletions pkg/jobs/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,14 +65,14 @@ func (*FakeNodeLiveness) ModuleTestingKnobs() {}
// Self implements the implicit storage.NodeLiveness interface. It uses NodeID
// as the node ID. On every call, a nonblocking send is performed over nl.ch to
// allow tests to execute a callback.
func (nl *FakeNodeLiveness) Self() (kvserverpb.Liveness, error) {
func (nl *FakeNodeLiveness) Self() (kvserverpb.Liveness, bool) {
select {
case nl.SelfCalledCh <- struct{}{}:
default:
}
nl.mu.Lock()
defer nl.mu.Unlock()
return *nl.mu.livenessMap[FakeNodeID.Get()], nil
return *nl.mu.livenessMap[FakeNodeID.Get()], true
}

// GetLivenesses implements the implicit storage.NodeLiveness interface.
Expand Down
6 changes: 3 additions & 3 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -784,10 +784,10 @@ func (r *Registry) maybeCancelJobsDeprecated(
// https://github.com/cockroachdb/cockroach/issues/47892
return
}
liveness, err := nl.Self()
if err != nil {
liveness, ok := nl.Self()
if !ok {
if nodeLivenessLogLimiter.ShouldLog() {
log.Warningf(ctx, "unable to get node liveness: %s", err)
log.Warning(ctx, "unable to get node liveness")
}
// Conservatively assume our lease has expired. Abort all jobs.
r.deprecatedCancelAll(ctx)
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1435,11 +1435,12 @@ func (m *multiTestContext) heartbeatLiveness(ctx context.Context, store int) err
m.mu.RLock()
nl := m.nodeLivenesses[store]
m.mu.RUnlock()
l, err := nl.Self()
if err != nil {
return err
l, ok := nl.Self()
if !ok {
return errors.New("liveness not found")
}

var err error
for r := retry.StartWithCtx(ctx, retry.Options{MaxRetries: 5}); r.Next(); {
if err = nl.Heartbeat(ctx, l); !errors.Is(err, kvserver.ErrEpochIncremented) {
break
Expand Down
7 changes: 2 additions & 5 deletions pkg/kv/kvserver/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -514,12 +514,9 @@ func (nl *NodeLiveness) SetDrainingInternal(
}

func (nl *NodeLiveness) SetDecommissioningInternal(
ctx context.Context,
nodeID roachpb.NodeID,
liveness LivenessRecord,
targetStatus kvserverpb.MembershipStatus,
ctx context.Context, oldLivenessRec LivenessRecord, targetStatus kvserverpb.MembershipStatus,
) (changeCommitted bool, err error) {
return nl.setMembershipStatusInternal(ctx, nodeID, liveness, targetStatus)
return nl.setMembershipStatusInternal(ctx, oldLivenessRec, targetStatus)
}

// GetCircuitBreaker returns the circuit breaker controlling
Expand Down
Loading

0 comments on commit d66a0f0

Please sign in to comment.