From 3f452202d26691963c2fd7c70e6a9616e51c8c58 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 22 Feb 2019 12:08:07 +0100 Subject: [PATCH 1/6] pprofui: allow hitting ui endpoints via `go tool pprof` Release note: None --- pkg/server/debug/pprofui/server.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/pkg/server/debug/pprofui/server.go b/pkg/server/debug/pprofui/server.go index b9f3d80a5dcc..000becc4aad1 100644 --- a/pkg/server/debug/pprofui/server.go +++ b/pkg/server/debug/pprofui/server.go @@ -215,8 +215,21 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { http.Error(w, err.Error(), http.StatusInternalServerError) return } + + // If this is a request issued by `go tool pprof`, just return the profile + // directly. This is convenient because it avoids having to expose the pprof + // endpoints separately, and also allows inserting hooks around CPU profiles + // in the future. + isGoPProf := strings.Contains(r.Header.Get("User-Agent"), "Go-http-client") origURL.Path = path.Join(origURL.Path, id, "flamegraph") - http.Redirect(w, r, origURL.String(), http.StatusTemporaryRedirect) + if !isGoPProf { + http.Redirect(w, r, origURL.String(), http.StatusTemporaryRedirect) + } else { + _ = s.storage.Get(id, func(r io.Reader) error { + _, err := io.Copy(w, r) + return err + }) + } } type fetcherFn func(_ string, _, _ time.Duration) (*profile.Profile, string, error) From 1d2c44084d2d06f5d3e7c681df427b99200957ae Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 22 Feb 2019 12:21:21 +0100 Subject: [PATCH 2/6] pprofui: add a hook around profile grabbing Release note: None --- pkg/server/debug/pprofui/server.go | 23 ++++++++++++++++++++--- pkg/server/debug/pprofui/server_test.go | 2 +- pkg/server/debug/server.go | 8 +++++++- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/pkg/server/debug/pprofui/server.go b/pkg/server/debug/pprofui/server.go index 000becc4aad1..9e52980923ed 100644 --- a/pkg/server/debug/pprofui/server.go +++ b/pkg/server/debug/pprofui/server.go @@ -44,12 +44,29 @@ type Server struct { storage Storage profileSem syncutil.Mutex profileTypes map[string]http.HandlerFunc + hook func(profile string, do func()) } -// NewServer creates a new Server backed by the supplied Storage. -func NewServer(storage Storage) *Server { +// NewServer creates a new Server backed by the supplied Storage and optionally +// a hook which is called when a new profile is created. The closure passed to +// the hook will carry out the work involved in creating the profile and must +// be called by the hook. The intention is that hook will be a method such as +// this: +// +// func hook(profile string, do func()) { +// if profile == "profile" { +// something.EnableProfilerLabels() +// defer something.DisableProfilerLabels() +// do() +// } +// } +func NewServer(storage Storage, hook func(profile string, do func())) *Server { + if hook == nil { + hook = func(_ string, do func()) { do() } + } s := &Server{ storage: storage, + hook: hook, } s.profileTypes = map[string]http.HandlerFunc{ @@ -195,7 +212,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { rw := &responseBridge{target: w} - fetchHandler(rw, req) + s.hook(profileName, func() { fetchHandler(rw, req) }) if rw.statusCode != http.StatusOK && rw.statusCode != 0 { return errors.Errorf("unexpected status: %d", rw.statusCode) diff --git a/pkg/server/debug/pprofui/server_test.go b/pkg/server/debug/pprofui/server_test.go index 56a921b48481..4b0677e58db0 100644 --- a/pkg/server/debug/pprofui/server_test.go +++ b/pkg/server/debug/pprofui/server_test.go @@ -24,7 +24,7 @@ import ( func TestServer(t *testing.T) { storage := NewMemStorage(1, 0) - s := NewServer(storage) + s := NewServer(storage, nil) for i := 0; i < 3; i++ { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index 984c115271c0..8e9124f9a4d5 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/pkg/errors" "github.com/rcrowley/go-metrics" "github.com/rcrowley/go-metrics/exp" @@ -118,7 +119,12 @@ func NewServer(st *cluster.Settings) *Server { } mux.HandleFunc("/debug/logspy", spy.handleDebugLogSpy) - ps := pprofui.NewServer(pprofui.NewMemStorage(1, 0)) + ps := pprofui.NewServer(pprofui.NewMemStorage(1, 0), func(profile string, do func()) { + tBegin := timeutil.Now() + log.Infof(context.Background(), "pprofui: recording %s", profile) + do() + log.Infof(context.Background(), "pprofui: recorded %s in %.2fs", profile, timeutil.Since(tBegin).Seconds()) + }) mux.Handle("/debug/pprof/ui/", http.StripPrefix("/debug/pprof/ui", ps)) return &Server{ From 78522c3624443a99a6eb8c482a110bb6854e13bf Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 22 Feb 2019 12:29:31 +0100 Subject: [PATCH 3/6] debug: let Settings know when CPU profile is active Release note: None --- pkg/server/debug/server.go | 12 +++++++++++- pkg/settings/cluster/settings.go | 21 +++++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index 8e9124f9a4d5..e7f20aaab2ff 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -121,8 +121,18 @@ func NewServer(st *cluster.Settings) *Server { ps := pprofui.NewServer(pprofui.NewMemStorage(1, 0), func(profile string, do func()) { tBegin := timeutil.Now() - log.Infof(context.Background(), "pprofui: recording %s", profile) + + extra := "" + if profile == "profile" { + extra = " (enabling profiler labels)" + log.Infof(context.Background(), "%s", profile) + st.SetCPUProfiling(true) + defer st.SetCPUProfiling(false) + } + log.Infof(context.Background(), "pprofui: recording %s%s", profile, extra) + do() + log.Infof(context.Background(), "pprofui: recorded %s in %.2fs", profile, timeutil.Since(tBegin).Seconds()) }) mux.Handle("/debug/pprof/ui/", http.StripPrefix("/debug/pprof/ui", ps)) diff --git a/pkg/settings/cluster/settings.go b/pkg/settings/cluster/settings.go index a30b50f33abc..3aaafea5dc3a 100644 --- a/pkg/settings/cluster/settings.go +++ b/pkg/settings/cluster/settings.go @@ -71,6 +71,27 @@ type Settings struct { ExternalIODir string Initialized bool + + // Set to 1 if a profile is active (if the profile is being grabbed through + // the `pprofui` server as opposed to the raw endpoint). + cpuProfiling int32 // atomic +} + +// IsCPUProfiling returns true if a pprofui CPU profile is being recorded. This can +// be used by moving parts across the system to add profiler labels which are +// too expensive to be enabled at all times. +func (s *Settings) IsCPUProfiling() bool { + return atomic.LoadInt32(&s.cpuProfiling) == 1 +} + +// SetCPUProfiling is called from the pprofui to inform the system that a CPU +// profile is being recorded. +func (s *Settings) SetCPUProfiling(to bool) { + i := int32(0) + if to { + i = 1 + } + atomic.StoreInt32(&s.cpuProfiling, i) } // NoSettings is used when a func requires a Settings but none is available From a0b98e46c35660b1c53acaee26ee85bea147ef46 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 22 Feb 2019 12:34:54 +0100 Subject: [PATCH 4/6] sql: add profiler labels during CPU profiling When pprofui CPU profiling is active, add the statement tag and anonymized statement string to the goroutine labels. For example, this is what you can see when running ./bin/workload run kv --read-percent 50 --init ``` $ pprof -seconds 10 http://localhost:8080/debug/pprof/ui/profile [...] (pprof) tags stmt.anonymized: Total 7.9s 4.0s (50.57%): UPSERT INTO kv(k, v) VALUES ($1, $2) 3.9s (49.43%): SELECT k, v FROM kv WHERE k IN ($1,) stmt.tag: Total 7.9s 4.0s (50.57%): INSERT 3.9s (49.43%): SELECT ``` The dot graphs are similarly annotated, though they require `dot` to be installed on the machine and thus won't be as useful on the pprofui itself. Profile tags are not propagated across RPC boundaries. That is, a node may have high CPU as a result of SQL queries not originating at the node itself, and no labels will be available. But perusing this diff, you may notice that any moving part in the system can sniff whether profiling is active, and can add labels in itself, so in principle we could add the application name or any other information that is propagated along with the transaction on the recipient node and track down problems that way. We may also be able to add tags based on RangeIDs to identify ranges which cause high CPU load. The possibilities are endless, and with this infra in place, it's trivial to quickly iterate on what's useful. Closes #30930. Release note (admin ui change): Running nodes can now be CPU profiled in a way that breaks down CPU usage by query (some restrictions apply). --- pkg/sql/conn_executor_exec.go | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index f3828b35a51a..b4a17206f67b 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 The Cockroach Authors. +// Copyright 2018 The Cockroach Authors.D // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ package sql import ( "context" "fmt" + "runtime/pprof" "strings" "time" @@ -90,7 +91,17 @@ func (ex *connExecutor) execStmt( case stateNoTxn: ev, payload = ex.execStmtInNoTxnState(ctx, stmt) case stateOpen: - ev, payload, err = ex.execStmtInOpenState(ctx, stmt, pinfo, res) + if ex.server.cfg.Settings.IsCPUProfiling() { + labels := pprof.Labels( + "stmt.tag", stmt.AST.StatementTag(), + "stmt.anonymized", stmt.AnonymizedStr, + ) + pprof.Do(ctx, labels, func(ctx context.Context) { + ev, payload, err = ex.execStmtInOpenState(ctx, stmt, pinfo, res) + }) + } else { + ev, payload, err = ex.execStmtInOpenState(ctx, stmt, pinfo, res) + } switch ev.(type) { case eventNonRetriableErr: ex.recordFailure() From 76c40720b106c984535644bc266ae71e8fb8a4fd Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 22 Feb 2019 12:50:04 +0100 Subject: [PATCH 5/6] ui: consolidate profiling debug links The UI URLs also work for pprof and have the nice side effect of letting the node add extra profiling information as applicable. Exposing the raw endpoints would make this likely to be forgotten. The raw endpoints are still there, so they can be used if necessary for some reason (that reason would constitute a bug). Release note: None --- pkg/server/debug/server.go | 1 - pkg/ui/src/views/reports/containers/debug/index.tsx | 12 ++++-------- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index e7f20aaab2ff..4dea33053bf0 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -125,7 +125,6 @@ func NewServer(st *cluster.Settings) *Server { extra := "" if profile == "profile" { extra = " (enabling profiler labels)" - log.Infof(context.Background(), "%s", profile) st.SetCPUProfiling(true) defer st.SetCPUProfiling(false) } diff --git a/pkg/ui/src/views/reports/containers/debug/index.tsx b/pkg/ui/src/views/reports/containers/debug/index.tsx index 8a62309a4e94..da81fe383ef9 100644 --- a/pkg/ui/src/views/reports/containers/debug/index.tsx +++ b/pkg/ui/src/views/reports/containers/debug/index.tsx @@ -218,7 +218,7 @@ export default function Debug() { - + @@ -226,14 +226,10 @@ export default function Debug() { - - - - - - - + + + From 9d7ca0ffa5f55fd5121bb2c50ee03724e181dc07 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Mon, 25 Feb 2019 12:27:58 +0100 Subject: [PATCH 6/6] server/debug: expose panicparse's UI at /debug/pprof/goroutineui [panicparse] is a nifty tool that preprocesses goroutine dumps with the goal of making them more digestible. To do so, it groups "similar" stacks and tries to highlight system vs user code. The grouping in particular is helpful since the situations in which we stare at goroutine dumps are often the same situations in which there are tons of goroutines all over the place. And even in a happy cluster, our thread pools show up with high multiplicity and occupy enormous amounts of terminal real estate. The UI sets some defaults that are hopefully sane. First, it won't try to let panicparse rummage through the source files to improve the display of arguments, as the source files won't be available in prod (and trying to find them pp will log annoying messages). Second, we operate at the most lenient similarity where two stack frames are considered "similar" no matter what the arguments to the method are. This groups most aggressively which I think is what we want, though if we find out otherwise it's always easy to download the raw dump and to use panicparse locally. Or, of course, we can plumb a GET parameter that lets you chose the similarity strategy. [panicparse]: https://github.com/maruel/panicparse/ Release note: None --- Gopkg.lock | 8 + Gopkg.toml | 4 + pkg/server/debug/goroutineui/dump.go | 100 +++++++++ pkg/server/debug/goroutineui/dump_test.go | 98 +++++++++ pkg/server/debug/goroutineui/html.go | 202 ++++++++++++++++++ pkg/server/debug/server.go | 15 ++ .../views/reports/containers/debug/index.tsx | 5 +- vendor | 2 +- 8 files changed, 432 insertions(+), 2 deletions(-) create mode 100644 pkg/server/debug/goroutineui/dump.go create mode 100644 pkg/server/debug/goroutineui/dump_test.go create mode 100644 pkg/server/debug/goroutineui/html.go diff --git a/Gopkg.lock b/Gopkg.lock index cdc07e883648..9d5b485e1e52 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -1028,6 +1028,13 @@ pruneopts = "UT" revision = "9f7362b77ad333b26c01c99de52a11bdb650ded2" +[[projects]] + digest = "1:2eeeebccad4f052e6037527e86b8114c6bfd184a97f84d4449a5ea6ad202c216" + name = "github.com/maruel/panicparse" + packages = ["stack"] + pruneopts = "UT" + revision = "f20d4c4d746f810c9110e21928d4135e1f2a3efa" + [[projects]] digest = "1:130d1249f8a867da8115c3de6cddf0ac29ca405117c88aa1541db690384a51c6" name = "github.com/marusama/semaphore" @@ -1786,6 +1793,7 @@ "github.com/lightstep/lightstep-tracer-go", "github.com/linkedin/goavro", "github.com/lufia/iostat", + "github.com/maruel/panicparse/stack", "github.com/marusama/semaphore", "github.com/mattn/go-isatty", "github.com/mattn/goveralls", diff --git a/Gopkg.toml b/Gopkg.toml index f10472fba056..7585e776e716 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -36,6 +36,10 @@ ignored = [ name = "github.com/docker/docker" branch = "master" +[[constraint]] + name = "github.com/maruel/panicparse" + revision = "f20d4c4d746f810c9110e21928d4135e1f2a3efa" + # https://github.com/getsentry/raven-go/pull/139 [[constraint]] name = "github.com/getsentry/raven-go" diff --git a/pkg/server/debug/goroutineui/dump.go b/pkg/server/debug/goroutineui/dump.go new file mode 100644 index 000000000000..a66227d11c08 --- /dev/null +++ b/pkg/server/debug/goroutineui/dump.go @@ -0,0 +1,100 @@ +// Copyright 2019 The Cockroach 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 goroutineui + +import ( + "bytes" + "io" + "io/ioutil" + "runtime" + "sort" + "strings" + "time" + + "github.com/maruel/panicparse/stack" +) + +// stacks is a wrapper for runtime.Stack that attempts to recover the data for all goroutines. +func stacks() []byte { + // We don't know how big the traces are, so grow a few times if they don't fit. Start large, though. + var trace []byte + for n := 1 << 20; /* 1mb */ n <= (1 << 29); /* 512mb */ n *= 2 { + trace = make([]byte, n) + nbytes := runtime.Stack(trace, true /* all */) + if nbytes < len(trace) { + return trace[:nbytes] + } + } + return trace +} + +// A Dump wraps a goroutine dump with functionality to output through panicparse. +type Dump struct { + err error + + now time.Time + buckets []*stack.Bucket +} + +// NewDump grabs a goroutine dump and associates it with the supplied time. +func NewDump(now time.Time) Dump { + return NewDumpFromBytes(now, stacks()) +} + +// NewDumpFromBytes is like NewDump, but treats the supplied bytes as a goroutine +// dump. +func NewDumpFromBytes(now time.Time, b []byte) Dump { + c, err := stack.ParseDump(bytes.NewReader(b), ioutil.Discard, true /* guesspaths */) + if err != nil { + return Dump{err: err} + } + return Dump{now: now, buckets: stack.Aggregate(c.Goroutines, stack.AnyValue)} +} + +// SortCountDesc rearranges the goroutine buckets such that higher multiplicities +// appear earlier. +func (d Dump) SortCountDesc() { + sort.Slice(d.buckets, func(i, j int) bool { + a, b := d.buckets[i], d.buckets[j] + return len(a.IDs) > len(b.IDs) + }) +} + +// SortWaitDesc rearranges the goroutine buckets such that goroutines that have +// longer wait times appear earlier. +func (d Dump) SortWaitDesc() { + sort.Slice(d.buckets, func(i, j int) bool { + a, b := d.buckets[i], d.buckets[j] + return a.SleepMax > b.SleepMax + }) +} + +// HTML writes the rendered output of panicparse into the supplied Writer. +func (d Dump) HTML(w io.Writer) error { + if d.err != nil { + return d.err + } + return writeToHTML(w, d.buckets, d.now) +} + +// HTMLString is like HTML, but returns a string. If an error occurs, its string +// representation is returned. +func (d Dump) HTMLString() string { + var w strings.Builder + if err := d.HTML(&w); err != nil { + return err.Error() + } + return w.String() +} diff --git a/pkg/server/debug/goroutineui/dump_test.go b/pkg/server/debug/goroutineui/dump_test.go new file mode 100644 index 000000000000..7e2074da5976 --- /dev/null +++ b/pkg/server/debug/goroutineui/dump_test.go @@ -0,0 +1,98 @@ +// Copyright 2019 The Cockroach 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 goroutineui + +import ( + "io/ioutil" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/assert" +) + +func TestDumpHTML(t *testing.T) { + defer leaktest.AfterTest(t)() + + now := time.Time{} + dump := NewDumpFromBytes(now, []byte(fixture)) + dump.SortWaitDesc() // noop + dump.SortCountDesc() // noop + act := dump.HTMLString() + + if false { + _ = ioutil.WriteFile("test.html", []byte(act), 0644) + } + assert.Equal(t, exp, act) +} + +// This is the output of debug.PrintStack() on the go playground. +const fixture = `goroutine 1 [running]: +runtime/debug.Stack(0x434070, 0xddb11, 0x0, 0x40e0f8) + /usr/local/go/src/runtime/debug/stack.go:24 +0xc0 +runtime/debug.PrintStack() + /usr/local/go/src/runtime/debug/stack.go:16 +0x20 +main.main() + /tmp/sandbox157492124/main.go:6 +0x20` + +const exp = ` + +PanicParse + + +
Generated on 0001-01-01 00:00:00 +0000 UTC. +
+
+ +

