cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.05k stars 3.8k forks source link

storage: Finish implementing ReplicaCorruptionError #4473

Closed bdarnell closed 8 years ago

bdarnell commented 8 years ago

ReplicaCorruptionError (and related code) is basically just a placeholder that does nothing. We need to be able to mark a replica (or a store?) as corrupt so that it will no longer participate in consensus and the range's data can be moved to a new replica.

vivekmenezes commented 8 years ago

Does it make sense here for a replica to mark itself as new so that the leaders sends it a brand new snapshot of the range? It's possible that the machine itself is having trouble, but then perhaps that's something to notify an admin about to take it out of service.

bdarnell commented 8 years ago

Note that while the error is named ReplicaCorruptionError, this predates our current Range/Replica nomenclature and often refers to Store-level corruption (if we get a rocksdb error we can't tell whether the whole store is in a bad state or if the damage can be isolated to a specific replica). In general we should be conservative and take machines out of service when a ReplicaCorruptionError occurs.

The raft protocol does not allow for a replica to revert itself like this; it will have to be removed and re-added with a new replica ID. And if that's going to be necessary then the new replica might as well be put on a different store instead of the one that is known to have problems.

bdarnell commented 8 years ago

As a starting point, maybeSetCorrupt should at minimum set a persistent flag. Replicas flagged as corrupt should not try to obtain a lease and all raft operations should be silently dropped (both incoming and outgoing messages). We also need to ensure that the replicas are moved to other nodes. We might want to gossip something so that other nodes can see, but it's tricky because we don't want to gossip for every replica on the store if the problem is at the store level. Maybe just gossip the store ID and we can conservatively remove all replicas from stores that have experienced any corruption.

tbg commented 8 years ago

Peter just introduced a "replica destroyed" error which blocks it from operating. At the very least, that should hold a corruption error (plus everything Ben said).

On Thu, Jun 23, 2016 at 5:31 PM Ben Darnell notifications@github.com wrote:

As a starting point, maybeSetCorrupt should at minimum set a persistent flag. Replicas flagged as corrupt should not try to obtain a lease and all raft operations should be silently dropped (both incoming and outgoing messages). We also need to ensure that the replicas are moved to other nodes. We might want to gossip something so that other nodes can see, but it's tricky because we don't want to gossip for every replica on the store if the problem is at the store level. Maybe just gossip the store ID and we can conservatively remove all replicas from stores that have experienced any corruption.

— You are receiving this because you are subscribed to this thread.

Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/4473#issuecomment-228190524, or mute the thread https://github.com/notifications/unsubscribe/AE135MPqJcCcoEMPd03fD3T_ZNph7R-Eks5qOvtMgaJpZM4HcoMe .

-- Tobias

d4l3k commented 8 years ago

To switch replicas off a corrupt store, am I correct in assuming that we can't do a standard ChangeReplicasTrigger because the on-disk raft log might be corrupt and there's no guarantee that command will be committed? Hence the gossip comments?

And we don't want to just silently stop accepting things to that store because we want faster rebalances?

@tschottdorf I'm not seeing the "replica destroyed" error anywhere in roachpb? Is it somewhere else?

petermattis commented 8 years ago

@d4l3k Not a specific error, but a holder for persistent errors: Replica.mu.destroyed.

d4l3k commented 8 years ago

@petermattis Thanks!

petermattis commented 8 years ago

Yes, executing the ChangeReplicas from the corrupt node looks problematic. I don't have a suggestion here other than we don't want to do that. Replication is normally only triggered from the replica holding the leader lease. Perhaps an RPC to the leader?

tbg commented 8 years ago

I suspect that ReplicaCorruptionError should really be a StoreCorruptionError at this point because that makes it much more powerful; if we deal with individual replicas, we must never use it when the Store's invariants might be in jeopardy (i.e. any panic that is close enough to any locking, for example). And on a storage layer failure we likely want to get rid of that disk anyway. If that's what we're doing, I figured we would gossip a distress signal for the whole store which gets picked up by the rebalancers. Obviously the Store would immediately drain all of its leases as well, after which point the problem isn't really getting data off that node but getting it onto other nodes.

On Wed, Jul 6, 2016 at 5:37 PM Peter Mattis notifications@github.com wrote:

Yes, executing the ChangeReplicas from the corrupt node looks problematic. I don't have a suggestion here other than we don't want to do that. Replication is normally only triggered from the replica holding the leader lease. Perhaps an RPC to the leader?

— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/4473#issuecomment-230914967, or mute the thread https://github.com/notifications/unsubscribe/AE135MiYRTSGUvFHtXSLOc4DW-Hz45qwks5qTCAXgaJpZM4HcoMe .

-- Tobias

petermattis commented 8 years ago

We currently create ReplicaCorruptionErrors for errors that are very likely programmer bugs. For example, Replica.applyRaftCommand creates one if the new index is not equal oldIndex+1. Marking the entire store as corrupt in those circumstances would be bad. I could very easily see a programming bug causing an entire cluster to become corrupt. I think we should aggressively alert about replication corruption errors and stop using the replica which has the error, but be conservative about ever marking a store as corrupt.

d4l3k commented 8 years ago

We could add two levels of corruption detection. One for replica level errors and then a store level error for failed checksums (which are more likely to be disk failures).

Do we have plans to add disk health checking in to cockroach such as the SMART utilities?

On Wed, Jul 6, 2016, 20:26 Peter Mattis notifications@github.com wrote:

We currently create ReplicaCorruptionErrors for errors that are very likely programmer bugs. For example, Replica.applyRaftCommand creates one if the new index is not equal oldIndex+1. Marking the entire store as corrupt in those circumstances would be bad. I could very easily see a programming bug causing an entire cluster to become corrupt. I think we should aggressively alert about replication corruption errors and stop using the replica which has the error, but be conservative about ever marking a store as corrupt.

— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/4473#issuecomment-230946536, or mute the thread https://github.com/notifications/unsubscribe/AA3fMP_PduoDysbIrm3L51RlJ4kNh2Zxks5qTEe7gaJpZM4HcoMe .

petermattis commented 8 years ago

We should file an issue to add disk health checking via SMART utilities. It's not on the immediate roadmap, but something for 2017.

A failed disk checksum can be an indication a disk is going bad, a random event or a bug in our code. I doubt a single failed disk checksum is sufficient evidence to evacuate a disk. I'd guess that our bugs are going to be far and away the most likely reason we're seeing corruption errors and these may manifest as both checksum failures as well as invariants being violated.

d4l3k commented 8 years ago

@petermattis What reason is there for https://github.com/cockroachdb/cockroach/blame/master/storage/replica.go#L250 ?

Wouldn't it be easier to return the corruption error so it would get propagated back to the range leader through GRPC?

petermattis commented 8 years ago

@d4l3k Until recently, we couldn't return errors from raft and returning an error from withRaftGroupLocked would cause a panic.

d4l3k commented 8 years ago

Until recently

So it's fine to return errors from raft now?

As far as I can tell all that should happen is that (*RaftTransport).processQueue will close the command queue for that replica and call the error handler. Closing the queue for a destroyed replica seems like the correct thing to happen.

petermattis commented 8 years ago

So it's fine to return errors from raft now?

It should be. You'll definitely want to test this.

bdarnell commented 8 years ago

Ultimately I think we want two levels of corruption error, one for the store and one for individual replicas. I'd say the store-level one is more important; it's safer to drain the whole store and start over from scratch than to assume that we'll be able to recover by destroying the corrupted replica. It's true that an overzealous StoreCorruptionError could knock out the whole cluster, but so could a single-replica corruption error on a meta range. We shouldn't eagerly delete data when a store or replica is corrupt, just leave it for an admin to look at (and possibly provide a tool to clear the corrupt flag if we can determine that it's a false alarm).

The panic in processRaft is still there; it's not yet safe to return an error from withRaftGroupLocked. The recent improvements in error handling make it possible to return errors from handleRaftMessage, which will cause the stream to be closed.