diff --git a/ddl/ddl.go b/ddl/ddl.go index de0a0894359a9..4a66300a5102d 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -218,6 +218,8 @@ type DDL interface { SchemaSyncer() SchemaSyncer // OwnerManager gets the owner manager. OwnerManager() owner.Manager + // GetID gets the ddl ID. + GetID() string // GetTableMaxRowID gets the max row ID of a normal table or a partition. GetTableMaxRowID(startTS uint64, tbl table.Table) (int64, bool, error) // SetBinlogClient sets the binlog client for DDL worker. It's exported for testing. @@ -433,6 +435,11 @@ func (d *ddl) OwnerManager() owner.Manager { return d.ownerManager } +// GetID implements DDL.GetID interface. +func (d *ddl) GetID() string { + return d.uuid +} + func checkJobMaxInterval(job *model.Job) time.Duration { // The job of adding index takes more time to process. // So it uses the longer time. diff --git a/ddl/syncer.go b/ddl/syncer.go index 9d4da63e2b26f..50b1c97e5a7b2 100644 --- a/ddl/syncer.go +++ b/ddl/syncer.go @@ -103,7 +103,11 @@ func NewSchemaSyncer(etcdCli *clientv3.Client, id string) SchemaSyncer { } } -func (s *schemaVersionSyncer) putKV(ctx context.Context, retryCnt int, key, val string, +// PutKVToEtcd puts key value to etcd. +// etcdCli is client of etcd. +// retryCnt is retry time when an error occurs. +// opts is configures of etcd Operations. +func PutKVToEtcd(ctx context.Context, etcdCli *clientv3.Client, retryCnt int, key, val string, opts ...clientv3.OpOption) error { var err error for i := 0; i < retryCnt; i++ { @@ -112,12 +116,12 @@ func (s *schemaVersionSyncer) putKV(ctx context.Context, retryCnt int, key, val } childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) - _, err = s.etcdCli.Put(childCtx, key, val, opts...) + _, err = etcdCli.Put(childCtx, key, val, opts...) cancel() if err == nil { return nil } - log.Warnf("[syncer] put schema version %s failed %v no.%d", val, err, i) + log.Warnf("[etcd-cli] put key: %s value: %s failed %v no.%d", key, val, err, i) time.Sleep(keyOpRetryInterval) } return errors.Trace(err) @@ -148,7 +152,7 @@ func (s *schemaVersionSyncer) Init(ctx context.Context) error { s.mu.globalVerCh = s.etcdCli.Watch(ctx, DDLGlobalSchemaVersion) s.mu.Unlock() - err = s.putKV(ctx, keyOpDefaultRetryCnt, s.selfSchemaVerPath, InitialVersion, + err = PutKVToEtcd(ctx, s.etcdCli, keyOpDefaultRetryCnt, s.selfSchemaVerPath, InitialVersion, clientv3.WithLease(s.session.Lease())) return errors.Trace(err) } @@ -176,7 +180,7 @@ func (s *schemaVersionSyncer) Restart(ctx context.Context) error { childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) defer cancel() - err = s.putKV(childCtx, putKeyRetryUnlimited, s.selfSchemaVerPath, InitialVersion, + err = PutKVToEtcd(childCtx, s.etcdCli, putKeyRetryUnlimited, s.selfSchemaVerPath, InitialVersion, clientv3.WithLease(s.session.Lease())) return errors.Trace(err) @@ -214,7 +218,7 @@ func (s *schemaVersionSyncer) WatchGlobalSchemaVer(ctx context.Context) { func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, version int64) error { startTime := time.Now() ver := strconv.FormatInt(version, 10) - err := s.putKV(ctx, putKeyNoRetry, s.selfSchemaVerPath, ver, + err := PutKVToEtcd(ctx, s.etcdCli, putKeyNoRetry, s.selfSchemaVerPath, ver, clientv3.WithLease(s.session.Lease())) metrics.UpdateSelfVersionHistogram.WithLabelValues(metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -227,8 +231,7 @@ func (s *schemaVersionSyncer) OwnerUpdateGlobalVersion(ctx context.Context, vers ver := strconv.FormatInt(version, 10) // TODO: If the version is larger than the original global version, we need set the version. // Otherwise, we'd better set the original global version. - err := s.putKV(ctx, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) - + err := PutKVToEtcd(ctx, s.etcdCli, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerUpdateGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } @@ -241,15 +244,22 @@ func (s *schemaVersionSyncer) RemoveSelfVersionPath() error { metrics.DeploySyncerHistogram.WithLabelValues(metrics.SyncerClear, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() + err = DeleteKeyFromEtcd(s.selfSchemaVerPath, s.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + return errors.Trace(err) +} + +// DeleteKeyFromEtcd deletes key value from etcd. +func DeleteKeyFromEtcd(key string, etcdCli *clientv3.Client, retryCnt int, timeout time.Duration) error { + var err error ctx := context.Background() - for i := 0; i < keyOpDefaultRetryCnt; i++ { - childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) - _, err = s.etcdCli.Delete(childCtx, s.selfSchemaVerPath) + for i := 0; i < retryCnt; i++ { + childCtx, cancel := context.WithTimeout(ctx, timeout) + _, err = etcdCli.Delete(childCtx, key) cancel() if err == nil { return nil } - log.Warnf("[syncer] remove schema version path %s failed %v no.%d", s.selfSchemaVerPath, err, i) + log.Warnf("[etcd-cli] delete key %s failed %v no.%d", key, err, i) } return errors.Trace(err) } @@ -283,7 +293,7 @@ func (s *schemaVersionSyncer) MustGetGlobalVersion(ctx context.Context) (int64, if err != nil { continue } - if err == nil && len(resp.Kvs) > 0 { + if len(resp.Kvs) > 0 { var ver int ver, err = strconv.Atoi(string(resp.Kvs[0].Value)) if err == nil { diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index 5915e29445701..819c7c1933c0a 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -96,6 +96,18 @@ timezone.* curl http://{TiDBIP}:10080/settings ``` +1. Get TiDB server information. + + ```shell + curl http://{TiDBIP}:10080/info + ``` + +1. Get TiDB cluster all servers information. + + ```shell + curl http://{TiDBIP}:10080/info/all + ``` + 1. Enable/Disable TiDB server general log ```shell diff --git a/domain/domain.go b/domain/domain.go index 3007b8afbb397..7f80aa0baf5f0 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -52,6 +52,7 @@ type Domain struct { statsHandle unsafe.Pointer statsLease time.Duration ddl ddl.DDL + info *InfoSyncer m sync.Mutex SchemaValidator SchemaValidator sysSessionPool *pools.ResourcePool @@ -251,6 +252,11 @@ func (do *Domain) DDL() ddl.DDL { return do.ddl } +// InfoSyncer gets infoSyncer from domain. +func (do *Domain) InfoSyncer() *InfoSyncer { + return do.info +} + // Store gets KV store from domain. func (do *Domain) Store() kv.Storage { return do.store @@ -359,6 +365,8 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) { break } do.SchemaValidator.Restart() + case <-do.info.Done(): + do.info.Restart(context.Background()) case <-do.exit: return } @@ -392,6 +400,9 @@ func (do *Domain) Close() { if do.ddl != nil { terror.Log(errors.Trace(do.ddl.Stop())) } + if do.info != nil { + do.info.RemoveServerInfo() + } close(do.exit) if do.etcdClient != nil { terror.Log(errors.Trace(do.etcdClient.Close())) @@ -500,6 +511,11 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R if err != nil { return errors.Trace(err) } + do.info = NewInfoSyncer(do.ddl.GetID(), do.etcdClient) + err = do.info.Init(ctx) + if err != nil { + return errors.Trace(err) + } err = do.Reload() if err != nil { return errors.Trace(err) diff --git a/domain/info.go b/domain/info.go new file mode 100644 index 0000000000000..6ab5a5f5d5bdb --- /dev/null +++ b/domain/info.go @@ -0,0 +1,220 @@ +// Copyright 2018 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 domain + +import ( + "encoding/json" + "fmt" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/coreos/etcd/clientv3/concurrency" + "github.com/juju/errors" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/printer" + log "github.com/sirupsen/logrus" + "golang.org/x/net/context" +) + +const ( + // ServerInformationPath store server information such as IP, port and so on. + ServerInformationPath = "/tidb/server/info" + // keyOpDefaultRetryCnt is the default retry count for etcd store. + keyOpDefaultRetryCnt = 2 + // keyOpDefaultTimeout is the default time out for etcd store. + keyOpDefaultTimeout = 1 * time.Second +) + +// InfoSessionTTL is the etcd session's TTL in seconds. It's exported for testing. +var InfoSessionTTL = 1 * 60 + +// InfoSyncer stores server info to etcd when the tidb-server starts and delete when tidb-server shuts down. +type InfoSyncer struct { + etcdCli *clientv3.Client + info *ServerInfo + serverInfoPath string + session *concurrency.Session +} + +// ServerInfo is server static information. +// It will not be updated when tidb-server running. So please only put static information in ServerInfo struct. +type ServerInfo struct { + ServerVersionInfo + ID string `json:"ddl_id"` + IP string `json:"ip"` + Port uint `json:"listening_port"` + StatusPort uint `json:"status_port"` + Lease string `json:"lease"` +} + +// ServerVersionInfo is the server version and git_hash. +type ServerVersionInfo struct { + Version string `json:"version"` + GitHash string `json:"git_hash"` +} + +// NewInfoSyncer return new InfoSyncer. It is exported for testing. +func NewInfoSyncer(id string, etcdCli *clientv3.Client) *InfoSyncer { + return &InfoSyncer{ + etcdCli: etcdCli, + info: getServerInfo(id), + serverInfoPath: fmt.Sprintf("%s/%s", ServerInformationPath, id), + } +} + +// Init creates a new etcd session and stores server info to etcd. +func (is *InfoSyncer) Init(ctx context.Context) error { + return errors.Trace(is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt)) +} + +// GetServerInfo gets self server static information. +func (is *InfoSyncer) GetServerInfo() *ServerInfo { + return is.info +} + +// GetServerInfoByID gets server static information from etcd. +func (is *InfoSyncer) GetServerInfoByID(ctx context.Context, id string) (*ServerInfo, error) { + if is.etcdCli == nil || id == is.info.ID { + return is.info, nil + } + key := fmt.Sprintf("%s/%s", ServerInformationPath, id) + infoMap, err := getInfo(ctx, is.etcdCli, key, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + if err != nil { + return nil, errors.Trace(err) + } + info, ok := infoMap[id] + if !ok { + return nil, errors.Errorf("[info-syncer] get %s failed", key) + } + return info, nil +} + +// GetAllServerInfo gets all servers static information from etcd. +func (is *InfoSyncer) GetAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { + allInfo := make(map[string]*ServerInfo) + if is.etcdCli == nil { + allInfo[is.info.ID] = is.info + return allInfo, nil + } + allInfo, err := getInfo(ctx, is.etcdCli, ServerInformationPath, keyOpDefaultRetryCnt, keyOpDefaultTimeout, clientv3.WithPrefix()) + if err != nil { + return nil, errors.Trace(err) + } + return allInfo, nil +} + +// storeServerInfo stores self server static information to etcd. +func (is *InfoSyncer) storeServerInfo(ctx context.Context) error { + if is.etcdCli == nil { + return nil + } + infoBuf, err := json.Marshal(is.info) + if err != nil { + return errors.Trace(err) + } + err = ddl.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.serverInfoPath, hack.String(infoBuf), clientv3.WithLease(is.session.Lease())) + return errors.Trace(err) +} + +// RemoveServerInfo remove self server static information from etcd. +func (is *InfoSyncer) RemoveServerInfo() { + if is.etcdCli == nil { + return + } + err := ddl.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + if err != nil { + log.Errorf("[info-syncer] remove server info failed %v", err) + } +} + +// Done returns a channel that closes when the info syncer is no longer being refreshed. +func (is InfoSyncer) Done() <-chan struct{} { + if is.etcdCli == nil { + return make(chan struct{}, 1) + } + return is.session.Done() +} + +// Restart restart the info syncer with new session leaseID and store server info to etcd again. +func (is *InfoSyncer) Restart(ctx context.Context) error { + return errors.Trace(is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt)) +} + +// newSessionAndStoreServerInfo creates a new etcd session and stores server info to etcd. +func (is *InfoSyncer) newSessionAndStoreServerInfo(ctx context.Context, retryCnt int) error { + if is.etcdCli == nil { + return nil + } + var err error + logPrefix := fmt.Sprintf("[Info-syncer] %s", is.serverInfoPath) + is.session, err = owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, InfoSessionTTL) + if err != nil { + return errors.Trace(err) + } + err = is.storeServerInfo(ctx) + return errors.Trace(err) +} + +// getInfo gets server information from etcd according to the key and opts. +func getInfo(ctx context.Context, etcdCli *clientv3.Client, key string, retryCnt int, timeout time.Duration, opts ...clientv3.OpOption) (map[string]*ServerInfo, error) { + var err error + var resp *clientv3.GetResponse + allInfo := make(map[string]*ServerInfo) + for i := 0; i < retryCnt; i++ { + select { + case <-ctx.Done(): + err = errors.Trace(ctx.Err()) + return nil, err + default: + } + childCtx, cancel := context.WithTimeout(ctx, timeout) + resp, err = etcdCli.Get(childCtx, key, opts...) + cancel() + if err != nil { + log.Infof("[info-syncer] get %s failed %v, continue checking.", key, err) + time.Sleep(200 * time.Millisecond) + continue + } + for _, kv := range resp.Kvs { + info := &ServerInfo{} + err = json.Unmarshal(kv.Value, info) + if err != nil { + log.Infof("[info-syncer] get %s, json.Unmarshal %v failed %v.", kv.Key, kv.Value, err) + return nil, errors.Trace(err) + } + allInfo[info.ID] = info + } + return allInfo, nil + } + return nil, errors.Trace(err) +} + +// getServerInfo gets self tidb server information. +func getServerInfo(id string) *ServerInfo { + cfg := config.GetGlobalConfig() + info := &ServerInfo{ + ID: id, + IP: cfg.AdvertiseAddress, + Port: cfg.Port, + StatusPort: cfg.Status.StatusPort, + Lease: cfg.Lease, + } + info.Version = mysql.ServerVersion + info.GitHash = printer.TiDBGitHash + return info +} diff --git a/server/http_handler.go b/server/http_handler.go index 6ac958a3faf3d..e3b2551968a60 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -334,6 +334,14 @@ type ddlHistoryJobHandler struct { *tikvHandlerTool } +type serverInfoHandler struct { + *tikvHandlerTool +} + +type allServerInfoHandler struct { + *tikvHandlerTool +} + // valueHandle is the handler for get value. type valueHandler struct { } @@ -1249,3 +1257,78 @@ func (h *mvccTxnHandler) handleMvccGetByTxn(params map[string]string) (interface } return h.getMvccByStartTs(uint64(startTS), startKey, endKey) } + +// serverInfo is used to report the servers info when do http request. +type serverInfo struct { + IsOwner bool `json:"is_owner"` + *domain.ServerInfo +} + +// ServeHTTP handles request of ddl server info. +func (h serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + do, err := session.GetDomain(h.store.(kv.Storage)) + if err != nil { + writeError(w, errors.New("create session error")) + log.Error(err) + return + } + info := serverInfo{} + info.ServerInfo = do.InfoSyncer().GetServerInfo() + info.IsOwner = do.DDL().OwnerManager().IsOwner() + writeData(w, info) +} + +// clusterServerInfo is used to report cluster servers info when do http request. +type clusterServerInfo struct { + ServersNum int `json:"servers_num,omitempty"` + OwnerID string `json:"owner_id"` + IsAllServerVersionConsistent bool `json:"is_all_server_version_consistent,omitempty"` + AllServersDiffVersions []domain.ServerVersionInfo `json:"all_servers_diff_versions,omitempty"` + AllServersInfo map[string]*domain.ServerInfo `json:"all_servers_info,omitempty"` +} + +// ServeHTTP handles request of all ddl servers info. +func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + do, err := session.GetDomain(h.store.(kv.Storage)) + if err != nil { + writeError(w, errors.New("create session error")) + log.Error(err) + return + } + ctx := context.Background() + allServersInfo, err := do.InfoSyncer().GetAllServerInfo(ctx) + if err != nil { + writeError(w, errors.New("ddl server information not found")) + log.Error(err) + return + } + ctx, cancel := context.WithTimeout(ctx, 3*time.Second) + ownerID, err := do.DDL().OwnerManager().GetOwnerID(ctx) + cancel() + if err != nil { + writeError(w, errors.New("ddl server information not found")) + log.Error(err) + return + } + allVersionsMap := map[domain.ServerVersionInfo]struct{}{} + allVersions := []domain.ServerVersionInfo{} + for _, v := range allServersInfo { + if _, ok := allVersionsMap[v.ServerVersionInfo]; ok { + continue + } + allVersionsMap[v.ServerVersionInfo] = struct{}{} + allVersions = append(allVersions, v.ServerVersionInfo) + } + clusterInfo := clusterServerInfo{ + ServersNum: len(allServersInfo), + OwnerID: ownerID, + // len(allVersions) = 1 indicates there has only 1 tidb version in cluster, so all server versions are consistent. + IsAllServerVersionConsistent: len(allVersions) == 1, + AllServersInfo: allServersInfo, + } + // if IsAllServerVersionConsistent is false, return the all tidb servers version. + if !clusterInfo.IsAllServerVersionConsistent { + clusterInfo.AllServersDiffVersions = allVersions + } + writeData(w, clusterInfo) +} diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 3404055719e82..0dc9b279a7100 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/printer" log "github.com/sirupsen/logrus" ) @@ -645,3 +646,64 @@ func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { } log.Fatalf("Failed to get profile for %d retries in every 10 ms", retryTime) } + +func (ts *HTTPHandlerTestSuite) TestServerInfo(c *C) { + ts.startServer(c) + defer ts.stopServer(c) + resp, err := http.Get("http://127.0.0.1:10090/info") + c.Assert(err, IsNil) + defer resp.Body.Close() + c.Assert(resp.StatusCode, Equals, http.StatusOK) + decoder := json.NewDecoder(resp.Body) + + info := serverInfo{} + err = decoder.Decode(&info) + c.Assert(err, IsNil) + + cfg := config.GetGlobalConfig() + c.Assert(info.IsOwner, IsTrue) + c.Assert(info.IP, Equals, cfg.AdvertiseAddress) + c.Assert(info.StatusPort, Equals, cfg.Status.StatusPort) + c.Assert(info.Lease, Equals, cfg.Lease) + c.Assert(info.Version, Equals, mysql.ServerVersion) + c.Assert(info.GitHash, Equals, printer.TiDBGitHash) + + store := ts.server.newTikvHandlerTool().store.(kv.Storage) + do, err := session.GetDomain(store.(kv.Storage)) + c.Assert(err, IsNil) + ddl := do.DDL() + c.Assert(info.ID, Equals, ddl.GetID()) +} + +func (ts *HTTPHandlerTestSuite) TestAllServerInfo(c *C) { + ts.startServer(c) + defer ts.stopServer(c) + resp, err := http.Get("http://127.0.0.1:10090/info/all") + c.Assert(err, IsNil) + defer resp.Body.Close() + c.Assert(resp.StatusCode, Equals, http.StatusOK) + decoder := json.NewDecoder(resp.Body) + + clusterInfo := clusterServerInfo{} + err = decoder.Decode(&clusterInfo) + c.Assert(err, IsNil) + + c.Assert(clusterInfo.IsAllServerVersionConsistent, IsTrue) + c.Assert(clusterInfo.ServersNum, Equals, 1) + + store := ts.server.newTikvHandlerTool().store.(kv.Storage) + do, err := session.GetDomain(store.(kv.Storage)) + c.Assert(err, IsNil) + ddl := do.DDL() + c.Assert(clusterInfo.OwnerID, Equals, ddl.GetID()) + serverInfo, ok := clusterInfo.AllServersInfo[ddl.GetID()] + c.Assert(ok, Equals, true) + + cfg := config.GetGlobalConfig() + c.Assert(serverInfo.IP, Equals, cfg.AdvertiseAddress) + c.Assert(serverInfo.StatusPort, Equals, cfg.Status.StatusPort) + c.Assert(serverInfo.Lease, Equals, cfg.Lease) + c.Assert(serverInfo.Version, Equals, mysql.ServerVersion) + c.Assert(serverInfo.GitHash, Equals, printer.TiDBGitHash) + c.Assert(serverInfo.ID, Equals, ddl.GetID()) +} diff --git a/server/http_status.go b/server/http_status.go index 0c6012fa31b49..357594d3cb695 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -52,8 +52,12 @@ func (s *Server) startHTTPServer() { router.Handle("/schema/{db}/{table}", schemaHandler{tikvHandlerTool}) router.Handle("/tables/{colID}/{colTp}/{colFlag}/{colLen}", valueHandler{}) router.Handle("/ddl/history", ddlHistoryJobHandler{tikvHandlerTool}) + + // HTTP path for get server info. + router.Handle("/info", serverInfoHandler{tikvHandlerTool}) + router.Handle("/info/all", allServerInfoHandler{tikvHandlerTool}) if s.cfg.Store == "tikv" { - // HTTP path for tikv + // HTTP path for tikv. router.Handle("/tables/{db}/{table}/regions", tableHandler{tikvHandlerTool, opTableRegions}) router.Handle("/tables/{db}/{table}/scatter", tableHandler{tikvHandlerTool, opTableScatter}) router.Handle("/tables/{db}/{table}/stop-scatter", tableHandler{tikvHandlerTool, opStopTableScatter}) diff --git a/util/memory/action.go b/util/memory/action.go index d06b33c22d733..06a2d39deded8 100644 --- a/util/memory/action.go +++ b/util/memory/action.go @@ -45,13 +45,13 @@ func (a *LogOnExceed) Action(t *Tracker) { } } -// PanicOnExceed panics when when memory usage exceeds memory quota. +// PanicOnExceed panics when memory usage exceeds memory quota. type PanicOnExceed struct { mutex sync.Mutex // For synchronization. acted bool } -// Action panics when when memory usage exceeds memory quota. +// Action panics when memory usage exceeds memory quota. func (a *PanicOnExceed) Action(t *Tracker) { a.mutex.Lock() if a.acted {