Running Routine

+ 1: running + +

Stack

+ + - stack.go:24 Stack({[{4407408 } {908049 } {0 } {4251896 }] [] false})
+ - stack.go:16 PrintStack({[] [] false})
+ - .:0 main({[] [] false})
+ + +
+` diff --git a/pkg/server/debug/goroutineui/html.go b/pkg/server/debug/goroutineui/html.go new file mode 100644 index 000000000000..0f68cdd623c4 --- /dev/null +++ b/pkg/server/debug/goroutineui/html.go @@ -0,0 +1,202 @@ +// Copyright 2019 The Cockroach 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. + +// Copyright 2017 Marc-Antoine Ruel. All rights reserved. +// Use of this source code is governed under the Apache License, Version 2.0 +// that can be found in the LICENSE file. + +// NB: this file's original lives at: +// https://github.com/maruel/panicparse/blob/master/internal/html.go +// +// Please modify this file only when absolutely necessary so that we can +// pick up updates easily. + +package goroutineui + +import ( + "html/template" + "io" + "time" + + "github.com/maruel/panicparse/stack" +) + +func writeToHTML(w io.Writer, buckets []*stack.Bucket, now time.Time) error { + m := template.FuncMap{ + "funcClass": funcClass, + "notoColorEmoji1F4A3": notoColorEmoji1F4A3, + } + if len(buckets) > 1 { + m["routineClass"] = routineClass + } else { + m["routineClass"] = func(bucket *stack.Bucket) template.HTML { return "Routine" } + } + t, err := template.New("htmlTpl").Funcs(m).Parse(htmlTpl) + if err != nil { + return err + } + data := struct { + Buckets []*stack.Bucket + Now time.Time + }{buckets, now.Truncate(time.Second)} + + return t.Execute(w, data) +} + +func funcClass(line *stack.Call) template.HTML { + if line.IsStdlib { + if line.Func.IsExported() { + return "FuncStdLibExported" + } + return "FuncStdLib" + } else if line.IsPkgMain() { + return "FuncMain" + } else if line.Func.IsExported() { + return "FuncOtherExported" + } + return "FuncOther" +} + +func routineClass(bucket *stack.Bucket) template.HTML { + if bucket.First { + return "RoutineFirst" + } + return "Routine" +} + +const htmlTpl = ` +{{- define "RenderCall" -}} +{{.SrcLine}} {{.Func.Name}}({{.Args}}) +{{- end -}} + + +PanicParse + + +
Generated on {{.Now.String}}. +
+
+{{range .Buckets}} +

