diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 96408c85a88f..6df4b420398f 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -11,7 +11,6 @@ package cli import ( - "bufio" "bytes" "context" "encoding/base64" @@ -41,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -1100,347 +1098,6 @@ func runDebugSyncBench(cmd *cobra.Command, args []string) error { return syncbench.Run(syncBenchOpts) } -var debugUnsafeRemoveDeadReplicasCmd = &cobra.Command{ - Use: "unsafe-remove-dead-replicas --dead-store-ids=[store ID,...] [path]", - Short: "Unsafely attempt to recover a range that has lost quorum (deprecated)", - Long: ` -DEPRECATED: use 'debug recover' instead. unsafe-remove-dead-replicas will be -removed in CockroachDB v23.1. - -This command is UNSAFE and should only be used with the supervision of -a Cockroach Labs engineer. It is a last-resort option to recover data -after multiple node failures. The recovered data is not guaranteed to -be consistent. If a suitable backup exists, restore it instead of -using this tool. - -The --dead-store-ids flag takes a comma-separated list of dead store IDs and -scans this store for any ranges unable to make progress (as indicated by the -remaining replicas not marked as dead). For each such replica in which the local -store is the voter with the highest StoreID, the range descriptors will be (only -when run on that store) rewritten in-place to reflect a replication -configuration in which the local store is the sole voter (and thus able to make -progress). - -The intention is for the tool to be run against all stores in the cluster, which -will attempt to recover all ranges in the system for which such an operation is -necessary. This is the safest and most straightforward option, but incurs global -downtime. When availability problems are isolated to a small number of ranges, -it is also possible to restrict the set of nodes to be restarted (all nodes that -have a store that has the highest voting StoreID in one of the ranges that need -to be recovered), and to perform the restarts one-by-one (assuming system ranges -are not affected). With this latter strategy, restarts of additional replicas -may still be necessary, owing to the fact that the former leaseholder's epoch -may still be live, even though that leaseholder may now be on the "unrecovered" -side of the range and thus still unavailable. This case can be detected via the -range status of the affected range(s) and by restarting the listed leaseholder. - -This command will prompt for confirmation before committing its changes. - -WARNINGS - -This tool may cause previously committed data to be lost. It does not preserve -atomicity of transactions, so further inconsistencies and undefined behavior may -result. In the worst case, a corruption of the cluster-internal metadata may -occur, which would complicate the recovery further. It is recommended to take a -filesystem-level backup or snapshot of the nodes to be affected before running -this command (it is not safe to take a filesystem-level backup of a running -node, but it is possible while the node is stopped). A cluster that had this -tool used against it is no longer fit for production use. It must be -re-initialized from a backup. - -Before proceeding at the yes/no prompt, review the ranges that are affected to -consider the possible impact of inconsistencies. Further remediation may be -necessary after running this tool, including dropping and recreating affected -indexes, or in the worst case creating a new backup or export of this cluster's -data for restoration into a brand new cluster. Because of the latter -possibilities, this tool is a slower means of disaster recovery than restoring -from a backup. - -Must only be used when the dead stores are lost and unrecoverable. If the dead -stores were to rejoin the cluster after this command was used, data may be -corrupted. - -After this command is used, the node should not be restarted until at least 10 -seconds have passed since it was stopped. Restarting it too early may lead to -things getting stuck (if it happens, it can be fixed by restarting a second -time). -`, - Args: cobra.ExactArgs(1), - RunE: clierrorplus.MaybeDecorateError(runDebugUnsafeRemoveDeadReplicas), -} - -var removeDeadReplicasOpts struct { - deadStoreIDs []int -} - -func runDebugUnsafeRemoveDeadReplicas(cmd *cobra.Command, args []string) error { - stopper := stop.NewStopper() - defer stopper.Stop(context.Background()) - - db, err := OpenEngine(args[0], stopper, storage.MustExist) - if err != nil { - return err - } - - deadStoreIDs := map[roachpb.StoreID]struct{}{} - for _, id := range removeDeadReplicasOpts.deadStoreIDs { - deadStoreIDs[roachpb.StoreID(id)] = struct{}{} - } - batch, err := removeDeadReplicas(db, deadStoreIDs) - if err != nil { - return err - } else if batch == nil { - fmt.Printf("Nothing to do\n") - return nil - } - defer batch.Close() - - fmt.Printf("Proceed with the above rewrites? [y/N] ") - - reader := bufio.NewReader(os.Stdin) - line, err := reader.ReadString('\n') - if err != nil { - return err - } - fmt.Printf("\n") - if line[0] == 'y' || line[0] == 'Y' { - fmt.Printf("Committing\n") - if err := batch.Commit(true); err != nil { - return err - } - } else { - fmt.Printf("Aborting\n") - } - return nil -} - -func removeDeadReplicas( - db storage.Engine, deadStoreIDs map[roachpb.StoreID]struct{}, -) (storage.Batch, error) { - clock := hlc.NewClockWithSystemTimeSource(0 /* maxOffset */) - - ctx := context.Background() - - storeIdent, err := kvserver.ReadStoreIdent(ctx, db) - if err != nil { - return nil, err - } - fmt.Printf("Scanning replicas on store %s for dead peers %v\n", storeIdent.String(), - removeDeadReplicasOpts.deadStoreIDs) - - if _, ok := deadStoreIDs[storeIdent.StoreID]; ok { - return nil, errors.Errorf("this store's ID (%s) marked as dead, aborting", storeIdent.StoreID) - } - - var newDescs []roachpb.RangeDescriptor - - err = kvserver.IterateRangeDescriptorsFromDisk(ctx, db, func(desc roachpb.RangeDescriptor) error { - numDeadPeers := 0 - allReplicas := desc.Replicas().Descriptors() - maxLiveVoter := roachpb.StoreID(-1) - for _, rep := range allReplicas { - if _, ok := deadStoreIDs[rep.StoreID]; ok { - numDeadPeers++ - continue - } - // The designated survivor will be the voter with the highest storeID. - // Note that an outgoing voter cannot be designated, as the only - // replication change it could make is to turn itself into a learner, at - // which point the range is completely messed up. - // - // Note: a better heuristic might be to choose the leaseholder store, not - // the largest store, as this avoids the problem of requests still hanging - // after running the tool in a rolling-restart fashion (when the lease- - // holder is under a valid epoch and was ont chosen as designated - // survivor). However, this choice is less deterministic, as leaseholders - // are more likely to change than replication configs. The hanging would - // independently be fixed by the below issue, so staying with largest store - // is likely the right choice. See: - // - // https://github.com/cockroachdb/cockroach/issues/33007 - if rep.IsVoterNewConfig() && rep.StoreID > maxLiveVoter { - maxLiveVoter = rep.StoreID - } - } - - // If there's no dead peer in this group (so can't hope to fix - // anything by rewriting the descriptor) or the current store is not the - // one we want to turn into the sole voter, don't do anything. - if numDeadPeers == 0 { - return nil - } - if storeIdent.StoreID != maxLiveVoter { - fmt.Printf("not designated survivor, skipping: %s\n", &desc) - return nil - } - - // The replica thinks it can make progress anyway, so we leave it alone. - if desc.Replicas().CanMakeProgress(func(rep roachpb.ReplicaDescriptor) bool { - _, ok := deadStoreIDs[rep.StoreID] - return !ok - }) { - fmt.Printf("replica has not lost quorum, skipping: %s\n", &desc) - return nil - } - - // We're the designated survivor and the range does not to be recovered. - // - // Rewrite the range as having a single replica. The winning replica is - // picked arbitrarily: the one with the highest store ID. This is not always - // the best option: it may lose writes that were committed on another - // surviving replica that had applied more of the raft log. However, in - // practice when we have multiple surviving replicas but still need this - // tool (because the replication factor was 4 or higher), we see that the - // logs are nearly always in sync and the choice doesn't matter. Correctly - // picking the replica with the longer log would complicate the use of this - // tool. - newDesc := desc - // Rewrite the replicas list. Bump the replica ID so that in case there are - // other surviving nodes that were members of the old incarnation of the - // range, they no longer recognize this revived replica (because they are - // not in sync with it). - replicas := []roachpb.ReplicaDescriptor{{ - NodeID: storeIdent.NodeID, - StoreID: storeIdent.StoreID, - ReplicaID: desc.NextReplicaID, - }} - newDesc.SetReplicas(roachpb.MakeReplicaSet(replicas)) - newDesc.NextReplicaID++ - fmt.Printf("replica has lost quorum, recovering: %s -> %s\n", &desc, &newDesc) - newDescs = append(newDescs, newDesc) - return nil - }) - if err != nil { - return nil, err - } - - if len(newDescs) == 0 { - return nil, nil - } - - batch := db.NewBatch() - for _, desc := range newDescs { - // Write the rewritten descriptor to the range-local descriptor - // key. We do not update the meta copies of the descriptor. - // Instead, we leave them in a temporarily inconsistent state and - // they will be overwritten when the cluster recovers and - // up-replicates this range from its single copy to multiple - // copies. We rely on the fact that all range descriptor updates - // start with a CPut on the range-local copy followed by a blind - // Put to the meta copy. - // - // For example, if we have replicas on s1-s4 but s3 and s4 are - // dead, we will rewrite the replica on s2 to have s2 as its only - // member only. When the cluster is restarted (and the dead nodes - // remain dead), the rewritten replica will be the only one able - // to make progress. It will elect itself leader and upreplicate. - // - // The old replica on s1 is untouched by this process. It will - // eventually either be overwritten by a new replica when s2 - // upreplicates, or it will be destroyed by the replica GC queue - // after upreplication has happened and s1 is no longer a member. - // (Note that in the latter case, consistency between s1 and s2 no - // longer matters; the consistency checker will only run on nodes - // that the new leader believes are members of the range). - // - // Note that this tool does not guarantee fully consistent - // results; the most recent writes to the raft log may have been - // lost. In the most unfortunate cases, this means that we would - // be "winding back" a split or a merge, which is almost certainly - // to result in irrecoverable corruption (for example, not only - // will individual values stored in the meta ranges diverge, but - // there will be keys not represented by any ranges or vice - // versa). - key := keys.RangeDescriptorKey(desc.StartKey) - sl := stateloader.Make(desc.RangeID) - ms, err := sl.LoadMVCCStats(ctx, batch) - if err != nil { - return nil, errors.Wrap(err, "loading MVCCStats") - } - err = storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc) - if wiErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &wiErr) { - if len(wiErr.Intents) != 1 { - return nil, errors.Errorf("expected 1 intent, found %d: %s", len(wiErr.Intents), wiErr) - } - intent := wiErr.Intents[0] - // We rely on the property that transactions involving the range - // descriptor always start on the range-local descriptor's key. When there - // is an intent, this means that it is likely that the transaction did not - // commit, so we abort the intent. - // - // However, this is not guaranteed. For one, applying a command is not - // synced to disk, so in theory whichever store becomes the designated - // survivor may temporarily have "forgotten" that the transaction - // committed in its applied state (it would still have the committed log - // entry, as this is durable state, so it would come back once the node - // was running, but we don't see that materialized state in - // unsafe-remove-dead-replicas). This is unlikely to be a problem in - // practice, since we assume that the store was shut down gracefully and - // besides, the write likely had plenty of time to make it to durable - // storage. More troubling is the fact that the designated survivor may - // simply not yet have learned that the transaction committed; it may not - // have been in the quorum and could've been slow to catch up on the log. - // It may not even have the intent; in theory the remaining replica could - // have missed any number of transactions on the range descriptor (even if - // they are in the log, they may not yet be applied, and the replica may - // not yet have learned that they are committed). This is particularly - // troubling when we miss a split, as the right-hand side of the split - // will exist in the meta ranges and could even be able to make progress. - // For yet another thing to worry about, note that the determinism (across - // different nodes) assumed in this tool can easily break down in similar - // ways (not all stores are going to have the same view of what the - // descriptors are), and so multiple replicas of a range may declare - // themselves the designated survivor. Long story short, use of this tool - // with or without the presence of an intent can - in theory - really - // tear the cluster apart. - // - // A solution to this would require a global view, where in a first step - // we collect from each store in the cluster the replicas present and - // compute from that a "recovery plan", i.e. set of replicas that will - // form the recovered keyspace. We may then find that no such recovery - // plan is trivially achievable, due to any of the above problems. But - // in the common case, we do expect one to exist. - fmt.Printf("aborting intent: %s (txn %s)\n", key, intent.Txn.ID) - - // A crude form of the intent resolution process: abort the - // transaction by deleting its record. - txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if _, err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { - return nil, err - } - update := roachpb.LockUpdate{ - Span: roachpb.Span{Key: intent.Key}, - Txn: intent.Txn, - Status: roachpb.ABORTED, - } - if _, err := storage.MVCCResolveWriteIntent(ctx, batch, &ms, update); err != nil { - return nil, err - } - // With the intent resolved, we can try again. - if err := storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc); err != nil { - return nil, err - } - } else if err != nil { - batch.Close() - return nil, err - } - // Write the new replica ID to RaftReplicaIDKey. - replicas := desc.Replicas().Descriptors() - if len(replicas) != 1 { - return nil, errors.Errorf("expected 1 replica, got %v", replicas) - } - if err := sl.SetRaftReplicaID(ctx, batch, replicas[0].ReplicaID); err != nil { - return nil, errors.Wrapf(err, "failed to write new replica ID for range %d", desc.RangeID) - } - // Update MVCC stats. - if err := sl.SetMVCCStats(ctx, batch, &ms); err != nil { - return nil, errors.Wrap(err, "updating MVCCStats") - } - } - - return batch, nil -} - var debugMergeLogsCmd = &cobra.Command{ Use: "merge-logs ", Short: "merge multiple log files from different machines into a single stream", @@ -1635,7 +1292,6 @@ var DebugCommandsRequiringEncryption = []*cobra.Command{ debugRaftLogCmd, debugRangeDataCmd, debugRangeDescriptorsCmd, - debugUnsafeRemoveDeadReplicasCmd, debugRecoverCollectInfoCmd, debugRecoverExecuteCmd, } @@ -1650,7 +1306,6 @@ var debugCmds = []*cobra.Command{ debugRaftLogCmd, debugRangeDataCmd, debugRangeDescriptorsCmd, - debugUnsafeRemoveDeadReplicasCmd, debugBallastCmd, debugCheckLogConfigCmd, debugDecodeKeyCmd, @@ -1767,10 +1422,6 @@ func init() { f.IntVarP(&debugCompactOpts.maxConcurrency, "max-concurrency", "c", debugCompactOpts.maxConcurrency, "maximum number of concurrent compactions") - f = debugUnsafeRemoveDeadReplicasCmd.Flags() - f.IntSliceVar(&removeDeadReplicasOpts.deadStoreIDs, "dead-store-ids", nil, - "list of dead store IDs") - f = debugRecoverCollectInfoCmd.Flags() f.VarP(&debugRecoverCollectInfoOpts.Stores, cliflags.RecoverStore.Name, cliflags.RecoverStore.Shorthand, cliflags.RecoverStore.Usage()) diff --git a/pkg/cli/debug_test.go b/pkg/cli/debug_test.go index efdc003be45d..0574deb16994 100644 --- a/pkg/cli/debug_test.go +++ b/pkg/cli/debug_test.go @@ -14,7 +14,6 @@ import ( "context" "fmt" "path/filepath" - "regexp" "sort" "strings" "testing" @@ -22,23 +21,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" ) func createStore(t *testing.T, path string) { @@ -99,290 +90,6 @@ func TestOpenReadOnlyStore(t *testing.T) { } } -func TestRemoveDeadReplicas(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, 75133, "flaky test") - defer log.Scope(t).Close(t) - - // This test is pretty slow under race (200+ cpu-seconds) because it - // uses multiple real disk-backed stores and goes through multiple - // cycles of rereplicating all ranges. - skip.UnderRace(t) - - ctx := context.Background() - - testCases := []struct { - survivingNodes, totalNodes, replicationFactor int - }{ - {1, 3, 3}, - {2, 4, 4}, - } - - for _, testCase := range testCases { - t.Run(fmt.Sprintf("%d/%d/r=%d", testCase.survivingNodes, testCase.totalNodes, - testCase.replicationFactor), - func(t *testing.T) { - baseDir, dirCleanupFn := testutils.TempDir(t) - defer dirCleanupFn() - - // The surviving nodes get a real store, others are just in memory. - var storePaths []string - clusterArgs := base.TestClusterArgs{ - ServerArgsPerNode: map[int]base.TestServerArgs{}, - } - deadReplicas := map[roachpb.StoreID]struct{}{} - - for i := 0; i < testCase.totalNodes; i++ { - args := base.TestServerArgs{} - args.ScanMaxIdleTime = time.Millisecond - storeID := roachpb.StoreID(i + 1) - if i < testCase.survivingNodes { - path := filepath.Join(baseDir, fmt.Sprintf("store%d", storeID)) - storePaths = append(storePaths, path) - // ServerArgsPerNode uses 0-based index, not node ID. - args.StoreSpecs = []base.StoreSpec{{Path: path}} - } else { - deadReplicas[storeID] = struct{}{} - } - clusterArgs.ServerArgsPerNode[i] = args - } - - // Start the cluster, let it replicate, then stop it. Since the - // non-surviving nodes use in-memory stores, this automatically - // causes the cluster to lose its quorum. - // - // While it's running, start a transaction and write an intent to - // one of the range descriptors (without committing or aborting the - // transaction). This exercises a special case in removeDeadReplicas. - func() { - tc := testcluster.StartTestCluster(t, testCase.totalNodes, clusterArgs) - defer tc.Stopper().Stop(ctx) - - s := sqlutils.MakeSQLRunner(tc.Conns[0]) - - // Set the replication factor on all zones. - func() { - rows := s.Query(t, "show all zone configurations") - defer rows.Close() - re := regexp.MustCompile(`num_replicas = \d+`) - var sqls []string - for rows.Next() { - var name, configSQL string - if err := rows.Scan(&name, &configSQL); err != nil { - t.Fatal(err) - } - sqls = append(sqls, configSQL) - } - for _, sql := range sqls { - if re.FindString(sql) != "" { - sql = re.ReplaceAllString(sql, - fmt.Sprintf("num_replicas = %d", testCase.replicationFactor)) - } else { - sql = fmt.Sprintf("%s, num_replicas = %d", sql, testCase.replicationFactor) - } - s.Exec(t, sql) - } - }() - if err := tc.WaitForFullReplication(); err != nil { - t.Fatal(err) - } - - // Perform a write, to ensure that pre-crash data is preserved. - // Creating a table causes extra friction in the test harness when - // we restart the cluster, so just write a setting. - s.Exec(t, "set cluster setting cluster.organization='remove dead replicas test'") - - txn := kv.NewTxn(ctx, tc.Servers[0].DB(), 1) - var desc roachpb.RangeDescriptor - // Pick one of the predefined split points. - rdKey := keys.RangeDescriptorKey(roachpb.RKey(keys.TimeseriesPrefix)) - if err := txn.GetProto(ctx, rdKey, &desc); err != nil { - t.Fatal(err) - } - desc.NextReplicaID++ - if err := txn.Put(ctx, rdKey, &desc); err != nil { - t.Fatal(err) - } - - // At this point the intent has been written to Pebble but this - // write was not synced (only the raft log append was synced). We - // need to force another sync, but we're far from the storage - // layer here so the easiest thing to do is simply perform a - // second write. This will force the first write to be persisted - // to disk (the second write may or may not make it to disk due to - // timing). - desc.NextReplicaID++ - if err := txn.Put(ctx, rdKey, &desc); err != nil { - t.Fatal(err) - } - - // We deliberately do not close this transaction so the intent is - // left behind. - }() - - // Open the surviving stores directly to repair them. - repairStore := func(idx int) error { - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - db, err := OpenEngine(storePaths[idx], stopper, storage.MustExist) - if err != nil { - return err - } - - batch, err := removeDeadReplicas(db, deadReplicas) - if err != nil { - return err - } - if batch != nil { - if err := batch.Commit(true); err != nil { - return err - } - batch.Close() - } - - // The repair process is idempotent and should give a nil batch the second time - batch, err = removeDeadReplicas(db, deadReplicas) - if err != nil { - return err - } - if batch != nil { - batch.Close() - return errors.New("expected nil batch on second attempt") - } - return nil - } - for i := 0; i < testCase.survivingNodes; i++ { - err := repairStore(i) - if err != nil { - t.Fatal(err) - } - } - - // Now that the data is salvaged, we can restart the cluster. - // The nodes with the in-memory stores will be assigned new, - // higher node IDs (e.g. in the 1/3 case, the restarted nodes - // will be 4 and 5). - // - // This can activate the adaptive zone config feature (using 5x - // replication for clusters of 5 nodes or more), so we must - // decommission the dead nodes for WaitForFullReplication to - // complete. (note that the cluster is working even when it - // doesn't consider itself fully replicated - that's what allows - // the decommissioning to succeed. We're just waiting for full - // replication so that we can validate that ranges were moved - // from e.g. {1,2,3} to {1,4,5}). - // - // Set replication mode to manual so that TestCluster doesn't call - // WaitForFullReplication before we've decommissioned the nodes. - clusterArgs.ReplicationMode = base.ReplicationManual - clusterArgs.ParallelStart = true - - // Sleep before restarting to allow all previous leases to expire. - // Without this sleep, the 2/4 case is flaky because sometimes - // node 1 will be left with an unexpired lease, which tricks - // some heuristics on node 2 into never campaigning even though it - // is the only surviving replica. - // - // The reason it never campaigns is incompletely understood. - // The result of shouldCampaignOnWake will correctly - // transition from false to true when the old lease expires, - // but nothing triggers a "wake" event after this point. - // - // TODO(bdarnell): This is really a bug in Replica.leaseStatus. - // It should not return VALID for a lease held by a node that - // has been removed from the configuration. - time.Sleep(10 * time.Second) - - tc := testcluster.StartTestCluster(t, testCase.totalNodes, clusterArgs) - defer tc.Stopper().Stop(ctx) - - grpcConn, err := tc.Server(0).RPCContext().GRPCDialNode( - tc.Server(0).ServingRPCAddr(), - tc.Server(0).NodeID(), - rpc.DefaultClass, - ).Connect(ctx) - if err != nil { - t.Fatal(err) - } - adminClient := serverpb.NewAdminClient(grpcConn) - - deadNodes := []roachpb.NodeID{} - for i := testCase.survivingNodes; i < testCase.totalNodes; i++ { - deadNodes = append(deadNodes, roachpb.NodeID(i+1)) - } - - if err := runDecommissionNodeImpl( - ctx, adminClient, nodeDecommissionWaitNone, deadNodes, tc.Server(0).NodeID(), - ); err != nil { - t.Fatal(err) - } - - for i := 0; i < len(tc.Servers); i++ { - err = tc.Servers[i].Stores().VisitStores(func(store *kvserver.Store) error { - store.SetReplicateQueueActive(true) - return nil - }) - if err != nil { - t.Fatal(err) - } - } - if err := tc.WaitForFullReplication(); err != nil { - t.Fatal(err) - } - - for i := 0; i < len(tc.Servers); i++ { - err = tc.Servers[i].Stores().VisitStores(func(store *kvserver.Store) error { - return store.ForceConsistencyQueueProcess() - }) - if err != nil { - t.Fatal(err) - } - } - - expectedReplicas := []string{} - for i := 0; i < testCase.totalNodes; i++ { - var storeID int - if i < testCase.survivingNodes { - // If the replica survived, just adjust from zero-based to one-based. - storeID = i + 1 - } else { - storeID = i + 1 + testCase.totalNodes - testCase.survivingNodes - } - expectedReplicas = append(expectedReplicas, fmt.Sprintf("%d", storeID)) - } - expectedReplicaStr := fmt.Sprintf("{%s}", strings.Join(expectedReplicas, ",")) - - s := sqlutils.MakeSQLRunner(tc.Conns[0]) - row := s.QueryRow(t, "select replicas from [show ranges from table system.namespace] limit 1") - var replicaStr string - row.Scan(&replicaStr) - if replicaStr != expectedReplicaStr { - t.Fatalf("expected replicas on %s but got %s", expectedReplicaStr, replicaStr) - } - - row = s.QueryRow(t, "show cluster setting cluster.organization") - var org string - row.Scan(&org) - - // If there was only one surviving node, we know we - // recovered the write we did at the start of the test. But - // if there were multiples, we might have picked a lagging - // replica that didn't have it. See comments around - // maxLivePeer in debug.go. - // - // TODO(bdarnell): It doesn't look to me like this is - // guaranteed even in the single-survivor case, but it's - // only flaky when there are multiple survivors. - if testCase.survivingNodes == 1 { - if org != "remove dead replicas test" { - t.Fatalf("expected old setting to be present, got %s instead", org) - } - } - }) - } -} - func TestParseGossipValues(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index 5e9866fc2ac2..0dc1df5ed174 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -223,27 +223,26 @@ func applyReplicaUpdate( // synced to disk, so in theory whichever store becomes the designated // survivor may temporarily have "forgotten" that the transaction // committed in its applied state (it would still have the committed log - // entry, as this is durable state, so it would come back once the node - // was running, but we don't see that materialized state in - // unsafe-remove-dead-replicas). This is unlikely to be a problem in - // practice, since we assume that the store was shut down gracefully and - // besides, the write likely had plenty of time to make it to durable - // storage. More troubling is the fact that the designated survivor may - // simply not yet have learned that the transaction committed; it may not - // have been in the quorum and could've been slow to catch up on the log. - // It may not even have the intent; in theory the remaining replica could - // have missed any number of transactions on the range descriptor (even if - // they are in the log, they may not yet be applied, and the replica may - // not yet have learned that they are committed). This is particularly - // troubling when we miss a split, as the right-hand side of the split - // will exist in the meta ranges and could even be able to make progress. - // For yet another thing to worry about, note that the determinism (across - // different nodes) assumed in this tool can easily break down in similar - // ways (not all stores are going to have the same view of what the - // descriptors are), and so multiple replicas of a range may declare - // themselves the designated survivor. Long story short, use of this tool - // with or without the presence of an intent can - in theory - really - // tear the cluster apart. + // entry, as this is durable state, so it would come back once the node was + // running, but we don't see that materialized state in `debug recover`). + // This is unlikely to be a problem in practice, since we assume that the + // store was shut down gracefully and besides, the write likely had plenty + // of time to make it to durable storage. More troubling is the fact that + // the designated survivor may simply not yet have learned that the + // transaction committed; it may not have been in the quorum and could've + // been slow to catch up on the log. It may not even have the intent; in + // theory the remaining replica could have missed any number of transactions + // on the range descriptor (even if they are in the log, they may not yet be + // applied, and the replica may not yet have learned that they are + // committed). This is particularly troubling when we miss a split, as the + // right-hand side of the split will exist in the meta ranges and could even + // be able to make progress. For yet another thing to worry about, note + // that the determinism (across different nodes) assumed in this tool can + // easily break down in similar ways (not all stores are going to have the + // same view of what the descriptors are), and so multiple replicas of a + // range may declare themselves the designated survivor. Long story short, + // use of this tool with or without the presence of an intent can - in + // theory - really tear the cluster apart. // // A solution to this would require a global view, where in a first step // we collect from each store in the cluster the replicas present and diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 16f1fb403528..264aee06f340 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -975,7 +975,7 @@ func (r *Replica) GetRangeInfo(ctx context.Context) roachpb.RangeInfo { if _, ok := desc.GetReplicaDescriptorByID(l.Replica.ReplicaID); !ok { // I wish this could be a Fatal, but unfortunately it's possible for the // lease to be incoherent with the descriptor after a leaseholder was - // brutally removed through `cockroach debug unsafe-remove-dead-replicas`. + // brutally removed through `cockroach debug recover`. log.Errorf(ctx, "leaseholder replica not in descriptor; desc: %s, lease: %s", desc, l) // Let's not return an incoherent lease; for example if we end up // returning it to a client through a br.RangeInfos, the client will freak diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 705236b99820..11d2dde22511 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -271,7 +271,7 @@ func (rgcq *replicaGCQueue) process( // the use of a snapshot when catching up to the new replica ID. // We don't normally expect to have a *higher* local replica ID // than the one in the meta descriptor, but it's possible after - // recovering with unsafe-remove-dead-replicas. + // recovering with "debug recover". return false, nil } else if sameRange { // We are no longer a member of this range, but the range still exists. diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index 09f001d68bc9..082fcac82453 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -1177,10 +1177,8 @@ func (r *Replica) leaseGoodToGoForStatusRLocked( // Hence, an extra check is in order: processRaftCommand makes sure that // lease requests for a replica not in the descriptor are bounced. // - // However, this is possible if the `cockroach debug - // unsafe-remove-dead-replicas` command has been used, so - // this is just a logged error instead of a fatal - // assertion. + // However, this is possible if the `cockroach debug recover` command has + // been used, so this is just a logged error instead of a fatal assertion. log.Errorf(ctx, "lease %s owned by replica %+v that no longer exists", st.Lease, st.Lease.Replica) }