diff --git a/etcdctl/README.md b/etcdctl/README.md index 3589f4d699d6..7b4bd2908d13 100644 --- a/etcdctl/README.md +++ b/etcdctl/README.md @@ -1129,6 +1129,43 @@ DOWNGRADE CANCEL cancels the ongoing downgrade action to cluster. ./etcdctl downgrade cancel Downgrade cancel success, cluster version 3.5 ``` +### DIAGNOSIS + +`etcdctl diagnosis [flags]` - Collects and analyzes troubleshooting data from a running etcd cluster. + +The `diagnosis` command gathers a concise set of diagnostic details from each cluster member by performing several checks, including: + + * **Membership checks**: Verifies the cluster membership information. + * **Endpoint status**: Retrieves the status of each endpoint. + * **Serializable and linearizable reads**: Performs read operations to validate data consistency. + * **Metrics snapshot**: Collects a small snapshot of key metrics. + +#### Flags + +- `--cluster`: use all endpoints discovered from the cluster member list. +- `--etcd-storage-quota-bytes`: expected etcd storage quota in bytes (value passed to etcd with `--quota-backend-bytes`). +- `-o, --output`: optional file path to write the JSON report; by default the report is written to stdout. Logs are written to stderr. + +Global flags (like `--endpoints`, TLS, auth, and timeouts) are shared with other `etcdctl` commands. See `etcdctl options` for the full list. + +#### Examples + +To perform analysis of a running etcd cluster, you can use the following command. This will collect and analyze data from all specified endpoints. + +```bash +etcdctl diagnosis --endpoints=https://10.0.1.10:2379,https://10.0.1.11:2379,https://10.0.1.12:2379 \ + --cacert ./ca.crt --key ./etcd-diagnosis.key --cert ./etcd-diagnosis.crt + +# Use cluster-discovered endpoints +etcdctl diagnosis --cluster + +# Write report to a file (logs still go to stderr) +etcdctl diagnosis -o report.json +``` + + +Example output: see [ctlv3/command/diagnosis/examples/etcd_diagnosis_report.json](ctlv3/command/diagnosis/examples/etcd_diagnosis_report.json) + ## Concurrency commands diff --git a/etcdctl/ctlv3/command/diagnosis/engine/diagnosis.go b/etcdctl/ctlv3/command/diagnosis/engine/diagnosis.go new file mode 100644 index 000000000000..48c4c78ecfc6 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/engine/diagnosis.go @@ -0,0 +1,40 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package engine + +import ( + "encoding/json" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" +) + +type report struct { + Input any `json:"input,omitempty"` + Results []any `json:"results,omitempty"` +} + +// Diagnose runs all provided plugins and returns a JSON report. +// It logs plugin progress and individual results to stderr. +func Diagnose(input any, plugins []intf.Plugin) ([]byte, error) { + rp := report{ + Input: input, + } + for _, plugin := range plugins { + result := plugin.Diagnose() + rp.Results = append(rp.Results, result) + } + + return json.MarshalIndent(rp, "", "\t") +} diff --git a/etcdctl/ctlv3/command/diagnosis/engine/intf/plugin.go b/etcdctl/ctlv3/command/diagnosis/engine/intf/plugin.go new file mode 100644 index 000000000000..65b3b6d95a39 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/engine/intf/plugin.go @@ -0,0 +1,31 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package intf + +type Plugin interface { + // Name returns the name of the plugin + Name() string + // Diagnose performs diagnosis and returns the result. If it fails + // to do the diagnosis for any reason, it gets the detailed reason + // included in the diagnosis result. + Diagnose() any +} + +// FailedResult is the result returned by a plugin if it fails to +// perform the diagnosis for any reason. +type FailedResult struct { + Name string `json:"name"` + Reason string `json:"reason"` +} diff --git a/etcdctl/ctlv3/command/diagnosis/examples/etcd_diagnosis_report.json b/etcdctl/ctlv3/command/diagnosis/examples/etcd_diagnosis_report.json new file mode 100644 index 000000000000..4eeb2030f93d --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/examples/etcd_diagnosis_report.json @@ -0,0 +1,181 @@ +{ + "input": { + "endpoints": [ + "http://127.0.0.1:2379" + ], + "useClusterEndpoints": true, + "dial-timeout": 2000000000, + "command-timeout": 5000000000, + "keep-alive-time": 2000000000, + "keep-alive-timeout": 5000000000, + "insecure": true, + "insecure-discovery": true, + "db-quota-bytes": 2147483648 + }, + "results": [ + { + "name": "membershipChecker", + "memberList": { + "header": { + "cluster_id": 17237436991929493444, + "member_id": 9372538179322589801, + "raft_term": 2 + }, + "members": [ + { + "ID": 9372538179322589801, + "name": "infra1", + "peerURLs": [ + "http://127.0.0.1:12380" + ], + "clientURLs": [ + "http://127.0.0.1:2379" + ] + }, + { + "ID": 10501334649042878790, + "name": "infra2", + "peerURLs": [ + "http://127.0.0.1:22380" + ], + "clientURLs": [ + "http://127.0.0.1:22379" + ] + }, + { + "ID": 18249187646912138824, + "name": "infra3", + "peerURLs": [ + "http://127.0.0.1:32380" + ], + "clientURLs": [ + "http://127.0.0.1:32379" + ] + } + ] + } + }, + { + "name": "epStatusChecker", + "summary": [ + "Successful" + ], + "epStatusList": [ + { + "endpoint": "http://127.0.0.1:2379", + "epStatus": { + "header": { + "cluster_id": 17237436991929493444, + "member_id": 9372538179322589801, + "revision": 1, + "raft_term": 2 + }, + "version": "3.5.9", + "dbSize": 98304, + "leader": 18249187646912138824, + "raftIndex": 8, + "raftTerm": 2, + "raftAppliedIndex": 8, + "dbSizeInUse": 98304 + } + }, + { + "endpoint": "http://127.0.0.1:22379", + "epStatus": { + "header": { + "cluster_id": 17237436991929493444, + "member_id": 10501334649042878790, + "revision": 1, + "raft_term": 2 + }, + "version": "3.5.9", + "dbSize": 98304, + "leader": 18249187646912138824, + "raftIndex": 8, + "raftTerm": 2, + "raftAppliedIndex": 8, + "dbSizeInUse": 98304 + } + }, + { + "endpoint": "http://127.0.0.1:32379", + "epStatus": { + "header": { + "cluster_id": 17237436991929493444, + "member_id": 18249187646912138824, + "revision": 1, + "raft_term": 2 + }, + "version": "3.5.9", + "dbSize": 98304, + "leader": 18249187646912138824, + "raftIndex": 8, + "raftTerm": 2, + "raftAppliedIndex": 8, + "dbSizeInUse": 98304 + } + } + ] + }, + { + "name": "serializableReadChecker", + "summary": "Successful", + "readResponses": [ + { + "endpoint": "http://127.0.0.1:2379", + "took": "686.5µs" + }, + { + "endpoint": "http://127.0.0.1:22379", + "took": "1.129291ms" + }, + { + "endpoint": "http://127.0.0.1:32379", + "took": "1.034625ms" + } + ] + }, + { + "name": "linearizableReadChecker", + "summary": "Successful", + "readResponses": [ + { + "endpoint": "http://127.0.0.1:2379", + "took": "1.286333ms" + }, + { + "endpoint": "http://127.0.0.1:22379", + "took": "890.417µs" + }, + { + "endpoint": "http://127.0.0.1:32379", + "took": "1.257791ms" + } + ] + }, + { + "name": "metricsChecker", + "summary": [ + "Successful" + ], + "epMetricsList": [ + { + "endpoint": "http://127.0.0.1:2379", + "took": "3.752625ms", + "epMetrics": { + "etcd_disk_backend_commit_duration_seconds_bucket": [ + "etcd_disk_backend_commit_duration_seconds_bucket{le=\"0.001\"} 0" + ], + "etcd_disk_wal_fsync_duration_seconds_bucket": [ + "etcd_disk_wal_fsync_duration_seconds_bucket{le=\"0.001\"} 0" + ], + "etcd_network_peer_round_trip_time_seconds_bucket": [ + "etcd_network_peer_round_trip_time_seconds_bucket{To=\"91bc3c398fb3c146\",le=\"0.0001\"} 2" + ], + "process_resident_memory_bytes": null + } + } + ] + } + ] +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/common/checker.go b/etcdctl/ctlv3/command/diagnosis/plugins/common/checker.go new file mode 100644 index 000000000000..f59f1f5e9cad --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/common/checker.go @@ -0,0 +1,32 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "time" + + clientv3 "go.etcd.io/etcd/client/v3" +) + +// Checker carries shared configuration for diagnosis plugins. +// It embeds generic options such as the etcd client configuration, +// resolved endpoints, and command timeout. +type Checker struct { + Cfg *clientv3.ConfigSpec + Endpoints []string + CommandTimeout time.Duration + DbQuotaBytes int + Name string +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/common/client.go b/etcdctl/ctlv3/command/diagnosis/plugins/common/client.go new file mode 100644 index 000000000000..76c4c4db05e8 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/common/client.go @@ -0,0 +1,40 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "go.uber.org/zap" + + "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" +) + +// NewClient creates an etcd client from the given configuration spec. +func NewClient(cfg *clientv3.ConfigSpec) (*clientv3.Client, error) { + lg, _ := logutil.CreateDefaultZapLogger(zap.InfoLevel) + cliCfg, err := clientv3.NewClientConfig(cfg, lg) + if err != nil { + return nil, err + } + return clientv3.New(*cliCfg) +} + +// ConfigWithEndpoint returns a shallow copy of cfg with Endpoints set to the +// provided single endpoint. +func ConfigWithEndpoint(cfg *clientv3.ConfigSpec, ep string) *clientv3.ConfigSpec { + c := *cfg + c.Endpoints = []string{ep} + return &c +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/epstatus/plugin.go b/etcdctl/ctlv3/command/diagnosis/plugins/epstatus/plugin.go new file mode 100644 index 000000000000..a83cfd6e928c --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/epstatus/plugin.go @@ -0,0 +1,188 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package epstatus + +import ( + "context" + "fmt" + "log" + "time" + + clientv3 "go.etcd.io/etcd/client/v3" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/common" +) + +type epStatusChecker struct { + common.Checker +} + +type epStatus struct { + Endpoint string `json:"endpoint,omitempty"` + EpStatus *clientv3.StatusResponse `json:"epStatus,omitempty"` +} + +type checkResult struct { + Name string `json:"name,omitempty"` + Summary []string `json:"summary,omitempty"` + EpStatusList []epStatus `json:"epStatusList,omitempty"` +} + +func NewPlugin(cfg *clientv3.ConfigSpec, eps []string, timeout time.Duration, dbQuota int) intf.Plugin { + return &epStatusChecker{ + Checker: common.Checker{ + Cfg: cfg, + Endpoints: eps, + CommandTimeout: timeout, + DbQuotaBytes: dbQuota, + Name: "epStatusChecker", + }, + } +} + +func (ck *epStatusChecker) Name() string { + return ck.Checker.Name +} + +func (ck *epStatusChecker) Diagnose() (result any) { + var err error + eps := ck.Endpoints + + defer func() { + if err != nil { + result = &intf.FailedResult{ + Name: ck.Name(), + Reason: err.Error(), + } + } + }() + + var ( + maxRetries = 3 + retries = 0 + shouldRetry = true + + chkResult = initCheckResult(ck.Name(), len(eps)) + ) + + for { + for i, ep := range eps { + chkResult.EpStatusList[i].Endpoint = ep + + cfg := common.ConfigWithEndpoint(ck.Cfg, ep) + c, err := common.NewClient(cfg) + if err != nil { + appendSummary(&chkResult, "Failed to create client for %q: %v", ep, err) + continue + } + ctx, cancel := context.WithTimeout(context.Background(), ck.CommandTimeout) + chkResult.EpStatusList[i].EpStatus, err = c.Status(ctx, ep) + cancel() + c.Close() + if err != nil { + appendSummary(&chkResult, "Failed to get endpoint status from %q: %v", ep, err) + continue + } + + if len(chkResult.EpStatusList[i].EpStatus.Errors) > 0 { + appendSummary(&chkResult, "Detected errors in endpoint %q: %v\n", ep, chkResult.EpStatusList[i].EpStatus.Errors) + shouldRetry = false + continue + } + + if i > 0 { + if !compareHardInfo(chkResult.EpStatusList[0].EpStatus, chkResult.EpStatusList[i].EpStatus) { + appendSummary(&chkResult, "Detected inconsistent hard endpoint info between %q and %q\n", eps[0], eps[i]) + shouldRetry = false + } + + if !shouldRetry { + continue + } + + if !compareSoftInfo(chkResult.EpStatusList[0].EpStatus, chkResult.EpStatusList[i].EpStatus) { + appendSummary(&chkResult, "Detected inconsistent soft endpoint info between %q and %q\n", eps[0], eps[i]) + } + } + } + + retries++ + + if len(chkResult.Summary) == 0 || !shouldRetry || retries >= maxRetries { + break + } + + chkResult = initCheckResult(ck.Name(), len(eps)) + log.Printf("Retrying checking endpoint status: %d/%d\n", retries, maxRetries) + time.Sleep(time.Second) + } + + checkDBSize(&chkResult, ck.DbQuotaBytes) + + if len(chkResult.Summary) == 0 { + chkResult.Summary = []string{"Successful"} + } + + result = chkResult + return result +} + +func initCheckResult(name string, epCount int) checkResult { + return checkResult{ + Name: name, + Summary: []string{}, + EpStatusList: make([]epStatus, epCount), + } +} + +func appendSummary(chkResult *checkResult, format string, v ...any) { + errMsg := fmt.Sprintf(format, v...) + log.Println(errMsg) + chkResult.Summary = append(chkResult.Summary, errMsg) +} + +func compareHardInfo(s1, s2 *clientv3.StatusResponse) bool { + if s1 == nil || s2 == nil { + return false + } + return s1.Header.ClusterId == s2.Header.ClusterId && + s1.Version == s2.Version && + s1.StorageVersion == s2.StorageVersion +} + +func compareSoftInfo(s1, s2 *clientv3.StatusResponse) bool { + if s1 == nil || s2 == nil { + return false + } + return s1.Header.Revision == s2.Header.Revision && + s1.RaftTerm == s2.RaftTerm && + s1.RaftIndex == s2.RaftIndex && + s1.RaftAppliedIndex == s2.RaftAppliedIndex && + s1.Leader == s2.Leader +} + +func checkDBSize(chkResult *checkResult, dbQuota int) { + for _, sts := range chkResult.EpStatusList { + if sts.EpStatus == nil { + continue + } + + freeSize := sts.EpStatus.DbSize - sts.EpStatus.DbSizeInUse + if freeSize > sts.EpStatus.DbSizeInUse && freeSize > 1_000_000_000 /* about 1GB */ || sts.EpStatus.DbSize >= int64(dbQuota*80/100) { + appendSummary(chkResult, "Detected large amount of db [free] space for endpoint %q, dbQuota: %d, dbSize: %d, dbSizeInUse: %d, dbSizeFree: %d", sts.Endpoint, dbQuota, sts.EpStatus.DbSize, sts.EpStatus.DbSizeInUse, freeSize) + } + } +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/membership/plugin.go b/etcdctl/ctlv3/command/diagnosis/plugins/membership/plugin.go new file mode 100644 index 000000000000..a9c605d89c1b --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/membership/plugin.go @@ -0,0 +1,119 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package membership + +import ( + "context" + "log" + "reflect" + "time" + + clientv3 "go.etcd.io/etcd/client/v3" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/common" +) + +type membershipChecker struct { + common.Checker +} + +type checkResult struct { + Name string `json:"name,omitempty"` + Summary string `json:"summary,omitempty"` + MemberList *clientv3.MemberListResponse `json:"memberList,omitempty"` + AllMemberLists []*clientv3.MemberListResponse `json:"allMemberLists,omitempty"` +} + +func NewPlugin(cfg *clientv3.ConfigSpec, eps []string, timeout time.Duration) intf.Plugin { + return &membershipChecker{ + Checker: common.Checker{ + Cfg: cfg, + Endpoints: eps, + CommandTimeout: timeout, + Name: "membershipChecker", + }, + } +} + +func (ck *membershipChecker) Name() string { + return ck.Checker.Name +} + +func (ck *membershipChecker) Diagnose() (result any) { + var err error + eps := ck.Endpoints + + defer func() { + if err != nil { + result = &intf.FailedResult{ + Name: ck.Name(), + Reason: err.Error(), + } + } + }() + + memberLists := make([]*clientv3.MemberListResponse, len(eps)) + detectedInconsistency := false + for i, ep := range eps { + cfg := common.ConfigWithEndpoint(ck.Cfg, ep) + c, err := common.NewClient(cfg) + if err != nil { + detectedInconsistency = true + log.Printf("Failed to create client for %q: %v\n", ep, err) + continue + } + ctx, cancel := context.WithTimeout(context.Background(), ck.CommandTimeout) + memberLists[i], err = c.MemberList(ctx, clientv3.WithSerializable()) + cancel() + c.Close() + if err != nil { + detectedInconsistency = true + log.Printf("Failed to get member list from %q: %v\n", ep, err) + continue + } + + if i > 0 { + if !compareMembers(memberLists[0], memberLists[i]) { + detectedInconsistency = true + log.Printf("Detected inconsistent member list between %q and %q\n", eps[0], eps[i]) + } + } + } + + if detectedInconsistency { + result = checkResult{ + Name: ck.Name(), + Summary: "Detected inconsistent member list between different members", + AllMemberLists: memberLists, + } + } else { + result = checkResult{ + Name: ck.Name(), + Summary: "Successful", + MemberList: memberLists[0], + } + } + + return result +} + +func compareMembers(m1, m2 *clientv3.MemberListResponse) bool { + if m1 == nil || m2 == nil { + return false + } + + return m1.Header.ClusterId == m2.Header.ClusterId && reflect.DeepEqual(m1.Members, m2.Members) +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/metrics/plugin.go b/etcdctl/ctlv3/command/diagnosis/plugins/metrics/plugin.go new file mode 100644 index 000000000000..911791de0899 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/metrics/plugin.go @@ -0,0 +1,179 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "crypto/tls" + "crypto/x509" + "fmt" + "io" + "log" + "net/http" + "net/url" + "os" + "strings" + "time" + + clientv3 "go.etcd.io/etcd/client/v3" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/common" +) + +var metricsNames = []string{ + "etcd_disk_wal_fsync_duration_seconds_bucket", + "etcd_disk_backend_commit_duration_seconds_bucket", + "etcd_network_peer_round_trip_time_seconds_bucket", + "process_resident_memory_bytes", + //"process_cpu_seconds_total", +} + +type metricsChecker struct { + common.Checker +} + +type epMetrics struct { + Endpoint string `json:"endpoint,omitempty"` + Took string `json:"took,omitempty"` + EpMetrics map[string][]string `json:"epMetrics,omitempty"` +} + +type checkResult struct { + Name string `json:"name,omitempty"` + Summary []string `json:"summary,omitempty"` + EpMetricsList []epMetrics `json:"epMetricsList,omitempty"` +} + +func NewPlugin(cfg *clientv3.ConfigSpec, eps []string, timeout time.Duration) intf.Plugin { + return &metricsChecker{ + Checker: common.Checker{ + Cfg: cfg, + Endpoints: eps, + CommandTimeout: timeout, + Name: "metricsChecker", + }, + } +} + +func (ck *metricsChecker) Name() string { + return ck.Checker.Name +} + +func (ck *metricsChecker) Diagnose() (result any) { + var err error + eps := ck.Endpoints + + defer func() { + if err != nil { + result = &intf.FailedResult{ + Name: ck.Name(), + Reason: err.Error(), + } + } + }() + + chkResult := checkResult{ + Name: ck.Name(), + Summary: []string{}, + EpMetricsList: make([]epMetrics, len(eps)), + } + + for i, ep := range eps { + chkResult.EpMetricsList[i].Endpoint = ep + + startTs := time.Now() + allMetrics, err := fetchMetrics(ck.Cfg, ep, ck.CommandTimeout) + chkResult.EpMetricsList[i].Took = time.Since(startTs).String() + if err != nil { + appendSummary(&chkResult, "Failed to get endpoint metrics from %q: %v", ep, err) + continue + } + + metricsMap := map[string][]string{} + for _, prefix := range metricsNames { + ret := metrics(allMetrics, prefix) + metricsMap[prefix] = ret + } + + chkResult.EpMetricsList[i].EpMetrics = metricsMap + } + + if len(chkResult.Summary) == 0 { + chkResult.Summary = []string{"Successful"} + } + + result = chkResult + return result +} + +func metrics(lines []string, prefix string) []string { + var ret []string + for _, line := range lines { + if strings.HasPrefix(line, prefix) { + ret = append(ret, line) + } + } + return ret +} + +func appendSummary(chkResult *checkResult, format string, v ...any) { + errMsg := fmt.Sprintf(format, v...) + log.Println(errMsg) + chkResult.Summary = append(chkResult.Summary, errMsg) +} + +func fetchMetrics(cfg *clientv3.ConfigSpec, ep string, timeout time.Duration) ([]string, error) { + if !strings.HasPrefix(ep, "http://") && !strings.HasPrefix(ep, "https://") { + ep = "http://" + ep + } + urlPath, err := url.JoinPath(ep, "metrics") + if err != nil { + return nil, fmt.Errorf("failed to join metrics url path: %w", err) + } + + client := &http.Client{Timeout: timeout} + if strings.HasPrefix(urlPath, "https://") && cfg.Secure != nil { + cert, err := tls.LoadX509KeyPair(cfg.Secure.Cert, cfg.Secure.Key) + if err != nil { + return nil, fmt.Errorf("failed to load certificate: %w", err) + } + caCert, err := os.ReadFile(cfg.Secure.Cacert) + if err != nil { + return nil, fmt.Errorf("failed to load CA: %w", err) + } + caCertPool := x509.NewCertPool() + caCertPool.AppendCertsFromPEM(caCert) + tr := &http.Transport{ + TLSClientConfig: &tls.Config{ + Certificates: []tls.Certificate{cert}, + RootCAs: caCertPool, + InsecureSkipVerify: cfg.Secure.InsecureSkipVerify, + }, + } + client.Transport = tr + } + resp, err := client.Get(urlPath) + if err != nil { + return nil, fmt.Errorf("http get failed: %w", err) + } + defer resp.Body.Close() + + data, err := io.ReadAll(resp.Body) + if err != nil { + return nil, fmt.Errorf("failed to read metrics response: %w", err) + } + + return strings.Split(string(data), "\n"), nil +} diff --git a/etcdctl/ctlv3/command/diagnosis/plugins/read/plugin.go b/etcdctl/ctlv3/command/diagnosis/plugins/read/plugin.go new file mode 100644 index 000000000000..5d7bcec71496 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis/plugins/read/plugin.go @@ -0,0 +1,146 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package read + +import ( + "context" + "errors" + "log" + "time" + + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/common" +) + +type readChecker struct { + common.Checker + linearizable bool +} + +type readResponse struct { + Endpoint string `json:"endpoint,omitempty"` + Took string `json:"took,omitempty"` + Error string `json:"error,omitempty"` +} +type checkResult struct { + Name string `json:"name,omitempty"` + Summary string `json:"summary,omitempty"` + ReadResponses []readResponse `json:"readResponses,omitempty"` +} + +func NewPlugin(cfg *clientv3.ConfigSpec, eps []string, timeout time.Duration, linearizable bool) intf.Plugin { + return &readChecker{ + Checker: common.Checker{ + Cfg: cfg, + Endpoints: eps, + CommandTimeout: timeout, + Name: generateName(linearizable), + }, + linearizable: linearizable, + } +} + +func (ck *readChecker) Name() string { + return ck.Checker.Name +} + +func generateName(linearizable bool) string { + if linearizable { + return "linearizableReadChecker" + } + return "serializableReadChecker" +} + +func (ck *readChecker) Diagnose() (result any) { + var err error + eps := ck.Endpoints + + defer func() { + if err != nil { + result = &intf.FailedResult{ + Name: ck.Name(), + Reason: err.Error(), + } + } + }() + + var ( + maxRetries = 3 + retries = 0 + + chkResult = initCheckResult(ck.Name(), len(eps)) + ) + + for { + shouldRetry := false + for i, ep := range eps { + chkResult.ReadResponses[i].Endpoint = ep + + startTs := time.Now() + cfg := common.ConfigWithEndpoint(ck.Cfg, ep) + c, err := common.NewClient(cfg) + if err != nil { + chkResult.ReadResponses[i].Error = err.Error() + shouldRetry = true + continue + } + ctx, cancel := context.WithTimeout(context.Background(), ck.CommandTimeout) + if ck.linearizable { + _, err = c.Get(ctx, "health") + } else { + _, err = c.Get(ctx, "health", clientv3.WithSerializable()) + } + cancel() + c.Close() + if err != nil && !errors.Is(err, rpctypes.ErrPermissionDenied) { + chkResult.ReadResponses[i].Error = err.Error() + shouldRetry = true + } + chkResult.ReadResponses[i].Took = time.Since(startTs).String() + } + + retries++ + + if !shouldRetry || retries >= maxRetries { + break + } + + chkResult = initCheckResult(ck.Name(), len(eps)) + log.Printf("Retrying checking read: %d/%d\n", retries, maxRetries) + time.Sleep(time.Second) + } + + chkResult.Summary = "Successful" + for _, resp := range chkResult.ReadResponses { + if len(resp.Error) > 0 { + chkResult.Summary = "Unsuccessful" + break + } + } + + result = chkResult + return result +} + +func initCheckResult(name string, epCount int) checkResult { + return checkResult{ + Name: name, + Summary: "", + ReadResponses: make([]readResponse, epCount), + } +} diff --git a/etcdctl/ctlv3/command/diagnosis_command.go b/etcdctl/ctlv3/command/diagnosis_command.go new file mode 100644 index 000000000000..4e62c2afc854 --- /dev/null +++ b/etcdctl/ctlv3/command/diagnosis_command.go @@ -0,0 +1,105 @@ +// Copyright 2025 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package command + +import ( + "fmt" + "os" + + "github.com/spf13/cobra" + + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/engine/intf" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/epstatus" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/membership" + "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/metrics" + readplugin "go.etcd.io/etcd/etcdctl/v3/ctlv3/command/diagnosis/plugins/read" + "go.etcd.io/etcd/pkg/v3/cobrautl" +) + +var ( + useCluster bool + dbQuotaBytes int + outputFile string +) + +// NewDiagnosisCommand returns the cobra command for "diagnosis". +func NewDiagnosisCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "diagnosis", + Short: "One-stop etcd diagnosis tool", + Run: runDiagnosis, + GroupID: groupClusterMaintenanceID, + } + + cmd.Flags().BoolVar(&useCluster, "cluster", false, "use all endpoints from the cluster member list") + cmd.Flags().IntVar(&dbQuotaBytes, "etcd-storage-quota-bytes", 2*1024*1024*1024, "etcd storage quota in bytes (the value passed to etcd instance by flag --quota-backend-bytes)") + cmd.Flags().StringVarP(&outputFile, "output", "o", "", "write report to file instead of stdout") + + return cmd +} + +func runDiagnosis(cmd *cobra.Command, args []string) { + cfg := clientConfigFromCmd(cmd) + cli := mustClientFromCmd(cmd) + defer cli.Close() + + eps := cfg.Endpoints + if useCluster { + ctx, cancel := commandCtx(cmd) + members, err := cli.MemberList(ctx) + cancel() + if err != nil { + fmt.Fprintf(os.Stderr, "failed to fetch member list: %v\n", err) + os.Exit(cobrautl.ExitError) + } + var clusterEps []string + for _, m := range members.Members { + clusterEps = append(clusterEps, m.ClientURLs...) + } + eps = clusterEps + cfg.Endpoints = eps + } + + timeout, err := cmd.Flags().GetDuration("command-timeout") + if err != nil { + fmt.Fprintf(os.Stderr, "failed to get command-timeout: %v\n", err) + os.Exit(cobrautl.ExitError) + } + + plugins := []intf.Plugin{ + membership.NewPlugin(cfg, eps, timeout), + epstatus.NewPlugin(cfg, eps, timeout, dbQuotaBytes), + readplugin.NewPlugin(cfg, eps, timeout, false), + readplugin.NewPlugin(cfg, eps, timeout, true), + metrics.NewPlugin(cfg, eps, timeout), + } + + report, err := engine.Diagnose(cfg, plugins) + if err != nil { + fmt.Fprintf(os.Stderr, "diagnosis failed: %v\n", err) + os.Exit(cobrautl.ExitError) + } + + if outputFile != "" { + if err := os.WriteFile(outputFile, report, 0o644); err != nil { + fmt.Fprintf(os.Stderr, "failed to write report: %v\n", err) + os.Exit(cobrautl.ExitError) + } + return + } + + fmt.Fprintln(os.Stdout, string(report)) +} diff --git a/etcdctl/ctlv3/ctl.go b/etcdctl/ctlv3/ctl.go index 039485ddf84d..2d382b5ee054 100644 --- a/etcdctl/ctlv3/ctl.go +++ b/etcdctl/ctlv3/ctl.go @@ -106,6 +106,7 @@ func init() { command.NewUserCommand(), command.NewRoleCommand(), command.NewCheckCommand(), + command.NewDiagnosisCommand(), command.NewCompletionCommand(), command.NewDowngradeCommand(), command.NewOptionsCommand(rootCmd),