{{if .First}}Running {{end}}Routine

+ {{len .IDs}}: {{.State}} + {{if .SleepMax -}} + {{- if ne .SleepMin .SleepMax}} [{{.SleepMin}}~{{.SleepMax}} minutes] + {{- else}} [{{.SleepMax}} minutes] + {{- end -}} + {{- end}} + {{if .Locked}} [locked] + {{- end -}} + {{- if .CreatedBy.SrcPath}} [Created by {{template "RenderCall" .CreatedBy}}] + {{- end -}} +

Stack

+ {{range .Signature.Stack.Calls}} + - {{template "RenderCall" .}}
+ {{- end}} + {{if .Stack.Elided}}(...)
{{end}} +{{end}} +
+` + +// notoColorEmoji1F4A3 is the bomb emoji U+1F4A3 in Noto Emoji as a PNG. +// +// Source: https://www.google.com/get/noto/help/emoji/smileys-people.html +// License: http://scripts.sil.org/cms/scripts/page.php?site_id=nrsi&id=OFL +// +// Created with: +// python -c "import base64;a=base64.b64encode(open('emoji_u1f4a3.png','rb').read()); print '\n'.join(a[i:i+70] for i in range(0,len(a),70))" +func notoColorEmoji1F4A3() template.HTML { + return "" + + "iVBORw0KGgoAAAANSUhEUgAAAIgAAACACAMAAADnN9ENAAAA5FBMVEVMaXFvTjQhISEhIS" + + "EhISEhISEhISEhISEhISEhISEhISEhISHRbBTRbBTRbBQhISHRbBTRbBTRbBTRbBQ6OjrR" + + "bBTZcBTsehbzfhf1fxfidRVOTk75oiL7uCr90zL3kB3/6zv2hhlHMR5OTk5KSkpKSkpRUV" + + "FKSkpKSkpMTExBQUE2NjYpKSlOTk5EREQ8PDw4ODgtLS1RUVFQUFAmJiYkJCQ0NDRLS0tT" + + "U1MoKCgyMjJHR0dTU1NdXV0rKytXV1dVVVUvLy9eXl5aWlpcXFxcXFxeXl5fX186OjphYW" + + "E/Pz9KSkrdB5CTAAAATHRSTlMAEUZggJjf/6/vcL86e1nP2//vmSC7//////9E////////" + + "/2WPpYHp////////////////////z/////8w6P////+p/8f///////+/QBb3BQAACWJJRE" + + "FUeAHslgWC6yAURUOk1N3d3d29+9/SzzxCStvQEfp9zgZycu8DnvTNN78YJCuqqsry77WQ" + + "NRum2BX0u7JQiYWJw/l7NBz4AderQkFut8frdn+kFBu2wodeYeHxBwjBkFd6StiFOYiboF" + + "CAxf9MRXHc9KGpqurDBpqghzcYuCOCeMp21oKelTBVCQt5kDiisXhCJ5aMQkFu6+lg4tDY" + + "r2rikRCPKFgQYlGeiYpN7OHbpMj8OgQ8PAGdWIIlnnwbFKYdtgDAJj+MDgZSX/Zwsx4mby" + + "YR6RbntRZVegQDX7/tI1YexMTNObQ+y992EUWRQJIJQjqTzWRyRjtRiMTqKtWQJCTCD4TM" + + "aS6bBzLGxLKRKNer1MELX0wEmYHk8pSMGUmIlMI+LC7uTeEQmhEvnZBC/kqGTolfkmSn72" + + "NPbFhoWOHswmczeYYC7YaV4MfBHl+BEYmCSJYVSUM3ukgRM5C7g4edqAqLMBq0m1sRh8oe" + + "Fl4zzp8swtFApXKlmkIkECD8+mpYEZ8iWZGq1YFKKazg582IDiu8bk7Ob5YaOnVCs9UWu+" + + "C99D7LWR5fVeY/YpVOp9Po9rp1g25/AIGIXmhp0yMLgSTIhcYDVYaj0ag1Hk/a0yZ1qVVK" + + "6KsmfnMVSbMepBkv32M2nw87rfZioatMxsveirqUBLoxHr1CJpvNZmBQSSB+icd6M9dFpt" + + "tt21CZ4EHfKKny9Ug4awA/kNRmt993loPBAFSICcbtFpRUeuViFIPFiENpp9NYHg6HexW8" + + "1Suaiaysscc8gsg6jdTxpFNf6oALUaEmBz2SsMBKEkjeL88Bt8VFWj1fCKvWdDolKmYoYD" + + "L5ejcSApNoMsZqBH+0byfbiStNEICbFZt/uIN3WtpASWIUwgOiZWgMyPL7v8+tCuIkBdlw" + + "W4WWHS/AdyKzRCEfXy7Iw+PHntlti+k0TZ1FKCJJgteV0wHGhr/1Lvp4/HGQvGdJVVVTZy" + + "HFl6TGDEIM3FiUIvnrv53zkXyH4PNzm5mknrhUNo7CUjAeSIbGmNW3Vih/gHGKY1jE53uc" + + "1BJYSOF4KCmM6YcqaPmvy/86l88MKPbzcXIKLKSwFJFUPMDtpvPDKT63cZKMJSCRglJE4k" + + "7x0hjTaZHAOpzjYBIKCpsThpQLSc4D3GaOdWQ0+CEFEo7HSkpIxjzAraXz4RxbURhJQQtK" + + "gYSdYE2mPMBtZYWxjMggIRYLKSiFks0Gw9nExjy16XBnxYBxNHghRSTcE65JEXM4rTl2qI" + + "OKkRdnIcWXcDgzXktam8s76zBQzOcZ4q6IsIBCSVVhYTmckpJ2HWBA8XoMMEri1oQnJ89L" + + "x++3cF6U46hYI8CQ4ks4HBzhYdHK0+TDc5ABxDsCCygi4ZpIJYvF0EIGqzsdffvtskschA" + + "4wLGHrcrRoCYfDSnBVe+nc5Yis4zAWRwYHFQwpFxKBuEq6Uyt5untBCs8hjB1DCiVnlfDg" + + "5PkCdzUT3TmYDINxezqH46jY7+3FxF0Vd75EVcLZdPPCmEH4cFb202RBfIdT2K4ONo5CyQ" + + "iSE+T5NJvu8q4z/LE/fBYWwsHY/Xgn45NxGEr8SvyDc4R06zt+W0S7wyGrk1MhdJAhkr2T" + + "rEXy09ng/toLL2SfcENkMHRoiYVkrERBnKQKriSynzmqORlAlMObjgyHs+G5kSXp5sGV/N" + + "jZQmQyKMQOxnNIpBI1m40sCSoJOjgPux0LKW4XQgkOjpqNBwm9wPYtJFGToeNaJaPjbPS2" + + "utRh98bvu/1rLZAMEFWISAjxl0RBNkE//Fb2U4sTBI5bkJ2/JBqCFCHfOH27mBMHKXzI4R" + + "pk/1PI8zmkCpnNx3aXaYh1NICkF5BZwKOkYz+1aBUS+OYmsp9aa8i+wWjUjuDc9BqvyPa9" + + "AkSW9b5Tg0yNeWm8ItusmkwuniP6wUrHBSTREFmSpk+R7dZMq4l6sh6uP1kdBLc09WQVyL" + + "D5Rc1eCGtAkiPk5pLIZDKuiH7EM40hD4R426oqufmlNwHEOs4hSdN7WmQhqYOoLxslEYd8" + + "1ejTK5C6MWTtIN6SuHPD4fgSOvxCrnznBZxfQtbPrOTi7kyJdkghNyCpMV+NIP31+4gQVs" + + "Itubg9yzVerqxyePWKhEHWDiLnhpVQAgoCBh08u7qQuyCv69FSKrmQgLLDm3jHEIdXiJ5M" + + "IOTxdZ0B4h0cSvzfnFswzh1SiJ5MACSyn7h89iuhJIMEFCoc49KhJxN2aghxlSiJvJdg1n" + + "DMnUMV4k0m9Dmysp/3zErOJKDAwrxahnKcCuFkAp6sjP20qauEwzmTgIJAAYbvuCjEhzT/" + + "0htkr/VmyX0VCSnOwoABR0EHBnOlkLw55Ct7HTvImUQoFsN4L1rVS0VVSMh9pD/P4pmTYD" + + "giUW98Y4/BPvRgJGnzG1pk698oiX4HblwK5ZC3rHSE31kf7FJOZxtfQgotEmGIQw1GEvLr" + + "dzCaJ6WthJKKElAQGs4Y1x0Bv2uYp9E8Ls8kpIhFEI5Bx5SO44nxIcG/9CL7GF2WM5FgPK" + + "DAwlBBBs4LHbqQgN++yCAeJcMzCSiwiCahAgyM5YZjFvZn4Kd4FJdOgo0VCizEwAAG69AO" + + "P5Ow9yOrOB6lw2FZniSWIhbJBAphYE+VI+yNEfMVxyZ/o8SjwMKIAgzUoR1MFfpH4EcTx8" + + "9HiVBgAeaUqTDoGMKhCgl/0TowcZFbCRcFFFokQJwx4FCM0PesrMTE6QISjwKLRBSWIWOh" + + "o4VCmBdjTL7IheIsxEioEMYVB9/FByYyxtQLSkiBBaFAFML4mWN235+OesaYzcKnwEIMEV" + + "CAcd2x4N9rQtMZGFPkXVJoIQYBAgrFUIOJghkcTtJ1ElBgAcZLSYVmSFK1qUHDqbqk0AIM" + + "AwQUYNChF4SDCU/HnZxNlxRY3qBhiPAUOsOC33Z3ZTWgxLOAg8AAhWJI2vhLONeEElqoEY" + + "JSKAdP7r15FIlgJBqhHVzUtiTLblBKOlqUVIsAx9LQ0aYkGTZlLCYtO3h2iobjKceG56XF" + + "PLwYm7pBKYupsRlE39rOk3GZLn51Owpj89VpmyH/lVCkv0LZYCoDjqXtdPoGqf5lQHlaGN" + + "Tx0L6BeegZJFnmV1djg6OitqPtRKSYZDrTMyrTOuC/BIJbGRhmXKfLktmkk8QwphfQRkA6" + + "jz1zIy+PAbsRbInQi8qgF6C4H9PvfQ2E8NXrR7z9tJvf+Z1/ANt+S+GBXoDpAAAAAElFTk" + + "SuQmCC" +} diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index 4dea33053bf0..69968d63298f 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -23,6 +23,7 @@ import ( "net/http/pprof" "strings" + "github.com/cockroachdb/cockroach/pkg/server/debug/goroutineui" "github.com/cockroachdb/cockroach/pkg/server/debug/pprofui" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -136,6 +137,20 @@ func NewServer(st *cluster.Settings) *Server { }) mux.Handle("/debug/pprof/ui/", http.StripPrefix("/debug/pprof/ui", ps)) + mux.HandleFunc("/debug/pprof/goroutineui/", func(w http.ResponseWriter, req *http.Request) { + dump := goroutineui.NewDump(timeutil.Now()) + + _ = req.ParseForm() + switch req.Form.Get("sort") { + case "count": + dump.SortCountDesc() + case "wait": + dump.SortWaitDesc() + default: + } + _ = dump.HTML(w) + }) + return &Server{ st: st, mux: mux, diff --git a/pkg/ui/src/views/reports/containers/debug/index.tsx b/pkg/ui/src/views/reports/containers/debug/index.tsx index da81fe383ef9..8299c40a4607 100644 --- a/pkg/ui/src/views/reports/containers/debug/index.tsx +++ b/pkg/ui/src/views/reports/containers/debug/index.tsx @@ -227,7 +227,10 @@ export default function Debug() {
- + + + + diff --git a/vendor b/vendor index d8a496f8197d..1c5942a982b5 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit d8a496f8197df890ec9aa21097af56e97fdbb37b +Subproject commit 1c5942a982b5e1b0b399a6fb4ae183d7651179ab