diff --git a/Makefile b/Makefile index 48ff0edc15129..e33389531a3c3 100644 --- a/Makefile +++ b/Makefile @@ -220,7 +220,7 @@ tools/bin/goword: tools/check/go.mod tools/bin/gometalinter: tools/check/go.mod cd tools/check; \ - $(GO) build -o ../bin/gometalinter gopkg.in/alecthomas/gometalinter.v2 + $(GO) build -o ../bin/gometalinter gopkg.in/alecthomas/gometalinter.v3 tools/bin/gosec: tools/check/go.mod cd tools/check; \ diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index abebc1277395f..fc7d892385f63 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -14,10 +14,12 @@ package bindinfo_test import ( + "context" "flag" "fmt" "os" "testing" + "time" . "github.com/pingcap/check" "github.com/pingcap/parser" @@ -115,22 +117,106 @@ func (s *testSuite) TestBindParse(c *C) { sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s')`, originSQL, bindSQL, defaultDb, status, charset, collation) tk.MustExec(sql) - bindHandle := bindinfo.NewHandle() - bindCacheUpdater := bindinfo.NewBindCacheUpdater(tk.Se, bindHandle, s.Parser) - err := bindCacheUpdater.Update(true) + bindHandle := bindinfo.NewBindHandle(tk.Se, s.Parser) + err := bindHandle.Update(true) c.Check(err, IsNil) - c.Check(len(bindHandle.Get()), Equals, 1) + c.Check(bindHandle.Size(), Equals, 1) - hash := parser.DigestHash("select * from t") - bindData := bindHandle.Get()[hash] + bindData := bindHandle.GetBindRecord("select * from t", "test") c.Check(bindData, NotNil) - c.Check(len(bindData), Equals, 1) - c.Check(bindData[0].OriginalSQL, Equals, "select * from t") - c.Check(bindData[0].BindSQL, Equals, "select * from t use index(index_t)") - c.Check(bindData[0].Db, Equals, "test") - c.Check(bindData[0].Status, Equals, "using") - c.Check(bindData[0].Charset, Equals, "utf8mb4") - c.Check(bindData[0].Collation, Equals, "utf8mb4_bin") - c.Check(bindData[0].CreateTime, NotNil) - c.Check(bindData[0].UpdateTime, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t") + c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t)") + c.Check(bindData.Db, Equals, "test") + c.Check(bindData.Status, Equals, "using") + c.Check(bindData.Charset, Equals, "utf8mb4") + c.Check(bindData.Collation, Equals, "utf8mb4_bin") + c.Check(bindData.CreateTime, NotNil) + c.Check(bindData.UpdateTime, NotNil) +} + +func (s *testSuite) TestGlobalBinding(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t(i int, s varchar(20))") + tk.MustExec("create table t1(i int, s varchar(20))") + tk.MustExec("create index index_t on t(i,s)") + + _, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100") + c.Assert(err, IsNil, Commentf("err %v", err)) + + time.Sleep(time.Second * 1) + _, err = tk.Exec("create global binding for select * from t where i>99 using select * from t use index(index_t) where i>99") + c.Assert(err, IsNil) + + bindData := s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") + c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") + c.Check(bindData.Db, Equals, "test") + c.Check(bindData.Status, Equals, "using") + c.Check(bindData.Charset, NotNil) + c.Check(bindData.Collation, NotNil) + c.Check(bindData.CreateTime, NotNil) + c.Check(bindData.UpdateTime, NotNil) + + rs, err := tk.Exec("show global bindings") + c.Assert(err, IsNil) + chk := rs.NewRecordBatch() + err = rs.Next(context.TODO(), chk) + c.Check(err, IsNil) + c.Check(chk.NumRows(), Equals, 1) + row := chk.GetRow(0) + c.Check(row.GetString(0), Equals, "select * from t where i > ?") + c.Check(row.GetString(1), Equals, "select * from t use index(index_t) where i>99") + c.Check(row.GetString(2), Equals, "test") + c.Check(row.GetString(3), Equals, "using") + c.Check(row.GetTime(4), NotNil) + c.Check(row.GetTime(5), NotNil) + c.Check(row.GetString(6), NotNil) + c.Check(row.GetString(7), NotNil) + + bindHandle := bindinfo.NewBindHandle(tk.Se, s.Parser) + err = bindHandle.Update(true) + c.Check(err, IsNil) + c.Check(bindHandle.Size(), Equals, 1) + + bindData = bindHandle.GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") + c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") + c.Check(bindData.Db, Equals, "test") + c.Check(bindData.Status, Equals, "using") + c.Check(bindData.Charset, NotNil) + c.Check(bindData.Collation, NotNil) + c.Check(bindData.CreateTime, NotNil) + c.Check(bindData.UpdateTime, NotNil) + + _, err = tk.Exec("DROP global binding for select * from t where i>100") + c.Check(err, IsNil) + bindData = s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, IsNil) + + bindHandle = bindinfo.NewBindHandle(tk.Se, s.Parser) + err = bindHandle.Update(true) + c.Check(err, IsNil) + c.Check(bindHandle.Size(), Equals, 0) + + bindData = bindHandle.GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, IsNil) + + rs, err = tk.Exec("show global bindings") + c.Assert(err, IsNil) + chk = rs.NewRecordBatch() + err = rs.Next(context.TODO(), chk) + c.Check(err, IsNil) + c.Check(chk.NumRows(), Equals, 0) + + _, err = tk.Exec("delete from mysql.bind_info") + c.Assert(err, IsNil) + + _, err = tk.Exec("create global binding for select * from t using select * from t1 use index for join(index_t)") + c.Assert(err, NotNil, Commentf("err %v", err)) } diff --git a/bindinfo/cache.go b/bindinfo/cache.go index bbb083100f98a..b8731628ec080 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -14,58 +14,35 @@ package bindinfo import ( - "context" - "fmt" - "sync/atomic" - - "github.com/pingcap/errors" - "github.com/pingcap/parser" "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/sqlexec" ) const ( - using = "using" + // using is the bind info's in use status. + using = "using" + // deleted is the bind info's deleted status. deleted = "deleted" ) // bindMeta stores the basic bind info and bindSql astNode. type bindMeta struct { - *bindRecord + *BindRecord ast ast.StmtNode //ast will be used to do query sql bind check } // cache is a k-v map, key is original sql, value is a slice of bindMeta. type cache map[string][]*bindMeta -// Handle holds an atomic cache. -type Handle struct { - atomic.Value -} - -// BindCacheUpdater is used to update the global cache. -// BindCacheUpdater will update the bind cache per 3 seconds in domain -// gorountine loop. When the tidb server first startup, the updater will load -// all bind info into memory; then load diff bind info per 3 second. -type BindCacheUpdater struct { - ctx sessionctx.Context - - parser *parser.Parser - lastUpdateTime types.Time - globalHandle *Handle -} - -type bindRecord struct { +// BindRecord represents a sql bind record retrieved from the storage. +type BindRecord struct { OriginalSQL string BindSQL string Db string // Status represents the status of the binding. It can only be one of the following values: - // 1. deleted: bindRecord is deleted, can not be used anymore. - // 2. using: bindRecord is in the normal active mode. + // 1. deleted: BindRecord is deleted, can not be used anymore. + // 2. using: BindRecord is in the normal active mode. Status string CreateTime types.Time UpdateTime types.Time @@ -73,86 +50,8 @@ type bindRecord struct { Collation string } -// NewBindCacheUpdater creates a new BindCacheUpdater. -func NewBindCacheUpdater(ctx sessionctx.Context, handle *Handle, parser *parser.Parser) *BindCacheUpdater { - return &BindCacheUpdater{ - ctx: ctx, - parser: parser, - globalHandle: handle, - } -} - -// NewHandle creates a Handle with a cache. -func NewHandle() *Handle { - handle := &Handle{} - return handle -} - -// Get gets cache from a Handle. -func (h *Handle) Get() cache { - bc := h.Load() - if bc != nil { - return bc.(map[string][]*bindMeta) - } - return make(map[string][]*bindMeta) -} - -// LoadDiff is used to load new bind info to cache bc. -func (bindCacheUpdater *BindCacheUpdater) loadDiff(sql string, bc cache) error { - recordSets, err := bindCacheUpdater.ctx.(sqlexec.SQLExecutor).Execute(context.Background(), sql) - if err != nil { - return errors.Trace(err) - } - - rs := recordSets[0] - defer terror.Call(rs.Close) - chkBatch := rs.NewRecordBatch() - for { - err = rs.Next(context.TODO(), chkBatch) - if err != nil || chkBatch.NumRows() == 0 { - return errors.Trace(err) - } - - it := chunk.NewIterator4Chunk(chkBatch.Chunk) - for row := it.Begin(); row != it.End(); row = it.Next() { - record := newBindMeta(row) - err = bc.appendNode(record, bindCacheUpdater.parser) - if err != nil { - return err - } - if record.UpdateTime.Compare(bindCacheUpdater.lastUpdateTime) == 1 { - bindCacheUpdater.lastUpdateTime = record.UpdateTime - } - } - } -} - -// Update updates the BindCacheUpdater's cache. -// The `fullLoad` is true only when tidb first startup, otherwise it is false. -func (bindCacheUpdater *BindCacheUpdater) Update(fullLoad bool) (err error) { - var sql string - bc := bindCacheUpdater.globalHandle.Get() - newBc := make(map[string][]*bindMeta, len(bc)) - for hash, bindDataArr := range bc { - newBc[hash] = append(newBc[hash], bindDataArr...) - } - - if fullLoad { - sql = "select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation from mysql.bind_info" - } else { - sql = fmt.Sprintf("select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation from mysql.bind_info where update_time > \"%s\"", bindCacheUpdater.lastUpdateTime.String()) - } - err = bindCacheUpdater.loadDiff(sql, newBc) - if err != nil { - return errors.Trace(err) - } - - bindCacheUpdater.globalHandle.Store(newBc) - return nil -} - -func newBindMeta(row chunk.Row) *bindRecord { - return &bindRecord{ +func newBindRecord(row chunk.Row) *BindRecord { + return &BindRecord{ OriginalSQL: row.GetString(0), BindSQL: row.GetString(1), Db: row.GetString(2), @@ -163,31 +62,3 @@ func newBindMeta(row chunk.Row) *bindRecord { Collation: row.GetString(7), } } - -func (b cache) appendNode(newBindRecord *bindRecord, sparser *parser.Parser) error { - hash := parser.DigestHash(newBindRecord.OriginalSQL) - if bindArr, ok := b[hash]; ok { - for idx, v := range bindArr { - if v.OriginalSQL == newBindRecord.OriginalSQL && v.Db == newBindRecord.Db { - b[hash] = append(b[hash][:idx], b[hash][idx+1:]...) - if len(b[hash]) == 0 { - delete(b, hash) - } - break - } - } - } - if newBindRecord.Status == deleted { - return nil - } - stmtNodes, _, err := sparser.Parse(newBindRecord.BindSQL, newBindRecord.Charset, newBindRecord.Collation) - if err != nil { - return errors.Trace(err) - } - newNode := &bindMeta{ - bindRecord: newBindRecord, - ast: stmtNodes[0], - } - b[hash] = append(b[hash], newNode) - return nil -} diff --git a/bindinfo/handle.go b/bindinfo/handle.go new file mode 100644 index 0000000000000..0a8e62d3e76e8 --- /dev/null +++ b/bindinfo/handle.go @@ -0,0 +1,388 @@ +// 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 bindinfo + +import ( + "bytes" + "context" + "fmt" + "go.uber.org/zap" + "sync" + "sync/atomic" + + "github.com/pingcap/parser" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" +) + +// BindHandle is used to handle all the sql bind operations. +type BindHandle struct { + sctx struct { + sync.Mutex + sessionctx.Context + } + + // bindInfo caches the sql bind info from storage. + // + // The Mutex protects that there is only one goroutine changes the content + // of atmoic.Value. + // + // NOTE: Concurrent Value Write: + // + // bindInfo.Lock() + // newCache := bindInfo.Value.Load() + // do the write operation on the newCache + // bindInfo.Value.Store(newCache) + // + // NOTE: Concurrent Value Read: + // + // cache := bindInfo.Load(). + // read the content + // + bindInfo struct { + sync.Mutex + atomic.Value + } + + parser *parser.Parser + lastUpdateTime types.Time +} + +// NewBindHandle creates a new BindHandle. +func NewBindHandle(ctx sessionctx.Context, parser *parser.Parser) *BindHandle { + handle := &BindHandle{parser: parser} + handle.sctx.Context = ctx + handle.bindInfo.Value.Store(make(cache, 32)) + return handle +} + +// Update updates the global sql bind cache. +func (h *BindHandle) Update(fullLoad bool) (err error) { + sql := "select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation from mysql.bind_info" + if !fullLoad { + sql += " where update_time >= \"" + h.lastUpdateTime.String() + "\"" + } + + // No need to acquire the session context lock for ExecRestrictedSQL, it + // uses another background session. + rows, _, err := h.sctx.Context.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(nil, sql) + if err != nil { + return err + } + + // Make sure there is only one goroutine writes the cache. + h.bindInfo.Lock() + newCache := h.bindInfo.Value.Load().(cache).copy() + defer func() { + h.bindInfo.Value.Store(newCache) + h.bindInfo.Unlock() + }() + + for _, row := range rows { + hash, meta, err := h.newBindMeta(newBindRecord(row)) + // Update lastUpdateTime to the newest one. + if meta.UpdateTime.Compare(h.lastUpdateTime) > 0 { + h.lastUpdateTime = meta.UpdateTime + } + if err != nil { + logutil.Logger(context.Background()).Error("update bindinfo failed", zap.Error(err)) + continue + } + + newCache.removeStaleBindMetas(hash, meta) + if meta.Status == using { + newCache[hash] = append(newCache[hash], meta) + } + } + return nil +} + +// AddBindRecord adds a BindRecord to the storage and bindMeta to the cache. +func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { + exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) + h.sctx.Lock() + _, err = exec.Execute(context.TODO(), "BEGIN") + if err != nil { + h.sctx.Unlock() + return + } + + defer func() { + if err != nil { + _, err1 := exec.Execute(context.TODO(), "ROLLBACK") + h.sctx.Unlock() + terror.Log(err1) + return + } + + _, err = exec.Execute(context.TODO(), "COMMIT") + h.sctx.Unlock() + if err != nil { + return + } + + // update the bindMeta to the cache. + hash, meta, err1 := h.newBindMeta(record) + if err1 != nil { + err = err1 + return + } + + h.appendBindMeta(hash, meta) + }() + + // remove all the unused sql binds. + _, err = exec.Execute(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db)) + if err != nil { + return err + } + + txn, err1 := h.sctx.Context.Txn(true) + if err1 != nil { + return err1 + } + record.CreateTime = types.Time{ + Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), + Type: mysql.TypeDatetime, + Fsp: 3, + } + record.UpdateTime = record.CreateTime + record.Status = using + record.BindSQL = h.getEscapeCharacter(record.BindSQL) + + // insert the BindRecord to the storage. + _, err = exec.Execute(context.TODO(), h.insertBindInfoSQL(record)) + return err +} + +// DropBindRecord drops a BindRecord to the storage and bindMeta int the cache. +func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { + exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) + h.sctx.Lock() + + _, err = exec.Execute(context.TODO(), "BEGIN") + if err != nil { + h.sctx.Unlock() + return + } + + defer func() { + if err != nil { + _, err1 := exec.Execute(context.TODO(), "ROLLBACK") + h.sctx.Unlock() + terror.Log(err1) + return + } + + _, err = exec.Execute(context.TODO(), "COMMIT") + h.sctx.Unlock() + if err != nil { + return + } + + hash, meta := h.newBindMetaWithoutAst(record) + h.removeBindMeta(hash, meta) + }() + + txn, err1 := h.sctx.Context.Txn(true) + if err1 != nil { + return err1 + } + + updateTs := types.Time{ + Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), + Type: mysql.TypeDatetime, + Fsp: 3, + } + record.Status = deleted + record.UpdateTime = updateTs + + _, err = exec.Execute(context.TODO(), h.logicalDeleteBindInfoSQL(record.OriginalSQL, record.Db, updateTs)) + return err +} + +// Size return the size of bind info cache. +func (h *BindHandle) Size() int { + size := 0 + for _, bindRecords := range h.bindInfo.Load().(cache) { + size += len(bindRecords) + } + return size +} + +// GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. +func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *bindMeta { + hash := parser.DigestHash(normdOrigSQL) + bindRecords := h.bindInfo.Load().(cache)[hash] + if bindRecords != nil { + for _, bindRecord := range bindRecords { + if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { + return bindRecord + } + } + } + return nil +} + +// GetAllBindRecord return all bind record in cache. +func (h *BindHandle) GetAllBindRecord() (bindRecords []*bindMeta) { + bindRecordMap := h.bindInfo.Load().(cache) + for _, bindRecord := range bindRecordMap { + bindRecords = append(bindRecords, bindRecord...) + } + return bindRecords +} + +func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *bindMeta, err error) { + hash = parser.DigestHash(record.OriginalSQL) + stmtNodes, _, err := h.parser.Parse(record.BindSQL, record.Charset, record.Collation) + if err != nil { + return "", nil, err + } + meta = &bindMeta{BindRecord: record, ast: stmtNodes[0]} + return hash, meta, nil +} + +func (h *BindHandle) newBindMetaWithoutAst(record *BindRecord) (hash string, meta *bindMeta) { + hash = parser.DigestHash(record.OriginalSQL) + meta = &bindMeta{BindRecord: record} + return hash, meta +} + +// appendBindMeta addes the bindMeta to the cache, all the stale bindMetas are +// removed from the cache after this operation. +func (h *BindHandle) appendBindMeta(hash string, meta *bindMeta) { + // Make sure there is only one goroutine writes the cache. + h.bindInfo.Lock() + newCache := h.bindInfo.Value.Load().(cache).copy() + defer func() { + h.bindInfo.Value.Store(newCache) + h.bindInfo.Unlock() + }() + + newCache.removeStaleBindMetas(hash, meta) + newCache[hash] = append(newCache[hash], meta) +} + +// removeBindMeta removes the bindMeta from the cache. +func (h *BindHandle) removeBindMeta(hash string, meta *bindMeta) { + h.bindInfo.Lock() + newCache := h.bindInfo.Value.Load().(cache).copy() + defer func() { + h.bindInfo.Value.Store(newCache) + h.bindInfo.Unlock() + }() + + newCache.removeDeletedBindMeta(hash, meta) +} + +// removeDeletedBindMeta removes all the bindMeta which originSQL and db are the same with the parameter's meta. +func (c cache) removeDeletedBindMeta(hash string, meta *bindMeta) { + metas, ok := c[hash] + if !ok { + return + } + + for i := len(metas) - 1; i >= 0; i-- { + if meta.isSame(meta) { + metas = append(metas[:i], metas[i+1:]...) + if len(metas) == 0 { + delete(c, hash) + return + } + } + } +} + +// removeStaleBindMetas removes all the stale bindMeta in the cache. +func (c cache) removeStaleBindMetas(hash string, meta *bindMeta) { + metas, ok := c[hash] + if !ok { + return + } + + // remove stale bindMetas. + for i := len(metas) - 1; i >= 0; i-- { + if metas[i].isStale(meta) { + metas = append(metas[:i], metas[i+1:]...) + if len(metas) == 0 { + delete(c, hash) + return + } + } + } +} + +func (c cache) copy() cache { + newCache := make(cache, len(c)) + for k, v := range c { + newCache[k] = v + } + return newCache +} + +// isStale checks whether this bindMeta is stale compared with the other bindMeta. +func (m *bindMeta) isStale(other *bindMeta) bool { + return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db && + m.UpdateTime.Compare(other.UpdateTime) <= 0 +} + +func (m *bindMeta) isSame(other *bindMeta) bool { + return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db +} + +func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db string) string { + return fmt.Sprintf( + "DELETE FROM mysql.bind_info WHERE original_sql='%s' AND default_db='%s'", + normdOrigSQL, + db, + ) +} + +func (h *BindHandle) insertBindInfoSQL(record *BindRecord) string { + return fmt.Sprintf(`INSERT INTO mysql.bind_info VALUES ('%s', '%s', '%s', '%s', '%s', '%s','%s', '%s')`, + record.OriginalSQL, + record.BindSQL, + record.Db, + record.Status, + record.CreateTime, + record.UpdateTime, + record.Charset, + record.Collation, + ) +} + +func (h *BindHandle) logicalDeleteBindInfoSQL(normdOrigSQL, db string, updateTs types.Time) string { + return fmt.Sprintf(`UPDATE mysql.bind_info SET status='%s',update_time='%s' WHERE original_sql='%s' and default_db='%s'`, + deleted, + updateTs, + normdOrigSQL, + db) +} + +func (h *BindHandle) getEscapeCharacter(str string) string { + var buffer bytes.Buffer + for _, v := range str { + if v == '\'' || v == '"' || v == '\\' { + buffer.WriteString("\\") + } + buffer.WriteString(string(v)) + } + return buffer.String() +} diff --git a/cmd/explaintest/r/access_path_selection.result b/cmd/explaintest/r/access_path_selection.result index f9a146c8b8331..ec0309e75dee7 100644 --- a/cmd/explaintest/r/access_path_selection.result +++ b/cmd/explaintest/r/access_path_selection.result @@ -55,7 +55,7 @@ ANALYZE TABLE unknown_correlation; EXPLAIN SELECT * FROM unknown_correlation WHERE a = 2 ORDER BY id limit 1; id count task operator info Limit_11 1.00 root offset:0, count:1 -└─TableReader_22 1.00 root data:Limit_21 - └─Limit_21 1.00 cop offset:0, count:1 - └─Selection_20 1.00 cop eq(test.unknown_correlation.a, 2) - └─TableScan_19 4.17 cop table:unknown_correlation, range:[-inf,+inf], keep order:true +└─TableReader_24 1.00 root data:Limit_23 + └─Limit_23 1.00 cop offset:0, count:1 + └─Selection_21 1.00 cop eq(test.unknown_correlation.a, 2) + └─TableScan_20 4.17 cop table:unknown_correlation, range:[-inf,+inf], keep order:true diff --git a/cmd/explaintest/r/explain.result b/cmd/explaintest/r/explain.result index 0055b8834a554..1fbaf91769480 100644 --- a/cmd/explaintest/r/explain.result +++ b/cmd/explaintest/r/explain.result @@ -26,14 +26,12 @@ drop table if exists t; create table t(id int primary key, a int, b int); explain select group_concat(a) from t group by id; id count task operator info -StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0, ",") -└─Projection_18 10000.00 root cast(test.t.a), test.t.id - └─TableReader_15 10000.00 root data:TableScan_14 - └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo +Projection_6 10000.00 root cast(test.t.a) +└─TableReader_8 10000.00 root data:TableScan_7 + └─TableScan_7 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select group_concat(a, b) from t group by id; id count task operator info -StreamAgg_8 8000.00 root group by:col_2, funcs:group_concat(col_0, col_1, ",") -└─Projection_18 10000.00 root cast(test.t.a), cast(test.t.b), test.t.id - └─TableReader_15 10000.00 root data:TableScan_14 - └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo +Projection_6 10000.00 root cast(test.t.a) +└─TableReader_8 10000.00 root data:TableScan_7 + └─TableScan_7 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo drop table t; diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index e23a148310563..66d38378b0f33 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -107,12 +107,12 @@ explain SELECT `ds`, `p1`, `p2`, `p3`, `p4`, `p5`, `p6_md5`, `p7_md5`, count(dic id count task operator info Projection_7 53.00 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, install_device └─Sort_8 53.00 root test.dt.ds2:desc - └─HashAgg_16 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) - └─IndexLookUp_17 53.00 root - ├─IndexScan_13 2650.00 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:pseudo + └─HashAgg_17 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) + └─IndexLookUp_18 53.00 root + ├─IndexScan_14 2650.00 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:pseudo └─HashAgg_11 53.00 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds), firstrow(test.dt.ds2), firstrow(test.dt.p1), firstrow(test.dt.p2), firstrow(test.dt.p3), firstrow(test.dt.p4), firstrow(test.dt.p5), firstrow(test.dt.p6_md5), firstrow(test.dt.p7_md5) - └─Selection_15 66.25 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) - └─TableScan_14 2650.00 cop table:dt, keep order:false, stats:pseudo + └─Selection_16 66.25 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) + └─TableScan_15 2650.00 cop table:dt, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t @@ -120,7 +120,7 @@ Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.d └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo └─IndexLookUp_33 3.33 root @@ -137,37 +137,37 @@ Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), not(isnull(gad.dic)) │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false, stats:pseudo └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info Projection_5 1.00 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, 3_col_0, 3_col_1 └─HashAgg_7 1.00 root group by:test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, funcs:count(1), count(distinct test.st.ip), firstrow(test.st.cm), firstrow(test.st.p1), firstrow(test.st.p2), firstrow(test.st.p3), firstrow(test.st.p4), firstrow(test.st.p5), firstrow(test.st.p6_md5), firstrow(test.st.p7_md5) - └─IndexLookUp_15 0.00 root - ├─IndexScan_12 250.00 cop table:st, index:t, range:[1478188800,1478275200], keep order:false, stats:pseudo - └─Selection_14 0.00 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") - └─TableScan_13 250.00 cop table:st, keep order:false, stats:pseudo + └─IndexLookUp_16 0.00 root + ├─IndexScan_13 250.00 cop table:st, index:t, range:[1478188800,1478275200], keep order:false, stats:pseudo + └─Selection_15 0.00 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") + └─TableScan_14 250.00 cop table:st, keep order:false, stats:pseudo explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 └─Limit_13 0.00 root offset:0, count:2000 └─IndexJoin_19 0.00 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic - ├─TableReader_41 0.00 root data:Selection_40 - │ └─Selection_40 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) - │ └─TableScan_39 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo + ├─TableReader_43 0.00 root data:Selection_42 + │ └─Selection_42 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─TableScan_41 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false, stats:pseudo + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false, stats:pseudo └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info Projection_5 1.00 root test.pp.pc, test.pp.cr, 3_col_0, 3_col_1, 3_col_2 └─HashAgg_7 1.00 root group by:test.pp.cr, test.pp.pc, funcs:count(distinct test.pp.uid), count(test.pp.oid), sum(test.pp.am), firstrow(test.pp.pc), firstrow(test.pp.cr) - └─IndexLookUp_21 0.00 root - ├─IndexScan_18 0.40 cop table:pp, index:uid, pi, range:[18089709 510017,18089709 510017], [18089709 520017,18089709 520017], [18090780 510017,18090780 510017], [18090780 520017,18090780 520017], keep order:false, stats:pseudo - └─Selection_20 0.00 cop eq(test.pp.ps, 2), ge(test.pp.ppt, 1478188800), lt(test.pp.ppt, 1478275200) - └─TableScan_19 0.40 cop table:pp, keep order:false, stats:pseudo + └─IndexLookUp_24 0.00 root + ├─IndexScan_21 0.40 cop table:pp, index:uid, pi, range:[18089709 510017,18089709 510017], [18089709 520017,18089709 520017], [18090780 510017,18090780 510017], [18090780 520017,18090780 520017], keep order:false, stats:pseudo + └─Selection_23 0.00 cop eq(test.pp.ps, 2), ge(test.pp.ppt, 1478188800), lt(test.pp.ppt, 1478275200) + └─TableScan_22 0.40 cop table:pp, keep order:false, stats:pseudo CREATE TABLE `tbl_001` (`a` int, `b` int); CREATE TABLE `tbl_002` (`a` int, `b` int); CREATE TABLE `tbl_003` (`a` int, `b` int); diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 5db97abb7f63e..805695e77132c 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -117,12 +117,12 @@ explain SELECT ds, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(dic) as install_dev id count task operator info Projection_7 21.40 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, install_device └─Sort_8 21.40 root test.dt.ds2:desc - └─HashAgg_16 21.40 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) - └─IndexLookUp_17 21.40 root - ├─IndexScan_13 128.32 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false + └─HashAgg_17 21.40 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) + └─IndexLookUp_18 21.40 root + ├─IndexScan_14 128.32 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false └─HashAgg_11 21.40 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds), firstrow(test.dt.ds2), firstrow(test.dt.p1), firstrow(test.dt.p2), firstrow(test.dt.p3), firstrow(test.dt.p4), firstrow(test.dt.p5), firstrow(test.dt.p6_md5), firstrow(test.dt.p7_md5) - └─Selection_15 21.43 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) - └─TableScan_14 128.32 cop table:dt, keep order:false + └─Selection_16 21.43 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) + └─TableScan_15 128.32 cop table:dt, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.bm = 0 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t @@ -133,7 +133,7 @@ Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908), not(isnull(gad.ip)) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -145,37 +145,37 @@ Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk. │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479), not(isnull(gad.dic)) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info Projection_5 39.28 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, 3_col_0, 3_col_1 └─HashAgg_7 39.28 root group by:test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, funcs:count(1), count(distinct test.st.ip), firstrow(test.st.cm), firstrow(test.st.p1), firstrow(test.st.p2), firstrow(test.st.p3), firstrow(test.st.p4), firstrow(test.st.p5), firstrow(test.st.p6_md5), firstrow(test.st.p7_md5) - └─IndexLookUp_15 39.38 root - ├─IndexScan_12 160.23 cop table:st, index:t, range:[1478188800,1478275200], keep order:false - └─Selection_14 39.38 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") - └─TableScan_13 160.23 cop table:st, keep order:false + └─IndexLookUp_16 39.38 root + ├─IndexScan_13 160.23 cop table:st, index:t, range:[1478188800,1478275200], keep order:false + └─Selection_15 39.38 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") + └─TableScan_14 160.23 cop table:st, keep order:false explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 └─Limit_13 428.32 root offset:0, count:2000 └─IndexJoin_19 428.32 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic - ├─TableReader_41 428.32 root data:Selection_40 - │ └─Selection_40 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) - │ └─TableScan_39 2000.00 cop table:dt, range:[0,+inf], keep order:false + ├─TableReader_43 428.32 root data:Selection_42 + │ └─Selection_42 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─TableScan_41 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info Projection_5 207.86 root test.pp.pc, test.pp.cr, 3_col_0, 3_col_1, 3_col_2 └─HashAgg_7 207.86 root group by:test.pp.cr, test.pp.pc, funcs:count(distinct test.pp.uid), count(test.pp.oid), sum(test.pp.am), firstrow(test.pp.pc), firstrow(test.pp.cr) - └─IndexLookUp_21 207.86 root - ├─IndexScan_15 627.00 cop table:pp, index:ps, range:[2,2], keep order:false - └─Selection_17 207.86 cop ge(test.pp.ppt, 1478188800), in(test.pp.pi, 510017, 520017), in(test.pp.uid, 18089709, 18090780), lt(test.pp.ppt, 1478275200) - └─TableScan_16 627.00 cop table:pp, keep order:false + └─IndexLookUp_24 207.86 root + ├─IndexScan_18 627.00 cop table:pp, index:ps, range:[2,2], keep order:false + └─Selection_20 207.86 cop ge(test.pp.ppt, 1478188800), in(test.pp.pi, 510017, 520017), in(test.pp.uid, 18089709, 18090780), lt(test.pp.ppt, 1478275200) + └─TableScan_19 627.00 cop table:pp, keep order:false drop table if exists tbl_001; CREATE TABLE tbl_001 (a int, b int); load stats 's/explain_complex_stats_tbl_001.json'; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index dce5e62f0e83f..400fddda2e81e 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -24,9 +24,9 @@ TableReader_5 10000.00 root data:TableScan_4 └─TableScan_4 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t1 order by c2; id count task operator info -IndexLookUp_12 10000.00 root -├─IndexScan_10 10000.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true, stats:pseudo -└─TableScan_11 10000.00 cop table:t1, keep order:false, stats:pseudo +IndexLookUp_13 10000.00 root +├─IndexScan_11 10000.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true, stats:pseudo +└─TableScan_12 10000.00 cop table:t1, keep order:false, stats:pseudo explain select * from t2 order by c2; id count task operator info Sort_4 10000.00 root test.t2.c2:asc @@ -43,11 +43,11 @@ IndexReader_6 10.00 root index:IndexScan_5 explain select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id count task operator info IndexJoin_12 4166.67 root left outer join, inner:IndexLookUp_11, outer key:test.t1.c2, inner key:test.t2.c1 -├─TableReader_24 3333.33 root data:TableScan_23 -│ └─TableScan_23 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo +├─TableReader_26 3333.33 root data:TableScan_25 +│ └─TableScan_25 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t2.c1)) - │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [eq(test.t2.c1, test.t1.c2)], keep order:false, stats:pseudo + │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [test.t1.c2], keep order:false, stats:pseudo └─TableScan_9 0.00 cop table:t2, keep order:false explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info @@ -61,8 +61,8 @@ explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 9990.00 root cast(join_agg_0) └─IndexJoin_14 9990.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 - ├─TableReader_13 1.00 root data:TableScan_12 - │ └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo + ├─TableReader_13 10.00 root data:TableScan_12 + │ └─TableScan_12 10.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo └─HashAgg_22 7992.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_23 7992.00 root data:HashAgg_17 └─HashAgg_17 7992.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) @@ -99,8 +99,8 @@ explain select c1 from t1 where c1 in (select c2 from t2); id count task operator info Projection_9 9990.00 root test.t1.c1 └─IndexJoin_12 9990.00 root inner join, inner:TableReader_11, outer key:test.t2.c2, inner key:test.t1.c1 - ├─TableReader_11 1.00 root data:TableScan_10 - │ └─TableScan_10 1.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo + ├─TableReader_11 10.00 root data:TableScan_10 + │ └─TableScan_10 10.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo └─HashAgg_20 7992.00 root group by:col_1, funcs:firstrow(col_0) └─TableReader_21 7992.00 root data:HashAgg_15 └─HashAgg_15 7992.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) @@ -126,11 +126,11 @@ Projection_12 10000.00 root eq(test.t1.c2, test.t2.c2) ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─Limit_21 1.00 root offset:0, count:1 - └─Projection_41 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_40 1.00 root - ├─Limit_39 1.00 cop offset:0, count:1 - │ └─IndexScan_37 1.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo - └─TableScan_38 1.00 cop table:t2, keep order:false + └─Projection_44 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_43 1.00 root + ├─Limit_42 1.00 cop offset:0, count:1 + │ └─IndexScan_40 1.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo + └─TableScan_41 1.00 cop table:t2, keep order:false explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -140,12 +140,12 @@ Limit_10 1.00 root offset:0, count:1 explain select * from t4 use index(idx) where a > 1 and b > 1 and c > 1 limit 1; id count task operator info Limit_9 1.00 root offset:0, count:1 -└─IndexLookUp_16 1.00 root - ├─Selection_13 3.00 cop gt(test.t4.b, 1) - │ └─IndexScan_11 9.00 cop table:t4, index:a, b, range:(1,+inf], keep order:false, stats:pseudo - └─Limit_15 1.00 cop offset:0, count:1 - └─Selection_14 1.00 cop gt(test.t4.c, 1) - └─TableScan_12 3.00 cop table:t4, keep order:false +└─IndexLookUp_17 1.00 root + ├─Selection_14 3.00 cop gt(test.t4.b, 1) + │ └─IndexScan_12 9.00 cop table:t4, index:a, b, range:(1,+inf], keep order:false, stats:pseudo + └─Limit_16 1.00 cop offset:0, count:1 + └─Selection_15 1.00 cop gt(test.t4.c, 1) + └─TableScan_13 3.00 cop table:t4, keep order:false explain select * from t4 where a > 1 and c > 1 limit 1; id count task operator info Limit_8 1.00 root offset:0, count:1 @@ -291,9 +291,9 @@ Projection_11 10000.00 root 9_aux_0 └─IndexJoin_44 12.50 root inner join, inner:TableReader_43, outer key:s.a, inner key:t1.a ├─TableReader_37 1.00 root data:TableScan_36 │ └─TableScan_36 1.00 cop table:s, range: decided by [eq(s.a, test.t.a)], keep order:false, stats:pseudo - └─TableReader_43 0.80 root data:Selection_42 - └─Selection_42 0.80 cop eq(t1.a, test.t.a) - └─TableScan_41 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─TableReader_43 10.00 root data:Selection_42 + └─Selection_42 10.00 cop eq(t1.a, test.t.a) + └─TableScan_41 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.a = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -304,8 +304,8 @@ Projection_11 10000.00 root 9_aux_0 └─IndexJoin_32 12.50 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a ├─IndexReader_27 10.00 root index:IndexScan_26 │ └─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo - └─TableReader_31 1.00 root data:TableScan_30 - └─TableScan_30 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─TableReader_31 10.00 root data:TableScan_30 + └─TableScan_30 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -318,8 +318,8 @@ Projection_11 10000.00 root 9_aux_0 │ ├─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo │ └─Selection_28 9.99 cop not(isnull(s.c)) │ └─TableScan_27 10.00 cop table:t, keep order:false, stats:pseudo - └─TableReader_33 1.00 root data:TableScan_32 - └─TableScan_32 1.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo + └─TableReader_33 10.00 root data:TableScan_32 + └─TableScan_32 10.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo insert into t values(1, 1, 1), (2, 2 ,2), (3, 3, 3), (4, 3, 4),(5,3,5); analyze table t; explain select t.c in (select count(*) from t s, t t1 where s.b = t.a and s.b = 3 and s.a = t1.a) from t; @@ -329,12 +329,11 @@ Projection_11 5.00 root 9_aux_0 ├─TableReader_15 5.00 root data:TableScan_14 │ └─TableScan_14 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_48 2.40 root inner join, inner:TableReader_47, outer key:s.a, inner key:t1.a - ├─IndexReader_40 2.40 root index:Selection_39 - │ └─Selection_39 2.40 cop eq(3, test.t.a) - │ └─IndexScan_38 3.00 cop table:s, index:b, range:[3,3], keep order:false - └─TableReader_47 0.80 root data:Selection_46 - └─Selection_46 0.80 cop eq(3, test.t.a) + └─IndexJoin_48 3.00 root inner join, inner:TableReader_47, outer key:s.a, inner key:t1.a + ├─IndexReader_40 3.00 root index:IndexScan_39 + │ └─IndexScan_39 3.00 cop table:s, index:b, range:[3,3], keep order:false + └─TableReader_47 4.00 root data:Selection_46 + └─Selection_46 4.00 cop eq(3, test.t.a) └─TableScan_45 1.00 cop table:t1, range: decided by [s.a], keep order:false explain select t.c in (select count(*) from t s left join t t1 on s.a = t1.a where 3 = t.a and s.b = 3) from t; id count task operator info @@ -343,13 +342,12 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─IndexJoin_42 2.40 root left outer join, inner:TableReader_41, outer key:s.a, inner key:t1.a - ├─IndexReader_34 2.40 root index:Selection_33 - │ └─Selection_33 2.40 cop eq(3, test.t.a) - │ └─IndexScan_32 3.00 cop table:s, index:b, range:[3,3], keep order:false - └─TableReader_41 0.80 root data:Selection_40 - └─Selection_40 0.80 cop eq(3, test.t.a) - └─TableScan_39 1.00 cop table:t1, range: decided by [s.a], keep order:false + └─Selection_36 2.40 root eq(3, test.t.a) + └─IndexJoin_40 3.00 root left outer join, inner:TableReader_39, outer key:s.a, inner key:t1.a + ├─IndexReader_33 3.00 root index:IndexScan_32 + │ └─IndexScan_32 3.00 cop table:s, index:b, range:[3,3], keep order:false + └─TableReader_39 1.00 root data:TableScan_38 + └─TableScan_38 1.00 cop table:t1, range: decided by [s.a], keep order:false explain select t.c in (select count(*) from t s right join t t1 on s.a = t1.a where 3 = t.a and t1.b = 3) from t; id count task operator info Projection_10 5.00 root 9_aux_0 @@ -357,13 +355,12 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─IndexJoin_42 2.40 root right outer join, inner:TableReader_41, outer key:t1.a, inner key:s.a - ├─TableReader_41 0.80 root data:Selection_40 - │ └─Selection_40 0.80 cop eq(3, test.t.a) - │ └─TableScan_39 1.00 cop table:s, range: decided by [t1.a], keep order:false - └─IndexReader_34 2.40 root index:Selection_33 - └─Selection_33 2.40 cop eq(3, test.t.a) - └─IndexScan_32 3.00 cop table:t1, index:b, range:[3,3], keep order:false + └─IndexJoin_39 2.40 root right outer join, inner:TableReader_38, outer key:t1.a, inner key:s.a + ├─TableReader_38 1.00 root data:TableScan_37 + │ └─TableScan_37 1.00 cop table:s, range: decided by [t1.a], keep order:false + └─IndexReader_33 2.40 root index:Selection_32 + └─Selection_32 2.40 cop eq(3, test.t.a) + └─IndexScan_31 3.00 cop table:t1, index:b, range:[3,3], keep order:false drop table if exists t; create table t(a int unsigned); explain select t.a = '123455' from t; diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 3a1ec46fb8eb6..6ec2cbcb6a57c 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -27,9 +27,9 @@ TableReader_5 1999.00 root data:TableScan_4 └─TableScan_4 1999.00 cop table:t1, range:[-inf,+inf], keep order:false explain select * from t1 order by c2; id count task operator info -IndexLookUp_12 1999.00 root -├─IndexScan_10 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true -└─TableScan_11 1999.00 cop table:t1, keep order:false +IndexLookUp_13 1999.00 root +├─IndexScan_11 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true +└─TableScan_12 1999.00 cop table:t1, keep order:false explain select * from t2 order by c2; id count task operator info Sort_4 1985.00 root test.t2.c2:asc @@ -46,14 +46,14 @@ IndexReader_6 0.00 root index:IndexScan_5 explain select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id count task operator info MergeJoin_7 2481.25 root left outer join, left key:test.t1.c2, right key:test.t2.c1 -├─IndexLookUp_17 1998.00 root -│ ├─Selection_16 1998.00 cop gt(test.t1.c1, 1) -│ │ └─IndexScan_14 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true -│ └─TableScan_15 1998.00 cop table:t1, keep order:false -└─Projection_22 1985.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_21 1985.00 root - ├─IndexScan_19 1985.00 cop table:t2, index:c1, range:[-inf,+inf], keep order:true - └─TableScan_20 1985.00 cop table:t2, keep order:false +├─IndexLookUp_18 1998.00 root +│ ├─Selection_17 1998.00 cop gt(test.t1.c1, 1) +│ │ └─IndexScan_15 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true +│ └─TableScan_16 1998.00 cop table:t1, keep order:false +└─Projection_24 1985.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_23 1985.00 root + ├─IndexScan_21 1985.00 cop table:t2, index:c1, range:[-inf,+inf], keep order:true + └─TableScan_22 1985.00 cop table:t2, keep order:false explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info Point_Get_1 1.00 root table:t1, handle:1 @@ -112,11 +112,11 @@ Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2) ├─TableReader_16 1999.00 root data:TableScan_15 │ └─TableScan_15 1999.00 cop table:t1, range:[-inf,+inf], keep order:false └─Limit_21 1.00 root offset:0, count:1 - └─Projection_41 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_40 1.00 root - ├─Limit_39 1.00 cop offset:0, count:1 - │ └─IndexScan_37 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true - └─TableScan_38 1.00 cop table:t2, keep order:false + └─Projection_44 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_43 1.00 root + ├─Limit_42 1.00 cop offset:0, count:1 + │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true + └─TableScan_41 1.00 cop table:t2, keep order:false explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -170,17 +170,17 @@ TableDual_5 0.00 root rows:0 explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1, 1; id count task operator info Limit_9 1.00 root offset:1, count:1 -└─IndexLookUp_14 1.00 root - ├─Limit_13 1.00 cop offset:0, count:2 - │ └─IndexScan_11 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_12 1.00 cop table:index_prune, keep order:false +└─IndexLookUp_15 1.00 root + ├─Limit_14 1.00 cop offset:0, count:2 + │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false + └─TableScan_13 1.00 cop table:index_prune, keep order:false explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1, 0; id count task operator info Limit_9 0.00 root offset:1, count:0 -└─IndexLookUp_14 0.00 root - ├─Limit_13 0.00 cop offset:0, count:1 - │ └─IndexScan_11 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_12 0.00 cop table:index_prune, keep order:false +└─IndexLookUp_15 0.00 root + ├─Limit_14 0.00 cop offset:0, count:1 + │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false + └─TableScan_13 0.00 cop table:index_prune, keep order:false explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 0, 1; id count task operator info Point_Get_1 1.00 root table:index_prune, index:a b diff --git a/cmd/explaintest/r/index_join.result b/cmd/explaintest/r/index_join.result index 9e0619b23d460..4f81db898cf7f 100644 --- a/cmd/explaintest/r/index_join.result +++ b/cmd/explaintest/r/index_join.result @@ -9,7 +9,7 @@ id count task operator info IndexJoin_16 5.00 root inner join, inner:IndexLookUp_15, outer key:test.t2.a, inner key:test.t1.a ├─IndexLookUp_15 0.00 root │ ├─Selection_14 0.00 cop not(isnull(test.t1.a)) -│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false +│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false │ └─TableScan_13 0.00 cop table:t1, keep order:false └─TableReader_19 1.00 root data:Selection_18 └─Selection_18 1.00 cop not(isnull(test.t2.a)) @@ -18,10 +18,10 @@ explain select * from t1 join t2 on t1.a=t2.a; id count task operator info Projection_6 5.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b └─IndexJoin_12 5.00 root inner join, inner:IndexLookUp_11, outer key:test.t2.a, inner key:test.t1.a - ├─TableReader_30 1.00 root data:Selection_29 - │ └─Selection_29 1.00 cop not(isnull(test.t2.a)) - │ └─TableScan_28 1.00 cop table:t2, range:[-inf,+inf], keep order:false + ├─TableReader_32 1.00 root data:Selection_31 + │ └─Selection_31 1.00 cop not(isnull(test.t2.a)) + │ └─TableScan_30 1.00 cop table:t2, range:[-inf,+inf], keep order:false └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t1.a)) - │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false + │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false └─TableScan_9 0.00 cop table:t1, keep order:false diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index bd2f4831dc171..a92ad36b9b19c 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -169,21 +169,21 @@ LIMIT 0, 5; id count task operator info Projection_13 0.00 root te.expect_time └─Limit_19 0.00 root offset:0, count:5 - └─IndexJoin_92 0.00 root left outer join, inner:IndexReader_91, outer key:tr.id, inner key:p.relate_id - ├─TopN_95 0.00 root te.expect_time:asc, offset:0, count:5 + └─IndexJoin_104 0.00 root left outer join, inner:IndexReader_103, outer key:tr.id, inner key:p.relate_id + ├─TopN_107 0.00 root te.expect_time:asc, offset:0, count:5 │ └─IndexJoin_36 0.00 root inner join, inner:IndexLookUp_35, outer key:tr.id, inner key:te.trade_id - │ ├─IndexLookUp_74 0.00 root - │ │ ├─Selection_72 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) - │ │ │ └─IndexScan_70 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo - │ │ └─Selection_73 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) - │ │ └─TableScan_71 0.00 cop table:tr, keep order:false + │ ├─IndexLookUp_85 0.00 root + │ │ ├─Selection_83 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) + │ │ │ └─IndexScan_81 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo + │ │ └─Selection_84 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) + │ │ └─TableScan_82 0.00 cop table:tr, keep order:false │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, tr.id)], keep order:false, stats:pseudo + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo │ └─Selection_34 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_33 10.00 cop table:te, keep order:false, stats:pseudo - └─IndexReader_91 0.00 root index:Selection_90 - └─Selection_90 0.00 cop not(isnull(p.relate_id)) - └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, tr.id)], keep order:false, stats:pseudo + └─IndexReader_103 0.00 root index:Selection_102 + └─Selection_102 0.00 cop not(isnull(p.relate_id)) + └─IndexScan_101 10.00 cop table:p, index:relate_id, range: decided by [tr.id], keep order:false, stats:pseudo desc select 1 as a from dual order by a limit 1; id count task operator info Projection_6 1.00 root 1 @@ -226,7 +226,7 @@ Limit_11 5.00 root offset:0, count:5 ├─TableReader_17 4.00 root data:TableScan_16 │ └─TableScan_16 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_14 10.00 root index:IndexScan_13 - └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo explain select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 @@ -235,7 +235,7 @@ Limit_12 5.00 root offset:0, count:5 ├─TableReader_19 4.00 root data:TableScan_18 │ └─TableScan_18 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_16 10.00 root index:IndexScan_15 - └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index a083977411ea6..21a9c49500f38 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -251,17 +251,17 @@ id count task operator info Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority └─TopN_17 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 └─HashAgg_23 40227041.09 root group by:col_4, col_5, col_6, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3) - └─Projection_59 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority + └─Projection_60 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority └─IndexJoin_29 91515927.49 root inner join, inner:IndexLookUp_28, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_49 22592975.51 root inner join, inner:TableReader_55, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─TableReader_55 1498236.00 root data:Selection_54 - │ │ └─Selection_54 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") - │ │ └─TableScan_53 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─TableReader_52 36870000.00 root data:Selection_51 - │ └─Selection_51 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) - │ └─TableScan_50 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + ├─HashRightJoin_50 22592975.51 root inner join, inner:TableReader_56, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─TableReader_56 1498236.00 root data:Selection_55 + │ │ └─Selection_55 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") + │ │ └─TableScan_54 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─TableReader_53 36870000.00 root data:Selection_52 + │ └─Selection_52 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) + │ └─TableScan_51 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false + ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_26 1.00 cop table:lineitem, keep order:false /* @@ -298,11 +298,11 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc └─Projection_12 1.00 root tpch.orders.o_orderpriority, 7_col_0 └─HashAgg_15 1.00 root group by:tpch.orders.o_orderpriority, funcs:count(1), firstrow(tpch.orders.o_orderpriority) └─IndexJoin_21 2340750.00 root semi join, inner:IndexLookUp_20, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─TableReader_33 2925937.50 root data:Selection_32 - │ └─Selection_32 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) - │ └─TableScan_31 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + ├─TableReader_34 2925937.50 root data:Selection_33 + │ └─Selection_33 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) + │ └─TableScan_32 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, keep order:false /* @@ -360,8 +360,8 @@ Sort_23 5.00 root revenue:desc │ │ │ └─TableScan_56 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false │ │ └─TableReader_59 300005811.00 root data:TableScan_58 │ │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ └─TableReader_37 0.80 root data:Selection_36 - │ └─Selection_36 0.80 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) + │ └─TableReader_37 11822812.50 root data:Selection_36 + │ └─Selection_36 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) │ └─TableScan_35 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─TableReader_30 1.00 root data:TableScan_29 └─TableScan_29 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false @@ -538,7 +538,7 @@ Sort_29 718.01 root all_nations.o_year:asc │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false │ │ └─TableReader_83 61674.00 root data:Selection_82 │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") @@ -612,7 +612,7 @@ Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc │ └─TableReader_40 1.00 root data:TableScan_39 │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)], keep order:false + ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false └─TableScan_33 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query @@ -673,7 +673,7 @@ Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, keep order:false /* @@ -769,11 +769,11 @@ id count task operator info Sort_9 1.00 root tpch.lineitem.l_shipmode:asc └─Projection_11 1.00 root tpch.lineitem.l_shipmode, 5_col_0, 5_col_1 └─HashAgg_14 1.00 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) - └─Projection_39 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), cast(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode + └─Projection_40 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), cast(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode └─IndexJoin_18 10023369.01 root inner join, inner:TableReader_17, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - ├─TableReader_35 10023369.01 root data:Selection_34 - │ └─Selection_34 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) - │ └─TableScan_33 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + ├─TableReader_36 10023369.01 root data:Selection_35 + │ └─Selection_35 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) + │ └─TableScan_34 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─TableReader_17 1.00 root data:TableScan_16 └─TableScan_16 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false /* @@ -936,7 +936,7 @@ Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_ty │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_26 1.00 root index:IndexScan_25 - │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false + │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -1042,7 +1042,7 @@ Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.or │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─IndexLookUp_37 1.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1096,8 +1096,8 @@ StreamAgg_13 1.00 root funcs:sum(col_0) ├─TableReader_22 6286493.79 root data:Selection_21 │ └─Selection_21 6286493.79 cop eq(tpch.lineitem.l_shipinstruct, "DELIVER IN PERSON"), in(tpch.lineitem.l_shipmode, "AIR", "AIR REG"), or(and(ge(tpch.lineitem.l_quantity, 4), le(tpch.lineitem.l_quantity, 14)), or(and(ge(tpch.lineitem.l_quantity, 18), le(tpch.lineitem.l_quantity, 28)), and(ge(tpch.lineitem.l_quantity, 29), le(tpch.lineitem.l_quantity, 39)))) │ └─TableScan_20 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─TableReader_28 0.80 root data:Selection_27 - └─Selection_27 0.80 cop ge(tpch.part.p_size, 1), or(and(eq(tpch.part.p_brand, "Brand#52"), and(in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG"), le(tpch.part.p_size, 5))), or(and(eq(tpch.part.p_brand, "Brand#11"), and(in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK"), le(tpch.part.p_size, 10))), and(eq(tpch.part.p_brand, "Brand#51"), and(in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG"), le(tpch.part.p_size, 15))))) + └─TableReader_28 8000000.00 root data:Selection_27 + └─Selection_27 8000000.00 cop ge(tpch.part.p_size, 1), or(and(eq(tpch.part.p_brand, "Brand#52"), and(in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG"), le(tpch.part.p_size, 5))), or(and(eq(tpch.part.p_brand, "Brand#11"), and(in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK"), le(tpch.part.p_size, 10))), and(eq(tpch.part.p_brand, "Brand#51"), and(in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG"), le(tpch.part.p_size, 15))))) └─TableScan_26 1.00 cop table:part, range: decided by [tpch.lineitem.l_partkey], keep order:false /* Q20 Potential Part Promotion Query @@ -1159,17 +1159,17 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc └─Projection_47 64006.34 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) └─Selection_48 64006.34 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) └─HashAgg_51 80007.93 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) - └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_80, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] + └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_81, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] ├─IndexJoin_59 321865.05 root inner join, inner:IndexLookUp_58, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_75 80007.93 root data:Selection_74 - │ │ └─Selection_74 80007.93 cop like(tpch.part.p_name, "green%", 92) - │ │ └─TableScan_73 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ ├─TableReader_76 80007.93 root data:Selection_75 + │ │ └─Selection_75 80007.93 cop like(tpch.part.p_name, "green%", 92) + │ │ └─TableScan_74 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false │ └─TableScan_57 1.00 cop table:partsupp, keep order:false - └─TableReader_80 44189356.65 root data:Selection_79 - └─Selection_79 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) - └─TableScan_78 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─TableReader_81 44189356.65 root data:Selection_80 + └─Selection_80 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) + └─TableScan_79 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q21 Suppliers Who Kept Orders Waiting Query This query identifies certain suppliers who were not able to ship required parts in a timely manner. @@ -1235,14 +1235,14 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ │ └─TableReader_80 240004648.80 root data:Selection_79 │ │ │ └─Selection_79 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) │ │ │ └─TableScan_78 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false - │ │ └─TableReader_61 0.80 root data:Selection_60 - │ │ └─Selection_60 0.80 cop eq(tpch.orders.o_orderstatus, "F") + │ │ └─TableReader_61 36517371.00 root data:Selection_60 + │ │ └─Selection_60 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l2.l_orderkey, l1.l_orderkey)], keep order:false + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false │ └─TableScan_54 1.00 cop table:lineitem, keep order:false └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l3.l_orderkey, l1.l_orderkey)], keep order:false + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) └─TableScan_37 1.00 cop table:lineitem, keep order:false /* diff --git a/cmd/explaintest/r/window_function.result b/cmd/explaintest/r/window_function.result index 3f4e6132897fe..7fc186eb98260 100644 --- a/cmd/explaintest/r/window_function.result +++ b/cmd/explaintest/r/window_function.result @@ -18,34 +18,34 @@ explain select sum(a) over(partition by a order by b) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc range between unbounded preceding and current row) - └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_10 10000.00 root data:TableScan_9 - └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_11 10000.00 root data:TableScan_10 + └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b rows unbounded preceding) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b rows unbounded preceding) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc rows between unbounded preceding and current row) - └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_10 10000.00 root data:TableScan_9 - └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_11 10000.00 root data:TableScan_10 + └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b rows between 1 preceding and 1 following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b rows between 1 preceding and 1 following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc rows between 1 preceding and 1 following) - └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_10 10000.00 root data:TableScan_9 - └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_11 10000.00 root data:TableScan_10 + └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b range between 1 preceding and 1 following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b range between 1 preceding and 1 following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc range between 1 preceding and 1 following) - └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_10 10000.00 root data:TableScan_9 - └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_11 10000.00 root data:TableScan_10 + └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by c range between interval '2:30' minute_second preceding and interval '2:30' minute_second following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by c range between interval '2:30' minute_second preceding and interval '2:30' minute_second following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.c asc range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) - └─Sort_11 10000.00 root test.t.a:asc, test.t.c:asc - └─TableReader_10 10000.00 root data:TableScan_9 - └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_12 10000.00 root test.t.a:asc, test.t.c:asc + └─TableReader_11 10000.00 root data:TableScan_10 + └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo diff --git a/cmd/importer/stats.go b/cmd/importer/stats.go index 1b67d94b9be16..50b332f821a80 100644 --- a/cmd/importer/stats.go +++ b/cmd/importer/stats.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/model" stats "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" @@ -34,12 +35,12 @@ func loadStats(tblInfo *model.TableInfo, path string) (*stats.Table, error) { if err != nil { return nil, errors.Trace(err) } - jsTable := &stats.JSONTable{} + jsTable := &handle.JSONTable{} err = json.Unmarshal(data, jsTable) if err != nil { return nil, errors.Trace(err) } - return stats.TableStatsFromJSON(tblInfo, tblInfo.ID, jsTable) + return handle.TableStatsFromJSON(tblInfo, tblInfo.ID, jsTable) } type histogram struct { diff --git a/cmd/pluginpkg/pluginpkg.go b/cmd/pluginpkg/pluginpkg.go index 335a8f5b93a49..f8577d523c4d1 100644 --- a/cmd/pluginpkg/pluginpkg.go +++ b/cmd/pluginpkg/pluginpkg.go @@ -89,7 +89,7 @@ func init() { } func usage() { - log.Printf("Usage: %s --pkg-dir [plugin source pkg folder] --outDir-dir [outDir-dir]\n", path.Base(os.Args[0])) + log.Printf("Usage: %s --pkg-dir [plugin source pkg folder] --out-dir [plugin packaged folder path]\n", path.Base(os.Args[0])) flag.PrintDefaults() os.Exit(1) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d01242d69fb55..a35f606a1c0eb 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + field_types "github.com/pingcap/parser/types" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta/autoid" @@ -440,6 +441,10 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o col.GeneratedStored = v.Stored _, dependColNames := findDependedColumnNames(colDef) col.Dependences = dependColNames + case ast.ColumnOptionCollate: + if field_types.HasCharset(colDef.Tp) { + col.FieldType.Collate = v.StrValue + } case ast.ColumnOptionFulltext: ctx.GetSessionVars().StmtCtx.AppendWarning(ErrTableCantHandleFt) } diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 85adb4b4e4fbd..824ddf4cb76dc 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -179,7 +179,7 @@ func (builder *RequestBuilder) SetConcurrency(concurrency int) *RequestBuilder { // TableRangesToKVRanges converts table ranges to "KeyRange". func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange { - if fb == nil || fb.Hist() == nil { + if fb == nil || fb.Hist == nil { return tableRangesToKVRangesWithoutSplit(tid, ranges) } krs := make([]kv.KeyRange, 0, len(ranges)) @@ -256,7 +256,7 @@ func TableHandlesToKVRanges(tid int64, handles []int64) []kv.KeyRange { // IndexRangesToKVRanges converts index ranges to "KeyRange". func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { - if fb == nil || fb.Hist() == nil { + if fb == nil || fb.Hist == nil { return indexRangesToKVWithoutSplit(sc, tid, idxID, ranges) } feedbackRanges := make([]*ranger.Range, 0, len(ranges)) @@ -268,7 +268,7 @@ func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, range feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true}) } - feedbackRanges = fb.Hist().SplitRange(sc, feedbackRanges, true) + feedbackRanges = fb.Hist.SplitRange(sc, feedbackRanges, true) krs := make([]kv.KeyRange, 0, len(feedbackRanges)) for _, ran := range feedbackRanges { low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() diff --git a/domain/domain.go b/domain/domain.go index 7255f6c5ca3b3..c7f18052ac240 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -15,7 +15,6 @@ package domain import ( "context" - "crypto/tls" "os" "sync" "sync/atomic" @@ -23,7 +22,7 @@ import ( "unsafe" "github.com/coreos/etcd/clientv3" - "github.com/grpc-ecosystem/go-grpc-prometheus" + grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/ngaut/pools" "github.com/ngaut/sync2" "github.com/pingcap/errors" @@ -43,7 +42,8 @@ import ( "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -58,7 +58,7 @@ type Domain struct { store kv.Storage infoHandle *infoschema.Handle privHandle *privileges.Handle - bindHandle *bindinfo.Handle + bindHandle *bindinfo.BindHandle statsHandle unsafe.Pointer statsLease time.Duration statsUpdating sync2.AtomicInt32 @@ -563,13 +563,6 @@ func (c *ddlCallback) OnChanged(err error) error { return nil } -// EtcdBackend is used for judging a storage is a real TiKV. -type EtcdBackend interface { - EtcdAddrs() []string - TLSConfig() *tls.Config - StartGCWorker() error -} - const resourceIdleTimeout = 3 * time.Minute // resources in the ResourcePool will be recycled after idleTimeout // NewDomain creates a new domain. Should not create multiple domains for the same store. @@ -589,7 +582,7 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio // Init initializes a domain. func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.Resource, error)) error { perfschema.Init() - if ebd, ok := do.store.(EtcdBackend); ok { + if ebd, ok := do.store.(tikv.EtcdBackend); ok { if addrs := ebd.EtcdAddrs(); addrs != nil { cfg := config.GetGlobalConfig() cli, err := clientv3.New(clientv3.Config{ @@ -785,7 +778,7 @@ func (do *Domain) PrivilegeHandle() *privileges.Handle { } // BindHandle returns domain's bindHandle. -func (do *Domain) BindHandle() *bindinfo.Handle { +func (do *Domain) BindHandle() *bindinfo.BindHandle { return do.bindHandle } @@ -793,10 +786,8 @@ func (do *Domain) BindHandle() *bindinfo.Handle { // be called only once in BootstrapSession. func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser) error { ctx.GetSessionVars().InRestrictedSQL = true - do.bindHandle = bindinfo.NewHandle() - - bindCacheUpdater := bindinfo.NewBindCacheUpdater(ctx, do.BindHandle(), parser) - err := bindCacheUpdater.Update(true) + do.bindHandle = bindinfo.NewBindHandle(ctx, parser) + err := do.bindHandle.Update(true) if err != nil { return err } @@ -812,7 +803,7 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser return case <-time.After(duration): } - err = bindCacheUpdater.Update(false) + err = do.bindHandle.Update(false) if err != nil { logutil.Logger(context.Background()).Error("update bindinfo failed", zap.Error(err)) } @@ -822,13 +813,13 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser } // StatsHandle returns the statistic handle. -func (do *Domain) StatsHandle() *statistics.Handle { - return (*statistics.Handle)(atomic.LoadPointer(&do.statsHandle)) +func (do *Domain) StatsHandle() *handle.Handle { + return (*handle.Handle)(atomic.LoadPointer(&do.statsHandle)) } // CreateStatsHandle is used only for test. func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) { - atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statistics.NewHandle(ctx, do.statsLease))) + atomic.StorePointer(&do.statsHandle, unsafe.Pointer(handle.NewHandle(ctx, do.statsLease))) } // StatsUpdating checks if the stats worker is updating. @@ -853,7 +844,7 @@ var RunAutoAnalyze = true // It should be called only once in BootstrapSession. func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { ctx.GetSessionVars().InRestrictedSQL = true - statsHandle := statistics.NewHandle(ctx, do.statsLease) + statsHandle := handle.NewHandle(ctx, do.statsLease) atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statsHandle)) do.ddl.RegisterEventCh(statsHandle.DDLEventCh()) if do.statsLease <= 0 { @@ -877,7 +868,7 @@ func (do *Domain) newStatsOwner() owner.Manager { if do.etcdClient == nil { statsOwner = owner.NewMockManager(id, cancelFunc) } else { - statsOwner = owner.NewOwnerManager(do.etcdClient, statistics.StatsPrompt, id, statistics.StatsOwnerKey, cancelFunc) + statsOwner = owner.NewOwnerManager(do.etcdClient, handle.StatsPrompt, id, handle.StatsOwnerKey, cancelFunc) } // TODO: Need to do something when err is not nil. err := statsOwner.CampaignOwner(cancelCtx) @@ -932,7 +923,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) logutil.Logger(context.Background()).Debug("handle ddl event failed", zap.Error(err)) } case <-deltaUpdateTicker.C: - err = statsHandle.DumpStatsDeltaToKV(statistics.DumpDelta) + err = statsHandle.DumpStatsDeltaToKV(handle.DumpDelta) if err != nil { logutil.Logger(context.Background()).Debug("dump stats delta failed", zap.Error(err)) } diff --git a/executor/adapter.go b/executor/adapter.go index 6ca2e3975f29b..983e071661ab8 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -413,12 +413,15 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { indexIDs = strings.Replace(fmt.Sprintf("%v", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) } execDetail := sessVars.StmtCtx.GetExecDetails() + copTaskInfo := sessVars.StmtCtx.CopTasksDetails() + statsInfos := a.getStatsInfo() + memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() if costTime < threshold { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) + logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql)) } else { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) + logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -444,6 +447,28 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } } +func (a *ExecStmt) getStatsInfo() map[string]uint64 { + var physicalPlan plannercore.PhysicalPlan + switch p := a.Plan.(type) { + case *plannercore.Insert: + physicalPlan = p.SelectPlan + case *plannercore.Update: + physicalPlan = p.SelectPlan + case *plannercore.Delete: + physicalPlan = p.SelectPlan + case plannercore.PhysicalPlan: + physicalPlan = p + } + + if physicalPlan == nil { + return nil + } + + statsInfos := make(map[string]uint64) + statsInfos = plannercore.CollectPlanStatsVersion(physicalPlan, statsInfos) + return statsInfos +} + // IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions: // 1. ctx is auto commit tagged // 2. txn is not valid diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index 20a212179ceca..c61bd792eebf2 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -141,29 +141,34 @@ func buildAggTesterWithFieldType(funcName string, ft *types.FieldType, numRows i dataType: ft, numRows: numRows, funcName: funcName, + dataGen: getDataGenFunc(ft), } for _, result := range results { pt.results = append(pt.results, types.NewDatum(result)) } + return pt +} + +func getDataGenFunc(ft *types.FieldType) func(i int) types.Datum { switch ft.Tp { case mysql.TypeLonglong: - pt.dataGen = func(i int) types.Datum { return types.NewIntDatum(int64(i)) } + return func(i int) types.Datum { return types.NewIntDatum(int64(i)) } case mysql.TypeFloat: - pt.dataGen = func(i int) types.Datum { return types.NewFloat32Datum(float32(i)) } + return func(i int) types.Datum { return types.NewFloat32Datum(float32(i)) } case mysql.TypeNewDecimal: - pt.dataGen = func(i int) types.Datum { return types.NewDecimalDatum(types.NewDecFromInt(int64(i))) } + return func(i int) types.Datum { return types.NewDecimalDatum(types.NewDecFromInt(int64(i))) } case mysql.TypeDouble: - pt.dataGen = func(i int) types.Datum { return types.NewFloat64Datum(float64(i)) } + return func(i int) types.Datum { return types.NewFloat64Datum(float64(i)) } case mysql.TypeString: - pt.dataGen = func(i int) types.Datum { return types.NewStringDatum(fmt.Sprintf("%d", i)) } + return func(i int) types.Datum { return types.NewStringDatum(fmt.Sprintf("%d", i)) } case mysql.TypeDate: - pt.dataGen = func(i int) types.Datum { return types.NewTimeDatum(types.TimeFromDays(int64(i + 365))) } + return func(i int) types.Datum { return types.NewTimeDatum(types.TimeFromDays(int64(i + 365))) } case mysql.TypeDuration: - pt.dataGen = func(i int) types.Datum { return types.NewDurationDatum(types.Duration{Duration: time.Duration(i)}) } + return func(i int) types.Datum { return types.NewDurationDatum(types.Duration{Duration: time.Duration(i)}) } case mysql.TypeJSON: - pt.dataGen = func(i int) types.Datum { return types.NewDatum(json.CreateBinary(int64(i))) } + return func(i int) types.Datum { return types.NewDatum(json.CreateBinary(int64(i))) } } - return pt + return nil } func (s *testSuite) testAggFunc(c *C, p aggTest) { diff --git a/executor/aggfuncs/window_func_test.go b/executor/aggfuncs/window_func_test.go new file mode 100644 index 0000000000000..d6c140d596d40 --- /dev/null +++ b/executor/aggfuncs/window_func_test.go @@ -0,0 +1,132 @@ +// 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 aggfuncs_test + +import ( + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/executor/aggfuncs" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/json" + "github.com/pingcap/tidb/util/chunk" +) + +type windowTest struct { + dataType *types.FieldType + numRows int + funcName string + args []expression.Expression + orderByCols []*expression.Column + results []types.Datum +} + +func (s *testSuite) testWindowFunc(c *C, p windowTest) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, p.numRows) + dataGen := getDataGenFunc(p.dataType) + for i := 0; i < p.numRows; i++ { + dt := dataGen(i) + srcChk.AppendDatum(0, &dt) + } + + desc := aggregation.NewAggFuncDesc(s.ctx, p.funcName, p.args, false) + finalFunc := aggfuncs.BuildWindowFunctions(s.ctx, desc, 0, p.orderByCols) + finalPr := finalFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) + + iter := chunk.NewIterator4Chunk(srcChk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) + } + + for i := 0; i < p.numRows; i++ { + finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + dt := resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[i]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0) + resultChk.Reset() + } + finalFunc.ResetPartialResult(finalPr) +} + +func buildWindowTester(funcName string, tp byte, constantArg uint64, orderByCols int, numRows int, results ...interface{}) windowTest { + pt := windowTest{ + dataType: types.NewFieldType(tp), + numRows: numRows, + funcName: funcName, + } + if funcName != ast.WindowFuncNtile { + pt.args = append(pt.args, &expression.Column{RetType: pt.dataType, Index: 0}) + } + if constantArg > 0 { + pt.args = append(pt.args, &expression.Constant{Value: types.NewUintDatum(constantArg)}) + } + if orderByCols > 0 { + pt.orderByCols = append(pt.orderByCols, &expression.Column{RetType: pt.dataType, Index: 0}) + } + + for _, result := range results { + pt.results = append(pt.results, types.NewDatum(result)) + } + return pt +} + +func (s *testSuite) TestWindowFunctions(c *C) { + tests := []windowTest{ + buildWindowTester(ast.WindowFuncCumeDist, mysql.TypeLonglong, 0, 0, 2, 1, 1), + buildWindowTester(ast.WindowFuncCumeDist, mysql.TypeLonglong, 0, 1, 4, 0.25, 0.5, 0.75, 1), + + buildWindowTester(ast.WindowFuncDenseRank, mysql.TypeLonglong, 0, 0, 2, 1, 1), + buildWindowTester(ast.WindowFuncDenseRank, mysql.TypeLonglong, 0, 1, 4, 1, 2, 3, 4), + + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeLonglong, 0, 1, 2, 0, 0), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeFloat, 0, 1, 2, 0, 0), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeDouble, 0, 1, 2, 0, 0), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeNewDecimal, 0, 1, 2, types.NewDecFromInt(0), types.NewDecFromInt(0)), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeString, 0, 1, 2, "0", "0"), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeDate, 0, 1, 2, types.TimeFromDays(365), types.TimeFromDays(365)), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeDuration, 0, 1, 2, types.Duration{Duration: time.Duration(0)}, types.Duration{Duration: time.Duration(0)}), + buildWindowTester(ast.WindowFuncFirstValue, mysql.TypeJSON, 0, 1, 2, json.CreateBinary(int64(0)), json.CreateBinary(int64(0))), + + buildWindowTester(ast.WindowFuncLag, mysql.TypeLonglong, 1, 0, 3, nil, 0, 1), + buildWindowTester(ast.WindowFuncLag, mysql.TypeLonglong, 2, 1, 4, nil, nil, 0, 1), + + buildWindowTester(ast.WindowFuncLastValue, mysql.TypeLonglong, 1, 0, 2, 1, 1), + + buildWindowTester(ast.WindowFuncLead, mysql.TypeLonglong, 1, 0, 3, 1, 2, nil), + buildWindowTester(ast.WindowFuncLead, mysql.TypeLonglong, 2, 0, 4, 2, 3, nil, nil), + + buildWindowTester(ast.WindowFuncNthValue, mysql.TypeLonglong, 2, 0, 3, 1, 1, 1), + buildWindowTester(ast.WindowFuncNthValue, mysql.TypeLonglong, 5, 0, 3, nil, nil, nil), + + buildWindowTester(ast.WindowFuncNtile, mysql.TypeLonglong, 3, 0, 4, 1, 1, 2, 3), + buildWindowTester(ast.WindowFuncNtile, mysql.TypeLonglong, 5, 0, 3, 1, 2, 3), + + buildWindowTester(ast.WindowFuncPercentRank, mysql.TypeLonglong, 0, 0, 3, 0, 0, 0), + buildWindowTester(ast.WindowFuncPercentRank, mysql.TypeLonglong, 0, 1, 4, 0, 0.3333333333333333, 0.6666666666666666, 1), + + buildWindowTester(ast.WindowFuncRank, mysql.TypeLonglong, 0, 0, 3, 1, 1, 1), + buildWindowTester(ast.WindowFuncRank, mysql.TypeLonglong, 0, 1, 4, 1, 2, 3, 4), + + buildWindowTester(ast.WindowFuncRowNumber, mysql.TypeLonglong, 0, 0, 4, 1, 2, 3, 4), + } + for _, test := range tests { + s.testWindowFunc(c, test) + } +} diff --git a/executor/analyze.go b/executor/analyze.go index 9a4eba1c105c4..7267ef6b5de5e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -332,7 +332,7 @@ func (e *AnalyzeColumnsExec) open() error { ranges = ranger.FullIntRange(false) } e.resultHandler = &tableResultHandler{} - firstPartRanges, secondPartRanges := splitRanges(ranges, true) + firstPartRanges, secondPartRanges := splitRanges(ranges, true, false) firstResult, err := e.buildResp(firstPartRanges) if err != nil { return err diff --git a/executor/bind.go b/executor/bind.go new file mode 100644 index 0000000000000..45262f39de75d --- /dev/null +++ b/executor/bind.go @@ -0,0 +1,84 @@ +// 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 executor + +import ( + "context" + + "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/bindinfo" + "github.com/pingcap/tidb/domain" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/util/chunk" +) + +// SQLBindExec represents a bind executor. +type SQLBindExec struct { + baseExecutor + + sqlBindOp plannercore.SQLBindOpType + normdOrigSQL string + bindSQL string + charset string + collation string + isGlobal bool + bindAst ast.StmtNode +} + +// Next implements the Executor Next interface. +func (e *SQLBindExec) Next(ctx context.Context, req *chunk.RecordBatch) error { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("SQLBindExec.Next", opentracing.ChildOf(span.Context())) + defer span1.Finish() + } + + req.Reset() + switch e.sqlBindOp { + case plannercore.OpSQLBindCreate: + return e.createSQLBind() + case plannercore.OpSQLBindDrop: + return e.dropSQLBind() + default: + return errors.Errorf("unsupported SQL bind operation: %v", e.sqlBindOp) + } +} + +func (e *SQLBindExec) dropSQLBind() error { + if !e.isGlobal { + return errors.New("drop non-global sql bind is not supported") + } + + record := &bindinfo.BindRecord{ + OriginalSQL: e.normdOrigSQL, + Db: e.ctx.GetSessionVars().CurrentDB, + } + return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(record) +} + +func (e *SQLBindExec) createSQLBind() error { + if !e.isGlobal { + return errors.New("create non-global sql bind is not supported") + } + + record := &bindinfo.BindRecord{ + OriginalSQL: e.normdOrigSQL, + BindSQL: e.bindSQL, + Db: e.ctx.GetSessionVars().CurrentDB, + Charset: e.charset, + Collation: e.collation, + } + return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(record) +} diff --git a/executor/builder.go b/executor/builder.go index 0874bf1e8a961..6ae2e361131f0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -177,6 +177,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildIndexLookUpReader(v) case *plannercore.PhysicalWindow: return b.buildWindow(v) + case *plannercore.SQLBindPlan: + return b.buildSQLBindExec(v) default: if mp, ok := p.(MockPhysicalPlan); ok { return mp.GetExecutor() @@ -519,6 +521,7 @@ func (b *executorBuilder) buildShow(v *plannercore.Show) Executor { Table: v.Table, Column: v.Column, User: v.User, + Roles: v.Roles, IfNotExists: v.IfNotExists, Flag: v.Flag, Full: v.Full, @@ -1250,7 +1253,7 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { // cols2Handle represents an mapper from column index to handle index. type cols2Handle struct { - // start/end represent the ordinal range [start, end) of the consecutive columns. + // start and end represent the ordinal range [start, end) of the consecutive columns. start, end int32 // handleOrdinal represents the ordinal of the handle column. handleOrdinal int32 @@ -2030,6 +2033,23 @@ func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) *WindowExec } } +func (b *executorBuilder) buildSQLBindExec(v *plannercore.SQLBindPlan) Executor { + base := newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()) + base.initCap = chunk.ZeroCapacity + + e := &SQLBindExec{ + baseExecutor: base, + sqlBindOp: v.SQLBindOp, + normdOrigSQL: v.NormdOrigSQL, + bindSQL: v.BindSQL, + charset: v.Charset, + collation: v.Collation, + isGlobal: v.IsGlobal, + bindAst: v.BindStmt, + } + return e +} + func getPhysicalTableID(t table.Table) int64 { if p, ok := t.(table.PhysicalTable); ok { return p.GetPhysicalID() diff --git a/executor/compiler.go b/executor/compiler.go index 3f62c5a0ca577..6070626337a0c 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -188,6 +188,22 @@ func getStmtDbLabel(stmtNode ast.StmtNode) map[string]struct{} { dbLabelSet[db] = struct{}{} } } + case *ast.CreateBindingStmt: + if x.OriginSel != nil { + originSelect := x.OriginSel.(*ast.SelectStmt) + dbLabels := getDbFromResultNode(originSelect.From.TableRefs) + for _, db := range dbLabels { + dbLabelSet[db] = struct{}{} + } + } + + if len(dbLabelSet) == 0 && x.HintedSel != nil { + hintedSelect := x.HintedSel.(*ast.SelectStmt) + dbLabels := getDbFromResultNode(hintedSelect.From.TableRefs) + for _, db := range dbLabels { + dbLabelSet[db] = struct{}{} + } + } } return dbLabelSet @@ -296,6 +312,8 @@ func GetStmtLabel(stmtNode ast.StmtNode) string { return "Prepare" case *ast.UseStmt: return "Use" + case *ast.CreateBindingStmt: + return "CreateBinding" } return "other" } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 93a52162291d9..bc43c6bd8dbf4 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -391,26 +392,82 @@ func (s *testSuite3) TestRenameTable(c *C) { tk.MustExec("drop database rename2") } -func (s *testSuite3) TestUnsupportedCharset(c *C) { +func (s *testSuite3) TestColumnCharsetAndCollate(c *C) { tk := testkit.NewTestKit(c, s.store) - dbName := "unsupported_charset" + dbName := "col_charset_collate" tk.MustExec("create database " + dbName) tk.MustExec("use " + dbName) tests := []struct { - charset string - valid bool + colType string + charset string + collates string + exptCharset string + exptCollate string + errMsg string }{ - {"charset UTF8 collate UTF8_bin", true}, - {"charset utf8mb4", true}, - {"charset utf16", false}, - {"charset latin1", true}, - {"charset binary", true}, - {"charset ascii", true}, + { + colType: "varchar(10)", + charset: "charset utf8", + collates: "collate utf8_bin", + exptCharset: "utf8", + exptCollate: "utf8_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset utf8mb4", + collates: "", + exptCharset: "utf8mb4", + exptCollate: "utf8mb4_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset utf16", + collates: "", + exptCharset: "", + exptCollate: "", + errMsg: "Unknown charset utf16", + }, + { + colType: "varchar(10)", + charset: "charset latin1", + collates: "", + exptCharset: "latin1", + exptCollate: "latin1_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset binary", + collates: "", + exptCharset: "binary", + exptCollate: "binary", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset ascii", + collates: "", + exptCharset: "ascii", + exptCollate: "ascii_bin", + errMsg: "", + }, } + sctx := tk.Se.(sessionctx.Context) + dm := domain.GetDomain(sctx) for i, tt := range tests { - sql := fmt.Sprintf("create table t%d (a varchar(10) %s)", i, tt.charset) - if tt.valid { + tblName := fmt.Sprintf("t%d", i) + sql := fmt.Sprintf("create table %s (a %s %s %s)", tblName, tt.colType, tt.charset, tt.collates) + if tt.errMsg == "" { tk.MustExec(sql) + is := dm.InfoSchema() + c.Assert(is, NotNil) + + tb, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + c.Assert(err, IsNil) + c.Assert(tb.Meta().Columns[0].Charset, Equals, tt.exptCharset, Commentf(sql)) + c.Assert(tb.Meta().Columns[0].Collate, Equals, tt.exptCollate, Commentf(sql)) } else { _, err := tk.Exec(sql) c.Assert(err, NotNil, Commentf(sql)) diff --git a/executor/distsql.go b/executor/distsql.go index c6f50ab471764..dc7309428df3b 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -160,7 +160,7 @@ func handleIsExtra(col *expression.Column) bool { return false } -func splitRanges(ranges []*ranger.Range, keepOrder bool) ([]*ranger.Range, []*ranger.Range) { +func splitRanges(ranges []*ranger.Range, keepOrder bool, desc bool) ([]*ranger.Range, []*ranger.Range) { if len(ranges) == 0 || ranges[0].LowVal[0].Kind() == types.KindInt64 { return ranges, nil } @@ -174,27 +174,37 @@ func splitRanges(ranges []*ranger.Range, keepOrder bool) ([]*ranger.Range, []*ra if !keepOrder { return append(unsignedRanges, signedRanges...), nil } + if desc { + return unsignedRanges, signedRanges + } return signedRanges, unsignedRanges } signedRanges := make([]*ranger.Range, 0, idx+1) unsignedRanges := make([]*ranger.Range, 0, len(ranges)-idx) signedRanges = append(signedRanges, ranges[0:idx]...) - signedRanges = append(signedRanges, &ranger.Range{ - LowVal: ranges[idx].LowVal, - LowExclude: ranges[idx].LowExclude, - HighVal: []types.Datum{types.NewUintDatum(math.MaxInt64)}, - }) - unsignedRanges = append(unsignedRanges, &ranger.Range{ - LowVal: []types.Datum{types.NewUintDatum(math.MaxInt64 + 1)}, - HighVal: ranges[idx].HighVal, - HighExclude: ranges[idx].HighExclude, - }) + if !(ranges[idx].LowVal[0].GetUint64() == math.MaxInt64 && ranges[idx].LowExclude) { + signedRanges = append(signedRanges, &ranger.Range{ + LowVal: ranges[idx].LowVal, + LowExclude: ranges[idx].LowExclude, + HighVal: []types.Datum{types.NewUintDatum(math.MaxInt64)}, + }) + } + if !(ranges[idx].HighVal[0].GetUint64() == math.MaxInt64+1 && ranges[idx].HighExclude) { + unsignedRanges = append(unsignedRanges, &ranger.Range{ + LowVal: []types.Datum{types.NewUintDatum(math.MaxInt64 + 1)}, + HighVal: ranges[idx].HighVal, + HighExclude: ranges[idx].HighExclude, + }) + } if idx < len(ranges) { unsignedRanges = append(unsignedRanges, ranges[idx+1:]...) } if !keepOrder { return append(unsignedRanges, signedRanges...), nil } + if desc { + return unsignedRanges, signedRanges + } return signedRanges, unsignedRanges } diff --git a/executor/distsql_test.go b/executor/distsql_test.go index d63624e883343..5ffb18036839b 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -172,3 +172,15 @@ func (s *testSuite3) TestUniqueKeyNullValueSelect(c *C) { res = tk.MustQuery("select * from t where id is null;") res.Check(testkit.Rows(" a", " b", " c")) } + +// TestIssue10178 contains tests for https://github.com/pingcap/tidb/issues/10178 . +func (s *testSuite3) TestIssue10178(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint unsigned primary key)") + tk.MustExec("insert into t values(9223372036854775807), (18446744073709551615)") + tk.MustQuery("select max(a) from t").Check(testkit.Rows("18446744073709551615")) + tk.MustQuery("select * from t where a > 9223372036854775807").Check(testkit.Rows("18446744073709551615")) + tk.MustQuery("select * from t where a < 9223372036854775808").Check(testkit.Rows("9223372036854775807")) +} diff --git a/executor/executor.go b/executor/executor.go index 60d8b40666edb..619d16a370b83 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1333,8 +1333,11 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.Priority = stmt.Priority case *ast.InsertStmt: sc.InInsertStmt = true + // For insert statement (not for update statement), disabling the StrictSQLMode + // should make TruncateAsWarning and DividedByZeroAsWarning, + // but should not make DupKeyAsWarning or BadNullAsWarning, sc.DupKeyAsWarning = stmt.IgnoreErr - sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.BadNullAsWarning = stmt.IgnoreErr sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() diff --git a/executor/executor_test.go b/executor/executor_test.go index f8b80e7a36818..60707de717b93 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1767,8 +1767,15 @@ func (s *testSuite) TestSQLMode(c *C) { tk.MustExec("set sql_mode = ''") tk.MustExec("insert t values ()") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1364 Field 'a' doesn't have a default value")) + _, err = tk.Exec("insert t values (null)") + c.Check(err, NotNil) + tk.MustExec("insert ignore t values (null)") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'a' cannot be null")) + tk.MustExec("insert t select null") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'a' cannot be null")) tk.MustExec("insert t values (1000)") - tk.MustQuery("select * from t").Check(testkit.Rows("0", "127")) + tk.MustQuery("select * from t order by a").Check(testkit.Rows("0", "0", "0", "127")) tk.MustExec("insert tdouble values (10.23)") tk.MustQuery("select * from tdouble").Check(testkit.Rows("9.99")) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index caa28fc21be4b..0df5fa6055edd 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -71,7 +71,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { " └─IndexLookUp_11 0.00 root ", " ├─Selection_10 0.00 cop not(isnull(test.t2.a))", " │ └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", - " └─TableScan_9 0.00 cop table:t2, keep order:false", + " └─TableScan_9 0.00 cop table:t2, keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( "2 2 2 2 2", diff --git a/executor/insert_common.go b/executor/insert_common.go index 6df2e1ab9e0bc..265215550f63b 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -301,6 +301,10 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, exec func(ctx c rows := make([][]types.Datum, 0, chk.Capacity()) sessVars := e.ctx.GetSessionVars() + if !sessVars.StrictSQLMode { + // If StrictSQLMode is disabled and it is a insert-select statement, it also handle BadNullAsWarning. + sessVars.StmtCtx.BadNullAsWarning = true + } batchInsert := sessVars.BatchInsert && !sessVars.InTxn() batchSize := sessVars.DMLBatchSize diff --git a/executor/load_stats.go b/executor/load_stats.go index ceacfbea6bda5..e55ada1e84b60 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" ) @@ -76,7 +76,7 @@ func (e *LoadStatsExec) Open(ctx context.Context) error { // Update updates the stats of the corresponding table according to the data. func (e *LoadStatsInfo) Update(data []byte) error { - jsonTbl := &statistics.JSONTable{} + jsonTbl := &handle.JSONTable{} if err := json.Unmarshal(data, jsonTbl); err != nil { return errors.Trace(err) } diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 5ae4f7479996d..2dd10fbfdc444 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -43,7 +43,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" @@ -599,17 +599,17 @@ func (s *seqTestSuite) TestShowStatsHealthy(c *C) { tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100")) tk.MustExec("insert into t values (1), (2)") do, _ := session.GetDomain(s.store) - do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll) + do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll) tk.MustExec("analyze table t") tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100")) tk.MustExec("insert into t values (3), (4), (5), (6), (7), (8), (9), (10)") - do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll) + do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll) do.StatsHandle().Update(do.InfoSchema()) tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 19")) tk.MustExec("analyze table t") tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100")) tk.MustExec("delete from t") - do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll) + do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll) do.StatsHandle().Update(do.InfoSchema()) tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 0")) } diff --git a/executor/show.go b/executor/show.go index c4322a876eb1e..ed0403c214710 100644 --- a/executor/show.go +++ b/executor/show.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb-tools/tidb-binlog/node" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/plugin" @@ -60,8 +61,9 @@ type ShowExec struct { Column *ast.ColumnName // Used for `desc table column`. Flag int // Some flag parsed from sql, such as FULL. Full bool - User *auth.UserIdentity // Used for show grants. - IfNotExists bool // Used for `show create database if not exists` + User *auth.UserIdentity // Used for show grants. + Roles []*auth.RoleIdentity // Used for show grants. + IfNotExists bool // Used for `show create database if not exists` // GlobalScope is used by show variables GlobalScope bool @@ -137,6 +139,8 @@ func (e *ShowExec) fetchAll() error { return e.fetchShowStatus() case ast.ShowTables: return e.fetchShowTables() + case ast.ShowOpenTables: + return e.fetchShowOpenTables() case ast.ShowTableStatus: return e.fetchShowTableStatus() case ast.ShowTriggers: @@ -168,6 +172,28 @@ func (e *ShowExec) fetchAll() error { return e.fetchShowMasterStatus() case ast.ShowPrivileges: return e.fetchShowPrivileges() + case ast.ShowBindings: + return e.fetchShowBind() + } + return nil +} + +func (e *ShowExec) fetchShowBind() error { + if !e.GlobalScope { + return errors.New("show non-global bind sql is not supported") + } + bindRecords := domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() + for _, bindData := range bindRecords { + e.appendRow([]interface{}{ + bindData.OriginalSQL, + bindData.BindSQL, + bindData.Db, + bindData.Status, + bindData.CreateTime, + bindData.UpdateTime, + bindData.Charset, + bindData.Collation, + }) } return nil } @@ -242,6 +268,12 @@ func (e *ShowExec) fetchShowProcessList() error { return nil } +func (e *ShowExec) fetchShowOpenTables() error { + // TiDB has no concept like mysql's "table cache" and "open table" + // For simplicity, we just return an empty result with the same structure as MySQL's SHOW OPEN TABLES + return nil +} + func (e *ShowExec) fetchShowTables() error { checker := privilege.GetPrivilegeManager(e.ctx) if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.DBIsVisible(e.DBName.O) { @@ -908,7 +940,15 @@ func (e *ShowExec) fetchShowGrants() error { if checker == nil { return errors.New("miss privilege checker") } - gs, err := checker.ShowGrants(e.ctx, e.User) + for _, r := range e.Roles { + if r.Hostname == "" { + r.Hostname = "%" + } + if !checker.FindEdge(e.ctx, r, e.User) { + return ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String()) + } + } + gs, err := checker.ShowGrants(e.ctx, e.User, e.Roles) if err != nil { return errors.Trace(err) } diff --git a/executor/show_test.go b/executor/show_test.go index 170321b5b9d87..bd623916447bd 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -382,6 +382,12 @@ func (s *testSuite2) TestShowSlow(c *C) { tk.MustQuery(`admin show slow top all 3`) } +func (s *testSuite2) TestShowOpenTables(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustQuery("show open tables") + tk.MustQuery("show open tables in test") +} + func (s *testSuite2) TestShowCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/simple.go b/executor/simple.go index 8f645b176d2f5..5cd1ef5259d95 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -87,11 +87,152 @@ func (e *SimpleExec) Next(ctx context.Context, req *chunk.RecordBatch) (err erro err = e.executeSetRole(x) case *ast.RevokeRoleStmt: err = e.executeRevokeRole(x) + case *ast.SetDefaultRoleStmt: + err = e.executeSetDefaultRole(x) } e.done = true return err } +func (e *SimpleExec) setDefaultRoleNone(s *ast.SetDefaultRoleStmt) error { + sqlExecutor := e.ctx.(sqlexec.SQLExecutor) + if _, err := sqlExecutor.Execute(context.Background(), "begin"); err != nil { + return err + } + for _, u := range s.UserList { + if u.Hostname == "" { + u.Hostname = "%" + } + sql := fmt.Sprintf("DELETE IGNORE FROM mysql.default_roles WHERE USER='%s' AND HOST='%s';", u.Username, u.Hostname) + if _, err := sqlExecutor.Execute(context.Background(), sql); err != nil { + logutil.Logger(context.Background()).Error(fmt.Sprintf("Error occur when executing %s", sql)) + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return err + } + } + if _, err := sqlExecutor.Execute(context.Background(), "commit"); err != nil { + return err + } + return nil +} + +func (e *SimpleExec) setDefaultRoleRegular(s *ast.SetDefaultRoleStmt) error { + for _, user := range s.UserList { + exists, err := userExists(e.ctx, user.Username, user.Hostname) + if err != nil { + return err + } + if !exists { + return ErrCannotUser.GenWithStackByArgs("SET DEFAULT ROLE", user.String()) + } + } + for _, role := range s.RoleList { + exists, err := userExists(e.ctx, role.Username, role.Hostname) + if err != nil { + return err + } + if !exists { + return ErrCannotUser.GenWithStackByArgs("SET DEFAULT ROLE", role.String()) + } + } + sqlExecutor := e.ctx.(sqlexec.SQLExecutor) + if _, err := sqlExecutor.Execute(context.Background(), "begin"); err != nil { + return err + } + for _, user := range s.UserList { + if user.Hostname == "" { + user.Hostname = "%" + } + sql := fmt.Sprintf("DELETE IGNORE FROM mysql.default_roles WHERE USER='%s' AND HOST='%s';", user.Username, user.Hostname) + if _, err := sqlExecutor.Execute(context.Background(), sql); err != nil { + logutil.Logger(context.Background()).Error(fmt.Sprintf("Error occur when executing %s", sql)) + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return err + } + for _, role := range s.RoleList { + sql := fmt.Sprintf("INSERT IGNORE INTO mysql.default_roles values('%s', '%s', '%s', '%s');", user.Hostname, user.Username, role.Hostname, role.Username) + checker := privilege.GetPrivilegeManager(e.ctx) + ok := checker.FindEdge(e.ctx, role, user) + if ok { + if _, err := sqlExecutor.Execute(context.Background(), sql); err != nil { + logutil.Logger(context.Background()).Error(fmt.Sprintf("Error occur when executing %s", sql)) + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return err + } + } else { + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return ErrRoleNotGranted.GenWithStackByArgs(role.String(), user.String()) + } + } + } + if _, err := sqlExecutor.Execute(context.Background(), "commit"); err != nil { + return err + } + return nil +} + +func (e *SimpleExec) setDefaultRoleAll(s *ast.SetDefaultRoleStmt) error { + for _, user := range s.UserList { + exists, err := userExists(e.ctx, user.Username, user.Hostname) + if err != nil { + return err + } + if !exists { + return ErrCannotUser.GenWithStackByArgs("SET DEFAULT ROLE", user.String()) + } + } + sqlExecutor := e.ctx.(sqlexec.SQLExecutor) + if _, err := sqlExecutor.Execute(context.Background(), "begin"); err != nil { + return err + } + for _, user := range s.UserList { + if user.Hostname == "" { + user.Hostname = "%" + } + sql := fmt.Sprintf("DELETE IGNORE FROM mysql.default_roles WHERE USER='%s' AND HOST='%s';", user.Username, user.Hostname) + if _, err := sqlExecutor.Execute(context.Background(), sql); err != nil { + logutil.Logger(context.Background()).Error(fmt.Sprintf("Error occur when executing %s", sql)) + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return err + } + sql = fmt.Sprintf("INSERT IGNORE INTO mysql.default_roles(HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER) "+ + "SELECT TO_HOST,TO_USER,FROM_HOST,FROM_USER FROM mysql.role_edges WHERE TO_HOST='%s' AND TO_USER='%s';", user.Hostname, user.Username) + if _, err := sqlExecutor.Execute(context.Background(), sql); err != nil { + if _, rollbackErr := sqlExecutor.Execute(context.Background(), "rollback"); rollbackErr != nil { + return rollbackErr + } + return err + } + } + if _, err := sqlExecutor.Execute(context.Background(), "commit"); err != nil { + return err + } + return nil +} + +func (e *SimpleExec) executeSetDefaultRole(s *ast.SetDefaultRoleStmt) error { + switch s.SetRoleOpt { + case ast.SetRoleAll: + return e.setDefaultRoleAll(s) + case ast.SetRoleNone: + return e.setDefaultRoleNone(s) + case ast.SetRoleRegular: + return e.setDefaultRoleRegular(s) + } + err := domain.GetDomain(e.ctx).PrivilegeHandle().Update(e.ctx.(sessionctx.Context)) + return err +} + func (e *SimpleExec) executeSetRole(s *ast.SetRoleStmt) error { checkDup := make(map[string]*auth.RoleIdentity, len(s.RoleList)) // Check whether RoleNameList contain duplicate role name. diff --git a/executor/simple_test.go b/executor/simple_test.go index 6cc38f1778af5..aaeda7a67beec 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -160,6 +160,56 @@ func (s *testSuite3) TestRole(c *C) { tk.MustExec(dropRoleSQL) } +func (s *testSuite3) TestDefaultRole(c *C) { + tk := testkit.NewTestKit(c, s.store) + + createRoleSQL := `CREATE ROLE r_1, r_2, r_3, u_1;` + tk.MustExec(createRoleSQL) + + tk.MustExec("insert into mysql.role_edges (FROM_HOST,FROM_USER,TO_HOST,TO_USER) values ('%','r_1','%','u_1')") + tk.MustExec("insert into mysql.role_edges (FROM_HOST,FROM_USER,TO_HOST,TO_USER) values ('%','r_2','%','u_1')") + + tk.MustExec("flush privileges;") + + setRoleSQL := `SET DEFAULT ROLE r_3 TO u_1;` + _, err := tk.Exec(setRoleSQL) + c.Check(err, NotNil) + + setRoleSQL = `SET DEFAULT ROLE r_1 TO u_1000;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, NotNil) + + setRoleSQL = `SET DEFAULT ROLE r_1, r_3 TO u_1;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, NotNil) + + setRoleSQL = `SET DEFAULT ROLE r_1 TO u_1;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, IsNil) + result := tk.MustQuery(`SELECT DEFAULT_ROLE_USER FROM mysql.default_roles WHERE USER="u_1"`) + result.Check(testkit.Rows("r_1")) + setRoleSQL = `SET DEFAULT ROLE r_2 TO u_1;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, IsNil) + result = tk.MustQuery(`SELECT DEFAULT_ROLE_USER FROM mysql.default_roles WHERE USER="u_1"`) + result.Check(testkit.Rows("r_2")) + + setRoleSQL = `SET DEFAULT ROLE ALL TO u_1;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, IsNil) + result = tk.MustQuery(`SELECT DEFAULT_ROLE_USER FROM mysql.default_roles WHERE USER="u_1"`) + result.Check(testkit.Rows("r_1", "r_2")) + + setRoleSQL = `SET DEFAULT ROLE NONE TO u_1;` + _, err = tk.Exec(setRoleSQL) + c.Check(err, IsNil) + result = tk.MustQuery(`SELECT DEFAULT_ROLE_USER FROM mysql.default_roles WHERE USER="u_1"`) + result.Check(nil) + + dropRoleSQL := `DROP USER r_1, r_2, r_3, u_1;` + tk.MustExec(dropRoleSQL) +} + func (s *testSuite3) TestUser(c *C) { tk := testkit.NewTestKit(c, s.store) // Make sure user test not in mysql.User. diff --git a/executor/table_reader.go b/executor/table_reader.go index 6f451d11b9f15..5f2785419b92e 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -99,11 +99,11 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { } e.resultHandler = &tableResultHandler{} - if e.feedback != nil && e.feedback.Hist() != nil { + if e.feedback != nil && e.feedback.Hist != nil { // EncodeInt don't need *statement.Context. - e.ranges = e.feedback.Hist().SplitRange(nil, e.ranges, false) + e.ranges = e.feedback.Hist.SplitRange(nil, e.ranges, false) } - firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder) + firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder, e.desc) firstResult, err := e.buildResp(ctx, firstPartRanges) if err != nil { e.feedback.Invalidate() @@ -176,10 +176,12 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } type tableResultHandler struct { - // If the pk is unsigned and we have KeepOrder=true. - // optionalResult handles the request whose range is in signed int range. - // result handles the request whose range is exceed signed int range. - // Otherwise, we just set optionalFinished true and the result handles the whole ranges. + // If the pk is unsigned and we have KeepOrder=true and want ascending order, + // `optionalResult` will handles the request whose range is in signed int range, and + // `result` will handle the request whose range is exceed signed int range. + // If we want descending order, `optionalResult` will handles the request whose range is exceed signed, and + // the `result` will handle the request whose range is in signed. + // Otherwise, we just set `optionalFinished` true and the `result` handles the whole ranges. optionalResult distsql.SelectResult result distsql.SelectResult diff --git a/executor/write_test.go b/executor/write_test.go index d6cf321a1447a..b553bde6ba788 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -719,6 +719,11 @@ commit;` tk.CheckLastMessage("") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) + + tk.MustExec(`SET sql_mode='';`) + _, err = tk.Exec(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) + c.Assert(err, NotNil) + tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) } func (s *testSuite2) TestInsertIgnoreOnDup(c *C) { diff --git a/expression/builtin_json.go b/expression/builtin_json.go index f2ba584a75d78..0693a43fde382 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -66,6 +67,7 @@ var ( _ builtinFunc = &builtinJSONMergeSig{} _ builtinFunc = &builtinJSONContainsSig{} _ builtinFunc = &builtinJSONDepthSig{} + _ builtinFunc = &builtinJSONSearchSig{} _ builtinFunc = &builtinJSONKeysSig{} _ builtinFunc = &builtinJSONKeys2ArgsSig{} _ builtinFunc = &builtinJSONLengthSig{} @@ -876,8 +878,121 @@ type jsonSearchFunctionClass struct { baseFunctionClass } +type builtinJSONSearchSig struct { + baseBuiltinFunc +} + +func (b *builtinJSONSearchSig) Clone() builtinFunc { + newSig := &builtinJSONSearchSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonSearchFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { - return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "JSON_SEARCH") + if err := c.verifyArgs(args); err != nil { + return nil, err + } + // json_doc, one_or_all, search_str[, escape_char[, path] ...]) + argTps := make([]types.EvalType, 0, len(args)) + argTps = append(argTps, types.ETJson) + for range args[1:] { + argTps = append(argTps, types.ETString) + } + bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + sig := &builtinJSONSearchSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_JsonSearchSig) + return sig, nil +} + +func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res json.BinaryJSON, isNull bool, err error) { + // json_doc + var obj json.BinaryJSON + obj, isNull, err = b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + + // one_or_all + var containType string + containType, isNull, err = b.args[1].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + if containType != json.ContainsPathAll && containType != json.ContainsPathOne { + return res, true, errors.AddStack(json.ErrInvalidJSONContainsPathType) + } + + // search_str & escape_char + var searchStr string + searchStr, isNull, err = b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + escape := byte('\\') + if len(b.args) >= 4 { + var escapeStr string + escapeStr, isNull, err = b.args[3].EvalString(b.ctx, row) + if err != nil { + return res, isNull, err + } + if isNull || len(escapeStr) == 0 { + escape = byte('\\') + } else if len(escapeStr) == 1 { + escape = byte(escapeStr[0]) + } else { + return res, true, errIncorrectArgs.GenWithStackByArgs("ESCAPE") + } + } + patChars, patTypes := stringutil.CompilePattern(searchStr, escape) + + // result + result := make([]interface{}, 0) + + // walk json_doc + walkFn := func(fullpath json.PathExpression, bj json.BinaryJSON) (stop bool, err error) { + if bj.TypeCode == json.TypeCodeString && stringutil.DoMatch(string(bj.GetString()), patChars, patTypes) { + result = append(result, fullpath.String()) + if containType == json.ContainsPathOne { + return true, nil + } + } + return false, nil + } + if len(b.args) >= 5 { // path... + pathExprs := make([]json.PathExpression, 0, len(b.args)-4) + for i := 4; i < len(b.args); i++ { + var s string + s, isNull, err = b.args[i].EvalString(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + var pathExpr json.PathExpression + pathExpr, err = json.ParseJSONPathExpr(s) + if err != nil { + return res, true, err + } + pathExprs = append(pathExprs, pathExpr) + } + err = obj.Walk(walkFn, pathExprs...) + if err != nil { + return res, true, err + } + } else { + err = obj.Walk(walkFn) + if err != nil { + return res, true, err + } + } + + // return + switch len(result) { + case 0: + return res, true, nil + case 1: + return json.CreateBinary(result[0]), false, nil + default: + return json.CreateBinary(result), false, nil + } } type jsonStorageSizeFunctionClass struct { diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index ebb1a6dcfe9f3..b544e36cc81d8 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -773,3 +773,77 @@ func (s *testEvaluatorSuite) TestJSONArrayAppend(c *C) { c.Assert(json.CompareBinary(j1, d.GetMysqlJSON()), Equals, 0, comment) } } + +func (s *testEvaluatorSuite) TestJSONSearch(c *C) { + defer testleak.AfterTest(c)() + fc := funcs[ast.JSONSearch] + jsonString := `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]` + jsonString2 := `["abc", [{"k": "10"}, "def"], {"x":"ab%d"}, {"y":"abcd"}]` + tbl := []struct { + input []interface{} + expected interface{} + success bool + }{ + // simple case + {[]interface{}{jsonString, `one`, `abc`}, `"$[0]"`, true}, + {[]interface{}{jsonString, `all`, `abc`}, `["$[0]", "$[2].x"]`, true}, + {[]interface{}{jsonString, `all`, `ghi`}, nil, true}, + {[]interface{}{jsonString, `all`, `10`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[*]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$**.k`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[*][0].k`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[1]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `10`, nil, `$[1][0]`}, `"$[1][0].k"`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`}, `"$[2].x"`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`, `$[0]`}, `["$[2].x", "$[0]"]`, true}, + {[]interface{}{jsonString, `all`, `abc`, nil, `$[2]`, `$[2]`}, `"$[2].x"`, true}, + + // search pattern + {[]interface{}{jsonString, `all`, `%a%`}, `["$[0]", "$[2].x"]`, true}, + {[]interface{}{jsonString, `all`, `%b%`}, `["$[0]", "$[2].x", "$[3].y"]`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[0]`}, `"$[0]"`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[2]`}, `"$[2].x"`, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[1]`}, nil, true}, + {[]interface{}{jsonString, `all`, `%b%`, ``, `$[1]`}, nil, true}, + {[]interface{}{jsonString, `all`, `%b%`, nil, `$[3]`}, `"$[3].y"`, true}, + {[]interface{}{jsonString2, `all`, `ab_d`}, `["$[2].x", "$[3].y"]`, true}, + + // escape char + {[]interface{}{jsonString2, `all`, `ab%d`}, `["$[2].x", "$[3].y"]`, true}, + {[]interface{}{jsonString2, `all`, `ab\%d`}, `"$[2].x"`, true}, + {[]interface{}{jsonString2, `all`, `ab|%d`, `|`}, `"$[2].x"`, true}, + + // error handle + {[]interface{}{nil, `all`, `abc`}, nil, true}, // NULL json + {[]interface{}{`a`, `all`, `abc`}, nil, false}, // non json + {[]interface{}{jsonString, `wrong`, `abc`}, nil, false}, // wrong one_or_all + {[]interface{}{jsonString, `all`, nil}, nil, true}, // NULL search_str + {[]interface{}{jsonString, `all`, `abc`, `??`}, nil, false}, // wrong escape_char + {[]interface{}{jsonString, `all`, `abc`, nil, nil}, nil, true}, // NULL path + {[]interface{}{jsonString, `all`, `abc`, nil, `$xx`}, nil, false}, // wrong path + } + for _, t := range tbl { + args := types.MakeDatums(t.input...) + f, err := fc.getFunction(s.ctx, s.datumsToConstants(args)) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if t.success { + c.Assert(err, IsNil) + switch x := t.expected.(type) { + case string: + var j1, j2 json.BinaryJSON + j1, err = json.ParseBinaryFromString(x) + c.Assert(err, IsNil) + j2 = d.GetMysqlJSON() + cmp := json.CompareBinary(j1, j2) + //fmt.Println(j1, j2) + c.Assert(cmp, Equals, 0) + case nil: + c.Assert(d.IsNull(), IsTrue) + } + } else { + c.Assert(err, NotNil) + } + } +} diff --git a/expression/constant_fold.go b/expression/constant_fold.go index 6bcc46a4f4112..adec69c78f8a0 100644 --- a/expression/constant_fold.go +++ b/expression/constant_fold.go @@ -117,7 +117,10 @@ func foldConstant(expr Expression) (Expression, bool) { constArgs[i] = One } } - dummyScalarFunc := NewFunctionInternal(x.GetCtx(), x.FuncName.L, x.GetType(), constArgs...) + dummyScalarFunc, err := NewFunctionBase(x.GetCtx(), x.FuncName.L, x.GetType(), constArgs...) + if err != nil { + return expr, isDeferredConst + } value, err := dummyScalarFunc.Eval(chunk.Row{}) if err != nil { return expr, isDeferredConst diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 7135905d80462..96ac0397d0cd5 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -444,6 +444,8 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti f = &builtinJSONLengthSig{base} case tipb.ScalarFuncSig_JsonDepthSig: f = &builtinJSONDepthSig{base} + case tipb.ScalarFuncSig_JsonSearchSig: + f = &builtinJSONSearchSig{base} case tipb.ScalarFuncSig_InInt: f = &builtinInIntSig{base} diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 34545f024ca1b..840fd5e89c755 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -17,6 +17,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -108,6 +109,17 @@ func (s *testEvalSuite) TestEval(c *C) { ), types.NewIntDatum(3), }, + { + scalarFunctionExpr(tipb.ScalarFuncSig_JsonSearchSig, + toPBFieldType(newJSONFieldType()), + jsonDatumExpr(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`), + datumExpr(c, types.NewBytesDatum([]byte(`all`))), + datumExpr(c, types.NewBytesDatum([]byte(`10`))), + datumExpr(c, types.NewBytesDatum([]byte(`\`))), + datumExpr(c, types.NewBytesDatum([]byte(`$**.k`))), + ), + newJSONDatum(c, `"$[1][0].k"`), + }, } sc := new(stmtctx.StatementContext) for _, tt := range tests { @@ -178,12 +190,15 @@ func datumExpr(c *C, d types.Datum) *tipb.Expr { return expr } -func jsonDatumExpr(c *C, s string) *tipb.Expr { - var d types.Datum +func newJSONDatum(c *C, s string) (d types.Datum) { j, err := json.ParseBinaryFromString(s) c.Assert(err, IsNil) d.SetMysqlJSON(j) - return datumExpr(c, d) + return d +} + +func jsonDatumExpr(c *C, s string) *tipb.Expr { + return datumExpr(c, newJSONDatum(c, s)) } func columnExpr(columnID int64) *tipb.Expr { @@ -214,6 +229,16 @@ func newIntFieldType() *types.FieldType { } } +func newJSONFieldType() *types.FieldType { + return &types.FieldType{ + Tp: mysql.TypeJSON, + Flen: types.UnspecifiedLength, + Decimal: 0, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + } +} + func scalarFunctionExpr(sigCode tipb.ScalarFuncSig, retType *tipb.FieldType, args ...*tipb.Expr) *tipb.Expr { return &tipb.Expr{ Tp: tipb.ExprType_ScalarFunc, diff --git a/expression/integration_test.go b/expression/integration_test.go index 6bd91b57691bf..33b8f5a8076da 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4213,3 +4213,25 @@ func (s *testIntegrationSuite) TestDaynameArithmetic(c *C) { tk.MustQuery(c.sql).Check(testkit.Rows(c.result)) } } + +func (s *testIntegrationSuite) TestIssue10156(c *C) { + tk := testkit.NewTestKit(c, s.store) + defer s.cleanEnv(c) + + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `t1` (`period_name` varchar(24) DEFAULT NULL ,`period_id` bigint(20) DEFAULT NULL ,`starttime` bigint(20) DEFAULT NULL)") + tk.MustExec("CREATE TABLE `t2` (`bussid` bigint(20) DEFAULT NULL,`ct` bigint(20) DEFAULT NULL)") + q := ` +select + a.period_name, + b.date8 +from + (select * from t1) a +left join + (select bussid,date(from_unixtime(ct)) date8 from t2) b +on + a.period_id = b.bussid +where + datediff(b.date8, date(from_unixtime(a.starttime))) >= 0` + tk.MustQuery(q) +} diff --git a/go.mod b/go.mod index ca535624af6d6..ecbd19cb3b8c6 100644 --- a/go.mod +++ b/go.mod @@ -51,7 +51,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30 + github.com/pingcap/parser v0.0.0-20190418074231-2cdba8195b40 github.com/pingcap/pd v2.1.0-rc.4+incompatible github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 diff --git a/go.sum b/go.sum index 6cf70f55d8c18..a53c14279e76e 100644 --- a/go.sum +++ b/go.sum @@ -119,8 +119,8 @@ github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 h1:32oF1/8lVnBR2JV github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30 h1:Cu+VJBHLUqI0TFj/0Kya4L1iHIJZ3VbtZcEwv+3zOxQ= -github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190418074231-2cdba8195b40 h1:fw59RpNk5buTOZUGtv5X2vJzgbF9yoJ/I1kEUFmgbt0= +github.com/pingcap/parser v0.0.0-20190418074231-2cdba8195b40/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE= github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 1b6f7a8ad43e8..17203cd52e760 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -48,6 +48,7 @@ var slowQueryCols = []columnInfo{ {variable.SlowLogIndexIDsStr, mysql.TypeVarchar, 100, 0, nil, nil}, {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogDigestStr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogStatsInfoStr, mysql.TypeVarchar, 512, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -139,6 +140,7 @@ type slowQueryTuple struct { indexIDs string isInternal bool digest string + statsInfo string sql string } @@ -217,6 +219,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.isInternal = value == "true" case variable.SlowLogDigestStr: st.digest = value + case variable.SlowLogStatsInfoStr: + st.statsInfo = value case variable.SlowLogQuerySQLStr: st.sql = value } @@ -244,6 +248,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record = append(record, types.NewStringDatum(st.indexIDs)) record = append(record, types.NewDatum(st.isInternal)) record = append(record, types.NewStringDatum(st.digest)) + record = append(record, types.NewStringDatum(st.statsInfo)) record = append(record, types.NewStringDatum(st.sql)) return record } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 00f5228ac5d70..f05eb28d7077b 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -31,6 +31,7 @@ func (s *testSuite) TestParseSlowLogFile(c *C) { # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:1,t2:2 select * from t;`) scanner := bufio.NewScanner(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") @@ -47,7 +48,7 @@ select * from t;`) } recordString += str } - expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,select * from t;" + expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,select * from t;" c.Assert(expectRecordString, Equals, recordString) } diff --git a/infoschema/tables.go b/infoschema/tables.go index 04d7b1845d42f..46cd30826d936 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -19,6 +19,7 @@ import ( "sync" "time" + "github.com/pingcap/errors" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -27,8 +28,11 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/helper" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/sqlexec" ) @@ -67,6 +71,7 @@ const ( tableProcesslist = "PROCESSLIST" tableTiDBIndexes = "TIDB_INDEXES" tableSlowLog = "SLOW_QUERY" + tableTiDBHotRegions = "TIDB_HOT_REGIONS" ) type columnInfo struct { @@ -544,6 +549,18 @@ var tableTiDBIndexesCols = []columnInfo{ {"INDEX_ID", mysql.TypeLonglong, 21, 0, nil, nil}, } +var tableTiDBHotRegionsCols = []columnInfo{ + {"TABLE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"INDEX_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"DB_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"TABLE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"INDEX_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"TYPE", mysql.TypeVarchar, 64, 0, nil, nil}, + {"MAX_HOT_DEGREE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"FLOW_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, +} + func dataForCharacterSets() (records [][]types.Datum) { charsets := charset.GetAllCharsets() @@ -1418,6 +1435,52 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ return rows } +func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") + } + allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(InfoSchema).AllSchemas() + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + metrics, err := tikvHelper.ScrapeHotInfo(pdapi.HotRead, allSchemas) + if err != nil { + return nil, err + } + records = append(records, dataForHotRegionByMetrics(metrics, "read")...) + metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) + if err != nil { + return nil, err + } + records = append(records, dataForHotRegionByMetrics(metrics, "write")...) + return records, nil +} + +func dataForHotRegionByMetrics(metrics map[helper.TblIndex]helper.RegionMetric, tp string) [][]types.Datum { + rows := make([][]types.Datum, 0, len(metrics)) + for tblIndex, regionMetric := range metrics { + row := make([]types.Datum, len(tableTiDBHotRegionsCols)) + if tblIndex.IndexName != "" { + row[1].SetInt64(tblIndex.IndexID) + row[4].SetString(tblIndex.IndexName) + } else { + row[1].SetNull() + row[4].SetNull() + } + row[0].SetInt64(tblIndex.TableID) + row[2].SetString(tblIndex.DbName) + row[3].SetString(tblIndex.TableName) + row[5].SetString(tp) + row[6].SetInt64(int64(regionMetric.MaxHotDegree)) + row[7].SetInt64(int64(regionMetric.Count)) + row[8].SetUint64(regionMetric.FlowBytes) + rows = append(rows, row) + } + return rows +} + var tableNameToColumns = map[string][]columnInfo{ tableSchemata: schemataCols, tableTables: tablesCols, @@ -1452,6 +1515,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableProcesslist: tableProcesslistCols, tableTiDBIndexes: tableTiDBIndexesCols, tableSlowLog: slowQueryCols, + tableTiDBHotRegions: tableTiDBHotRegionsCols, } func createInfoSchemaTable(handle *Handle, meta *model.TableInfo) *infoschemaTable { @@ -1545,6 +1609,8 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows = dataForProcesslist(ctx) case tableSlowLog: fullRows, err = dataForSlowLog(ctx) + case tableTiDBHotRegions: + fullRows, err = dataForTiDBHotRegions(ctx) } if err != nil { return nil, err diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 5a805d3b1aa8f..a6f229bd32f66 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/testkit" @@ -137,22 +137,22 @@ func (s *testTableSuite) TestDataForTableStatsField(c *C) { tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( testkit.Rows("0 0 0 0")) tk.MustExec(`insert into t(c, d, e) values(1, 2, "c"), (2, 3, "d"), (3, 4, "e")`) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( testkit.Rows("3 17 51 3")) tk.MustExec(`insert into t(c, d, e) values(4, 5, "f")`) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( testkit.Rows("4 17 68 4")) tk.MustExec("delete from t where c >= 3") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( testkit.Rows("2 17 34 2")) tk.MustExec("delete from t where c=3") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( testkit.Rows("2 17 34 2")) @@ -168,7 +168,7 @@ func (s *testTableSuite) TestCharacterSetCollations(c *C) { // The is_default column is not important // but the id's are used by client libraries and must be stable tk.MustQuery("SELECT character_set_name, id, sortlen FROM information_schema.collations ORDER BY collation_name").Check( - testkit.Rows("armscii8 64 1", "armscii8 32 1", "ascii 65 1", "ascii 11 1", "big5 84 1", "big5 1 1", "binary 63 1", "cp1250 66 1", "cp1250 44 1", "cp1250 34 1", "cp1250 26 1", "cp1250 99 1", "cp1251 50 1", "cp1251 14 1", "cp1251 51 1", "cp1251 52 1", "cp1251 23 1", "cp1256 67 1", "cp1256 57 1", "cp1257 58 1", "cp1257 59 1", "cp1257 29 1", "cp850 80 1", "cp850 4 1", "cp852 81 1", "cp852 40 1", "cp866 68 1", "cp866 36 1", "cp932 96 1", "cp932 95 1", "dec8 69 1", "dec8 3 1", "eucjpms 98 1", "eucjpms 97 1", "euckr 85 1", "euckr 19 1", "gb2312 86 1", "gb2312 24 1", "gbk 87 1", "gbk 28 1", "geostd8 93 1", "geostd8 92 1", "greek 70 1", "greek 25 1", "hebrew 71 1", "hebrew 16 1", "hp8 72 1", "hp8 6 1", "keybcs2 73 1", "keybcs2 37 1", "koi8r 74 1", "koi8r 7 1", "koi8u 75 1", "koi8u 22 1", "latin1 47 1", "latin1 15 1", "latin1 48 1", "latin1 49 1", "latin1 5 1", "latin1 31 1", "latin1 94 1", "latin1 8 1", "latin2 77 1", "latin2 27 1", "latin2 2 1", "latin2 9 1", "latin2 21 1", "latin5 78 1", "latin5 30 1", "latin7 79 1", "latin7 20 1", "latin7 41 1", "latin7 42 1", "macce 43 1", "macce 38 1", "macroman 53 1", "macroman 39 1", "sjis 88 1", "sjis 13 1", "swe7 82 1", "swe7 10 1", "tis620 89 1", "tis620 18 1", "ucs2 90 1", "ucs2 149 1", "ucs2 138 1", "ucs2 139 1", "ucs2 145 1", "ucs2 134 1", "ucs2 35 1", "ucs2 159 1", "ucs2 148 1", "ucs2 146 1", "ucs2 129 1", "ucs2 130 1", "ucs2 140 1", "ucs2 144 1", "ucs2 133 1", "ucs2 143 1", "ucs2 131 1", "ucs2 147 1", "ucs2 141 1", "ucs2 132 1", "ucs2 142 1", "ucs2 135 1", "ucs2 136 1", "ucs2 137 1", "ucs2 150 1", "ucs2 128 1", "ucs2 151 1", "ujis 91 1", "ujis 12 1", "utf16 55 1", "utf16 122 1", "utf16 111 1", "utf16 112 1", "utf16 118 1", "utf16 107 1", "utf16 54 1", "utf16 121 1", "utf16 119 1", "utf16 102 1", "utf16 103 1", "utf16 113 1", "utf16 117 1", "utf16 106 1", "utf16 116 1", "utf16 104 1", "utf16 120 1", "utf16 114 1", "utf16 105 1", "utf16 115 1", "utf16 108 1", "utf16 109 1", "utf16 110 1", "utf16 123 1", "utf16 101 1", "utf16 124 1", "utf16le 62 1", "utf16le 56 1", "utf32 61 1", "utf32 181 1", "utf32 170 1", "utf32 171 1", "utf32 177 1", "utf32 166 1", "utf32 60 1", "utf32 180 1", "utf32 178 1", "utf32 161 1", "utf32 162 1", "utf32 172 1", "utf32 176 1", "utf32 165 1", "utf32 175 1", "utf32 163 1", "utf32 179 1", "utf32 173 1", "utf32 164 1", "utf32 174 1", "utf32 167 1", "utf32 168 1", "utf32 169 1", "utf32 182 1", "utf32 160 1", "utf32 183 1", "utf8 83 1", "utf8 213 1", "utf8 202 1", "utf8 203 1", "utf8 209 1", "utf8 198 1", "utf8 33 1", "utf8 223 1", "utf8 212 1", "utf8 210 1", "utf8 193 1", "utf8 194 1", "utf8 204 1", "utf8 208 1", "utf8 197 1", "utf8 207 1", "utf8 195 1", "utf8 211 1", "utf8 205 1", "utf8 196 1", "utf8 206 1", "utf8 199 1", "utf8 200 1", "utf8 201 1", "utf8 214 1", "utf8 192 1", "utf8 215 1", "utf8mb4 46 1", "utf8mb4 245 1", "utf8mb4 234 1", "utf8mb4 235 1", "utf8mb4 241 1", "utf8mb4 230 1", "utf8mb4 45 1", "utf8mb4 244 1", "utf8mb4 242 1", "utf8mb4 225 1", "utf8mb4 226 1", "utf8mb4 236 1", "utf8mb4 240 1", "utf8mb4 229 1", "utf8mb4 239 1", "utf8mb4 227 1", "utf8mb4 243 1", "utf8mb4 237 1", "utf8mb4 228 1", "utf8mb4 238 1", "utf8mb4 231 1", "utf8mb4 232 1", "utf8mb4 233 1", "utf8mb4 246 1", "utf8mb4 224 1", "utf8mb4 247 1")) + testkit.Rows("armscii8 64 1", "armscii8 32 1", "ascii 65 1", "ascii 11 1", "big5 84 1", "big5 1 1", "binary 63 1", "cp1250 66 1", "cp1250 44 1", "cp1250 34 1", "cp1250 26 1", "cp1250 99 1", "cp1251 50 1", "cp1251 14 1", "cp1251 51 1", "cp1251 52 1", "cp1251 23 1", "cp1256 67 1", "cp1256 57 1", "cp1257 58 1", "cp1257 59 1", "cp1257 29 1", "cp850 80 1", "cp850 4 1", "cp852 81 1", "cp852 40 1", "cp866 68 1", "cp866 36 1", "cp932 96 1", "cp932 95 1", "dec8 69 1", "dec8 3 1", "eucjpms 98 1", "eucjpms 97 1", "euckr 85 1", "euckr 19 1", "gb2312 86 1", "gb2312 24 1", "gbk 87 1", "gbk 28 1", "geostd8 93 1", "geostd8 92 1", "greek 70 1", "greek 25 1", "hebrew 71 1", "hebrew 16 1", "hp8 72 1", "hp8 6 1", "keybcs2 73 1", "keybcs2 37 1", "koi8r 74 1", "koi8r 7 1", "koi8u 75 1", "koi8u 22 1", "latin1 47 1", "latin1 15 1", "latin1 48 1", "latin1 49 1", "latin1 5 1", "latin1 31 1", "latin1 94 1", "latin1 8 1", "latin2 77 1", "latin2 27 1", "latin2 2 1", "latin2 9 1", "latin2 21 1", "latin5 78 1", "latin5 30 1", "latin7 79 1", "latin7 20 1", "latin7 41 1", "latin7 42 1", "macce 43 1", "macce 38 1", "macroman 53 1", "macroman 39 1", "sjis 88 1", "sjis 13 1", "swe7 82 1", "swe7 10 1", "tis620 89 1", "tis620 18 1", "ucs2 90 1", "ucs2 149 1", "ucs2 138 1", "ucs2 139 1", "ucs2 145 1", "ucs2 134 1", "ucs2 35 1", "ucs2 159 1", "ucs2 148 1", "ucs2 146 1", "ucs2 129 1", "ucs2 130 1", "ucs2 140 1", "ucs2 144 1", "ucs2 133 1", "ucs2 143 1", "ucs2 131 1", "ucs2 147 1", "ucs2 141 1", "ucs2 132 1", "ucs2 142 1", "ucs2 135 1", "ucs2 136 1", "ucs2 137 1", "ucs2 150 1", "ucs2 128 1", "ucs2 151 1", "ujis 91 1", "ujis 12 1", "utf16 55 1", "utf16 122 1", "utf16 111 1", "utf16 112 1", "utf16 118 1", "utf16 107 1", "utf16 54 1", "utf16 121 1", "utf16 119 1", "utf16 102 1", "utf16 103 1", "utf16 113 1", "utf16 117 1", "utf16 106 1", "utf16 116 1", "utf16 104 1", "utf16 120 1", "utf16 114 1", "utf16 105 1", "utf16 115 1", "utf16 108 1", "utf16 109 1", "utf16 110 1", "utf16 123 1", "utf16 101 1", "utf16 124 1", "utf16le 62 1", "utf16le 56 1", "utf32 61 1", "utf32 181 1", "utf32 170 1", "utf32 171 1", "utf32 177 1", "utf32 166 1", "utf32 60 1", "utf32 180 1", "utf32 178 1", "utf32 161 1", "utf32 162 1", "utf32 172 1", "utf32 176 1", "utf32 165 1", "utf32 175 1", "utf32 163 1", "utf32 179 1", "utf32 173 1", "utf32 164 1", "utf32 174 1", "utf32 167 1", "utf32 168 1", "utf32 169 1", "utf32 182 1", "utf32 160 1", "utf32 183 1", "utf8 83 1", "utf8 213 1", "utf8 202 1", "utf8 203 1", "utf8 209 1", "utf8 198 1", "utf8 33 1", "utf8 223 1", "utf8 212 1", "utf8 210 1", "utf8 193 1", "utf8 194 1", "utf8 204 1", "utf8 208 1", "utf8 197 1", "utf8 207 1", "utf8 195 1", "utf8 211 1", "utf8 205 1", "utf8 196 1", "utf8 206 1", "utf8 199 1", "utf8 200 1", "utf8 201 1", "utf8 214 1", "utf8 192 1", "utf8 215 1", "utf8mb4 255 1", "utf8mb4 46 1", "utf8mb4 245 1", "utf8mb4 234 1", "utf8mb4 235 1", "utf8mb4 241 1", "utf8mb4 230 1", "utf8mb4 45 1", "utf8mb4 244 1", "utf8mb4 242 1", "utf8mb4 225 1", "utf8mb4 226 1", "utf8mb4 236 1", "utf8mb4 240 1", "utf8mb4 229 1", "utf8mb4 239 1", "utf8mb4 227 1", "utf8mb4 243 1", "utf8mb4 237 1", "utf8mb4 228 1", "utf8mb4 238 1", "utf8mb4 231 1", "utf8mb4 232 1", "utf8mb4 233 1", "utf8mb4 246 1", "utf8mb4 224 1", "utf8mb4 247 1")) // Test charset/collation in information_schema.COLUMNS table. tk.MustExec("DROP DATABASE IF EXISTS charset_collate_test") @@ -339,6 +339,7 @@ func (s *testTableSuite) TestSlowQuery(c *C) { # DB: test # Is_internal: false # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:1,t2:2 select * from t_slim;`)) c.Assert(f.Close(), IsNil) c.Assert(err, IsNil) @@ -346,8 +347,8 @@ select * from t_slim;`)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) tk.MustExec("set time_zone = '+08:00';") re := tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) } diff --git a/kv/kv.go b/kv/kv.go index 53a305b571d98..2a959e61df001 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -271,6 +271,12 @@ type Storage interface { GetOracle() oracle.Oracle // SupportDeleteRange gets the storage support delete range or not. SupportDeleteRange() (supported bool) + // Name gets the name of the storage engine + Name() string + // Describe returns of brief introduction of the storage + Describe() string + // ShowStatus returns the specified status of the storage + ShowStatus(ctx context.Context, key string) (interface{}, error) } // FnKeyCmp is the function for iterator the keys diff --git a/kv/mock.go b/kv/mock.go index f81560fc74ac4..a3fc01f4908a8 100644 --- a/kv/mock.go +++ b/kv/mock.go @@ -171,6 +171,18 @@ func (s *mockStorage) SupportDeleteRange() (supported bool) { return false } +func (s *mockStorage) Name() string { + return "KVMockStorage" +} + +func (s *mockStorage) Describe() string { + return "KVMockStorage is a mock Store implementation, only for unittests in KV package" +} + +func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, nil +} + // MockTxn is used for test cases that need more interfaces than Transaction. type MockTxn interface { Transaction diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 77ae0789d4b75..ca5b6fd01eef4 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -47,7 +48,7 @@ func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) e if err != nil { return err } - statsTbl := &statistics.JSONTable{} + statsTbl := &handle.JSONTable{} err = json.Unmarshal(bytes, statsTbl) if err != nil { return err @@ -105,7 +106,7 @@ func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) { c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)") testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) testKit.MustQuery("explain select * from t1, t2 where t1.a = t2.a").Check(testkit.Rows( "HashLeftJoin_8 7.49 root inner join, inner:TableReader_15, equal:[eq(test.t1.a, test.t2.a)]", @@ -195,7 +196,7 @@ func (s *testAnalyzeSuite) TestTableDual(c *C) { testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) testKit.MustQuery(`explain select * from t where 1 = 0`).Check(testkit.Rows( @@ -225,12 +226,12 @@ func (s *testAnalyzeSuite) TestEstimation(c *C) { testKit.MustExec("insert into t select * from t") h := dom.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) testKit.MustExec("analyze table t") for i := 1; i <= 8; i++ { testKit.MustExec("delete from t where a = ?", i) } - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) testKit.MustQuery("explain select count(*) from t group by a").Check(testkit.Rows( "HashAgg_9 2.00 root group by:col_1, funcs:count(col_0)", @@ -567,12 +568,12 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { } h := dom.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) testKit.MustExec("analyze table t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) statistics.RatioOfPseudoEstimate = 10.0 testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows( diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 517f76aaa08ca..6beeeeba536fe 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -318,13 +318,13 @@ type Show struct { Column *ast.ColumnName // Used for `desc table column`. Flag int // Some flag parsed from sql, such as FULL. Full bool - User *auth.UserIdentity // Used for show grants. - IfNotExists bool // Used for `show create database if not exists` + User *auth.UserIdentity // Used for show grants. + Roles []*auth.RoleIdentity // Used for show grants. + IfNotExists bool // Used for `show create database if not exists` Conditions []expression.Expression - // Used by show variables - GlobalScope bool + GlobalScope bool // Used by show variables } // Set represents a plan for set stmt. @@ -334,6 +334,29 @@ type Set struct { VarAssigns []*expression.VarAssignment } +// SQLBindOpType repreents the SQL bind type +type SQLBindOpType int + +const ( + // OpSQLBindCreate represents the operation to create a SQL bind. + OpSQLBindCreate SQLBindOpType = iota + // OpSQLBindDrop represents the operation to drop a SQL bind. + OpSQLBindDrop +) + +// SQLBindPlan represents a plan for SQL bind. +type SQLBindPlan struct { + baseSchemaProducer + + SQLBindOp SQLBindOpType + NormdOrigSQL string + BindSQL string + IsGlobal bool + BindStmt ast.StmtNode + Charset string + Collation string +} + // Simple represents a simple statement plan which doesn't need any optimization. type Simple struct { baseSchemaProducer diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index f3c1105d2f129..cf042cfe3f08d 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" @@ -516,7 +517,10 @@ func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Col ts.SetSchema(ds.schema) ts.stats = property.NewSimpleStats(1) - ts.stats.UsePseudoStats = ds.statisticTable.Pseudo + ts.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + ts.stats.StatsVersion = statistics.PseudoVersion + } copTask := &copTask{ tablePlan: ts, @@ -563,7 +567,10 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn rowCount = ds.statisticTable.PseudoAvgCountPerValue() } is.stats = property.NewSimpleStats(rowCount) - is.stats.UsePseudoStats = ds.statisticTable.Pseudo + is.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + is.stats.StatsVersion = statistics.PseudoVersion + } cop := &copTask{ indexPlan: is, diff --git a/planner/core/explain.go b/planner/core/explain.go index fe59ad9375d93..50cd85cb2e19f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/statistics" ) // ExplainInfo implements PhysicalPlan interface. @@ -74,7 +75,7 @@ func (p *PhysicalIndexScan) ExplainInfo() string { if p.Desc { buffer.WriteString(", desc") } - if p.stats.UsePseudoStats { + if p.stats.StatsVersion == statistics.PseudoVersion { buffer.WriteString(", stats:pseudo") } return buffer.String() @@ -121,7 +122,7 @@ func (p *PhysicalTableScan) ExplainInfo() string { if p.Desc { buffer.WriteString(", desc") } - if p.stats.UsePseudoStats { + if p.stats.StatsVersion == statistics.PseudoVersion { buffer.WriteString(", stats:pseudo") } return buffer.String() diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index b810e72eda078..8be78abc1f2fc 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "golang.org/x/tools/container/intsets" @@ -504,7 +505,11 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) } is.stats = property.NewSimpleStats(rowCount) - is.stats.UsePseudoStats = ds.statisticTable.Pseudo + is.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + is.stats.StatsVersion = statistics.PseudoVersion + } + cop.cst = rowCount * scanFactor task = cop if candidate.isMatchProp { @@ -653,7 +658,11 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) } ts.stats = property.NewSimpleStats(rowCount) - ts.stats.UsePseudoStats = ds.statisticTable.Pseudo + ts.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + ts.stats.StatsVersion = statistics.PseudoVersion + } + copTask.cst = rowCount * scanFactor if candidate.isMatchProp { if prop.Items[0].Desc { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index c92e6c177b193..81529c780bbd4 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -391,3 +391,26 @@ type PhysicalWindow struct { OrderBy []property.Item Frame *WindowFrame } + +// CollectPlanStatsVersion uses to collect the statistics version of the plan. +func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) map[string]uint64 { + for _, child := range plan.Children() { + statsInfos = CollectPlanStatsVersion(child, statsInfos) + } + switch copPlan := plan.(type) { + case *PhysicalTableReader: + statsInfos = CollectPlanStatsVersion(copPlan.tablePlan, statsInfos) + case *PhysicalIndexReader: + statsInfos = CollectPlanStatsVersion(copPlan.indexPlan, statsInfos) + case *PhysicalIndexLookUpReader: + // For index loop up, only the indexPlan is necessary, + // because they use the same stats and we do not set the stats info for tablePlan. + statsInfos = CollectPlanStatsVersion(copPlan.indexPlan, statsInfos) + case *PhysicalIndexScan: + statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + case *PhysicalTableScan: + statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + } + + return statsInfos +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5213025746245..c8b82bf4b21f0 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -20,6 +20,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" @@ -268,10 +269,14 @@ func (b *PlanBuilder) Build(node ast.Node) (Plan, error) { case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, *ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt, *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt, - *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt: + *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt: return b.buildSimple(node.(ast.StmtNode)) case ast.DDLNode: return b.buildDDL(x) + case *ast.CreateBindingStmt: + return b.buildCreateBindPlan(x) + case *ast.DropBindingStmt: + return b.buildDropBindPlan(x) case *ast.ChangeStmt: return b.buildChange(x) } @@ -361,6 +366,31 @@ func (b *PlanBuilder) buildSet(v *ast.SetStmt) (Plan, error) { return p, nil } +func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) { + p := &SQLBindPlan{ + SQLBindOp: OpSQLBindDrop, + NormdOrigSQL: parser.Normalize(v.OriginSel.Text()), + IsGlobal: v.GlobalScope, + } + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) + return p, nil +} + +func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (Plan, error) { + charSet, collation := b.ctx.GetSessionVars().GetCharsetInfo() + p := &SQLBindPlan{ + SQLBindOp: OpSQLBindCreate, + NormdOrigSQL: parser.Normalize(v.OriginSel.Text()), + BindSQL: v.HintedSel.Text(), + IsGlobal: v.GlobalScope, + BindStmt: v.HintedSel, + Charset: charSet, + Collation: collation, + } + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) + return p, nil +} + // detectSelectAgg detects an aggregate function or GROUP BY clause. func (b *PlanBuilder) detectSelectAgg(sel *ast.SelectStmt) bool { if sel.GroupBy != nil { @@ -1008,6 +1038,7 @@ func (b *PlanBuilder) buildShow(show *ast.ShowStmt) (Plan, error) { Flag: show.Flag, Full: show.Full, User: show.User, + Roles: show.Roles, IfNotExists: show.IfNotExists, GlobalScope: show.GlobalScope, }.Init(b.ctx) @@ -1095,7 +1126,7 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { err := ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "", err) } - case *ast.AlterUserStmt: + case *ast.AlterUserStmt, *ast.SetDefaultRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "", err) case *ast.GrantStmt: @@ -1886,6 +1917,9 @@ func buildShowSchema(s *ast.ShowStmt, isView bool) (schema *expression.Schema) { names = []string{"Engine", "Support", "Comment", "Transactions", "XA", "Savepoints"} case ast.ShowDatabases: names = []string{"Database"} + case ast.ShowOpenTables: + names = []string{"Database", "Table", "In_use", "Name_locked"} + ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeLong} case ast.ShowTables: names = []string{fmt.Sprintf("Tables_in_%s", s.DBName)} if s.Full { @@ -1981,6 +2015,9 @@ func buildShowSchema(s *ast.ShowStmt, isView bool) (schema *expression.Schema) { case ast.ShowPrivileges: names = []string{"Privilege", "Context", "Comment"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar} + case ast.ShowBindings: + names = []string{"Original_sql", "Bind_sql", "Default_db", "Status", "Create_time", "Update_time", "Charset", "Collation"} + ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar} } schema = expression.NewSchema(make([]*expression.Column, 0, len(names))...) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 862d0ae44dab4..a99f3c5bb7fd7 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -111,6 +111,8 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { return in, true case *ast.Join: p.checkNonUniqTableAlias(node) + case *ast.CreateBindingStmt: + p.checkBindGrammar(node) case *ast.RecoverTableStmt: // The specified table in recover table statement maybe already been dropped. // So skip check table name here, otherwise, recover table [table_name] syntax will return @@ -122,6 +124,15 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { return in, p.err != nil } +func (p *preprocessor) checkBindGrammar(createBindingStmt *ast.CreateBindingStmt) { + originSQL := parser.Normalize(createBindingStmt.OriginSel.(*ast.SelectStmt).Text()) + hintedSQL := parser.Normalize(createBindingStmt.HintedSel.(*ast.SelectStmt).Text()) + + if originSQL != hintedSQL { + p.err = errors.Errorf("hinted sql and origin sql don't match when hinted sql erase the hint info, after erase hint info, originSQL:%s, hintedSQL:%s", originSQL, hintedSQL) + } +} + func (p *preprocessor) Leave(in ast.Node) (out ast.Node, ok bool) { switch x := in.(type) { case *ast.CreateTableStmt: diff --git a/planner/core/stats.go b/planner/core/stats.go index 57333e534edd5..0df8d0eb38e8c 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -92,11 +92,15 @@ func (ds *DataSource) getColumnNDV(colID int64) (ndv float64) { func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) (*property.StatsInfo, *statistics.HistColl) { profile := &property.StatsInfo{ - RowCount: float64(ds.statisticTable.Count), - Cardinality: make([]float64, len(ds.Columns)), - HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), - UsePseudoStats: ds.statisticTable.Pseudo, + RowCount: float64(ds.statisticTable.Count), + Cardinality: make([]float64, len(ds.Columns)), + HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), + StatsVersion: ds.statisticTable.Version, } + if ds.statisticTable.Pseudo { + profile.StatsVersion = statistics.PseudoVersion + } + for i, col := range ds.Columns { profile.Cardinality[i] = ds.getColumnNDV(col.ID) } diff --git a/planner/property/stats_info.go b/planner/property/stats_info.go index 8881f044d7ba5..21894731c2cab 100644 --- a/planner/property/stats_info.go +++ b/planner/property/stats_info.go @@ -25,9 +25,9 @@ type StatsInfo struct { Cardinality []float64 HistColl *statistics.HistColl - // UsePseudoStats indicates whether the StatsInfo is calculated using the - // pseudo statistics on a table. - UsePseudoStats bool + // StatsVersion indicates the statistics version of a table. + // If the StatsInfo is calculated using the pseudo statistics on a table, StatsVersion will be PseudoVersion. + StatsVersion uint64 } // NewSimpleStats creates a simple StatsInfo with rowCount. @@ -48,10 +48,10 @@ func (s *StatsInfo) Count() int64 { // Scale receives a selectivity and multiplies it with RowCount and Cardinality. func (s *StatsInfo) Scale(factor float64) *StatsInfo { profile := &StatsInfo{ - RowCount: s.RowCount * factor, - Cardinality: make([]float64, len(s.Cardinality)), - HistColl: s.HistColl, - UsePseudoStats: s.UsePseudoStats, + RowCount: s.RowCount * factor, + Cardinality: make([]float64, len(s.Cardinality)), + HistColl: s.HistColl, + StatsVersion: s.StatsVersion, } for i := range profile.Cardinality { profile.Cardinality[i] = s.Cardinality[i] * factor diff --git a/privilege/privilege.go b/privilege/privilege.go index b6aa136c6c806..332e8229ea93a 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -29,7 +29,7 @@ func (k keyType) String() string { // Manager is the interface for providing privilege related operations. type Manager interface { // ShowGrants shows granted privileges for user. - ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity) ([]string, error) + ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) ([]string, error) // GetEncodedPassword shows the encoded password for user. GetEncodedPassword(user, host string) string @@ -56,6 +56,12 @@ type Manager interface { // ActiveRoles active roles for current session. // The first illegal role will be returned. ActiveRoles(ctx sessionctx.Context, roleList []*auth.RoleIdentity) (bool, string) + + // FindEdge find if there is an edge between role and user. + FindEdge(ctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool + + // GetDefaultRoles returns all default roles for certain user. + GetDefaultRoles(user, host string) []*auth.RoleIdentity } const key keyType = 0 diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 2731f6f39dad1..85121d30440e4 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -68,7 +68,7 @@ type dbRecord struct { User string Privileges mysql.PrivilegeType - // patChars is compiled from Host and DB, cached for pattern match performance. + // hostPatChars is compiled from Host and DB, cached for pattern match performance. hostPatChars []byte hostPatTypes []byte @@ -105,9 +105,21 @@ type columnsPrivRecord struct { patTypes []byte } -// RoleGraphEdgesTable is used to cache relationship between and role. +// defaultRoleRecord is used to cache mysql.default_roles +type defaultRoleRecord struct { + Host string + User string + DefaultRoleUser string + DefaultRoleHost string + + // patChars is compiled from Host, cached for pattern match performance. + patChars []byte + patTypes []byte +} + +// roleGraphEdgesTable is used to cache relationship between and role. type roleGraphEdgesTable struct { - roleList map[string]bool + roleList map[string]*auth.RoleIdentity } // Find method is used to find role from table @@ -125,11 +137,39 @@ func (g roleGraphEdgesTable) Find(user, host string) bool { // MySQLPrivilege is the in-memory cache of mysql privilege tables. type MySQLPrivilege struct { - User []UserRecord - DB []dbRecord - TablesPriv []tablesPrivRecord - ColumnsPriv []columnsPrivRecord - RoleGraph map[string]roleGraphEdgesTable + User []UserRecord + DB []dbRecord + TablesPriv []tablesPrivRecord + ColumnsPriv []columnsPrivRecord + DefaultRoles []defaultRoleRecord + RoleGraph map[string]roleGraphEdgesTable +} + +// FindAllRole is used to find all roles grant to this user. +func (p *MySQLPrivilege) FindAllRole(activeRoles []*auth.RoleIdentity) []*auth.RoleIdentity { + queue, head := make([]*auth.RoleIdentity, 0, len(activeRoles)), 0 + for _, r := range activeRoles { + queue = append(queue, r) + } + // Using breadth first search to find all roles grant to this user. + visited, ret := make(map[string]bool), make([]*auth.RoleIdentity, 0) + for head < len(queue) { + role := queue[head] + if _, ok := visited[role.String()]; !ok { + visited[role.String()] = true + ret = append(ret, role) + key := role.Username + "@" + role.Hostname + if edgeTable, ok := p.RoleGraph[key]; ok { + for _, v := range edgeTable.roleList { + if _, ok := visited[v.String()]; !ok { + queue = append(queue, v) + } + } + } + } + head += 1 + } + return ret } // FindRole is used to detect whether there is edges between users and roles. @@ -166,6 +206,14 @@ func (p *MySQLPrivilege) LoadAll(ctx sessionctx.Context) error { log.Warn("mysql.tables_priv missing") } + err = p.LoadDefaultRoles(ctx) + if err != nil { + if !noSuchTable(err) { + return errors.Trace(err) + } + log.Warn("mysql.default_roles missing") + } + err = p.LoadColumnsPrivTable(ctx) if err != nil { if !noSuchTable(err) { @@ -316,6 +364,11 @@ func (p *MySQLPrivilege) LoadColumnsPrivTable(ctx sessionctx.Context) error { return p.loadTable(ctx, "select HIGH_PRIORITY Host,DB,User,Table_name,Column_name,Timestamp,Column_priv from mysql.columns_priv", p.decodeColumnsPrivTableRow) } +// LoadDefaultRoles loads the mysql.columns_priv table from database. +func (p *MySQLPrivilege) LoadDefaultRoles(ctx sessionctx.Context) error { + return p.loadTable(ctx, "select HOST, USER, DEFAULT_ROLE_HOST, DEFAULT_ROLE_USER from mysql.default_roles", p.decodeDefaultRoleTableRow) +} + func (p *MySQLPrivilege) loadTable(sctx sessionctx.Context, sql string, decodeTableRow func(chunk.Row, []*ast.ResultField) error) error { ctx := context.Background() @@ -448,10 +501,29 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*ast.ResultFie toKey := toUser + "@" + toHost roleGraph, ok := p.RoleGraph[toKey] if !ok { - roleGraph = roleGraphEdgesTable{roleList: make(map[string]bool)} + roleGraph = roleGraphEdgesTable{roleList: make(map[string]*auth.RoleIdentity)} p.RoleGraph[toKey] = roleGraph } - roleGraph.roleList[fromKey] = true + roleGraph.roleList[fromKey] = &auth.RoleIdentity{Username: fromUser, Hostname: fromHost} + return nil +} + +func (p *MySQLPrivilege) decodeDefaultRoleTableRow(row chunk.Row, fs []*ast.ResultField) error { + var value defaultRoleRecord + for i, f := range fs { + switch { + case f.ColumnAsName.L == "host": + value.Host = row.GetString(i) + value.patChars, value.patTypes = stringutil.CompilePattern(value.Host, '\\') + case f.ColumnAsName.L == "user": + value.User = row.GetString(i) + case f.ColumnAsName.L == "default_role_host": + value.DefaultRoleHost = row.GetString(i) + case f.ColumnAsName.L == "default_role_user": + value.DefaultRoleUser = row.GetString(i) + } + } + p.DefaultRoles = append(p.DefaultRoles, value) return nil } @@ -522,6 +594,10 @@ func (record *columnsPrivRecord) match(user, host, db, table, col string) bool { patternMatch(host, record.patChars, record.patTypes) } +func (record *defaultRoleRecord) match(user, host string) bool { + return record.User == user && patternMatch(host, record.patChars, record.patTypes) +} + // patternMatch matches "%" the same way as ".*" in regular expression, for example, // "10.0.%" would match "10.0.1" "10.0.1.118" ... func patternMatch(str string, patChars, patTypes []byte) bool { @@ -651,21 +727,33 @@ func (p *MySQLPrivilege) DBIsVisible(user, host, db string) bool { return false } -func (p *MySQLPrivilege) showGrants(user, host string) []string { +func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentity) []string { var gs []string var hasGlobalGrant bool = false - // Show global grants + // Some privileges may granted from role inheritance. + // We should find these inheritance relationship. + allRoles := p.FindAllRole(roles) + // Show global grants. + var currentPriv mysql.PrivilegeType + var g string for _, record := range p.User { if record.User == user && record.Host == host { hasGlobalGrant = true - g := userPrivToString(record.Privileges) - if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON *.* TO '%s'@'%s'`, g, record.User, record.Host) - gs = append(gs, s) + currentPriv |= record.Privileges + } else { + for _, r := range allRoles { + if record.User == r.Username && record.Host == r.Hostname { + hasGlobalGrant = true + currentPriv |= record.Privileges + } } - break // it's unique } } + g = userPrivToString(currentPriv) + if len(g) > 0 { + s := fmt.Sprintf(`GRANT %s ON *.* TO '%s'@'%s'`, g, user, host) + gs = append(gs, s) + } // This is a mysql convention. if len(gs) == 0 && hasGlobalGrant { @@ -673,28 +761,81 @@ func (p *MySQLPrivilege) showGrants(user, host string) []string { gs = append(gs, s) } - // Show db scope grants + // Show db scope grants. + dbPrivTable := make(map[string]mysql.PrivilegeType) for _, record := range p.DB { if record.User == user && record.Host == host { - g := dbPrivToString(record.Privileges) - if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON %s.* TO '%s'@'%s'`, g, record.DB, record.User, record.Host) - gs = append(gs, s) + if _, ok := dbPrivTable[record.DB]; ok { + dbPrivTable[record.DB] |= record.Privileges + } else { + dbPrivTable[record.DB] = record.Privileges } + } else { + for _, r := range allRoles { + if record.User == r.Username && record.Host == r.Hostname { + if _, ok := dbPrivTable[record.DB]; ok { + dbPrivTable[record.DB] |= record.Privileges + } else { + dbPrivTable[record.DB] = record.Privileges + } + } + } + } + } + for dbName, priv := range dbPrivTable { + g := dbPrivToString(priv) + if len(g) > 0 { + s := fmt.Sprintf(`GRANT %s ON %s.* TO '%s'@'%s'`, g, dbName, user, host) + gs = append(gs, s) } } - // Show table scope grants + // Show table scope grants. + tablePrivTable := make(map[string]mysql.PrivilegeType) for _, record := range p.TablesPriv { + recordKey := record.DB + "." + record.TableName if record.User == user && record.Host == host { - g := tablePrivToString(record.TablePriv) - if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON %s.%s TO '%s'@'%s'`, g, record.DB, record.TableName, record.User, record.Host) - gs = append(gs, s) + if _, ok := dbPrivTable[record.DB]; ok { + tablePrivTable[recordKey] |= record.TablePriv + } else { + tablePrivTable[recordKey] = record.TablePriv } + } else { + for _, r := range allRoles { + if record.User == r.Username && record.Host == r.Hostname { + if _, ok := dbPrivTable[record.DB]; ok { + tablePrivTable[recordKey] |= record.TablePriv + } else { + tablePrivTable[recordKey] = record.TablePriv + } + } + } + } + } + for k, priv := range tablePrivTable { + g := tablePrivToString(priv) + if len(g) > 0 { + s := fmt.Sprintf(`GRANT %s ON %s TO '%s'@'%s'`, g, k, user, host) + gs = append(gs, s) } } + // Show role grants. + graphKey := user + "@" + host + edgeTable, ok := p.RoleGraph[graphKey] + g = "" + if ok { + for k := range edgeTable.roleList { + role := strings.Split(k, "@") + roleName, roleHost := role[0], role[1] + if g != "" { + g += ", " + } + g += fmt.Sprintf("'%s'@'%s'", roleName, roleHost) + } + s := fmt.Sprintf(`GRANT %s TO '%s'@'%s'`, g, user, host) + gs = append(gs, s) + } return gs } @@ -766,6 +907,16 @@ func appendUserPrivilegesTableRow(rows [][]types.Datum, user UserRecord) [][]typ return rows } +func (p *MySQLPrivilege) getDefaultRoles(user, host string) []*auth.RoleIdentity { + ret := make([]*auth.RoleIdentity, 0) + for _, r := range p.DefaultRoles { + if r.match(user, host) { + ret = append(ret, &auth.RoleIdentity{Username: r.DefaultRoleUser, Hostname: r.DefaultRoleHost}) + } + } + return ret +} + // Handle wraps MySQLPrivilege providing thread safe access. type Handle struct { priv atomic.Value diff --git a/privilege/privileges/cache_test.go b/privilege/privileges/cache_test.go index 032b3586ee120..fe9e6c740035e 100644 --- a/privilege/privileges/cache_test.go +++ b/privilege/privileges/cache_test.go @@ -134,6 +134,25 @@ func (s *testCacheSuite) TestLoadColumnsPrivTable(c *C) { c.Assert(p.ColumnsPriv[1].ColumnPriv, Equals, mysql.SelectPriv) } +func (s *testCacheSuite) TestLoadDefaultRoleTable(c *C) { + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + defer se.Close() + mustExec(c, se, "use mysql;") + mustExec(c, se, "truncate table default_roles") + + mustExec(c, se, `INSERT INTO mysql.default_roles VALUES ("%", "test_default_roles", "localhost", "r_1")`) + mustExec(c, se, `INSERT INTO mysql.default_roles VALUES ("%", "test_default_roles", "localhost", "r_2")`) + var p privileges.MySQLPrivilege + err = p.LoadDefaultRoles(se) + c.Assert(err, IsNil) + c.Assert(p.DefaultRoles[0].Host, Equals, `%`) + c.Assert(p.DefaultRoles[0].User, Equals, "test_default_roles") + c.Assert(p.DefaultRoles[0].DefaultRoleHost, Equals, "localhost") + c.Assert(p.DefaultRoles[0].DefaultRoleUser, Equals, "r_1") + c.Assert(p.DefaultRoles[1].DefaultRoleHost, Equals, "localhost") +} + func (s *testCacheSuite) TestPatternMatch(c *C) { se, err := session.CreateSession4Test(s.store) c.Assert(err, IsNil) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index a8d8a6cd7d816..25e35b17027e7 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -175,7 +175,7 @@ func (p *UserPrivileges) UserPrivilegesTable() [][]types.Datum { } // ShowGrants implements privilege.Manager ShowGrants interface. -func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity) (grants []string, err error) { +func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) (grants []string, err error) { mysqlPrivilege := p.Handle.Get() u := user.Username h := user.Hostname @@ -183,7 +183,7 @@ func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdent u = user.AuthUsername h = user.AuthHostname } - grants = mysqlPrivilege.showGrants(u, h) + grants = mysqlPrivilege.showGrants(u, h, roles) if len(grants) == 0 { err = errNonexistingGrant.GenWithStackByArgs(u, h) } @@ -206,3 +206,21 @@ func (p *UserPrivileges) ActiveRoles(ctx sessionctx.Context, roleList []*auth.Ro ctx.GetSessionVars().ActiveRoles = roleList return true, "" } + +// FindEdge implements privilege.Manager FindRelationship interface. +func (p *UserPrivileges) FindEdge(ctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool { + mysqlPrivilege := p.Handle.Get() + ok := mysqlPrivilege.FindRole(user.Username, user.Hostname, role) + if !ok { + logutil.Logger(context.Background()).Error("find role failed", zap.Stringer("role", role)) + return false + } + return true +} + +// GetDefaultRoles returns all default roles for certain user. +func (p *UserPrivileges) GetDefaultRoles(user, host string) []*auth.RoleIdentity { + mysqlPrivilege := p.Handle.Get() + ret := mysqlPrivilege.getDefaultRoles(user, host) + return ret +} diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index faa66d18ea3ed..85dc130203fc6 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -147,34 +147,34 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { mustExec(c, se, `GRANT Index ON *.* TO 'show'@'localhost';`) pc := privilege.GetPrivilegeManager(se) - gs, err := pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err := pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) c.Assert(gs[0], Equals, `GRANT Index ON *.* TO 'show'@'localhost'`) mustExec(c, se, `GRANT Select ON *.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) c.Assert(gs[0], Equals, `GRANT Select,Index ON *.* TO 'show'@'localhost'`) // The order of privs is the same with AllGlobalPrivs mustExec(c, se, `GRANT Update ON *.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) c.Assert(gs[0], Equals, `GRANT Select,Update,Index ON *.* TO 'show'@'localhost'`) // All privileges mustExec(c, se, `GRANT ALL ON *.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) c.Assert(gs[0], Equals, `GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`) // Add db scope privileges mustExec(c, se, `GRANT Select ON test.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 2) expected := []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, @@ -182,7 +182,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { c.Assert(testutil.CompareUnorderedStringSlice(gs, expected), IsTrue) mustExec(c, se, `GRANT Index ON test1.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 3) expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, @@ -191,7 +191,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { c.Assert(testutil.CompareUnorderedStringSlice(gs, expected), IsTrue) mustExec(c, se, `GRANT ALL ON test1.* TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 3) expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, @@ -201,7 +201,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { // Add table scope privileges mustExec(c, se, `GRANT Update ON test.test TO 'show'@'localhost';`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 4) expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, @@ -215,7 +215,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { mustExec(c, se, `REVOKE Select on test.* FROM 'show'@'localhost'`) mustExec(c, se, `REVOKE ALL ON test1.* FROM 'show'@'localhost'`) mustExec(c, se, `REVOKE UPDATE on test.test FROM 'show'@'localhost'`) - gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) c.Assert(gs[0], Equals, `GRANT USAGE ON *.* TO 'show'@'localhost'`) @@ -226,11 +226,51 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { mustExec(c, se, `DROP USER 'show'@'localhost'`) // This should now return an error - _, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}) + _, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, NotNil) // cant show grants for non-existent errNonexistingGrant := terror.ClassPrivilege.New(mysql.ErrNonexistingGrant, mysql.MySQLErrName[mysql.ErrNonexistingGrant]) c.Assert(terror.ErrorEqual(err, errNonexistingGrant), IsTrue) + + // Test SHOW GRANTS with USING roles. + mustExec(c, se, `CREATE ROLE 'r1', 'r2'`) + mustExec(c, se, `GRANT SELECT ON test.* TO 'r1'`) + mustExec(c, se, `GRANT INSERT, UPDATE ON test.* TO 'r2'`) + mustExec(c, se, `CREATE USER 'testrole'@'localhost' IDENTIFIED BY 'u1pass'`) + mustExec(c, se, `GRANT 'r1', 'r2' TO 'testrole'@'localhost'`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, nil) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 2) + roles := make([]*auth.RoleIdentity, 0) + roles = append(roles, &auth.RoleIdentity{Username: "r2", Hostname: "%"}) + mustExec(c, se, `GRANT DELETE ON test.* TO 'testrole'@'localhost'`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, roles) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 3) + roles = append(roles, &auth.RoleIdentity{Username: "r1", Hostname: "%"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, roles) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 3) + mustExec(c, se, `GRANT INSERT, DELETE ON test.test TO 'r2'`) + mustExec(c, se, `GRANT UPDATE ON a.b TO 'testrole'@'localhost'`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, roles) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 5) + mustExec(c, se, `DROP ROLE 'r1', 'r2'`) + mustExec(c, se, `DROP USER 'testrole'@'localhost'`) + mustExec(c, se, `CREATE ROLE 'r1', 'r2'`) + mustExec(c, se, `GRANT SELECT ON test.* TO 'r2'`) + mustExec(c, se, `CREATE USER 'testrole'@'localhost' IDENTIFIED BY 'u1pass'`) + mustExec(c, se, `GRANT 'r1' TO 'testrole'@'localhost'`) + mustExec(c, se, `GRANT 'r2' TO 'r1'`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, nil) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 2) + roles = make([]*auth.RoleIdentity, 0) + roles = append(roles, &auth.RoleIdentity{Username: "r1", Hostname: "%"}) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "testrole", Hostname: "localhost"}, roles) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 3) } func (s *testPrivilegeSuite) TestDropTablePriv(c *C) { @@ -494,6 +534,29 @@ func (s *testPrivilegeSuite) TestGetEncodedPassword(c *C) { c.Assert(pc.GetEncodedPassword("test_encode_u", "localhost"), Equals, "*81F5E21E35407D884A6CD4A731AEBFB6AF209E1B") } +func (s *testPrivilegeSuite) TestDefaultRoles(c *C) { + rootSe := newSession(c, s.store, s.dbName) + mustExec(c, rootSe, `CREATE USER 'testdefault'@'localhost';`) + mustExec(c, rootSe, `CREATE ROLE 'testdefault_r1'@'localhost', 'testdefault_r2'@'localhost';`) + mustExec(c, rootSe, `GRANT 'testdefault_r1'@'localhost', 'testdefault_r2'@'localhost' TO 'testdefault'@'localhost';`) + + se := newSession(c, s.store, s.dbName) + pc := privilege.GetPrivilegeManager(se) + + ret := pc.GetDefaultRoles("testdefault", "localhost") + c.Assert(len(ret), Equals, 0) + + mustExec(c, rootSe, `SET DEFAULT ROLE ALL TO 'testdefault'@'localhost';`) + mustExec(c, rootSe, `flush privileges;`) + ret = pc.GetDefaultRoles("testdefault", "localhost") + c.Assert(len(ret), Equals, 2) + + mustExec(c, rootSe, `SET DEFAULT ROLE NONE TO 'testdefault'@'localhost';`) + mustExec(c, rootSe, `flush privileges;`) + ret = pc.GetDefaultRoles("testdefault", "localhost") + c.Assert(len(ret), Equals, 0) +} + func mustExec(c *C, se session.Session, sql string) { _, err := se.Execute(context.Background(), sql) c.Assert(err, IsNil) diff --git a/server/http_handler.go b/server/http_handler.go index aaf2758c8bae3..3a2db8174d8e4 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -53,6 +53,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/pdapi" log "github.com/sirupsen/logrus" "go.uber.org/zap" ) @@ -78,18 +79,10 @@ const qTableID = "table_id" const qLimit = "limit" const ( - protocol = "http://" headerContentType = "Content-Type" contentTypeJSON = "application/json" - hotRead = "/pd/api/v1/hotspot/regions/read" - hotWrite = "/pd/api/v1/hotspot/regions/write" ) -type kvStore interface { - GetRegionCache() *tikv.RegionCache - SendReq(bo *tikv.Backoffer, req *tikvrpc.Request, regionID tikv.RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) -} - func writeError(w http.ResponseWriter, err error) { w.WriteHeader(http.StatusBadRequest) _, err = w.Write([]byte(err.Error())) @@ -286,7 +279,7 @@ func (t *tikvHandlerTool) getTable(dbName, tableName string) (*model.TableInfo, } func (t *tikvHandlerTool) schema() (infoschema.InfoSchema, error) { - session, err := session.CreateSession(t.Store.(kv.Storage)) + session, err := session.CreateSession(t.Store) if err != nil { return nil, errors.Trace(err) } @@ -301,82 +294,6 @@ func (t *tikvHandlerTool) handleMvccGetByHex(params map[string]string) (interfac return t.GetMvccByEncodedKey(encodedKey) } -func (t *tikvHandlerTool) scrapeHotInfo(rw string) (map[tblIndex]helper.RegionMetric, error) { - regionMetrics, err := t.FetchHotRegion(rw) - if err != nil { - return nil, err - } - - tblIdx, err := t.fetchRegionTableIndex(regionMetrics) - if err != nil { - return nil, err - } - return tblIdx, nil -} - -// tblIndex presents the aggregate key that combined with db,table,index -type tblIndex struct { - DbName string `json:"db_name"` - TableName string `json:"table_name"` - IndexName string `json:"index_name"` -} - -func (t *tikvHandlerTool) fetchRegionTableIndex(metrics map[uint64]helper.RegionMetric) (map[tblIndex]helper.RegionMetric, error) { - schema, err := t.schema() - if err != nil { - return nil, err - } - - idxMetrics := make(map[tblIndex]helper.RegionMetric) - for regionID, regionMetric := range metrics { - region, err := t.RegionCache.LocateRegionByID(tikv.NewBackoffer(context.Background(), 500), regionID) - if err != nil { - logutil.Logger(context.Background()).Error("locate region failed", zap.Error(err)) - continue - } - - hotRange, err := NewRegionFrameRange(region) - if err != nil { - return nil, err - } - - f := t.findTableIndexOfRegion(schema, hotRange) - if f != nil { - idx := tblIndex{DbName: f.DBName, TableName: f.TableName, IndexName: f.IndexName} - metric, exists := idxMetrics[idx] - if !exists { - metric = regionMetric - metric.Count++ - idxMetrics[idx] = metric - } else { - metric.FlowBytes += regionMetric.FlowBytes - if metric.MaxHotDegree < regionMetric.MaxHotDegree { - metric.MaxHotDegree = regionMetric.MaxHotDegree - } - metric.Count++ - } - } - } - - return idxMetrics, nil -} - -func (t *tikvHandlerTool) findTableIndexOfRegion(schema infoschema.InfoSchema, hotRange *RegionFrameRange) *FrameItem { - for _, db := range schema.AllSchemas() { - for _, tbl := range db.Tables { - if f := hotRange.getRecordFrame(tbl.ID, db.Name.O, tbl.Name.O); f != nil { - return f - } - for _, idx := range tbl.Indices { - if f := hotRange.getIndexFrame(tbl.ID, idx.ID, db.Name.O, tbl.Name.O, idx.Name.O); f != nil { - return f - } - } - } - } - return nil -} - // settingsHandler is the handler for list tidb server settings. type settingsHandler struct { } @@ -553,24 +470,24 @@ type IndexRegions struct { // RegionDetail is the response data for get region by ID // it includes indices and records detail in current region. type RegionDetail struct { - RegionID uint64 `json:"region_id"` - StartKey []byte `json:"start_key"` - EndKey []byte `json:"end_key"` - Frames []*FrameItem `json:"frames"` + RegionID uint64 `json:"region_id"` + StartKey []byte `json:"start_key"` + EndKey []byte `json:"end_key"` + Frames []*helper.FrameItem `json:"frames"` } // addTableInRange insert a table into RegionDetail // with index's id or record in the range if r. -func (rt *RegionDetail) addTableInRange(dbName string, curTable *model.TableInfo, r *RegionFrameRange) { +func (rt *RegionDetail) addTableInRange(dbName string, curTable *model.TableInfo, r *helper.RegionFrameRange) { tName := curTable.Name.String() tID := curTable.ID for _, index := range curTable.Indices { - if f := r.getIndexFrame(tID, index.ID, dbName, tName, index.Name.String()); f != nil { + if f := r.GetIndexFrame(tID, index.ID, dbName, tName, index.Name.String()); f != nil { rt.Frames = append(rt.Frames, f) } } - if f := r.getRecordFrame(tID, dbName, tName); f != nil { + if f := r.GetRecordFrame(tID, dbName, tName); f != nil { rt.Frames = append(rt.Frames, f) } } @@ -867,7 +784,7 @@ func (h ddlResignOwnerHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques func (h tableHandler) getPDAddr() ([]string, error) { var pdAddrs []string - etcd, ok := h.Store.(domain.EtcdBackend) + etcd, ok := h.Store.(tikv.EtcdBackend) if !ok { return nil, errors.New("not implemented") } @@ -1069,7 +986,7 @@ func (h tableHandler) handleDiskUsageRequest(schema infoschema.InfoSchema, tbl t } type hotRegion struct { - tblIndex + helper.TblIndex helper.RegionMetric } type hotRegions []hotRegion @@ -1111,17 +1028,22 @@ func (h regionHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } if router == "RegionHot" { - hotRead, err := h.scrapeHotInfo(hotRead) + schema, err := h.schema() if err != nil { writeError(w, err) return } - hotWrite, err := h.scrapeHotInfo(hotWrite) + hotRead, err := h.ScrapeHotInfo(pdapi.HotRead, schema.AllSchemas()) if err != nil { writeError(w, err) return } - asSortedEntry := func(metric map[tblIndex]helper.RegionMetric) hotRegions { + hotWrite, err := h.ScrapeHotInfo(pdapi.HotWrite, schema.AllSchemas()) + if err != nil { + writeError(w, err) + return + } + asSortedEntry := func(metric map[helper.TblIndex]helper.RegionMetric) hotRegions { hs := make(hotRegions, 0, len(metric)) for key, value := range metric { hs = append(hs, hotRegion{key, value}) @@ -1152,7 +1074,7 @@ func (h regionHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } - frameRange, err := NewRegionFrameRange(region) + frameRange, err := helper.NewRegionFrameRange(region) if err != nil { writeError(w, err) return @@ -1225,95 +1147,6 @@ func NewFrameItemFromRegionKey(key []byte) (frame *FrameItem, err error) { return } -// NewRegionFrameRange init a NewRegionFrameRange with region info. -func NewRegionFrameRange(region *tikv.KeyLocation) (idxRange *RegionFrameRange, err error) { - var first, last *FrameItem - // check and init first frame - if len(region.StartKey) > 0 { - first, err = NewFrameItemFromRegionKey(region.StartKey) - if err != nil { - return - } - } else { // empty startKey means start with -infinite - first = &FrameItem{ - IndexID: int64(math.MinInt64), - IsRecord: false, - TableID: int64(math.MinInt64), - } - } - - // check and init last frame - if len(region.EndKey) > 0 { - last, err = NewFrameItemFromRegionKey(region.EndKey) - if err != nil { - return - } - } else { // empty endKey means end with +infinite - last = &FrameItem{ - TableID: int64(math.MaxInt64), - IndexID: int64(math.MaxInt64), - IsRecord: true, - } - } - - idxRange = &RegionFrameRange{ - region: region, - first: first, - last: last, - } - return idxRange, nil -} - -// getRecordFrame returns the record frame of a table. If the table's records -// are not covered by this frame range, it returns nil. -func (r *RegionFrameRange) getRecordFrame(tableID int64, dbName, tableName string) *FrameItem { - if tableID == r.first.TableID && r.first.IsRecord { - r.first.DBName, r.first.TableName = dbName, tableName - return r.first - } - if tableID == r.last.TableID && r.last.IsRecord { - r.last.DBName, r.last.TableName = dbName, tableName - return r.last - } - - if tableID >= r.first.TableID && tableID < r.last.TableID { - return &FrameItem{ - DBName: dbName, - TableName: tableName, - TableID: tableID, - IsRecord: true, - } - } - return nil -} - -// getIndexFrame returns the indnex frame of a table. If the table's indices are -// not covered by this frame range, it returns nil. -func (r *RegionFrameRange) getIndexFrame(tableID, indexID int64, dbName, tableName, indexName string) *FrameItem { - if tableID == r.first.TableID && !r.first.IsRecord && indexID == r.first.IndexID { - r.first.DBName, r.first.TableName, r.first.IndexName = dbName, tableName, indexName - return r.first - } - if tableID == r.last.TableID && indexID == r.last.IndexID { - r.last.DBName, r.last.TableName, r.last.IndexName = dbName, tableName, indexName - return r.last - } - - greaterThanFirst := tableID > r.first.TableID || (tableID == r.first.TableID && !r.first.IsRecord && indexID > r.first.IndexID) - lessThanLast := tableID < r.last.TableID || (tableID == r.last.TableID && (r.last.IsRecord || indexID < r.last.IndexID)) - if greaterThanFirst && lessThanLast { - return &FrameItem{ - DBName: dbName, - TableName: tableName, - TableID: tableID, - IsRecord: false, - IndexName: indexName, - IndexID: indexID, - } - } - return nil -} - // parseQuery is used to parse query string in URL with shouldUnescape, due to golang http package can not distinguish // query like "?a=" and "?a". We rewrite it to separate these two queries. e.g. // "?a=" which means that a is an empty string ""; diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 23dc8efe57ec4..9480924c375c3 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" @@ -91,15 +92,14 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRange(c *C) { StartKey: startKey, EndKey: endKey, } - r, err := NewRegionFrameRange(region) + r, err := helper.NewRegionFrameRange(region) c.Assert(err, IsNil) - c.Assert(r.first.IndexID, Equals, sIndex) - c.Assert(r.first.IsRecord, IsFalse) - c.Assert(r.first.RecordID, Equals, int64(0)) - c.Assert(r.first.IndexValues, DeepEquals, expectIndexValues) - c.Assert(r.last.IsRecord, IsTrue) - c.Assert(r.last.RecordID, Equals, recordID) - c.Assert(r.last.IndexValues, IsNil) + c.Assert(r.First.IndexID, Equals, sIndex) + c.Assert(r.First.IsRecord, IsFalse) + c.Assert(r.First.RecordID, Equals, int64(0)) + c.Assert(r.First.IndexValues, DeepEquals, expectIndexValues) + c.Assert(r.Last.RecordID, Equals, recordID) + c.Assert(r.Last.IndexValues, IsNil) testCases := []struct { tableID int64 @@ -118,11 +118,11 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRange(c *C) { {10, 1, false}, } for _, t := range testCases { - var f *FrameItem + var f *helper.FrameItem if t.indexID == 0 { - f = r.getRecordFrame(t.tableID, "", "") + f = r.GetRecordFrame(t.tableID, "", "") } else { - f = r.getIndexFrame(t.tableID, t.indexID, "", "", "") + f = r.GetIndexFrame(t.tableID, t.indexID, "", "", "") } if t.isCover { c.Assert(f, NotNil) @@ -141,12 +141,12 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRangeWithEndNoLimit(c *C) { StartKey: startKey, EndKey: endKey, } - r, err := NewRegionFrameRange(region) + r, err := helper.NewRegionFrameRange(region) c.Assert(err, IsNil) - c.Assert(r.first.IsRecord, IsTrue) - c.Assert(r.last.IsRecord, IsTrue) - c.Assert(r.getRecordFrame(300, "", ""), NotNil) - c.Assert(r.getIndexFrame(200, 100, "", "", ""), NotNil) + c.Assert(r.First.IsRecord, IsTrue) + c.Assert(r.Last.IsRecord, IsTrue) + c.Assert(r.GetRecordFrame(300, "", ""), NotNil) + c.Assert(r.GetIndexFrame(200, 100, "", "", ""), NotNil) } func (ts *HTTPHandlerTestSuite) TestRegionIndexRangeWithStartNoLimit(c *C) { @@ -158,12 +158,12 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRangeWithStartNoLimit(c *C) { StartKey: startKey, EndKey: endKey, } - r, err := NewRegionFrameRange(region) + r, err := helper.NewRegionFrameRange(region) c.Assert(err, IsNil) - c.Assert(r.first.IsRecord, IsFalse) - c.Assert(r.last.IsRecord, IsTrue) - c.Assert(r.getRecordFrame(3, "", ""), NotNil) - c.Assert(r.getIndexFrame(8, 1, "", "", ""), NotNil) + c.Assert(r.First.IsRecord, IsFalse) + c.Assert(r.Last.IsRecord, IsTrue) + c.Assert(r.GetRecordFrame(3, "", ""), NotNil) + c.Assert(r.GetIndexFrame(8, 1, "", "", ""), NotNil) } func (ts *HTTPHandlerTestSuite) TestRegionsAPI(c *C) { diff --git a/server/server_test.go b/server/server_test.go index 57179c4c091dd..8271c2391b584 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -804,7 +804,10 @@ func runTestShowProcessList(c *C) { func runTestAuth(c *C) { runTests(c, nil, func(dbt *DBTest) { dbt.mustExec(`CREATE USER 'authtest'@'%' IDENTIFIED BY '123';`) + dbt.mustExec(`CREATE ROLE 'authtest_r1'@'%';`) dbt.mustExec(`GRANT ALL on test.* to 'authtest'`) + dbt.mustExec(`GRANT authtest_r1 to 'authtest'`) + dbt.mustExec(`SET DEFAULT ROLE authtest_r1 TO authtest`) dbt.mustExec(`FLUSH PRIVILEGES;`) }) runTests(c, func(config *mysql.Config) { @@ -823,6 +826,21 @@ func runTestAuth(c *C) { c.Assert(err, NotNil, Commentf("Wrong password should be failed")) db.Close() + // Test for loading active roles. + db, err = sql.Open("mysql", getDSN(func(config *mysql.Config) { + config.User = "authtest" + config.Passwd = "123" + })) + c.Assert(err, IsNil) + rows, err := db.Query("select current_role;") + c.Assert(err, IsNil) + c.Assert(rows.Next(), IsTrue) + var outA string + err = rows.Scan(&outA) + c.Assert(err, IsNil) + c.Assert(outA, Equals, "`authtest_r1`@`%`") + db.Close() + // Test login use IP that not exists in mysql.user. runTests(c, nil, func(dbt *DBTest) { dbt.mustExec(`CREATE USER 'authtest2'@'localhost' IDENTIFIED BY '123';`) diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index 292433e7fd8a4..8a7b5ff47113e 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" ) @@ -120,11 +120,11 @@ func (ds *testDumpStatsSuite) prepareData(c *C) { h.HandleDDLEvent(<-h.DDLEventCh()) dbt.mustExec("create index c on test (a, b)") dbt.mustExec("insert test values (1, 's')") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) dbt.mustExec("analyze table test") dbt.mustExec("insert into test(a,b) values (1, 'v'),(3, 'vvv'),(5, 'vv')") is := ds.sh.do.InfoSchema() - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) } diff --git a/session/bootstrap.go b/session/bootstrap.go index e19f2631a6977..2998f53747a5e 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -222,10 +222,11 @@ const ( bind_sql text NOT NULL , default_db text NOT NULL, status text NOT NULL, - create_time timestamp NOT NULL, - update_time timestamp NOT NULL, + create_time timestamp(3) NOT NULL, + update_time timestamp(3) NOT NULL, charset text NOT NULL, collation text NOT NULL, + INDEX sql_index(original_sql(1024),default_db(1024)) COMMENT "accelerate the speed when add global binding query", INDEX time_index(update_time) COMMENT "accelerate the speed when querying with last update time" ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;` @@ -315,6 +316,7 @@ const ( version26 = 26 version27 = 27 version28 = 28 + version29 = 29 ) func checkBootstrapped(s Session) (bool, error) { @@ -486,6 +488,10 @@ func upgrade(s Session) { upgradeToVer28(s) } + if ver == version28 { + upgradeToVer29(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -768,6 +774,12 @@ func upgradeToVer28(s Session) { doReentrantDDL(s, CreateBindInfoTable) } +func upgradeToVer29(s Session) { + doReentrantDDL(s, "ALTER TABLE mysql.bind_info change create_time create_time timestamp(3)") + doReentrantDDL(s, "ALTER TABLE mysql.bind_info change update_time update_time timestamp(3)") + doReentrantDDL(s, "ALTER TABLE mysql.bind_info add index sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. diff --git a/session/session.go b/session/session.go index ecf1ac65d7c5e..4a02acc0c55a4 100644 --- a/session/session.go +++ b/session/session.go @@ -54,7 +54,8 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -152,7 +153,7 @@ type session struct { sessionVars *variable.SessionVars sessionManager util.SessionManager - statsCollector *statistics.SessionStatsCollector + statsCollector *handle.SessionStatsCollector // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker } @@ -1196,6 +1197,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []by user.AuthUsername, user.AuthHostname, success = pm.ConnectionVerification(user.Username, user.Hostname, authentication, salt) if success { s.sessionVars.User = user + s.sessionVars.ActiveRoles = pm.GetDefaultRoles(user.AuthUsername, user.AuthHostname) return true } else if user.Hostname == variable.DefHostname { logutil.Logger(context.Background()).Error("user connection verification failed", @@ -1213,6 +1215,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []by AuthUsername: u, AuthHostname: h, } + s.sessionVars.ActiveRoles = pm.GetDefaultRoles(u, h) return true } } @@ -1265,7 +1268,6 @@ func CreateSession(store kv.Storage) (Session, error) { Handle: do.PrivilegeHandle(), } privilege.BindPrivilegeManager(s, pm) - // Add stats collector, and it will be freed by background stats worker // which periodically updates stats using the collected data. if do.StatsHandle() != nil && do.StatsUpdating() { @@ -1363,7 +1365,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } - if raw, ok := store.(domain.EtcdBackend); ok { + if raw, ok := store.(tikv.EtcdBackend); ok { err = raw.StartGCWorker() if err != nil { return nil, err @@ -1450,7 +1452,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 28 + currentBootstrapVersion = 29 ) func getStoreBootstrapVersion(store kv.Storage) int64 { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index de6cd9fb1857e..d36a8c6e49ca5 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -15,6 +15,7 @@ package stmtctx import ( "math" + "sort" "sync" "time" @@ -95,6 +96,7 @@ type StatementContext struct { warnings []SQLWarn histogramsNotLoad bool execDetails execdetails.ExecDetails + allExecDetails []*execdetails.ExecDetails } // PrevAffectedRows is the affected-rows value(DDL is 0, DML is the number of affected rows). PrevAffectedRows int64 @@ -376,6 +378,8 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.touched = 0 sc.mu.message = "" sc.mu.warnings = nil + sc.mu.execDetails = execdetails.ExecDetails{} + sc.mu.allExecDetails = make([]*execdetails.ExecDetails, 0, 4) sc.mu.Unlock() sc.TableIDs = sc.TableIDs[:0] sc.IndexIDs = sc.IndexIDs[:0] @@ -392,6 +396,7 @@ func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails, c sc.mu.execDetails.RequestCount++ sc.mu.execDetails.TotalKeys += details.TotalKeys sc.mu.execDetails.ProcessedKeys += details.ProcessedKeys + sc.mu.allExecDetails = append(sc.mu.allExecDetails, details) } sc.mu.execDetails.CommitDetail = commitDetails sc.mu.Unlock() @@ -423,3 +428,46 @@ func (sc *StatementContext) ShouldIgnoreOverflowError() bool { } return false } + +// CopTasksDetails returns some useful information of cop-tasks during execution. +func (sc *StatementContext) CopTasksDetails() *CopTasksDetails { + sc.mu.Lock() + defer sc.mu.Unlock() + n := len(sc.mu.allExecDetails) + d := &CopTasksDetails{NumCopTasks: n} + if n == 0 { + return d + } + d.AvgProcessTime = sc.mu.execDetails.ProcessTime / time.Duration(n) + d.AvgWaitTime = sc.mu.execDetails.WaitTime / time.Duration(n) + + sort.Slice(sc.mu.allExecDetails, func(i, j int) bool { + return sc.mu.allExecDetails[i].ProcessTime < sc.mu.allExecDetails[j].ProcessTime + }) + d.P90ProcessTime = sc.mu.allExecDetails[n*9/10].ProcessTime + d.MaxProcessTime = sc.mu.allExecDetails[n-1].ProcessTime + d.MaxProcessAddress = sc.mu.allExecDetails[n-1].CalleeAddress + + sort.Slice(sc.mu.allExecDetails, func(i, j int) bool { + return sc.mu.allExecDetails[i].WaitTime < sc.mu.allExecDetails[j].WaitTime + }) + d.P90WaitTime = sc.mu.allExecDetails[n*9/10].WaitTime + d.MaxWaitTime = sc.mu.allExecDetails[n-1].WaitTime + d.MaxWaitAddress = sc.mu.allExecDetails[n-1].CalleeAddress + return d +} + +//CopTasksDetails collects some useful information of cop-tasks during execution. +type CopTasksDetails struct { + NumCopTasks int + + AvgProcessTime time.Duration + P90ProcessTime time.Duration + MaxProcessAddress string + MaxProcessTime time.Duration + + AvgWaitTime time.Duration + P90WaitTime time.Duration + MaxWaitAddress string + MaxWaitTime time.Duration +} diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go new file mode 100644 index 0000000000000..3ef7fee235459 --- /dev/null +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -0,0 +1,46 @@ +// 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 stmtctx + +import ( + "fmt" + "testing" + "time" + + "github.com/pingcap/tidb/util/execdetails" +) + +func TestCopTasksDetails(t *testing.T) { + ctx := new(StatementContext) + for i := 0; i < 100; i++ { + d := &execdetails.ExecDetails{ + CalleeAddress: fmt.Sprintf("%v", i+1), + ProcessTime: time.Second * time.Duration(i+1), + WaitTime: time.Millisecond * time.Duration(i+1), + } + ctx.MergeExecDetails(d, nil) + } + c := ctx.CopTasksDetails() + if c.NumCopTasks != 100 || + c.AvgProcessTime != time.Second*101/2 || + c.P90ProcessTime != time.Second*91 || + c.MaxProcessTime != time.Second*100 || + c.MaxProcessAddress != "100" || + c.AvgWaitTime != time.Millisecond*101/2 || + c.P90WaitTime != time.Millisecond*91 || + c.MaxWaitTime != time.Millisecond*100 || + c.MaxWaitAddress != "100" { + t.Fatal(c) + } +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 7f7fb1d822088..03104eb888e2c 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -895,6 +895,16 @@ const ( SlowLogDigestStr = "Digest" // SlowLogQuerySQLStr is slow log field name. SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. + // SlowLogStatsInfoStr is plan stats info. + SlowLogStatsInfoStr = "Stats" + // SlowLogNumCopTasksStr is the number of cop-tasks. + SlowLogNumCopTasksStr = "Num_cop_tasks" + // SlowLogCopProcessStr includes some useful information about cop-tasks' process time. + SlowLogCopProcessStr = "Cop_process" + // SlowLogCopWaitStr includes some useful information about cop-tasks' wait time. + SlowLogCopWaitStr = "Cop_wait" + // SlowLogMemMax is the max number bytes of memory used in this statement. + SlowLogMemMax = "Mem_max" ) // SlowLogFormat uses for formatting slow log. @@ -908,8 +918,15 @@ const ( // # DB: test // # Index_ids: [1,2] // # Is_internal: false +// # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +// # Stats: t1:1,t2:2 +// # Num_cop_tasks: 10 +// # Cop_process: Avg_time: 1s P90_time: 2s Max_time: 3s Max_addr: 10.6.131.78 +// # Cop_wait: Avg_time: 10ms P90_time: 20ms Max_time: 30ms Max_Addr: 10.6.131.79 +// # Memory_max: 4096 // select * from t_slim; -func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest, sql string) string { +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest string, + statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, memMax int64, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() buf.WriteString(SlowLogRowPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") @@ -933,6 +950,38 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe if len(digest) > 0 { buf.WriteString(SlowLogRowPrefixStr + SlowLogDigestStr + SlowLogSpaceMarkStr + digest + "\n") } + if len(statsInfos) > 0 { + buf.WriteString(SlowLogRowPrefixStr + SlowLogStatsInfoStr + SlowLogSpaceMarkStr) + firstComma := false + vStr := "" + for k, v := range statsInfos { + if v == 0 { + vStr = "pseudo" + } else { + vStr = strconv.FormatUint(v, 10) + + } + if firstComma { + buf.WriteString("," + k + ":" + vStr) + } else { + buf.WriteString(k + ":" + vStr) + firstComma = true + } + } + buf.WriteString("\n") + } + if copTasks != nil { + buf.WriteString(SlowLogRowPrefixStr + SlowLogNumCopTasksStr + SlowLogSpaceMarkStr + strconv.FormatInt(int64(copTasks.NumCopTasks), 10) + "\n") + buf.WriteString(SlowLogRowPrefixStr + SlowLogCopProcessStr + SlowLogSpaceMarkStr + + fmt.Sprintf("Avg_time: %v P90_time: %v Max_time: %v Max_addr: %v", copTasks.AvgProcessTime, + copTasks.P90ProcessTime, copTasks.MaxProcessTime, copTasks.MaxProcessAddress) + "\n") + buf.WriteString(SlowLogRowPrefixStr + SlowLogCopWaitStr + SlowLogSpaceMarkStr + + fmt.Sprintf("Avg_time: %v P90_time: %v Max_time: %v Max_Addr: %v", copTasks.AvgWaitTime, + copTasks.P90WaitTime, copTasks.MaxWaitTime, copTasks.MaxWaitAddress) + "\n") + } + if memMax > 0 { + buf.WriteString(SlowLogRowPrefixStr + SlowLogMemMax + SlowLogSpaceMarkStr + strconv.FormatInt(memMax, 10) + "\n") + } if len(sql) == 0 { sql = ";" } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 9421bcbd84736..9cec3189c8742 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser" "github.com/pingcap/parser/auth" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" ) @@ -104,6 +105,20 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { TotalKeys: 10000, ProcessedKeys: 20001, } + statsInfos := make(map[string]uint64) + statsInfos["t1"] = 0 + copTasks := &stmtctx.CopTasksDetails{ + NumCopTasks: 10, + AvgProcessTime: time.Second, + P90ProcessTime: time.Second * 2, + MaxProcessAddress: "10.6.131.78", + MaxProcessTime: time.Second * 3, + AvgWaitTime: time.Millisecond * 10, + P90WaitTime: time.Millisecond * 20, + MaxWaitTime: time.Millisecond * 30, + MaxWaitAddress: "10.6.131.79", + } + var memMax int64 = 2333 resultString := `# Txn_start_ts: 406649736972468225 # User: root@192.168.0.1 # Conn_ID: 1 @@ -113,9 +128,14 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # Index_ids: [1,2] # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:pseudo +# Num_cop_tasks: 10 +# Cop_process: Avg_time: 1s P90_time: 2s Max_time: 3s Max_addr: 10.6.131.78 +# Cop_wait: Avg_time: 10ms P90_time: 20ms Max_time: 30ms Max_Addr: 10.6.131.79 +# Mem_max: 2333 select * from t;` sql := "select * from t" digest := parser.DigestHash(sql) - logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, sql) + logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, statsInfos, copTasks, memMax, sql) c.Assert(logString, Equals, resultString) } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 18915ba5df25b..675bb15372115 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -145,7 +145,8 @@ func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch { return c } -func encodeCMSketch(c *CMSketch) ([]byte, error) { +// EncodeCMSketch encodes the given CMSketch to byte slice. +func EncodeCMSketch(c *CMSketch) ([]byte, error) { if c == nil || c.count == 0 { return nil, nil } @@ -153,7 +154,8 @@ func encodeCMSketch(c *CMSketch) ([]byte, error) { return p.Marshal() } -func decodeCMSketch(data []byte) (*CMSketch, error) { +// DecodeCMSketch decode a CMSketch from the given byte slice. +func DecodeCMSketch(data []byte) (*CMSketch, error) { if data == nil { return nil, nil } @@ -191,7 +193,8 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { return true } -func (c *CMSketch) copy() *CMSketch { +// Copy makes a copy for current CMSketch. +func (c *CMSketch) Copy() *CMSketch { if c == nil { return nil } diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index cc64256c3df2a..44e820d0138d0 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -120,10 +120,10 @@ func (s *testStatisticsSuite) TestCMSketchCoding(c *C) { lSketch.table[i][j] = math.MaxUint32 } } - bytes, err := encodeCMSketch(lSketch) + bytes, err := EncodeCMSketch(lSketch) c.Assert(err, IsNil) c.Assert(len(bytes), Equals, 61455) - rSketch, err := decodeCMSketch(bytes) + rSketch, err := DecodeCMSketch(bytes) c.Assert(err, IsNil) c.Assert(lSketch.Equal(rSketch), IsTrue) } diff --git a/statistics/feedback.go b/statistics/feedback.go index d5b26a0d7b12b..32e52c6fab27d 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -17,7 +17,6 @@ import ( "bytes" "context" "encoding/gob" - "fmt" "math" "math/rand" "sort" @@ -40,24 +39,24 @@ import ( "go.uber.org/zap" ) -// feedback represents the total scan count in range [lower, upper). -type feedback struct { - lower *types.Datum - upper *types.Datum - count int64 - repeat int64 +// Feedback represents the total scan count in range [lower, upper). +type Feedback struct { + Lower *types.Datum + Upper *types.Datum + Count int64 + Repeat int64 } // QueryFeedback is used to represent the query feedback info. It contains the query's scan ranges and number of rows // in each range. type QueryFeedback struct { - physicalID int64 - hist *Histogram - tp int - feedback []feedback - expected int64 // expected is the expected scan count of corresponding query. + PhysicalID int64 + Hist *Histogram + Tp int + Feedback []Feedback + Expected int64 // Expected is the Expected scan count of corresponding query. actual int64 // actual is the actual scan count of corresponding query. - valid bool // valid represents the whether this query feedback is still valid. + Valid bool // Valid represents the whether this query feedback is still Valid. desc bool // desc represents the corresponding query is desc scan. } @@ -66,16 +65,16 @@ func NewQueryFeedback(physicalID int64, hist *Histogram, expected int64, desc bo if hist != nil && hist.Len() == 0 { hist = nil } - tp := pkType - if hist != nil && hist.isIndexHist() { - tp = indexType + tp := PkType + if hist != nil && hist.IsIndexHist() { + tp = IndexType } return &QueryFeedback{ - physicalID: physicalID, - valid: true, - tp: tp, - hist: hist, - expected: expected, + PhysicalID: physicalID, + Valid: true, + Tp: tp, + Hist: hist, + Expected: expected, desc: desc, } } @@ -87,12 +86,24 @@ var ( FeedbackProbability = 0.0 ) +// CalcErrorRate calculates the error rate the current QueryFeedback. +func (q *QueryFeedback) CalcErrorRate() float64 { + expected := float64(q.Expected) + if q.actual == 0 { + if expected == 0 { + return 0 + } + return 1 + } + return math.Abs(expected-float64(q.actual)) / float64(q.actual) +} + // CollectFeedback decides whether to collect the feedback. It returns false when: // 1: the histogram is nil or has no buckets; // 2: the number of scan ranges exceeds the limit because it may affect the performance; // 3: it does not pass the probabilistic sampler. func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { - if q.hist == nil || q.hist.Len() == 0 { + if q.Hist == nil || q.Hist.Len() == 0 { return false } if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability { @@ -103,9 +114,9 @@ func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { // DecodeToRanges decode the feedback to ranges. func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { - ranges := make([]*ranger.Range, 0, len(q.feedback)) - for _, val := range q.feedback { - low, high := *val.lower, *val.upper + ranges := make([]*ranger.Range, 0, len(q.Feedback)) + for _, val := range q.Feedback { + low, high := *val.Lower, *val.Upper var lowVal, highVal []types.Datum if isIndex { var err error @@ -119,11 +130,11 @@ func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { return nil, errors.Trace(err) } } else { - _, lowInt, err := codec.DecodeInt(val.lower.GetBytes()) + _, lowInt, err := codec.DecodeInt(val.Lower.GetBytes()) if err != nil { return nil, errors.Trace(err) } - _, highInt, err := codec.DecodeInt(val.upper.GetBytes()) + _, highInt, err := codec.DecodeInt(val.Upper.GetBytes()) if err != nil { return nil, errors.Trace(err) } @@ -139,55 +150,51 @@ func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { return ranges, nil } -func (q *QueryFeedback) decodeIntValues() *QueryFeedback { +// DecodeIntValues is called when the current Feedback stores encoded int values. +func (q *QueryFeedback) DecodeIntValues() *QueryFeedback { nq := &QueryFeedback{} - nq.feedback = make([]feedback, 0, len(q.feedback)) - for _, fb := range q.feedback { - _, lowInt, err := codec.DecodeInt(fb.lower.GetBytes()) + nq.Feedback = make([]Feedback, 0, len(q.Feedback)) + for _, fb := range q.Feedback { + _, lowInt, err := codec.DecodeInt(fb.Lower.GetBytes()) if err != nil { - logutil.Logger(context.Background()).Debug("decode feedback lower bound value to integer failed", zap.Binary("value", fb.lower.GetBytes()), zap.Error(err)) + logutil.Logger(context.Background()).Debug("decode feedback lower bound value to integer failed", zap.Binary("value", fb.Lower.GetBytes()), zap.Error(err)) continue } - _, highInt, err := codec.DecodeInt(fb.upper.GetBytes()) + _, highInt, err := codec.DecodeInt(fb.Upper.GetBytes()) if err != nil { - logutil.Logger(context.Background()).Debug("decode feedback upper bound value to integer failed", zap.Binary("value", fb.upper.GetBytes()), zap.Error(err)) + logutil.Logger(context.Background()).Debug("decode feedback upper bound value to integer failed", zap.Binary("value", fb.Upper.GetBytes()), zap.Error(err)) continue } low, high := types.NewIntDatum(lowInt), types.NewIntDatum(highInt) - nq.feedback = append(nq.feedback, feedback{lower: &low, upper: &high, count: fb.count}) + nq.Feedback = append(nq.Feedback, Feedback{Lower: &low, Upper: &high, Count: fb.Count}) } return nq } // StoreRanges stores the ranges for update. func (q *QueryFeedback) StoreRanges(ranges []*ranger.Range) { - q.feedback = make([]feedback, 0, len(ranges)) + q.Feedback = make([]Feedback, 0, len(ranges)) for _, ran := range ranges { - q.feedback = append(q.feedback, feedback{&ran.LowVal[0], &ran.HighVal[0], 0, 0}) + q.Feedback = append(q.Feedback, Feedback{&ran.LowVal[0], &ran.HighVal[0], 0, 0}) } } // Invalidate is used to invalidate the query feedback. func (q *QueryFeedback) Invalidate() { - q.feedback = nil - q.hist = nil - q.valid = false + q.Feedback = nil + q.Hist = nil + q.Valid = false q.actual = -1 } // Actual gets the actual row count. func (q *QueryFeedback) Actual() int64 { - if !q.valid { + if !q.Valid { return -1 } return q.actual } -// Hist gets the histogram. -func (q *QueryFeedback) Hist() *Histogram { - return q.hist -} - // Update updates the query feedback. `startKey` is the start scan key of the partial result, used to find // the range for update. `counts` is the scan counts of each range, used to update the feedback count info. func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { @@ -202,18 +209,18 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { } metrics.DistSQLScanKeysPartialHistogram.Observe(float64(sum)) q.actual += sum - if !q.valid || q.hist == nil { + if !q.Valid || q.Hist == nil { return } - if q.tp == indexType { + if q.Tp == IndexType { startKey = tablecodec.CutIndexPrefix(startKey) } else { startKey = tablecodec.CutRowKeyPrefix(startKey) } // Find the range that startKey falls in. - idx := sort.Search(len(q.feedback), func(i int) bool { - return bytes.Compare(q.feedback[i].lower.GetBytes(), startKey) > 0 + idx := sort.Search(len(q.Feedback), func(i int) bool { + return bytes.Compare(q.Feedback[i].Lower.GetBytes(), startKey) > 0 }) idx-- if idx < 0 { @@ -228,18 +235,18 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { } // Update the feedback count info. for i, count := range counts { - if i+idx >= len(q.feedback) { + if i+idx >= len(q.Feedback) { q.Invalidate() break } - q.feedback[i+idx].count += count + q.Feedback[i+idx].Count += count } return } // BucketFeedback stands for all the feedback for a bucket. type BucketFeedback struct { - feedback []feedback // All the feedback info in the same bucket. + feedback []Feedback // All the feedback info in the same bucket. lower *types.Datum // The lower bound of the new bucket. upper *types.Datum // The upper bound of the new bucket. } @@ -265,8 +272,8 @@ func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, // adjustFeedbackBoundaries adjust the feedback boundaries according to the `min` and `max`. // If the feedback has no intersection with `min` and `max`, we could just skip this feedback. -func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) { - result, err := outOfRange(sc, min, max, f.lower) +func (f *Feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) { + result, err := outOfRange(sc, min, max, f.Lower) if err != nil { return false, err } @@ -274,9 +281,9 @@ func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, m return true, nil } if result < 0 { - f.lower = min + f.Lower = min } - result, err = outOfRange(sc, min, max, f.upper) + result, err = outOfRange(sc, min, max, f.Upper) if err != nil { return false, err } @@ -284,7 +291,7 @@ func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, m return true, nil } if result > 0 { - f.upper = max + f.Upper = max } return false, nil } @@ -292,13 +299,13 @@ func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, m // buildBucketFeedback build the feedback for each bucket from the histogram feedback. func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) { bktID2FB := make(map[int]*BucketFeedback) - if len(feedback.feedback) == 0 { + if len(feedback.Feedback) == 0 { return bktID2FB, 0 } total := 0 sc := &stmtctx.StatementContext{TimeZone: time.UTC} - min, max := getMinValue(h.Tp), getMaxValue(h.Tp) - for _, fb := range feedback.feedback { + min, max := GetMinValue(h.Tp), GetMaxValue(h.Tp) + for _, fb := range feedback.Feedback { skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) if err != nil { logutil.Logger(context.Background()).Debug("adjust feedback boundaries failed", zap.Error(err)) @@ -307,7 +314,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket if skip { continue } - idx, _ := h.Bounds.LowerBound(0, fb.lower) + idx, _ := h.Bounds.LowerBound(0, fb.Lower) bktIdx := 0 // The last bucket also stores the feedback that falls outside the upper bound. if idx >= h.Bounds.NumRows()-2 { @@ -315,7 +322,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } else { bktIdx = idx / 2 // Make sure that this feedback lies within the bucket. - if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, fb.upper) < 0 { + if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, fb.Upper) < 0 { continue } } @@ -327,21 +334,21 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } bkt.feedback = append(bkt.feedback, fb) // Update the bound if necessary. - res, err := bkt.lower.CompareDatum(nil, fb.lower) + res, err := bkt.lower.CompareDatum(nil, fb.Lower) if err != nil { - logutil.Logger(context.Background()).Debug("compare datum failed", zap.Any("value1", bkt.lower), zap.Any("value2", fb.lower), zap.Error(err)) + logutil.Logger(context.Background()).Debug("compare datum failed", zap.Any("value1", bkt.lower), zap.Any("value2", fb.Lower), zap.Error(err)) continue } if res > 0 { - bkt.lower = fb.lower + bkt.lower = fb.Lower } - res, err = bkt.upper.CompareDatum(nil, fb.upper) + res, err = bkt.upper.CompareDatum(nil, fb.Upper) if err != nil { - logutil.Logger(context.Background()).Debug("compare datum failed", zap.Any("value1", bkt.upper), zap.Any("value2", fb.upper), zap.Error(err)) + logutil.Logger(context.Background()).Debug("compare datum failed", zap.Any("value1", bkt.upper), zap.Any("value2", fb.Upper), zap.Error(err)) continue } if res < 0 { - bkt.upper = fb.upper + bkt.upper = fb.Upper } } return bktID2FB, total @@ -352,7 +359,7 @@ func (b *BucketFeedback) getBoundaries(num int) []types.Datum { // Get all the possible new boundaries. vals := make([]types.Datum, 0, len(b.feedback)*2+2) for _, fb := range b.feedback { - vals = append(vals, *fb.lower, *fb.upper) + vals = append(vals, *fb.Lower, *fb.Upper) } vals = append(vals, *b.lower) err := types.SortDatums(nil, vals) @@ -389,7 +396,7 @@ func (b *BucketFeedback) getBoundaries(num int) []types.Datum { // There are only two types of datum in bucket: one is `Blob`, which is for index; the other one // is `Int`, which is for primary key. -type bucket = feedback +type bucket = Feedback // splitBucket firstly splits this "BucketFeedback" to "newNumBkts" new buckets, // calculates the count for each new bucket, merge the new bucket whose count @@ -402,7 +409,7 @@ func (b *BucketFeedback) splitBucket(newNumBkts int, totalCount float64, originB for i := 1; i < len(bounds); i++ { newBkt := bucket{&bounds[i-1], bounds[i].Copy(), 0, 0} // get bucket count - _, ratio := getOverlapFraction(feedback{b.lower, b.upper, int64(originBucketCount), 0}, newBkt) + _, ratio := getOverlapFraction(Feedback{b.lower, b.upper, int64(originBucketCount), 0}, newBkt) countInNewBkt := originBucketCount * ratio countInNewBkt = b.refineBucketCount(newBkt, countInNewBkt) // do not split if the count of result bucket is too small. @@ -410,7 +417,7 @@ func (b *BucketFeedback) splitBucket(newNumBkts int, totalCount float64, originB bounds[i] = bounds[i-1] continue } - newBkt.count = int64(countInNewBkt) + newBkt.Count = int64(countInNewBkt) bkts = append(bkts, newBkt) // To guarantee that each bucket's range will not overlap. setNextValue(&bounds[i]) @@ -420,19 +427,19 @@ func (b *BucketFeedback) splitBucket(newNumBkts int, totalCount float64, originB // getOverlapFraction gets the overlap fraction of feedback and bucket range. In order to get the bucket count, it also // returns the ratio between bucket fraction and feedback fraction. -func getOverlapFraction(fb feedback, bkt bucket) (float64, float64) { +func getOverlapFraction(fb Feedback, bkt bucket) (float64, float64) { datums := make([]types.Datum, 0, 4) - datums = append(datums, *fb.lower, *fb.upper) - datums = append(datums, *bkt.lower, *bkt.upper) + datums = append(datums, *fb.Lower, *fb.Upper) + datums = append(datums, *bkt.Lower, *bkt.Upper) err := types.SortDatums(nil, datums) if err != nil { return 0, 0 } minValue, maxValue := &datums[0], &datums[3] - fbLower := calcFraction4Datums(minValue, maxValue, fb.lower) - fbUpper := calcFraction4Datums(minValue, maxValue, fb.upper) - bktLower := calcFraction4Datums(minValue, maxValue, bkt.lower) - bktUpper := calcFraction4Datums(minValue, maxValue, bkt.upper) + fbLower := calcFraction4Datums(minValue, maxValue, fb.Lower) + fbUpper := calcFraction4Datums(minValue, maxValue, fb.Upper) + bktLower := calcFraction4Datums(minValue, maxValue, bkt.Lower) + bktUpper := calcFraction4Datums(minValue, maxValue, bkt.Upper) ratio := (bktUpper - bktLower) / (fbUpper - fbLower) // full overlap if fbLower <= bktLower && bktUpper <= fbUpper { @@ -456,7 +463,7 @@ func (b *BucketFeedback) refineBucketCount(bkt bucket, defaultCount float64) flo // choose the max overlap fraction if fraction > bestFraction { bestFraction = fraction - count = float64(fb.count) * ratio + count = float64(fb.Count) * ratio } } return count @@ -497,16 +504,16 @@ const ( // getBucketScore gets the score for merge this bucket with previous one. // TODO: We also need to consider the bucket hit count. func getBucketScore(bkts []bucket, totalCount float64, id int) bucketScore { - preCount, count := float64(bkts[id-1].count), float64(bkts[id].count) + preCount, count := float64(bkts[id-1].Count), float64(bkts[id].Count) // do not merge if the result bucket is too large if (preCount + count) > maxBucketFraction*totalCount { return bucketScore{id, math.MaxFloat64} } - // merge them if the result bucket is already too small. + // Merge them if the result bucket is already too small. if (preCount + count) < minBucketFraction*totalCount { return bucketScore{id, 0} } - low, mid, high := bkts[id-1].lower, bkts[id-1].upper, bkts[id].upper + low, mid, high := bkts[id-1].Lower, bkts[id-1].Upper, bkts[id].Upper // If we choose to merge, err is the absolute estimate error for the previous bucket. err := calcFraction4Datums(low, high, mid)*(preCount+count) - preCount return bucketScore{id, math.Abs(err / (preCount + count))} @@ -537,9 +544,9 @@ func mergeBuckets(bkts []bucket, isNewBuckets []bool, totalCount float64) []buck for i := range bkts { // Merge this bucket with last one. if idCursor < mergeCount && ids[idCursor] == i { - bkts[bktCursor-1].upper = bkts[i].upper - bkts[bktCursor-1].count += bkts[i].count - bkts[bktCursor-1].repeat = bkts[i].repeat + bkts[bktCursor-1].Upper = bkts[i].Upper + bkts[bktCursor-1].Count += bkts[i].Count + bkts[bktCursor-1].Repeat = bkts[i].Repeat idCursor++ } else { bkts[bktCursor] = bkts[i] @@ -566,7 +573,7 @@ func splitBuckets(h *Histogram, feedback *QueryFeedback) ([]bucket, []bool, int6 } // Distribute the total split count to bucket based on number of bucket feedback. newBktNums := splitCount * len(bktFB.feedback) / numTotalFBs - bkts := bktFB.splitBucket(newBktNums, h.totalRowCount(), float64(h.bucketCount(i))) + bkts := bktFB.splitBucket(newBktNums, h.TotalRowCount(), float64(h.bucketCount(i))) buckets = append(buckets, bkts...) if len(bkts) == 1 { isNewBuckets = append(isNewBuckets, false) @@ -578,7 +585,7 @@ func splitBuckets(h *Histogram, feedback *QueryFeedback) ([]bucket, []bool, int6 } totCount := int64(0) for _, bkt := range buckets { - totCount += bkt.count + totCount += bkt.Count } return buckets, isNewBuckets, totCount } @@ -589,21 +596,21 @@ func UpdateHistogram(h *Histogram, feedback *QueryFeedback) *Histogram { buckets = mergeBuckets(buckets, isNewBuckets, float64(totalCount)) hist := buildNewHistogram(h, buckets) // Update the NDV of primary key column. - if feedback.tp == pkType { - hist.NDV = int64(hist.totalRowCount()) + if feedback.Tp == PkType { + hist.NDV = int64(hist.TotalRowCount()) } return hist } // UpdateCMSketch updates the CMSketch by feedback. -func UpdateCMSketch(c *CMSketch, eqFeedbacks []feedback) *CMSketch { +func UpdateCMSketch(c *CMSketch, eqFeedbacks []Feedback) *CMSketch { if c == nil || len(eqFeedbacks) == 0 { return c } - newCMSketch := c.copy() + newCMSketch := c.Copy() for _, fb := range eqFeedbacks { - h1, h2 := murmur3.Sum128(fb.lower.GetBytes()) - newCMSketch.setValue(h1, h2, uint32(fb.count)) + h1, h2 := murmur3.Sum128(fb.Lower.GetBytes()) + newCMSketch.setValue(h1, h2, uint32(fb.Count)) } return newCMSketch } @@ -612,8 +619,8 @@ func buildNewHistogram(h *Histogram, buckets []bucket) *Histogram { hist := NewHistogram(h.ID, h.NDV, h.NullCount, h.LastUpdateVersion, h.Tp, len(buckets), h.TotColSize) preCount := int64(0) for _, bkt := range buckets { - hist.AppendBucket(bkt.lower, bkt.upper, bkt.count+preCount, bkt.repeat) - preCount += bkt.count + hist.AppendBucket(bkt.Lower, bkt.Upper, bkt.Count+preCount, bkt.Repeat) + preCount += bkt.Count } return hist } @@ -632,21 +639,21 @@ type queryFeedback struct { func encodePKFeedback(q *QueryFeedback) (*queryFeedback, error) { pb := &queryFeedback{} - for _, fb := range q.feedback { + for _, fb := range q.Feedback { // There is no need to update the point queries. - if bytes.Compare(kv.Key(fb.lower.GetBytes()).PrefixNext(), fb.upper.GetBytes()) >= 0 { + if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { continue } - _, low, err := codec.DecodeInt(fb.lower.GetBytes()) + _, low, err := codec.DecodeInt(fb.Lower.GetBytes()) if err != nil { return nil, errors.Trace(err) } - _, high, err := codec.DecodeInt(fb.upper.GetBytes()) + _, high, err := codec.DecodeInt(fb.Upper.GetBytes()) if err != nil { return nil, errors.Trace(err) } pb.IntRanges = append(pb.IntRanges, low, high) - pb.Counts = append(pb.Counts, fb.count) + pb.Counts = append(pb.Counts, fb.Count) } return pb, nil } @@ -654,14 +661,14 @@ func encodePKFeedback(q *QueryFeedback) (*queryFeedback, error) { func encodeIndexFeedback(q *QueryFeedback) *queryFeedback { pb := &queryFeedback{} var pointCounts []int64 - for _, fb := range q.feedback { - if bytes.Compare(kv.Key(fb.lower.GetBytes()).PrefixNext(), fb.upper.GetBytes()) >= 0 { - h1, h2 := murmur3.Sum128(fb.lower.GetBytes()) + for _, fb := range q.Feedback { + if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { + h1, h2 := murmur3.Sum128(fb.Lower.GetBytes()) pb.HashValues = append(pb.HashValues, h1, h2) - pointCounts = append(pointCounts, fb.count) + pointCounts = append(pointCounts, fb.Count) } else { - pb.IndexRanges = append(pb.IndexRanges, fb.lower.GetBytes(), fb.upper.GetBytes()) - pb.Counts = append(pb.Counts, fb.count) + pb.IndexRanges = append(pb.IndexRanges, fb.Lower.GetBytes(), fb.Upper.GetBytes()) + pb.Counts = append(pb.Counts, fb.Count) } } pb.Counts = append(pb.Counts, pointCounts...) @@ -671,30 +678,31 @@ func encodeIndexFeedback(q *QueryFeedback) *queryFeedback { func encodeColumnFeedback(q *QueryFeedback) (*queryFeedback, error) { pb := &queryFeedback{} sc := stmtctx.StatementContext{TimeZone: time.UTC} - for _, fb := range q.feedback { - lowerBytes, err := codec.EncodeKey(&sc, nil, *fb.lower) + for _, fb := range q.Feedback { + lowerBytes, err := codec.EncodeKey(&sc, nil, *fb.Lower) if err != nil { return nil, errors.Trace(err) } - upperBytes, err := codec.EncodeKey(&sc, nil, *fb.upper) + upperBytes, err := codec.EncodeKey(&sc, nil, *fb.Upper) if err != nil { return nil, errors.Trace(err) } pb.ColumnRanges = append(pb.ColumnRanges, lowerBytes, upperBytes) - pb.Counts = append(pb.Counts, fb.count) + pb.Counts = append(pb.Counts, fb.Count) } return pb, nil } -func encodeFeedback(q *QueryFeedback) ([]byte, error) { +// EncodeFeedback encodes the given feedback to byte slice. +func EncodeFeedback(q *QueryFeedback) ([]byte, error) { var pb *queryFeedback var err error - switch q.tp { - case pkType: + switch q.Tp { + case PkType: pb, err = encodePKFeedback(q) - case indexType: + case IndexType: pb = encodeIndexFeedback(q) - case colType: + case ColType: pb, err = encodeColumnFeedback(q) } if err != nil { @@ -710,11 +718,11 @@ func encodeFeedback(q *QueryFeedback) ([]byte, error) { } func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch) { - q.tp = indexType + q.Tp = IndexType // decode the index range feedback for i := 0; i < len(pb.IndexRanges); i += 2 { lower, upper := types.NewBytesDatum(pb.IndexRanges[i]), types.NewBytesDatum(pb.IndexRanges[i+1]) - q.feedback = append(q.feedback, feedback{&lower, &upper, pb.Counts[i/2], 0}) + q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0}) } if c != nil { // decode the index point feedback, just set value count in CM Sketch @@ -726,7 +734,7 @@ func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch) { } func decodeFeedbackForPK(q *QueryFeedback, pb *queryFeedback, isUnsigned bool) { - q.tp = pkType + q.Tp = PkType // decode feedback for primary key for i := 0; i < len(pb.IntRanges); i += 2 { var lower, upper types.Datum @@ -737,12 +745,12 @@ func decodeFeedbackForPK(q *QueryFeedback, pb *queryFeedback, isUnsigned bool) { lower.SetInt64(pb.IntRanges[i]) upper.SetInt64(pb.IntRanges[i+1]) } - q.feedback = append(q.feedback, feedback{&lower, &upper, pb.Counts[i/2], 0}) + q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0}) } } func decodeFeedbackForColumn(q *QueryFeedback, pb *queryFeedback) error { - q.tp = colType + q.Tp = ColType for i := 0; i < len(pb.ColumnRanges); i += 2 { low, err := codec.DecodeRange(pb.ColumnRanges[i], 1) if err != nil { @@ -752,12 +760,13 @@ func decodeFeedbackForColumn(q *QueryFeedback, pb *queryFeedback) error { if err != nil { return errors.Trace(err) } - q.feedback = append(q.feedback, feedback{&low[0], &high[0], pb.Counts[i/2], 0}) + q.Feedback = append(q.Feedback, Feedback{&low[0], &high[0], pb.Counts[i/2], 0}) } return nil } -func decodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, isUnsigned bool) error { +// DecodeFeedback decodes a byte slice to feedback. +func DecodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, isUnsigned bool) error { buf := bytes.NewBuffer(val) dec := gob.NewDecoder(buf) pb := &queryFeedback{} @@ -780,26 +789,26 @@ func decodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, isUnsigned bool) // Equal tests if two query feedback equal, it is only used in test. func (q *QueryFeedback) Equal(rq *QueryFeedback) bool { - if len(q.feedback) != len(rq.feedback) { + if len(q.Feedback) != len(rq.Feedback) { return false } - for i, fb := range q.feedback { - rfb := rq.feedback[i] - if fb.count != rfb.count { + for i, fb := range q.Feedback { + rfb := rq.Feedback[i] + if fb.Count != rfb.Count { return false } - if fb.lower.Kind() == types.KindInt64 { - if fb.lower.GetInt64() != rfb.lower.GetInt64() { + if fb.Lower.Kind() == types.KindInt64 { + if fb.Lower.GetInt64() != rfb.Lower.GetInt64() { return false } - if fb.upper.GetInt64() != rfb.upper.GetInt64() { + if fb.Upper.GetInt64() != rfb.Upper.GetInt64() { return false } } else { - if !bytes.Equal(fb.lower.GetBytes(), rfb.lower.GetBytes()) { + if !bytes.Equal(fb.Lower.GetBytes(), rfb.Lower.GetBytes()) { return false } - if !bytes.Equal(fb.upper.GetBytes(), rfb.upper.GetBytes()) { + if !bytes.Equal(fb.Upper.GetBytes(), rfb.Upper.GetBytes()) { return false } } @@ -807,53 +816,12 @@ func (q *QueryFeedback) Equal(rq *QueryFeedback) bool { return true } -// recalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. -func (q *QueryFeedback) recalculateExpectCount(h *Handle) error { - t, ok := h.statsCache.Load().(statsCache)[q.physicalID] - if !ok { - return nil - } - tablePseudo := t.Pseudo || t.IsOutdated() - if !tablePseudo { - return nil - } - isIndex := q.hist.Tp.Tp == mysql.TypeBlob - id := q.hist.ID - if isIndex && (t.Indices[id] == nil || !t.Indices[id].NotAccurate()) { - return nil - } - if !isIndex && (t.Columns[id] == nil || !t.Columns[id].NotAccurate()) { - return nil - } - - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - return errors.Trace(err) - } - expected := 0.0 - if isIndex { - idx := t.Indices[id] - expected, err = idx.getRowCount(sc, ranges, t.ModifyCount) - expected *= idx.getIncreaseFactor(t.Count) - } else { - c := t.Columns[id] - expected, err = c.getColumnRowCount(sc, ranges, t.ModifyCount) - expected *= c.getIncreaseFactor(t.Count) - } - if err != nil { - return errors.Trace(err) - } - q.expected = int64(expected) - return nil -} - -// splitFeedbackByQueryType splits the feedbacks into equality feedbacks and range feedbacks. -func splitFeedbackByQueryType(feedbacks []feedback) ([]feedback, []feedback) { - var eqFB, ranFB []feedback +// SplitFeedbackByQueryType splits the feedbacks into equality feedbacks and range feedbacks. +func SplitFeedbackByQueryType(feedbacks []Feedback) ([]Feedback, []Feedback) { + var eqFB, ranFB []Feedback for _, fb := range feedbacks { // Use `>=` here because sometimes the lower is equal to upper. - if bytes.Compare(kv.Key(fb.lower.GetBytes()).PrefixNext(), fb.upper.GetBytes()) >= 0 { + if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { eqFB = append(eqFB, fb) } else { ranFB = append(ranFB, fb) @@ -862,272 +830,6 @@ func splitFeedbackByQueryType(feedbacks []feedback) ([]feedback, []feedback) { return eqFB, ranFB } -// formatBuckets formats bucket from lowBkt to highBkt. -func formatBuckets(hg *Histogram, lowBkt, highBkt, idxCols int) string { - if lowBkt == highBkt { - return hg.bucketToString(lowBkt, idxCols) - } - if lowBkt+1 == highBkt { - return fmt.Sprintf("%s, %s", hg.bucketToString(lowBkt, 0), hg.bucketToString(highBkt, 0)) - } - // do not care the middle buckets - return fmt.Sprintf("%s, (%d buckets, total count %d), %s", hg.bucketToString(lowBkt, 0), - highBkt-lowBkt-1, hg.Buckets[highBkt-1].Count-hg.Buckets[lowBkt].Count, hg.bucketToString(highBkt, 0)) -} - -func colRangeToStr(c *Column, ran *ranger.Range, actual int64, factor float64) string { - lowCount, lowBkt := c.lessRowCountWithBktIdx(ran.LowVal[0]) - highCount, highBkt := c.lessRowCountWithBktIdx(ran.HighVal[0]) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, buckets: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&c.Histogram, lowBkt, highBkt, 0)) -} - -func logForPK(prefix string, c *Column, ranges []*ranger.Range, actual []int64, factor float64) { - for i, ran := range ranges { - if ran.LowVal[0].GetInt64()+1 >= ran.HighVal[0].GetInt64() { - continue - } - logutil.Logger(context.Background()).Debug(prefix, zap.String("column", c.Info.Name.O), zap.String("rangeStr", colRangeToStr(c, ran, actual[i], factor))) - } -} - -func logForIndexRange(idx *Index, ran *ranger.Range, actual int64, factor float64) string { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - lb, err := codec.EncodeKey(sc, nil, ran.LowVal...) - if err != nil { - return "" - } - rb, err := codec.EncodeKey(sc, nil, ran.HighVal...) - if err != nil { - return "" - } - if idx.CMSketch != nil && bytes.Compare(kv.Key(lb).PrefixNext(), rb) >= 0 { - str, err := types.DatumsToString(ran.LowVal, true) - if err != nil { - return "" - } - return fmt.Sprintf("value: %s, actual: %d, expected: %d", str, actual, int64(float64(idx.QueryBytes(lb))*factor)) - } - l, r := types.NewBytesDatum(lb), types.NewBytesDatum(rb) - lowCount, lowBkt := idx.lessRowCountWithBktIdx(l) - highCount, highBkt := idx.lessRowCountWithBktIdx(r) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, histogram: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&idx.Histogram, lowBkt, highBkt, len(idx.Info.Columns))) -} - -func logForIndex(prefix string, t *Table, idx *Index, ranges []*ranger.Range, actual []int64, factor float64) { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - if idx.CMSketch == nil || idx.statsVer != version1 { - for i, ran := range ranges { - logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - } - return - } - for i, ran := range ranges { - rangePosition := getOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - continue - } - equalityString, err := types.DatumsToString(ran.LowVal[:rangePosition], true) - if err != nil { - continue - } - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - continue - } - equalityCount := idx.CMSketch.QueryBytes(bytes) - rang := ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - } - colName := idx.Info.Columns[rangePosition].Name.L - // prefer index stats over column stats - if idxHist := t.indexStartWithColumn(colName); idxHist != nil && idxHist.Histogram.Len() > 0 { - rangeString := logForIndexRange(idxHist, &rang, -1, factor) - logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), - zap.String("range", rangeString)) - } else if colHist := t.columnByName(colName); colHist != nil && colHist.Histogram.Len() > 0 { - rangeString := colRangeToStr(colHist, &rang, -1, factor) - logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), - zap.String("range", rangeString)) - } else { - count, err := getPseudoRowCountByColumnRanges(sc, float64(t.Count), []*ranger.Range{&rang}, 0) - if err == nil { - logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), - zap.Stringer("range", &rang), zap.Float64("pseudo count", math.Round(count))) - } - } - } -} - -func (q *QueryFeedback) logDetailedInfo(h *Handle) { - t, ok := h.statsCache.Load().(statsCache)[q.physicalID] - if !ok { - return - } - isIndex := q.hist.isIndexHist() - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - logutil.Logger(context.Background()).Debug("decode to ranges failed", zap.Error(err)) - return - } - actual := make([]int64, 0, len(q.feedback)) - for _, fb := range q.feedback { - actual = append(actual, fb.count) - } - logPrefix := fmt.Sprintf("[stats-feedback] %s", t.name) - if isIndex { - idx := t.Indices[q.hist.ID] - if idx == nil || idx.Histogram.Len() == 0 { - return - } - logForIndex(logPrefix, t, idx, ranges, actual, idx.getIncreaseFactor(t.Count)) - } else { - c := t.Columns[q.hist.ID] - if c == nil || c.Histogram.Len() == 0 { - return - } - logForPK(logPrefix, c, ranges, actual, c.getIncreaseFactor(t.Count)) - } -} - -// minAdjustFactor is the minimum adjust factor of each index feedback. -// We use it to avoid adjusting too much when the assumption of independence failed. -const minAdjustFactor = 0.7 - -// getNewCountForIndex adjust the estimated `eqCount` and `rangeCount` according to the real count. -// We assumes that `eqCount` and `rangeCount` contribute the same error rate. -func getNewCountForIndex(eqCount, rangeCount, totalCount, realCount float64) (float64, float64) { - estimate := (eqCount / totalCount) * (rangeCount / totalCount) * totalCount - if estimate <= 1 { - return eqCount, rangeCount - } - adjustFactor := math.Sqrt(realCount / estimate) - adjustFactor = math.Max(adjustFactor, minAdjustFactor) - return eqCount * adjustFactor, rangeCount * adjustFactor -} - -// dumpFeedbackForIndex dumps the feedback for index. -// For queries that contains both equality and range query, we will split them and update accordingly. -func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { - idx, ok := t.Indices[q.hist.ID] - if !ok { - return nil - } - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - if idx.CMSketch == nil || idx.statsVer != version1 { - return h.dumpFeedbackToKV(q) - } - ranges, err := q.DecodeToRanges(true) - if err != nil { - logutil.Logger(context.Background()).Debug("decode feedback ranges fail", zap.Error(err)) - return nil - } - for i, ran := range ranges { - rangePosition := getOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - continue - } - - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - logutil.Logger(context.Background()).Debug("encode keys fail", zap.Error(err)) - continue - } - equalityCount := float64(idx.CMSketch.QueryBytes(bytes)) * idx.getIncreaseFactor(t.Count) - rang := ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - } - colName := idx.Info.Columns[rangePosition].Name.L - var rangeCount float64 - rangeFB := &QueryFeedback{physicalID: q.physicalID} - // prefer index stats over column stats - if idx := t.indexStartWithColumn(colName); idx != nil && idx.Histogram.Len() != 0 { - rangeCount, err = t.GetRowCountByIndexRanges(sc, idx.ID, []*ranger.Range{&rang}) - rangeFB.tp, rangeFB.hist = indexType, &idx.Histogram - } else if col := t.columnByName(colName); col != nil && col.Histogram.Len() != 0 { - rangeCount, err = t.GetRowCountByColumnRanges(sc, col.ID, []*ranger.Range{&rang}) - rangeFB.tp, rangeFB.hist = colType, &col.Histogram - } else { - continue - } - if err != nil { - logutil.Logger(context.Background()).Debug("get row count by ranges fail", zap.Error(err)) - continue - } - - equalityCount, rangeCount = getNewCountForIndex(equalityCount, rangeCount, float64(t.Count), float64(q.feedback[i].count)) - value := types.NewBytesDatum(bytes) - q.feedback[i] = feedback{lower: &value, upper: &value, count: int64(equalityCount)} - err = rangeFB.dumpRangeFeedback(sc, h, &rang, rangeCount) - if err != nil { - logutil.Logger(context.Background()).Debug("dump range feedback fail", zap.Error(err)) - continue - } - } - return errors.Trace(h.dumpFeedbackToKV(q)) -} - -func (q *QueryFeedback) dumpRangeFeedback(sc *stmtctx.StatementContext, h *Handle, ran *ranger.Range, rangeCount float64) error { - lowIsNull := ran.LowVal[0].IsNull() - if q.tp == indexType { - lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) - if err != nil { - return errors.Trace(err) - } - upper, err := codec.EncodeKey(sc, nil, ran.HighVal[0]) - if err != nil { - return errors.Trace(err) - } - ran.LowVal[0].SetBytes(lower) - ran.HighVal[0].SetBytes(upper) - } else { - if !supportColumnType(q.hist.Tp) { - return nil - } - if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = getMinValue(q.hist.Tp) - } - if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = getMaxValue(q.hist.Tp) - } - } - ranges := q.hist.SplitRange(sc, []*ranger.Range{ran}, q.tp == indexType) - counts := make([]float64, 0, len(ranges)) - sum := 0.0 - for i, r := range ranges { - // Though after `SplitRange`, we may have ranges like `[l, r]`, we still use - // `betweenRowCount` to compute the estimation since the ranges of feedback are all in `[l, r)` - // form, that is to say, we ignore the exclusiveness of ranges from `SplitRange` and just use - // its result of boundary values. - count := q.hist.betweenRowCount(r.LowVal[0], r.HighVal[0]) - // We have to include `NullCount` of histogram for [l, r) cases where l is null because `betweenRowCount` - // does not include null values of lower bound. - if i == 0 && lowIsNull { - count += float64(q.hist.NullCount) - } - sum += count - counts = append(counts, count) - } - if sum <= 1 { - return nil - } - // We assume that each part contributes the same error rate. - adjustFactor := rangeCount / sum - for i, r := range ranges { - q.feedback = append(q.feedback, feedback{lower: &r.LowVal[0], upper: &r.HighVal[0], count: int64(counts[i] * adjustFactor)}) - } - return errors.Trace(h.dumpFeedbackToKV(q)) -} - // setNextValue sets the next value for the given datum. For types like float, // we do not set because it is not discrete and does not matter too much when estimating the scalar info. func setNextValue(d *types.Datum) { @@ -1152,8 +854,8 @@ func setNextValue(d *types.Datum) { } } -// supportColumnType checks if the type of the column can be updated by feedback. -func supportColumnType(ft *types.FieldType) bool { +// SupportColumnType checks if the type of the column can be updated by feedback. +func SupportColumnType(ft *types.FieldType) bool { switch ft.Tp { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeFloat, mysql.TypeDouble, mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, @@ -1164,7 +866,8 @@ func supportColumnType(ft *types.FieldType) bool { } } -func getMaxValue(ft *types.FieldType) (max types.Datum) { +// GetMaxValue returns the max value datum for each type. +func GetMaxValue(ft *types.FieldType) (max types.Datum) { switch ft.Tp { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: if mysql.HasUnsignedFlag(ft.Flag) { @@ -1198,7 +901,8 @@ func getMaxValue(ft *types.FieldType) (max types.Datum) { return } -func getMinValue(ft *types.FieldType) (min types.Datum) { +// GetMinValue returns the min value datum for each type. +func GetMinValue(ft *types.FieldType) (min types.Datum) { switch ft.Tp { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: if mysql.HasUnsignedFlag(ft.Flag) { diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 5c71aa0013da2..86c0f2b5480f3 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -25,13 +25,13 @@ var _ = Suite(&testFeedbackSuite{}) type testFeedbackSuite struct { } -func newFeedback(lower, upper, count int64) feedback { +func newFeedback(lower, upper, count int64) Feedback { low, upp := types.NewIntDatum(lower), types.NewIntDatum(upper) - return feedback{&low, &upp, count, 0} + return Feedback{&low, &upp, count, 0} } -func genFeedbacks(lower, upper int64) []feedback { - var feedbacks []feedback +func genFeedbacks(lower, upper int64) []Feedback { + var feedbacks []Feedback for i := lower; i < upper; i++ { feedbacks = append(feedbacks, newFeedback(i, upper, upper-i+1)) } @@ -54,7 +54,7 @@ func genHistogram() *Histogram { } func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { - feedbacks := []feedback{ + feedbacks := []Feedback{ newFeedback(0, 1, 10000), newFeedback(1, 2, 1), newFeedback(2, 3, 3), @@ -65,11 +65,11 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { feedbacks = append(feedbacks, genFeedbacks(21, 60)...) q := NewQueryFeedback(0, genHistogram(), 0, false) - q.feedback = feedbacks + q.Feedback = feedbacks originBucketCount := defaultBucketCount defaultBucketCount = 7 defer func() { defaultBucketCount = originBucketCount }() - c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals, + c.Assert(UpdateHistogram(q.Hist, q).ToString(0), Equals, "column:0 ndv:10057 totColSize:0\n"+ "num: 10000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ "num: 8 lower_bound: 2 upper_bound: 7 repeats: 0\n"+ @@ -82,14 +82,14 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { func (s *testFeedbackSuite) TestSplitBuckets(c *C) { // test bucket split - feedbacks := []feedback{newFeedback(0, 1, 1)} + feedbacks := []Feedback{newFeedback(0, 1, 1)} for i := 0; i < 100; i++ { feedbacks = append(feedbacks, newFeedback(10, 15, 5)) } q := NewQueryFeedback(0, genHistogram(), 0, false) - q.feedback = feedbacks - buckets, isNewBuckets, totalCount := splitBuckets(q.Hist(), q) - c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, + q.Feedback = feedbacks + buckets, isNewBuckets, totalCount := splitBuckets(q.Hist, q) + c.Assert(buildNewHistogram(q.Hist, buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ "num: 1 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0\n"+ @@ -101,14 +101,14 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { c.Assert(totalCount, Equals, int64(6)) // test do not split if the bucket count is too small - feedbacks = []feedback{newFeedback(0, 1, 100000)} + feedbacks = []Feedback{newFeedback(0, 1, 100000)} for i := 0; i < 100; i++ { feedbacks = append(feedbacks, newFeedback(10, 15, 1)) } q = NewQueryFeedback(0, genHistogram(), 0, false) - q.feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) - c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, + q.Feedback = feedbacks + buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q) + c.Assert(buildNewHistogram(q.Hist, buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ "num: 100000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0\n"+ @@ -128,9 +128,9 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { feedbacks = append(feedbacks, newFeedback(0, 10, 1)) } q = NewQueryFeedback(0, h, 0, false) - q.feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) - c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, + q.Feedback = feedbacks + buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q) + c.Assert(buildNewHistogram(q.Hist, buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ "num: 1000000 lower_bound: 0 upper_bound: 1000000 repeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false}) @@ -144,9 +144,9 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { feedbacks = append(feedbacks, newFeedback(0, 10, 1)) } q = NewQueryFeedback(0, h, 0, false) - q.feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) - c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, + q.Feedback = feedbacks + buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q) + c.Assert(buildNewHistogram(q.Hist, buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ "num: 1 lower_bound: 0 upper_bound: 10 repeats: 0\n"+ "num: 0 lower_bound: 11 upper_bound: 1000000 repeats: 0") @@ -180,7 +180,7 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { "num: 100000 lower_bound: 1 upper_bound: 2 repeats: 0\n" + "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0", }, - // test do not merge if the result bucket count is too large + // test do not Merge if the result bucket count is too large { points: []int64{1, 2, 2, 3, 3, 4, 4, 5}, counts: []int64{1, 1, 100000, 100000}, @@ -215,29 +215,29 @@ func encodeInt(v int64) *types.Datum { func (s *testFeedbackSuite) TestFeedbackEncoding(c *C) { hist := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 0, 0) - q := &QueryFeedback{hist: hist, tp: pkType} - q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(3), 1, 0}) - q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(5), 1, 0}) - val, err := encodeFeedback(q) + q := &QueryFeedback{Hist: hist, Tp: PkType} + q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0}) + q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(5), 1, 0}) + val, err := EncodeFeedback(q) c.Assert(err, IsNil) rq := &QueryFeedback{} - c.Assert(decodeFeedback(val, rq, nil, false), IsNil) - for _, fb := range rq.feedback { - fb.lower.SetBytes(codec.EncodeInt(nil, fb.lower.GetInt64())) - fb.upper.SetBytes(codec.EncodeInt(nil, fb.upper.GetInt64())) + c.Assert(DecodeFeedback(val, rq, nil, false), IsNil) + for _, fb := range rq.Feedback { + fb.Lower.SetBytes(codec.EncodeInt(nil, fb.Lower.GetInt64())) + fb.Upper.SetBytes(codec.EncodeInt(nil, fb.Upper.GetInt64())) } c.Assert(q.Equal(rq), IsTrue) hist.Tp = types.NewFieldType(mysql.TypeBlob) - q = &QueryFeedback{hist: hist} - q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(3), 1, 0}) - q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(1), 1, 0}) - val, err = encodeFeedback(q) + q = &QueryFeedback{Hist: hist} + q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0}) + q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(1), 1, 0}) + val, err = EncodeFeedback(q) c.Assert(err, IsNil) rq = &QueryFeedback{} cms := NewCMSketch(4, 4) - c.Assert(decodeFeedback(val, rq, cms, false), IsNil) + c.Assert(DecodeFeedback(val, rq, cms, false), IsNil) c.Assert(cms.QueryBytes(codec.EncodeInt(nil, 0)), Equals, uint32(1)) - q.feedback = q.feedback[:1] + q.Feedback = q.Feedback[:1] c.Assert(q.Equal(rq), IsTrue) } diff --git a/statistics/handle.go b/statistics/handle.go deleted file mode 100644 index bdc924b8eaf31..0000000000000 --- a/statistics/handle.go +++ /dev/null @@ -1,311 +0,0 @@ -// Copyright 2017 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 statistics - -import ( - "context" - "fmt" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/parser/model" - "github.com/pingcap/tidb/ddl/util" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/sqlexec" - "go.uber.org/zap" -) - -type statsCache map[int64]*Table - -// Handle can update stats info periodically. -type Handle struct { - mu struct { - sync.Mutex - ctx sessionctx.Context - // lastVersion is the latest update version before last lease. - lastVersion uint64 - // rateMap contains the error rate delta from feedback. - rateMap errorRateDeltaMap - // pid2tid is the map from partition ID to table ID. - pid2tid map[int64]int64 - // schemaVersion is the version of information schema when `pid2tid` is built. - schemaVersion int64 - } - - restrictedExec sqlexec.RestrictedSQLExecutor - - statsCache atomic.Value - // ddlEventCh is a channel to notify a ddl operation has happened. - // It is sent only by owner or the drop stats executor, and read by stats handle. - ddlEventCh chan *util.Event - // listHead contains all the stats collector required by session. - listHead *SessionStatsCollector - // globalMap contains all the delta map from collectors when we dump them to KV. - globalMap tableDeltaMap - // feedback is used to store query feedback info. - feedback []*QueryFeedback - - Lease time.Duration -} - -// Clear the statsCache, only for test. -func (h *Handle) Clear() { - h.mu.Lock() - h.statsCache.Store(statsCache{}) - h.mu.lastVersion = 0 - for len(h.ddlEventCh) > 0 { - <-h.ddlEventCh - } - h.feedback = h.feedback[:0] - h.mu.ctx.GetSessionVars().InitChunkSize = 1 - h.mu.ctx.GetSessionVars().MaxChunkSize = 32 - h.listHead = &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)} - h.globalMap = make(tableDeltaMap) - h.mu.rateMap = make(errorRateDeltaMap) - h.mu.Unlock() -} - -// MaxQueryFeedbackCount is the max number of feedback that cache in memory. -var MaxQueryFeedbackCount = 1 << 10 - -// NewHandle creates a Handle for update stats. -func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { - handle := &Handle{ - ddlEventCh: make(chan *util.Event, 100), - listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, - globalMap: make(tableDeltaMap), - Lease: lease, - feedback: make([]*QueryFeedback, 0, MaxQueryFeedbackCount), - } - // It is safe to use it concurrently because the exec won't touch the ctx. - if exec, ok := ctx.(sqlexec.RestrictedSQLExecutor); ok { - handle.restrictedExec = exec - } - handle.mu.ctx = ctx - handle.mu.rateMap = make(errorRateDeltaMap) - handle.statsCache.Store(statsCache{}) - return handle -} - -// GetQueryFeedback gets the query feedback. It is only use in test. -func (h *Handle) GetQueryFeedback() []*QueryFeedback { - defer func() { - h.feedback = h.feedback[:0] - }() - return h.feedback -} - -// DurationToTS converts duration to timestamp. -func DurationToTS(d time.Duration) uint64 { - return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) -} - -// Update reads stats meta from store and updates the stats map. -func (h *Handle) Update(is infoschema.InfoSchema) error { - lastVersion := h.LastUpdateVersion() - // We need this because for two tables, the smaller version may write later than the one with larger version. - // Consider the case that there are two tables A and B, their version and commit time is (A0, A1) and (B0, B1), - // and A0 < B0 < B1 < A1. We will first read the stats of B, and update the lastVersion to B0, but we cannot read - // the table stats of A0 if we read stats that greater than lastVersion which is B0. - // We can read the stats if the diff between commit time and version is less than three lease. - offset := DurationToTS(3 * h.Lease) - if lastVersion >= offset { - lastVersion = lastVersion - offset - } else { - lastVersion = 0 - } - sql := fmt.Sprintf("SELECT version, table_id, modify_count, count from mysql.stats_meta where version > %d order by version", lastVersion) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql) - if err != nil { - return errors.Trace(err) - } - - tables := make([]*Table, 0, len(rows)) - deletedTableIDs := make([]int64, 0, len(rows)) - for _, row := range rows { - version := row.GetUint64(0) - physicalID := row.GetInt64(1) - modifyCount := row.GetInt64(2) - count := row.GetInt64(3) - lastVersion = version - h.mu.Lock() - table, ok := h.getTableByPhysicalID(is, physicalID) - h.mu.Unlock() - if !ok { - logutil.Logger(context.Background()).Debug("unknown physical ID in stats meta table, maybe it has been dropped", zap.Int64("ID", physicalID)) - deletedTableIDs = append(deletedTableIDs, physicalID) - continue - } - tableInfo := table.Meta() - tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false) - // Error is not nil may mean that there are some ddl changes on this table, we will not update it. - if err != nil { - logutil.Logger(context.Background()).Debug("error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) - continue - } - if tbl == nil { - deletedTableIDs = append(deletedTableIDs, physicalID) - continue - } - tbl.Version = version - tbl.Count = count - tbl.ModifyCount = modifyCount - tbl.name = getFullTableName(is, tableInfo) - tables = append(tables, tbl) - } - h.mu.Lock() - h.mu.lastVersion = lastVersion - h.UpdateTableStats(tables, deletedTableIDs) - h.mu.Unlock() - return nil -} - -func (h *Handle) getTableByPhysicalID(is infoschema.InfoSchema, physicalID int64) (table.Table, bool) { - if is.SchemaMetaVersion() != h.mu.schemaVersion { - h.mu.schemaVersion = is.SchemaMetaVersion() - h.mu.pid2tid = buildPartitionID2TableID(is) - } - if id, ok := h.mu.pid2tid[physicalID]; ok { - return is.TableByID(id) - } - return is.TableByID(physicalID) -} - -func buildPartitionID2TableID(is infoschema.InfoSchema) map[int64]int64 { - mapper := make(map[int64]int64) - for _, db := range is.AllSchemas() { - tbls := db.Tables - for _, tbl := range tbls { - pi := tbl.GetPartitionInfo() - if pi == nil { - continue - } - for _, def := range pi.Definitions { - mapper[def.ID] = tbl.ID - } - } - } - return mapper -} - -// GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. -func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *Table { - return h.GetPartitionStats(tblInfo, tblInfo.ID) -} - -// GetPartitionStats retrieves the partition stats from cache. -func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64) *Table { - tbl, ok := h.statsCache.Load().(statsCache)[pid] - if !ok { - tbl = PseudoTable(tblInfo) - tbl.PhysicalID = pid - h.UpdateTableStats([]*Table{tbl}, nil) - return tbl - } - return tbl -} - -func (h *Handle) copyFromOldCache() statsCache { - newCache := statsCache{} - oldCache := h.statsCache.Load().(statsCache) - for k, v := range oldCache { - newCache[k] = v - } - return newCache -} - -// UpdateTableStats updates the statistics table cache using copy on write. -func (h *Handle) UpdateTableStats(tables []*Table, deletedIDs []int64) { - newCache := h.copyFromOldCache() - for _, tbl := range tables { - id := tbl.PhysicalID - newCache[id] = tbl - } - for _, id := range deletedIDs { - delete(newCache, id) - } - h.statsCache.Store(newCache) -} - -// LoadNeededHistograms will load histograms for those needed columns. -func (h *Handle) LoadNeededHistograms() error { - cols := histogramNeededColumns.allCols() - for _, col := range cols { - tbl, ok := h.statsCache.Load().(statsCache)[col.tableID] - if !ok { - continue - } - tbl = tbl.copy() - c, ok := tbl.Columns[col.columnID] - if !ok || c.Len() > 0 { - histogramNeededColumns.delete(col) - continue - } - hg, err := h.histogramFromStorage(col.tableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) - if err != nil { - return errors.Trace(err) - } - cms, err := h.cmSketchFromStorage(col.tableID, 0, col.columnID) - if err != nil { - return errors.Trace(err) - } - tbl.Columns[c.ID] = &Column{ - PhysicalID: col.tableID, - Histogram: *hg, - Info: c.Info, - CMSketch: cms, - Count: int64(hg.totalRowCount()), - isHandle: c.isHandle, - } - h.UpdateTableStats([]*Table{tbl}, nil) - histogramNeededColumns.delete(col) - } - return nil -} - -// LastUpdateVersion gets the last update version. -func (h *Handle) LastUpdateVersion() uint64 { - h.mu.Lock() - defer h.mu.Unlock() - return h.mu.lastVersion -} - -// SetLastUpdateVersion sets the last update version. -func (h *Handle) SetLastUpdateVersion(version uint64) { - h.mu.Lock() - defer h.mu.Unlock() - h.mu.lastVersion = version -} - -// FlushStats flushes the cached stats update into store. -func (h *Handle) FlushStats() { - for len(h.ddlEventCh) > 0 { - e := <-h.ddlEventCh - if err := h.HandleDDLEvent(e); err != nil { - logutil.Logger(context.Background()).Debug("[stats] handle ddl event fail", zap.Error(err)) - } - } - if err := h.DumpStatsDeltaToKV(DumpAll); err != nil { - logutil.Logger(context.Background()).Debug("[stats] dump stats delta fail", zap.Error(err)) - } - if err := h.DumpStatsFeedbackToKV(); err != nil { - logutil.Logger(context.Background()).Debug("[stats] dump stats feedback fail", zap.Error(err)) - } -} diff --git a/statistics/bootstrap.go b/statistics/handle/bootstrap.go similarity index 85% rename from statistics/bootstrap.go rename to statistics/handle/bootstrap.go index b68893b363179..42a19b21d0499 100644 --- a/statistics/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( "context" @@ -23,6 +23,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" @@ -30,7 +31,7 @@ import ( "go.uber.org/zap" ) -func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { physicalID := row.GetInt64(1) table, ok := h.getTableByPhysicalID(is, physicalID) @@ -39,24 +40,24 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache continue } tableInfo := table.Meta() - newHistColl := HistColl{ + newHistColl := statistics.HistColl{ PhysicalID: physicalID, HavePhysicalID: true, Count: row.GetInt64(3), ModifyCount: row.GetInt64(2), - Columns: make(map[int64]*Column, len(tableInfo.Columns)), - Indices: make(map[int64]*Index, len(tableInfo.Indices)), + Columns: make(map[int64]*statistics.Column, len(tableInfo.Columns)), + Indices: make(map[int64]*statistics.Index, len(tableInfo.Indices)), } - tbl := &Table{ + tbl := &statistics.Table{ HistColl: newHistColl, Version: row.GetUint64(0), - name: getFullTableName(is, tableInfo), + Name: getFullTableName(is, tableInfo), } tables[physicalID] = tbl } } -func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { +func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (StatsCache, error) { h.mu.Lock() defer h.mu.Unlock() sql := "select HIGH_PRIORITY version, table_id, modify_count, count from mysql.stats_meta" @@ -67,7 +68,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { if err != nil { return nil, errors.Trace(err) } - tables := statsCache{} + tables := StatsCache{} req := rc[0].NewRecordBatch() iter := chunk.NewIterator4Chunk(req.Chunk) for { @@ -83,7 +84,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { return tables, nil } -func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { table, ok := tables[row.GetInt64(0)] if !ok { @@ -102,13 +103,13 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables stat if idxInfo == nil { continue } - cms, err := decodeCMSketch(row.GetBytes(6)) + cms, err := statistics.DecodeCMSketch(row.GetBytes(6)) if err != nil { cms = nil terror.Log(errors.Trace(err)) } - hist := NewHistogram(id, ndv, nullCount, version, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) - table.Indices[hist.ID] = &Index{Histogram: *hist, CMSketch: cms, Info: idxInfo, statsVer: row.GetInt64(8)} + hist := statistics.NewHistogram(id, ndv, nullCount, version, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) + table.Indices[hist.ID] = &statistics.Index{Histogram: *hist, CMSketch: cms, Info: idxInfo, StatsVer: row.GetInt64(8)} } else { var colInfo *model.ColumnInfo for _, col := range tbl.Meta().Columns { @@ -120,20 +121,20 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables stat if colInfo == nil { continue } - hist := NewHistogram(id, ndv, nullCount, version, &colInfo.FieldType, 0, totColSize) + hist := statistics.NewHistogram(id, ndv, nullCount, version, &colInfo.FieldType, 0, totColSize) hist.Correlation = row.GetFloat64(9) - table.Columns[hist.ID] = &Column{ + table.Columns[hist.ID] = &statistics.Column{ Histogram: *hist, PhysicalID: table.PhysicalID, Info: colInfo, Count: nullCount, - isHandle: tbl.Meta().PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), + IsHandle: tbl.Meta().PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), } } } } -func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables statsCache) error { +func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables StatsCache) error { h.mu.Lock() defer h.mu.Unlock() sql := "select HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation from mysql.stats_histograms" @@ -159,7 +160,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables statsCache return nil } -func initStatsBuckets4Chunk(ctx sessionctx.Context, tables statsCache, iter *chunk.Iterator4Chunk) { +func initStatsBuckets4Chunk(ctx sessionctx.Context, tables StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tableID, isIndex, histID := row.GetInt64(0), row.GetInt64(1), row.GetInt64(2) table, ok := tables[tableID] @@ -167,7 +168,7 @@ func initStatsBuckets4Chunk(ctx sessionctx.Context, tables statsCache, iter *chu continue } var lower, upper types.Datum - var hist *Histogram + var hist *statistics.Histogram if isIndex > 0 { index, ok := table.Indices[histID] if !ok { @@ -205,7 +206,7 @@ func initStatsBuckets4Chunk(ctx sessionctx.Context, tables statsCache, iter *chu } } -func (h *Handle) initStatsBuckets(tables statsCache) error { +func (h *Handle) initStatsBuckets(tables StatsCache) error { h.mu.Lock() defer h.mu.Unlock() sql := "select HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound from mysql.stats_buckets order by table_id, is_index, hist_id, bucket_id" @@ -262,7 +263,7 @@ func (h *Handle) InitStats(is infoschema.InfoSchema) error { if err != nil { return errors.Trace(err) } - h.statsCache.Store(tables) + h.StatsCache.Store(tables) return nil } diff --git a/statistics/ddl.go b/statistics/handle/ddl.go similarity index 99% rename from statistics/ddl.go rename to statistics/handle/ddl.go index fa390ce664605..14f6baec5f412 100644 --- a/statistics/ddl.go +++ b/statistics/handle/ddl.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( "context" diff --git a/statistics/ddl_test.go b/statistics/handle/ddl_test.go similarity index 99% rename from statistics/ddl_test.go rename to statistics/handle/ddl_test.go index 0521988b4aed8..769a7093bb59d 100644 --- a/statistics/ddl_test.go +++ b/statistics/handle/ddl_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package handle_test import ( . "github.com/pingcap/check" diff --git a/statistics/dump.go b/statistics/handle/dump.go similarity index 86% rename from statistics/dump.go rename to statistics/handle/dump.go index e6043e9b2109d..1a4098bfde8f4 100644 --- a/statistics/dump.go +++ b/statistics/handle/dump.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( "time" @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tipb/go-tipb" ) @@ -44,15 +45,15 @@ type jsonColumn struct { LastUpdateVersion uint64 `json:"last_update_version"` } -func dumpJSONCol(hist *Histogram, CMSketch *CMSketch) *jsonColumn { +func dumpJSONCol(hist *statistics.Histogram, CMSketch *statistics.CMSketch) *jsonColumn { jsonCol := &jsonColumn{ - Histogram: HistogramToProto(hist), + Histogram: statistics.HistogramToProto(hist), NullCount: hist.NullCount, TotColSize: hist.TotColSize, LastUpdateVersion: hist.LastUpdateVersion, } if CMSketch != nil { - jsonCol.CMSketch = CMSketchToProto(CMSketch) + jsonCol.CMSketch = statistics.CMSketchToProto(CMSketch) } return jsonCol } @@ -170,16 +171,16 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, } // TableStatsFromJSON loads statistic from JSONTable and return the Table of statistic. -func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *JSONTable) (*Table, error) { - newHistColl := HistColl{ +func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *JSONTable) (*statistics.Table, error) { + newHistColl := statistics.HistColl{ PhysicalID: physicalID, HavePhysicalID: true, Count: jsonTbl.Count, ModifyCount: jsonTbl.ModifyCount, - Columns: make(map[int64]*Column, len(jsonTbl.Columns)), - Indices: make(map[int64]*Index, len(jsonTbl.Indices)), + Columns: make(map[int64]*statistics.Column, len(jsonTbl.Columns)), + Indices: make(map[int64]*statistics.Index, len(jsonTbl.Indices)), } - tbl := &Table{ + tbl := &statistics.Table{ HistColl: newHistColl, } for id, jsonIdx := range jsonTbl.Indices { @@ -187,11 +188,11 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *J if idxInfo.Name.L != id { continue } - hist := HistogramFromProto(jsonIdx.Histogram) + hist := statistics.HistogramFromProto(jsonIdx.Histogram) hist.ID, hist.NullCount, hist.LastUpdateVersion = idxInfo.ID, jsonIdx.NullCount, jsonIdx.LastUpdateVersion - idx := &Index{ + idx := &statistics.Index{ Histogram: *hist, - CMSketch: CMSketchFromProto(jsonIdx.CMSketch), + CMSketch: statistics.CMSketchFromProto(jsonIdx.CMSketch), Info: idxInfo, } tbl.Indices[idx.ID] = idx @@ -203,21 +204,21 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *J if colInfo.Name.L != id { continue } - hist := HistogramFromProto(jsonCol.Histogram) - count := int64(hist.totalRowCount()) + hist := statistics.HistogramFromProto(jsonCol.Histogram) + count := int64(hist.TotalRowCount()) sc := &stmtctx.StatementContext{TimeZone: time.UTC} hist, err := hist.ConvertTo(sc, &colInfo.FieldType) if err != nil { return nil, errors.Trace(err) } hist.ID, hist.NullCount, hist.LastUpdateVersion, hist.TotColSize = colInfo.ID, jsonCol.NullCount, jsonCol.LastUpdateVersion, jsonCol.TotColSize - col := &Column{ + col := &statistics.Column{ PhysicalID: physicalID, Histogram: *hist, - CMSketch: CMSketchFromProto(jsonCol.CMSketch), + CMSketch: statistics.CMSketchFromProto(jsonCol.CMSketch), Info: colInfo, Count: count, - isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), + IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), } tbl.Columns[col.ID] = col } diff --git a/statistics/dump_test.go b/statistics/handle/dump_test.go similarity index 94% rename from statistics/dump_test.go rename to statistics/handle/dump_test.go index 7f753e193dbab..a56e1163de907 100644 --- a/statistics/dump_test.go +++ b/statistics/handle/dump_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package handle_test import ( "fmt" @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/testkit" ) @@ -34,14 +35,14 @@ func (s *testStatsSuite) TestConversion(c *C) { tk.MustExec("insert into t(a,b) values (1, 1),(3, 1),(5, 10)") is := s.do.InfoSchema() h := s.do.StatsHandle() - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta()) c.Assert(err, IsNil) - loadTbl, err := statistics.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl) + loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl) c.Assert(err, IsNil) tbl := h.GetTableStats(tableInfo.Meta()) diff --git a/statistics/gc.go b/statistics/handle/gc.go similarity index 99% rename from statistics/gc.go rename to statistics/handle/gc.go index 4a95935457179..bbca79a972013 100644 --- a/statistics/gc.go +++ b/statistics/handle/gc.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( "context" diff --git a/statistics/gc_test.go b/statistics/handle/gc_test.go similarity index 99% rename from statistics/gc_test.go rename to statistics/handle/gc_test.go index c6b96889b022e..bfb5b0a90ffe7 100644 --- a/statistics/gc_test.go +++ b/statistics/handle/gc_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package handle_test import ( "math" diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go new file mode 100644 index 0000000000000..7545d22e35cdf --- /dev/null +++ b/statistics/handle/handle.go @@ -0,0 +1,642 @@ +// Copyright 2017 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 handle + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" + "go.uber.org/zap" +) + +// StatsCache caches the tables in memory for Handle. +type StatsCache map[int64]*statistics.Table + +// Handle can update stats info periodically. +type Handle struct { + mu struct { + sync.Mutex + ctx sessionctx.Context + // lastVersion is the latest update version before last lease. + lastVersion uint64 + // rateMap contains the error rate delta from feedback. + rateMap errorRateDeltaMap + // pid2tid is the map from partition ID to table ID. + pid2tid map[int64]int64 + // schemaVersion is the version of information schema when `pid2tid` is built. + schemaVersion int64 + } + + restrictedExec sqlexec.RestrictedSQLExecutor + + StatsCache atomic.Value + // ddlEventCh is a channel to notify a ddl operation has happened. + // It is sent only by owner or the drop stats executor, and read by stats handle. + ddlEventCh chan *util.Event + // listHead contains all the stats collector required by session. + listHead *SessionStatsCollector + // globalMap contains all the delta map from collectors when we dump them to KV. + globalMap tableDeltaMap + // feedback is used to store query feedback info. + feedback []*statistics.QueryFeedback + + Lease time.Duration +} + +// Clear the StatsCache, only for test. +func (h *Handle) Clear() { + h.mu.Lock() + h.StatsCache.Store(StatsCache{}) + h.mu.lastVersion = 0 + for len(h.ddlEventCh) > 0 { + <-h.ddlEventCh + } + h.feedback = h.feedback[:0] + h.mu.ctx.GetSessionVars().InitChunkSize = 1 + h.mu.ctx.GetSessionVars().MaxChunkSize = 32 + h.listHead = &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)} + h.globalMap = make(tableDeltaMap) + h.mu.rateMap = make(errorRateDeltaMap) + h.mu.Unlock() +} + +// MaxQueryFeedbackCount is the max number of feedback that cache in memory. +var MaxQueryFeedbackCount = 1 << 10 + +// NewHandle creates a Handle for update stats. +func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { + handle := &Handle{ + ddlEventCh: make(chan *util.Event, 100), + listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, + globalMap: make(tableDeltaMap), + Lease: lease, + feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount), + } + // It is safe to use it concurrently because the exec won't touch the ctx. + if exec, ok := ctx.(sqlexec.RestrictedSQLExecutor); ok { + handle.restrictedExec = exec + } + handle.mu.ctx = ctx + handle.mu.rateMap = make(errorRateDeltaMap) + handle.StatsCache.Store(StatsCache{}) + return handle +} + +// GetQueryFeedback gets the query feedback. It is only use in test. +func (h *Handle) GetQueryFeedback() []*statistics.QueryFeedback { + defer func() { + h.feedback = h.feedback[:0] + }() + return h.feedback +} + +// DurationToTS converts duration to timestamp. +func DurationToTS(d time.Duration) uint64 { + return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) +} + +// Update reads stats meta from store and updates the stats map. +func (h *Handle) Update(is infoschema.InfoSchema) error { + lastVersion := h.LastUpdateVersion() + // We need this because for two tables, the smaller version may write later than the one with larger version. + // Consider the case that there are two tables A and B, their version and commit time is (A0, A1) and (B0, B1), + // and A0 < B0 < B1 < A1. We will first read the stats of B, and update the lastVersion to B0, but we cannot read + // the table stats of A0 if we read stats that greater than lastVersion which is B0. + // We can read the stats if the diff between commit time and version is less than three lease. + offset := DurationToTS(3 * h.Lease) + if lastVersion >= offset { + lastVersion = lastVersion - offset + } else { + lastVersion = 0 + } + sql := fmt.Sprintf("SELECT version, table_id, modify_count, count from mysql.stats_meta where version > %d order by version", lastVersion) + rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql) + if err != nil { + return errors.Trace(err) + } + + tables := make([]*statistics.Table, 0, len(rows)) + deletedTableIDs := make([]int64, 0, len(rows)) + for _, row := range rows { + version := row.GetUint64(0) + physicalID := row.GetInt64(1) + modifyCount := row.GetInt64(2) + count := row.GetInt64(3) + lastVersion = version + h.mu.Lock() + table, ok := h.getTableByPhysicalID(is, physicalID) + h.mu.Unlock() + if !ok { + logutil.Logger(context.Background()).Debug("unknown physical ID in stats meta table, maybe it has been dropped", zap.Int64("ID", physicalID)) + deletedTableIDs = append(deletedTableIDs, physicalID) + continue + } + tableInfo := table.Meta() + tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false) + // Error is not nil may mean that there are some ddl changes on this table, we will not update it. + if err != nil { + logutil.Logger(context.Background()).Debug("error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) + continue + } + if tbl == nil { + deletedTableIDs = append(deletedTableIDs, physicalID) + continue + } + tbl.Version = version + tbl.Count = count + tbl.ModifyCount = modifyCount + tbl.Name = getFullTableName(is, tableInfo) + tables = append(tables, tbl) + } + h.mu.Lock() + h.mu.lastVersion = lastVersion + h.UpdateTableStats(tables, deletedTableIDs) + h.mu.Unlock() + return nil +} + +func (h *Handle) getTableByPhysicalID(is infoschema.InfoSchema, physicalID int64) (table.Table, bool) { + if is.SchemaMetaVersion() != h.mu.schemaVersion { + h.mu.schemaVersion = is.SchemaMetaVersion() + h.mu.pid2tid = buildPartitionID2TableID(is) + } + if id, ok := h.mu.pid2tid[physicalID]; ok { + return is.TableByID(id) + } + return is.TableByID(physicalID) +} + +func buildPartitionID2TableID(is infoschema.InfoSchema) map[int64]int64 { + mapper := make(map[int64]int64) + for _, db := range is.AllSchemas() { + tbls := db.Tables + for _, tbl := range tbls { + pi := tbl.GetPartitionInfo() + if pi == nil { + continue + } + for _, def := range pi.Definitions { + mapper[def.ID] = tbl.ID + } + } + } + return mapper +} + +// GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. +func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *statistics.Table { + return h.GetPartitionStats(tblInfo, tblInfo.ID) +} + +// GetPartitionStats retrieves the partition stats from cache. +func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64) *statistics.Table { + tbl, ok := h.StatsCache.Load().(StatsCache)[pid] + if !ok { + tbl = statistics.PseudoTable(tblInfo) + tbl.PhysicalID = pid + h.UpdateTableStats([]*statistics.Table{tbl}, nil) + return tbl + } + return tbl +} + +func (h *Handle) copyFromOldCache() StatsCache { + newCache := StatsCache{} + oldCache := h.StatsCache.Load().(StatsCache) + for k, v := range oldCache { + newCache[k] = v + } + return newCache +} + +// UpdateTableStats updates the statistics table cache using copy on write. +func (h *Handle) UpdateTableStats(tables []*statistics.Table, deletedIDs []int64) { + newCache := h.copyFromOldCache() + for _, tbl := range tables { + id := tbl.PhysicalID + newCache[id] = tbl + } + for _, id := range deletedIDs { + delete(newCache, id) + } + h.StatsCache.Store(newCache) +} + +// LoadNeededHistograms will load histograms for those needed columns. +func (h *Handle) LoadNeededHistograms() error { + cols := statistics.HistogramNeededColumns.AllCols() + for _, col := range cols { + tbl, ok := h.StatsCache.Load().(StatsCache)[col.TableID] + if !ok { + continue + } + tbl = tbl.Copy() + c, ok := tbl.Columns[col.ColumnID] + if !ok || c.Len() > 0 { + statistics.HistogramNeededColumns.Delete(col) + continue + } + hg, err := h.histogramFromStorage(col.TableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) + if err != nil { + return errors.Trace(err) + } + cms, err := h.cmSketchFromStorage(col.TableID, 0, col.ColumnID) + if err != nil { + return errors.Trace(err) + } + tbl.Columns[c.ID] = &statistics.Column{ + PhysicalID: col.TableID, + Histogram: *hg, + Info: c.Info, + CMSketch: cms, + Count: int64(hg.TotalRowCount()), + IsHandle: c.IsHandle, + } + h.UpdateTableStats([]*statistics.Table{tbl}, nil) + statistics.HistogramNeededColumns.Delete(col) + } + return nil +} + +// LastUpdateVersion gets the last update version. +func (h *Handle) LastUpdateVersion() uint64 { + h.mu.Lock() + defer h.mu.Unlock() + return h.mu.lastVersion +} + +// SetLastUpdateVersion sets the last update version. +func (h *Handle) SetLastUpdateVersion(version uint64) { + h.mu.Lock() + defer h.mu.Unlock() + h.mu.lastVersion = version +} + +// FlushStats flushes the cached stats update into store. +func (h *Handle) FlushStats() { + for len(h.ddlEventCh) > 0 { + e := <-h.ddlEventCh + if err := h.HandleDDLEvent(e); err != nil { + logutil.Logger(context.Background()).Debug("[stats] handle ddl event fail", zap.Error(err)) + } + } + if err := h.DumpStatsDeltaToKV(DumpAll); err != nil { + logutil.Logger(context.Background()).Debug("[stats] dump stats delta fail", zap.Error(err)) + } + if err := h.DumpStatsFeedbackToKV(); err != nil { + logutil.Logger(context.Background()).Debug("[stats] dump stats feedback fail", zap.Error(err)) + } +} + +func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*statistics.CMSketch, error) { + selSQL := fmt.Sprintf("select cm_sketch from mysql.stats_histograms where table_id = %d and is_index = %d and hist_id = %d", tblID, isIndex, histID) + rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + if err != nil { + return nil, errors.Trace(err) + } + if len(rows) == 0 { + return nil, nil + } + return statistics.DecodeCMSketch(rows[0].GetBytes(0)) +} + +func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo) error { + histID := row.GetInt64(2) + distinct := row.GetInt64(3) + histVer := row.GetUint64(4) + nullCount := row.GetInt64(5) + idx := table.Indices[histID] + errorRate := statistics.ErrorRate{} + if statistics.IsAnalyzed(row.GetInt64(8)) { + h.mu.Lock() + h.mu.rateMap.clear(table.PhysicalID, histID, true) + h.mu.Unlock() + } else if idx != nil { + errorRate = idx.ErrorRate + } + for _, idxInfo := range tableInfo.Indices { + if histID != idxInfo.ID { + continue + } + if idx == nil || idx.LastUpdateVersion < histVer { + hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) + if err != nil { + return errors.Trace(err) + } + cms, err := h.cmSketchFromStorage(table.PhysicalID, 1, idxInfo.ID) + if err != nil { + return errors.Trace(err) + } + idx = &statistics.Index{Histogram: *hg, CMSketch: cms, Info: idxInfo, ErrorRate: errorRate, StatsVer: row.GetInt64(7)} + } + break + } + if idx != nil { + table.Indices[histID] = idx + } else { + logutil.Logger(context.Background()).Debug("we cannot find index id in table info. It may be deleted.", zap.Int64("indexID", histID), zap.String("table", tableInfo.Name.O)) + } + return nil +} + +func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo, loadAll bool) error { + histID := row.GetInt64(2) + distinct := row.GetInt64(3) + histVer := row.GetUint64(4) + nullCount := row.GetInt64(5) + totColSize := row.GetInt64(6) + correlation := row.GetFloat64(9) + col := table.Columns[histID] + errorRate := statistics.ErrorRate{} + if statistics.IsAnalyzed(row.GetInt64(8)) { + h.mu.Lock() + h.mu.rateMap.clear(table.PhysicalID, histID, false) + h.mu.Unlock() + } else if col != nil { + errorRate = col.ErrorRate + } + for _, colInfo := range tableInfo.Columns { + if histID != colInfo.ID { + continue + } + isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) + // We will not load buckets if: + // 1. Lease > 0, and: + // 2. this column is not handle, and: + // 3. the column doesn't has buckets before, and: + // 4. loadAll is false. + notNeedLoad := h.Lease > 0 && + !isHandle && + (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) && + !loadAll + if notNeedLoad { + count, err := h.columnCountFromStorage(table.PhysicalID, histID) + if err != nil { + return errors.Trace(err) + } + col = &statistics.Column{ + PhysicalID: table.PhysicalID, + Histogram: *statistics.NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), + Info: colInfo, + Count: count + nullCount, + ErrorRate: errorRate, + IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), + } + col.Histogram.Correlation = correlation + break + } + if col == nil || col.LastUpdateVersion < histVer || loadAll { + hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) + if err != nil { + return errors.Trace(err) + } + cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID) + if err != nil { + return errors.Trace(err) + } + col = &statistics.Column{ + PhysicalID: table.PhysicalID, + Histogram: *hg, + Info: colInfo, + CMSketch: cms, + Count: int64(hg.TotalRowCount()), + ErrorRate: errorRate, + IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), + } + break + } + if col.TotColSize != totColSize { + newCol := *col + newCol.TotColSize = totColSize + col = &newCol + } + break + } + if col != nil { + table.Columns[col.ID] = col + } else { + // If we didn't find a Column or Index in tableInfo, we won't load the histogram for it. + // But don't worry, next lease the ddl will be updated, and we will load a same table for two times to + // avoid error. + logutil.Logger(context.Background()).Debug("we cannot find column in table info now. It may be deleted", zap.Int64("colID", histID), zap.String("table", tableInfo.Name.O)) + } + return nil +} + +// tableStatsFromStorage loads table stats info from storage. +func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool) (*statistics.Table, error) { + table, ok := h.StatsCache.Load().(StatsCache)[physicalID] + // If table stats is pseudo, we also need to copy it, since we will use the column stats when + // the average error rate of it is small. + if !ok { + histColl := statistics.HistColl{ + PhysicalID: physicalID, + HavePhysicalID: true, + Columns: make(map[int64]*statistics.Column, len(tableInfo.Columns)), + Indices: make(map[int64]*statistics.Index, len(tableInfo.Indices)), + } + table = &statistics.Table{ + HistColl: histColl, + } + } else { + // We copy it before writing to avoid race. + table = table.Copy() + } + table.Pseudo = false + selSQL := fmt.Sprintf("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation from mysql.stats_histograms where table_id = %d", physicalID) + rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + if err != nil { + return nil, errors.Trace(err) + } + // Check deleted table. + if len(rows) == 0 { + return nil, nil + } + for _, row := range rows { + if row.GetInt64(1) > 0 { + if err := h.indexStatsFromStorage(row, table, tableInfo); err != nil { + return nil, errors.Trace(err) + } + } else { + if err := h.columnStatsFromStorage(row, table, tableInfo, loadAll); err != nil { + return nil, errors.Trace(err) + } + } + } + return table, nil +} + +// SaveStatsToStorage saves the stats to storage. +func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, isAnalyzed int64) (err error) { + h.mu.Lock() + defer h.mu.Unlock() + ctx := context.TODO() + exec := h.mu.ctx.(sqlexec.SQLExecutor) + _, err = exec.Execute(ctx, "begin") + if err != nil { + return errors.Trace(err) + } + defer func() { + err = finishTransaction(context.Background(), exec, err) + }() + txn, err := h.mu.ctx.Txn(true) + if err != nil { + return errors.Trace(err) + } + + version := txn.StartTS() + var sql string + // If the count is less than 0, then we do not want to update the modify count and count. + if count >= 0 { + sql = fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count) values (%d, %d, %d)", version, tableID, count) + } else { + sql = fmt.Sprintf("update mysql.stats_meta set version = %d where table_id = %d", version, tableID) + } + _, err = exec.Execute(ctx, sql) + if err != nil { + return + } + data, err := statistics.EncodeCMSketch(cms) + if err != nil { + return + } + flag := 0 + if isAnalyzed == 1 { + flag = statistics.AnalyzeFlag + } + replaceSQL := fmt.Sprintf("replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, flag, correlation) values (%d, %d, %d, %d, %d, %d, X'%X', %d, %d, %d, %f)", + tableID, isIndex, hg.ID, hg.NDV, version, hg.NullCount, data, hg.TotColSize, statistics.CurStatsVersion, flag, hg.Correlation) + _, err = exec.Execute(ctx, replaceSQL) + if err != nil { + return + } + deleteSQL := fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, hg.ID) + _, err = exec.Execute(ctx, deleteSQL) + if err != nil { + return + } + sc := h.mu.ctx.GetSessionVars().StmtCtx + for i := range hg.Buckets { + count := hg.Buckets[i].Count + if i > 0 { + count -= hg.Buckets[i-1].Count + } + var upperBound types.Datum + upperBound, err = hg.GetUpper(i).ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) + if err != nil { + return + } + var lowerBound types.Datum + lowerBound, err = hg.GetLower(i).ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) + if err != nil { + return + } + insertSQL := fmt.Sprintf("insert into mysql.stats_buckets(table_id, is_index, hist_id, bucket_id, count, repeats, lower_bound, upper_bound) values(%d, %d, %d, %d, %d, %d, X'%X', X'%X')", tableID, isIndex, hg.ID, i, count, hg.Buckets[i].Repeat, lowerBound.GetBytes(), upperBound.GetBytes()) + _, err = exec.Execute(ctx, insertSQL) + if err != nil { + return + } + } + return +} + +// SaveMetaToStorage will save stats_meta to storage. +func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error) { + h.mu.Lock() + defer h.mu.Unlock() + ctx := context.TODO() + exec := h.mu.ctx.(sqlexec.SQLExecutor) + _, err = exec.Execute(ctx, "begin") + if err != nil { + return errors.Trace(err) + } + defer func() { + err = finishTransaction(ctx, exec, err) + }() + txn, err := h.mu.ctx.Txn(true) + if err != nil { + return errors.Trace(err) + } + var sql string + version := txn.StartTS() + sql = fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count, modify_count) values (%d, %d, %d, %d)", version, tableID, count, modifyCount) + _, err = exec.Execute(ctx, sql) + return +} + +func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (*statistics.Histogram, error) { + selSQL := fmt.Sprintf("select count, repeats, lower_bound, upper_bound from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d order by bucket_id", tableID, isIndex, colID) + rows, fields, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + if err != nil { + return nil, errors.Trace(err) + } + bucketSize := len(rows) + hg := statistics.NewHistogram(colID, distinct, nullCount, ver, tp, bucketSize, totColSize) + hg.Correlation = corr + totalCount := int64(0) + for i := 0; i < bucketSize; i++ { + count := rows[i].GetInt64(0) + repeats := rows[i].GetInt64(1) + var upperBound, lowerBound types.Datum + if isIndex == 1 { + lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) + upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) + } else { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + d := rows[i].GetDatum(2, &fields[2].Column.FieldType) + lowerBound, err = d.ConvertTo(sc, tp) + if err != nil { + return nil, errors.Trace(err) + } + d = rows[i].GetDatum(3, &fields[3].Column.FieldType) + upperBound, err = d.ConvertTo(sc, tp) + if err != nil { + return nil, errors.Trace(err) + } + } + totalCount += count + hg.AppendBucket(&lowerBound, &upperBound, totalCount, repeats) + } + hg.PreCalculateScalar() + return hg, nil +} + +func (h *Handle) columnCountFromStorage(tableID, colID int64) (int64, error) { + selSQL := fmt.Sprintf("select sum(count) from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d", tableID, 0, colID) + rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + if err != nil { + return 0, errors.Trace(err) + } + if rows[0].IsNull(0) { + return 0, nil + } + return rows[0].GetMyDecimal(0).ToInt() +} diff --git a/statistics/handle_test.go b/statistics/handle/handle_test.go similarity index 98% rename from statistics/handle_test.go rename to statistics/handle/handle_test.go index 162a5f3d4c403..2be9b8e798194 100644 --- a/statistics/handle_test.go +++ b/statistics/handle/handle_test.go @@ -11,10 +11,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package handle_test import ( "fmt" + "testing" "time" . "github.com/pingcap/check" @@ -25,12 +26,17 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" ) +func TestT(t *testing.T) { + TestingT(t) +} + func cleanEnv(c *C, store kv.Storage, do *domain.Domain) { tk := testkit.NewTestKit(c, store) tk.MustExec("use test") @@ -220,7 +226,7 @@ func (s *testStatsSuite) TestAvgColLen(c *C) { func (s *testStatsSuite) TestDurationToTS(c *C) { tests := []time.Duration{time.Millisecond, time.Second, time.Minute, time.Hour} for _, t := range tests { - ts := statistics.DurationToTS(t) + ts := handle.DurationToTS(t) c.Assert(oracle.ExtractPhysical(ts)*int64(time.Millisecond), Equals, int64(t)) } } @@ -236,7 +242,7 @@ func (s *testStatsSuite) TestVersion(c *C) { tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() - h := statistics.NewHandle(testKit.Se, time.Millisecond) + h := handle.NewHandle(testKit.Se, time.Millisecond) unit := oracle.ComposeTS(1, 0) testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) @@ -324,7 +330,7 @@ func (s *testStatsSuite) TestLoadHist(c *C) { for i := 0; i < rowCount; i++ { testKit.MustExec("insert into t values('bb','sdfga')") } - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(do.InfoSchema()) newStatsTbl := h.GetTableStats(tableInfo) // The stats table is updated. diff --git a/statistics/update.go b/statistics/handle/update.go similarity index 57% rename from statistics/update.go rename to statistics/handle/update.go index 27fa115527710..75b810bcacd8b 100644 --- a/statistics/update.go +++ b/statistics/handle/update.go @@ -11,9 +11,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( + "bytes" "context" "fmt" "math" @@ -27,11 +28,17 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -55,8 +62,8 @@ func (m tableDeltaMap) update(id int64, delta int64, count int64, colSize *map[i type errorRateDelta struct { PkID int64 - PkErrorRate *ErrorRate - IdxErrorRate map[int64]*ErrorRate + PkErrorRate *statistics.ErrorRate + IdxErrorRate map[int64]*statistics.ErrorRate } type errorRateDeltaMap map[int64]errorRateDelta @@ -65,18 +72,18 @@ func (m errorRateDeltaMap) update(tableID int64, histID int64, rate float64, isI item := m[tableID] if isIndex { if item.IdxErrorRate == nil { - item.IdxErrorRate = make(map[int64]*ErrorRate) + item.IdxErrorRate = make(map[int64]*statistics.ErrorRate) } if item.IdxErrorRate[histID] == nil { - item.IdxErrorRate[histID] = &ErrorRate{} + item.IdxErrorRate[histID] = &statistics.ErrorRate{} } - item.IdxErrorRate[histID].update(rate) + item.IdxErrorRate[histID].Update(rate) } else { if item.PkErrorRate == nil { item.PkID = histID - item.PkErrorRate = &ErrorRate{} + item.PkErrorRate = &statistics.ErrorRate{} } - item.PkErrorRate.update(rate) + item.PkErrorRate.Update(rate) } m[tableID] = item } @@ -86,19 +93,19 @@ func (m errorRateDeltaMap) merge(deltaMap errorRateDeltaMap) { tbl := m[tableID] for histID, errorRate := range item.IdxErrorRate { if tbl.IdxErrorRate == nil { - tbl.IdxErrorRate = make(map[int64]*ErrorRate) + tbl.IdxErrorRate = make(map[int64]*statistics.ErrorRate) } if tbl.IdxErrorRate[histID] == nil { - tbl.IdxErrorRate[histID] = &ErrorRate{} + tbl.IdxErrorRate[histID] = &statistics.ErrorRate{} } - tbl.IdxErrorRate[histID].merge(errorRate) + tbl.IdxErrorRate[histID].Merge(errorRate) } if item.PkErrorRate != nil { if tbl.PkErrorRate == nil { tbl.PkID = item.PkID - tbl.PkErrorRate = &ErrorRate{} + tbl.PkErrorRate = &statistics.ErrorRate{} } - tbl.PkErrorRate.merge(item.PkErrorRate) + tbl.PkErrorRate.Merge(item.PkErrorRate) } m[tableID] = tbl } @@ -130,7 +137,7 @@ type SessionStatsCollector struct { sync.Mutex mapper tableDeltaMap - feedback []*QueryFeedback + feedback []*statistics.QueryFeedback rateMap errorRateDeltaMap next *SessionStatsCollector // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. @@ -151,7 +158,7 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi s.mapper.update(id, delta, count, colSize) } -func mergeQueryFeedback(lq []*QueryFeedback, rq []*QueryFeedback) []*QueryFeedback { +func mergeQueryFeedback(lq []*statistics.QueryFeedback, rq []*statistics.QueryFeedback) []*statistics.QueryFeedback { for _, q := range rq { if len(lq) >= MaxQueryFeedbackCount { break @@ -170,37 +177,27 @@ var ( // StoreQueryFeedback will merges the feedback into stats collector. func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle) error { - q := feedback.(*QueryFeedback) + q := feedback.(*statistics.QueryFeedback) // TODO: If the error rate is small or actual scan count is small, we do not need to store the feed back. - if !q.valid || q.hist == nil { + if !q.Valid || q.Hist == nil { return nil } - err := q.recalculateExpectCount(h) + err := h.RecalculateExpectCount(q) if err != nil { return errors.Trace(err) } - expected := float64(q.expected) - var rate float64 - if q.actual == 0 { - if expected == 0 { - rate = 0 - } else { - rate = 1 - } - } else { - rate = math.Abs(expected-float64(q.actual)) / float64(q.actual) - } - if rate >= MinLogErrorRate && (q.actual >= MinLogScanCount || q.expected >= MinLogScanCount) { + rate := q.CalcErrorRate() + if rate >= MinLogErrorRate && (q.Actual() >= MinLogScanCount || q.Expected >= MinLogScanCount) { metrics.SignificantFeedbackCounter.Inc() if log.GetLevel() == zap.DebugLevel { - q.logDetailedInfo(h) + h.logDetailedInfo(q) } } metrics.StatsInaccuracyRate.Observe(rate) s.Lock() defer s.Unlock() - isIndex := q.tp == indexType - s.rateMap.update(q.physicalID, q.hist.ID, rate, isIndex) + isIndex := q.Tp == statistics.IndexType + s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) if len(s.feedback) < MaxQueryFeedbackCount { s.feedback = append(s.feedback, q) } @@ -233,7 +230,7 @@ func needDumpStatsDelta(h *Handle, id int64, item variable.TableDelta, currentTi if item.InitTime.IsZero() { item.InitTime = currentTime } - tbl, ok := h.statsCache.Load().(statsCache)[id] + tbl, ok := h.StatsCache.Load().(StatsCache)[id] if !ok { // No need to dump if the stats is invalid. return false @@ -373,12 +370,12 @@ func (h *Handle) DumpStatsFeedbackToKV() error { var err error var successCount int for _, fb := range h.feedback { - if fb.tp == pkType { - err = h.dumpFeedbackToKV(fb) + if fb.Tp == statistics.PkType { + err = h.DumpFeedbackToKV(fb) } else { - t, ok := h.statsCache.Load().(statsCache)[fb.physicalID] + t, ok := h.StatsCache.Load().(StatsCache)[fb.PhysicalID] if ok { - err = dumpFeedbackForIndex(h, fb, t) + err = h.DumpFeedbackForIndex(fb, t) } } if err != nil { @@ -390,18 +387,19 @@ func (h *Handle) DumpStatsFeedbackToKV() error { return errors.Trace(err) } -func (h *Handle) dumpFeedbackToKV(fb *QueryFeedback) error { - vals, err := encodeFeedback(fb) +// DumpFeedbackToKV dumps the given feedback to physical kv layer. +func (h *Handle) DumpFeedbackToKV(fb *statistics.QueryFeedback) error { + vals, err := statistics.EncodeFeedback(fb) if err != nil { logutil.Logger(context.Background()).Debug("error occurred when encoding feedback", zap.Error(err)) return nil } var isIndex int64 - if fb.tp == indexType { + if fb.Tp == statistics.IndexType { isIndex = 1 } sql := fmt.Sprintf("insert into mysql.stats_feedback (table_id, hist_id, is_index, feedback) values "+ - "(%d, %d, %d, X'%X')", fb.physicalID, fb.hist.ID, isIndex, vals) + "(%d, %d, %d, X'%X')", fb.PhysicalID, fb.Hist.ID, isIndex, vals) h.mu.Lock() _, err = h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql) h.mu.Unlock() @@ -421,54 +419,54 @@ func (h *Handle) UpdateStatsByLocalFeedback(is infoschema.InfoSchema) { h.sweepList() for _, fb := range h.feedback { h.mu.Lock() - table, ok := h.getTableByPhysicalID(is, fb.physicalID) + table, ok := h.getTableByPhysicalID(is, fb.PhysicalID) h.mu.Unlock() if !ok { continue } - tblStats := h.GetPartitionStats(table.Meta(), fb.physicalID) - newTblStats := tblStats.copy() - if fb.tp == indexType { - idx, ok := tblStats.Indices[fb.hist.ID] + tblStats := h.GetPartitionStats(table.Meta(), fb.PhysicalID) + newTblStats := tblStats.Copy() + if fb.Tp == statistics.IndexType { + idx, ok := tblStats.Indices[fb.Hist.ID] if !ok || idx.Histogram.Len() == 0 { continue } newIdx := *idx - eqFB, ranFB := splitFeedbackByQueryType(fb.feedback) - newIdx.CMSketch = UpdateCMSketch(idx.CMSketch, eqFB) - newIdx.Histogram = *UpdateHistogram(&idx.Histogram, &QueryFeedback{feedback: ranFB}) + eqFB, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) + newIdx.CMSketch = statistics.UpdateCMSketch(idx.CMSketch, eqFB) + newIdx.Histogram = *statistics.UpdateHistogram(&idx.Histogram, &statistics.QueryFeedback{Feedback: ranFB}) newIdx.Histogram.PreCalculateScalar() - newTblStats.Indices[fb.hist.ID] = &newIdx + newTblStats.Indices[fb.Hist.ID] = &newIdx } else { - col, ok := tblStats.Columns[fb.hist.ID] + col, ok := tblStats.Columns[fb.Hist.ID] if !ok || col.Histogram.Len() == 0 { continue } newCol := *col // only use the range query to update primary key - _, ranFB := splitFeedbackByQueryType(fb.feedback) - newFB := &QueryFeedback{feedback: ranFB} - newFB = newFB.decodeIntValues() - newCol.Histogram = *UpdateHistogram(&col.Histogram, newFB) - newTblStats.Columns[fb.hist.ID] = &newCol + _, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) + newFB := &statistics.QueryFeedback{Feedback: ranFB} + newFB = newFB.DecodeIntValues() + newCol.Histogram = *statistics.UpdateHistogram(&col.Histogram, newFB) + newTblStats.Columns[fb.Hist.ID] = &newCol } - h.UpdateTableStats([]*Table{newTblStats}, nil) + h.UpdateTableStats([]*statistics.Table{newTblStats}, nil) } } // UpdateErrorRate updates the error rate of columns from h.rateMap to cache. func (h *Handle) UpdateErrorRate(is infoschema.InfoSchema) { h.mu.Lock() - tbls := make([]*Table, 0, len(h.mu.rateMap)) + tbls := make([]*statistics.Table, 0, len(h.mu.rateMap)) for id, item := range h.mu.rateMap { table, ok := h.getTableByPhysicalID(is, id) if !ok { continue } - tbl := h.GetPartitionStats(table.Meta(), id).copy() + tbl := h.GetPartitionStats(table.Meta(), id).Copy() if item.PkErrorRate != nil && tbl.Columns[item.PkID] != nil { col := *tbl.Columns[item.PkID] - col.ErrorRate.merge(item.PkErrorRate) + col.ErrorRate.Merge(item.PkErrorRate) tbl.Columns[item.PkID] = &col } for key, val := range item.IdxErrorRate { @@ -476,7 +474,7 @@ func (h *Handle) UpdateErrorRate(is infoschema.InfoSchema) { continue } idx := *tbl.Indices[key] - idx.ErrorRate.merge(val) + idx.ErrorRate.Merge(val) tbl.Indices[key] = &idx } tbls = append(tbls, tbl) @@ -531,20 +529,20 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch if !ok { return nil } - var tbl *Table + var tbl *statistics.Table if table.Meta().GetPartitionInfo() != nil { tbl = h.GetPartitionStats(table.Meta(), physicalTableID) } else { tbl = h.GetTableStats(table.Meta()) } - var cms *CMSketch - var hist *Histogram + var cms *statistics.CMSketch + var hist *statistics.Histogram if isIndex == 1 { idx, ok := tbl.Indices[histID] if ok && idx.Histogram.Len() > 0 { idxHist := idx.Histogram hist = &idxHist - cms = idx.CMSketch.copy() + cms = idx.CMSketch.Copy() } } else { col, ok := tbl.Columns[histID] @@ -557,9 +555,9 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch if hist == nil { return nil } - q := &QueryFeedback{} + q := &statistics.QueryFeedback{} for _, row := range rows { - err1 := decodeFeedback(row.GetBytes(3), q, cms, mysql.HasUnsignedFlag(hist.Tp.Flag)) + err1 := statistics.DecodeFeedback(row.GetBytes(3), q, cms, mysql.HasUnsignedFlag(hist.Tp.Flag)) if err1 != nil { logutil.Logger(context.Background()).Debug("decode feedback failed", zap.Error(err)) } @@ -578,8 +576,8 @@ func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { return errors.Trace(err) } -func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *QueryFeedback, hist *Histogram, cms *CMSketch) error { - hist = UpdateHistogram(hist, q) +func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch) error { + hist = statistics.UpdateHistogram(hist, q) err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, 0) metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() return errors.Trace(err) @@ -596,7 +594,7 @@ const ( var AutoAnalyzeMinCnt int64 = 1000 // TableAnalyzed checks if the table is analyzed. -func TableAnalyzed(tbl *Table) bool { +func TableAnalyzed(tbl *statistics.Table) bool { for _, col := range tbl.Columns { if col.Count > 0 { return true @@ -631,7 +629,7 @@ func withinTimePeriod(start, end, now time.Time) bool { // 2. If the table had been analyzed before, we need to analyze it when // "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio" and the current time is // between `start` and `end`. -func NeedAnalyzeTable(tbl *Table, limit time.Duration, autoAnalyzeRatio float64, start, end, now time.Time) (bool, string) { +func NeedAnalyzeTable(tbl *statistics.Table, limit time.Duration, autoAnalyzeRatio float64, start, end, now time.Time) (bool, string) { analyzed := TableAnalyzed(tbl) if !analyzed { t := time.Unix(0, oracle.ExtractPhysical(tbl.Version)*int64(time.Millisecond)) @@ -736,7 +734,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) { return } -func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *Table, start, end time.Time, ratio float64, sql string) bool { +func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *statistics.Table, start, end time.Time, ratio float64, sql string) bool { if statsTbl.Pseudo || statsTbl.Count < AutoAnalyzeMinCnt { return false } @@ -771,3 +769,310 @@ func (h *Handle) execAutoAnalyze(sql string) { metrics.AutoAnalyzeCounter.WithLabelValues("succ").Inc() } } + +// formatBuckets formats bucket from lowBkt to highBkt. +func formatBuckets(hg *statistics.Histogram, lowBkt, highBkt, idxCols int) string { + if lowBkt == highBkt { + return hg.BucketToString(lowBkt, idxCols) + } + if lowBkt+1 == highBkt { + return fmt.Sprintf("%s, %s", hg.BucketToString(lowBkt, 0), hg.BucketToString(highBkt, 0)) + } + // do not care the middle buckets + return fmt.Sprintf("%s, (%d buckets, total count %d), %s", hg.BucketToString(lowBkt, 0), + highBkt-lowBkt-1, hg.Buckets[highBkt-1].Count-hg.Buckets[lowBkt].Count, hg.BucketToString(highBkt, 0)) +} + +func colRangeToStr(c *statistics.Column, ran *ranger.Range, actual int64, factor float64) string { + lowCount, lowBkt := c.LessRowCountWithBktIdx(ran.LowVal[0]) + highCount, highBkt := c.LessRowCountWithBktIdx(ran.HighVal[0]) + return fmt.Sprintf("range: %s, actual: %d, expected: %d, buckets: {%s}", ran.String(), actual, + int64((highCount-lowCount)*factor), formatBuckets(&c.Histogram, lowBkt, highBkt, 0)) +} + +func logForIndexRange(idx *statistics.Index, ran *ranger.Range, actual int64, factor float64) string { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + lb, err := codec.EncodeKey(sc, nil, ran.LowVal...) + if err != nil { + return "" + } + rb, err := codec.EncodeKey(sc, nil, ran.HighVal...) + if err != nil { + return "" + } + if idx.CMSketch != nil && bytes.Compare(kv.Key(lb).PrefixNext(), rb) >= 0 { + str, err := types.DatumsToString(ran.LowVal, true) + if err != nil { + return "" + } + return fmt.Sprintf("value: %s, actual: %d, expected: %d", str, actual, int64(float64(idx.QueryBytes(lb))*factor)) + } + l, r := types.NewBytesDatum(lb), types.NewBytesDatum(rb) + lowCount, lowBkt := idx.LessRowCountWithBktIdx(l) + highCount, highBkt := idx.LessRowCountWithBktIdx(r) + return fmt.Sprintf("range: %s, actual: %d, expected: %d, histogram: {%s}", ran.String(), actual, + int64((highCount-lowCount)*factor), formatBuckets(&idx.Histogram, lowBkt, highBkt, len(idx.Info.Columns))) +} + +func logForIndex(prefix string, t *statistics.Table, idx *statistics.Index, ranges []*ranger.Range, actual []int64, factor float64) { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + if idx.CMSketch == nil || idx.StatsVer != statistics.Version1 { + for i, ran := range ranges { + logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) + } + return + } + for i, ran := range ranges { + rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) + // only contains range or equality query + if rangePosition == 0 || rangePosition == len(ran.LowVal) { + logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) + continue + } + equalityString, err := types.DatumsToString(ran.LowVal[:rangePosition], true) + if err != nil { + continue + } + bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) + if err != nil { + continue + } + equalityCount := idx.CMSketch.QueryBytes(bytes) + rang := ranger.Range{ + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + } + colName := idx.Info.Columns[rangePosition].Name.L + // prefer index stats over column stats + if idxHist := t.IndexStartWithColumn(colName); idxHist != nil && idxHist.Histogram.Len() > 0 { + rangeString := logForIndexRange(idxHist, &rang, -1, factor) + logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), + zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), + zap.String("range", rangeString)) + } else if colHist := t.ColumnByName(colName); colHist != nil && colHist.Histogram.Len() > 0 { + rangeString := colRangeToStr(colHist, &rang, -1, factor) + logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), + zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), + zap.String("range", rangeString)) + } else { + count, err := statistics.GetPseudoRowCountByColumnRanges(sc, float64(t.Count), []*ranger.Range{&rang}, 0) + if err == nil { + logutil.Logger(context.Background()).Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), + zap.String("equality", equalityString), zap.Uint32("expected equality", equalityCount), + zap.Stringer("range", &rang), zap.Float64("pseudo count", math.Round(count))) + } + } + } +} + +func (h *Handle) logDetailedInfo(q *statistics.QueryFeedback) { + t, ok := h.StatsCache.Load().(StatsCache)[q.PhysicalID] + if !ok { + return + } + isIndex := q.Hist.IsIndexHist() + ranges, err := q.DecodeToRanges(isIndex) + if err != nil { + logutil.Logger(context.Background()).Debug("decode to ranges failed", zap.Error(err)) + return + } + actual := make([]int64, 0, len(q.Feedback)) + for _, fb := range q.Feedback { + actual = append(actual, fb.Count) + } + logPrefix := fmt.Sprintf("[stats-feedback] %s", t.Name) + if isIndex { + idx := t.Indices[q.Hist.ID] + if idx == nil || idx.Histogram.Len() == 0 { + return + } + logForIndex(logPrefix, t, idx, ranges, actual, idx.GetIncreaseFactor(t.Count)) + } else { + c := t.Columns[q.Hist.ID] + if c == nil || c.Histogram.Len() == 0 { + return + } + logForPK(logPrefix, c, ranges, actual, c.GetIncreaseFactor(t.Count)) + } +} + +func logForPK(prefix string, c *statistics.Column, ranges []*ranger.Range, actual []int64, factor float64) { + for i, ran := range ranges { + if ran.LowVal[0].GetInt64()+1 >= ran.HighVal[0].GetInt64() { + continue + } + logutil.Logger(context.Background()).Debug(prefix, zap.String("column", c.Info.Name.O), zap.String("rangeStr", colRangeToStr(c, ran, actual[i], factor))) + } +} + +// RecalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. +func (h *Handle) RecalculateExpectCount(q *statistics.QueryFeedback) error { + t, ok := h.StatsCache.Load().(StatsCache)[q.PhysicalID] + if !ok { + return nil + } + tablePseudo := t.Pseudo || t.IsOutdated() + if !tablePseudo { + return nil + } + isIndex := q.Hist.Tp.Tp == mysql.TypeBlob + id := q.Hist.ID + if isIndex && (t.Indices[id] == nil || !t.Indices[id].NotAccurate()) { + return nil + } + if !isIndex && (t.Columns[id] == nil || !t.Columns[id].NotAccurate()) { + return nil + } + + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + ranges, err := q.DecodeToRanges(isIndex) + if err != nil { + return errors.Trace(err) + } + expected := 0.0 + if isIndex { + idx := t.Indices[id] + expected, err = idx.GetRowCount(sc, ranges, t.ModifyCount) + expected *= idx.GetIncreaseFactor(t.Count) + } else { + c := t.Columns[id] + expected, err = c.GetColumnRowCount(sc, ranges, t.ModifyCount) + expected *= c.GetIncreaseFactor(t.Count) + } + if err != nil { + return errors.Trace(err) + } + q.Expected = int64(expected) + return nil +} + +func (h *Handle) dumpRangeFeedback(sc *stmtctx.StatementContext, ran *ranger.Range, rangeCount float64, q *statistics.QueryFeedback) error { + lowIsNull := ran.LowVal[0].IsNull() + if q.Tp == statistics.IndexType { + lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) + if err != nil { + return errors.Trace(err) + } + upper, err := codec.EncodeKey(sc, nil, ran.HighVal[0]) + if err != nil { + return errors.Trace(err) + } + ran.LowVal[0].SetBytes(lower) + ran.HighVal[0].SetBytes(upper) + } else { + if !statistics.SupportColumnType(q.Hist.Tp) { + return nil + } + if ran.LowVal[0].Kind() == types.KindMinNotNull { + ran.LowVal[0] = statistics.GetMinValue(q.Hist.Tp) + } + if ran.HighVal[0].Kind() == types.KindMaxValue { + ran.HighVal[0] = statistics.GetMaxValue(q.Hist.Tp) + } + } + ranges := q.Hist.SplitRange(sc, []*ranger.Range{ran}, q.Tp == statistics.IndexType) + counts := make([]float64, 0, len(ranges)) + sum := 0.0 + for i, r := range ranges { + // Though after `SplitRange`, we may have ranges like `[l, r]`, we still use + // `betweenRowCount` to compute the estimation since the ranges of feedback are all in `[l, r)` + // form, that is to say, we ignore the exclusiveness of ranges from `SplitRange` and just use + // its result of boundary values. + count := q.Hist.BetweenRowCount(r.LowVal[0], r.HighVal[0]) + // We have to include `NullCount` of histogram for [l, r) cases where l is null because `betweenRowCount` + // does not include null values of lower bound. + if i == 0 && lowIsNull { + count += float64(q.Hist.NullCount) + } + sum += count + counts = append(counts, count) + } + if sum <= 1 { + return nil + } + // We assume that each part contributes the same error rate. + adjustFactor := rangeCount / sum + for i, r := range ranges { + q.Feedback = append(q.Feedback, statistics.Feedback{Lower: &r.LowVal[0], Upper: &r.HighVal[0], Count: int64(counts[i] * adjustFactor)}) + } + return errors.Trace(h.DumpFeedbackToKV(q)) +} + +// DumpFeedbackForIndex dumps the feedback for index. +// For queries that contains both equality and range query, we will split them and Update accordingly. +func (h *Handle) DumpFeedbackForIndex(q *statistics.QueryFeedback, t *statistics.Table) error { + idx, ok := t.Indices[q.Hist.ID] + if !ok { + return nil + } + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + if idx.CMSketch == nil || idx.StatsVer != statistics.Version1 { + return h.DumpFeedbackToKV(q) + } + ranges, err := q.DecodeToRanges(true) + if err != nil { + logutil.Logger(context.Background()).Debug("decode feedback ranges fail", zap.Error(err)) + return nil + } + for i, ran := range ranges { + rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) + // only contains range or equality query + if rangePosition == 0 || rangePosition == len(ran.LowVal) { + continue + } + + bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) + if err != nil { + logutil.Logger(context.Background()).Debug("encode keys fail", zap.Error(err)) + continue + } + equalityCount := float64(idx.CMSketch.QueryBytes(bytes)) * idx.GetIncreaseFactor(t.Count) + rang := ranger.Range{ + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + } + colName := idx.Info.Columns[rangePosition].Name.L + var rangeCount float64 + rangeFB := &statistics.QueryFeedback{PhysicalID: q.PhysicalID} + // prefer index stats over column stats + if idx := t.IndexStartWithColumn(colName); idx != nil && idx.Histogram.Len() != 0 { + rangeCount, err = t.GetRowCountByIndexRanges(sc, idx.ID, []*ranger.Range{&rang}) + rangeFB.Tp, rangeFB.Hist = statistics.IndexType, &idx.Histogram + } else if col := t.ColumnByName(colName); col != nil && col.Histogram.Len() != 0 { + rangeCount, err = t.GetRowCountByColumnRanges(sc, col.ID, []*ranger.Range{&rang}) + rangeFB.Tp, rangeFB.Hist = statistics.ColType, &col.Histogram + } else { + continue + } + if err != nil { + logutil.Logger(context.Background()).Debug("get row count by ranges fail", zap.Error(err)) + continue + } + + equalityCount, rangeCount = getNewCountForIndex(equalityCount, rangeCount, float64(t.Count), float64(q.Feedback[i].Count)) + value := types.NewBytesDatum(bytes) + q.Feedback[i] = statistics.Feedback{Lower: &value, Upper: &value, Count: int64(equalityCount)} + err = h.dumpRangeFeedback(sc, &rang, rangeCount, rangeFB) + if err != nil { + logutil.Logger(context.Background()).Debug("dump range feedback fail", zap.Error(err)) + continue + } + } + return errors.Trace(h.DumpFeedbackToKV(q)) +} + +// minAdjustFactor is the minimum adjust factor of each index feedback. +// We use it to avoid adjusting too much when the assumption of independence failed. +const minAdjustFactor = 0.7 + +// getNewCountForIndex adjust the estimated `eqCount` and `rangeCount` according to the real count. +// We assumes that `eqCount` and `rangeCount` contribute the same error rate. +func getNewCountForIndex(eqCount, rangeCount, totalCount, realCount float64) (float64, float64) { + estimate := (eqCount / totalCount) * (rangeCount / totalCount) * totalCount + if estimate <= 1 { + return eqCount, rangeCount + } + adjustFactor := math.Sqrt(realCount / estimate) + adjustFactor = math.Max(adjustFactor, minAdjustFactor) + return eqCount * adjustFactor, rangeCount * adjustFactor +} diff --git a/statistics/update_list_test.go b/statistics/handle/update_list_test.go similarity index 98% rename from statistics/update_list_test.go rename to statistics/handle/update_list_test.go index f8428be602986..3a4e7e9d22b89 100644 --- a/statistics/update_list_test.go +++ b/statistics/handle/update_list_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics +package handle import ( . "github.com/pingcap/check" diff --git a/statistics/update_test.go b/statistics/handle/update_test.go similarity index 93% rename from statistics/update_test.go rename to statistics/handle/update_test.go index 351412aa2b5cd..e9fa144196745 100644 --- a/statistics/update_test.go +++ b/statistics/handle/update_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package handle_test import ( "fmt" @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -94,7 +95,7 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { h.HandleDDLEvent(<-h.DDLEventCh()) h.HandleDDLEvent(<-h.DDLEventCh()) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) @@ -110,7 +111,7 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) @@ -125,7 +126,7 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("insert into t1 values(1, 2)") } testKit.MustExec("commit") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) @@ -141,7 +142,7 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("update t2 set c2 = c1") } testKit.MustExec("commit") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) @@ -151,7 +152,7 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("begin") testKit.MustExec("delete from t1") testKit.MustExec("commit") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(0)) @@ -163,23 +164,23 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { rs.Check(testkit.Rows("0", "0", "10", "10")) // test dump delta only when `modify count / count` is greater than the ratio. - originValue := statistics.DumpStatsDeltaRatio - statistics.DumpStatsDeltaRatio = 0.5 + originValue := handle.DumpStatsDeltaRatio + handle.DumpStatsDeltaRatio = 0.5 defer func() { - statistics.DumpStatsDeltaRatio = originValue + handle.DumpStatsDeltaRatio = originValue }() - statistics.DumpStatsDeltaRatio = 0.5 + handle.DumpStatsDeltaRatio = 0.5 for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values (1,2)") } - h.DumpStatsDeltaToKV(statistics.DumpDelta) + h.DumpStatsDeltaToKV(handle.DumpDelta) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) // not dumped testKit.MustExec("insert into t1 values (1,2)") - h.DumpStatsDeltaToKV(statistics.DumpDelta) + h.DumpStatsDeltaToKV(handle.DumpDelta) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) @@ -205,7 +206,7 @@ func (s *testStatsSuite) TestRollback(c *C) { tableInfo := tbl.Meta() h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats := h.GetTableStats(tableInfo) @@ -240,7 +241,7 @@ func (s *testStatsSuite) TestMultiSession(c *C) { h.HandleDDLEvent(<-h.DDLEventCh()) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) @@ -260,7 +261,7 @@ func (s *testStatsSuite) TestMultiSession(c *C) { testKit.Se.Close() testKit2.Se.Close() - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) @@ -289,14 +290,14 @@ func (s *testStatsSuite) TestTxnWithFailure(c *C) { for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(?, 2)", i) } - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) // have not commit c.Assert(stats1.Count, Equals, int64(0)) testKit.MustExec("commit") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) @@ -304,13 +305,13 @@ func (s *testStatsSuite) TestTxnWithFailure(c *C) { _, err = testKit.Exec("insert into t1 values(0, 2)") c.Assert(err, NotNil) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) testKit.MustExec("insert into t1 values(-1, 2)") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1+1)) @@ -336,7 +337,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { bColID := tableInfo.Columns[1].ID testKit.MustExec(`insert into t values (1, "a"), (7, "a")`) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) @@ -346,7 +347,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { } testKit.MustExec(`update t set a = a + 1, b = "aa"`) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) @@ -356,7 +357,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { } testKit.MustExec("delete from t") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) @@ -372,10 +373,10 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { testKit.MustExec("use test") testKit.MustExec("create table t (a varchar(20))") - statistics.AutoAnalyzeMinCnt = 0 + handle.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { - statistics.AutoAnalyzeMinCnt = 1000 + handle.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() @@ -393,7 +394,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { _, err = testKit.Exec("insert into t values ('ss')") c.Assert(err, IsNil) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) h.Update(is) @@ -411,7 +412,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { defer func() { h.Lease = 0 }() _, err = testKit.Exec("insert into t values ('fff')") c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) h.HandleAutoAnalyze(is) h.Update(is) @@ -421,7 +422,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { _, err = testKit.Exec("insert into t values ('fff')") c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) h.HandleAutoAnalyze(is) h.Update(is) @@ -431,7 +432,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { _, err = testKit.Exec("insert into t values ('eee')") c.Assert(err, IsNil) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) h.Update(is) @@ -471,10 +472,10 @@ func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { testKit.MustExec("create table t (a int) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") testKit.MustExec("analyze table t") - statistics.AutoAnalyzeMinCnt = 0 + handle.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { - statistics.AutoAnalyzeMinCnt = 1000 + handle.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() @@ -491,7 +492,7 @@ func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { c.Assert(stats.Count, Equals, int64(0)) testKit.MustExec("insert into t values (1)") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) stats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) @@ -514,12 +515,12 @@ func (s *testStatsSuite) TestTableAnalyzed(c *C) { h.Update(is) statsTbl := h.GetTableStats(tableInfo) - c.Assert(statistics.TableAnalyzed(statsTbl), IsFalse) + c.Assert(handle.TableAnalyzed(statsTbl), IsFalse) testKit.MustExec("analyze table t") h.Update(is) statsTbl = h.GetTableStats(tableInfo) - c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) + c.Assert(handle.TableAnalyzed(statsTbl), IsTrue) h.Clear() oriLease := h.Lease @@ -530,7 +531,7 @@ func (s *testStatsSuite) TestTableAnalyzed(c *C) { }() h.Update(is) statsTbl = h.GetTableStats(tableInfo) - c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) + c.Assert(handle.TableAnalyzed(statsTbl), IsTrue) } func (s *testStatsSuite) TestUpdateErrorRate(c *C) { @@ -553,14 +554,14 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { testKit.MustExec("insert into t values (1, 3)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2, 3)") testKit.MustExec("insert into t values (5, 3)") testKit.MustExec("insert into t values (8, 3)") testKit.MustExec("insert into t values (12, 3)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) is = s.do.InfoSchema() h.Update(is) @@ -575,7 +576,7 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where a between 1 and 10") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) @@ -587,7 +588,7 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { c.Assert(tbl.Indices[bID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where b between 2 and 10") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) @@ -597,7 +598,7 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(1)) testKit.MustExec("analyze table t") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) h.Update(is) tbl = h.GetTableStats(tblInfo) c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(0)) @@ -624,14 +625,14 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { testKit.MustExec("insert into t values (1)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2)") testKit.MustExec("insert into t values (5)") testKit.MustExec("insert into t values (8)") testKit.MustExec("insert into t values (12)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) is = s.do.InfoSchema() h.Update(is) @@ -646,7 +647,7 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where a between 1 and 10") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) @@ -768,7 +769,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) @@ -784,7 +785,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { // Feedback from limit executor may not be accurate. testKit.MustQuery("select * from t where t.a <= 5 limit 1") - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) @@ -792,7 +793,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { statistics.MaxNumberOfRanges = 0 for _, t := range tests { testKit.MustQuery(t.sql) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) } @@ -802,7 +803,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { statistics.MaxNumberOfRanges = oriNumber for _, t := range tests { testKit.MustQuery(t.sql) - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) } @@ -812,7 +813,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { for _, t := range tests { testKit.MustQuery(t.sql) } - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) testKit.MustExec("drop stats t") c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) @@ -822,7 +823,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { for _, t := range tests { testKit.MustQuery(t.sql) } - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) testKit.MustExec("drop table t") c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) @@ -891,7 +892,7 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) @@ -938,17 +939,17 @@ func (s *testStatsSuite) TestOutOfOrderUpdate(c *C) { // Simulate the case that another tidb has inserted some value, but delta info has not been dumped to kv yet. testKit.MustExec("insert into t values (2,2),(4,5)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 1 where table_id = %d", tableInfo.ID)) testKit.MustExec("delete from t") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("1")) // Now another tidb has updated the delta info. testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 3 where table_id = %d", tableInfo.ID)) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("0")) } @@ -1082,20 +1083,20 @@ func (s *testStatsSuite) TestLogDetailedInfo(c *C) { defer cleanEnv(c, s.store, s.do) oriProbability := statistics.FeedbackProbability - oriMinLogCount := statistics.MinLogScanCount - oriMinError := statistics.MinLogErrorRate + oriMinLogCount := handle.MinLogScanCount + oriMinError := handle.MinLogErrorRate oriLevel := log.GetLevel() oriLease := s.do.StatsHandle().Lease defer func() { statistics.FeedbackProbability = oriProbability - statistics.MinLogScanCount = oriMinLogCount - statistics.MinLogErrorRate = oriMinError + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriMinError s.do.StatsHandle().Lease = oriLease log.SetLevel(oriLevel) }() statistics.FeedbackProbability = 1 - statistics.MinLogScanCount = 0 - statistics.MinLogErrorRate = 0 + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 s.do.StatsHandle().Lease = 1 testKit := testkit.NewTestKit(c, s.store) @@ -1249,7 +1250,7 @@ func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { c.Assert(err, IsNil) now, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, test.now, time.UTC) c.Assert(err, IsNil) - needAnalyze, reason := statistics.NeedAnalyzeTable(test.tbl, test.limit, test.ratio, start, end, now) + needAnalyze, reason := handle.NeedAnalyzeTable(test.tbl, test.limit, test.ratio, start, end, now) c.Assert(needAnalyze, Equals, test.result) c.Assert(strings.HasPrefix(reason, test.reason), IsTrue) } @@ -1272,12 +1273,12 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { } h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (1, %d, %d)", i, i)) } - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) is := s.do.InfoSchema() h.Update(is) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1316,7 +1317,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { } for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) h.Update(is) @@ -1387,7 +1388,7 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { } for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) h.Update(is) @@ -1417,12 +1418,12 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) tests := []struct { sql string hist string @@ -1457,7 +1458,7 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) @@ -1486,12 +1487,12 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) } h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) } - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) tests := []struct { sql string hist string @@ -1515,7 +1516,7 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) @@ -1536,7 +1537,7 @@ func (s *testStatsSuite) TestLoadHistCorrelation(c *C) { testKit.MustExec("use test") testKit.MustExec("create table t(c int)") testKit.MustExec("insert into t values(1),(2),(3),(4),(5)") - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t") h.Clear() c.Assert(h.Update(s.do.InfoSchema()), IsNil) diff --git a/statistics/histogram.go b/statistics/histogram.go index ce47f988ea471..2642ccd6dd520 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -191,164 +190,18 @@ func HistogramEqual(a, b *Histogram, ignoreID bool) bool { return bytes.Equal([]byte(a.ToString(0)), []byte(b.ToString(0))) } +// constants for stats version. These const can be used for solving compatibility issue. const ( - // constants for stats version - curStatsVersion = version1 - version1 = 1 - - // constants for column flag - analyzeFlag = 1 + CurStatsVersion = Version1 + Version1 = 1 ) -func isAnalyzed(flag int64) bool { - return (flag & analyzeFlag) > 0 -} - -// SaveStatsToStorage saves the stats to storage. -func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *Histogram, cms *CMSketch, isAnalyzed int64) (err error) { - h.mu.Lock() - defer h.mu.Unlock() - ctx := context.TODO() - exec := h.mu.ctx.(sqlexec.SQLExecutor) - _, err = exec.Execute(ctx, "begin") - if err != nil { - return errors.Trace(err) - } - defer func() { - err = finishTransaction(context.Background(), exec, err) - }() - txn, err := h.mu.ctx.Txn(true) - if err != nil { - return errors.Trace(err) - } +// AnalyzeFlag is one for column flag. We can use IsAnalyzed to check whether this column is analyzed or not. +const AnalyzeFlag = 1 - version := txn.StartTS() - var sql string - // If the count is less than 0, then we do not want to update the modify count and count. - if count >= 0 { - sql = fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count) values (%d, %d, %d)", version, tableID, count) - } else { - sql = fmt.Sprintf("update mysql.stats_meta set version = %d where table_id = %d", version, tableID) - } - _, err = exec.Execute(ctx, sql) - if err != nil { - return - } - data, err := encodeCMSketch(cms) - if err != nil { - return - } - flag := 0 - if isAnalyzed == 1 { - flag = analyzeFlag - } - replaceSQL := fmt.Sprintf("replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, flag, correlation) values (%d, %d, %d, %d, %d, %d, X'%X', %d, %d, %d, %f)", - tableID, isIndex, hg.ID, hg.NDV, version, hg.NullCount, data, hg.TotColSize, curStatsVersion, flag, hg.Correlation) - _, err = exec.Execute(ctx, replaceSQL) - if err != nil { - return - } - deleteSQL := fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, hg.ID) - _, err = exec.Execute(ctx, deleteSQL) - if err != nil { - return - } - sc := h.mu.ctx.GetSessionVars().StmtCtx - for i := range hg.Buckets { - count := hg.Buckets[i].Count - if i > 0 { - count -= hg.Buckets[i-1].Count - } - var upperBound types.Datum - upperBound, err = hg.GetUpper(i).ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) - if err != nil { - return - } - var lowerBound types.Datum - lowerBound, err = hg.GetLower(i).ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) - if err != nil { - return - } - insertSQL := fmt.Sprintf("insert into mysql.stats_buckets(table_id, is_index, hist_id, bucket_id, count, repeats, lower_bound, upper_bound) values(%d, %d, %d, %d, %d, %d, X'%X', X'%X')", tableID, isIndex, hg.ID, i, count, hg.Buckets[i].Repeat, lowerBound.GetBytes(), upperBound.GetBytes()) - _, err = exec.Execute(ctx, insertSQL) - if err != nil { - return - } - } - return -} - -// SaveMetaToStorage will save stats_meta to storage. -func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error) { - h.mu.Lock() - defer h.mu.Unlock() - ctx := context.TODO() - exec := h.mu.ctx.(sqlexec.SQLExecutor) - _, err = exec.Execute(ctx, "begin") - if err != nil { - return errors.Trace(err) - } - defer func() { - err = finishTransaction(ctx, exec, err) - }() - txn, err := h.mu.ctx.Txn(true) - if err != nil { - return errors.Trace(err) - } - var sql string - version := txn.StartTS() - sql = fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count, modify_count) values (%d, %d, %d, %d)", version, tableID, count, modifyCount) - _, err = exec.Execute(ctx, sql) - return -} - -func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (*Histogram, error) { - selSQL := fmt.Sprintf("select count, repeats, lower_bound, upper_bound from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d order by bucket_id", tableID, isIndex, colID) - rows, fields, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) - if err != nil { - return nil, errors.Trace(err) - } - bucketSize := len(rows) - hg := NewHistogram(colID, distinct, nullCount, ver, tp, bucketSize, totColSize) - hg.Correlation = corr - totalCount := int64(0) - for i := 0; i < bucketSize; i++ { - count := rows[i].GetInt64(0) - repeats := rows[i].GetInt64(1) - var upperBound, lowerBound types.Datum - if isIndex == 1 { - lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) - upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) - } else { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - d := rows[i].GetDatum(2, &fields[2].Column.FieldType) - lowerBound, err = d.ConvertTo(sc, tp) - if err != nil { - return nil, errors.Trace(err) - } - d = rows[i].GetDatum(3, &fields[3].Column.FieldType) - upperBound, err = d.ConvertTo(sc, tp) - if err != nil { - return nil, errors.Trace(err) - } - } - totalCount += count - hg.AppendBucket(&lowerBound, &upperBound, totalCount, repeats) - } - hg.PreCalculateScalar() - return hg, nil -} - -func (h *Handle) columnCountFromStorage(tableID, colID int64) (int64, error) { - selSQL := fmt.Sprintf("select sum(count) from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d", tableID, 0, colID) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) - if err != nil { - return 0, errors.Trace(err) - } - if rows[0].IsNull(0) { - return 0, nil - } - return rows[0].GetMyDecimal(0).ToInt() +// IsAnalyzed checks whether this flag contains AnalyzeFlag. +func IsAnalyzed(flag int64) bool { + return (flag & AnalyzeFlag) > 0 } // ValueToString converts a possible encoded value to a formatted string. If the value is encoded, then @@ -368,7 +221,8 @@ func ValueToString(value *types.Datum, idxCols int) (string, error) { return str, nil } -func (hg *Histogram) bucketToString(bktID, idxCols int) string { +// BucketToString change the given bucket to string format. +func (hg *Histogram) BucketToString(bktID, idxCols int) string { upperVal, err := ValueToString(hg.GetUpper(bktID), idxCols) terror.Log(errors.Trace(err)) lowerVal, err := ValueToString(hg.GetLower(bktID), idxCols) @@ -385,7 +239,7 @@ func (hg *Histogram) ToString(idxCols int) string { strs = append(strs, fmt.Sprintf("column:%d ndv:%d totColSize:%d", hg.ID, hg.NDV, hg.TotColSize)) } for i := 0; i < hg.Len(); i++ { - strs = append(strs, hg.bucketToString(i, idxCols)) + strs = append(strs, hg.BucketToString(i, idxCols)) } return strings.Join(strs, "\n") } @@ -419,8 +273,8 @@ func (hg *Histogram) greaterRowCount(value types.Datum) float64 { return gtCount } -// lessRowCount estimates the row count where the column less than value. -func (hg *Histogram) lessRowCountWithBktIdx(value types.Datum) (float64, int) { +// LessRowCountWithBktIdx estimates the row count where the column less than value. +func (hg *Histogram) LessRowCountWithBktIdx(value types.Datum) (float64, int) { // All the values are null. if hg.Bounds.NumRows() == 0 { return 0, 0 @@ -446,12 +300,12 @@ func (hg *Histogram) lessRowCountWithBktIdx(value types.Datum) (float64, int) { } func (hg *Histogram) lessRowCount(value types.Datum) float64 { - result, _ := hg.lessRowCountWithBktIdx(value) + result, _ := hg.LessRowCountWithBktIdx(value) return result } -// betweenRowCount estimates the row count where column greater or equal to a and less than b. -func (hg *Histogram) betweenRowCount(a, b types.Datum) float64 { +// BetweenRowCount estimates the row count where column greater or equal to a and less than b. +func (hg *Histogram) BetweenRowCount(a, b types.Datum) float64 { lessCountA := hg.lessRowCount(a) lessCountB := hg.lessRowCount(b) // If lessCountA is not less than lessCountB, it may be that they fall to the same bucket and we cannot estimate @@ -464,13 +318,14 @@ func (hg *Histogram) betweenRowCount(a, b types.Datum) float64 { return lessCountB - lessCountA } -func (hg *Histogram) totalRowCount() float64 { +// TotalRowCount returns the total count of this histogram. +func (hg *Histogram) TotalRowCount() float64 { return hg.notNullCount() + float64(hg.NullCount) } // notNullCount indicates the count of non-null values in column histogram and single-column index histogram, // for multi-column index histogram, since we cannot define null for the row, we treat all rows as non-null, that means, -// notNullCount would return same value as totalRowCount for multi-column index histograms. +// notNullCount would return same value as TotalRowCount for multi-column index histograms. func (hg *Histogram) notNullCount() float64 { if hg.Len() == 0 { return 0 @@ -478,7 +333,7 @@ func (hg *Histogram) notNullCount() float64 { return float64(hg.Buckets[hg.Len()-1].Count) } -// mergeBuckets is used to merge every two neighbor buckets. +// mergeBuckets is used to Merge every two neighbor buckets. func (hg *Histogram) mergeBuckets(bucketIdx int) { curBuck := 0 c := chunk.NewChunkWithCapacity([]*types.FieldType{hg.Tp}, bucketIdx) @@ -499,9 +354,9 @@ func (hg *Histogram) mergeBuckets(bucketIdx int) { return } -// getIncreaseFactor will return a factor of data increasing after the last analysis. -func (hg *Histogram) getIncreaseFactor(totalCount int64) float64 { - columnCount := hg.totalRowCount() +// GetIncreaseFactor will return a factor of data increasing after the last analysis. +func (hg *Histogram) GetIncreaseFactor(totalCount int64) float64 { + columnCount := hg.TotalRowCount() if columnCount == 0 { // avoid dividing by 0 return 1.0 @@ -509,7 +364,7 @@ func (hg *Histogram) getIncreaseFactor(totalCount int64) float64 { return float64(totalCount) / columnCount } -// validRange checks if the range is valid, it is used by `SplitRange` to remove the invalid range, +// validRange checks if the range is Valid, it is used by `SplitRange` to remove the invalid range, // the possible types of range are index key range and handle key range. func validRange(sc *stmtctx.StatementContext, ran *ranger.Range, encoded bool) bool { var low, high []byte @@ -635,7 +490,8 @@ func (hg *Histogram) popFirstBucket() { hg.Bounds = c } -func (hg *Histogram) isIndexHist() bool { +// IsIndexHist checks whether current histogram is one for index. +func (hg *Histogram) IsIndexHist() bool { return hg.Tp.Tp == mysql.TypeBlob } @@ -692,7 +548,7 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, // AvgCountPerNotNullValue gets the average row count per value by the data of histogram. func (hg *Histogram) AvgCountPerNotNullValue(totalCount int64) float64 { - factor := hg.getIncreaseFactor(totalCount) + factor := hg.GetIncreaseFactor(totalCount) totalNotNull := hg.notNullCount() * factor curNDV := float64(hg.NDV) * factor if curNDV == 0 { @@ -729,12 +585,14 @@ func (e *ErrorRate) NotAccurate() bool { return e.ErrorTotal/float64(e.QueryTotal) > MaxErrorRate } -func (e *ErrorRate) update(rate float64) { +// Update updates the ErrorRate. +func (e *ErrorRate) Update(rate float64) { e.QueryTotal++ e.ErrorTotal += rate } -func (e *ErrorRate) merge(rate *ErrorRate) { +// Merge range merges two ErrorRate. +func (e *ErrorRate) Merge(rate *ErrorRate) { e.QueryTotal += rate.QueryTotal e.ErrorTotal += rate.ErrorTotal } @@ -746,7 +604,7 @@ type Column struct { PhysicalID int64 Count int64 Info *model.ColumnInfo - isHandle bool + IsHandle bool ErrorRate } @@ -754,7 +612,9 @@ func (c *Column) String() string { return c.Histogram.ToString(0) } -var histogramNeededColumns = neededColumnMap{cols: map[tableColumnID]struct{}{}} +// HistogramNeededColumns stores the columns whose Histograms need to be loaded from physical kv layer. +// Currently, we only load index/pk's Histogram from kv automatically. Columns' are loaded by needs. +var HistogramNeededColumns = neededColumnMap{cols: map[tableColumnID]struct{}{}} // IsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it // as need histogram. @@ -764,9 +624,9 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { } if c.NDV > 0 && c.Len() == 0 && sc != nil { sc.SetHistogramsNotLoad() - histogramNeededColumns.insert(tableColumnID{tableID: c.PhysicalID, columnID: c.Info.ID}) + HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) } - return c.totalRowCount() == 0 || (c.NDV > 0 && c.Len() == 0) + return c.TotalRowCount() == 0 || (c.NDV > 0 && c.Len() == 0) } func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, modifyCount int64) (float64, error) { @@ -787,8 +647,8 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, mo return c.Histogram.equalRowCount(val), nil } -// getColumnRowCount estimates the row count by a slice of Range. -func (c *Column) getColumnRowCount(sc *stmtctx.StatementContext, ranges []*ranger.Range, modifyCount int64) (float64, error) { +// GetColumnRowCount estimates the row count by a slice of Range. +func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*ranger.Range, modifyCount int64) (float64, error) { var rowCount float64 for _, rg := range ranges { cmp, err := rg.LowVal[0].CompareDatum(sc, &rg.HighVal[0]) @@ -808,7 +668,7 @@ func (c *Column) getColumnRowCount(sc *stmtctx.StatementContext, ranges []*range continue } // The interval case. - cnt := c.betweenRowCount(rg.LowVal[0], rg.HighVal[0]) + cnt := c.BetweenRowCount(rg.LowVal[0], rg.HighVal[0]) if (c.outOfRange(rg.LowVal[0]) && !rg.LowVal[0].IsNull()) || c.outOfRange(rg.HighVal[0]) { cnt += float64(modifyCount) / outOfRangeBetweenRate } @@ -834,8 +694,8 @@ func (c *Column) getColumnRowCount(sc *stmtctx.StatementContext, ranges []*range } rowCount += cnt } - if rowCount > c.totalRowCount() { - rowCount = c.totalRowCount() + if rowCount > c.TotalRowCount() { + rowCount = c.TotalRowCount() } else if rowCount < 0 { rowCount = 0 } @@ -847,7 +707,7 @@ type Index struct { Histogram *CMSketch ErrorRate - statsVer int64 // statsVer is the version of the current stats, used to maintain compatibility + StatsVer int64 // StatsVer is the version of the current stats, used to maintain compatibility Info *model.IndexInfo } @@ -857,7 +717,7 @@ func (idx *Index) String() string { // IsInvalid checks if this index is invalid. func (idx *Index) IsInvalid(collPseudo bool) bool { - return (collPseudo && idx.NotAccurate()) || idx.totalRowCount() == 0 + return (collPseudo && idx.NotAccurate()) || idx.TotalRowCount() == 0 } var nullKeyBytes, _ = codec.EncodeKey(nil, nil, types.NewDatum(nil)) @@ -878,7 +738,9 @@ func (idx *Index) equalRowCount(sc *stmtctx.StatementContext, b []byte, modifyCo return idx.Histogram.equalRowCount(val), nil } -func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*ranger.Range, modifyCount int64) (float64, error) { +// GetRowCount returns the row count of the given ranges. +// It uses the modifyCount to adjust the influence of modifications on the table. +func (idx *Index) GetRowCount(sc *stmtctx.StatementContext, indexRanges []*ranger.Range, modifyCount int64) (float64, error) { totalCount := float64(0) isSingleCol := len(idx.Info.Columns) == 1 for _, indexRange := range indexRanges { @@ -912,7 +774,7 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range } l := types.NewBytesDatum(lb) r := types.NewBytesDatum(rb) - totalCount += idx.betweenRowCount(l, r) + totalCount += idx.BetweenRowCount(l, r) lowIsNull := bytes.Equal(lb, nullKeyBytes) if (idx.outOfRange(l) && !(isSingleCol && lowIsNull)) || idx.outOfRange(r) { totalCount += float64(modifyCount) / outOfRangeBetweenRate @@ -921,8 +783,8 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range totalCount += float64(idx.NullCount) } } - if totalCount > idx.totalRowCount() { - totalCount = idx.totalRowCount() + if totalCount > idx.TotalRowCount() { + totalCount = idx.TotalRowCount() } return totalCount, nil } @@ -933,7 +795,7 @@ type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range) // TODO: Datum is not efficient, try to avoid using it here. // Also, there're redundant calculation with Selectivity(). We need to reduce it too. func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error { - cntPerVal := int64(oldHist.AvgCountPerNotNullValue(int64(oldHist.totalRowCount()))) + cntPerVal := int64(oldHist.AvgCountPerNotNullValue(int64(oldHist.TotalRowCount()))) var totCnt int64 for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(ranges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { for highRangeIdx < len(ranges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &ranges[highRangeIdx].HighVal[0]) >= 0 { @@ -977,7 +839,7 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode ranLowEncode, ranHighEncode []byte err error ) - newIndexHist := &Index{Info: idx.Info, statsVer: idx.statsVer, CMSketch: idx.CMSketch} + newIndexHist := &Index{Info: idx.Info, StatsVer: idx.StatsVer, CMSketch: idx.CMSketch} newIndexHist.Histogram = *NewHistogram(idx.ID, int64(float64(idx.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) lowBucketIdx, highBucketIdx := 0, 0 @@ -1030,7 +892,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta Count: coll.Count, } for _, node := range statsNodes { - if node.Tp == indexType { + if node.Tp == IndexType { idxHist, ok := coll.Indices[node.ID] if !ok { continue @@ -1051,7 +913,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta newCol := &Column{ PhysicalID: oldCol.PhysicalID, Info: oldCol.Info, - isHandle: oldCol.isHandle, + IsHandle: oldCol.IsHandle, CMSketch: oldCol.CMSketch, } newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) @@ -1069,7 +931,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } } } - if oldCol.isHandle { + if oldCol.IsHandle { err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, splitRanges, coll.GetRowCountByIntColumnRanges) } else { err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, splitRanges, coll.GetRowCountByColumnRanges) diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 73b7daaa64ecc..e131b143e7306 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -34,14 +34,14 @@ func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { sc := ctx.GetSessionVars().StmtCtx intCol := &Column{} intCol.Histogram = *NewHistogram(1, 30, 30, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) - intCol.isHandle = true + intCol.IsHandle = true for i := 0; i < 10; i++ { intCol.Bounds.AppendInt64(0, int64(i*3)) intCol.Bounds.AppendInt64(0, int64(i*3+2)) intCol.Buckets = append(intCol.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) } coll.Columns[1] = intCol - node := &StatsNode{ID: 1, Tp: pkType, Selectivity: 0.56} + node := &StatsNode{ID: 1, Tp: PkType, Selectivity: 0.56} node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums(2)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(5), HighVal: types.MakeDatums(6)}) @@ -76,7 +76,7 @@ num: 30 lower_bound: 27 upper_bound: 29 repeats: 0` stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 300}) stringCol.PreCalculateScalar() coll.Columns[2] = stringCol - node2 := &StatsNode{ID: 2, Tp: colType, Selectivity: 0.6} + node2 := &StatsNode{ID: 2, Tp: ColType, Selectivity: 0.6} node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa")}) node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) @@ -106,7 +106,7 @@ num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` idx.Buckets = append(idx.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) } idx.PreCalculateScalar() - node3 := &StatsNode{ID: 0, Tp: indexType, Selectivity: 0.47} + node3 := &StatsNode{ID: 0, Tp: IndexType, Selectivity: 0.47} node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(10), HighVal: types.MakeDatums(13)}) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 0af6e942311a7..fef5c54907d25 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -47,9 +47,9 @@ type StatsNode struct { // The type of the StatsNode. const ( - indexType = iota - pkType - colType + IndexType = iota + PkType + ColType ) const unknownColumnID = math.MinInt64 @@ -193,9 +193,9 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp if err != nil { return 0, nil, errors.Trace(err) } - nodes = append(nodes, &StatsNode{Tp: colType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) - if colInfo.isHandle { - nodes[len(nodes)-1].Tp = pkType + nodes = append(nodes, &StatsNode{Tp: ColType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) + if colInfo.IsHandle { + nodes[len(nodes)-1].Tp = PkType var cnt float64 cnt, err = coll.GetRowCountByIntColumnRanges(sc, id, ranges) if err != nil { @@ -228,7 +228,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp } selectivity := cnt / float64(coll.Count) nodes = append(nodes, &StatsNode{ - Tp: indexType, + Tp: IndexType, ID: id, mask: maskCovered, Ranges: ranges, @@ -299,7 +299,7 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { mask := int64(math.MaxInt64) for { // Choose the index that covers most. - bestID, bestCount, bestTp, bestNumCols, bestMask := -1, 0, colType, 0, int64(0) + bestID, bestCount, bestTp, bestNumCols, bestMask := -1, 0, ColType, 0, int64(0) for i, set := range nodes { if marked[i] { continue @@ -314,7 +314,7 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { // (1): The stats type, always prefer the primary key or index. // (2): The number of expression that it covers, the more the better. // (3): The number of columns that it contains, the less the better. - if (bestTp == colType && set.Tp != colType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { + if (bestTp == ColType && set.Tp != ColType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { bestID, bestCount, bestTp, bestNumCols, bestMask = i, bits, set.Tp, set.numCols, curMask } } @@ -322,7 +322,7 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { break } - // update the mask, remove the bit that nodes[bestID].mask has. + // Update the mask, remove the bit that nodes[bestID].mask has. mask &^= bestMask newBlocks = append(newBlocks, nodes[bestID]) diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 338b4e3039c6d..efe34266f3c5d 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -18,25 +18,130 @@ import ( "math" "os" "runtime/pprof" + "strings" "testing" "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) const eps = 1e-9 +var _ = Suite(&testStatsSuite{}) + +type testStatsSuite struct { + store kv.Storage + do *domain.Domain + hook *logHook +} + +func (s *testStatsSuite) SetUpSuite(c *C) { + testleak.BeforeTest() + // Add the hook here to avoid data race. + s.registerHook() + var err error + s.store, s.do, err = newStoreWithBootstrap(0) + c.Assert(err, IsNil) +} + +func (s *testStatsSuite) TearDownSuite(c *C) { + s.do.Close() + s.store.Close() + testleak.AfterTest(c)() +} + +func (s *testStatsSuite) registerHook() { + conf := &log.Config{Level: "info", File: log.FileLogConfig{}} + _, r, _ := log.InitLogger(conf) + s.hook = &logHook{r.Core, ""} + lg := zap.New(s.hook) + log.ReplaceGlobals(lg, r) +} + +type logHook struct { + zapcore.Core + results string +} + +func (h *logHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { + message := entry.Message + if idx := strings.Index(message, "[stats"); idx != -1 { + h.results = h.results + message + for _, f := range fields { + h.results = h.results + ", " + f.Key + "=" + h.field2String(f) + } + } + return nil +} + +func (h *logHook) field2String(field zapcore.Field) string { + switch field.Type { + case zapcore.StringType: + return field.String + case zapcore.Int64Type, zapcore.Int32Type, zapcore.Uint32Type: + return fmt.Sprintf("%v", field.Integer) + case zapcore.Float64Type: + return fmt.Sprintf("%v", math.Float64frombits(uint64(field.Integer))) + case zapcore.StringerType: + return field.Interface.(fmt.Stringer).String() + } + return "not support" +} + +func (h *logHook) Check(e zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if h.Enabled(e.Level) { + return ce.AddCore(e, h) + } + return ce +} + +func newStoreWithBootstrap(statsLease time.Duration) (kv.Storage, *domain.Domain, error) { + store, err := mockstore.NewMockTikvStore() + if err != nil { + return nil, nil, errors.Trace(err) + } + session.SetSchemaLease(0) + session.SetStatsLease(statsLease) + domain.RunAutoAnalyze = false + do, err := session.BootstrapSession(store) + do.SetStatsUpdating(true) + return store, do, errors.Trace(err) +} + +func cleanEnv(c *C, store kv.Storage, do *domain.Domain) { + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + r := tk.MustQuery("show tables") + for _, tb := range r.Rows() { + tableName := tb[0] + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } + tk.MustExec("delete from mysql.stats_meta") + tk.MustExec("delete from mysql.stats_histograms") + tk.MustExec("delete from mysql.stats_buckets") + do.StatsHandle().Clear() +} + // generateIntDatum will generate a datum slice, every dimension is begin from 0, end with num - 1. // If dimension is x, num is y, the total number of datum is y^x. And This slice is sorted. func (s *testStatsSuite) generateIntDatum(dimension, num int) ([]types.Datum, error) { @@ -242,12 +347,12 @@ func (s *testStatsSuite) TestEstimationForUnknownValues(c *C) { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } h := s.do.StatsHandle() - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) testKit.MustExec("analyze table t") for i := 0; i < 10; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i+10)) } - h.DumpStatsDeltaToKV(statistics.DumpAll) + h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(s.do.InfoSchema()), IsNil) table, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) @@ -369,7 +474,7 @@ func (s *testStatsSuite) TestColumnIndexNullEstimation(c *C) { testKit.MustExec("create table t(a int, b int, c int, index idx_b(b), index idx_c_a(c, a))") testKit.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null);") h := s.do.StatsHandle() - c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t") testKit.MustQuery(`explain select b from t where b is null`).Check(testkit.Rows( "IndexReader_6 4.00 root index:IndexScan_5", diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 316dbd90f24fd..4ee497514d156 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -265,7 +265,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(count, Equals, 0.0) count = col.equalRowCount(types.NewIntDatum(200000000)) c.Check(count, Equals, 0.0) - count = col.betweenRowCount(types.NewIntDatum(3000), types.NewIntDatum(3500)) + count = col.BetweenRowCount(types.NewIntDatum(3000), types.NewIntDatum(3500)) c.Check(int(count), Equals, 4994) count = col.lessRowCount(types.NewIntDatum(1)) c.Check(int(count), Equals, 9) @@ -295,9 +295,9 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(int(count), Equals, 1) count = col.lessRowCount(encodeKey(types.NewIntDatum(20000))) c.Check(int(count), Equals, 19999) - count = col.betweenRowCount(encodeKey(types.NewIntDatum(30000)), encodeKey(types.NewIntDatum(35000))) + count = col.BetweenRowCount(encodeKey(types.NewIntDatum(30000)), encodeKey(types.NewIntDatum(35000))) c.Check(int(count), Equals, 4999) - count = col.betweenRowCount(encodeKey(types.MinNotNullDatum()), encodeKey(types.NewIntDatum(0))) + count = col.BetweenRowCount(encodeKey(types.MinNotNullDatum()), encodeKey(types.NewIntDatum(0))) c.Check(int(count), Equals, 0) count = col.lessRowCount(encodeKey(types.NewIntDatum(0))) c.Check(int(count), Equals, 0) @@ -312,7 +312,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(int(count), Equals, 1) count = col.lessRowCount(types.NewIntDatum(20000)) c.Check(int(count), Equals, 20000) - count = col.betweenRowCount(types.NewIntDatum(30000), types.NewIntDatum(35000)) + count = col.BetweenRowCount(types.NewIntDatum(30000), types.NewIntDatum(35000)) c.Check(int(count), Equals, 5000) count = col.greaterRowCount(types.NewIntDatum(1001)) c.Check(int(count), Equals, 98998) @@ -398,7 +398,7 @@ func (s *testStatisticsSuite) TestMergeHistogram(c *C) { c.Assert(err, IsNil) c.Assert(h.NDV, Equals, t.ndv) c.Assert(h.Len(), Equals, t.bucketNum) - c.Assert(int64(h.totalRowCount()), Equals, t.leftNum+t.rightNum) + c.Assert(int64(h.TotalRowCount()), Equals, t.leftNum+t.rightNum) expectLower := types.NewIntDatum(t.leftLower) cmp, err := h.GetLower(0).CompareDatum(sc, &expectLower) c.Assert(err, IsNil) @@ -455,7 +455,7 @@ func (s *testStatisticsSuite) TestColumnRange(c *C) { col := &Column{Histogram: *hg, CMSketch: buildCMSketch(s.rc.(*recordSet).data), Info: &model.ColumnInfo{}} tbl := &Table{ HistColl: HistColl{ - Count: int64(col.totalRowCount()), + Count: int64(col.TotalRowCount()), Columns: make(map[int64]*Column), }, } @@ -524,7 +524,7 @@ func (s *testStatisticsSuite) TestIntColumnRanges(c *C) { col := &Column{Histogram: *hg, Info: &model.ColumnInfo{}} tbl := &Table{ HistColl: HistColl{ - Count: int64(col.totalRowCount()), + Count: int64(col.TotalRowCount()), Columns: make(map[int64]*Column), }, } @@ -616,7 +616,7 @@ func (s *testStatisticsSuite) TestIndexRanges(c *C) { idx := &Index{Histogram: *hg, CMSketch: cms, Info: idxInfo} tbl := &Table{ HistColl: HistColl{ - Count: int64(idx.totalRowCount()), + Count: int64(idx.TotalRowCount()), Indices: make(map[int64]*Index), }, } diff --git a/statistics/table.go b/statistics/table.go index 9f9b3bb3977e1..ce666369d845f 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -14,7 +14,6 @@ package statistics import ( - "context" "fmt" "math" "strings" @@ -26,11 +25,8 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" - "go.uber.org/zap" ) const ( @@ -45,11 +41,14 @@ const ( outOfRangeBetweenRate = 100 ) +// PseudoVersion means the pseudo statistics version is 0. +const PseudoVersion uint64 = 0 + // Table represents statistics for a table. type Table struct { HistColl Version uint64 - name string + Name string } // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. @@ -69,7 +68,8 @@ type HistColl struct { ModifyCount int64 // Total modify count in a table. } -func (t *Table) copy() *Table { +// Copy copies the current table. +func (t *Table) Copy() *Table { newHistColl := HistColl{ PhysicalID: t.PhysicalID, HavePhysicalID: t.HavePhysicalID, @@ -88,189 +88,11 @@ func (t *Table) copy() *Table { nt := &Table{ HistColl: newHistColl, Version: t.Version, - name: t.name, + Name: t.Name, } return nt } -func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*CMSketch, error) { - selSQL := fmt.Sprintf("select cm_sketch from mysql.stats_histograms where table_id = %d and is_index = %d and hist_id = %d", tblID, isIndex, histID) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) - if err != nil { - return nil, errors.Trace(err) - } - if len(rows) == 0 { - return nil, nil - } - return decodeCMSketch(rows[0].GetBytes(0)) -} - -func (h *Handle) indexStatsFromStorage(row chunk.Row, table *Table, tableInfo *model.TableInfo) error { - histID := row.GetInt64(2) - distinct := row.GetInt64(3) - histVer := row.GetUint64(4) - nullCount := row.GetInt64(5) - idx := table.Indices[histID] - errorRate := ErrorRate{} - if isAnalyzed(row.GetInt64(8)) { - h.mu.Lock() - h.mu.rateMap.clear(table.PhysicalID, histID, true) - h.mu.Unlock() - } else if idx != nil { - errorRate = idx.ErrorRate - } - for _, idxInfo := range tableInfo.Indices { - if histID != idxInfo.ID { - continue - } - if idx == nil || idx.LastUpdateVersion < histVer { - hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) - if err != nil { - return errors.Trace(err) - } - cms, err := h.cmSketchFromStorage(table.PhysicalID, 1, idxInfo.ID) - if err != nil { - return errors.Trace(err) - } - idx = &Index{Histogram: *hg, CMSketch: cms, Info: idxInfo, ErrorRate: errorRate, statsVer: row.GetInt64(7)} - } - break - } - if idx != nil { - table.Indices[histID] = idx - } else { - logutil.Logger(context.Background()).Debug("we cannot find index id in table info. It may be deleted.", zap.Int64("indexID", histID), zap.String("table", tableInfo.Name.O)) - } - return nil -} - -func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo *model.TableInfo, loadAll bool) error { - histID := row.GetInt64(2) - distinct := row.GetInt64(3) - histVer := row.GetUint64(4) - nullCount := row.GetInt64(5) - totColSize := row.GetInt64(6) - correlation := row.GetFloat64(9) - col := table.Columns[histID] - errorRate := ErrorRate{} - if isAnalyzed(row.GetInt64(8)) { - h.mu.Lock() - h.mu.rateMap.clear(table.PhysicalID, histID, false) - h.mu.Unlock() - } else if col != nil { - errorRate = col.ErrorRate - } - for _, colInfo := range tableInfo.Columns { - if histID != colInfo.ID { - continue - } - isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) - // We will not load buckets if: - // 1. Lease > 0, and: - // 2. this column is not handle, and: - // 3. the column doesn't has buckets before, and: - // 4. loadAll is false. - notNeedLoad := h.Lease > 0 && - !isHandle && - (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) && - !loadAll - if notNeedLoad { - count, err := h.columnCountFromStorage(table.PhysicalID, histID) - if err != nil { - return errors.Trace(err) - } - col = &Column{ - PhysicalID: table.PhysicalID, - Histogram: *NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), - Info: colInfo, - Count: count + nullCount, - ErrorRate: errorRate, - isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), - } - col.Histogram.Correlation = correlation - break - } - if col == nil || col.LastUpdateVersion < histVer || loadAll { - hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) - if err != nil { - return errors.Trace(err) - } - cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID) - if err != nil { - return errors.Trace(err) - } - col = &Column{ - PhysicalID: table.PhysicalID, - Histogram: *hg, - Info: colInfo, - CMSketch: cms, - Count: int64(hg.totalRowCount()), - ErrorRate: errorRate, - isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), - } - break - } - if col.TotColSize != totColSize { - newCol := *col - newCol.TotColSize = totColSize - col = &newCol - } - break - } - if col != nil { - table.Columns[col.ID] = col - } else { - // If we didn't find a Column or Index in tableInfo, we won't load the histogram for it. - // But don't worry, next lease the ddl will be updated, and we will load a same table for two times to - // avoid error. - logutil.Logger(context.Background()).Debug("we cannot find column in table info now. It may be deleted", zap.Int64("colID", histID), zap.String("table", tableInfo.Name.O)) - } - return nil -} - -// tableStatsFromStorage loads table stats info from storage. -func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool) (*Table, error) { - table, ok := h.statsCache.Load().(statsCache)[physicalID] - // If table stats is pseudo, we also need to copy it, since we will use the column stats when - // the average error rate of it is small. - if !ok { - histColl := HistColl{ - PhysicalID: physicalID, - HavePhysicalID: true, - Columns: make(map[int64]*Column, len(tableInfo.Columns)), - Indices: make(map[int64]*Index, len(tableInfo.Indices)), - } - table = &Table{ - HistColl: histColl, - } - } else { - // We copy it before writing to avoid race. - table = table.copy() - } - table.Pseudo = false - selSQL := fmt.Sprintf("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation from mysql.stats_histograms where table_id = %d", physicalID) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) - if err != nil { - return nil, errors.Trace(err) - } - // Check deleted table. - if len(rows) == 0 { - return nil, nil - } - for _, row := range rows { - if row.GetInt64(1) > 0 { - if err := h.indexStatsFromStorage(row, table, tableInfo); err != nil { - return nil, errors.Trace(err) - } - } else { - if err := h.columnStatsFromStorage(row, table, tableInfo, loadAll); err != nil { - return nil, errors.Trace(err) - } - } - } - return table, nil -} - // String implements Stringer interface. func (t *Table) String() string { strs := make([]string, 0, len(t.Columns)+1) @@ -284,7 +106,8 @@ func (t *Table) String() string { return strings.Join(strs, "\n") } -func (t *Table) indexStartWithColumn(colName string) *Index { +// IndexStartWithColumn finds the first index whose first column is the given column. +func (t *Table) IndexStartWithColumn(colName string) *Index { for _, index := range t.Indices { if index.Info.Columns[0].Name.L == colName { return index @@ -293,7 +116,8 @@ func (t *Table) indexStartWithColumn(colName string) *Index { return nil } -func (t *Table) columnByName(colName string) *Column { +// ColumnByName finds the statistics.Column for the given column. +func (t *Table) ColumnByName(colName string) *Column { for _, c := range t.Columns { if c.Info.Name.L == colName { return c @@ -303,8 +127,8 @@ func (t *Table) columnByName(colName string) *Column { } type tableColumnID struct { - tableID int64 - columnID int64 + TableID int64 + ColumnID int64 } type neededColumnMap struct { @@ -312,7 +136,7 @@ type neededColumnMap struct { cols map[tableColumnID]struct{} } -func (n *neededColumnMap) allCols() []tableColumnID { +func (n *neededColumnMap) AllCols() []tableColumnID { n.m.Lock() keys := make([]tableColumnID, 0, len(n.cols)) for key := range n.cols { @@ -328,7 +152,7 @@ func (n *neededColumnMap) insert(col tableColumnID) { n.m.Unlock() } -func (n *neededColumnMap) delete(col tableColumnID) { +func (n *neededColumnMap) Delete(col tableColumnID) { n.m.Lock() delete(n.cols, col) n.m.Unlock() @@ -352,7 +176,7 @@ func (t *Table) ColumnGreaterRowCount(sc *stmtctx.StatementContext, value types. if !ok || c.IsInvalid(sc, t.Pseudo) { return float64(t.Count) / pseudoLessRate } - return c.greaterRowCount(value) * c.getIncreaseFactor(t.Count) + return c.greaterRowCount(value) * c.GetIncreaseFactor(t.Count) } // ColumnLessRowCount estimates the row count where the column less than value. Note that null values are not counted. @@ -361,7 +185,7 @@ func (t *Table) ColumnLessRowCount(sc *stmtctx.StatementContext, value types.Dat if !ok || c.IsInvalid(sc, t.Pseudo) { return float64(t.Count) / pseudoLessRate } - return c.lessRowCount(value) * c.getIncreaseFactor(t.Count) + return c.lessRowCount(value) * c.GetIncreaseFactor(t.Count) } // ColumnBetweenRowCount estimates the row count where column greater or equal to a and less than b. @@ -370,11 +194,11 @@ func (t *Table) ColumnBetweenRowCount(sc *stmtctx.StatementContext, a, b types.D if !ok || c.IsInvalid(sc, t.Pseudo) { return float64(t.Count) / pseudoBetweenRate } - count := c.betweenRowCount(a, b) + count := c.BetweenRowCount(a, b) if a.IsNull() { count += float64(c.NullCount) } - return count * c.getIncreaseFactor(t.Count) + return count * c.GetIncreaseFactor(t.Count) } // ColumnEqualRowCount estimates the row count where the column equals to value. @@ -384,7 +208,7 @@ func (t *Table) ColumnEqualRowCount(sc *stmtctx.StatementContext, value types.Da return float64(t.Count) / pseudoEqualRate, nil } result, err := c.equalRowCount(sc, value, t.ModifyCount) - result *= c.getIncreaseFactor(t.Count) + result *= c.GetIncreaseFactor(t.Count) return result, errors.Trace(err) } @@ -400,8 +224,8 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sc *stmtctx.StatementContext, } return getPseudoRowCountByUnsignedIntRanges(intRanges, float64(coll.Count)), nil } - result, err := c.getColumnRowCount(sc, intRanges, coll.ModifyCount) - result *= c.getIncreaseFactor(coll.Count) + result, err := c.GetColumnRowCount(sc, intRanges, coll.ModifyCount) + result *= c.GetIncreaseFactor(coll.Count) return result, errors.Trace(err) } @@ -409,10 +233,10 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sc *stmtctx.StatementContext, func (coll *HistColl) GetRowCountByColumnRanges(sc *stmtctx.StatementContext, colID int64, colRanges []*ranger.Range) (float64, error) { c, ok := coll.Columns[colID] if !ok || c.IsInvalid(sc, coll.Pseudo) { - return getPseudoRowCountByColumnRanges(sc, float64(coll.Count), colRanges, 0) + return GetPseudoRowCountByColumnRanges(sc, float64(coll.Count), colRanges, 0) } - result, err := c.getColumnRowCount(sc, colRanges, coll.ModifyCount) - result *= c.getIncreaseFactor(coll.Count) + result, err := c.GetColumnRowCount(sc, colRanges, coll.ModifyCount) + result *= c.GetIncreaseFactor(coll.Count) return result, errors.Trace(err) } @@ -428,12 +252,12 @@ func (coll *HistColl) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idx } var result float64 var err error - if idx.CMSketch != nil && idx.statsVer == version1 { + if idx.CMSketch != nil && idx.StatsVer == Version1 { result, err = coll.getIndexRowCount(sc, idxID, indexRanges) } else { - result, err = idx.getRowCount(sc, indexRanges, coll.ModifyCount) + result, err = idx.GetRowCount(sc, indexRanges, coll.ModifyCount) } - result *= idx.getIncreaseFactor(coll.Count) + result *= idx.GetIncreaseFactor(coll.Count) return result, errors.Trace(err) } @@ -442,9 +266,9 @@ func (t *Table) PseudoAvgCountPerValue() float64 { return float64(t.Count) / pseudoEqualRate } -// getOrdinalOfRangeCond gets the ordinal of the position range condition, +// GetOrdinalOfRangeCond gets the ordinal of the position range condition, // if not exist, it returns the end position. -func getOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int { +func GetOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int { for i := range ran.LowVal { a, b := ran.LowVal[i], ran.HighVal[i] cmp, err := a.CompareDatum(sc, &b) @@ -529,12 +353,12 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idxID int64 idx := coll.Indices[idxID] totalCount := float64(0) for _, ran := range indexRanges { - rangePosition := getOrdinalOfRangeCond(sc, ran) + rangePosition := GetOrdinalOfRangeCond(sc, ran) // If first one is range, just use the previous way to estimate; if it is [NULL, NULL] range // on single-column index, use previous way as well, because CMSketch does not contain null // values in this case. if rangePosition == 0 || isSingleColIdxNullRange(idx, ran) { - count, err := idx.getRowCount(sc, []*ranger.Range{ran}, coll.ModifyCount) + count, err := idx.GetRowCount(sc, []*ranger.Range{ran}, coll.ModifyCount) if err != nil { return 0, errors.Trace(err) } @@ -553,13 +377,13 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idxID int64 // so we use heuristic methods to estimate the selectivity. if idx.NDV > 0 && len(ran.LowVal) == len(idx.Info.Columns) && rangePosition == len(ran.LowVal) { // for equality queries - selectivity = float64(coll.ModifyCount) / float64(idx.NDV) / idx.totalRowCount() + selectivity = float64(coll.ModifyCount) / float64(idx.NDV) / idx.TotalRowCount() } else { // for range queries - selectivity = float64(coll.ModifyCount) / outOfRangeBetweenRate / idx.totalRowCount() + selectivity = float64(coll.ModifyCount) / outOfRangeBetweenRate / idx.TotalRowCount() } } else { - selectivity = float64(idx.CMSketch.QueryBytes(bytes)) / float64(idx.totalRowCount()) + selectivity = float64(idx.CMSketch.QueryBytes(bytes)) / float64(idx.TotalRowCount()) } // use histogram to estimate the range condition if rangePosition != len(ran.LowVal) { @@ -587,12 +411,12 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idxID int64 if err != nil { return 0, errors.Trace(err) } - selectivity = selectivity * count / float64(idx.totalRowCount()) + selectivity = selectivity * count / float64(idx.TotalRowCount()) } - totalCount += selectivity * float64(idx.totalRowCount()) + totalCount += selectivity * float64(idx.TotalRowCount()) } - if totalCount > idx.totalRowCount() { - totalCount = idx.totalRowCount() + if totalCount > idx.TotalRowCount() { + totalCount = idx.TotalRowCount() } return totalCount, nil } @@ -617,7 +441,7 @@ func PseudoTable(tblInfo *model.TableInfo) *Table { t.Columns[col.ID] = &Column{ PhysicalID: fakePhysicalID, Info: col, - isHandle: tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag), + IsHandle: tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag), } } } @@ -648,7 +472,7 @@ func getPseudoRowCountByIndexRanges(sc *stmtctx.StatementContext, indexRanges [] if i >= len(indexRange.LowVal) { i = len(indexRange.LowVal) - 1 } - rowCount, err := getPseudoRowCountByColumnRanges(sc, tableRowCount, []*ranger.Range{indexRange}, i) + rowCount, err := GetPseudoRowCountByColumnRanges(sc, tableRowCount, []*ranger.Range{indexRange}, i) if err != nil { return 0, errors.Trace(err) } @@ -666,7 +490,8 @@ func getPseudoRowCountByIndexRanges(sc *stmtctx.StatementContext, indexRanges [] return totalCount, nil } -func getPseudoRowCountByColumnRanges(sc *stmtctx.StatementContext, tableRowCount float64, columnRanges []*ranger.Range, colIdx int) (float64, error) { +// GetPseudoRowCountByColumnRanges calculate the row count by the ranges if there's no statistics information for this column. +func GetPseudoRowCountByColumnRanges(sc *stmtctx.StatementContext, tableRowCount float64, columnRanges []*ranger.Range, colIdx int) (float64, error) { var rowCount float64 var err error for _, ran := range columnRanges { diff --git a/store/helper/helper.go b/store/helper/helper.go index ee1ff6284eff9..392a67404704b 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -14,17 +14,20 @@ package helper import ( + "bytes" "context" "encoding/json" + "math" "net/http" "time" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/domain" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -69,19 +72,19 @@ func (h *Helper) GetMvccByEncodedKey(encodedKey kv.Key) (*kvrpcpb.MvccGetByKeyRe // StoreHotRegionInfos records all hog region stores. // it's the response of PD. type StoreHotRegionInfos struct { - AsPeer map[uint64]*hotRegionsStat `json:"as_peer"` - AsLeader map[uint64]*hotRegionsStat `json:"as_leader"` + AsPeer map[uint64]*HotRegionsStat `json:"as_peer"` + AsLeader map[uint64]*HotRegionsStat `json:"as_leader"` } -// hotRegions records echo store's hot region. +// HotRegionsStat records echo store's hot region. // it's the response of PD. -type hotRegionsStat struct { - RegionsStat []regionStat `json:"statistics"` +type HotRegionsStat struct { + RegionsStat []RegionStat `json:"statistics"` } -// regionStat records each hot region's statistics +// RegionStat records each hot region's statistics // it's the response of PD. -type regionStat struct { +type RegionStat struct { RegionID uint64 `json:"region_id"` FlowBytes uint64 `json:"flow_bytes"` HotDegree int `json:"hot_degree"` @@ -94,9 +97,18 @@ type RegionMetric struct { Count int `json:"region_count"` } +// ScrapeHotInfo gets the needed hot region information by the url given. +func (h *Helper) ScrapeHotInfo(rw string, allSchemas []*model.DBInfo) (map[TblIndex]RegionMetric, error) { + regionMetrics, err := h.FetchHotRegion(rw) + if err != nil { + return nil, err + } + return h.FetchRegionTableIndex(regionMetrics, allSchemas) +} + // FetchHotRegion fetches the hot region information from PD's http api. func (h *Helper) FetchHotRegion(rw string) (map[uint64]RegionMetric, error) { - etcd, ok := h.Store.(domain.EtcdBackend) + etcd, ok := h.Store.(tikv.EtcdBackend) if !ok { return nil, errors.WithStack(errors.New("not implemented")) } @@ -133,3 +145,223 @@ func (h *Helper) FetchHotRegion(rw string) (map[uint64]RegionMetric, error) { } return metric, nil } + +// TblIndex stores the things to index one table. +type TblIndex struct { + DbName string + TableName string + TableID int64 + IndexName string + IndexID int64 +} + +// FrameItem includes a index's or record's meta data with table's info. +type FrameItem struct { + DBName string `json:"db_name"` + TableName string `json:"table_name"` + TableID int64 `json:"table_id"` + IsRecord bool `json:"is_record"` + RecordID int64 `json:"record_id,omitempty"` + IndexName string `json:"index_name,omitempty"` + IndexID int64 `json:"index_id,omitempty"` + IndexValues []string `json:"index_values,omitempty"` +} + +// RegionFrameRange contains a frame range info which the region covered. +type RegionFrameRange struct { + First *FrameItem // start frame of the region + Last *FrameItem // end frame of the region + region *tikv.KeyLocation // the region +} + +// FetchRegionTableIndex constructs a map that maps a table to its hot region information by the given raw hot region metrics. +func (h *Helper) FetchRegionTableIndex(metrics map[uint64]RegionMetric, allSchemas []*model.DBInfo) (map[TblIndex]RegionMetric, error) { + idxMetrics := make(map[TblIndex]RegionMetric) + for regionID, regionMetric := range metrics { + region, err := h.RegionCache.LocateRegionByID(tikv.NewBackoffer(context.Background(), 500), regionID) + if err != nil { + logutil.Logger(context.Background()).Error("locate region failed", zap.Error(err)) + continue + } + + hotRange, err := NewRegionFrameRange(region) + if err != nil { + return nil, err + } + + f := h.FindTableIndexOfRegion(allSchemas, hotRange) + if f != nil { + idx := TblIndex{ + DbName: f.DBName, + TableName: f.TableName, + TableID: f.TableID, + IndexName: f.IndexName, + IndexID: f.IndexID, + } + metric, exists := idxMetrics[idx] + if !exists { + metric = regionMetric + metric.Count++ + idxMetrics[idx] = metric + } else { + metric.FlowBytes += regionMetric.FlowBytes + if metric.MaxHotDegree < regionMetric.MaxHotDegree { + metric.MaxHotDegree = regionMetric.MaxHotDegree + } + metric.Count++ + } + } + } + + return idxMetrics, nil +} + +// FindTableIndexOfRegion finds what table is involved in this hot region. And constructs the new frame item for future use. +func (h *Helper) FindTableIndexOfRegion(allSchemas []*model.DBInfo, hotRange *RegionFrameRange) *FrameItem { + for _, db := range allSchemas { + for _, tbl := range db.Tables { + if f := hotRange.GetRecordFrame(tbl.ID, db.Name.O, tbl.Name.O); f != nil { + return f + } + for _, idx := range tbl.Indices { + if f := hotRange.GetIndexFrame(tbl.ID, idx.ID, db.Name.O, tbl.Name.O, idx.Name.O); f != nil { + return f + } + } + } + } + return nil +} + +// NewRegionFrameRange init a NewRegionFrameRange with region info. +func NewRegionFrameRange(region *tikv.KeyLocation) (idxRange *RegionFrameRange, err error) { + var first, last *FrameItem + // check and init first frame + if len(region.StartKey) > 0 { + first, err = NewFrameItemFromRegionKey(region.StartKey) + if err != nil { + return + } + } else { // empty startKey means start with -infinite + first = &FrameItem{ + IndexID: int64(math.MinInt64), + IsRecord: false, + TableID: int64(math.MinInt64), + } + } + + // check and init last frame + if len(region.EndKey) > 0 { + last, err = NewFrameItemFromRegionKey(region.EndKey) + if err != nil { + return + } + } else { // empty endKey means end with +infinite + last = &FrameItem{ + TableID: int64(math.MaxInt64), + IndexID: int64(math.MaxInt64), + IsRecord: true, + } + } + + idxRange = &RegionFrameRange{ + region: region, + First: first, + Last: last, + } + return idxRange, nil +} + +// NewFrameItemFromRegionKey creates a FrameItem with region's startKey or endKey, +// returns err when key is illegal. +func NewFrameItemFromRegionKey(key []byte) (frame *FrameItem, err error) { + frame = &FrameItem{} + frame.TableID, frame.IndexID, frame.IsRecord, err = tablecodec.DecodeKeyHead(key) + if err == nil { + if frame.IsRecord { + _, frame.RecordID, err = tablecodec.DecodeRecordKey(key) + } else { + _, _, frame.IndexValues, err = tablecodec.DecodeIndexKey(key) + } + logutil.Logger(context.Background()).Warn("decode region key failed", zap.ByteString("key", key), zap.Error(err)) + // Ignore decode errors. + err = nil + return + } + if bytes.HasPrefix(key, tablecodec.TablePrefix()) { + // If SplitTable is enabled, the key may be `t{id}`. + if len(key) == tablecodec.TableSplitKeyLen { + frame.TableID = tablecodec.DecodeTableID(key) + return frame, nil + } + return nil, errors.Trace(err) + } + + // key start with tablePrefix must be either record key or index key + // That's means table's record key and index key are always together + // in the continuous interval. And for key with prefix smaller than + // tablePrefix, is smaller than all tables. While for key with prefix + // bigger than tablePrefix, means is bigger than all tables. + err = nil + if bytes.Compare(key, tablecodec.TablePrefix()) < 0 { + frame.TableID = math.MinInt64 + frame.IndexID = math.MinInt64 + frame.IsRecord = false + return + } + // bigger than tablePrefix, means is bigger than all tables. + frame.TableID = math.MaxInt64 + frame.TableID = math.MaxInt64 + frame.IsRecord = true + return +} + +// GetRecordFrame returns the record frame of a table. If the table's records +// are not covered by this frame range, it returns nil. +func (r *RegionFrameRange) GetRecordFrame(tableID int64, dbName, tableName string) *FrameItem { + if tableID == r.First.TableID && r.First.IsRecord { + r.First.DBName, r.First.TableName = dbName, tableName + return r.First + } + if tableID == r.Last.TableID && r.Last.IsRecord { + r.Last.DBName, r.Last.TableName = dbName, tableName + return r.Last + } + + if tableID >= r.First.TableID && tableID < r.Last.TableID { + return &FrameItem{ + DBName: dbName, + TableName: tableName, + TableID: tableID, + IsRecord: true, + } + } + return nil +} + +// GetIndexFrame returns the indnex frame of a table. If the table's indices are +// not covered by this frame range, it returns nil. +func (r *RegionFrameRange) GetIndexFrame(tableID, indexID int64, dbName, tableName, indexName string) *FrameItem { + if tableID == r.First.TableID && !r.First.IsRecord && indexID == r.First.IndexID { + r.First.DBName, r.First.TableName, r.First.IndexName = dbName, tableName, indexName + return r.First + } + if tableID == r.Last.TableID && indexID == r.Last.IndexID { + r.Last.DBName, r.Last.TableName, r.Last.IndexName = dbName, tableName, indexName + return r.Last + } + + greaterThanFirst := tableID > r.First.TableID || (tableID == r.First.TableID && !r.First.IsRecord && indexID > r.First.IndexID) + lessThanLast := tableID < r.Last.TableID || (tableID == r.Last.TableID && (r.Last.IsRecord || indexID < r.Last.IndexID)) + if greaterThanFirst && lessThanLast { + return &FrameItem{ + DBName: dbName, + TableName: tableName, + TableID: tableID, + IsRecord: false, + IndexName: indexName, + IndexID: indexID, + } + } + return nil +} diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index 697a46c5b5b81..aa803a0ae3626 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package helper +package helper_test import ( "crypto/tls" @@ -24,9 +24,11 @@ import ( "github.com/gorilla/mux" . "github.com/pingcap/check" "github.com/pingcap/log" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/util/pdapi" "go.uber.org/zap" ) @@ -71,11 +73,11 @@ func (s *HelperTestSuite) SetUpSuite(c *C) { } func (s *HelperTestSuite) TestHotRegion(c *C) { - helper := Helper{ + helper := helper.Helper{ Store: s.store, RegionCache: s.store.GetRegionCache(), } - regionMetric, err := helper.FetchHotRegion("/pd/api/v1/hotspot/regions/read") + regionMetric, err := helper.FetchHotRegion(pdapi.HotRead) c.Assert(err, IsNil, Commentf("err: %+v", err)) c.Assert(fmt.Sprintf("%v", regionMetric), Equals, "map[1:{100 1 0}]") } @@ -93,8 +95,8 @@ func (s *HelperTestSuite) mockPDHTTPServer(c *C) { func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http.Request) { w.Header().Set("Content-Type", "application/json") w.WriteHeader(http.StatusOK) - regionsStat := hotRegionsStat{ - []regionStat{ + regionsStat := helper.HotRegionsStat{ + RegionsStat: []helper.RegionStat{ { FlowBytes: 100, RegionID: 1, @@ -102,8 +104,8 @@ func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http }, }, } - resp := StoreHotRegionInfos{ - AsLeader: make(map[uint64]*hotRegionsStat), + resp := helper.StoreHotRegionInfos{ + AsLeader: make(map[uint64]*helper.HotRegionsStat), } resp.AsLeader[0] = ®ionsStat data, err := json.MarshalIndent(resp, "", " ") diff --git a/store/tikv/client.go b/store/tikv/client.go index 0de1b543cba50..6b8f69aa4495a 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -133,13 +133,16 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { } logutil.Logger(context.Background()).Error("batchRecvLoop error when receive", zap.Error(err)) - // Hold the lock to forbid batchSendLoop using the old client. - c.clientLock.Lock() - c.failPendingRequests(err) // fail all pending requests. - for { // try to re-create the streaming in the loop. + for { // try to re-create the streaming in the loop. + // Hold the lock to forbid batchSendLoop using the old client. + c.clientLock.Lock() + c.failPendingRequests(err) // fail all pending requests. + // Re-establish a application layer stream. TCP layer is handled by gRPC. tikvClient := tikvpb.NewTikvClient(c.conn) streamClient, err := tikvClient.BatchCommands(context.TODO()) + c.clientLock.Unlock() + if err == nil { logutil.Logger(context.Background()).Info("batchRecvLoop re-create streaming success") c.client = streamClient @@ -149,7 +152,6 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { // TODO: Use a more smart backoff strategy. time.Sleep(time.Second) } - c.clientLock.Unlock() continue } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 34f40ea63daf1..fd942936fe727 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -24,9 +24,9 @@ import ( "time" "github.com/coreos/etcd/clientv3" - "github.com/grpc-ecosystem/go-grpc-prometheus" + grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/pingcap/errors" - "github.com/pingcap/pd/client" + pd "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -123,6 +123,13 @@ func (d Driver) Open(path string) (kv.Storage, error) { return s, nil } +// EtcdBackend is used for judging a storage is a real TiKV. +type EtcdBackend interface { + EtcdAddrs() []string + TLSConfig() *tls.Config + StartGCWorker() error +} + // update oracle's lastTS every 2000ms. var oracleUpdateInterval = 2000 @@ -336,6 +343,18 @@ func (s *tikvStore) GetOracle() oracle.Oracle { return s.oracle } +func (s *tikvStore) Name() string { + return "TiKV" +} + +func (s *tikvStore) Describe() string { + return "TiKV is a distributed transactional key-value database" +} + +func (s *tikvStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, kv.ErrNotImplemented +} + func (s *tikvStore) SupportDeleteRange() (supported bool) { return !s.mock } diff --git a/table/column.go b/table/column.go index f4fc28d360b78..48ec41eff751a 100644 --- a/table/column.go +++ b/table/column.go @@ -426,11 +426,16 @@ func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo) (t if col.IsGenerated() { return types.Datum{}, nil } - sc := ctx.GetSessionVars().StmtCtx + vars := ctx.GetSessionVars() + sc := vars.StmtCtx if sc.BadNullAsWarning { sc.AppendWarning(ErrColumnCantNull.GenWithStackByArgs(col.Name)) return GetZeroValue(col), nil } + if !vars.StrictSQLMode { + sc.AppendWarning(ErrNoDefaultValue.GenWithStackByArgs(col.Name)) + return GetZeroValue(col), nil + } return types.Datum{}, ErrNoDefaultValue.GenWithStackByArgs(col.Name) } diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 7eeea8e57cb59..a4ae973da727a 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -372,25 +372,23 @@ func CutRowNew(data []byte, colIDs map[int64]int) ([][]byte, error) { cnt int b []byte err error + cid int64 ) row := make([][]byte, len(colIDs)) for len(data) > 0 && cnt < len(colIDs) { // Get col id. - b, data, err = codec.CutOne(data) - if err != nil { - return nil, errors.Trace(err) - } - _, cid, err := codec.DecodeOne(b) + data, cid, err = codec.CutColumnID(data) if err != nil { return nil, errors.Trace(err) } + // Get col value. b, data, err = codec.CutOne(data) if err != nil { return nil, errors.Trace(err) } - id := cid.GetInt64() - offset, ok := colIDs[id] + + offset, ok := colIDs[cid] if ok { row[offset] = b cnt++ diff --git a/tidb-server/main.go b/tidb-server/main.go index cf295c4565543..73ee8610c0fca 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -24,7 +24,7 @@ import ( "sync/atomic" "time" - opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" kvstore "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" @@ -464,7 +465,7 @@ func setGlobalVars() { session.SetStatsLease(statsLeaseDuration) domain.RunAutoAnalyze = cfg.Performance.RunAutoAnalyze statistics.FeedbackProbability = cfg.Performance.FeedbackProbability - statistics.MaxQueryFeedbackCount = int(cfg.Performance.QueryFeedbackLimit) + handle.MaxQueryFeedbackCount = int(cfg.Performance.QueryFeedbackLimit) statistics.RatioOfPseudoEstimate = cfg.Performance.PseudoEstimateRatio ddl.RunWorker = cfg.RunDDL if cfg.SplitTable { diff --git a/tools/check/go.mod b/tools/check/go.mod index 4886811f71e56..ca5d580f6d6a4 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -14,6 +14,7 @@ require ( github.com/pelletier/go-toml v1.2.0 // indirect github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 gopkg.in/alecthomas/gometalinter.v2 v2.0.12 // indirect + gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20170321130658-9670b87a702e // indirect gopkg.in/yaml.v2 v2.2.2 // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index b7575b113d530..3e1fbb82e2d3e 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -58,6 +58,8 @@ golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563 h1:NIou6eNFigscvKJmsbyez16 golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= gopkg.in/alecthomas/gometalinter.v2 v2.0.12 h1:/xBWwtjmOmVxn8FXfIk9noV8m2E2Id9jFfUY/Mh9QAI= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= +gopkg.in/alecthomas/gometalinter.v3 v3.0.0 h1:tKnpLD70cWDacxrv9JZ4atld7RPoPiHOBfad6mPmyBw= +gopkg.in/alecthomas/gometalinter.v3 v3.0.0/go.mod h1:sE0aqUDPY4ibZWdfOxx4ZVG9CD+Y5I1H+Snwv8a3r/s= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20170321130658-9670b87a702e h1:/Kkmnr1GwjGf9vuTsP/PZLLQxJolIZl7UGN+Or0co1A= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20170321130658-9670b87a702e/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index e20b745658ab8..9cc87569e3542 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -149,6 +149,67 @@ func decodeEscapedUnicode(s []byte) (char [4]byte, size int, err error) { return } +// quoteString escapes interior quote and other characters for JSON_QUOTE +// https://dev.mysql.com/doc/refman/5.7/en/json-creation-functions.html#function_json-quote +// TODO: add JSON_QUOTE builtin +func quoteString(s string) string { + var escapeByteMap = map[byte]string{ + '\\': "\\\\", + '"': "\\\"", + '\b': "\\b", + '\f': "\\f", + '\n': "\\n", + '\r': "\\r", + '\t': "\\t", + } + + ret := new(bytes.Buffer) + ret.WriteByte('"') + + start := 0 + hasEscaped := false + + for i := 0; i < len(s); { + if b := s[i]; b < utf8.RuneSelf { + escaped, ok := escapeByteMap[b] + if ok { + if start < i { + ret.WriteString(s[start:i]) + } + hasEscaped = true + ret.WriteString(escaped) + i++ + start = i + } else { + i++ + } + } else { + c, size := utf8.DecodeRune([]byte(s[i:])) + if c == utf8.RuneError && size == 1 { // refer to codes of `binary.marshalStringTo` + if start < i { + ret.WriteString(s[start:i]) + } + hasEscaped = true + ret.WriteString(`\ufffd`) + i += size + start = i + continue + } + i += size + } + } + + if start < len(s) { + ret.WriteString(s[start:]) + } + + if hasEscaped { + ret.WriteByte('"') + return ret.String() + } + return ret.String()[1:] +} + // Extract receives several path expressions as arguments, matches them in bj, and returns: // ret: target JSON matched any path expressions. maybe autowrapped as an array. // found: true if any path expressions matched. @@ -778,3 +839,148 @@ func (bj BinaryJSON) GetElemDepth() int { return 1 } } + +// extractCallbackFn: the type of CALLBACK function for extractToCallback +type extractCallbackFn func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) + +// extractToCallback: callback alternative of extractTo +// would be more effective when walk through the whole JSON is unnecessary +// NOTICE: path [0] & [*] for JSON object other than array is INVALID, which is different from extractTo. +func (bj BinaryJSON) extractToCallback(pathExpr PathExpression, callbackFn extractCallbackFn, fullpath PathExpression) (stop bool, err error) { + if len(pathExpr.legs) == 0 { + return callbackFn(fullpath, bj) + } + + currentLeg, subPathExpr := pathExpr.popOneLeg() + if currentLeg.typ == pathLegIndex && bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + if currentLeg.arrayIndex == arrayIndexAsterisk { + for i := 0; i < elemCount; i++ { + //buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneIndexLeg(i) + stop, err = bj.arrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if currentLeg.arrayIndex < elemCount { + //buf = bj.arrayGetElem(currentLeg.arrayIndex).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneIndexLeg(currentLeg.arrayIndex) + stop, err = bj.arrayGetElem(currentLeg.arrayIndex).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if currentLeg.typ == pathLegKey && bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + if currentLeg.dotKey == "*" { + for i := 0; i < elemCount; i++ { + //buf = bj.objectGetVal(i).extractTo(buf, subPathExpr) + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = bj.objectGetVal(i).extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else { + child, ok := bj.objectSearchKey(hack.Slice(currentLeg.dotKey)) + if ok { + //buf = child.extractTo(buf, subPathExpr) + path := fullpath.pushBackOneKeyLeg(currentLeg.dotKey) + stop, err = child.extractToCallback(subPathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } + } else if currentLeg.typ == pathLegDoubleAsterisk { + //buf = bj.extractTo(buf, subPathExpr) + stop, err = bj.extractToCallback(subPathExpr, callbackFn, fullpath) + if stop || err != nil { + return + } + + if bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + //buf = bj.arrayGetElem(i).extractTo(buf, pathExpr) + path := fullpath.pushBackOneIndexLeg(i) + stop, err = bj.arrayGetElem(i).extractToCallback(pathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } else if bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + //buf = bj.objectGetVal(i).extractTo(buf, pathExpr) + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = bj.objectGetVal(i).extractToCallback(pathExpr, callbackFn, path) + if stop || err != nil { + return + } + } + } + } + return false, nil +} + +// BinaryJSONWalkFunc is used as callback function for BinaryJSON.Walk +type BinaryJSONWalkFunc func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) + +// Walk traverse BinaryJSON objects +func (bj BinaryJSON) Walk(walkFn BinaryJSONWalkFunc, pathExprList ...PathExpression) (err error) { + pathSet := make(map[string]bool) + + var doWalk extractCallbackFn + doWalk = func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + pathStr := fullpath.String() + if _, ok := pathSet[pathStr]; ok { + return false, nil + } + + stop, err = walkFn(fullpath, bj) + pathSet[pathStr] = true + if stop || err != nil { + return + } + + if bj.TypeCode == TypeCodeArray { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + path := fullpath.pushBackOneIndexLeg(i) + stop, err = doWalk(path, bj.arrayGetElem(i)) + if stop || err != nil { + return + } + } + } else if bj.TypeCode == TypeCodeObject { + elemCount := bj.GetElemCount() + for i := 0; i < elemCount; i++ { + path := fullpath.pushBackOneKeyLeg(string(bj.objectGetKey(i))) + stop, err = doWalk(path, bj.objectGetVal(i)) + if stop || err != nil { + return + } + } + } + return false, nil + } + + fullpath := PathExpression{legs: make([]pathLeg, 0, 32), flags: pathExpressionFlag(0)} + if len(pathExprList) > 0 { + for _, pathExpr := range pathExprList { + var stop bool + stop, err = bj.extractToCallback(pathExpr, doWalk, fullpath) + if stop || err != nil { + return err + } + } + } else { + _, err = doWalk(fullpath, bj) + if err != nil { + return + } + } + return nil +} diff --git a/types/json/binary_test.go b/types/json/binary_test.go index 73d30c8fc6f2c..6ab74d0739602 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -132,6 +132,27 @@ func (s *testJSONSuite) TestBinaryJSONUnquote(c *C) { } } +func (s *testJSONSuite) TestQuoteString(c *C) { + var tests = []struct { + j string + quoted string + }{ + {j: "3", quoted: `3`}, + {j: "hello, \"escaped quotes\" world", quoted: `"hello, \"escaped quotes\" world"`}, + {j: "你", quoted: `你`}, + {j: "true", quoted: `true`}, + {j: "null", quoted: `null`}, + {j: `"`, quoted: `"\""`}, + {j: `'`, quoted: `'`}, + {j: `''`, quoted: `''`}, + {j: ``, quoted: ``}, + {j: "\\ \" \b \f \n \r \t", quoted: `"\\ \" \b \f \n \r \t"`}, + } + for _, tt := range tests { + c.Assert(quoteString(tt.j), Equals, tt.quoted) + } +} + func (s *testJSONSuite) TestBinaryJSONModify(c *C) { c.Parallel() var tests = []struct { @@ -420,3 +441,142 @@ func (s *testJSONSuite) TestFunctions(c *C) { c.Assert(n, Equals, 0) c.Assert(err, ErrorMatches, "Cant peek from empty bytes") } + +func (s *testJSONSuite) TestBinaryJSONExtractCallback(c *C) { + bj1 := mustParseBinaryFromString(c, `{"\"hello\"": "world", "a": [1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}], "b": true, "c": ["d"]}`) + bj2 := mustParseBinaryFromString(c, `[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]`) + + type ExpectedPair struct { + path string + bj BinaryJSON + } + var tests = []struct { + bj BinaryJSON + pathExpr string + expected []ExpectedPair + }{ + {bj1, "$.a", []ExpectedPair{ + {"$.a", mustParseBinaryFromString(c, `[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]`)}, + }}, + {bj2, "$.a", []ExpectedPair{}}, + {bj1, "$[0]", []ExpectedPair{}}, // in extractToCallback/Walk/Search, DON'T autowraped bj as an array. + {bj2, "$[0]", []ExpectedPair{ + {"$[0]", mustParseBinaryFromString(c, `{"a": 1, "b": true}`)}, + }}, + {bj1, "$.a[2].aa", []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + }}, + {bj1, "$.a[*].aa", []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + {"$.a[4].aa", mustParseBinaryFromString(c, `"cc"`)}, + }}, + {bj1, "$.*[0]", []ExpectedPair{ + // {"$.\"hello\"[0]", mustParseBinaryFromString(c, `"world"`)}, // NO autowraped as an array. + {"$.a[0]", mustParseBinaryFromString(c, `1`)}, + // {"$.b[0]", mustParseBinaryFromString(c, `true`)}, // NO autowraped as an array. + {"$.c[0]", mustParseBinaryFromString(c, `"d"`)}, + }}, + {bj1, `$.a[*]."aa"`, []ExpectedPair{ + {"$.a[2].aa", mustParseBinaryFromString(c, `"bb"`)}, + {"$.a[4].aa", mustParseBinaryFromString(c, `"cc"`)}, + }}, + {bj1, `$."\"hello\""`, []ExpectedPair{ + {`$."\"hello\""`, mustParseBinaryFromString(c, `"world"`)}, + }}, + {bj1, `$**[1]`, []ExpectedPair{ + {`$.a[1]`, mustParseBinaryFromString(c, `"2"`)}, + }}, + } + + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.pathExpr) + c.Assert(err, IsNil) + + count := 0 + cb := func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + c.Assert(count, Less, len(tt.expected)) + if count < len(tt.expected) { + c.Assert(fullpath.String(), Equals, tt.expected[count].path) + c.Assert(bj.String(), Equals, tt.expected[count].bj.String()) + } + count++ + return false, nil + } + fullpath := PathExpression{legs: make([]pathLeg, 0), flags: pathExpressionFlag(0)} + _, err = tt.bj.extractToCallback(pe, cb, fullpath) + c.Assert(err, IsNil) + c.Assert(count, Equals, len(tt.expected)) + } +} + +func (s *testJSONSuite) TestBinaryJSONWalk(c *C) { + bj1 := mustParseBinaryFromString(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`) + bj2 := mustParseBinaryFromString(c, `{}`) + + type ExpectedPair struct { + path string + bj BinaryJSON + } + var tests = []struct { + bj BinaryJSON + paths []string + expected []ExpectedPair + }{ + {bj1, []string{}, []ExpectedPair{ + {`$`, mustParseBinaryFromString(c, `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]`)}, + {`$[0]`, mustParseBinaryFromString(c, `"abc"`)}, + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + {`$[2]`, mustParseBinaryFromString(c, `{"x":"abc"}`)}, + {`$[2].x`, mustParseBinaryFromString(c, `"abc"`)}, + {`$[3]`, mustParseBinaryFromString(c, `{"y":"bcd"}`)}, + {`$[3].y`, mustParseBinaryFromString(c, `"bcd"`)}, + }}, + {bj1, []string{`$[1]`}, []ExpectedPair{ + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + }}, + {bj1, []string{`$[1]`, `$[1]`}, []ExpectedPair{ // test for unique + {`$[1]`, mustParseBinaryFromString(c, `[{"k": "10"}, "def"]`)}, + {`$[1][0]`, mustParseBinaryFromString(c, `{"k": "10"}`)}, + {`$[1][0].k`, mustParseBinaryFromString(c, `"10"`)}, + {`$[1][1]`, mustParseBinaryFromString(c, `"def"`)}, + }}, + {bj1, []string{`$.m`}, []ExpectedPair{}}, + {bj2, []string{}, []ExpectedPair{ + {`$`, mustParseBinaryFromString(c, `{}`)}, + }}, + } + + for _, tt := range tests { + count := 0 + cb := func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + c.Assert(count, Less, len(tt.expected)) + if count < len(tt.expected) { + c.Assert(fullpath.String(), Equals, tt.expected[count].path) + c.Assert(bj.String(), Equals, tt.expected[count].bj.String()) + } + count++ + return false, nil + } + + var err error + if len(tt.paths) > 0 { + peList := make([]PathExpression, 0, len(tt.paths)) + for _, path := range tt.paths { + pe, errPath := ParseJSONPathExpr(path) + c.Assert(errPath, IsNil) + peList = append(peList, pe) + } + err = tt.bj.Walk(cb, peList...) + } else { + err = tt.bj.Walk(cb) + } + c.Assert(err, IsNil) + c.Assert(count, Equals, len(tt.expected)) + } +} diff --git a/types/json/path_expr.go b/types/json/path_expr.go index 7ed5276231ae8..ede4ce11aa89b 100644 --- a/types/json/path_expr.go +++ b/types/json/path_expr.go @@ -117,6 +117,30 @@ func (pe PathExpression) popOneLastLeg() (PathExpression, pathLeg) { return PathExpression{legs: pe.legs[:lastLegIdx]}, lastLeg } +// pushBackOneIndexLeg pushback one leg of INDEX type +func (pe PathExpression) pushBackOneIndexLeg(index int) PathExpression { + newPe := PathExpression{ + legs: append(pe.legs, pathLeg{typ: pathLegIndex, arrayIndex: index}), + flags: pe.flags, + } + if index == -1 { + newPe.flags |= pathExpressionContainsAsterisk + } + return newPe +} + +// pushBackOneKeyLeg pushback one leg of KEY type +func (pe PathExpression) pushBackOneKeyLeg(key string) PathExpression { + newPe := PathExpression{ + legs: append(pe.legs, pathLeg{typ: pathLegKey, dotKey: key}), + flags: pe.flags, + } + if key == "*" { + newPe.flags |= pathExpressionContainsAsterisk + } + return newPe +} + // ContainsAnyAsterisk returns true if pe contains any asterisk. func (pe PathExpression) ContainsAnyAsterisk() bool { return pe.flags.containsAnyAsterisk() @@ -212,3 +236,27 @@ func isBlank(c rune) bool { } return false } + +func (pe PathExpression) String() string { + var s strings.Builder + + s.WriteString("$") + for _, leg := range pe.legs { + switch leg.typ { + case pathLegIndex: + if leg.arrayIndex == -1 { + s.WriteString("[*]") + } else { + s.WriteString("[") + s.WriteString(strconv.Itoa(leg.arrayIndex)) + s.WriteString("]") + } + case pathLegKey: + s.WriteString(".") + s.WriteString(quoteString(leg.dotKey)) + case pathLegDoubleAsterisk: + s.WriteString("**") + } + } + return s.String() +} diff --git a/types/json/path_expr_test.go b/types/json/path_expr_test.go index 4525b7fe064ee..5efc648283d0a 100644 --- a/types/json/path_expr_test.go +++ b/types/json/path_expr_test.go @@ -62,3 +62,68 @@ func (s *testJSONSuite) TestValidatePathExpr(c *C) { } } } + +func (s *testJSONSuite) TestPathExprToString(c *C) { + var tests = []struct { + exprString string + }{ + {"$.a[1]"}, + {"$.a[*]"}, + {"$.*[2]"}, + {"$**.a[3]"}, + {`$."\"hello\""`}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + c.Assert(pe.String(), Equals, tt.exprString) + } +} + +func (s *testJSONSuite) TestPushBackOneIndexLeg(c *C) { + var tests = []struct { + exprString string + index int + expected string + containsAnyAsterisk bool + }{ + {"$", 1, "$[1]", false}, + {"$.a[1]", 1, "$.a[1][1]", false}, + {"$.a[1]", -1, "$.a[1][*]", true}, + {"$.a[*]", 10, "$.a[*][10]", true}, + {"$.*[2]", 2, "$.*[2][2]", true}, + {"$**.a[3]", 3, "$**.a[3][3]", true}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + + pe = pe.pushBackOneIndexLeg(tt.index) + c.Assert(pe.String(), Equals, tt.expected) + c.Assert(pe.ContainsAnyAsterisk(), Equals, tt.containsAnyAsterisk) + } +} + +func (s *testJSONSuite) TestPushBackOneKeyLeg(c *C) { + var tests = []struct { + exprString string + key string + expected string + containsAnyAsterisk bool + }{ + {"$", "aa", "$.aa", false}, + {"$.a[1]", "aa", "$.a[1].aa", false}, + {"$.a[1]", "*", "$.a[1].*", true}, + {"$.a[*]", "k", "$.a[*].k", true}, + {"$.*[2]", "bb", "$.*[2].bb", true}, + {"$**.a[3]", "cc", "$**.a[3].cc", true}, + } + for _, tt := range tests { + pe, err := ParseJSONPathExpr(tt.exprString) + c.Assert(err, IsNil) + + pe = pe.pushBackOneKeyLeg(tt.key) + c.Assert(pe.String(), Equals, tt.expected) + c.Assert(pe.ContainsAnyAsterisk(), Equals, tt.containsAnyAsterisk) + } +} diff --git a/util/codec/codec.go b/util/codec/codec.go index 0af81e36ae5f7..fde8afe1b7bd3 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -429,6 +429,17 @@ func CutOne(b []byte) (data []byte, remain []byte, err error) { return b[:l], b[l:], nil } +// CutColumnID cuts the column ID from b. +// It will return the remains as byte slice and column ID +func CutColumnID(b []byte) (remain []byte, n int64, err error) { + if len(b) < 1 { + return nil, 0, errors.New("invalid encoded key") + } + // skip the flag + b = b[1:] + return DecodeVarint(b) +} + // SetRawValues set raw datum values from a row data. func SetRawValues(data []byte, values []types.Datum) error { for i := 0; i < len(values); i++ { diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 2c49c345940e4..54de745d4301f 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -46,6 +46,7 @@ type Tracker struct { label string // Label of this "Tracker". bytesConsumed int64 // Consumed bytes. bytesLimit int64 // Negative value means no limit. + maxConsumed int64 // max number of bytes consumed during execution. actionOnExceed ActionOnExceed parent *Tracker // The parent memory tracker. } @@ -142,6 +143,19 @@ func (t *Tracker) Consume(bytes int64) { if atomic.AddInt64(&tracker.bytesConsumed, bytes) >= tracker.bytesLimit && tracker.bytesLimit > 0 { rootExceed = tracker } + + if tracker.parent == nil { + // since we only need a total memory usage during execution, + // we only record max consumed bytes in root(statement-level) for performance. + for { + maxNow := atomic.LoadInt64(&tracker.maxConsumed) + consumed := atomic.LoadInt64(&tracker.bytesConsumed) + if consumed > maxNow && !atomic.CompareAndSwapInt64(&tracker.maxConsumed, maxNow, consumed) { + continue + } + break + } + } } if rootExceed != nil { rootExceed.actionOnExceed.Action(rootExceed) @@ -153,6 +167,11 @@ func (t *Tracker) BytesConsumed() int64 { return atomic.LoadInt64(&t.bytesConsumed) } +// MaxConsumed returns max number of bytes consumed during execution. +func (t *Tracker) MaxConsumed() int64 { + return atomic.LoadInt64(&t.maxConsumed) +} + // String returns the string representation of this Tracker tree. func (t *Tracker) String() string { buffer := bytes.NewBufferString("\n") diff --git a/util/memory/tracker_test.go b/util/memory/tracker_test.go index 91ffab8e88522..8868571279500 100644 --- a/util/memory/tracker_test.go +++ b/util/memory/tracker_test.go @@ -14,10 +14,12 @@ package memory import ( + "math/rand" "os" "sync" "testing" + "github.com/cznic/mathutil" . "github.com/pingcap/check" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testleak" @@ -190,6 +192,33 @@ func (s *testSuite) TestToString(c *C) { `) } +func (s *testSuite) TestMaxConsumed(c *C) { + r := NewTracker("root", -1) + c1 := NewTracker("child 1", -1) + c2 := NewTracker("child 2", -1) + cc1 := NewTracker("child of child 1", -1) + + c1.AttachTo(r) + c2.AttachTo(r) + cc1.AttachTo(c1) + + ts := []*Tracker{r, c1, c2, cc1} + var consumed, maxConsumed int64 + for i := 0; i < 10; i++ { + t := ts[rand.Intn(len(ts))] + b := rand.Int63n(1000) - 500 + if consumed+b < 0 { + b = -consumed + } + consumed += b + t.Consume(b) + maxConsumed = mathutil.MaxInt64(maxConsumed, consumed) + + c.Assert(r.BytesConsumed(), Equals, consumed) + c.Assert(r.MaxConsumed(), Equals, maxConsumed) + } +} + func BenchmarkConsume(b *testing.B) { tracker := NewTracker("root", -1) b.RunParallel(func(pb *testing.PB) { diff --git a/util/mock/store.go b/util/mock/store.go index b385ebd2ea037..032201713311a 100644 --- a/util/mock/store.go +++ b/util/mock/store.go @@ -14,6 +14,8 @@ package mock import ( + "context" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -49,3 +51,14 @@ func (s *Store) CurrentVersion() (kv.Version, error) { return kv.Version{}, nil // SupportDeleteRange implements kv.Storage interface. func (s *Store) SupportDeleteRange() bool { return false } + +// Name implements kv.Storage interface. +func (s *Store) Name() string { return "UtilMockStorage" } + +// Describe implements kv.Storage interface. +func (s *Store) Describe() string { + return "UtilMockStorage is a mock Store implementation, only for unittests in util package" +} + +// ShowStatus implements kv.Storage interface. +func (s *Store) ShowStatus(ctx context.Context, key string) (interface{}, error) { return nil, nil } diff --git a/util/pdapi/const.go b/util/pdapi/const.go new file mode 100644 index 0000000000000..1cf4d935ff4d0 --- /dev/null +++ b/util/pdapi/const.go @@ -0,0 +1,20 @@ +// 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 pdapi + +// HotRead / HotWrite is the pd apis to get the corresponding hot region information. +const ( + HotRead = "/pd/api/v1/hotspot/regions/read" + HotWrite = "/pd/api/v1/hotspot/regions/read" +)