Skip to content

Commit

Permalink
Merge #93157 #95289
Browse files Browse the repository at this point in the history
93157: loqrecovery: add replica info collection to admin server r=erikgrinaker,knz a=aliher1911

This commit adds replica info and range descriptor collection from partially available cluster that lost quorum on some ranges. Collection is done using AdminServer calls. Cluster wide call performs fan-out using node info from gossip. Local replica info collection on nodes is done by scanning storages.

Release note: File format used for transient loss of quorum recovery files has changed. It is not possible to use replica info files generated by earlier versions to be used with current and future versions.

Fixes #93040

Note that we recovery doesn't need collect command to be "remote capable" but allowing this makes development and debugging simpler as you can create local snapshot from a cluster that you can subsequently verify you planning against.

95289: sql: populate `pg_proc.prokind` r=rafiss a=knz

Needed for #88061.

Fixes #95288.
Epic: CRDB-23454

Co-authored-by: Oleg Afanasyev <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Jan 17, 2023
3 parents 369b442 + 76fc533 + 02015bc commit aaba7ad
Show file tree
Hide file tree
Showing 24 changed files with 1,207 additions and 157 deletions.
19 changes: 19 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -7241,12 +7241,31 @@ Support status: [reserved](#support-status)
| ----- | ---- | ----- | ----------- | -------------- |
| range_descriptor | [cockroach.roachpb.RangeDescriptor](#cockroach.server.serverpb.RecoveryCollectReplicaInfoResponse-cockroach.roachpb.RangeDescriptor) | | | [reserved](#support-status) |
| replica_info | [cockroach.kv.kvserver.loqrecovery.loqrecoverypb.ReplicaInfo](#cockroach.server.serverpb.RecoveryCollectReplicaInfoResponse-cockroach.kv.kvserver.loqrecovery.loqrecoverypb.ReplicaInfo) | | | [reserved](#support-status) |
| node_stream_restarted | [RecoveryCollectReplicaRestartNodeStream](#cockroach.server.serverpb.RecoveryCollectReplicaInfoResponse-cockroach.server.serverpb.RecoveryCollectReplicaRestartNodeStream) | | | [reserved](#support-status) |






<a name="cockroach.server.serverpb.RecoveryCollectReplicaInfoResponse-cockroach.server.serverpb.RecoveryCollectReplicaRestartNodeStream"></a>
#### RecoveryCollectReplicaRestartNodeStream

RecoveryCollectReplicaRestartNodeStream is sent by collector node to client
if it experiences a transient failure collecting data from one of the nodes.
This message instructs client to drop any data that it collected locally
for specified node as streaming for this node would be restarted.
This mechanism is needed to avoid restarting the whole collection procedure
in large cluster if one of the nodes fails transiently.

| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [int32](#cockroach.server.serverpb.RecoveryCollectReplicaInfoResponse-int32) | | | [reserved](#support-status) |






## RecoveryCollectLocalReplicaInfo

Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/intentresolver:intentresolver_test",
"//pkg/kv/kvserver/liveness:liveness_test",
"//pkg/kv/kvserver/logstore:logstore_test",
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_test",
"//pkg/kv/kvserver/loqrecovery:loqrecovery_test",
"//pkg/kv/kvserver/multiqueue:multiqueue_test",
"//pkg/kv/kvserver/protectedts/ptcache:ptcache_test",
Expand Down Expand Up @@ -1200,6 +1201,7 @@ GO_TARGETS = [
"//pkg/kv/kvserver/logstore:logstore",
"//pkg/kv/kvserver/logstore:logstore_test",
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb",
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_test",
"//pkg/kv/kvserver/loqrecovery:loqrecovery",
"//pkg/kv/kvserver/loqrecovery:loqrecovery_test",
"//pkg/kv/kvserver/multiqueue:multiqueue",
Expand Down
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,4 +53,5 @@ type TestingKnobs struct {
ExternalConnection ModuleTestingKnobs
EventExporter ModuleTestingKnobs
EventLog ModuleTestingKnobs
LOQRecovery ModuleTestingKnobs
}
3 changes: 1 addition & 2 deletions pkg/cli/cliflags/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -1822,8 +1822,7 @@ See start --help for more flag details and examples.
}

ConfirmActions = FlagInfo{
Name: "confirm",
Shorthand: "p",
Name: "confirm",
Description: `
Confirm action:
<PRE>
Expand Down
113 changes: 81 additions & 32 deletions pkg/cli/debug_recover_loss_of_quorum.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,13 @@ Now the cluster could be started again.
RunE: UsageAndErr,
}

var recoverCommands = []*cobra.Command{
debugRecoverCollectInfoCmd,
debugRecoverPlanCmd,
//debugRecoverStagePlan,
//debugRecoverVerify,
}

func init() {
debugRecoverCmd.AddCommand(
debugRecoverCollectInfoCmd,
Expand All @@ -174,18 +181,28 @@ func init() {

var debugRecoverCollectInfoCmd = &cobra.Command{
Use: "collect-info [destination-file]",
Short: "collect replica information from the given stores",
Short: "collect replica information from a cluster",
Long: `
Collect information about replicas by reading data from underlying stores. Store
locations must be provided using --store flags.
Collect information about replicas in the cluster.
The command can collect data from an online or an offline cluster.
In the first case, the address of a single healthy cluster node must be provided
using the --host flag. This designated node will handle collection of data from
all surviving nodes.
In the second case data is read directly from local stores on each node.
CockroachDB must not be running on any node. The location of each store must be
provided using the --store flag. The command must be executed for all surviving
stores.
Multiple store locations can be provided to the command to collect all info
from all stores on a node at once. It is also possible to call it per store, in
that case all resulting files should be fed to the plan subcommand.
Collected information is written to a destination file if file name is provided,
or to stdout.
Multiple store locations could be provided to the command to collect all info from
node at once. It is also possible to call it per store, in that case all resulting
files should be fed to plan subcommand.
See debug recover command help for more details on how to use this command.
`,
Args: cobra.MaximumNArgs(1),
Expand All @@ -197,28 +214,49 @@ var debugRecoverCollectInfoOpts struct {
}

func runDebugDeadReplicaCollect(cmd *cobra.Command, args []string) error {
// We must have cancellable context here to obtain grpc client connection.
ctx, cancel := context.WithCancel(cmd.Context())
defer cancel()
stopper := stop.NewStopper()
defer stopper.Stop(cmd.Context())
defer stopper.Stop(ctx)

var replicaInfo loqrecoverypb.ClusterReplicaInfo
var stats loqrecovery.CollectionStats

var stores []storage.Engine
for _, storeSpec := range debugRecoverCollectInfoOpts.Stores.Specs {
db, err := OpenEngine(storeSpec.Path, stopper, storage.MustExist, storage.ReadOnly)
if len(debugRecoverCollectInfoOpts.Stores.Specs) == 0 {
c, finish, err := getAdminClient(ctx, serverCfg)
if err != nil {
return errors.Wrapf(err, "failed to open store at path %q, ensure that store path is "+
"correct and that it is not used by another process", storeSpec.Path)
return errors.Wrapf(err, "failed to get admin connection to cluster")
}
defer finish()
replicaInfo, stats, err = loqrecovery.CollectRemoteReplicaInfo(ctx, c)
if err != nil {
return errors.WithHint(errors.Wrap(err,
"failed to retrieve replica info from cluster"),
"Check cluster health and retry the operation.")
}
} else {
var stores []storage.Engine
for _, storeSpec := range debugRecoverCollectInfoOpts.Stores.Specs {
db, err := OpenEngine(storeSpec.Path, stopper, storage.MustExist, storage.ReadOnly)
if err != nil {
return errors.WithHint(errors.Wrapf(err,
"failed to open store at path %q", storeSpec.Path),
"Ensure that store path is correct and that it is not used by another process.")
}
stores = append(stores, db)
}
var err error
replicaInfo, stats, err = loqrecovery.CollectStoresReplicaInfo(ctx, stores)
if err != nil {
return errors.Wrapf(err, "failed to collect replica info from local stores")
}
stores = append(stores, db)
}

replicaInfo, err := loqrecovery.CollectReplicaInfo(cmd.Context(), stores)
if err != nil {
return err
}

var writer io.Writer = os.Stdout
if len(args) > 0 {
filename := args[0]
if _, err = os.Stat(filename); err == nil {
if _, err := os.Stat(filename); err == nil {
return errors.Newf("file %q already exists", filename)
}

Expand All @@ -230,14 +268,20 @@ func runDebugDeadReplicaCollect(cmd *cobra.Command, args []string) error {
writer = outFile
}
jsonpb := protoutil.JSONPb{Indent: " "}
var out []byte
if out, err = jsonpb.Marshal(replicaInfo); err != nil {
out, err := jsonpb.Marshal(&replicaInfo)
if err != nil {
return errors.Wrap(err, "failed to marshal collected replica info")
}
if _, err = writer.Write(out); err != nil {
if _, err := writer.Write(out); err != nil {
return errors.Wrap(err, "failed to write collected replica info")
}
_, _ = fmt.Fprintf(stderr, "Collected info about %d replicas.\n", len(replicaInfo.Replicas))
_, _ = fmt.Fprintf(stderr, `Collected recovery info from:
nodes %d
stores %d
Collected info:
replicas %d
range descriptors %d
`, stats.Nodes, stats.Stores, replicaInfo.ReplicaCount(), stats.Descriptors)
return nil
}

Expand Down Expand Up @@ -278,7 +322,7 @@ func runDebugPlanReplicaRemoval(cmd *cobra.Command, args []string) error {
deadStoreIDs = append(deadStoreIDs, roachpb.StoreID(id))
}

plan, report, err := loqrecovery.PlanReplicas(cmd.Context(), replicas, deadStoreIDs)
plan, report, err := loqrecovery.PlanReplicas(cmd.Context(), replicas.LocalInfo, deadStoreIDs)
if err != nil {
return err
}
Expand Down Expand Up @@ -377,7 +421,7 @@ Discarded live replicas: %d

jsonpb := protoutil.JSONPb{Indent: " "}
var out []byte
if out, err = jsonpb.Marshal(plan); err != nil {
if out, err = jsonpb.Marshal(&plan); err != nil {
return errors.Wrap(err, "failed to marshal recovery plan")
}
if _, err = writer.Write(out); err != nil {
Expand All @@ -393,20 +437,25 @@ Discarded live replicas: %d
return nil
}

func readReplicaInfoData(fileNames []string) ([]loqrecoverypb.NodeReplicaInfo, error) {
var replicas []loqrecoverypb.NodeReplicaInfo
func readReplicaInfoData(fileNames []string) (loqrecoverypb.ClusterReplicaInfo, error) {
var replicas loqrecoverypb.ClusterReplicaInfo
for _, filename := range fileNames {
data, err := os.ReadFile(filename)
if err != nil {
return nil, errors.Wrapf(err, "failed to read replica info file %q", filename)
return loqrecoverypb.ClusterReplicaInfo{}, errors.Wrapf(err, "failed to read replica info file %q", filename)
}

var nodeReplicas loqrecoverypb.NodeReplicaInfo
var nodeReplicas loqrecoverypb.ClusterReplicaInfo
jsonpb := protoutil.JSONPb{}
if err = jsonpb.Unmarshal(data, &nodeReplicas); err != nil {
return nil, errors.Wrapf(err, "failed to unmarshal replica info from file %q", filename)
return loqrecoverypb.ClusterReplicaInfo{}, errors.WithHint(errors.Wrapf(err,
"failed to unmarshal replica info from file %q", filename),
"Ensure that replica info file is generated with the same binary version and file is not corrupted.")
}
if err = replicas.Merge(nodeReplicas); err != nil {
return loqrecoverypb.ClusterReplicaInfo{}, errors.Wrapf(err,
"failed to merge replica info from file %q", filename)
}
replicas = append(replicas, nodeReplicas)
}
return replicas, nil
}
Expand Down
Loading

0 comments on commit aaba7ad

Please sign in to comment.