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

stability: crash on uninitialized replica #14193

Closed
mberhault opened this issue Mar 16, 2017 · 28 comments
Closed

stability: crash on uninitialized replica #14193

mberhault opened this issue Mar 16, 2017 · 28 comments

Comments

@mberhault
Copy link
Contributor

cockroach-omega-0003 was just upgraded to beta candidate: 10c2544
Crashed after ~9 minutes: stack trace and log entries pertaining to r3363:

I170315 22:00:02.654651 212946 storage/replica_raftstorage.go:596  [n3,s3,r3363/?:{-},@c423fa5880] applying preemptive snapshot at index 477966 (id=92ec7568, encoded size=33915347, 33 rocksdb batches, 6 log entries)
I170315 22:00:03.126140 212946 storage/replica_raftstorage.go:604  [n3,s3,r3363/?:/Table/51/1/592{56971…-62622…},@c423fa5880] applied preemptive snapshot in 471ms [clear=115ms batch=258ms entries=0ms commit=98ms]
I170315 22:00:04.916455 1199 storage/store.go:3191  [n3,s3,r3363/4:/Table/51/1/592{56971…-62622…},@c423fa5880] added to replica GC queue (peer suggestion)
I170315 22:00:05.335444 168 storage/store.go:2205  [replicaGC,n3,s3,r3363/4:/Table/51/1/592{56971…-62622…},@c423fa5880] removing replica
I170315 22:00:05.607439 168 storage/replica.go:721  [replicaGC,n3,s3,r3363/4:/Table/51/1/592{56971…-62622…},@c423fa5880] removed 237762 (237751+11) keys in 272ms [clear=210ms commit=62ms]
E170315 22:00:05.609776 1199 storage/store.go:3189  [n3,s3,r3363/5:{-},@c4317bce00] unable to add to replica GC queue: replica not initialized
I170315 22:00:05.658301 168 storage/store.go:2205  [replicaGC,n3,s3,r3363/5:{-},@c4317bce00] removing replica
F170315 22:00:05.658344 168 storage/store.go:2224  [replicaGC,n3,s3,r3363/5:{-},@c4317bce00] unexpectedly overlapped by <nil>
goroutine 168 [running]:
github.com/cockroachdb/cockroach/pkg/util/log.getStacks(0x1, 0x273d8ac2, 0x2de2b20, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:856 +0xa7
github.com/cockroachdb/cockroach/pkg/util/log.(*loggingT).outputLogEntry(0x2de4320, 0xc400000004, 0x2585a27, 0x10, 0x8b0, 0xc42cbe6690, 0x4a)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:721 +0x74f
github.com/cockroachdb/cockroach/pkg/util/log.addStructured(0x7f68fabb2800, 0xc42ac069c0, 0x4, 0x2, 0x1cb6436, 0x1d, 0xc426fed860, 0x1, 0x1)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/structured.go:146 +0x27b
github.com/cockroachdb/cockroach/pkg/util/log.logDepth(0x7f68fabb2800, 0xc42ac069c0, 0x1, 0xc400000004, 0x1cb6436, 0x1d, 0xc426fed860, 0x1, 0x1)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:67 +0x8c
github.com/cockroachdb/cockroach/pkg/util/log.Fatalf(0x7f68fabb2800, 0xc42ac069c0, 0x1cb6436, 0x1d, 0xc426fed860, 0x1, 0x1)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:151 +0x7e
github.com/cockroachdb/cockroach/pkg/storage.(*Store).removeReplicaImpl(0xc420ba8000, 0x7f68fabb2800, 0xc42ac069c0, 0xc4317bce00, 0x1, 0x2e273a8, 0x0, 0x0, 0xc423e8d2c0, 0x27, ...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:2224 +0x2a6
github.com/cockroachdb/cockroach/pkg/storage.(*Store).RemoveReplica(0xc420ba8000, 0x7f68fabb2800, 0xc42ac069c0, 0xc4317bce00, 0x1, 0x2e273a8, 0x0, 0x0, 0xc423e8d2c0, 0x27, ...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:2196 +0x12d
github.com/cockroachdb/cockroach/pkg/storage.(*replicaGCQueue).process(0xc42096fec0, 0x7f68fabb2800, 0xc42ac069c0, 0xc4317bce00, 0xc423dfe000, 0x2b, 0x40, 0x2912ee0, 0xc427802d00)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_gc_queue.go:219 +0x37a
github.com/cockroachdb/cockroach/pkg/storage.(*baseQueue).processReplica(0xc42024c000, 0x7f68fac519f8, 0xc423e8c330, 0xc4317bce00, 0xc4201326c0, 0x0, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/queue.go:603 +0x39a
github.com/cockroachdb/cockroach/pkg/storage.(*baseQueue).processLoop.func1.2()
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/queue.go:514 +0xf2
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunTask(0xc4204506e0, 0xc420567ea0, 0x0, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:239 +0x105
github.com/cockroachdb/cockroach/pkg/storage.(*baseQueue).processLoop.func1()
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/queue.go:523 +0x368
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker.func1(0xc4204506e0, 0xc4225da1a0)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:209 +0x7d
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker
        /go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:210 +0x66

The code in question is: pkg/storage/store.go:

  if placeholder := s.getOverlappingKeyRangeLocked(desc); placeholder != rep {
    // This is a fatal error because uninitialized replicas shouldn't make it
    // this far. This method will need some changes when we introduce GC of
    // uninitialized replicas.
    s.mu.Unlock()
    log.Fatalf(ctx, "unexpectedly overlapped by %v", placeholder)
  }
@mberhault
Copy link
Contributor Author

CC #14167

@mberhault
Copy link
Contributor Author

@a-robinson: omega is running with COCKROACH_ENABLE_LOAD_BASED_LEASE_REBALANCING

@a-robinson
Copy link
Contributor

Ack, thanks for the ping. Also cc @BramGruneir, since this looks potentially related to some of the problems seen since merging #13973

@a-robinson
Copy link
Contributor

Also @tamird, who originally wrote the check that's assuming the replica is initialized here, but it certainly looks like the issue is with the replica being removed twice, which is likely related to @BramGruneir's recent changes.

@tamird
Copy link
Contributor

tamird commented Mar 16, 2017 via email

@petermattis
Copy link
Collaborator

Also consider that this could have been a lurking bug that was only revealed by recent changes.

@mberhault
Copy link
Contributor Author

backed up logs go cockroach-omega-0003:~/logs.14193 and restarted cockroach

@petermattis
Copy link
Collaborator

Removing an uninitialized replica will always result in the unexpectedly overlapped by <nil> fatal error. The code path this is being called from is:

	} else if desc.RangeID != replyDesc.RangeID {
		// If we get a different range ID back, then the range has been merged
		// away. But currentMember is true, so we are still a member of the
		// subsuming range. Shut down raft processing for the former range
		// and delete any remaining metadata, but do not delete the data.
		rgcq.metrics.RemoveReplicaCount.Inc(1)
		if log.V(1) {
			log.Infof(ctx, "removing merged range")
		}
		if err := repl.store.RemoveReplica(ctx, repl, replyDesc, false); err != nil {
			return err
		}

@bdarnell Clearly the range wasn't merged because we don't have merging turned on. How else could we be in this code path?

@a-robinson
Copy link
Contributor

This appears to be an existing race between replica creation and replica deletion that's getting made much more likely by bad behavior in the replicate queue and allocator. I haven't yet confidently determined the logic flaw allowing the race, but the bad behavior in the replicate queue and allocator is quite clear. This is what was happening on node 6, the leaseholder for r3363, that caused node 3 to hit the race:

Leaseholder n6 decides to rebalance away from itself
It starts by adding replica to n3
Enqueues itself back in replicate queue so that it can transfer away its lease
A call is made into ChangeReplicas with a request to add n3's new replica (3,3,4)
n6's ADD_REPLICA command for n3's replica (ID=4) gets proposed to raft
n6's replicate queue's ShouldQueue gets called to see if further action is needed, and it decides that an AllocatorRemove is needed
1.4 seconds later, n6's replicate queue's processOneChange gets called, and it again decides that an AllocatorRemove is needed
It decides to remove n3's replica [1]
It then again decides that it needs to transfer its lease so it re-enqueues itself in the replicate queue [2]
When it comes back through the queue again, it decides it should add a replica to n3 and generates a preemptive snapshot
A call is made into ChangeReplicas with a request to add n3's new replica (3,3,5)
n6's ADD_REPLICA command for n3's replica (ID=5) gets proposed to raft
n6's replicate queue gets called on the range again and it decides to remove n3's replica and quickly proposes to raft to remove (3,3,5)
n6 again decides more rebalancing is needed and re-enqueues the range into its replicate queue
The replicate queue again decides to add a replica to n3, but now n3 is unresponsive because it has crashed

[1] remove candidates: [
s1, valid:true, con:1.00, ranges:6748, details:(diversity=0.00, preferred=0, converge=1.00)
s5, valid:true, con:0.00, ranges:7059, details:(diversity=0.00, preferred=0, converge=0.00)
s3, valid:true, con:0.00, ranges:7069, details:(diversity=0.00, preferred=0, converge=0.00)
s6, valid:true, con:0.00, ranges:7354, details:(diversity=0.00, preferred=0, converge=0.00)]

[2] ShouldTransferLease source (lease-holder=6):
candidate: avg-ranges=7110.5 avg-leases=2287.1666666666665
1: ranges=6748 leases=91 fraction-used=0.78
2: ranges=7075 leases=793 fraction-used=0.78
4: ranges=7358 leases=3622 fraction-used=0.78
6: ranges=7354 leases=4125 fraction-used=0.78
5: ranges=7059 leases=3252 fraction-used=0.79
3: ranges=7069 leases=1840 fraction-used=0.79

There are a few pieces of bad behavior here.

  1. n6 decided to remove the replica on n3 immediately after deciding to add it. I believe storage: allocator will actively move replicas due to zone configs #14106 might fix this, right @BramGruneir?
  2. Given those remove candidates, why would n6 choose to remove n3 anyways? Shouldn't it prefer removing itself given the number of ranges on it? I haven't checked out the code about this yet, but wonder if something about n3's replica being behind is causing it to get preferentially removed here.
  3. n6 re-enqueued itself expressly because it decided that it should transfer its lease, but then used that next pass through the replicate queue to add a new replica instead of transferring its lease.

In short, #13973 has made replica thrashing more likely by enabling things to get put back in the replicate queue more often. I think we may want to hold off on the release until this bad behavior and/or the underlying race is fixed.

@bdarnell
Copy link
Contributor

I think we may want to hold off on the release until this bad behavior and/or the underlying race is fixed.

Agreed. Should we revert #13973 while we work on fixing this? And if we do, do we want to go ahead with the release or wait until next week?

@a-robinson
Copy link
Contributor

@petermattis What's happening on node 3 looks something like:

  1. applying preemptive snapshot
  2. applied preemptive snapshot
  3. added to replica GC queue
  4. removeReplicaImpl called
  5. destroyDataRaftMuLocked finished its work on the replica
  6. Immediately afterward (2ms), an attempt to add replica ID 5 to the GC queue failed due to it not being initialized
  7. removeReplicaImpl called

Note that the second preemptive snapshot started streaming from n6 at 22:00:04.918343 and finished streaming at 22:00:05.334711, yet there's nothing in n3's logs about it ever starting to apply it. n3 was working on removing its old keys until 22:00:05.607439, and (presumably as soon as it could get the relevant mutex) noticed the request to GC its new replica at 22:00:05.609776, still before it could start applying the preemptive snapshot. Somehow the remove replica logic got triggered at 22:00:05.658301 without the snapshot getting applied in the middle, leading to the crash.

@a-robinson
Copy link
Contributor

@bdarnell Reverting #13973 and releasing without it would be safe. I think we can continue pulling on the two threads separately without needing that to still be in master.

@BramGruneir, can you pull on the replicate queue / allocator behavior thread?

@a-robinson
Copy link
Contributor

And when I say "safe", I mean as safe as any of our past releases.

@a-robinson a-robinson mentioned this issue Mar 16, 2017
8 tasks
@a-robinson
Copy link
Contributor

The test failure in https://teamcity.cockroachdb.com/viewLog.html?buildId=172955&tab=buildLog#_ has the same pattern - add replica on node, remove replica on node, add replica on node, remove replica on node. All in quick succession. There has to be something about that pattern.

@BramGruneir
Copy link
Member

For the trashing, specifically the adding and then subsequent removal from 3 twice... this shouldn't happen before #14106. Since this is using the older code, it has to first pass the shouldRebalance function which should reject just this type of thrashing. Regardless, it seems very suspect.

Do you happen to have the logs with the remove candidates and rebalance candidates for either of those quick add then remove cases?

Reverting #13973 is fine for now, for the beta at least, but to be clear, it's just patching over the other deeper issue.

@a-robinson
Copy link
Contributor

The logs with all the rebalancing decisions can be found in /home/cockroach/cockroach.stderr.14 on omega 6. I also have a local copy if that gets wiped for some reason. I'd do the grepping for you, but the line breaks in the candidate log messages would cause some relevant info to be lost.

@a-robinson
Copy link
Contributor

Ok, I was able to reproduce the failure by stressing TestRemovePlaceholderRace really hard with a race-enabled build. It's convenient that test already existed, because it's basically the perfect test for reproducing this failure. Here's what happened.

Replica 38 got put in gc queue, but before replica 38 could be removed, a preemptive streaming snapshot was sent for the range, and that snapshot snuck through the logic in processRaftRequest in a nasty way:

  1. req.ToReplica.ReplicaID was 0 but the local replica's ID was 38
  2. getOrCreateReplica should probably consider that an error, but doesn't, instead happily returning the existing replica
  3. Execution continued through processRaftRequest without needing to create a placeholder because there was already an initialized replica for the range
  4. Execution goes into the code that's intended for preemptive snapshots because req.ToReplica.ReplicaID == 0 and req.Message.Type == raftpb.MsgSnap
  5. A new raft node is created (raft.NewRawNode)
  6. Raft silently drops the snapshot on the floor (https://github.com/cockroachdb/vendored/blob/4ee3d80ba6f886de06169985276d0a7cf9152e2c/github.com/coreos/etcd/raft/raft.go#L1111)
  7. processRaftRequest thinks the snapshot applied and returns happily
  8. The replica gc work for repl38 gets done, removing it from the replicasByKey btree
  9. A new raft request came in for the range (as part of the next REMOVE_REPLICA transaction, I believe), causing an empty replica to get created for the range (vendor/github.com/coreos/etcd/raft/raft.go:567 [s2,r1/39:{-},@c420948000] 27 became follower at term 0). Note that the new replica does not get added to replicasByKey because the code assumes that there's a snapshot on the way that will tell us what the range's start/end keys are (
    // Add the range to range map, but not replicasByKey since the range's start
    )
  10. At this point, the damage is done. We go through the rest of the replica's life without it ever learning what its key range is (because the node that sent it the snapshot thinks the snapshot applied successfully). The replica never gets added to replicasByKey, and in fact the final lookup of getOverlappingKeyRangeLocked was likely called with empty start/end keys.

I won't have time to fix this today, but there are a couple potential ways that we could do so. And for historical reference, @tschottdorf expressed some concern about parts of this logic in the past - #7830.

@a-robinson
Copy link
Contributor

And if anyone wants to check out the logs from the repro failure that helped me understand this, I ran

make stressrace PKG=./storage/ TESTS=TestRemovePlaceholderRace TESTFLAGS='--verbosity=3' STRESSFLAGS='-stderr -maxfails 1 -p 64'

against commit 150b52d and got the output in the file below. The real key was pumping up the verbosity so that raft's info logs got printed.

raft_panic2.txt

@petermattis
Copy link
Collaborator

Nice debugging, @a-robinson.

@spencerkimball
Copy link
Member

spencerkimball commented Mar 18, 2017 via email

a-robinson added a commit to a-robinson/cockroach that referenced this issue Mar 20, 2017
* Add more detail to Fatal log message hit by cockroachdb#14193
* Improve nasty indentation in `processRaftRequest` by moving the error
  check above the normal code path
* Fix `removePlaceholderLocked` method comment
a-robinson added a commit to a-robinson/cockroach that referenced this issue Mar 20, 2017
* Add more detail to Fatal log message hit by cockroachdb#14193
* Improve nasty indentation in `processRaftRequest` by moving the error
  check above the normal code path
* Fix `removePlaceholderLocked` method comment
@bdarnell
Copy link
Contributor

At this point, the damage is done. We go through the rest of the replica's life without it ever learning what its key range is (because the node that sent it the snapshot thinks the snapshot applied successfully). The replica never gets added to replicasByKey, and in fact the final lookup of getOverlappingKeyRangeLocked was likely called with empty start/end keys.

But what is the damage? All of this looks fine to me. The preemptive snapshot was applied successfully, but preemptive snapshots are handled entirely through our code and don't (shouldn't) communicate anything back to the raft leader's view of the new replica's state. The leader wouldn't believe that the snapshot had been applied successfully unless there had been another message after the snapshot, which would have informed this node that its new replica ID is 39 (and would have in turn prevented the replica GC, barring some synchronization failure).

Deleting a replica in between a preemptive snapshot and its first real use is unfortunate (it causes wasted work since we have to send a raft snapshot afterwards), but it's supposed to all work. Maybe there's an interaction with heartbeat coalescing that is making the leader think the new replica is alive when that replica doesn't know it yet?

@a-robinson
Copy link
Contributor

Thanks for correcting me there. I might have other faulty assumptions as well, so let me take another close look at things and we can chat later this afternoon.

@a-robinson
Copy link
Contributor

I'm not seeing

  1. any logic that protects (*Store).processRaftRequest from calling raftGroup.Step on an uninitialized replica
  2. any logic that protects the replica GC queue / (*Store).removeReplicaImpl from trying to remove an uninitialized replica, despite the explicit comment saying that doing so is protected against

For the first point, you make it sound like it's unexpected for a leader to send a request to a follower that isn't alive yet. What's the expected ordering of events there? It seems possible that the successful sending of a preemptive snapshot is being used as a justification for assuming that the replica exists and is ready to receive traffic.

@bdarnell
Copy link
Contributor

When you say "uninitialized", do you mean "doesn't know its replica ID" or "doesn't know its key range"? (Replica.IsInitialized refers to the latter, so we should avoid using "initialized" for the former too)

For the first point, you make it sound like it's unexpected for a leader to send a request to a follower that isn't alive yet.

It is expected that the leader will send requests to followers that don't know their replica ID yet (and may not even exist, which could indicate that either a message was delayed until after a replica GC or an early GC destroyed a replica which has been created by a preemptive snapshot). Messages sent by raft (i.e. everything but preemptive snapshots and coalesced heartbeats) always include the replica ID.

What's the expected ordering of events there?

The normal order is:

  1. Leader decides to add replica and sends preemptive snapshot (with destination replicaID=0). If no existing replica exists (the expected case), it is created with replica ID 0. If there is an existing replica, it keeps its replica ID. The snapshot is applied to disk, making the replica Initialized in the process. Raft also generates a response (MsgAppResp) with a dummy address, which is discarded (because the preemptive snapshot path calls raftGroup.Ready() but only processes the snapshot part of the result)
  2. Leader performs the ChangeReplicas transaction, allocating a replica ID for the new replica and adding it to the raft configuration.
  3. After ChangeReplicas commits, the leader will attempt to send a raft message to the new replica (I think we expect an empty MsgApp to start). This message will always include the new replica ID as its destination.
  4. When the new replica receives its first message from the leader, it updates its replica ID (this is the first time it learns about its new replica ID). It then sends a response (MsgAppResp).
  5. Once the leader has received an MsgAppResp, then it marks the new replica as having acknowledged receipt of a snapshot, and will start sending messages that may cause panics if the contents of that snapshot have been lost.

So in order to get this panic, the GC must happen sometime after step 4.

It seems possible that the successful sending of a preemptive snapshot is being used as a justification for assuming that the replica exists and is ready to receive traffic.

I don't see any way for this to happen because we don't even allocate a new replica ID until after the snapshot has been sent.

any logic that protects the replica GC queue / (*Store).removeReplicaImpl from trying to remove an uninitialized replica, despite the explicit comment saying that doing so is protected against

The only caller of Store.RemoveReplica is replicaGCQueue, which only ever processes initialized replicas

@a-robinson
Copy link
Contributor

When you say "uninitialized", do you mean "doesn't know its replica ID" or "doesn't know its key range"?

I mean the key range, i.e. the same thing as IsInitialized.

The only caller of Store.RemoveReplica is replicaGCQueue, which only ever processes initialized replicas

It only ever enqueues initialized replicas, but it only enqueues them by their range ID, not the Replica pointer or replica ID, so it is conceivable that a copy of the old replica could have made it through these checks and caused the new replica to get GC'ed.

@a-robinson
Copy link
Contributor

As for the raft panic due to an existing commit index of 0 (#14231), I'm missing where the protection that prevents an incoming raft request from causing a panic is meant to be. This appears to be the entire code path from an incoming raft request, with an uninitialized replica and raft group being created along the way as needed.

raftTransport.RaftMessageBatch -> raftTransport.handleRaftRequest -> store.HandleRaftRequest (-> store.uncoalesceBeats) -> store.HandleRaftUncoalescedRequest (scheduler.EnqueueRaftRequest -> scheduler.worker -> store.processRaftRequest if respStream is non-nil) - > store.processRaftRequest (in which an uninitialized replica will be created) -> replica.withRaftGroup (in which a raft group with no valid info will be created) -> raftGroup.Step

Are we really just relying on raft messages to be delivered in order and only when a replica exists? I have to be missing something.

@a-robinson
Copy link
Contributor

Going back to the replica GC log.Fatal (rather than the raft panic), some extra logging has proven that it's caused by a replica being added to the GC queue multiple times along with some insufficient locking/ordering guarantees. To summarize:

  1. The replica gets added to the replica GC queue multiple times
    1. replicaGCQueue.Add gets called one or more times from store.HandleRaftResponse, once for each raft response that contains a roachpb.ReplicaTooOldError. In my test's logs this was getting called around twice per replica most of the time.
    2. replicaGCQueue.Add gets called once from replica.handleReplicatedEvalResult when evaluating the REMOVE_REPLICA command.
  2. The replica GC queue starts processing the replica after the first of the above replicaGCQueue.Add calls completes. We pop the replica off the queue when we start processing it, so the second Add call succeeds because the replica has not yet been GC'ed but it still valid. This is where the trouble starts.
  3. The replica gets fully removed, and the store calls scanner.RemoveReplica, which will trigger the scanner to remove the replica from all of its queues
  4. Its successor is quickly created by a raft MsgHeartbeat without receiving a snapshot (which arrived before the previous replica was GC'ed), leaving it in an uninitialized state
  5. The replica GC queue immediately starts processing the new replica
  6. The scanner finally gets around to removing the replica from all of its queues. Unfortunately, it was too late, since the replica GC queue already started processing it
  7. The replica GC queue "finishes" processing the new, uninitialized replica when the log.Fatal in store.removeReplicaImpl gets hit

There are still some breadcrumbs I need to track down (or have explained to me) about the raft panic, but I'm pretty sure this settles the log.Fatal.

@bdarnell
Copy link
Contributor

Are we really just relying on raft messages to be delivered in order and only when a replica exists? I have to be missing something.

No, we don't rely on in-order delivery, and it is possible for a raft message to be delivered when no replica exists (in which case it will be created by that message). We rely on two properties:

  • Getting a response from a replica indicates that that replica exists, and any knowledge implied by that response is durable and will never regress (for that replica ID).
  • When a replica ceases to exist, no new replica with that ReplicaID will ever be created. (This is enforced by tombstones and the fact that replica IDs increase monotonically)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

7 participants