From 49e6d3931ba9e8731c695e08025acf7b0a1ab2b5 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Wed, 1 Jul 2020 18:13:11 -0700 Subject: [PATCH] cli: add commands for managing statement diagnostics This change adds a `statement-diag` command, with the following subcommands: ``` list list available bundles and outstanding activation requests download download statement diagnostics bundle into a zip file delete delete statement diagnostics bundles cancel cancel outstanding activation requests ``` Fixes #48597. Release note (cli change): A new set of `statement-diag` CLI commands that can be used to manage statement diagnostics. --- pkg/cli/cli.go | 1 + pkg/cli/cli_test.go | 3 +- pkg/cli/cliflags/flags.go | 10 + pkg/cli/context.go | 11 + pkg/cli/flags.go | 8 + pkg/cli/main_test.go | 5 + pkg/cli/statement_diag.go | 358 +++++++++++++++++++++++++++++++++ pkg/cli/statement_diag_test.go | 152 ++++++++++++++ pkg/sql/explain_bundle.go | 4 +- 9 files changed, 550 insertions(+), 2 deletions(-) create mode 100644 pkg/cli/statement_diag.go create mode 100644 pkg/cli/statement_diag_test.go diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go index 88d85bbe1e4c..e5213b1efc10 100644 --- a/pkg/cli/cli.go +++ b/pkg/cli/cli.go @@ -195,6 +195,7 @@ func init() { quitCmd, sqlShellCmd, + stmtDiagCmd, authCmd, nodeCmd, dumpCmd, diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index 537cf5ddc1a6..c04584a53d29 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -297,7 +297,7 @@ func isSQLCommand(args []string) bool { return false } switch args[0] { - case "sql", "dump", "workload", "nodelocal": + case "sql", "dump", "workload", "nodelocal", "statement-diag": return true case "node": if len(args) == 0 { @@ -1403,6 +1403,7 @@ Available Commands: init initialize a cluster cert create ca, node, and client certs sql open a sql shell + statement-diag commands for managing statement diagnostics bundles auth-session log in and out of HTTP sessions node list, inspect, drain or remove nodes dump dump sql tables diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 8357ca24482b..10fd738aa455 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -1188,4 +1188,14 @@ other items retrieved by the zip command may still consider confidential data or PII. `, } + + StmtDiagDeleteAll = FlagInfo{ + Name: "all", + Description: `Delete all bundles.`, + } + + StmtDiagCancelAll = FlagInfo{ + Name: "all", + Description: `Cancel all outstanding requests.`, + } ) diff --git a/pkg/cli/context.go b/pkg/cli/context.go index a6560f3e464a..420d351705e8 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -49,6 +49,7 @@ func initCLIDefaults() { setNetworkBenchContextDefaults() setSqlfmtContextDefaults() setDemoContextDefaults() + setStmtDiagContextDefaults() setAuthContextDefaults() initPreFlagsDefaults() @@ -506,6 +507,16 @@ func setDemoContextDefaults() { demoCtx.insecure = false } +// stmtDiagCtx captures the command-line parameters of the 'statement-diag' +// command. +var stmtDiagCtx struct { + all bool +} + +func setStmtDiagContextDefaults() { + stmtDiagCtx.all = false +} + // GetServerCfgStores provides direct public access to the StoreSpecList inside // serverCfg. This is used by CCL code to populate some fields. // diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index ac4074810dfc..645a84008502 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -535,6 +535,7 @@ func init() { clientCmds = append(clientCmds, nodeCmds...) clientCmds = append(clientCmds, systemBenchCmds...) clientCmds = append(clientCmds, nodeLocalCmds...) + clientCmds = append(clientCmds, stmtDiagCmds...) for _, cmd := range clientCmds { f := cmd.PersistentFlags() varFlag(f, addrSetter{&cliCtx.clientConnHost, &cliCtx.clientConnPort}, cliflags.ClientHost) @@ -645,6 +646,7 @@ func init() { sqlCmds := []*cobra.Command{sqlShellCmd, dumpCmd, demoCmd} sqlCmds = append(sqlCmds, authCmds...) sqlCmds = append(sqlCmds, demoCmd.Commands()...) + sqlCmds = append(sqlCmds, stmtDiagCmds...) sqlCmds = append(sqlCmds, nodeLocalCmds...) for _, cmd := range sqlCmds { f := cmd.Flags() @@ -732,6 +734,12 @@ func init() { stringFlag(f, &startCtx.geoLibsDir, cliflags.GeoLibsDir) } + // statement-diag command. + { + boolFlag(stmtDiagDeleteCmd.Flags(), &stmtDiagCtx.all, cliflags.StmtDiagDeleteAll) + boolFlag(stmtDiagCancelCmd.Flags(), &stmtDiagCtx.all, cliflags.StmtDiagCancelAll) + } + // sqlfmt command. { f := sqlfmtCmd.Flags() diff --git a/pkg/cli/main_test.go b/pkg/cli/main_test.go index e860d1f0d64f..a233ca2e699b 100644 --- a/pkg/cli/main_test.go +++ b/pkg/cli/main_test.go @@ -13,6 +13,7 @@ package cli_test import ( "os" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/security" @@ -30,6 +31,10 @@ func TestMain(m *testing.M) { // a version injected. Pretend to be a very up-to-date version. defer build.TestingOverrideTag("v999.0.0")() + // The times for Example_statement_diag are reported in the local timezone. + // Fix it to UTC so the output is always the same. + time.Local = time.UTC + serverutils.InitTestServerFactory(server.TestServerFactory) os.Exit(m.Run()) } diff --git a/pkg/cli/statement_diag.go b/pkg/cli/statement_diag.go new file mode 100644 index 000000000000..518d0eecbdaa --- /dev/null +++ b/pkg/cli/statement_diag.go @@ -0,0 +1,358 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package cli + +import ( + "bytes" + "database/sql/driver" + "fmt" + "io" + "os" + "strconv" + "text/tabwriter" + "time" + + "github.com/cockroachdb/errors" + "github.com/spf13/cobra" +) + +var stmtDiagCmd = &cobra.Command{ + Use: "statement-diag [command]", + Short: "commands for managing statement diagnostics bundles", + Long: `This set of commands can be used to manage and download statement diagnostic +bundles, and to cancel outstanding diagnostics activation requests. Statement +diagnostics can be activated from the UI or using EXPLAIN ANALYZE (DEBUG).`, + RunE: usageAndErr, +} + +var stmtDiagListCmd = &cobra.Command{ + Use: "list [options]", + Short: "list available bundles and outstanding activation requests", + Long: `List statements diagnostics that are available for download and outstanding +diagnostics activation requests.`, + Args: cobra.NoArgs, + RunE: MaybeDecorateGRPCError(runStmtDiagList), +} + +func runStmtDiagList(cmd *cobra.Command, args []string) error { + const timeFmt = "2006-01-02 15:04:05 MST" + + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + // -- List bundles -- + + rows, err := conn.Query( + `SELECT id, statement_fingerprint, collected_at + FROM system.statement_diagnostics + WHERE error IS NULL + ORDER BY collected_at DESC`, + nil, /* args */ + ) + if err != nil { + return err + } + vals := make([]driver.Value, 3) + var buf bytes.Buffer + w := tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0) + fmt.Fprint(w, " ID\tCollection time\tStatement\n") + num := 0 + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + id := vals[0].(int64) + stmt := vals[1].(string) + t := vals[2].(time.Time) + fmt.Fprintf(w, " %d\t%s\t%s\n", id, t.Local().Format(timeFmt), stmt) + num++ + } + if err := rows.Close(); err != nil { + return err + } + if num == 0 { + fmt.Printf("No statement diagnostics bundles available.\n") + } else { + fmt.Printf("Statement diagnostics bundles:\n") + _ = w.Flush() + // When we show a list of bundles, we want an extra blank line. + fmt.Println(buf.String()) + } + + // -- List outstanding activation requests -- + + rows, err = conn.Query( + `SELECT id, statement_fingerprint, requested_at + FROM system.statement_diagnostics_requests + WHERE NOT completed + ORDER BY requested_at DESC`, + nil, /* args */ + ) + if err != nil { + return err + } + + buf.Reset() + w = tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0) + fmt.Fprint(w, " ID\tActivation time\tStatement\n") + num = 0 + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + id := vals[0].(int64) + stmt := vals[1].(string) + t := vals[2].(time.Time) + fmt.Fprintf(w, " %d\t%s\t%s\n", id, t.Local().Format(timeFmt), stmt) + num++ + } + if err := rows.Close(); err != nil { + return err + } + if num == 0 { + fmt.Printf("No outstanding activation requests.\n") + } else { + fmt.Printf("Outstanding activation requests:\n") + _ = w.Flush() + fmt.Print(buf.String()) + } + + return nil +} + +var stmtDiagDownloadCmd = &cobra.Command{ + Use: "download [options]", + Short: "download statement diagnostics bundle into a zip file", + Long: `Download statements diagnostics bundle into a zip file, using an ID returned by +the list command.`, + Args: cobra.ExactArgs(2), + RunE: MaybeDecorateGRPCError(runStmtDiagDownload), +} + +func runStmtDiagDownload(cmd *cobra.Command, args []string) error { + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid bundle id") + } + filename := args[1] + + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + // Retrieve the chunk IDs; these are stored in an INT ARRAY column. + rows, err := conn.Query( + "SELECT unnest(bundle_chunks) FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + if err != nil { + return err + } + var chunkIDs []int64 + vals := make([]driver.Value, 1) + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + chunkIDs = append(chunkIDs, vals[0].(int64)) + } + if err := rows.Close(); err != nil { + return err + } + + if len(chunkIDs) == 0 { + return errors.Newf("no statement diagnostics bundle with ID %d", id) + } + + // Create the file and write out the chunks. + out, err := os.Create(filename) + if err != nil { + return err + } + + for _, chunkID := range chunkIDs { + data, err := conn.QueryRow( + "SELECT data FROM system.statement_bundle_chunks WHERE id = $1", + []driver.Value{chunkID}, + ) + if err != nil { + _ = out.Close() + return err + } + if _, err := out.Write(data[0].([]byte)); err != nil { + _ = out.Close() + return err + } + } + + return out.Close() +} + +var stmtDiagDeleteCmd = &cobra.Command{ + Use: "delete { --all | }", + Short: "delete statement diagnostics bundles", + Long: `Delete a statement diagnostics bundle using an ID returned by the list +command, or delete all bundles.`, + Args: cobra.MaximumNArgs(1), + RunE: MaybeDecorateGRPCError(runStmtDiagDelete), +} + +func runStmtDiagDelete(cmd *cobra.Command, args []string) error { + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + if stmtDiagCtx.all { + if len(args) > 0 { + return errors.New("extra arguments with --all") + } + return runStmtDiagDeleteAll(conn) + } + if len(args) != 1 { + return fmt.Errorf("accepts 1 arg, received %d", len(args)) + } + + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid id") + } + + _, err = conn.QueryRow( + "SELECT 1 FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + if err != nil { + if err == io.EOF { + return errors.Newf("no statement diagnostics bundle with ID %d", id) + } + return err + } + + return conn.ExecTxn(func(conn *sqlConn) error { + // Delete the request metadata. + if err := conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE statement_diagnostics_id = $1", + []driver.Value{id}, + ); err != nil { + return err + } + // Delete the bundle chunks. + if err := conn.Exec( + `DELETE FROM system.statement_bundle_chunks + WHERE id IN ( + SELECT unnest(bundle_chunks) FROM system.statement_diagnostics WHERE id = $1 + )`, + []driver.Value{id}, + ); err != nil { + return err + } + // Finally, delete the diagnostics entry. + return conn.Exec( + "DELETE FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + }) +} + +func runStmtDiagDeleteAll(conn *sqlConn) error { + return conn.ExecTxn(func(conn *sqlConn) error { + // Delete the request metadata. + if err := conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE completed", + nil, + ); err != nil { + return err + } + // Delete all bundle chunks. + if err := conn.Exec( + `DELETE FROM system.statement_bundle_chunks WHERE true`, + nil, + ); err != nil { + return err + } + // Finally, delete the diagnostics entry. + return conn.Exec( + "DELETE FROM system.statement_diagnostics WHERE true", + nil, + ) + }) +} + +var stmtDiagCancelCmd = &cobra.Command{ + Use: "cancel { -all | }", + Short: "cancel outstanding activation requests", + Long: `Cancel an outstanding activation request, using an ID returned by the +list command, or cancel all outstanding requests.`, + Args: cobra.MaximumNArgs(1), + RunE: MaybeDecorateGRPCError(runStmtDiagCancel), +} + +func runStmtDiagCancel(cmd *cobra.Command, args []string) error { + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + if stmtDiagCtx.all { + if len(args) > 0 { + return errors.New("extra arguments with --all") + } + return conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE NOT completed", + nil, + ) + } + if len(args) != 1 { + return fmt.Errorf("accepts 1 arg, received %d", len(args)) + } + + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid id") + } + + _, err = conn.QueryRow( + "DELETE FROM system.statement_diagnostics_requests WHERE id = $1 RETURNING id", + []driver.Value{id}, + ) + if err != nil { + if err == io.EOF { + return errors.Newf("no outstanding activation requests with ID %d", id) + } + return err + } + return nil +} + +var stmtDiagCmds = []*cobra.Command{ + stmtDiagListCmd, + stmtDiagDownloadCmd, + stmtDiagDeleteCmd, + stmtDiagCancelCmd, +} + +func init() { + stmtDiagCmd.AddCommand(stmtDiagCmds...) +} diff --git a/pkg/cli/statement_diag_test.go b/pkg/cli/statement_diag_test.go new file mode 100644 index 000000000000..ce65bc8ced25 --- /dev/null +++ b/pkg/cli/statement_diag_test.go @@ -0,0 +1,152 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package cli + +import ( + "context" + "fmt" + "io/ioutil" + "os" + + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func Example_statement_diag() { + c := newCLITest(cliTestParams{}) + defer c.cleanup() + + // First, set up some diagnostics state. + commands := []string{ + `INSERT INTO system.statement_bundle_chunks(id, data) VALUES (1001, 'chunk1'), (1002, 'chunk2'), (1003, 'chunk3')`, + + `INSERT INTO system.statement_diagnostics(id, statement_fingerprint, statement, collected_at, bundle_chunks) + VALUES (10, 'SELECT _ FROM _', 'SELECT a FROM t', '2010-01-02 03:04:05', ARRAY[1001]), + (20, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:06', ARRAY[1001,1002,1003]), + (30, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:07', ARRAY[1001])`, + + `INSERT INTO system.statement_diagnostics_requests(id, completed, statement_fingerprint, statement_diagnostics_id, requested_at) + VALUES (1, TRUE, 'SELECT _ FROM _', 10, '2010-01-02 03:04:00'), + (2, TRUE, 'SELECT _ FROM _ WHERE _ > _', 20, '2010-01-02 03:04:02'), + (3, TRUE, 'SELECT _ FROM _ WHERE _ > _', 30, '2010-01-02 03:04:05'), + (4, FALSE, 'SELECT _ + _', NULL, '2010-01-02 03:04:10'), + (5, FALSE, 'SELECT _ - _', NULL, '2010-01-02 03:04:11'), + (6, FALSE, 'SELECT _ / _', NULL, '2010-01-02 03:04:12')`, + } + + for _, cmd := range commands { + _, err := c.RunWithCaptureArgs([]string{"sql", "-e", cmd}) + if err != nil { + log.Fatalf(context.Background(), "Couldn't execute sql: %s", err) + } + } + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "download", "13", "foo.zip"}) + tmpfile, err := ioutil.TempFile("", "bundle-*.zip") + if err != nil { + log.Fatalf(context.Background(), "Couldn't execute sql: %s", err) + } + bundleFile := tmpfile.Name() + _ = tmpfile.Close() + defer func() { _ = os.Remove(bundleFile) }() + + fmt.Printf("statement-diag download 20 tempfile.zip\n") + _, err = c.RunWithCaptureArgs([]string{"statement-diag", "download", "20", bundleFile}) + if err != nil { + log.Fatalf(context.Background(), "Error downloading bundle: %s", err) + } + data, err := ioutil.ReadFile(bundleFile) + if err != nil { + log.Fatalf(context.Background(), "Error reading bundle: %s", err) + } + fmt.Printf("bundle data: %s\n", data) + + c.RunWithArgs([]string{"statement-diag", "download", "xx"}) + c.RunWithArgs([]string{"statement-diag", "delete", "--all", "20"}) + c.RunWithArgs([]string{"statement-diag", "delete", "20", "30"}) + c.RunWithArgs([]string{"statement-diag", "delete", "xx"}) + c.RunWithArgs([]string{"statement-diag", "delete", "13"}) + c.RunWithArgs([]string{"statement-diag", "delete", "10"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "delete", "--all"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "xx"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "5", "6"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "--all", "5"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "4"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "--all"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + + // Output: + // statement-diag list + // Statement diagnostics bundles: + // ID Collection time Statement + // 30 2010-01-02 03:04:07 UTC SELECT _ FROM _ WHERE _ > _ + // 20 2010-01-02 03:04:06 UTC SELECT _ FROM _ WHERE _ > _ + // 10 2010-01-02 03:04:05 UTC SELECT _ FROM _ + // + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag download 13 foo.zip + // ERROR: no statement diagnostics bundle with ID 13 + // statement-diag download 20 tempfile.zip + // bundle data: chunk1chunk2chunk3 + // statement-diag download xx + // ERROR: accepts 2 arg(s), received 1 + // statement-diag delete --all 20 + // ERROR: extra arguments with --all + // statement-diag delete 20 30 + // ERROR: accepts at most 1 arg(s), received 2 + // statement-diag delete xx + // ERROR: invalid id + // statement-diag delete 13 + // ERROR: no statement diagnostics bundle with ID 13 + // statement-diag delete 10 + // statement-diag list + // Statement diagnostics bundles: + // ID Collection time Statement + // 30 2010-01-02 03:04:07 UTC SELECT _ FROM _ WHERE _ > _ + // 20 2010-01-02 03:04:06 UTC SELECT _ FROM _ WHERE _ > _ + // + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag delete --all + // statement-diag list + // No statement diagnostics bundles available. + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag cancel xx + // ERROR: invalid id + // statement-diag cancel 5 6 + // ERROR: accepts at most 1 arg(s), received 2 + // statement-diag cancel --all 5 + // ERROR: extra arguments with --all + // statement-diag cancel 4 + // statement-diag list + // No statement diagnostics bundles available. + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // statement-diag cancel --all + // statement-diag list + // No statement diagnostics bundles available. + // No outstanding activation requests. +} diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 267ac505a946..f6c29a5fb02d 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -74,9 +74,11 @@ func setExplainBundleResult( text = []string{ "Statement diagnostics bundle generated. Download from the Admin UI (Advanced", - "Debug -> Statement Diagnostics History) or use the direct link below.", + "Debug -> Statement Diagnostics History), via the direct link below, or using", + "the command line.", fmt.Sprintf("Admin UI: %s", execCfg.AdminURL()), fmt.Sprintf("Direct link: %s/_admin/v1/stmtbundle/%d", execCfg.AdminURL(), diagID), + "Command line: cockroach statement-diag list / download", } }()