Skip to content

Commit

Permalink
server: add debug/pprof/ui/cpu/?node=all endpoint for cluster-wide …
Browse files Browse the repository at this point in the history
…CPU profile

This change adds a debug endpoint to the status server that collects
a CPU profiles from all nodes in the cluster and merges them before
rendering the profile. The merge is performed using `pprof.Merge`
https://pkg.go.dev/github.com/google/pprof/profile#Merge.

A future patch will teach the jobs page to filter out samples from
this cluster-wide profile that are only labelled with the job's job ID.
In this way a user can get the cluster-wide CPU profile for a particular
job making it easier to identify the cause for a hotspot or a
performance regression.

Release note: None
  • Loading branch information
adityamaru committed May 11, 2023
1 parent a628ff8 commit ae19af3
Show file tree
Hide file tree
Showing 3 changed files with 109 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,7 @@ go_library(
"@com_github_cockroachdb_redact//:redact",
"@com_github_getsentry_sentry_go//:sentry-go",
"@com_github_gogo_protobuf//proto",
"@com_github_google_pprof//profile",
"@com_github_gorilla_mux//:mux",
"@com_github_grpc_ecosystem_grpc_gateway//runtime:go_default_library",
"@com_github_grpc_ecosystem_grpc_gateway//utilities:go_default_library",
Expand Down
103 changes: 103 additions & 0 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uint128"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/google/pprof/profile"
gwruntime "github.com/grpc-ecosystem/grpc-gateway/runtime"
raft "go.etcd.io/raft/v3"
"google.golang.org/grpc"
Expand Down Expand Up @@ -1464,6 +1465,102 @@ func (s *statusServer) Stacks(
return stacksLocal(req)
}

// fetchProfileFromAllNodes fetches the CPU profiles from all live nodes in the
// cluster and merges the samples across all profiles.
func (s *statusServer) fetchProfileFromAllNodes(
ctx context.Context, req *serverpb.ProfileRequest,
) (*serverpb.JSONResponse, error) {
type profData struct {
data []byte
err error
}
type profDataResponse struct {
profDataByNodeID map[roachpb.NodeID]*profData
}
response := profDataResponse{profDataByNodeID: make(map[roachpb.NodeID]*profData)}

resp, err := s.Node(ctx, &serverpb.NodeRequest{NodeId: "local"})
if err != nil {
return nil, err
}
senderServerVersion := resp.Desc.ServerVersion
dialFn := func(ctx context.Context, nodeID roachpb.NodeID) (interface{}, error) {
client, err := s.dialNode(ctx, nodeID)
return client, err
}
nodeFn := func(ctx context.Context, client interface{}, nodeID roachpb.NodeID) (interface{}, error) {
// pprof.Merge can only merge samples collected on the same binary version.
// So we must first check that the node is running the same binary version
// as the node that is going to merge the profiles.
statusClient := client.(serverpb.StatusClient)
resp, err := statusClient.Node(ctx, &serverpb.NodeRequest{NodeId: "local"})
if err != nil {
return nodeID, err
}
nodeServerVersion := resp.Desc.ServerVersion
if !senderServerVersion.Equal(nodeServerVersion) {
return nil, errors.Newf("node %d has a different server version %s to the node merging CPU profiles %s",
nodeID, nodeServerVersion.String(), senderServerVersion.String())
}
var pd *profData
err = contextutil.RunWithTimeout(ctx, "fetch cpu profile", 1*time.Minute, func(ctx context.Context) error {
log.Infof(ctx, "fetching a CPU profile for %d", resp.Desc.NodeID)
resp, err := statusClient.Profile(ctx, &serverpb.ProfileRequest{
NodeId: fmt.Sprintf("%d", nodeID),
Type: serverpb.ProfileRequest_CPU,
Seconds: req.Seconds,
Labels: req.Labels,
})
if err != nil {
return err
}
pd = &profData{data: resp.Data}
return nil
})
return pd, err
}
responseFn := func(nodeID roachpb.NodeID, resp interface{}) {
profResp := resp.(*profData)
response.profDataByNodeID[nodeID] = profResp
}
errorFn := func(nodeID roachpb.NodeID, err error) {
response.profDataByNodeID[nodeID] = &profData{err: err}
}
if err := s.iterateNodes(ctx, "cluster-wide CPU profile", dialFn, nodeFn, responseFn, errorFn); err != nil {
return nil, serverError(ctx, err)
}

profs := make([]*profile.Profile, 0, len(response.profDataByNodeID))
for nodeID, pd := range response.profDataByNodeID {
if len(pd.data) == 0 && pd.err == nil {
log.Warningf(ctx, "no profile collected for node %d", nodeID)
continue // skipped node
}

if pd.err != nil {
log.Warningf(ctx, "failed to collect profile for node %d: %v", nodeID, pd.err)
continue
}

p, err := profile.ParseData(pd.data)
if err != nil {
return nil, err
}
p.Comments = append(p.Comments, fmt.Sprintf("n%d", nodeID))
profs = append(profs, p)
}
mergedProfiles, err := profile.Merge(profs)
if err != nil {
return nil, err
}

var buf bytes.Buffer
if err := mergedProfiles.Write(&buf); err != nil {
return nil, status.Errorf(codes.Internal, err.Error())
}
return &serverpb.JSONResponse{Data: buf.Bytes()}, nil
}

// TODO(tschottdorf): significant overlap with /debug/pprof/heap, except that
// this one allows querying by NodeID.
//
Expand All @@ -1481,6 +1578,12 @@ func (s *statusServer) Profile(
return nil, err
}

// If the request is for "all" nodes then we collect profiles from all nodes
// in the cluster and merge them before returning to the user.
if req.NodeId == "all" {
return s.fetchProfileFromAllNodes(ctx, req)
}

nodeID, local, err := s.parseNodeID(req.NodeId)
if err != nil {
return nil, status.Errorf(codes.InvalidArgument, err.Error())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -445,6 +445,11 @@ export default function Debug() {
url="debug/pprof/ui/cpu/"
params={{ node: nodeID, seconds: "5", labels: "true" }}
/>
<DebugTableLink
name="Cluster-wide CPU Profile (profiles all nodes; MEMORY OVERHEAD)"
url="debug/pprof/ui/cpu/"
params={{ node: "all", seconds: "5", labels: "true" }}
/>
<DebugTableLink
name="Block"
url="debug/pprof/ui/block/"
Expand Down

0 comments on commit ae19af3

Please sign in to comment.