diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 37384f99daab9..7debcf0ce31a2 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -1441,67 +1441,6 @@ func (ts *HTTPHandlerTestSuite) TestCheckCN(c *C) { c.Assert(err, NotNil) } -func (ts *HTTPHandlerTestSuite) TestZipInfoForSQL(c *C) { - ts.startServer(c) - defer ts.stopServer(c) - - db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) - defer func() { - err := db.Close() - c.Assert(err, IsNil) - }() - dbt := &DBTest{c, db} - - dbt.mustExec("use test") - dbt.mustExec("create table if not exists t (a int)") - - urlValues := url.Values{ - "sql": {"select * from t"}, - "current_db": {"test"}, - } - resp, err := ts.formStatus("/debug/sub-optimal-plan", urlValues) - c.Assert(err, IsNil) - c.Assert(resp.StatusCode, Equals, http.StatusOK) - b, err := httputil.DumpResponse(resp, true) - c.Assert(err, IsNil) - c.Assert(len(b), Greater, 0) - c.Assert(resp.Body.Close(), IsNil) - - resp, err = ts.formStatus("/debug/sub-optimal-plan?pprof_time=5&timeout=0", urlValues) - c.Assert(err, IsNil) - c.Assert(resp.StatusCode, Equals, http.StatusOK) - b, err = httputil.DumpResponse(resp, true) - c.Assert(err, IsNil) - c.Assert(len(b), Greater, 0) - c.Assert(resp.Body.Close(), IsNil) - - resp, err = ts.formStatus("/debug/sub-optimal-plan?pprof_time=5", urlValues) - c.Assert(err, IsNil) - c.Assert(resp.StatusCode, Equals, http.StatusOK) - b, err = httputil.DumpResponse(resp, true) - c.Assert(err, IsNil) - c.Assert(len(b), Greater, 0) - c.Assert(resp.Body.Close(), IsNil) - - resp, err = ts.formStatus("/debug/sub-optimal-plan?timeout=1", urlValues) - c.Assert(err, IsNil) - c.Assert(resp.StatusCode, Equals, http.StatusOK) - b, err = httputil.DumpResponse(resp, true) - c.Assert(err, IsNil) - c.Assert(len(b), Greater, 0) - c.Assert(resp.Body.Close(), IsNil) - - urlValues.Set("current_db", "non_exists_db") - resp, err = ts.formStatus("/debug/sub-optimal-plan", urlValues) - c.Assert(err, IsNil) - c.Assert(resp.StatusCode, Equals, http.StatusInternalServerError) - b, err = io.ReadAll(resp.Body) - c.Assert(err, IsNil) - c.Assert(string(b), Equals, "use database non_exists_db failed, err: [schema:1049]Unknown database 'non_exists_db'\n") - c.Assert(resp.Body.Close(), IsNil) -} - func (ts *HTTPHandlerTestSuite) TestFailpointHandler(c *C) { defer ts.stopServer(c) diff --git a/server/http_status.go b/server/http_status.go index ae1cdf6434e47..5748fdb67ea7b 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -299,8 +299,6 @@ func (s *Server) startHTTPServer() { err = zw.Close() terror.Log(err) }) - fetcher := sqlInfoFetcher{store: tikvHandlerTool.Store} - serverMux.HandleFunc("/debug/sub-optimal-plan", fetcher.zipInfoForSQL) // failpoint is enabled only for tests so we can add some http APIs here for tests. failpoint.Inject("enableTestAPI", func() { diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go deleted file mode 100644 index 57f51f544b90b..0000000000000 --- a/server/sql_info_fetcher.go +++ /dev/null @@ -1,338 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package server - -import ( - "archive/zip" - "bytes" - "context" - "encoding/json" - "fmt" - "net/http" - "strconv" - "strings" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/parser" - "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/model" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/topsql/tracecpu" -) - -type sqlInfoFetcher struct { - store kv.Storage - do *domain.Domain - s session.Session -} - -type tableNamePair struct { - DBName string - TableName string -} - -type tableNameExtractor struct { - curDB string - names map[tableNamePair]struct{} -} - -func (tne *tableNameExtractor) Enter(in ast.Node) (ast.Node, bool) { - if _, ok := in.(*ast.TableName); ok { - return in, true - } - return in, false -} - -func (tne *tableNameExtractor) Leave(in ast.Node) (ast.Node, bool) { - if t, ok := in.(*ast.TableName); ok { - tp := tableNamePair{DBName: t.Schema.L, TableName: t.Name.L} - if tp.DBName == "" { - tp.DBName = tne.curDB - } - if _, ok := tne.names[tp]; !ok { - tne.names[tp] = struct{}{} - } - } - return in, true -} - -func (sh *sqlInfoFetcher) zipInfoForSQL(w http.ResponseWriter, r *http.Request) { - var err error - sh.s, err = session.CreateSession(sh.store) - if err != nil { - serveError(w, http.StatusInternalServerError, fmt.Sprintf("create session failed, err: %v", err)) - return - } - defer sh.s.Close() - - sh.do = domain.GetDomain(sh.s) - reqCtx := r.Context() - sql := r.FormValue("sql") - pprofTimeString := r.FormValue("pprof_time") - timeoutString := r.FormValue("timeout") - curDB := strings.ToLower(r.FormValue("current_db")) - if curDB != "" { - _, err = sh.s.ExecuteInternal(context.Background(), "use %n", curDB) - if err != nil { - serveError(w, http.StatusInternalServerError, fmt.Sprintf("use database %v failed, err: %v", curDB, err)) - return - } - } - var ( - pprofTime int - timeout int - ) - if pprofTimeString != "" { - pprofTime, err = strconv.Atoi(pprofTimeString) - if err != nil { - serveError(w, http.StatusBadRequest, "invalid value for pprof_time, please input a int value larger than 5") - return - } - } - if pprofTimeString != "" && pprofTime < 5 { - serveError(w, http.StatusBadRequest, "pprof time is too short, please input a int value larger than 5") - } - if timeoutString != "" { - timeout, err = strconv.Atoi(timeoutString) - if err != nil { - serveError(w, http.StatusBadRequest, "invalid value for timeout") - return - } - } - if timeout < pprofTime { - timeout = pprofTime - } - pairs, err := sh.extractTableNames(sql, curDB) - if err != nil { - serveError(w, http.StatusBadRequest, fmt.Sprintf("invalid SQL text, err: %v", err)) - return - } - zw := zip.NewWriter(w) - defer func() { - terror.Log(zw.Close()) - }() - for pair := range pairs { - jsonTbl, err := sh.getStatsForTable(pair) - if err != nil { - err = sh.writeErrFile(zw, fmt.Sprintf("%v.%v.stats.err.txt", pair.DBName, pair.TableName), err) - terror.Log(err) - continue - } - statsFw, err := zw.Create(fmt.Sprintf("%v.%v.json", pair.DBName, pair.TableName)) - if err != nil { - terror.Log(err) - continue - } - data, err := json.Marshal(jsonTbl) - if err != nil { - err = sh.writeErrFile(zw, fmt.Sprintf("%v.%v.stats.err.txt", pair.DBName, pair.TableName), err) - terror.Log(err) - continue - } - _, err = statsFw.Write(data) - if err != nil { - err = sh.writeErrFile(zw, fmt.Sprintf("%v.%v.stats.err.txt", pair.DBName, pair.TableName), err) - terror.Log(err) - continue - } - } - for pair := range pairs { - err = sh.getShowCreateTable(pair, zw) - if err != nil { - err = sh.writeErrFile(zw, fmt.Sprintf("%v.%v.schema.err.txt", pair.DBName, pair.TableName), err) - terror.Log(err) - return - } - } - // If we don't catch profile. We just get a explain result. - if pprofTime == 0 { - recordSets, err := sh.s.(sqlexec.SQLExecutor).Execute(reqCtx, fmt.Sprintf("explain %s", sql)) - if len(recordSets) > 0 { - defer terror.Call(recordSets[0].Close) - } - if err != nil { - err = sh.writeErrFile(zw, "explain.err.txt", err) - terror.Log(err) - return - } - sRows, err := session.ResultSetToStringSlice(reqCtx, sh.s, recordSets[0]) - if err != nil { - err = sh.writeErrFile(zw, "explain.err.txt", err) - terror.Log(err) - return - } - fw, err := zw.Create("explain.txt") - if err != nil { - terror.Log(err) - return - } - for _, row := range sRows { - fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) - } - } else { - // Otherwise we catch a profile and run `EXPLAIN ANALYZE` result. - ctx, cancelFunc := context.WithCancel(reqCtx) - timer := time.NewTimer(time.Second * time.Duration(timeout)) - resultChan := make(chan *explainAnalyzeResult) - go sh.getExplainAnalyze(ctx, sql, resultChan) - errChan := make(chan error) - var buf bytes.Buffer - go sh.catchCPUProfile(reqCtx, pprofTime, &buf, errChan) - select { - case result := <-resultChan: - timer.Stop() - cancelFunc() - if result.err != nil { - err = sh.writeErrFile(zw, "explain_analyze.err.txt", result.err) - terror.Log(err) - return - } - if len(result.rows) == 0 { - break - } - fw, err := zw.Create("explain_analyze.txt") - if err != nil { - terror.Log(err) - break - } - for _, row := range result.rows { - fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) - } - case <-timer.C: - cancelFunc() - } - err = dumpCPUProfile(errChan, &buf, zw) - if err != nil { - err = sh.writeErrFile(zw, "profile.err.txt", err) - terror.Log(err) - return - } - } -} - -func dumpCPUProfile(errChan chan error, buf *bytes.Buffer, zw *zip.Writer) error { - err := <-errChan - if err != nil { - return err - } - fw, err := zw.Create("profile") - if err != nil { - return err - } - _, err = fw.Write(buf.Bytes()) - if err != nil { - return err - } - return nil -} - -func (sh *sqlInfoFetcher) writeErrFile(zw *zip.Writer, name string, err error) error { - fw, err1 := zw.Create(name) - if err1 != nil { - return err1 - } - fmt.Fprintf(fw, "error: %v", err) - return nil -} - -type explainAnalyzeResult struct { - rows [][]string - err error -} - -func (sh *sqlInfoFetcher) getExplainAnalyze(ctx context.Context, sql string, resultChan chan<- *explainAnalyzeResult) { - recordSets, err := sh.s.(sqlexec.SQLExecutor).Execute(ctx, fmt.Sprintf("explain analyze %s", sql)) - if err != nil { - resultChan <- &explainAnalyzeResult{err: err} - return - } - rows, err := session.ResultSetToStringSlice(ctx, sh.s, recordSets[0]) - if err != nil { - terror.Log(err) - return - } - if len(recordSets) > 0 { - terror.Call(recordSets[0].Close) - } - resultChan <- &explainAnalyzeResult{rows: rows} -} - -func (sh *sqlInfoFetcher) catchCPUProfile(ctx context.Context, sec int, buf *bytes.Buffer, errChan chan<- error) { - if err := tracecpu.StartCPUProfile(buf); err != nil { - errChan <- err - return - } - sleepWithCtx(ctx, time.Duration(sec)*time.Second) - err := tracecpu.StopCPUProfile() - errChan <- err -} - -func (sh *sqlInfoFetcher) getStatsForTable(pair tableNamePair) (*handle.JSONTable, error) { - is := sh.do.InfoSchema() - h := sh.do.StatsHandle() - tbl, err := is.TableByName(model.NewCIStr(pair.DBName), model.NewCIStr(pair.TableName)) - if err != nil { - return nil, err - } - js, err := h.DumpStatsToJSON(pair.DBName, tbl.Meta(), nil) - return js, err -} - -func (sh *sqlInfoFetcher) getShowCreateTable(pair tableNamePair, zw *zip.Writer) error { - recordSets, err := sh.s.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show create table `%v`.`%v`", pair.DBName, pair.TableName)) - if len(recordSets) > 0 { - defer terror.Call(recordSets[0].Close) - } - if err != nil { - return err - } - sRows, err := session.ResultSetToStringSlice(context.Background(), sh.s, recordSets[0]) - if err != nil { - terror.Log(err) - return nil - } - fw, err := zw.Create(fmt.Sprintf("%v.%v.schema.txt", pair.DBName, pair.TableName)) - if err != nil { - terror.Log(err) - return nil - } - for _, row := range sRows { - fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) - } - return nil -} - -func (sh *sqlInfoFetcher) extractTableNames(sql, curDB string) (map[tableNamePair]struct{}, error) { - p := parser.New() - charset, collation := sh.s.GetSessionVars().GetCharsetInfo() - stmts, _, err := p.Parse(sql, charset, collation) - if err != nil { - return nil, err - } - if len(stmts) > 1 { - return nil, errors.Errorf("Only 1 statement is allowed") - } - extractor := &tableNameExtractor{ - curDB: curDB, - names: make(map[tableNamePair]struct{}), - } - stmts[0].Accept(extractor) - return extractor.names, nil -}