diff --git a/bindinfo/bind.go b/bindinfo/bind.go new file mode 100644 index 0000000000000..9d70aadf198a4 --- /dev/null +++ b/bindinfo/bind.go @@ -0,0 +1,169 @@ +// 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 "github.com/pingcap/parser/ast" + +// BindHint will add hints for originStmt according to hintedStmt' hints. +func BindHint(originStmt, hintedStmt ast.StmtNode) ast.StmtNode { + switch x := originStmt.(type) { + case *ast.SelectStmt: + return selectBind(x, hintedStmt.(*ast.SelectStmt)) + default: + return originStmt + } +} + +func selectBind(originalNode, hintedNode *ast.SelectStmt) *ast.SelectStmt { + if hintedNode.TableHints != nil { + originalNode.TableHints = hintedNode.TableHints + } + if originalNode.From != nil { + originalNode.From.TableRefs = resultSetNodeBind(originalNode.From.TableRefs, hintedNode.From.TableRefs).(*ast.Join) + } + if originalNode.Where != nil { + originalNode.Where = exprBind(originalNode.Where, hintedNode.Where).(ast.ExprNode) + } + + if originalNode.Having != nil { + originalNode.Having.Expr = exprBind(originalNode.Having.Expr, hintedNode.Having.Expr) + } + + if originalNode.OrderBy != nil { + originalNode.OrderBy = orderByBind(originalNode.OrderBy, hintedNode.OrderBy) + } + + if originalNode.Fields != nil { + origFields := originalNode.Fields.Fields + hintFields := hintedNode.Fields.Fields + for idx := range origFields { + origFields[idx].Expr = exprBind(origFields[idx].Expr, hintFields[idx].Expr) + } + } + return originalNode +} + +func orderByBind(originalNode, hintedNode *ast.OrderByClause) *ast.OrderByClause { + for idx := 0; idx < len(originalNode.Items); idx++ { + originalNode.Items[idx].Expr = exprBind(originalNode.Items[idx].Expr, hintedNode.Items[idx].Expr) + } + return originalNode +} + +func exprBind(originalNode, hintedNode ast.ExprNode) ast.ExprNode { + switch v := originalNode.(type) { + case *ast.SubqueryExpr: + if v.Query != nil { + v.Query = resultSetNodeBind(v.Query, hintedNode.(*ast.SubqueryExpr).Query) + } + case *ast.ExistsSubqueryExpr: + if v.Sel != nil { + v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.ExistsSubqueryExpr).Sel.(*ast.SubqueryExpr).Query) + } + case *ast.PatternInExpr: + if v.Sel != nil { + v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.PatternInExpr).Sel.(*ast.SubqueryExpr).Query) + } + case *ast.BinaryOperationExpr: + if v.L != nil { + v.L = exprBind(v.L, hintedNode.(*ast.BinaryOperationExpr).L) + } + if v.R != nil { + v.R = exprBind(v.R, hintedNode.(*ast.BinaryOperationExpr).R) + } + case *ast.IsNullExpr: + if v.Expr != nil { + v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsNullExpr).Expr) + } + case *ast.IsTruthExpr: + if v.Expr != nil { + v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsTruthExpr).Expr) + } + case *ast.PatternLikeExpr: + if v.Pattern != nil { + v.Pattern = exprBind(v.Pattern, hintedNode.(*ast.PatternLikeExpr).Pattern) + } + case *ast.CompareSubqueryExpr: + if v.L != nil { + v.L = exprBind(v.L, hintedNode.(*ast.CompareSubqueryExpr).L) + } + if v.R != nil { + v.R = exprBind(v.R, hintedNode.(*ast.CompareSubqueryExpr).R) + } + case *ast.BetweenExpr: + if v.Left != nil { + v.Left = exprBind(v.Left, hintedNode.(*ast.BetweenExpr).Left) + } + if v.Right != nil { + v.Right = exprBind(v.Right, hintedNode.(*ast.BetweenExpr).Right) + } + case *ast.UnaryOperationExpr: + if v.V != nil { + v.V = exprBind(v.V, hintedNode.(*ast.UnaryOperationExpr).V) + } + case *ast.CaseExpr: + if v.Value != nil { + v.Value = exprBind(v.Value, hintedNode.(*ast.CaseExpr).Value) + } + if v.ElseClause != nil { + v.ElseClause = exprBind(v.ElseClause, hintedNode.(*ast.CaseExpr).ElseClause) + } + } + return originalNode +} + +func resultSetNodeBind(originalNode, hintedNode ast.ResultSetNode) ast.ResultSetNode { + switch x := originalNode.(type) { + case *ast.Join: + return joinBind(x, hintedNode.(*ast.Join)) + case *ast.TableSource: + ts, _ := hintedNode.(*ast.TableSource) + switch v := x.Source.(type) { + case *ast.SelectStmt: + x.Source = selectBind(v, ts.Source.(*ast.SelectStmt)) + case *ast.UnionStmt: + x.Source = unionSelectBind(v, hintedNode.(*ast.TableSource).Source.(*ast.UnionStmt)) + case *ast.TableName: + x.Source.(*ast.TableName).IndexHints = ts.Source.(*ast.TableName).IndexHints + } + return x + case *ast.SelectStmt: + return selectBind(x, hintedNode.(*ast.SelectStmt)) + case *ast.UnionStmt: + return unionSelectBind(x, hintedNode.(*ast.UnionStmt)) + default: + return x + } +} + +func joinBind(originalNode, hintedNode *ast.Join) *ast.Join { + if originalNode.Left != nil { + originalNode.Left = resultSetNodeBind(originalNode.Left, hintedNode.Left) + } + + if hintedNode.Right != nil { + originalNode.Right = resultSetNodeBind(originalNode.Right, hintedNode.Right) + } + + return originalNode +} + +func unionSelectBind(originalNode, hintedNode *ast.UnionStmt) ast.ResultSetNode { + selects := originalNode.SelectList.Selects + for i := len(selects) - 1; i >= 0; i-- { + originalNode.SelectList.Selects[i] = selectBind(selects[i], hintedNode.SelectList.Selects[i]) + } + + return originalNode +} diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index fc7d892385f63..e37ae0f45eff3 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -220,3 +220,207 @@ func (s *testSuite) TestGlobalBinding(c *C) { _, 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)) } + +func (s *testSuite) TestSessionBinding(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 session 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 session binding for select * from t where i>99 using select * from t use index(index_t) where i>99") + c.Assert(err, IsNil) + + handle := tk.Se.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindData := handle.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, 0) + + rs, err = tk.Exec("show session 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) + + _, err = tk.Exec("drop session binding for select * from t where i>99") + c.Assert(err, IsNil) + bindData = handle.GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") + c.Check(bindData.Status, Equals, "deleted") +} + +func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustQuery("explain SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) +} + +func (s *testSuite) TestExplain(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustQuery("explain SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id") +} + +func (s *testSuite) TestErrorBind(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)) + + 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>100") + 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) + + tk.MustExec("drop index index_t on t") + _, err = tk.Exec("select * from t where i > 10") + c.Check(err, IsNil) + + s.domain.BindHandle().DropInvalidBindRecord() + + 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) +} diff --git a/bindinfo/cache.go b/bindinfo/cache.go index b8731628ec080..a4c2785eb9c64 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -20,20 +20,22 @@ import ( ) const ( - // using is the bind info's in use status. - using = "using" + // Using is the bind info's in use status. + Using = "using" // deleted is the bind info's deleted status. deleted = "deleted" + // Invalid is the bind info's invalid status. + Invalid = "invalid" ) -// bindMeta stores the basic bind info and bindSql astNode. -type bindMeta struct { +// BindMeta stores the basic bind info and bindSql astNode. +type BindMeta struct { *BindRecord - ast ast.StmtNode //ast will be used to do query sql bind check + 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 +// cache is a k-v map, key is original sql, value is a slice of BindMeta. +type cache map[string][]*BindMeta // BindRecord represents a sql bind record retrieved from the storage. type BindRecord struct { diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 0a8e62d3e76e8..fc33f2cd3442c 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -20,6 +20,7 @@ import ( "go.uber.org/zap" "sync" "sync/atomic" + "time" "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" @@ -31,7 +32,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" ) -// BindHandle is used to handle all the sql bind operations. +// BindHandle is used to handle all global sql bind operations. type BindHandle struct { sctx struct { sync.Mutex @@ -60,15 +61,28 @@ type BindHandle struct { atomic.Value } + // invalidBindRecordMap indicates the invalid bind records found during querying. + // A record will be deleted from this map, after 2 bind-lease, after it is dropped from the kv. + invalidBindRecordMap struct { + sync.Mutex + atomic.Value + } + parser *parser.Parser lastUpdateTime types.Time } +type invalidBindRecordMap struct { + bindRecord *BindRecord + droppedTime time.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)) + handle.invalidBindRecordMap.Value.Store(make(map[string]*invalidBindRecordMap)) return handle } @@ -106,14 +120,14 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { } newCache.removeStaleBindMetas(hash, meta) - if meta.Status == using { + if meta.Status == Using { newCache[hash] = append(newCache[hash], meta) } } return nil } -// AddBindRecord adds a BindRecord to the storage and bindMeta to the cache. +// 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() @@ -137,7 +151,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return } - // update the bindMeta to the cache. + // update the BindMeta to the cache. hash, meta, err1 := h.newBindMeta(record) if err1 != nil { err = err1 @@ -163,7 +177,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { Fsp: 3, } record.UpdateTime = record.CreateTime - record.Status = using + record.Status = Using record.BindSQL = h.getEscapeCharacter(record.BindSQL) // insert the BindRecord to the storage. @@ -171,7 +185,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return err } -// DropBindRecord drops a BindRecord to the storage and bindMeta int the cache. +// 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() @@ -196,7 +210,7 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { return } - hash, meta := h.newBindMetaWithoutAst(record) + hash, meta := newBindMetaWithoutAst(record) h.removeBindMeta(hash, meta) }() @@ -217,6 +231,44 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { return err } +// DropInvalidBindRecord execute the drop bindRecord task. +func (h *BindHandle) DropInvalidBindRecord() { + invalidBindRecordMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Load().(map[string]*invalidBindRecordMap)) + for key, invalidBindRecord := range invalidBindRecordMap { + if invalidBindRecord.droppedTime.IsZero() { + err := h.DropBindRecord(invalidBindRecord.bindRecord) + if err != nil { + logutil.Logger(context.Background()).Error("DropInvalidBindRecord failed", zap.Error(err)) + } + invalidBindRecord.droppedTime = time.Now() + continue + } + + if time.Since(invalidBindRecord.droppedTime) > 6*time.Second { + delete(invalidBindRecordMap, key) + } + } + h.invalidBindRecordMap.Store(invalidBindRecordMap) +} + +// AddDropInvalidBindTask add bindRecord to invalidBindRecordMap when the bindRecord need to be deleted. +func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { + key := invalidBindRecord.OriginalSQL + ":" + invalidBindRecord.Db + if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { + return + } + h.invalidBindRecordMap.Lock() + defer h.invalidBindRecordMap.Unlock() + if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { + return + } + newMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)) + newMap[key] = &invalidBindRecordMap{ + bindRecord: invalidBindRecord, + } + h.invalidBindRecordMap.Store(newMap) +} + // Size return the size of bind info cache. func (h *BindHandle) Size() int { size := 0 @@ -227,21 +279,12 @@ func (h *BindHandle) Size() int { } // 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 +func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { + return h.bindInfo.Load().(cache).getBindRecord(normdOrigSQL, db) } // GetAllBindRecord return all bind record in cache. -func (h *BindHandle) GetAllBindRecord() (bindRecords []*bindMeta) { +func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindMeta) { bindRecordMap := h.bindInfo.Load().(cache) for _, bindRecord := range bindRecordMap { bindRecords = append(bindRecords, bindRecord...) @@ -249,25 +292,25 @@ func (h *BindHandle) GetAllBindRecord() (bindRecords []*bindMeta) { return bindRecords } -func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *bindMeta, err error) { +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]} + meta = &BindMeta{BindRecord: record, Ast: stmtNodes[0]} return hash, meta, nil } -func (h *BindHandle) newBindMetaWithoutAst(record *BindRecord) (hash string, meta *bindMeta) { +func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { hash = parser.DigestHash(record.OriginalSQL) - meta = &bindMeta{BindRecord: record} + meta = &BindMeta{BindRecord: record} return hash, meta } -// appendBindMeta addes the bindMeta to the cache, all the stale bindMetas are +// 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) { +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() @@ -280,8 +323,8 @@ func (h *BindHandle) appendBindMeta(hash string, meta *bindMeta) { newCache[hash] = append(newCache[hash], meta) } -// removeBindMeta removes the bindMeta from the cache. -func (h *BindHandle) removeBindMeta(hash string, meta *bindMeta) { +// 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() { @@ -292,8 +335,8 @@ func (h *BindHandle) removeBindMeta(hash string, meta *bindMeta) { 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) { +// 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 @@ -310,8 +353,8 @@ func (c cache) removeDeletedBindMeta(hash string, meta *bindMeta) { } } -// removeStaleBindMetas removes all the stale bindMeta in the cache. -func (c cache) removeStaleBindMetas(hash string, meta *bindMeta) { +// removeStaleBindMetas removes all the stale BindMeta in the cache. +func (c cache) removeStaleBindMetas(hash string, meta *BindMeta) { metas, ok := c[hash] if !ok { return @@ -337,13 +380,34 @@ func (c cache) copy() cache { return newCache } -// isStale checks whether this bindMeta is stale compared with the other bindMeta. -func (m *bindMeta) isStale(other *bindMeta) bool { +func copyInvalidBindRecordMap(oldMap map[string]*invalidBindRecordMap) map[string]*invalidBindRecordMap { + newMap := make(map[string]*invalidBindRecordMap, len(oldMap)) + for k, v := range oldMap { + newMap[k] = v + } + return newMap +} + +func (c cache) getBindRecord(normdOrigSQL, db string) *BindMeta { + hash := parser.DigestHash(normdOrigSQL) + bindRecords := c[hash] + if bindRecords != nil { + for _, bindRecord := range bindRecords { + if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { + return bindRecord + } + } + } + return nil +} + +// 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 { +func (m *BindMeta) isSame(other *BindMeta) bool { return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db } diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go new file mode 100644 index 0000000000000..f343b3ca8e24d --- /dev/null +++ b/bindinfo/session_handle.go @@ -0,0 +1,112 @@ +// 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 ( + "time" + + "github.com/pingcap/parser" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +// SessionHandle is used to handle all session sql bind operations. +type SessionHandle struct { + ch cache + parser *parser.Parser +} + +// NewSessionBindHandle creates a new SessionBindHandle. +func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { + sessionHandle := &SessionHandle{parser: parser} + sessionHandle.ch = make(cache) + return sessionHandle +} + +// appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are +// removed from the cache after this operation. +func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) { + // Make sure there is only one goroutine writes the cache. + h.ch.removeStaleBindMetas(hash, meta) + h.ch[hash] = append(h.ch[hash], meta) +} + +func (h *SessionHandle) 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 +} + +// AddBindRecord new a BindRecord with BindMeta, add it to the cache. +func (h *SessionHandle) AddBindRecord(record *BindRecord) error { + record.CreateTime = types.Time{ + Time: types.FromGoTime(time.Now()), + Type: mysql.TypeDatetime, + Fsp: 3, + } + record.UpdateTime = record.CreateTime + + // update the BindMeta to the cache. + hash, meta, err := h.newBindMeta(record) + if err == nil { + h.appendBindMeta(hash, meta) + } + return err +} + +// DropBindRecord drops a BindRecord in the cache. +func (h *SessionHandle) DropBindRecord(record *BindRecord) { + meta := &BindMeta{BindRecord: record} + meta.Status = deleted + hash := parser.DigestHash(record.OriginalSQL) + h.ch.removeDeletedBindMeta(hash, meta) + h.appendBindMeta(hash, meta) +} + +// GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. +func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { + hash := parser.DigestHash(normdOrigSQL) + bindRecords := h.ch[hash] + if bindRecords != nil { + for _, bindRecord := range bindRecords { + if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { + return bindRecord + } + } + } + return nil +} + +// GetAllBindRecord return all session bind info. +func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { + for _, bindRecord := range h.ch { + bindRecords = append(bindRecords, bindRecord...) + } + return bindRecords +} + +// sessionBindInfoKeyType is a dummy type to avoid naming collision in context. +type sessionBindInfoKeyType int + +// String defines a Stringer function for debugging and pretty printing. +func (k sessionBindInfoKeyType) String() string { + return "session_bindinfo" +} + +// SessionBindInfoKeyType is a variable key for store session bind info. +const SessionBindInfoKeyType sessionBindInfoKeyType = 0 diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index f8b87a8ded5fe..e23a148310563 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -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 [gad.aid gad.ip], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], 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,7 +137,7 @@ 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 [gad.aid gad.dic], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], 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; @@ -157,7 +157,7 @@ Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr. │ └─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 └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false, stats:pseudo + ├─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 └─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; diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 830fcaba61da5..5db97abb7f63e 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -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 [gad.aid gad.ip], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], 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,7 +145,7 @@ 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 [gad.aid gad.dic], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], 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; @@ -165,7 +165,7 @@ Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, r │ └─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 └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false + ├─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 └─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; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 3a64d978d4b8d..1497fb03490de 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -47,7 +47,7 @@ IndexJoin_12 4166.67 root left outer join, inner:IndexLookUp_11, outer key:test. │ └─TableScan_23 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 [test.t1.c2], keep order:false, stats:pseudo + │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [eq(test.t2.c1, test.t1.c2)], keep order:false, stats:pseudo └─TableScan_9 0.00 cop table:t2, keep order:false, stats:pseudo explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info diff --git a/cmd/explaintest/r/index_join.result b/cmd/explaintest/r/index_join.result index 07d177671c292..6d5555bc8993e 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 [test.t2.a], keep order:false +│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false │ └─TableScan_13 0.00 cop table:t1, keep order:false, stats:pseudo └─TableReader_19 1.00 root data:Selection_18 └─Selection_18 1.00 cop not(isnull(test.t2.a)) @@ -23,5 +23,5 @@ Projection_6 5.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b │ └─TableScan_28 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 [test.t2.a], keep order:false + │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false └─TableScan_9 0.00 cop table:t1, keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 132df50fc2f19..e8ba90678635a 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -178,12 +178,12 @@ Projection_13 0.00 root te.expect_time │ │ └─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, stats:pseudo │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, 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 [tr.id], keep order:false, stats:pseudo + └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, 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 [t1.a], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, 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 [t1.a], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, 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 5ee157d771853..a083977411ea6 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -261,7 +261,7 @@ Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderd │ └─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 └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─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 └─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 /* @@ -302,7 +302,7 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc │ └─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 └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─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 └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, 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 [tpch.orders.o_orderkey], keep order:false + │ │ │ ├─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 │ │ │ └─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 [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + ├─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 └─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 [tpch.orders.o_orderkey], keep order:false + ├─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 └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, 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 [tpch.part.p_partkey], keep order:false + │ └─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 └─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 [tpch.orders.o_orderkey], keep order:false + ├─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 └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1165,7 +1165,7 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc │ │ └─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 │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ ├─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 │ └─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) @@ -1239,10 +1239,10 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ └─Selection_60 0.80 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 [l1.l_orderkey], keep order:false + │ ├─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 │ └─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 [l1.l_orderkey], keep order:false + ├─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 └─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/config/config.go b/config/config.go index f5112120320d7..cada69fe3edbd 100644 --- a/config/config.go +++ b/config/config.go @@ -14,17 +14,23 @@ package config import ( + "bytes" + "context" "crypto/tls" "crypto/x509" "fmt" "io/ioutil" + "os" + "reflect" "strings" + "sync" "time" "github.com/BurntSushi/toml" "github.com/pingcap/errors" "github.com/pingcap/tidb/util/logutil" tracing "github.com/uber/jaeger-client-go/config" + "go.uber.org/atomic" ) // Config number limitations @@ -370,7 +376,14 @@ var defaultConf = Config{ }, } -var globalConf = defaultConf +var ( + globalConf = atomic.Value{} + reloadConfPath = "" + confReloader func(nc, c *Config) + confReloadLock sync.Mutex + supportedReloadConfigs = make(map[string]struct{}, 32) + supportedReloadConfList = make([]string, 0, 32) +) // NewConfig creates a new config instance with default value. func NewConfig() *Config { @@ -378,11 +391,89 @@ func NewConfig() *Config { return &conf } +// SetConfReloader sets reload config path and a reloader. +// It should be called only once at start time. +func SetConfReloader(cpath string, reloader func(nc, c *Config), confItems ...string) { + reloadConfPath = cpath + confReloader = reloader + for _, item := range confItems { + supportedReloadConfigs[item] = struct{}{} + supportedReloadConfList = append(supportedReloadConfList, item) + } +} + // GetGlobalConfig returns the global configuration for this server. // It should store configuration from command line and configuration file. // Other parts of the system can read the global configuration use this function. func GetGlobalConfig() *Config { - return &globalConf + return globalConf.Load().(*Config) +} + +// ReloadGlobalConfig reloads global configuration for this server. +func ReloadGlobalConfig() error { + confReloadLock.Lock() + defer confReloadLock.Unlock() + + nc := NewConfig() + if err := nc.Load(reloadConfPath); err != nil { + return err + } + if err := nc.Valid(); err != nil { + return err + } + c := GetGlobalConfig() + + diffs := collectsDiff(*nc, *c, "") + if len(diffs) == 0 { + return nil + } + var formattedDiff bytes.Buffer + for k, vs := range diffs { + formattedDiff.WriteString(fmt.Sprintf(", %v:%v->%v", k, vs[1], vs[0])) + } + unsupported := make([]string, 0, 2) + for k := range diffs { + if _, ok := supportedReloadConfigs[k]; !ok { + unsupported = append(unsupported, k) + } + } + if len(unsupported) > 0 { + return fmt.Errorf("reloading config %v is not supported, only %v are supported now, "+ + "your changes%s", unsupported, supportedReloadConfList, formattedDiff.String()) + } + + confReloader(nc, c) + globalConf.Store(nc) + logutil.Logger(context.Background()).Info("reload config changes" + formattedDiff.String()) + return nil +} + +// collectsDiff collects different config items. +// map[string][]string -> map[field path][]{new value, old value} +func collectsDiff(i1, i2 interface{}, fieldPath string) map[string][]interface{} { + diff := make(map[string][]interface{}) + t := reflect.TypeOf(i1) + if t.Kind() != reflect.Struct { + if reflect.DeepEqual(i1, i2) { + return diff + } + diff[fieldPath] = []interface{}{i1, i2} + return diff + } + + v1 := reflect.ValueOf(i1) + v2 := reflect.ValueOf(i2) + for i := 0; i < v1.NumField(); i++ { + p := t.Field(i).Name + if fieldPath != "" { + p = fieldPath + "." + p + } + m := collectsDiff(v1.Field(i).Interface(), v2.Field(i).Interface(), p) + for k, v := range m { + diff[k] = v + } + } + return diff } // Load loads config options from a toml file. @@ -406,6 +497,51 @@ func (c *Config) Load(confFile string) error { return err } +// Valid checks if this config is valid. +func (c *Config) Valid() error { + if c.Security.SkipGrantTable && !hasRootPrivilege() { + return fmt.Errorf("TiDB run with skip-grant-table need root privilege") + } + if _, ok := ValidStorage[c.Store]; !ok { + nameList := make([]string, 0, len(ValidStorage)) + for k, v := range ValidStorage { + if v { + nameList = append(nameList, k) + } + } + return fmt.Errorf("invalid store=%s, valid storages=%v", c.Store, nameList) + } + if c.Store == "mocktikv" && !c.RunDDL { + return fmt.Errorf("can't disable DDL on mocktikv") + } + if c.Log.File.MaxSize > MaxLogFileSize { + return fmt.Errorf("invalid max log file size=%v which is larger than max=%v", c.Log.File.MaxSize, MaxLogFileSize) + } + c.OOMAction = strings.ToLower(c.OOMAction) + + // lower_case_table_names is allowed to be 0, 1, 2 + if c.LowerCaseTableNames < 0 || c.LowerCaseTableNames > 2 { + return fmt.Errorf("lower-case-table-names should be 0 or 1 or 2") + } + + if c.TxnLocalLatches.Enabled && c.TxnLocalLatches.Capacity == 0 { + return fmt.Errorf("txn-local-latches.capacity can not be 0") + } + + // For tikvclient. + if c.TiKVClient.GrpcConnectionCount == 0 { + return fmt.Errorf("grpc-connection-count should be greater than 0") + } + if c.TiKVClient.MaxTxnTimeUse == 0 { + return fmt.Errorf("max-txn-time-use should be greater than 0") + } + return nil +} + +func hasRootPrivilege() bool { + return os.Geteuid() == 0 +} + // ToLogConfig converts *Log to *logutil.LogConfig. func (l *Log) ToLogConfig() *logutil.LogConfig { return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.DisableTimestamp) @@ -433,6 +569,7 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { } func init() { + globalConf.Store(&defaultConf) if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true } diff --git a/config/config_test.go b/config/config_test.go index 06110c2da9cc6..237bf1a1b85ba 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -169,3 +169,28 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1 c.Assert(os.Remove(certFile), IsNil) c.Assert(os.Remove(keyFile), IsNil) } + +func (s *testConfigSuite) TestConfigDiff(c *C) { + c1 := NewConfig() + c2 := &Config{} + *c2 = *c1 + c1.OOMAction = "c1" + c2.OOMAction = "c2" + c1.MemQuotaQuery = 2333 + c2.MemQuotaQuery = 3222 + c1.Performance.CrossJoin = true + c2.Performance.CrossJoin = false + c1.Performance.FeedbackProbability = 2333 + c2.Performance.FeedbackProbability = 23.33 + + diffs := collectsDiff(*c1, *c2, "") + c.Assert(len(diffs), Equals, 4) + c.Assert(diffs["OOMAction"][0], Equals, "c1") + c.Assert(diffs["OOMAction"][1], Equals, "c2") + c.Assert(diffs["MemQuotaQuery"][0], Equals, int64(2333)) + c.Assert(diffs["MemQuotaQuery"][1], Equals, int64(3222)) + c.Assert(diffs["Performance.CrossJoin"][0], Equals, true) + c.Assert(diffs["Performance.CrossJoin"][1], Equals, false) + c.Assert(diffs["Performance.FeedbackProbability"][0], Equals, float64(2333)) + c.Assert(diffs["Performance.FeedbackProbability"][1], Equals, float64(23.33)) +} diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 171a5128b27b4..5ee6b31999c74 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -551,7 +551,7 @@ func (s *testIntegrationSuite7) TestNullGeneratedColumn(c *C) { tk.MustExec("CREATE TABLE `t` (" + "`a` int(11) DEFAULT NULL," + "`b` int(11) DEFAULT NULL," + - "`c` int(11) GENERATED ALWAYS AS (`a` + `b`) VIRTUAL DEFAULT NULL," + + "`c` int(11) GENERATED ALWAYS AS (`a` + `b`) VIRTUAL," + "`h` varchar(10) DEFAULT NULL," + "`m` int(11) DEFAULT NULL" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") @@ -751,6 +751,51 @@ func (s *testIntegrationSuite7) TestCaseInsensitiveCharsetAndCollate(c *C) { tk.MustExec("create table t2(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8 COLLATE=utf8_BIN;") tk.MustExec("create table t3(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8mb4 COLLATE=utf8MB4_BIN;") tk.MustExec("create table t4(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8mb4 COLLATE=utf8MB4_general_ci;") + + tk.MustExec("create table t5(a varchar(20)) ENGINE=InnoDB DEFAULT CHARSET=UTF8MB4 COLLATE=UTF8MB4_GENERAL_CI;") + tk.MustExec("insert into t5 values ('特克斯和凯科斯群岛')") + + db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test_charset_collate")) + c.Assert(ok, IsTrue) + tbl := testGetTableByName(c, s.ctx, "test_charset_collate", "t5") + tblInfo := tbl.Meta().Clone() + c.Assert(tblInfo.Charset, Equals, "utf8mb4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + + tblInfo.Version = model.TableInfoVersion2 + tblInfo.Charset = "UTF8MB4" + + updateTableInfo := func(tblInfo *model.TableInfo) { + mockCtx := mock.NewContext() + mockCtx.Store = s.store + err := mockCtx.NewTxn(context.Background()) + c.Assert(err, IsNil) + txn, err := mockCtx.Txn(true) + c.Assert(err, IsNil) + mt := meta.NewMeta(txn) + c.Assert(ok, IsTrue) + err = mt.UpdateTable(db.ID, tblInfo) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + } + updateTableInfo(tblInfo) + tk.MustExec("alter table t5 add column b varchar(10);") // load latest schema. + + tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() + c.Assert(tblInfo.Charset, Equals, "utf8mb4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + + // For model.TableInfoVersion3, it is believed that all charsets / collations are lower-cased, do not do case-convert + tblInfo = tblInfo.Clone() + tblInfo.Version = model.TableInfoVersion3 + tblInfo.Charset = "UTF8MB4" + updateTableInfo(tblInfo) + tk.MustExec("alter table t5 add column c varchar(10);") // load latest schema. + + tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() + c.Assert(tblInfo.Charset, Equals, "UTF8MB4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") } func (s *testIntegrationSuite3) TestZeroFillCreateTable(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index c5867b6648576..5c10247ba7f76 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2292,7 +2292,7 @@ func (s *testDBSuite4) TestCheckTooBigFieldLength(c *C) { assertErrorCode(c, s.tk, "alter table tr_04 add column b varchar(20000) charset utf8mb4;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "alter table tr_04 convert to character set utf8mb4;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "create table tr (id int, name varchar(30000), purchased date ) default charset=utf8 collate=utf8_bin;", tmysql.ErrTooBigFieldlength) - assertErrorCode(c, s.tk, "create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8;", tmysql.ErrTooBigFieldlength) + assertErrorCode(c, s.tk, "create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8_bin;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "create table tr (id int, name varchar(65536), purchased date ) default charset=latin1;", tmysql.ErrTooBigFieldlength) s.tk.MustExec("drop table if exists tr_05;") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index fc27a1be8e2d0..16650d6dbe9d6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1819,6 +1819,11 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 if err != nil { return errors.Trace(err) } + // If newBase < autoIncID, we need to do a rebase before returning. + // Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000. + // If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B, + // and TiDB-B finds 100 < 30001 but returns without any handling, + // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. newBase = mathutil.MaxInt64(newBase, autoIncID) job := &model.Job{ SchemaID: schema.ID, diff --git a/ddl/table.go b/ddl/table.go index 22c8a15fe6b42..dd604a3d8c7c3 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -537,6 +537,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // No need to check `newBase` again, because `RebaseAutoID` will do this check. tblInfo.AutoIncID = newBase tbl, err := getTable(store, schemaID, tblInfo) if err != nil { diff --git a/domain/domain.go b/domain/domain.go index 45b8a1d5cec5a..814452ea9870a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -194,6 +194,7 @@ func (do *Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, // schema is not public, can't be used outside. continue } + infoschema.ConvertCharsetCollateToLowerCaseIfNeed(tbl) di.Tables = append(di.Tables, tbl) } } @@ -791,6 +792,12 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser return err } + do.loadBindInfoLoop() + do.handleInvalidBindTaskLoop() + return nil +} + +func (do *Domain) loadBindInfoLoop() { duration := 3 * time.Second do.wg.Add(1) go func() { @@ -802,13 +809,29 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser return case <-time.After(duration): } - err = do.bindHandle.Update(false) + err := do.bindHandle.Update(false) if err != nil { logutil.Logger(context.Background()).Error("update bindinfo failed", zap.Error(err)) } } }() - return nil +} + +func (do *Domain) handleInvalidBindTaskLoop() { + handleInvalidTaskDuration := 3 * time.Second + do.wg.Add(1) + go func() { + defer do.wg.Done() + defer recoverInDomain("loadBindInfoLoop-dropInvalidBindInfo", false) + for { + select { + case <-do.exit: + return + case <-time.After(handleInvalidTaskDuration): + } + do.bindHandle.DropInvalidBindRecord() + } + }() } // StatsHandle returns the statistic handle. diff --git a/executor/adapter.go b/executor/adapter.go index 983e071661ab8..07ac3fcfd0fa1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -290,16 +290,6 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co var err error defer func() { terror.Log(e.Close()) - txnTS := uint64(0) - // Don't active pending txn here. - if txn, err1 := sctx.Txn(false); err1 != nil { - logutil.Logger(ctx).Error("get current transaction failed", zap.Error(err)) - } else { - if txn.Valid() { - txnTS = txn.StartTS() - } - } - a.LogSlowQuery(txnTS, err == nil) a.logAudit() }() @@ -316,11 +306,11 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { if _, ok := a.Plan.(*plannercore.Execute); !ok { // Do not sync transaction for Execute statement, because the real optimization work is done in // "ExecuteExec.Build". - isPointGet, err := IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, a.Plan) + useMaxTS, err := IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, a.Plan) if err != nil { return nil, err } - if isPointGet { + if useMaxTS { logutil.Logger(context.Background()).Debug("init txnStartTS with MaxUint64", zap.Uint64("conn", ctx.GetSessionVars().ConnectionID), zap.String("text", a.Text)) err = ctx.InitTxnWithStartTS(math.MaxUint64) } else if ctx.GetSessionVars().SnapshotTS != 0 { @@ -335,7 +325,7 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { stmtCtx := ctx.GetSessionVars().StmtCtx if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority { switch { - case isPointGet: + case useMaxTS: stmtCtx.Priority = kv.PriorityHigh case a.Expensive: stmtCtx.Priority = kv.PriorityLow @@ -472,7 +462,7 @@ func (a *ExecStmt) getStatsInfo() map[string]uint64 { // IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions: // 1. ctx is auto commit tagged // 2. txn is not valid -// 2. plan is point get by pk or unique key +// 2. plan is point get by pk, or point get by unique index (no double read) func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannercore.Plan) (bool, error) { // check auto commit if !ctx.GetSessionVars().IsAutocommit() { @@ -500,14 +490,14 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannerco case *plannercore.PhysicalIndexReader: indexScan := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil - case *plannercore.PhysicalIndexLookUpReader: - indexScan := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) - return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil case *plannercore.PhysicalTableReader: tableScan := v.TablePlans[0].(*plannercore.PhysicalTableScan) return len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx.GetSessionVars().StmtCtx), nil case *plannercore.PointGetPlan: - return true, nil + // If the PointGetPlan needs to read data using unique index (double read), we + // can't use max uint64, because using math.MaxUint64 can't guarantee repeatable-read + // and the data and index would be inconsistent! + return v.IndexInfo == nil, nil default: return false, nil } diff --git a/executor/analyze.go b/executor/analyze.go index 9c0e671d77270..5c27ba9e17d39 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -16,6 +16,7 @@ package executor import ( "bytes" "context" + "math" "math/rand" "runtime" "sort" @@ -24,6 +25,7 @@ import ( "sync/atomic" "time" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/debugpb" "github.com/pingcap/parser/model" @@ -145,14 +147,18 @@ const ( colTask taskType = iota idxTask fastTask + pkIncrementalTask + idxIncrementalTask ) type analyzeTask struct { - taskType taskType - idxExec *AnalyzeIndexExec - colExec *AnalyzeColumnsExec - fastExec *AnalyzeFastExec - job *statistics.AnalyzeJob + taskType taskType + idxExec *AnalyzeIndexExec + colExec *AnalyzeColumnsExec + fastExec *AnalyzeFastExec + idxIncrementalExec *analyzeIndexIncrementalExec + colIncrementalExec *analyzePKIncrementalExec + job *statistics.AnalyzeJob } var errAnalyzeWorkerPanic = errors.New("analyze worker panic") @@ -160,8 +166,8 @@ var errAnalyzeWorkerPanic = errors.New("analyze worker panic") func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- analyzeResult, isCloseChanThread bool) { defer func() { e.wg.Done() - e.wg.Wait() if isCloseChanThread { + e.wg.Wait() close(resultCh) } if r := recover(); r != nil { @@ -180,25 +186,41 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- if !ok { break } + task.job.Start() switch task.taskType { case colTask: task.colExec.job = task.job - task.job.Start() resultCh <- analyzeColumnsPushdown(task.colExec) case idxTask: task.idxExec.job = task.job - task.job.Start() resultCh <- analyzeIndexPushdown(task.idxExec) case fastTask: + task.fastExec.job = task.job + task.job.Start() for _, result := range analyzeFastExec(task.fastExec) { resultCh <- result } + case pkIncrementalTask: + task.colIncrementalExec.job = task.job + resultCh <- analyzePKIncremental(task.colIncrementalExec) + case idxIncrementalTask: + task.idxIncrementalExec.job = task.job + resultCh <- analyzeIndexIncremental(task.idxIncrementalExec) } } } func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) analyzeResult { - hist, cms, err := idxExec.buildStats() + ranges := ranger.FullRange() + // For single-column index, we do not load null rows from TiKV, so the built histogram would not include + // null values, and its `NullCount` would be set by result of another distsql call to get null rows. + // For multi-column index, we cannot define null for the rows, so we still use full range, and the rows + // containing null fields would exist in built histograms. Note that, the `NullCount` of histograms for + // multi-column index is always 0 then. + if len(idxExec.idxInfo.Columns) == 1 { + ranges = ranger.FullNotNullRange() + } + hist, cms, err := idxExec.buildStats(ranges, true) if err != nil { return analyzeResult{Err: err, job: idxExec.job} } @@ -257,21 +279,12 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang return nil } -func (e *AnalyzeIndexExec) open() error { - ranges := ranger.FullRange() - // For single-column index, we do not load null rows from TiKV, so the built histogram would not include - // null values, and its `NullCount` would be set by result of another distsql call to get null rows. - // For multi-column index, we cannot define null for the rows, so we still use full range, and the rows - // containing null fields would exist in built histograms. Note that, the `NullCount` of histograms for - // multi-column index is always 0 then. - if len(e.idxInfo.Columns) == 1 { - ranges = ranger.FullNotNullRange() - } +func (e *AnalyzeIndexExec) open(ranges []*ranger.Range, considerNull bool) error { err := e.fetchAnalyzeResult(ranges, false) if err != nil { return err } - if len(e.idxInfo.Columns) == 1 { + if considerNull && len(e.idxInfo.Columns) == 1 { ranges = ranger.NullRange() err = e.fetchAnalyzeResult(ranges, true) if err != nil { @@ -309,19 +322,16 @@ func (e *AnalyzeIndexExec) buildStatsFromResult(result distsql.SelectResult, nee if needCMS { if resp.Cms == nil { logutil.Logger(context.TODO()).Warn("nil CMS in response", zap.String("table", e.idxInfo.Table.O), zap.String("index", e.idxInfo.Name.O)) - } else { - err := cms.MergeCMSketch(statistics.CMSketchFromProto(resp.Cms)) - if err != nil { - return nil, nil, err - } + } else if err := cms.MergeCMSketch(statistics.CMSketchFromProto(resp.Cms)); err != nil { + return nil, nil, err } } } return hist, cms, nil } -func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statistics.CMSketch, err error) { - if err = e.open(); err != nil { +func (e *AnalyzeIndexExec) buildStats(ranges []*ranger.Range, considerNull bool) (hist *statistics.Histogram, cms *statistics.CMSketch, err error) { + if err = e.open(ranges, considerNull); err != nil { return nil, nil, err } defer func() { @@ -345,7 +355,13 @@ func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statis } func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) analyzeResult { - hists, cms, err := colExec.buildStats() + var ranges []*ranger.Range + if colExec.pkInfo != nil { + ranges = ranger.FullIntRange(mysql.HasUnsignedFlag(colExec.pkInfo.Flag)) + } else { + ranges = ranger.FullIntRange(false) + } + hists, cms, err := colExec.buildStats(ranges) if err != nil { return analyzeResult{Err: err, job: colExec.job} } @@ -377,13 +393,7 @@ type AnalyzeColumnsExec struct { job *statistics.AnalyzeJob } -func (e *AnalyzeColumnsExec) open() error { - var ranges []*ranger.Range - if e.pkInfo != nil { - ranges = ranger.FullIntRange(mysql.HasUnsignedFlag(e.pkInfo.Flag)) - } else { - ranges = ranger.FullIntRange(false) - } +func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { e.resultHandler = &tableResultHandler{} firstPartRanges, secondPartRanges := splitRanges(ranges, true, false) firstResult, err := e.buildResp(firstPartRanges) @@ -425,8 +435,8 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } -func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []*statistics.CMSketch, err error) { - if err = e.open(); err != nil { +func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range) (hists []*statistics.Histogram, cms []*statistics.CMSketch, err error) { + if err = e.open(ranges); err != nil { return nil, nil, err } defer func() { @@ -507,7 +517,7 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms [] func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { hists, cms, err := exec.buildStats() if err != nil { - return []analyzeResult{{Err: err}} + return []analyzeResult{{Err: err, job: exec.job}} } var results []analyzeResult hasPKInfo := 0 @@ -522,6 +532,7 @@ func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { Cms: []*statistics.CMSketch{cms[i]}, IsIndex: 1, Count: hists[i].NullCount, + job: exec.job, } if hists[i].Len() > 0 { idxResult.Count += hists[i].Buckets[hists[i].Len()-1].Count @@ -535,6 +546,7 @@ func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { Hist: hists[:hasPKInfo+len(exec.colsInfo)], Cms: cms[:hasPKInfo+len(exec.colsInfo)], Count: hist.NullCount, + job: exec.job, } if hist.Len() > 0 { colResult.Count += hist.Buckets[hist.Len()-1].Count @@ -560,6 +572,7 @@ type AnalyzeFastExec struct { idxsInfo []*model.IndexInfo concurrency int maxNumBuckets uint64 + tblInfo *model.TableInfo cache *tikv.RegionCache wg *sync.WaitGroup sampLocs chan *tikv.KeyLocation @@ -569,6 +582,7 @@ type AnalyzeFastExec struct { scanTasks []*tikv.KeyLocation collectors []*statistics.SampleCollector randSeed int64 + job *statistics.AnalyzeJob } func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild *bool, err *error, sampTasks *[]*AnalyzeFastTask) { @@ -627,12 +641,12 @@ func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild } } -// buildSampTask return tow variable, the first bool is whether the task meeting region error +// buildSampTask returns tow variables, the first bool is whether the task meets region error // and need to rebuild. func (e *AnalyzeFastExec) buildSampTask() (needRebuild bool, err error) { // Do get regions row count. bo := tikv.NewBackoffer(context.Background(), 500) - atomic.StoreUint64(&e.rowCount, 0) + e.rowCount = 0 needRebuildForRoutine := make([]bool, e.concurrency) errs := make([]error, e.concurrency) sampTasksForRoutine := make([][]*AnalyzeFastTask, e.concurrency) @@ -721,6 +735,11 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if err != nil { return err } + var rowID int64 + rowID, err = tablecodec.DecodeRowKey(sKey) + if err != nil { + return err + } // Update the primary key collector. if hasPKInfo > 0 { v, ok := values[e.pkInfo.ID] @@ -735,7 +754,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[0].Samples[samplePos] == nil { e.collectors[0].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[0].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[0].Samples[samplePos].RowID = rowID e.collectors[0].Samples[samplePos].Value = v } // Update the columns' collectors. @@ -747,7 +766,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[hasPKInfo+j].Samples[samplePos] == nil { e.collectors[hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[hasPKInfo+j].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[hasPKInfo+j].Samples[samplePos].RowID = rowID e.collectors[hasPKInfo+j].Samples[samplePos].Value = v } // Update the indexes' collectors. @@ -773,7 +792,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] == nil { e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].RowID = rowID e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].Value = types.NewBytesDatum(bytes) } return nil @@ -844,9 +863,7 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) error { } func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *error) { - defer func() { - e.wg.Done() - }() + defer e.wg.Done() var snapshot kv.Snapshot snapshot, *err = e.ctx.GetStore().(tikv.Storage).GetSnapshot(kv.MaxVersion) rander := rand.New(rand.NewSource(e.randSeed + int64(workID))) @@ -869,9 +886,6 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e if *err != nil { return } - if maxRowID <= minRowID { - continue - } keys := make([]kv.Key, 0, task.SampSize) for i := 0; i < int(task.SampSize); i++ { @@ -897,8 +911,37 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e } func (e *AnalyzeFastExec) buildHist(ID int64, collector *statistics.SampleCollector, tp *types.FieldType) (*statistics.Histogram, error) { - // TODO: build histogram and cmsketch here for one collector. - return nil, nil + // build collector properties. + collector.Samples = collector.Samples[:e.sampCursor] + sort.Slice(collector.Samples, func(i, j int) bool { return collector.Samples[i].RowID < collector.Samples[j].RowID }) + collector.CalcTotalSize() + data := make([][]byte, 0, len(collector.Samples)) + for i, sample := range collector.Samples { + sample.Ordinal = i + if sample.Value.IsNull() { + collector.NullCount++ + continue + } + bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, sample.Value) + if err != nil { + return nil, err + } + data = append(data, bytes) + } + stats := domain.GetDomain(e.ctx).StatsHandle() + rowCount := int64(e.rowCount) + if stats.Lease > 0 { + rowCount = mathutil.MinInt64(stats.GetTableStats(e.tblInfo).Count, rowCount) + } + // build CMSketch + var ndv, scaleRatio uint64 + collector.CMSketch, ndv, scaleRatio = statistics.NewCMSketchWithTopN(defaultCMSketchDepth, defaultCMSketchWidth, data, 20, uint64(rowCount)) + // build Histogram + hist, err := statistics.BuildColumnHist(e.ctx, int64(e.maxNumBuckets), ID, collector, tp, rowCount, int64(ndv), collector.NullCount*int64(scaleRatio)) + if err != nil { + return nil, err + } + return hist, nil } func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMSketch, error) { @@ -974,6 +1017,8 @@ func (e *AnalyzeFastExec) buildStats() (hists []*statistics.Histogram, cms []*st return nil, nil, errors.Errorf(errMsg, maxBuildTimes) } + defer e.job.Update(int64(e.rowCount)) + // If total row count of the table is smaller than 2*MaxSampleSize, we // translate all the sample tasks to scan tasks. if e.rowCount < uint64(MaxSampleSize)*2 { @@ -1009,6 +1054,7 @@ type AnalyzeTestFastExec struct { IdxsInfo []*model.IndexInfo Concurrency int Collectors []*statistics.SampleCollector + TblInfo *model.TableInfo } // TestFastSample only test the fast sample in unit test. @@ -1020,11 +1066,95 @@ func (e *AnalyzeTestFastExec) TestFastSample() error { e.concurrency = e.Concurrency e.physicalTableID = e.PhysicalTableID e.wg = &sync.WaitGroup{} + e.job = &statistics.AnalyzeJob{} + e.tblInfo = e.TblInfo _, _, err := e.buildStats() e.Collectors = e.collectors return err } +type analyzeIndexIncrementalExec struct { + AnalyzeIndexExec + index *statistics.Index +} + +func analyzeIndexIncremental(idxExec *analyzeIndexIncrementalExec) analyzeResult { + idx := idxExec.index + highBound := idx.Histogram.GetUpper(idx.Len() - 1) + values, err := codec.Decode(highBound.GetBytes(), len(idxExec.idxInfo.Columns)) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + ran := ranger.Range{LowVal: values, HighVal: []types.Datum{types.MaxValueDatum()}} + hist, cms, err := idxExec.buildStats([]*ranger.Range{&ran}, false) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + oldHist, oldCMS, err := idx.RemoveUpperBound(idxExec.ctx.GetSessionVars().StmtCtx, values) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + hist, err = statistics.MergeHistograms(idxExec.ctx.GetSessionVars().StmtCtx, oldHist, hist, int(idxExec.maxNumBuckets)) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + if oldCMS != nil && cms != nil { + err = cms.MergeCMSketch(oldCMS) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + } + result := analyzeResult{ + PhysicalTableID: idxExec.physicalTableID, + Hist: []*statistics.Histogram{hist}, + Cms: []*statistics.CMSketch{cms}, + IsIndex: 1, + job: idxExec.job, + } + result.Count = hist.NullCount + if hist.Len() > 0 { + result.Count += hist.Buckets[hist.Len()-1].Count + } + return result +} + +type analyzePKIncrementalExec struct { + AnalyzeColumnsExec + pkStats *statistics.Column +} + +func analyzePKIncremental(colExec *analyzePKIncrementalExec) analyzeResult { + pkStats := colExec.pkStats + high := pkStats.GetUpper(pkStats.Len() - 1) + var maxVal types.Datum + if mysql.HasUnsignedFlag(colExec.pkInfo.Flag) { + maxVal = types.NewUintDatum(math.MaxUint64) + } else { + maxVal = types.NewIntDatum(math.MaxInt64) + } + ran := ranger.Range{LowVal: []types.Datum{*high}, LowExclude: true, HighVal: []types.Datum{maxVal}} + hists, _, err := colExec.buildStats([]*ranger.Range{&ran}) + if err != nil { + return analyzeResult{Err: err, job: colExec.job} + } + hist := hists[0] + oldHist := pkStats.Histogram.Copy() + hist, err = statistics.MergeHistograms(colExec.ctx.GetSessionVars().StmtCtx, oldHist, hist, int(colExec.maxNumBuckets)) + if err != nil { + return analyzeResult{Err: err, job: colExec.job} + } + result := analyzeResult{ + PhysicalTableID: colExec.physicalTableID, + Hist: []*statistics.Histogram{hist}, + Cms: []*statistics.CMSketch{nil}, + job: colExec.job, + } + if hist.Len() > 0 { + result.Count += hist.Buckets[hist.Len()-1].Count + } + return result +} + // analyzeResult is used to represent analyze result. type analyzeResult struct { // PhysicalTableID is the id of a partition or a table. diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 1c9c8c7060649..c33880d4f4b90 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -140,6 +140,8 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { ) c.Assert(err, IsNil) var dom *domain.Domain + session.SetStatsLease(0) + session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) tk := testkit.NewTestKit(c, store) @@ -183,6 +185,7 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { IdxsInfo: indicesInfo, Concurrency: 1, PhysicalTableID: tbl.(table.PhysicalTable).GetPhysicalID(), + TblInfo: tblInfo, } err = mockExec.TestFastSample() c.Assert(err, IsNil) @@ -197,5 +200,107 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { vals[i] = append(vals[i], s) } } - c.Assert(fmt.Sprintln(vals), Equals, "[[0 34 35 57 4 24 6 25 58 9 10 11 12 30 14 52 29 17 44 54] [0 34 35 57 4 24 6 25 58 9 10 11 12 30 14 52 29 17 44 54]]\n") + c.Assert(fmt.Sprintln(vals), Equals, "[[0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58] [0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58]]\n") +} + +func (s *testSuite1) TestFastAnalyze(c *C) { + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + store, err := mockstore.NewMockTikvStore( + mockstore.WithCluster(cluster), + ) + c.Assert(err, IsNil) + var dom *domain.Domain + session.SetStatsLease(0) + session.SetSchemaLease(0) + dom, err = session.BootstrapSession(store) + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + executor.MaxSampleSize = 1000 + executor.RandSeed = 123 + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, index index_b(b))") + tk.MustExec("set @@session.tidb_enable_fast_analyze=1") + tk.MustExec("set @@session.tidb_build_stats_concurrency=1") + for i := 0; i < 3000; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tid := tblInfo.Meta().ID + + // construct 5 regions split by {600, 1200, 1800, 2400} + splitKeys := generateTableSplitKeyForInt(tid, []int{600, 1200, 1800, 2400}) + manipulateCluster(cluster, splitKeys) + + tk.MustExec("analyze table t with 5 buckets") + + is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := table.Meta() + tbl := dom.StatsHandle().GetTableStats(tableInfo) + sTbl := fmt.Sprintln(tbl) + matched := false + if sTbl == "Table:39 Count:3000\n"+ + "column:1 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "column:2 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "index:1 ndv:3000\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" || + sTbl == "Table:39 Count:3000\n"+ + "column:2 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "column:1 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "index:1 ndv:3000\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" { + matched = true + } + c.Assert(matched, Equals, true) +} + +func (s *testSuite1) TestAnalyzeIncremental(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 int, b int, primary key(a), index idx(b))") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows()) + tk.MustExec("insert into t values (1,1)") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t idx 1 0 1 1 1 1")) + tk.MustExec("insert into t values (2,2)") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) + tk.MustExec("analyze incremental table t index") + // Result should not change. + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) } diff --git a/executor/bind.go b/executor/bind.go index 45262f39de75d..d2c2034851bde 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -57,28 +57,30 @@ func (e *SQLBindExec) Next(ctx context.Context, req *chunk.RecordBatch) error { } 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, } + if !e.isGlobal { + handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + handle.DropBindRecord(record) + return nil + } 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, + Status: bindinfo.Using, + } + if !e.isGlobal { + handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + return handle.AddBindRecord(record) } return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(record) } diff --git a/executor/builder.go b/executor/builder.go index 55142e42995cf..596b64c72b48c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -1351,7 +1350,7 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } -func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64) *AnalyzeIndexExec { +func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64, autoAnalyze string) *analyzeTask { _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) e := &AnalyzeIndexExec{ ctx: b.ctx, @@ -1374,10 +1373,35 @@ func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeInde width := int32(defaultCMSketchWidth) e.analyzePB.IdxReq.CmsketchDepth = &depth e.analyzePB.IdxReq.CmsketchWidth = &width - return e + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O} + return &analyzeTask{taskType: idxTask, idxExec: e, job: job} +} + +func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64) *analyzeTask { + h := domain.GetDomain(b.ctx).StatsHandle() + statsTbl := h.GetPartitionStats(&model.TableInfo{}, task.PhysicalTableID) + analyzeTask := b.buildAnalyzeIndexPushdown(task, maxNumBuckets, "") + if statsTbl.Pseudo { + return analyzeTask + } + idx, ok := statsTbl.Indices[task.IndexInfo.ID] + // TODO: If the index contains feedback, we may use other strategy. + if !ok || idx.Len() == 0 || idx.ContainsFeedback() { + return analyzeTask + } + exec := analyzeTask.idxExec + if idx.CMSketch != nil { + width, depth := idx.CMSketch.GetWidthAndDepth() + exec.analyzePB.IdxReq.CmsketchWidth = &width + exec.analyzePB.IdxReq.CmsketchDepth = &depth + } + analyzeTask.taskType = idxIncrementalTask + analyzeTask.idxIncrementalExec = &analyzeIndexIncrementalExec{AnalyzeIndexExec: *analyzeTask.idxExec, index: idx} + analyzeTask.job = &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "analyze incremental index " + task.IndexInfo.Name.O} + return analyzeTask } -func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) *AnalyzeColumnsExec { +func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64, autoAnalyze string) *analyzeTask { cols := task.ColsInfo if task.PKInfo != nil { cols = append([]*model.ColumnInfo{task.PKInfo}, cols...) @@ -1409,7 +1433,27 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo CmsketchWidth: &width, } b.err = plannercore.SetPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, cols) - return e + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze columns"} + return &analyzeTask{taskType: colTask, colExec: e, job: job} +} + +func (b *executorBuilder) buildAnalyzePKIncremental(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) *analyzeTask { + h := domain.GetDomain(b.ctx).StatsHandle() + statsTbl := h.GetPartitionStats(&model.TableInfo{}, task.PhysicalTableID) + analyzeTask := b.buildAnalyzeColumnsPushdown(task, maxNumBuckets, "") + if statsTbl.Pseudo { + return analyzeTask + } + col, ok := statsTbl.Columns[task.PKInfo.ID] + // TODO: If the primary key contains feedback, we may use other strategy. + if !ok || col.Len() == 0 || col.ContainsFeedback() { + return analyzeTask + } + exec := analyzeTask.colExec + analyzeTask.taskType = pkIncrementalTask + analyzeTask.colIncrementalExec = &analyzePKIncrementalExec{AnalyzeColumnsExec: *exec, pkStats: col} + analyzeTask.job = &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "analyze incremental primary key"} + return analyzeTask } func (b *executorBuilder) buildAnalyzeFastColumn(e *AnalyzeExec, task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) { @@ -1435,9 +1479,11 @@ func (b *executorBuilder) buildAnalyzeFastColumn(e *AnalyzeExec, task plannercor colsInfo: task.ColsInfo, pkInfo: task.PKInfo, maxNumBuckets: maxNumBuckets, + tblInfo: task.TblInfo, concurrency: concurrency, wg: &sync.WaitGroup{}, }, + job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "fast analyze columns"}, }) } } @@ -1464,9 +1510,11 @@ func (b *executorBuilder) buildAnalyzeFastIndex(e *AnalyzeExec, task plannercore physicalTableID: task.PhysicalTableID, idxsInfo: []*model.IndexInfo{task.IndexInfo}, maxNumBuckets: maxNumBuckets, + tblInfo: task.TblInfo, concurrency: concurrency, wg: &sync.WaitGroup{}, }, + job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: "fast analyze index " + task.IndexInfo.Name.O}, }) } } @@ -1483,28 +1531,28 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) Executor { autoAnalyze = "auto " } for _, task := range v.ColTasks { - if enableFastAnalyze { - b.buildAnalyzeFastColumn(e, task, v.MaxNumBuckets) + if task.Incremental { + e.tasks = append(e.tasks, b.buildAnalyzePKIncremental(task, v.MaxNumBuckets)) } else { - e.tasks = append(e.tasks, &analyzeTask{ - taskType: colTask, - colExec: b.buildAnalyzeColumnsPushdown(task, v.MaxNumBuckets), - job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze columns"}, - }) + if enableFastAnalyze { + b.buildAnalyzeFastColumn(e, task, v.MaxNumBuckets) + } else { + e.tasks = append(e.tasks, b.buildAnalyzeColumnsPushdown(task, v.MaxNumBuckets, autoAnalyze)) + } } if b.err != nil { return nil } } for _, task := range v.IdxTasks { - if enableFastAnalyze { - b.buildAnalyzeFastIndex(e, task, v.MaxNumBuckets) + if task.Incremental { + e.tasks = append(e.tasks, b.buildAnalyzeIndexIncremental(task, v.MaxNumBuckets)) } else { - e.tasks = append(e.tasks, &analyzeTask{ - taskType: idxTask, - idxExec: b.buildAnalyzeIndexPushdown(task, v.MaxNumBuckets), - job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O}, - }) + if enableFastAnalyze { + b.buildAnalyzeFastIndex(e, task, v.MaxNumBuckets) + } else { + e.tasks = append(e.tasks, b.buildAnalyzeIndexPushdown(task, v.MaxNumBuckets, autoAnalyze)) + } } if b.err != nil { return nil @@ -1625,6 +1673,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) isOuterJoin: v.JoinType.IsOuterJoin(), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, + lastColHelper: v.CompareFilters, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { @@ -1863,25 +1912,25 @@ type dataReaderBuilder struct { selectResultHook // for testing } -func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, datums [][]types.Datum, - IndexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { +func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { switch v := builder.Plan.(type) { case *plannercore.PhysicalTableReader: - return builder.buildTableReaderForIndexJoin(ctx, v, datums) + return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents) case *plannercore.PhysicalIndexReader: - return builder.buildIndexReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalIndexLookUpReader: - return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalUnionScan: - return builder.buildUnionScanForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildUnionScanForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) } return nil, errors.New("Wrong plan type for dataReaderBuilder") } func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { childBuilder := &dataReaderBuilder{Plan: v.Children()[0], executorBuilder: builder.executorBuilder} - reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff) + reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1894,14 +1943,14 @@ func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context return us, nil } -func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, datums [][]types.Datum) (Executor, error) { +func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent) (Executor, error) { e, err := buildNoRangeTableReader(builder.executorBuilder, v) if err != nil { return nil, err } - handles := make([]int64, 0, len(datums)) - for _, datum := range datums { - handles = append(handles, datum[0].GetInt64()) + handles := make([]int64, 0, len(lookUpContents)) + for _, content := range lookUpContents { + handles = append(handles, content.keys[0].GetInt64()) } return builder.buildTableReaderFromHandles(ctx, e, handles) } @@ -1935,12 +1984,12 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex } func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexReader(builder.executorBuilder, v) if err != nil { return nil, err } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1949,12 +1998,12 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexLookUpReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexLookUpReader(builder.executorBuilder, v) if err != nil { return nil, err } - e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.index.ID, values, indexRanges, keyOff2IdxOff) + e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1963,17 +2012,40 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context } // buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan. -func buildKvRangesForIndexJoin(sc *stmtctx.StatementContext, tableID, indexID int64, keyDatums [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) ([]kv.KeyRange, error) { - kvRanges := make([]kv.KeyRange, 0, len(indexRanges)*len(keyDatums)) - for _, val := range keyDatums { - for _, ran := range indexRanges { +func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, + ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) ([]kv.KeyRange, error) { + kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) + lastPos := len(ranges[0].LowVal) - 1 + sc := ctx.GetSessionVars().StmtCtx + for _, content := range lookUpContents { + for _, ran := range ranges { for keyOff, idxOff := range keyOff2IdxOff { - ran.LowVal[idxOff] = val[keyOff] - ran.HighVal[idxOff] = val[keyOff] + ran.LowVal[idxOff] = content.keys[keyOff] + ran.HighVal[idxOff] = content.keys[keyOff] + } + } + if cwc != nil { + nextColRanges, err := cwc.BuildRangesByRow(ctx, content.row) + if err != nil { + return nil, err } + for _, nextColRan := range nextColRanges { + for _, ran := range ranges { + ran.LowVal[lastPos] = nextColRan.LowVal[0] + ran.HighVal[lastPos] = nextColRan.HighVal[0] + ran.LowExclude = nextColRan.LowExclude + ran.HighExclude = nextColRan.HighExclude + } + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) + if err != nil { + return nil, errors.Trace(err) + } + kvRanges = append(kvRanges, tmpKvRanges...) + } + continue } - tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, indexRanges, nil) + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) if err != nil { return nil, err } diff --git a/executor/compiler.go b/executor/compiler.go index 23b2239475ca9..f233bdfe833cd 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -16,10 +16,14 @@ package executor import ( "context" "fmt" + "strings" "github.com/opentracing/opentracing-go" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner" @@ -49,11 +53,24 @@ type Compiler struct { // Compile compiles an ast.StmtNode to a physical plan. func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStmt, error) { + return c.compile(ctx, stmtNode, false) +} + +// SkipBindCompile compiles an ast.StmtNode to a physical plan without SQL bind. +func (c *Compiler) SkipBindCompile(ctx context.Context, node ast.StmtNode) (*ExecStmt, error) { + return c.compile(ctx, node, true) +} + +func (c *Compiler) compile(ctx context.Context, stmtNode ast.StmtNode, skipBind bool) (*ExecStmt, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("executor.Compile", opentracing.ChildOf(span.Context())) defer span1.Finish() } + if !skipBind { + stmtNode = addHint(c.Ctx, stmtNode) + } + infoSchema := GetInfoSchema(c.Ctx) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err @@ -367,3 +384,46 @@ func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema { } return is } + +func addHint(ctx sessionctx.Context, stmtNode ast.StmtNode) ast.StmtNode { + if ctx.Value(bindinfo.SessionBindInfoKeyType) == nil { //when the domain is initializing, the bind will be nil. + return stmtNode + } + switch x := stmtNode.(type) { + case *ast.ExplainStmt: + switch x.Stmt.(type) { + case *ast.SelectStmt: + normalizeExplainSQL := parser.Normalize(x.Text()) + idx := strings.Index(normalizeExplainSQL, "select") + normalizeSQL := normalizeExplainSQL[idx:] + x.Stmt = addHintForSelect(normalizeSQL, ctx, x.Stmt) + } + return x + case *ast.SelectStmt: + return addHintForSelect(parser.Normalize(x.Text()), ctx, x) + default: + return stmtNode + } +} + +func addHintForSelect(normdOrigSQL string, ctx sessionctx.Context, stmt ast.StmtNode) ast.StmtNode { + sessionHandle := ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindRecord := sessionHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) + if bindRecord != nil { + if bindRecord.Status == bindinfo.Invalid { + return stmt + } + if bindRecord.Status == bindinfo.Using { + return bindinfo.BindHint(stmt, bindRecord.Ast) + } + } + globalHandle := domain.GetDomain(ctx).BindHandle() + bindRecord = globalHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) + if bindRecord == nil { + bindRecord = globalHandle.GetBindRecord(normdOrigSQL, "") + } + if bindRecord != nil { + return bindinfo.BindHint(stmt, bindRecord.Ast) + } + return stmt +} diff --git a/executor/distsql.go b/executor/distsql.go index e93d67cb16670..5bd77376292c1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -519,6 +519,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha e.tblWorkerWg.Add(lookupConcurrencyLimit) for i := 0; i < lookupConcurrencyLimit; i++ { worker := &tableWorker{ + idxLookup: e, workCh: workCh, finished: e.finished, buildTblReader: e.buildTableReader, @@ -732,6 +733,7 @@ func (w *indexWorker) buildTableTask(handles []int64) *lookupTableTask { // tableWorker is used by IndexLookUpExecutor to maintain table lookup background goroutines. type tableWorker struct { + idxLookup *IndexLookUpExecutor workCh <-chan *lookupTableTask finished <-chan struct{} buildTblReader func(ctx context.Context, handles []int64) (Executor, error) @@ -809,6 +811,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er task.rows = append(task.rows, row) } } + memUsage = int64(cap(task.rows)) * int64(unsafe.Sizeof(chunk.Row{})) task.memUsage += memUsage task.memTracker.Consume(memUsage) @@ -824,14 +827,23 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er sort.Sort(task) } - if w.isCheckOp && handleCnt != len(task.rows) { - obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) - for _, row := range task.rows { - handle := row.GetInt64(w.handleIdx) - obtainedHandlesMap[handle] = struct{}{} + if handleCnt != len(task.rows) { + if w.isCheckOp { + obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) + for _, row := range task.rows { + handle := row.GetInt64(w.handleIdx) + obtainedHandlesMap[handle] = struct{}{} + } + return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d, missing handles %v in a batch", + w.idxLookup.index.Name.O, handleCnt, len(task.rows), GetLackHandles(task.handles, obtainedHandlesMap)) + } + + if len(w.idxLookup.tblPlans) == 1 { + // table scan in double read can never has conditions according to convertToIndexScan. + // if this table scan has no condition, the number of rows it returns must equal to the length of handles. + return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", + w.idxLookup.index.Name.O, handleCnt, len(task.rows)) } - return errors.Errorf("handle count %d isn't equal to value count %d, missing handles %v in a batch", - handleCnt, len(task.rows), GetLackHandles(task.handles, obtainedHandlesMap)) } return nil diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 5ffb18036839b..814e6f47fe1bb 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -25,7 +25,10 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) @@ -184,3 +187,53 @@ func (s *testSuite3) TestIssue10178(c *C) { tk.MustQuery("select * from t where a > 9223372036854775807").Check(testkit.Rows("18446744073709551615")) tk.MustQuery("select * from t where a < 9223372036854775808").Check(testkit.Rows("9223372036854775807")) } + +func (s *testSuite3) TestInconsistentIndex(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 int, b int, index idx_a(a))") + is := s.domain.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + idx := tbl.Meta().FindIndexByName("idx_a") + idxOp := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), idx) + ctx := mock.NewContext() + ctx.Store = s.store + + for i := 0; i < 10; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i)) + c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil) + } + + for i := 0; i < 10; i++ { + tk.MustExec(fmt.Sprintf("update t set a=%d where a=%d", i, i+10)) + c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil) + } + + for i := 0; i < 10; i++ { + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = idxOp.Create(ctx, txn, types.MakeDatums(i+10), int64(100+i)) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + err = tk.QueryToErr("select * from t use index(idx_a) where a >= 0") + c.Assert(err.Error(), Equals, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11)) + + // if has other conditions, the inconsistent index check doesn't work. + err = tk.QueryToErr("select * from t where a>=0 and b<10") + c.Assert(err, IsNil) + } + + // fix inconsistent problem to pass CI + for i := 0; i < 10; i++ { + txn, err := s.store.Begin() + c.Assert(err, IsNil) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), int64(100+i), nil) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + } +} diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index c2c274455c7e2..b237236e9b31b 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -15,7 +15,6 @@ package executor import ( "context" - "time" . "github.com/pingcap/check" "github.com/pingcap/parser/ast" @@ -23,7 +22,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -137,7 +135,7 @@ func buildSchema(names []string, ftypes []byte) *expression.Schema { return schema } -func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { +func (s *testExecSuite) TestBuildKvRangesForIndexJoinWithoutCwc(c *C) { indexRanges := make([]*ranger.Range, 0, 6) indexRanges = append(indexRanges, generateIndexRange(1, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(1, 1, 2, 1, 1)) @@ -146,16 +144,16 @@ func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { indexRanges = append(indexRanges, generateIndexRange(2, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(2, 1, 2, 1, 1)) - joinKeyRows := make([][]types.Datum, 0, 5) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 3)) + joinKeyRows := make([]*indexJoinLookUpContent, 0, 5) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 3)}) keyOff2IdxOff := []int{1, 3} - sc := &stmtctx.StatementContext{TimeZone: time.Local} - kvRanges, err := buildKvRangesForIndexJoin(sc, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff) + ctx := mock.NewContext() + kvRanges, err := buildKvRangesForIndexJoin(ctx, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff, nil) c.Assert(err, IsNil) // Check the kvRanges is in order. for i, kvRange := range kvRanges { diff --git a/executor/executor_test.go b/executor/executor_test.go index 60707de717b93..9f24b23c0206a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -29,6 +29,7 @@ import ( "github.com/golang/protobuf/proto" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/parser" @@ -66,6 +67,7 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" + "github.com/tiancaiamao/debugger" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -1853,7 +1855,8 @@ func (s *testSuite) TestIsPointGet(c *C) { tk.MustExec("use mysql") ctx := tk.Se.(sessionctx.Context) tests := map[string]bool{ - "select * from help_topic where name='aaa'": true, + "select * from help_topic where name='aaa'": false, + "select 1 from help_topic where name='aaa'": true, "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } @@ -1872,6 +1875,34 @@ func (s *testSuite) TestIsPointGet(c *C) { } } +func (s *testSuite) TestPointGetRepeatableRead(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest") + + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use test") + tk1.MustExec(`create table point_get (a int, b int, c int, + primary key k_a(a), + unique key k_b(b))`) + tk1.MustExec("insert into point_get values (1, 1, 1)") + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use test") + + go func() { + ctx := context.WithValue(context.Background(), "pointGetRepeatableReadTest", true) + rs, err := tk1.Se.Execute(ctx, "select c from point_get where b = 1") + c.Assert(err, IsNil) + result := tk1.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute sql fail")) + result.Check(testkit.Rows("1")) + }() + + label := debugger.Bind("point-get-g2") + debugger.Continue("point-get-g1") + debugger.Breakpoint(label) + tk2.MustExec("update point_get set b = 2, c = 2 where a = 1") + debugger.Continue("point-get-g1") +} + func (s *testSuite) TestRow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index e49baf3c6290a..3d9371d14d396 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -73,6 +74,9 @@ type IndexLookUpJoin struct { keyOff2IdxOff []int innerPtrBytes [][]byte + // lastColHelper store the information for last col if there's complicated filter like col > x_col and col < x_col + 100. + lastColHelper *plannercore.ColWithCmpFuncManager + memTracker *memory.Tracker // track memory usage. } @@ -132,8 +136,9 @@ type innerWorker struct { ctx sessionctx.Context executorChk *chunk.Chunk - indexRanges []*ranger.Range - keyOff2IdxOff []int + indexRanges []*ranger.Range + nextColCompareFilters *plannercore.ColWithCmpFuncManager + keyOff2IdxOff []int } // Open implements the Executor interface. @@ -209,13 +214,14 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork copiedRanges = append(copiedRanges, ran.Clone()) } iw := &innerWorker{ - innerCtx: e.innerCtx, - outerCtx: e.outerCtx, - taskCh: taskCh, - ctx: e.ctx, - executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), - indexRanges: copiedRanges, - keyOff2IdxOff: e.keyOff2IdxOff, + innerCtx: e.innerCtx, + outerCtx: e.outerCtx, + taskCh: taskCh, + ctx: e.ctx, + executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), + indexRanges: copiedRanges, + keyOff2IdxOff: e.keyOff2IdxOff, + nextColCompareFilters: e.lastColHelper, } return iw } @@ -447,13 +453,18 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { } } +type indexJoinLookUpContent struct { + keys []types.Datum + row chunk.Row +} + func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { - dLookUpKeys, err := iw.constructDatumLookupKeys(task) + lookUpContents, err := iw.constructLookupContent(task) if err != nil { return err } - dLookUpKeys = iw.sortAndDedupDatumLookUpKeys(dLookUpKeys) - err = iw.fetchInnerResults(ctx, task, dLookUpKeys) + lookUpContents = iw.sortAndDedupLookUpContents(lookUpContents) + err = iw.fetchInnerResults(ctx, task, lookUpContents) if err != nil { return err } @@ -464,8 +475,8 @@ func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) err return nil } -func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types.Datum, error) { - dLookUpKeys := make([][]types.Datum, 0, task.outerResult.NumRows()) +func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoinLookUpContent, error) { + lookUpContents := make([]*indexJoinLookUpContent, 0, task.outerResult.NumRows()) keyBuf := make([]byte, 0, 64) for i := 0; i < task.outerResult.NumRows(); i++ { dLookUpKey, err := iw.constructDatumLookupKey(task, i) @@ -484,11 +495,11 @@ func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types } // Store the encoded lookup key in chunk, so we can use it to lookup the matched inners directly. task.encodedLookUpKeys.AppendBytes(0, keyBuf) - dLookUpKeys = append(dLookUpKeys, dLookUpKey) + lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: task.outerResult.GetRow(i)}) } task.memTracker.Consume(task.encodedLookUpKeys.MemoryUsage()) - return dLookUpKeys, nil + return lookUpContents, nil } func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) ([]types.Datum, error) { @@ -510,6 +521,10 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) innerColType := iw.rowTypes[iw.keyCols[i]] innerValue, err := outerValue.ConvertTo(sc, innerColType) if err != nil { + // If the converted outerValue overflows, we don't need to lookup it. + if terror.ErrorEqual(err, types.ErrOverflow) { + return nil, nil + } return nil, err } cmp, err := outerValue.CompareDatum(sc, &innerValue) @@ -525,20 +540,23 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) return dLookupKey, nil } -func (iw *innerWorker) sortAndDedupDatumLookUpKeys(dLookUpKeys [][]types.Datum) [][]types.Datum { - if len(dLookUpKeys) < 2 { - return dLookUpKeys +func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLookUpContent) []*indexJoinLookUpContent { + if len(lookUpContents) < 2 { + return lookUpContents } sc := iw.ctx.GetSessionVars().StmtCtx - sort.Slice(dLookUpKeys, func(i, j int) bool { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[j]) - return cmp < 0 + sort.Slice(lookUpContents, func(i, j int) bool { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[j].keys) + if cmp != 0 || iw.nextColCompareFilters == nil { + return cmp < 0 + } + return iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[j].row) < 0 }) - deDupedLookupKeys := dLookUpKeys[:1] - for i := 1; i < len(dLookUpKeys); i++ { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[i-1]) - if cmp != 0 { - deDupedLookupKeys = append(deDupedLookupKeys, dLookUpKeys[i]) + deDupedLookupKeys := lookUpContents[:1] + for i := 1; i < len(lookUpContents); i++ { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys) + if cmp != 0 || (iw.nextColCompareFilters != nil && iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[i-1].row) != 0) { + deDupedLookupKeys = append(deDupedLookupKeys, lookUpContents[i]) } } return deDupedLookupKeys @@ -558,8 +576,8 @@ func compareRow(sc *stmtctx.StatementContext, left, right []types.Datum) int { return 0 } -func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, dLookUpKeys [][]types.Datum) error { - innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, iw.indexRanges, iw.keyOff2IdxOff) +func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, lookUpContent []*indexJoinLookUpContent) error { + innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters) if err != nil { return err } diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 2eb60cc747856..60ebb5e1e811b 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -70,7 +70,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { "└─UnionScan_12 0.00 root not(isnull(test.t2.a))", " └─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 [test.t1.a], keep order:false, stats:pseudo", + " │ └─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, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( @@ -88,7 +88,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { " └─UnionScan_11 0.00 root not(isnull(test.t2.a))", " └─IndexReader_10 0.00 root index:Selection_9", " └─Selection_9 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ t1.a, t2.a from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( "2 2", @@ -117,7 +117,7 @@ func (s *testSuite1) TestBatchIndexJoinUnionScan(c *C) { " └─UnionScan_26 0.00 root not(isnull(test.t2.a))", " └─IndexReader_25 0.00 root index:Selection_24", " └─Selection_24 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ count(*) from t1 join t2 on t1.a = t2.id").Check(testkit.Rows( "4", @@ -143,3 +143,12 @@ func (s *testSuite1) TestInapplicableIndexJoinHint(c *C) { tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 right join t2 on t1.a=t2.a;`).Check(testkit.Rows()) tk.MustQuery(`show warnings;`).Check(testkit.Rows(`Warning 1815 Optimizer Hint /*+ TIDB_INLJ(t2) */ is inapplicable`)) } + +func (s *testSuite) TestIndexJoinOverflow(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec(`drop table if exists t1, t2`) + tk.MustExec(`create table t1(a int)`) + tk.MustExec(`insert into t1 values (-1)`) + tk.MustExec(`create table t2(a int unsigned, index idx(a));`) + tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a;`).Check(testkit.Rows()) +} diff --git a/executor/point_get.go b/executor/point_get.go index 1d170b8dfe087..17f76e6079ae9 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -16,6 +16,7 @@ package executor import ( "context" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/tiancaiamao/debugger" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -88,6 +90,14 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err if err1 != nil { return err1 } + + failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { + if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { + label := debugger.Bind("point-get-g1") + debugger.Breakpoint(label) + } + }) + handleVal, err1 := e.get(idxKey) if err1 != nil && !kv.ErrNotExist.Equal(err1) { return err1 @@ -99,7 +109,16 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err if err1 != nil { return err1 } + + failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { + if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { + label := debugger.Bind("point-get-g1") + debugger.Continue("point-get-g2") + debugger.Breakpoint(label) + } + }) } + key := tablecodec.EncodeRowKeyWithHandle(e.tblInfo.ID, e.handle) val, err := e.get(key) if err != nil && !kv.ErrNotExist.Equal(err) { diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index cbc4d95475b03..46f4f64a5a41d 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -45,7 +45,7 @@ func (s *seqTestSuite) TestPrepared(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -262,7 +262,7 @@ func (s *seqTestSuite) TestPreparedLimitOffset(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -305,7 +305,7 @@ func (s *seqTestSuite) TestPreparedNullParam(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -349,7 +349,7 @@ func (s *seqTestSuite) TestPrepareWithAggregation(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -384,7 +384,7 @@ func (s *seqTestSuite) TestPreparedIssue7579(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -439,7 +439,7 @@ func (s *seqTestSuite) TestPreparedInsert(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -522,7 +522,7 @@ func (s *seqTestSuite) TestPreparedUpdate(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -582,7 +582,7 @@ func (s *seqTestSuite) TestPreparedDelete(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -636,7 +636,7 @@ func (s *seqTestSuite) TestPrepareDealloc(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -681,7 +681,7 @@ func (s *seqTestSuite) TestPreparedIssue8153(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -739,7 +739,7 @@ func (s *seqTestSuite) TestPreparedIssue8644(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7bc877411f3de..e6b3cf3727647 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -777,14 +777,14 @@ func (s *seqTestSuite) TestCartesianProduct(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(c1 int)") - plannercore.AllowCartesianProduct = false + plannercore.AllowCartesianProduct.Store(false) err := tk.ExecToErr("select * from t t1, t t2") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) err = tk.ExecToErr("select * from t t1 left join t t2 on 1") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) err = tk.ExecToErr("select * from t t1 right join t t2 on 1") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) - plannercore.AllowCartesianProduct = true + plannercore.AllowCartesianProduct.Store(true) } type checkPrioClient struct { diff --git a/executor/show.go b/executor/show.go index 99b1aa7a51050..5d9b129755502 100644 --- a/executor/show.go +++ b/executor/show.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/etcd" "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb-tools/tidb-binlog/node" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -182,10 +183,13 @@ func (e *ShowExec) fetchAll() error { } func (e *ShowExec) fetchShowBind() error { + var bindRecords []*bindinfo.BindMeta if !e.GlobalScope { - return errors.New("show non-global bind sql is not supported") + handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindRecords = handle.GetAllBindRecord() + } else { + bindRecords = domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() } - bindRecords := domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() for _, bindData := range bindRecords { e.appendRow([]interface{}{ bindData.OriginalSQL, diff --git a/expression/util.go b/expression/util.go index d11f3b45a54b8..881a8baa19a9b 100644 --- a/expression/util.go +++ b/expression/util.go @@ -44,6 +44,19 @@ func Filter(result []Expression, input []Expression, filter func(Expression) boo return result } +// FilterOutInPlace do the filtering out in place. +// The remained are the ones who doesn't match the filter, storing in the original slice. +// The filteredOut are the ones match the filter, storing in a new slice. +func FilterOutInPlace(input []Expression, filter func(Expression) bool) (remained, filteredOut []Expression) { + for i := len(input) - 1; i >= 0; i-- { + if filter(input[i]) { + filteredOut = append(filteredOut, input[i]) + input = append(input[:i], input[i+1:]...) + } + } + return input, filteredOut +} + // ExtractColumns extracts all columns from an expression. func ExtractColumns(expr Expression) (cols []*Column) { // Pre-allocate a slice to reduce allocation, 8 doesn't have special meaning. diff --git a/expression/util_test.go b/expression/util_test.go index 4aaef213bb483..1d9f9b936860f 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -83,6 +83,20 @@ func (s *testUtilSuite) TestFilter(c *check.C) { c.Assert(result, check.HasLen, 1) } +func (s *testUtilSuite) TestFilterOutInPlace(c *check.C) { + conditions := []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.EQ, newColumn(1), newColumn(2)), + newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + } + remained, filtered := FilterOutInPlace(conditions, isLogicOrFunction) + c.Assert(len(remained), check.Equals, 2) + c.Assert(remained[0].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(remained[1].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(len(filtered), check.Equals, 1) + c.Assert(filtered[0].(*ScalarFunction).FuncName.L, check.Equals, "or") +} + func isLogicOrFunction(e Expression) bool { if f, ok := e.(*ScalarFunction); ok { return f.FuncName.L == ast.LogicOr diff --git a/go.mod b/go.mod index 4392ba138def8..19a9d81e6ca3c 100644 --- a/go.mod +++ b/go.mod @@ -39,10 +39,10 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2 + github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible - github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 + github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 github.com/prometheus/client_golang v0.9.0 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39 // indirect @@ -55,11 +55,13 @@ require ( github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 + github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 github.com/twinj/uuid v1.0.0 github.com/uber-go/atomic v1.3.2 // indirect github.com/uber/jaeger-client-go v2.15.0+incompatible github.com/uber/jaeger-lib v1.5.0 // indirect github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d // indirect + go.uber.org/atomic v1.3.2 go.uber.org/zap v1.9.1 golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb // indirect diff --git a/go.sum b/go.sum index 26440dba9819c..7cddadde9083e 100644 --- a/go.sum +++ b/go.sum @@ -160,14 +160,14 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= 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-20190424024541-e2cdb851bce2 h1:WZPcqMEpW1PFsusiCEiNlatZdI8dGURXneIUrFNmehI= -github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 h1:m35H8yhrMcATKp8yUVUHAmvuzIT8xjL+ialXyBrBWHU= +github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 h1:wnjdQRhybddDesBVBKyOLUPgDaOFdtqA92pduBgWvVQ= -github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 h1:rRMLMjIMFulCX9sGKZ1hoov/iROMsKyC8Snc02nSukw= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -215,6 +215,8 @@ github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2t github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= +github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 h1:P4sgavMKEdqNOws2VfR2c/Bye9nYFgV8gHyiW1wpQhE= +github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22/go.mod h1:qaShs3uDBYnvaQZJAJ6PjPg8kuAHR9zUJ8ilSLK1y/w= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6 h1:lYIiVDtZnyTWlNwiAxLj0bbpTcx1BWCFhXjfsvmPdNc= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= @@ -290,7 +292,6 @@ google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3 gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/infoschema/builder.go b/infoschema/builder.go index 65eba8279c1b0..5b0446648f46f 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -16,6 +16,7 @@ package infoschema import ( "fmt" "sort" + "strings" "github.com/pingcap/errors" "github.com/pingcap/parser/charset" @@ -172,6 +173,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i fmt.Sprintf("(Table ID %d)", tableID), ) } + ConvertCharsetCollateToLowerCaseIfNeed(tblInfo) ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) if alloc == nil { @@ -197,6 +199,20 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i return nil } +// ConvertCharsetCollateToLowerCaseIfNeed convert the charset / collation of table and its columns to lower case, +// if the table's version is prior to TableInfoVersion3. +func ConvertCharsetCollateToLowerCaseIfNeed(tbInfo *model.TableInfo) { + if tbInfo.Version >= model.TableInfoVersion3 { + return + } + tbInfo.Charset = strings.ToLower(tbInfo.Charset) + tbInfo.Collate = strings.ToLower(tbInfo.Collate) + for _, col := range tbInfo.Columns { + col.Charset = strings.ToLower(col.Charset) + col.Collate = strings.ToLower(col.Collate) + } +} + // ConvertOldVersionUTF8ToUTF8MB4IfNeed convert old version UTF8 to UTF8MB4 if config.TreatOldVersionUTF8AsUTF8MB4 is enable. func ConvertOldVersionUTF8ToUTF8MB4IfNeed(tbInfo *model.TableInfo) { if tbInfo.Version >= model.TableInfoVersion2 || !config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 { diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 17203cd52e760..6afa6dab3ea47 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -49,6 +49,15 @@ var slowQueryCols = []columnInfo{ {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.SlowLogCopProcAvg, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcP90, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcMax, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcAddr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogCopWaitAvg, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitP90, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitMax, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitAddr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogMemMax, mysql.TypeLonglong, 20, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -94,7 +103,7 @@ func ParseSlowLog(tz *time.Location, scanner *bufio.Scanner) ([][]types.Datum, e if startFlag { // Parse slow log field. - if strings.Contains(line, variable.SlowLogRowPrefixStr) { + if strings.HasPrefix(line, variable.SlowLogRowPrefixStr) { line = line[len(variable.SlowLogRowPrefixStr):] fieldValues := strings.Split(line, " ") for i := 0; i < len(fieldValues)-1; i += 2 { @@ -125,23 +134,32 @@ func ParseSlowLog(tz *time.Location, scanner *bufio.Scanner) ([][]types.Datum, e } type slowQueryTuple struct { - time time.Time - txnStartTs uint64 - user string - connID uint64 - queryTime float64 - processTime float64 - waitTime float64 - backOffTime float64 - requestCount uint64 - totalKeys uint64 - processKeys uint64 - db string - indexIDs string - isInternal bool - digest string - statsInfo string - sql string + time time.Time + txnStartTs uint64 + user string + connID uint64 + queryTime float64 + processTime float64 + waitTime float64 + backOffTime float64 + requestCount uint64 + totalKeys uint64 + processKeys uint64 + db string + indexIDs string + isInternal bool + digest string + statsInfo string + avgProcessTime float64 + p90ProcessTime float64 + maxProcessTime float64 + maxProcessAddress string + avgWaitTime float64 + p90WaitTime float64 + maxWaitTime float64 + maxWaitAddress string + memMax int64 + sql string } func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) error { @@ -221,6 +239,52 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.digest = value case variable.SlowLogStatsInfoStr: st.statsInfo = value + case variable.SlowLogCopProcAvg: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.avgProcessTime = num + case variable.SlowLogCopProcP90: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.p90ProcessTime = num + case variable.SlowLogCopProcMax: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.maxProcessTime = num + case variable.SlowLogCopProcAddr: + st.maxProcessAddress = value + case variable.SlowLogCopWaitAvg: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.avgWaitTime = num + case variable.SlowLogCopWaitP90: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.p90WaitTime = num + case variable.SlowLogCopWaitMax: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.maxWaitTime = num + case variable.SlowLogCopWaitAddr: + st.maxWaitAddress = value + case variable.SlowLogMemMax: + num, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return errors.AddStack(err) + } + st.memMax = num case variable.SlowLogQuerySQLStr: st.sql = value } @@ -249,6 +313,15 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { 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.NewFloat64Datum(st.avgProcessTime)) + record = append(record, types.NewFloat64Datum(st.p90ProcessTime)) + record = append(record, types.NewFloat64Datum(st.maxProcessTime)) + record = append(record, types.NewStringDatum(st.maxProcessAddress)) + record = append(record, types.NewFloat64Datum(st.avgWaitTime)) + record = append(record, types.NewFloat64Datum(st.p90WaitTime)) + record = append(record, types.NewFloat64Datum(st.maxWaitTime)) + record = append(record, types.NewStringDatum(st.maxWaitAddress)) + record = append(record, types.NewIntDatum(st.memMax)) record = append(record, types.NewStringDatum(st.sql)) return record } @@ -257,7 +330,11 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { func ParseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) if err != nil { - err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) + // This is for compatibility. + t, err = time.Parse(logutil.OldSlowLogTimeFormat, s) + if err != nil { + err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) + } } return t, err } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index f05eb28d7077b..3c75daecd5c2c 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -25,13 +25,16 @@ import ( func (s *testSuite) TestParseSlowLogFile(c *C) { slowLog := bytes.NewBufferString( - `# Time: 2019-01-24-22:32:29.313255 +0800 + `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 # Query_time: 0.216905 # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:1,t2:2 +# Cop_proc_avg: 0.1 Cop_proc_p90: 0.2 Cop_proc_max: 0.03 Cop_proc_addr: 127.0.0.1:20160 +# Cop_wait_avg: 0.05 Cop_wait_p90: 0.6 Cop_wait_max: 0.8 Cop_wait_addr: 0.0.0.0:20160 +# Mem_max: 70724 select * from t;`) scanner := bufio.NewScanner(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") @@ -48,18 +51,53 @@ 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,t1:1,t2:2,select * from t;" + expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,select * from t;" c.Assert(expectRecordString, Equals, recordString) + + // fix sql contain '# ' bug + slowLog = bytes.NewBufferString( + `# Time: 2019-04-28T15:24:04.309074+08:00 +select a# from t; +# Time: 2019-01-24T22:32:29.313255+08:00 +# Txn_start_ts: 405888132465033227 +# Query_time: 0.216905 +# 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) + _, err = infoschema.ParseSlowLog(loc, scanner) + c.Assert(err, IsNil) + + // test for time format compatibility. + slowLog = bytes.NewBufferString( + `# Time: 2019-04-28T15:24:04.309074+08:00 +select * from t; +# Time: 2019-04-24-19:41:21.716221 +0800 +select * from t; +`) + scanner = bufio.NewScanner(slowLog) + rows, err = infoschema.ParseSlowLog(loc, scanner) + c.Assert(err, IsNil) + c.Assert(len(rows) == 2, IsTrue) + t0Str, err := rows[0][0].ToString() + c.Assert(err, IsNil) + c.Assert(t0Str, Equals, "2019-04-28 15:24:04.309074") + t1Str, err := rows[1][0].ToString() + c.Assert(err, IsNil) + c.Assert(t1Str, Equals, "2019-04-24 19:41:21.716221") } func (s *testSuite) TestSlowLogParseTime(c *C) { - t1Str := "2019-01-24-22:32:29.313255 +0800" - t2Str := "2019-01-24-22:32:29.313255" + t1Str := "2019-01-24T22:32:29.313255+08:00" + t2Str := "2019-01-24T22:32:29.313255" t1, err := infoschema.ParseTime(t1Str) c.Assert(err, IsNil) loc, err := time.LoadLocation("Asia/Shanghai") c.Assert(err, IsNil) - t2, err := time.ParseInLocation("2006-01-02-15:04:05.999999999", t2Str, loc) + t2, err := time.ParseInLocation("2006-01-02T15:04:05.999999999", t2Str, loc) c.Assert(err, IsNil) c.Assert(t1.Unix(), Equals, t2.Unix()) t1Format := t1.In(loc).Format(logutil.SlowLogTimeFormat) diff --git a/infoschema/tables.go b/infoschema/tables.go index 67646bd64a684..697a68a7d1997 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -14,6 +14,7 @@ package infoschema import ( + "encoding/json" "fmt" "sort" "sync" @@ -33,7 +34,9 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + binaryJson "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" ) @@ -73,7 +76,10 @@ const ( tableTiDBIndexes = "TIDB_INDEXES" tableSlowLog = "SLOW_QUERY" tableTiDBHotRegions = "TIDB_HOT_REGIONS" + tableTiKVStoreStatus = "TIKV_STORE_STATUS" tableAnalyzeStatus = "ANALYZE_STATUS" + tableTiKVRegionStatus = "TIKV_REGION_STATUS" + tableTiKVRegionPeers = "TIKV_REGION_PEERS" ) type columnInfo struct { @@ -563,6 +569,28 @@ var tableTiDBHotRegionsCols = []columnInfo{ {"FLOW_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, } +var tableTiKVStoreStatusCols = []columnInfo{ + {"STORE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"ADDRESS", mysql.TypeVarchar, 64, 0, nil, nil}, + {"STORE_STATE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"STORE_STATE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LABEL", mysql.TypeJSON, 51, 0, nil, nil}, + {"VERSION", mysql.TypeVarchar, 64, 0, nil, nil}, + {"CAPACITY", mysql.TypeVarchar, 64, 0, nil, nil}, + {"AVAILABLE", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LEADER_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"START_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"LAST_HEARTBEAT_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"UPTIME", mysql.TypeVarchar, 64, 0, nil, nil}, +} + var tableAnalyzeStatusCols = []columnInfo{ {"TABLE_SCHEMA", mysql.TypeVarchar, 64, 0, nil, nil}, {"TABLE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, @@ -573,6 +601,171 @@ var tableAnalyzeStatusCols = []columnInfo{ {"STATE", mysql.TypeVarchar, 64, 0, nil, nil}, } +var tableTiKVRegionStatusCols = []columnInfo{ + {"REGION_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"START_KEY", mysql.TypeBlob, types.UnspecifiedLength, 0, nil, nil}, + {"END_KEY", mysql.TypeBlob, types.UnspecifiedLength, 0, nil, nil}, + {"EPOCH_CONF_VER", mysql.TypeLonglong, 21, 0, nil, nil}, + {"EPOCH_VERSION", mysql.TypeLonglong, 21, 0, nil, nil}, + {"WRITTEN_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, + {"READ_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, + {"APPROXIMATE_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"APPROXIMATE_KEYS", mysql.TypeLonglong, 21, 0, nil, nil}, +} + +var tableTiKVRegionPeersCols = []columnInfo{ + {"REGION_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"PEER_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"STORE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"IS_LEARNER", mysql.TypeTiny, 1, mysql.NotNullFlag, 0, nil}, + {"IS_LEADER", mysql.TypeTiny, 1, mysql.NotNullFlag, 0, nil}, + {"STATUS", mysql.TypeVarchar, 10, 0, 0, nil}, + {"DOWN_SECONDS", mysql.TypeLonglong, 21, 0, 0, nil}, +} + +func dataForTiKVRegionStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + regionsStat, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + for _, regionStat := range regionsStat.Regions { + row := make([]types.Datum, len(tableTiKVRegionStatusCols)) + row[0].SetInt64(regionStat.ID) + row[1].SetString(regionStat.StartKey) + row[2].SetString(regionStat.EndKey) + row[3].SetInt64(regionStat.Epoch.ConfVer) + row[4].SetInt64(regionStat.Epoch.Version) + row[5].SetInt64(regionStat.WrittenBytes) + row[6].SetInt64(regionStat.ReadBytes) + row[7].SetInt64(regionStat.ApproximateSize) + row[8].SetInt64(regionStat.ApproximateKeys) + records = append(records, row) + } + return records, nil +} + +const ( + normalPeer = "NORMAL" + pendingPeer = "PENDING" + downPeer = "DOWN" +) + +func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + regionsStat, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + for _, regionStat := range regionsStat.Regions { + pendingPeerIDSet := set.NewInt64Set() + for _, peer := range regionStat.PendingPeers { + pendingPeerIDSet.Insert(peer.ID) + } + downPeerMap := make(map[int64]int64) + for _, peerStat := range regionStat.DownPeers { + downPeerMap[peerStat.ID] = peerStat.DownSec + } + for _, peer := range regionStat.Peers { + row := make([]types.Datum, len(tableTiKVRegionPeersCols)) + row[0].SetInt64(regionStat.ID) + row[1].SetInt64(peer.ID) + row[2].SetInt64(peer.StoreID) + if peer.ID == regionStat.Leader.ID { + row[3].SetInt64(1) + } else { + row[3].SetInt64(0) + } + if peer.IsLearner { + row[4].SetInt64(1) + } else { + row[4].SetInt64(0) + } + if pendingPeerIDSet.Exist(peer.ID) { + row[5].SetString(pendingPeer) + } else if downSec, ok := downPeerMap[peer.ID]; ok { + row[5].SetString(downPeer) + row[6].SetInt64(downSec) + } else { + row[5].SetString(normalPeer) + } + records = append(records, row) + } + } + return records, nil +} + +func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV store status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + storesStat, err := tikvHelper.GetStoresStat() + if err != nil { + return nil, err + } + for _, storeStat := range storesStat.Stores { + row := make([]types.Datum, len(tableTiKVStoreStatusCols)) + row[0].SetInt64(storeStat.Store.ID) + row[1].SetString(storeStat.Store.Address) + row[2].SetInt64(storeStat.Store.State) + row[3].SetString(storeStat.Store.StateName) + data, err := json.Marshal(storeStat.Store.Labels) + if err != nil { + return nil, err + } + bj := binaryJson.BinaryJSON{} + if err = bj.UnmarshalJSON(data); err != nil { + return nil, err + } + row[4].SetMysqlJSON(bj) + row[5].SetString(storeStat.Store.Version) + row[6].SetString(storeStat.Status.Capacity) + row[7].SetString(storeStat.Status.Available) + row[8].SetInt64(storeStat.Status.LeaderCount) + row[9].SetInt64(storeStat.Status.LeaderWeight) + row[10].SetInt64(storeStat.Status.LeaderScore) + row[11].SetInt64(storeStat.Status.LeaderSize) + row[12].SetInt64(storeStat.Status.RegionCount) + row[13].SetInt64(storeStat.Status.RegionWeight) + row[14].SetInt64(storeStat.Status.RegionScore) + row[15].SetInt64(storeStat.Status.RegionSize) + startTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.StartTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[16].SetMysqlTime(startTs) + lastHeartbeatTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.LastHeartbeatTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[17].SetMysqlTime(lastHeartbeatTs) + row[18].SetString(storeStat.Status.Uptime) + records = append(records, row) + } + return records, nil +} + func dataForCharacterSets() (records [][]types.Datum) { charsets := charset.GetSupportedCharsets() @@ -1552,7 +1745,10 @@ var tableNameToColumns = map[string][]columnInfo{ tableTiDBIndexes: tableTiDBIndexesCols, tableSlowLog: slowQueryCols, tableTiDBHotRegions: tableTiDBHotRegionsCols, + tableTiKVStoreStatus: tableTiKVStoreStatusCols, tableAnalyzeStatus: tableAnalyzeStatusCols, + tableTiKVRegionStatus: tableTiKVRegionStatusCols, + tableTiKVRegionPeers: tableTiKVRegionPeersCols, } func createInfoSchemaTable(handle *Handle, meta *model.TableInfo) *infoschemaTable { @@ -1648,8 +1844,14 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForSlowLog(ctx) case tableTiDBHotRegions: fullRows, err = dataForTiDBHotRegions(ctx) + case tableTiKVStoreStatus: + fullRows, err = dataForTiKVStoreStatus(ctx) case tableAnalyzeStatus: fullRows = DataForAnalyzeStatus() + case tableTiKVRegionStatus: + fullRows, err = dataForTiKVRegionStatus(ctx) + case tableTiKVRegionPeers: + fullRows, err = dataForTikVRegionPeers(ctx) } if err != nil { return nil, err diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index d01d69b20fb0f..31020e99ed846 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -329,7 +329,7 @@ func (s *testTableSuite) TestSlowQuery(c *C) { f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) c.Assert(err, IsNil) defer os.Remove(slowLogFileName) - _, err = f.Write([]byte(`# Time: 2019-02-12-19:33:56.571953 +0800 + _, err = f.Write([]byte(`# Time: 2019-02-12T19:33:56.571953+08:00 # Txn_start_ts: 406315658548871171 # User: root@127.0.0.1 # Conn_ID: 6 @@ -341,6 +341,9 @@ func (s *testTableSuite) TestSlowQuery(c *C) { # Is_internal: false # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:1,t2:2 +# Cop_proc_avg: 0.1 Cop_proc_p90: 0.2 Cop_proc_max: 0.03 Cop_proc_addr: 127.0.0.1:20160 +# Cop_wait_avg: 0.05 Cop_wait_p90: 0.6 Cop_wait_max: 0.8 Cop_wait_addr: 0.0.0.0:20160 +# Mem_max: 70724 select * from t_slim;`)) c.Assert(f.Close(), IsNil) c.Assert(err, IsNil) @@ -348,10 +351,11 @@ 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|t1:1,t2:2|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|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|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|t1:1,t2:2|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|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|select * from t_slim;")) } func (s *testTableSuite) TestForAnalyzeStatus(c *C) { diff --git a/planner/core/cache.go b/planner/core/cache.go index b8fcc669ceecf..c4bd455ff1d9d 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/kvcache" + atomic2 "go.uber.org/atomic" ) var ( @@ -34,7 +35,7 @@ var ( // PreparedPlanCacheMemoryGuardRatio stores the global config "prepared-plan-cache-memory-guard-ratio". PreparedPlanCacheMemoryGuardRatio float64 // PreparedPlanCacheMaxMemory stores the max memory size defined in the global config "performance-max-memory". - PreparedPlanCacheMaxMemory uint64 + PreparedPlanCacheMaxMemory atomic2.Uint64 ) const ( diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index eef47f0f053ae..f817600b4e1aa 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -216,9 +216,9 @@ func (s *testAnalyzeSuite) TestEstimation(c *C) { defer func() { dom.Close() store.Close() - statistics.RatioOfPseudoEstimate = 0.7 + statistics.RatioOfPseudoEstimate.Store(0.7) }() - statistics.RatioOfPseudoEstimate = 10.0 + statistics.RatioOfPseudoEstimate.Store(10.0) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") @@ -575,13 +575,13 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { testKit.MustExec("insert into t select * from t") h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) - statistics.RatioOfPseudoEstimate = 10.0 + statistics.RatioOfPseudoEstimate.Store(10.0) testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows( "TableReader_7 35.91 root data:Selection_6", "└─Selection_6 35.91 cop le(test.t.a, 5), le(test.t.b, 5)", " └─TableScan_5 80.00 cop table:t, range:[-inf,+inf], keep order:false", )) - statistics.RatioOfPseudoEstimate = 0.7 + statistics.RatioOfPseudoEstimate.Store(0.7) testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows( "IndexLookUp_11 8.84 root ", "├─IndexScan_8 26.59 cop table:t, index:a, range:[-inf,5], keep order:false, stats:pseudo", @@ -910,7 +910,7 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", "└─IndexReader_8 0.00 root index:Selection_7", " └─Selection_7 0.00 cop not(isnull(test.t2.a)), not(isnull(test.t2.c))", - " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [test.t1.a test.t1.c], keep order:false, stats:pseudo", + " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [eq(test.t2.a, test.t1.a) gt(test.t2.b, minus(test.t1.b, 1)) lt(test.t2.b, plus(test.t1.b, 1))], keep order:false, stats:pseudo", )) tk.MustExec("create table t(a int, b int, index idx_ab(a, b))") @@ -1036,7 +1036,7 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { " │ └─TopN_30 1.00 cop t1.a:asc, offset:0, count:1", " │ └─IndexScan_29 6.00 cop table:t1, index:b, range:[-inf,6], keep order:false", " └─IndexReader_57 1.04 root index:IndexScan_56", - " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [t1.a], keep order:false", + " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [eq(t2.b, t1.a)], keep order:false", )) // Desc TableScan. tk.MustExec("truncate table t") diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 933c48e4697cc..ea467b3ef637d 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -427,20 +427,21 @@ type analyzeInfo struct { PartitionName string // PhysicalTableID is the id for a partition or a table. PhysicalTableID int64 - PKInfo *model.ColumnInfo - ColsInfo []*model.ColumnInfo + Incremental bool } // AnalyzeColumnsTask is used for analyze columns. type AnalyzeColumnsTask struct { PKInfo *model.ColumnInfo ColsInfo []*model.ColumnInfo + TblInfo *model.TableInfo analyzeInfo } // AnalyzeIndexTask is used for analyze index. type AnalyzeIndexTask struct { IndexInfo *model.IndexInfo + TblInfo *model.TableInfo analyzeInfo } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 6c5d08e5140a5..e8076fbf96477 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -14,20 +14,25 @@ package core import ( + "bytes" + "context" "fmt" "math" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "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" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" + "go.uber.org/zap" ) func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { @@ -317,10 +322,21 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int) // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. -func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, - innerPlan PhysicalPlan, ranges []*ranger.Range, keyOff2IdxOff []int) []PhysicalPlan { +func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerIdx int, innerPlan PhysicalPlan, + ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCmpFuncManager) []PhysicalPlan { joinType := p.JoinType outerSchema := p.children[outerIdx].Schema() + var ( + innerJoinKeys []*expression.Column + outerJoinKeys []*expression.Column + ) + if outerIdx == 0 { + outerJoinKeys = p.LeftJoinKeys + innerJoinKeys = p.RightJoinKeys + } else { + innerJoinKeys = p.LeftJoinKeys + outerJoinKeys = p.RightJoinKeys + } all, _ := prop.AllSameOrder() // If the order by columns are not all from outer child, index join cannot promise the order. if !prop.AllColsFromSchema(outerSchema) || !all { @@ -358,6 +374,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJ innerPlan: innerPlan, KeyOff2IdxOff: newKeyOff, Ranges: ranges, + CompareFilters: compareFilters, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) join.SetSchema(p.schema) return []PhysicalPlan{join} @@ -414,40 +431,77 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou innerPlan := p.constructInnerTableScan(ds, pkCol, outerJoinKeys, us) // Since the primary key means one value corresponding to exact one row, this will always be a no worse one // comparing to other index. - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff) + return p.constructIndexJoin(prop, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) } } - var ( - bestIndexInfo *model.IndexInfo - rangesOfBest []*ranger.Range - maxUsedCols int - remainedOfBest []expression.Expression - keyOff2IdxOff []int - ) + helper := &indexJoinBuildHelper{join: p} for _, path := range ds.possibleAccessPaths { if path.isTablePath { continue } indexInfo := path.index - ranges, remained, tmpKeyOff2IdxOff := p.buildRangeForIndexJoin(indexInfo, ds, innerJoinKeys) - // We choose the index by the number of used columns of the range, the much the better. - // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. - // But obviously when the range is nil, we don't need index join. - if len(ranges) > 0 && len(ranges[0].LowVal) > maxUsedCols { - bestIndexInfo = indexInfo - maxUsedCols = len(ranges[0].LowVal) - rangesOfBest = ranges - remainedOfBest = remained - keyOff2IdxOff = tmpKeyOff2IdxOff - } - } - if bestIndexInfo != nil { - innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us) - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff) + err := helper.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) + if err != nil { + logutil.Logger(context.Background()).Warn("build index join failed", zap.Error(err)) + } + } + if helper.chosenIndexInfo != nil { + keyOff2IdxOff := make([]int, len(innerJoinKeys)) + for i := range keyOff2IdxOff { + keyOff2IdxOff[i] = -1 + } + for idxOff, keyOff := range helper.idxOff2KeyOff { + if keyOff != -1 { + keyOff2IdxOff[keyOff] = idxOff + } + } + idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo) + rangeInfo := helper.buildRangeDecidedByInformation(idxCols, outerJoinKeys) + innerPlan := p.constructInnerIndexScan(ds, helper.chosenIndexInfo, helper.chosenRemained, outerJoinKeys, us, rangeInfo) + return p.constructIndexJoin(prop, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager) } return nil } +type indexJoinBuildHelper struct { + join *LogicalJoin + + chosenIndexInfo *model.IndexInfo + maxUsedCols int + chosenAccess []expression.Expression + chosenRemained []expression.Expression + idxOff2KeyOff []int + lastColManager *ColWithCmpFuncManager + chosenRanges []*ranger.Range + + curPossibleUsedKeys []*expression.Column + curNotUsedIndexCols []*expression.Column + curNotUsedColLens []int + curIdxOff2KeyOff []int +} + +func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column, outerJoinKeys []*expression.Column) string { + buffer := bytes.NewBufferString("[") + isFirst := true + for idxOff, keyOff := range ijHelper.idxOff2KeyOff { + if keyOff == -1 { + continue + } + if !isFirst { + buffer.WriteString(" ") + } else { + isFirst = false + } + buffer.WriteString(fmt.Sprintf("eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) + } + for _, access := range ijHelper.chosenAccess { + // Since now there must be eq/in condition so here we can just append space directly. + buffer.WriteString(fmt.Sprintf(" %v", access)) + } + buffer.WriteString("]") + return buffer.String() +} + // constructInnerTableScan is specially used to construct the inner plan for PhysicalIndexJoin. func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Column, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { ranges := ranger.FullIntRange(mysql.HasUnsignedFlag(pk.RetType.Flag)) @@ -491,7 +545,8 @@ func (p *LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physi } // constructInnerIndexScan is specially used to construct the inner plan for PhysicalIndexJoin. -func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, remainedConds []expression.Expression, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { +func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, filterConds []expression.Expression, + outerJoinKeys []*expression.Column, us *LogicalUnionScan, rangeInfo string) PhysicalPlan { is := PhysicalIndexScan{ Table: ds.tableInfo, TableAsName: ds.TableAsName, @@ -501,9 +556,8 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn dataSourceSchema: ds.schema, KeepOrder: false, Ranges: ranger.FullRange(), - rangeDecidedBy: outerJoinKeys, + rangeInfo: rangeInfo, }.Init(ds.ctx) - is.filterCondition = remainedConds var rowCount float64 idxHist, ok := ds.statisticTable.Indices[idx.ID] @@ -529,7 +583,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn } is.initSchema(ds.id, idx, cop.tablePlan != nil) - indexConds, tblConds := splitIndexFilterConditions(remainedConds, idx.Columns, ds.tableInfo) + indexConds, tblConds := splitIndexFilterConditions(filterConds, idx.Columns, ds.tableInfo) path := &accessPath{indexFilters: indexConds, tableFilters: tblConds, countAfterIndex: math.MaxFloat64} is.addPushedDownSelection(cop, ds, math.MaxFloat64, path) t := finishCopTask(ds.ctx, cop) @@ -537,92 +591,360 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn return p.constructInnerUnionScan(us, reader) } -// buildRangeForIndexJoin checks whether this index can be used for building index join and return the range if this index is ok. -// If this index is invalid, just return nil range. -func (p *LogicalJoin) buildRangeForIndexJoin(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ( - []*ranger.Range, []expression.Expression, []int) { - idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.Schema().Columns, indexInfo) - if len(idxCols) == 0 { - return nil, nil, nil - } +var symmetricOp = map[string]string{ + ast.LT: ast.GT, + ast.GE: ast.LE, + ast.GT: ast.LT, + ast.LE: ast.GE, +} - // Extract the filter to calculate access and the filters that must be remained ones. - access, eqConds, remained, keyOff2IdxOff := p.buildFakeEqCondsForIndexJoin(innerJoinKeys, idxCols, colLengths, innerPlan.pushedDownConds) +// ColWithCmpFuncManager is used in index join to handle the column with compare functions(>=, >, <, <=). +// It stores the compare functions and build ranges in execution phase. +type ColWithCmpFuncManager struct { + targetCol *expression.Column + colLength int + OpType []string + opArg []expression.Expression + tmpConstant []*expression.Constant + affectedColSchema *expression.Schema + compareFuncs []chunk.CompareFunc +} - if len(keyOff2IdxOff) == 0 { - return nil, nil, nil +func (cwc *ColWithCmpFuncManager) appendNewExpr(opName string, arg expression.Expression, affectedCols []*expression.Column) { + cwc.OpType = append(cwc.OpType, opName) + cwc.opArg = append(cwc.opArg, arg) + cwc.tmpConstant = append(cwc.tmpConstant, &expression.Constant{RetType: cwc.targetCol.RetType}) + for _, col := range affectedCols { + if cwc.affectedColSchema.Contains(col) { + continue + } + cwc.compareFuncs = append(cwc.compareFuncs, chunk.GetCompareFunc(col.RetType)) + cwc.affectedColSchema.Append(col) } +} - // In `buildFakeEqCondsForIndexJoin`, we construct the equal conditions for join keys and remove filters that contain the join keys' column. - // When t1.a = t2.a and t1.a > 1, we can also guarantee that t1.a > 1 won't be chosen as the access condition. - // So the equal conditions we built can be successfully used to build a range if they can be used. They won't be affected by the existing filters. - res, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, access, idxCols, colLengths) - if err != nil { - terror.Log(err) - return nil, nil, nil +// CompareRow compares the rows for deduplicate. +func (cwc *ColWithCmpFuncManager) CompareRow(lhs, rhs chunk.Row) int { + for i, col := range cwc.affectedColSchema.Columns { + ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) + if ret != 0 { + return ret + } } + return 0 +} - // Guarantee res.AccessConds is not empty. - if len(res.AccessConds) == 0 { - return nil, nil, nil +// BuildRangesByRow will build range of the given row. It will eval each function's arg then call BuildRange. +func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { + exprs := make([]expression.Expression, len(cwc.OpType)) + for i, opType := range cwc.OpType { + constantArg, err := cwc.opArg[i].Eval(row) + if err != nil { + return nil, err + } + cwc.tmpConstant[i].Value = constantArg + newExpr, err := expression.NewFunction(ctx, opType, types.NewFieldType(mysql.TypeTiny), cwc.targetCol, cwc.tmpConstant[i]) + if err != nil { + return nil, err + } + exprs = append(exprs, newExpr) + } + ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.targetCol.RetType, cwc.colLength) + if err != nil { + return nil, err } + return ranges, nil +} - // Find invalid fake condition and modify the joinKey's idxOff to -1. - var invalidFakeConds []expression.Expression - for i, eqCond := range eqConds { - if !expression.Contains(res.AccessConds, eqCond) { - keyOff2IdxOff[i] = -1 - invalidFakeConds = append(invalidFakeConds, eqCond) +func (cwc *ColWithCmpFuncManager) resolveIndices(schema *expression.Schema) (err error) { + for i := range cwc.opArg { + cwc.opArg[i], err = cwc.opArg[i].ResolveIndices(schema) + if err != nil { + return err } } + return nil +} - // Filter out invalidFakeConds from res.RemainedConds. - for _, cond := range res.RemainedConds { - if !expression.Contains(invalidFakeConds, cond) { - remained = append(remained, cond) +// String implements Stringer interface. +func (cwc *ColWithCmpFuncManager) String() string { + buffer := bytes.NewBufferString("") + for i := range cwc.OpType { + buffer.WriteString(fmt.Sprintf("%v(%v, %v)", cwc.OpType[i], cwc.targetCol, cwc.opArg[i])) + if i < len(cwc.OpType)-1 { + buffer.WriteString(" ") } } - - return res.Ranges, remained, keyOff2IdxOff + return buffer.String() } -func (p *LogicalJoin) buildFakeEqCondsForIndexJoin(keys, idxCols []*expression.Column, colLengths []int, - innerFilters []expression.Expression) (accesses, eqConds, remained []expression.Expression, keyOff2IdxOff []int) { - // Check whether all join keys match one column from index. - keyOff2IdxOff = joinKeysMatchIndex(keys, idxCols, colLengths) - if keyOff2IdxOff == nil { - return nil, nil, nil, nil +func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int) bool { + tmpSchema := expression.NewSchema(innerKeys...) + ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols)) + ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols)) + ijHelper.curNotUsedColLens = make([]int, 0, len(idxCols)) + keyMatched := false + for i, idxCol := range idxCols { + ijHelper.curIdxOff2KeyOff[i] = tmpSchema.ColumnIndex(idxCol) + if ijHelper.curIdxOff2KeyOff[i] >= 0 { + keyMatched = true + continue + } + ijHelper.curNotUsedIndexCols = append(ijHelper.curNotUsedIndexCols, idxCol) + ijHelper.curNotUsedColLens = append(ijHelper.curNotUsedColLens, colLens[i]) } + return keyMatched +} - usableKeys := make([]*expression.Column, 0, len(keys)) +// findUsefulEqAndInFilters analyzes the pushedDownConds held by inner child and split them to three parts. +// usefulEqOrInFilters is the continuous eq/in conditions on current unused index columns. +// uselessFilters is the conditions which cannot be used for building ranges. +// remainingRangeCandidates is the other conditions for future use. +func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression) { + uselessFilters = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + var remainedEqOrIn []expression.Expression + // Extract the eq/in functions of possible join key. + // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. + usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _ = ranger.ExtractEqAndInCondition( + innerPlan.ctx, innerPlan.pushedDownConds, + ijHelper.curNotUsedIndexCols, + ijHelper.curNotUsedColLens, + ) + uselessFilters = append(uselessFilters, remainedEqOrIn...) + return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates +} - conds := make([]expression.Expression, 0, len(keys)+len(innerFilters)) - eqConds = make([]expression.Expression, 0, len(keys)) - // Construct a fake equal expression for every join key for calculating the range. - for i, key := range keys { - if keyOff2IdxOff[i] < 0 { +// buildLastColManager analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. +// The returned value is just for outputting explain information +func (ijHelper *indexJoinBuildHelper) buildLastColManager(nextCol *expression.Column, + innerPlan *DataSource, cwc *ColWithCmpFuncManager) []expression.Expression { + var lastColAccesses []expression.Expression +loopOtherConds: + for _, filter := range ijHelper.join.OtherConditions { + sf, ok := filter.(*expression.ScalarFunction) + if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { + continue + } + var funcName string + var anotherArg expression.Expression + if lCol, ok := sf.GetArgs()[0].(*expression.Column); ok && lCol.Equal(nil, nextCol) { + anotherArg = sf.GetArgs()[1] + funcName = sf.FuncName.L + } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { + anotherArg = sf.GetArgs()[0] + // The column manager always build expression in the form of col op arg1. + // So we need use the symmetric one of the current function. + funcName = symmetricOp[sf.FuncName.L] + } else { continue } - usableKeys = append(usableKeys, key) - // Int datum 1 can convert to all column's type(numeric type, string type, json, time type, enum, set) safely. - fakeConstant := &expression.Constant{Value: types.NewIntDatum(1), RetType: key.GetType()} - eqFunc := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), key, fakeConstant) - conds = append(conds, eqFunc) - eqConds = append(eqConds, eqFunc) + affectedCols := expression.ExtractColumns(anotherArg) + if len(affectedCols) == 0 { + continue + } + for _, col := range affectedCols { + if innerPlan.schema.Contains(col) { + continue loopOtherConds + } + } + lastColAccesses = append(lastColAccesses, sf) + cwc.appendNewExpr(funcName, anotherArg, affectedCols) } + return lastColAccesses +} - // Look into every `innerFilter`, if it contains join keys' column, put this filter into `remained` part directly. - remained = make([]expression.Expression, 0, len(innerFilters)) - for _, filter := range innerFilters { - affectedCols := expression.ExtractColumns(filter) - if expression.ColumnSliceIsIntersect(affectedCols, usableKeys) { - remained = append(remained, filter) +// removeUselessEqAndInFunc removes the useless eq/in conditions. It's designed for the following case: +// t1 join t2 on t1.a=t2.a and t1.c=t2.c where t1.b > t2.b-10 and t1.b < t2.b+10 there's index(a, b, c) on t1. +// In this case the curIdxOff2KeyOff is [0 -1 1] and the notKeyEqAndIn is []. +// It's clearly that the column c cannot be used to access data. So we need to remove it and reset the IdxOff2KeyOff to +// [0 -1 -1]. +// So that we can use t1.a=t2.a and t1.b > t2.b-10 and t1.b < t2.b+10 to build ranges then access data. +func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc( + idxCols []*expression.Column, + notKeyEqAndIn []expression.Expression) ( + usefulEqAndIn, uselessOnes []expression.Expression, +) { + ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) + for idxColPos, notKeyColPos := 0, 0; idxColPos < len(idxCols); idxColPos++ { + if ijHelper.curIdxOff2KeyOff[idxColPos] != -1 { + ijHelper.curPossibleUsedKeys = append(ijHelper.curPossibleUsedKeys, idxCols[idxColPos]) + continue + } + if notKeyColPos < len(notKeyEqAndIn) && ijHelper.curNotUsedIndexCols[notKeyColPos].Equal(nil, idxCols[idxColPos]) { + notKeyColPos++ continue } - conds = append(conds, filter) + for i := idxColPos + 1; i < len(idxCols); i++ { + ijHelper.curIdxOff2KeyOff[i] = -1 + } + remained := make([]expression.Expression, 0, len(notKeyEqAndIn)-notKeyColPos) + remained = append(remained, notKeyEqAndIn[notKeyColPos:]...) + notKeyEqAndIn = notKeyEqAndIn[:notKeyColPos] + return notKeyEqAndIn, remained + } + return notKeyEqAndIn, nil +} + +func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) error { + idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) + if len(idxCols) == 0 { + return nil + } + accesses := make([]expression.Expression, 0, len(idxCols)) + // If no index column appears in join key, we just break. + // TODO: It may meet this case: There's no join key condition, but have compare filters. + // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. + // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join + // to speed up. + if !ijHelper.checkIndex(innerJoinKeys, idxCols, colLengths) { + return nil + } + notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) + var remainedEqAndIn []expression.Expression + notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(idxCols, notKeyEqAndIn) + matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) + if matchedKeyCnt <= 0 { + return nil + } + accesses = append(accesses, notKeyEqAndIn...) + remained = append(remained, remainedEqAndIn...) + lastColPos := matchedKeyCnt + len(notKeyEqAndIn) + // If all the index columns are covered by eq/in conditions, we don't need to consider other conditions anymore. + if lastColPos == len(idxCols) { + remained = append(remained, rangeFilterCandidates...) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, false) + if err != nil { + return err + } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil + } + lastPossibleCol := idxCols[lastColPos] + lastColManager := &ColWithCmpFuncManager{ + targetCol: lastPossibleCol, + colLength: colLengths[lastColPos], + affectedColSchema: expression.NewSchema(), + } + lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, innerPlan, lastColManager) + // If the column manager holds no expression, then we fallback to find whether there're useful normal filters + if len(lastColAccess) == 0 { + colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) + var ranges, nextColRange []*ranger.Range + var err error + if len(colAccesses) > 0 { + nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, colLengths[lastColPos]) + if err != nil { + return err + } + } + ranges, err = ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nextColRange, false) + if err != nil { + return err + } + remained = append(remained, colRemained...) + if colLengths[lastColPos] != types.UnspecifiedLength { + remained = append(remained, colAccesses...) + } + accesses = append(accesses, colAccesses...) + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil } + accesses = append(accesses, lastColAccess...) + remained = append(remained, rangeFilterCandidates...) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, true) + if err != nil { + return err + } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, lastColManager) + return nil +} - return conds, eqConds, remained, keyOff2IdxOff +func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, idxInfo *model.IndexInfo, accesses, + remained []expression.Expression, lastColManager *ColWithCmpFuncManager) { + // We choose the index by the number of used columns of the range, the much the better. + // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. + // But obviously when the range is nil, we don't need index join. + if len(ranges) > 0 && len(ranges[0].LowVal) > ijHelper.maxUsedCols { + ijHelper.chosenIndexInfo = idxInfo + ijHelper.maxUsedCols = len(ranges[0].LowVal) + ijHelper.chosenRanges = ranges + ijHelper.chosenAccess = accesses + ijHelper.chosenRemained = remained + ijHelper.idxOff2KeyOff = ijHelper.curIdxOff2KeyOff + ijHelper.lastColManager = lastColManager + } +} + +func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { + pointLength := matchedKeyCnt + len(eqAndInFuncs) + if nextColRange != nil { + for _, colRan := range nextColRange { + // The range's exclude status is the same with last col's. + ran := &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength+1), + HighVal: make([]types.Datum, pointLength, pointLength+1), + LowExclude: colRan.LowExclude, + HighExclude: colRan.HighExclude, + } + ran.LowVal = append(ran.LowVal, colRan.LowVal[0]) + ran.HighVal = append(ran.HighVal, colRan.HighVal[0]) + ranges = append(ranges, ran) + } + } else if haveExtraCol { + // Reserve a position for the last col. + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength+1, pointLength+1), + HighVal: make([]types.Datum, pointLength+1, pointLength+1), + }) + } else { + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength), + HighVal: make([]types.Datum, pointLength, pointLength), + }) + } + emptyRow := chunk.Row{} + for i, j := 0, 0; j < len(eqAndInFuncs); i++ { + // This position is occupied by join key. + if ijHelper.curIdxOff2KeyOff[i] != -1 { + continue + } + sf := eqAndInFuncs[j].(*expression.ScalarFunction) + // Deal with the first two args. + if _, ok := sf.GetArgs()[0].(*expression.Column); ok { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[1].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } else { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[0].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } + // If the length of in function's constant list is more than one, we will expand ranges. + curRangeLen := len(ranges) + for argIdx := 2; argIdx < len(sf.GetArgs()); argIdx++ { + newRanges := make([]*ranger.Range, 0, curRangeLen) + for oldRangeIdx := 0; oldRangeIdx < curRangeLen; oldRangeIdx++ { + newRange := ranges[oldRangeIdx].Clone() + newRange.LowVal[i], err = sf.GetArgs()[argIdx].Eval(emptyRow) + if err != nil { + return nil, err + } + newRange.HighVal[i] = newRange.LowVal[i] + newRanges = append(newRanges, newRange) + } + ranges = append(ranges, newRanges...) + } + j++ + } + return ranges, nil } // tryToGetIndexJoin will get index join by hints. If we can generate a valid index join by hint, the second return value @@ -651,10 +973,6 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ } }() - if len(p.EqualConditions) == 0 { - return nil, false - } - switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: join := p.getIndexJoinByOuterIdx(prop, 0) diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go new file mode 100644 index 0000000000000..a7cf2a21a9a1f --- /dev/null +++ b/planner/core/exhaust_physical_plans_test.go @@ -0,0 +1,238 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/types" +) + +func (s *testUnitTestSuit) rewriteSimpleExpr(str string, schema *expression.Schema) ([]expression.Expression, error) { + if str == "" { + return nil, nil + } + filters, err := expression.ParseSimpleExprsWithSchema(s.ctx, str, schema) + if err != nil { + return nil, err + } + if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { + filters = expression.FlattenCNFConditions(sf) + } + return filters, nil +} + +func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { + s.ctx.GetSessionVars().PlanID = -1 + joinNode := LogicalJoin{}.Init(s.ctx) + dataSourceNode := DataSource{}.Init(s.ctx) + dsSchema := expression.NewSchema() + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("a"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("b"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("c"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("d"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dataSourceNode.schema = dsSchema + outerChildSchema := expression.NewSchema() + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("e"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("f"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("g"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("h"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + joinNode.SetSchema(expression.MergeSchema(dsSchema, outerChildSchema)) + var idxCols []*model.IndexColumn + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("a"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("b"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("c"), Length: 2}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("d"), Length: types.UnspecifiedLength}) + idxInfo := &model.IndexInfo{Columns: idxCols} + + tests := []struct { + innerKeys []*expression.Column + pushedDownConds string + otherConds string + ranges string + idxOff2KeyOff string + accesses string + remained string + compareFilters string + }{ + // Join key not continuous and no pushed filter to match. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "", + otherConds: "", + ranges: "[[NULL,NULL]]", + idxOff2KeyOff: "[0 -1 -1 -1]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Join key and pushed eq filter not continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[2]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[]", + idxOff2KeyOff: "[]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[[1 NULL,1 NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1)]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous and there're correlated filters. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < concat(g, \"ab\")", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))", + }, + // cast function won't be involved. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < g + 10", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g)]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g)", + }, + // Can deal with prefix index correctly. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1 and c > 'a' and c < 'aaaaaa'", + otherConds: "", + ranges: "[(1 NULL \"a\",1 NULL \"[97 97]\"]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + remained: "[gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "", + ranges: "[[1 NULL \"a\",1 NULL \"a\"] [2 NULL \"a\",2 NULL \"a\"] [3 NULL \"a\",3 NULL \"a\"] [1 NULL \"b\",1 NULL \"b\"] [2 NULL \"b\",2 NULL \"b\"] [3 NULL \"b\",3 NULL \"b\"] [1 NULL \"c\",1 NULL \"c\"] [2 NULL \"c\",2 NULL \"c\"] [3 NULL \"c\",3 NULL \"c\"]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c)]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions with correlated filters.. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "d > h and d < h + 100", + ranges: "[[1 NULL \"a\" NULL,1 NULL \"a\" NULL] [2 NULL \"a\" NULL,2 NULL \"a\" NULL] [3 NULL \"a\" NULL,3 NULL \"a\" NULL] [1 NULL \"b\" NULL,1 NULL \"b\" NULL] [2 NULL \"b\" NULL,2 NULL \"b\" NULL] [3 NULL \"b\" NULL,3 NULL \"b\" NULL] [1 NULL \"c\" NULL,1 NULL \"c\" NULL] [2 NULL \"c\" NULL,2 NULL \"c\" NULL] [3 NULL \"c\" NULL,3 NULL \"c\" NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c) gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))", + }, + // Join keys are not continuous and the pushed key connect the key but not eq/in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "b > 1", + otherConds: "", + ranges: "[(NULL 1,NULL +inf]]", + idxOff2KeyOff: "[0 -1 -1 -1]", + accesses: "[gt(test.t.b, 1)]", + remained: "[]", + compareFilters: "", + }, + } + for i, tt := range tests { + pushed, err := s.rewriteSimpleExpr(tt.pushedDownConds, dsSchema) + c.Assert(err, IsNil) + dataSourceNode.pushedDownConds = pushed + others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema) + c.Assert(err, IsNil) + joinNode.OtherConditions = others + helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil} + err = helper.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges, Commentf("test case: #%v", i)) + c.Assert(fmt.Sprintf("%v", helper.idxOff2KeyOff), Equals, tt.idxOff2KeyOff) + c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses) + c.Assert(fmt.Sprintf("%v", helper.chosenRemained), Equals, tt.remained) + c.Assert(fmt.Sprintf("%v", helper.lastColManager), Equals, tt.compareFilters) + } +} diff --git a/planner/core/explain.go b/planner/core/explain.go index 18fe8f8f998df..ad801dbaecd8f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -58,8 +58,8 @@ func (p *PhysicalIndexScan) ExplainInfo() string { break } } - if len(p.rangeDecidedBy) > 0 { - fmt.Fprintf(buffer, ", range: decided by %v", p.rangeDecidedBy) + if len(p.rangeInfo) > 0 { + fmt.Fprintf(buffer, ", range: decided by %v", p.rangeInfo) } else if haveCorCol { fmt.Fprintf(buffer, ", range: decided by %v", p.AccessCondition) } else if len(p.Ranges) > 0 { diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index d5e163d1e931b..0f36b2bc32eb2 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -710,7 +710,7 @@ func (er *expressionRewriter) handleInSubquery(v *ast.PatternInExpr) (ast.Node, // We need to try to eliminate the agg and the projection produced by this operation. er.b.optFlag |= flagEliminateAgg er.b.optFlag |= flagEliminateProjection - er.b.optFlag |= flagJoinReOrderGreedy + er.b.optFlag |= flagJoinReOrder // Build distinct for the inner query. agg := er.b.buildDistinct(np, np.Schema().Len()) for _, col := range agg.schema.Columns { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 80df8bcf548c8..b71546145175b 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -477,7 +477,6 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid IdxColLens: path.idxColLens, AccessCondition: path.accessConds, Ranges: path.ranges, - filterCondition: path.indexFilters, dataSourceSchema: ds.schema, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, @@ -734,7 +733,7 @@ func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float6 return 0, false, corr } sc := ds.ctx.GetSessionVars().StmtCtx - ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType) + ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType, types.UnspecifiedLength) if len(ranges) == 0 || err != nil { return 0, err == nil, corr } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d00f28d561647..b9dcec36df240 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -390,7 +390,7 @@ func (b *PlanBuilder) buildJoin(joinNode *ast.Join) (LogicalPlan, error) { joinPlan.JoinType = RightOuterJoin resetNotNullFlag(joinPlan.schema, 0, leftPlan.Schema().Len()) default: - b.optFlag = b.optFlag | flagJoinReOrderGreedy + b.optFlag = b.optFlag | flagJoinReOrder joinPlan.JoinType = InnerJoin } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index a1e795bf59ed9..578155d266563 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -916,7 +916,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { p, err := BuildLogicalPlan(s.ctx, stmt, s.is) c.Assert(err, IsNil) - p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrderGreedy, p.(LogicalPlan)) + p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) c.Assert(err, IsNil) c.Assert(ToString(p), Equals, tt.best, Commentf("for %s", tt.sql)) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 1baf596854a66..e0fd6c507c304 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,13 +24,14 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "go.uber.org/atomic" ) // OptimizeAstNode optimizes the query to a physical plan directly. var OptimizeAstNode func(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (Plan, error) // AllowCartesianProduct means whether tidb allows cartesian join without equal conditions. -var AllowCartesianProduct = true +var AllowCartesianProduct = atomic.NewBool(true) const ( flagPrunColumns uint64 = 1 << iota @@ -44,7 +45,7 @@ const ( flagPartitionProcessor flagPushDownAgg flagPushDownTopN - flagJoinReOrderGreedy + flagJoinReOrder ) var optRuleList = []logicalOptRule{ @@ -102,7 +103,7 @@ func DoOptimize(flag uint64, logic LogicalPlan) (PhysicalPlan, error) { if err != nil { return nil, err } - if !AllowCartesianProduct && existsCartesianProduct(logic) { + if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, errors.Trace(ErrCartesianProductUnsupported) } physical, err := physicalOptimize(logic) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index bfa7598adf72a..81529c780bbd4 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -90,7 +90,6 @@ type PhysicalIndexScan struct { // AccessCondition is used to calculate range. AccessCondition []expression.Expression - filterCondition []expression.Expression Table *model.TableInfo Index *model.IndexInfo @@ -115,7 +114,7 @@ type PhysicalIndexScan struct { // It is used for query feedback. Hist *statistics.Histogram - rangeDecidedBy []*expression.Column + rangeInfo string // The index scan may be on a partition. isPartition bool @@ -235,6 +234,12 @@ type PhysicalIndexJoin struct { Ranges []*ranger.Range // KeyOff2IdxOff maps the offsets in join key to the offsets in the index. KeyOff2IdxOff []int + // CompareFilters stores the filters for last column if those filters need to be evaluated during execution. + // e.g. select * from t where t.a = t1.a and t.b > t1.b and t.b < t1.b+10 + // If there's index(t.a, t.b). All the filters can be used to construct index range but t.b > t1.b and t.b < t1.b=10 + // need to be evaluated after we fetch the data of t1. + // This struct stores them and evaluate them to ranges. + CompareFilters *ColWithCmpFuncManager } // PhysicalMergeJoin represents merge join for inner/ outer join. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 64bef437ed633..1bde08d6f95b5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -436,13 +436,13 @@ func getPathByIndexName(paths []*accessPath, idxName model.CIStr, tblInfo *model return path } } - if isPrimaryIndexHint(idxName) && tblInfo.PKIsHandle { + if isPrimaryIndex(idxName) && tblInfo.PKIsHandle { return tablePath } return nil } -func isPrimaryIndexHint(indexName model.CIStr) bool { +func isPrimaryIndex(indexName model.CIStr) bool { return indexName.L == "primary" } @@ -808,20 +808,22 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) } for _, idx := range idxInfo { for i, id := range physicalIDs { - info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{ IndexInfo: idx, analyzeInfo: info, + TblInfo: tbl.TableInfo, }) } } if len(colInfo) > 0 || pkInfo != nil { for i, id := range physicalIDs { - info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{ PKInfo: pkInfo, ColsInfo: colInfo, analyzeInfo: info, + TblInfo: tbl.TableInfo, }) } } @@ -837,13 +839,21 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) return nil, err } for _, idxName := range as.IndexNames { + if isPrimaryIndex(idxName) && tblInfo.PKIsHandle { + pkCol := tblInfo.GetPkColInfo() + for i, id := range physicalIDs { + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info}) + } + continue + } idx := tblInfo.FindIndexByName(idxName.L) if idx == nil || idx.State != model.StatePublic { return nil, ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tblInfo.Name.O) } for i, id := range physicalIDs { - info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info}) + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } return p, nil @@ -859,11 +869,18 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) (Plan, erro for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for i, id := range physicalIDs { - info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info}) + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } } + if tblInfo.PKIsHandle { + pkCol := tblInfo.GetPkColInfo() + for i, id := range physicalIDs { + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info}) + } + } return p, nil } diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index b2f8858b36b5b..6a548bca53cdc 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -51,7 +51,7 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c int, key idx_bc(b,c))") diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index f5af8954eec94..b681b19f4a64a 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -58,7 +58,7 @@ func (s *testPrepareSuite) TestPrepareCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c int, index idx1(b, a), index idx2(b))") @@ -108,7 +108,7 @@ func (s *testPrepareSuite) TestPrepareCacheIndexScan(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, primary key (a, b))") @@ -142,7 +142,7 @@ func (s *testPlanSuite) TestPrepareCacheDeferredFunction(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -202,7 +202,7 @@ func (s *testPrepareSuite) TestPrepareCacheNow(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec(`prepare stmt1 from "select now(), current_timestamp(), utc_timestamp(), unix_timestamp(), sleep(0.1), now(), current_timestamp(), utc_timestamp(), unix_timestamp()"`) // When executing one statement at the first time, we don't usTestPrepareCacheDeferredFunctione cache, so we need to execute it at least twice to test the cache. @@ -286,7 +286,7 @@ func (s *testPrepareSuite) TestPrepareTableAsNameOnGroupByWithCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1 ( diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 3cabf66819691..ec415eaf9bef6 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -193,6 +193,19 @@ func (p *PhysicalIndexJoin) ResolveIndices() (err error) { return err } } + if p.CompareFilters != nil { + err = p.CompareFilters.resolveIndices(p.children[p.OuterIndex].Schema()) + if err != nil { + return err + } + for i := range p.CompareFilters.affectedColSchema.Columns { + resolvedCol, err1 := p.CompareFilters.affectedColSchema.Columns[i].ResolveIndices(p.children[p.OuterIndex].Schema()) + if err1 != nil { + return err1 + } + p.CompareFilters.affectedColSchema.Columns[i] = resolvedCol.(*expression.Column) + } + } return } diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 8e5a1c4f4d7d2..fac63d725cbb5 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -71,11 +71,19 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP ctx: ctx, otherConds: otherConds, } - groupSolver := &joinReorderGreedySingleGroupSolver{ - baseSingleGroupJoinOrderSolver: baseGroupSolver, - eqEdges: eqEdges, + if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold { + groupSolver := &joinReorderGreedySolver{ + baseSingleGroupJoinOrderSolver: baseGroupSolver, + eqEdges: eqEdges, + } + p, err = groupSolver.solve(curJoinGroup) + } else { + dpSolver := &joinReorderDPSolver{ + baseSingleGroupJoinOrderSolver: baseGroupSolver, + } + dpSolver.newJoin = dpSolver.newJoinWithEdges + p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges)) } - p, err = groupSolver.solve(curJoinGroup) if err != nil { return nil, err } @@ -143,22 +151,15 @@ func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild Logical return join } -func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(eqEdges []*expression.ScalarFunction, remainedOtherConds []expression.Expression, - lChild, rChild LogicalPlan) (*LogicalJoin, []expression.Expression) { +func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild LogicalPlan, eqEdges []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { newJoin := s.newCartesianJoin(lChild, rChild) newJoin.EqualConditions = eqEdges + newJoin.OtherConditions = otherConds for _, eqCond := range newJoin.EqualConditions { newJoin.LeftJoinKeys = append(newJoin.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) newJoin.RightJoinKeys = append(newJoin.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) } - for i := len(remainedOtherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(remainedOtherConds[i]) - if newJoin.schema.ColumnsIndices(cols) != nil { - newJoin.OtherConditions = append(newJoin.OtherConditions, remainedOtherConds[i]) - remainedOtherConds = append(remainedOtherConds[:i], remainedOtherConds[i+1:]...) - } - } - return newJoin, remainedOtherConds + return newJoin } // calcJoinCumCost calculates the cumulative cost of the join node. diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index f4820fe7380fd..18b549d7813c8 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -18,24 +18,39 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx" ) type joinReorderDPSolver struct { - ctx sessionctx.Context - newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan + *baseSingleGroupJoinOrderSolver + newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan } -type joinGroupEdge struct { +type joinGroupEqEdge struct { nodeIDs []int edge *expression.ScalarFunction } -func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression.Expression) (LogicalPlan, error) { - adjacents := make([][]int, len(joinGroup)) - totalEdges := make([]joinGroupEdge, 0, len(conds)) - addEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { - totalEdges = append(totalEdges, joinGroupEdge{ +type joinGroupNonEqEdge struct { + nodeIDs []int + nodeIDMask uint + expr expression.Expression +} + +func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expression.Expression) (LogicalPlan, error) { + for _, node := range joinGroup { + _, err := node.recursiveDeriveStats() + if err != nil { + return nil, err + } + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: s.baseNodeCumCost(node), + }) + } + adjacents := make([][]int, len(s.curJoinGroup)) + totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) + addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { + totalEqEdges = append(totalEqEdges, joinGroupEqEdge{ nodeIDs: []int{node1, node2}, edge: edgeContent, }) @@ -43,7 +58,7 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. adjacents[node2] = append(adjacents[node2], node1) } // Build Graph for join group - for _, cond := range conds { + for _, cond := range eqConds { sf := cond.(*expression.ScalarFunction) lCol := sf.GetArgs()[0].(*expression.Column) rCol := sf.GetArgs()[1].(*expression.Column) @@ -55,7 +70,26 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. if err != nil { return nil, err } - addEdge(lIdx, rIdx, sf) + addEqEdge(lIdx, rIdx, sf) + } + totalNonEqEdges := make([]joinGroupNonEqEdge, 0, len(s.otherConds)) + for _, cond := range s.otherConds { + cols := expression.ExtractColumns(cond) + mask := uint(0) + ids := make([]int, 0, len(cols)) + for _, col := range cols { + idx, err := findNodeIndexInGroup(joinGroup, col) + if err != nil { + return nil, err + } + ids = append(ids, idx) + mask |= 1 << uint(idx) + } + totalNonEqEdges = append(totalNonEqEdges, joinGroupNonEqEdge{ + nodeIDs: ids, + nodeIDMask: mask, + expr: cond, + }) } visited := make([]bool, len(joinGroup)) nodeID2VisitID := make([]int, len(joinGroup)) @@ -66,15 +100,37 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. continue } visitID2NodeID := s.bfsGraph(i, visited, adjacents, nodeID2VisitID) + nodeIDMask := uint(0) + for _, nodeID := range visitID2NodeID { + nodeIDMask |= 1 << uint(nodeID) + } + var subNonEqEdges []joinGroupNonEqEdge + for i := len(totalNonEqEdges) - 1; i >= 0; i-- { + // If this edge is not the subset of the current sub graph. + if totalNonEqEdges[i].nodeIDMask&nodeIDMask != totalNonEqEdges[i].nodeIDMask { + continue + } + newMask := uint(0) + for _, nodeID := range totalNonEqEdges[i].nodeIDs { + newMask |= 1 << uint(nodeID2VisitID[nodeID]) + } + totalNonEqEdges[i].nodeIDMask = newMask + subNonEqEdges = append(subNonEqEdges, totalNonEqEdges[i]) + totalNonEqEdges = append(totalNonEqEdges[:i], totalNonEqEdges[i+1:]...) + } // Do DP on each sub graph. - join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEdges) + join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEqEdges, subNonEqEdges) if err != nil { return nil, err } joins = append(joins, join) } + remainedOtherConds := make([]expression.Expression, 0, len(totalNonEqEdges)) + for _, edge := range totalNonEqEdges { + remainedOtherConds = append(remainedOtherConds, edge.expr) + } // Build bushy tree for cartesian joins. - return s.makeBushyJoin(joins), nil + return s.makeBushyJoin(joins, remainedOtherConds), nil } // bfsGraph bfs a sub graph starting at startPos. And relabel its label for future use. @@ -98,13 +154,16 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents return visitID2NodeID } -func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, totalEdges []joinGroupEdge) (LogicalPlan, error) { - nodeCnt := uint(len(newPos2OldPos)) - bestPlan := make([]LogicalPlan, 1< join.statsInfo().Count()+bestCost[remain]+bestCost[sub] { - bestPlan[nodeBitmap] = join - bestCost[nodeBitmap] = join.statsInfo().Count() + bestCost[remain] + bestCost[sub] + curCost := s.calcJoinCumCost(join, bestPlan[sub], bestPlan[remain]) + if bestPlan[nodeBitmap] == nil { + bestPlan[nodeBitmap] = &jrNode{ + p: join, + cumCost: curCost, + } + } else if bestPlan[nodeBitmap].cumCost > curCost { + bestPlan[nodeBitmap].p = join + bestPlan[nodeBitmap].cumCost = curCost } } } - return bestPlan[(1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) - } else if (leftMask&(1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) + if ((leftMask&(1< 0 && (rightMask&(1< 0) || ((leftMask&(1< 0 && (rightMask&(1< 0) { + usedEqEdges = append(usedEqEdges, edge) } } - return usedEdges + for _, edge := range totalNonEqEdges { + // If the result is false, means that the current group hasn't covered the columns involved in the expression. + if edge.nodeIDMask&(leftMask|rightMask) != edge.nodeIDMask { + continue + } + // Check whether this expression is only built from one side of the join. + if edge.nodeIDMask&leftMask == 0 || edge.nodeIDMask&rightMask == 0 { + continue + } + otherConds = append(otherConds, edge.expr) + } + return usedEqEdges, otherConds } -func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEdge) (LogicalPlan, error) { +func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEqEdge, otherConds []expression.Expression) (LogicalPlan, error) { var eqConds []*expression.ScalarFunction for _, edge := range edges { lCol := edge.edge.GetArgs()[0].(*expression.Column) @@ -165,13 +244,13 @@ func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, e eqConds = append(eqConds, newSf) } } - join := s.newJoin(leftPlan, rightPlan, eqConds) + join := s.newJoin(leftPlan, rightPlan, eqConds, otherConds) _, err := join.recursiveDeriveStats() return join, err } // Make cartesian join as bushy tree. -func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { +func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, otherConds []expression.Expression) LogicalPlan { for len(cartesianJoinGroup) > 1 { resultJoinGroup := make([]LogicalPlan, 0, len(cartesianJoinGroup)) for i := 0; i < len(cartesianJoinGroup); i += 2 { @@ -179,7 +258,15 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) Lo resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } - resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil)) + // TODO:Since the other condition may involve more than two tables, e.g. t1.a = t2.b+t3.c. + // So We'll need a extra stage to deal with it. + // Currently, we just add it when building cartesianJoinGroup. + mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) + var usedOtherConds []expression.Expression + otherConds, usedOtherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds)) } cartesianJoinGroup = resultJoinGroup } @@ -194,3 +281,14 @@ func findNodeIndexInGroup(group []LogicalPlan, col *expression.Column) (int, err } return -1, ErrUnknownColumn.GenWithStackByArgs(col, "JOIN REORDER RULE") } + +func (s *joinReorderDPSolver) newJoinWithConds(leftPlan, rightPlan LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { + join := s.newCartesianJoin(leftPlan, rightPlan) + join.EqualConditions = eqConds + join.OtherConditions = otherConds + for _, eqCond := range join.EqualConditions { + join.LeftJoinKeys = append(join.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) + join.RightJoinKeys = append(join.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) + } + return join +} diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index c3d2790e18fb7..72e25a6507f4b 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -56,7 +56,7 @@ func (mj *mockLogicalJoin) recursiveDeriveStats() (*property.StatsInfo, error) { return mj.statsMap[mj.involvedNodeSet], nil } -func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan { +func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { retJoin := mockLogicalJoin{}.init(s.ctx) retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) retJoin.statsMap = s.statsMap @@ -145,7 +145,7 @@ func (s *testJoinReorderDPSuite) makeStatsMapForTPCHQ5() { } -func (s *testJoinReorderDPSuite) newDataSource(name string) LogicalPlan { +func (s *testJoinReorderDPSuite) newDataSource(name string, count int) LogicalPlan { ds := DataSource{}.Init(s.ctx) tan := model.NewCIStr(name) ds.TableAsName = &tan @@ -158,6 +158,9 @@ func (s *testJoinReorderDPSuite) newDataSource(name string) LogicalPlan { DBName: model.NewCIStr("test"), RetType: types.NewFieldType(mysql.TypeLonglong), }) + ds.stats = &property.StatsInfo{ + RowCount: float64(count), + } return ds } @@ -174,12 +177,12 @@ func (s *testJoinReorderDPSuite) planToString(plan LogicalPlan) string { func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { s.makeStatsMapForTPCHQ5() joinGroups := make([]LogicalPlan, 0, 6) - joinGroups = append(joinGroups, s.newDataSource("lineitem")) - joinGroups = append(joinGroups, s.newDataSource("orders")) - joinGroups = append(joinGroups, s.newDataSource("customer")) - joinGroups = append(joinGroups, s.newDataSource("supplier")) - joinGroups = append(joinGroups, s.newDataSource("nation")) - joinGroups = append(joinGroups, s.newDataSource("region")) + joinGroups = append(joinGroups, s.newDataSource("lineitem", 59986052)) + joinGroups = append(joinGroups, s.newDataSource("orders", 15000000)) + joinGroups = append(joinGroups, s.newDataSource("customer", 1500000)) + joinGroups = append(joinGroups, s.newDataSource("supplier", 100000)) + joinGroups = append(joinGroups, s.newDataSource("nation", 25)) + joinGroups = append(joinGroups, s.newDataSource("region", 5)) var eqConds []expression.Expression eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[1].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[1].Schema().Columns[0], joinGroups[2].Schema().Columns[0])) @@ -189,7 +192,9 @@ func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) solver := &joinReorderDPSolver{ - ctx: s.ctx, + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: s.ctx, + }, newJoin: s.newMockJoin, } result, err := solver.solve(joinGroups, eqConds) @@ -199,12 +204,14 @@ func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { func (s *testJoinReorderDPSuite) TestDPReorderAllCartesian(c *C) { joinGroup := make([]LogicalPlan, 0, 4) - joinGroup = append(joinGroup, s.newDataSource("a")) - joinGroup = append(joinGroup, s.newDataSource("b")) - joinGroup = append(joinGroup, s.newDataSource("c")) - joinGroup = append(joinGroup, s.newDataSource("d")) + joinGroup = append(joinGroup, s.newDataSource("a", 100)) + joinGroup = append(joinGroup, s.newDataSource("b", 100)) + joinGroup = append(joinGroup, s.newDataSource("c", 100)) + joinGroup = append(joinGroup, s.newDataSource("d", 100)) solver := &joinReorderDPSolver{ - ctx: s.ctx, + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: s.ctx, + }, newJoin: s.newMockJoin, } result, err := solver.solve(joinGroup, nil) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 5260fa1bde4b4..6bdf993ec54ba 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/expression" ) -type joinReorderGreedySingleGroupSolver struct { +type joinReorderGreedySolver struct { *baseSingleGroupJoinOrderSolver eqEdges []*expression.ScalarFunction } @@ -40,7 +40,7 @@ type joinReorderGreedySingleGroupSolver struct { // // For the nodes and join trees which don't have a join equal condition to // connect them, we make a bushy join tree to do the cartesian joins finally. -func (s *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { +func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { for _, node := range joinNodePlans { _, err := node.recursiveDeriveStats() if err != nil { @@ -67,7 +67,7 @@ func (s *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) return s.makeBushyJoin(cartesianGroup), nil } -func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNode, error) { +func (s *joinReorderGreedySolver) constructConnectedJoinTree() (*jrNode, error) { curJoinTree := s.curJoinGroup[0] s.curJoinGroup = s.curJoinGroup[1:] for { @@ -106,7 +106,7 @@ func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNo return curJoinTree, nil } -func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { +func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { var usedEdges []*expression.ScalarFunction remainOtherConds := make([]expression.Expression, len(s.otherConds)) copy(remainOtherConds, s.otherConds) @@ -123,5 +123,10 @@ func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode if len(usedEdges) == 0 { return nil, nil } - return s.newJoinWithEdges(usedEdges, remainOtherConds, leftNode, rightNode) + var otherConds []expression.Expression + mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) + remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index c5d7ec75cb851..b3a6200151ce8 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" ) @@ -186,7 +187,7 @@ func (s *partitionProcessor) canBePruned(sctx sessionctx.Context, partCol *expre // handle the null condition, while calculate range can prune something like: // "select * from t where t is null" accessConds := ranger.ExtractAccessConditionsForColumn(conds, partCol.UniqueID) - r, err := ranger.BuildColumnRange(accessConds, sctx.GetSessionVars().StmtCtx, partCol.RetType) + r, err := ranger.BuildColumnRange(accessConds, sctx.GetSessionVars().StmtCtx, partCol.RetType, types.UnspecifiedLength) if err != nil { return false, err } diff --git a/server/http_handler.go b/server/http_handler.go index 3a2db8174d8e4..2e3814ab24656 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -590,6 +590,19 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } +// configReloadHandler is the handler for reloading config online. +type configReloadHandler struct { +} + +// ServeHTTP handles request of reloading config for this server. +func (h configReloadHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + if err := config.ReloadGlobalConfig(); err != nil { + writeError(w, err) + } else { + writeData(w, "success!") + } +} + // ServeHTTP recovers binlog service. func (h binlogRecover) ServeHTTP(w http.ResponseWriter, req *http.Request) { binloginfo.DisableSkipBinlogFlag() diff --git a/server/http_status.go b/server/http_status.go index 62bd51c548c4f..af90a49e54e00 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -60,6 +60,7 @@ func (s *Server) startHTTPServer() { router.Handle("/stats/dump/{db}/{table}", s.newStatsHandler()).Name("StatsDump") router.Handle("/settings", settingsHandler{}).Name("Settings") + router.Handle("/reload-config", configReloadHandler{}).Name("ConfigReload") router.Handle("/binlog/recover", binlogRecover{}).Name("BinlogRecover") tikvHandlerTool := s.newTikvHandlerTool() diff --git a/server/tidb_test.go b/server/tidb_test.go index c325fd33d3916..faafef9c164ae 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -210,7 +210,7 @@ func (ts *TidbTestSuite) TestSocket(c *C) { // If parentCert and parentCertKey is specified, the new certificate will be signed by the parentCert. // Otherwise, the new certificate will be self-signed and is a CA. func generateCert(sn int, commonName string, parentCert *x509.Certificate, parentCertKey *rsa.PrivateKey, outKeyFile string, outCertFile string) (*x509.Certificate, *rsa.PrivateKey, error) { - privateKey, err := rsa.GenerateKey(rand.Reader, 512) + privateKey, err := rsa.GenerateKey(rand.Reader, 528) if err != nil { return nil, nil, errors.Trace(err) } @@ -295,11 +295,6 @@ func registerTLSConfig(configName string, caCertPath string, clientCertPath stri } func (ts *TidbTestSuite) TestTLS(c *C) { - preEnv := os.Getenv("GODEBUG") - os.Setenv("GODEBUG", "tls13=0") - defer func() { - os.Setenv("GODEBUG", preEnv) - }() // Generate valid TLS certificates. caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") c.Assert(err, IsNil) diff --git a/session/bootstrap.go b/session/bootstrap.go index 2998f53747a5e..34a0503a051a7 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -248,6 +248,16 @@ const ( DEFAULT_ROLE_USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', PRIMARY KEY (HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER) )` + + // CreateStatsTopNTable stores topn data of a cmsketch with top n. + CreateStatsTopNTable = `CREATE TABLE if not exists mysql.stats_top_n ( + table_id bigint(64) NOT NULL, + is_index tinyint(2) NOT NULL, + hist_id bigint(64) NOT NULL, + value longblob, + count bigint(64) UNSIGNED NOT NULL, + index tbl(table_id, is_index, hist_id) + );` ) // bootstrap initiates system DB for a store. @@ -317,6 +327,7 @@ const ( version27 = 27 version28 = 28 version29 = 29 + version30 = 30 ) func checkBootstrapped(s Session) (bool, error) { @@ -492,6 +503,10 @@ func upgrade(s Session) { upgradeToVer29(s) } + if ver < version30 { + upgradeToVer30(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -780,6 +795,10 @@ func upgradeToVer29(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.bind_info add index sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) } +func upgradeToVer30(s Session) { + mustExecute(s, CreateStatsTopNTable) +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. @@ -836,6 +855,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateDefaultRolesTable) // Create bind_info table. mustExecute(s, CreateBindInfoTable) + // Create stats_topn_store table. + mustExecute(s, CreateStatsTopNTable) } // doDMLWorks executes DML statements in bootstrap stage. diff --git a/session/session.go b/session/session.go index 5d045cd473387..1f504b9dd4ddd 100644 --- a/session/session.go +++ b/session/session.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" @@ -459,18 +460,12 @@ func (s *session) CommitTxn(ctx context.Context) error { defer span1.Finish() } - stmt := executor.ExecStmt{ - Text: "commitTxn", - Ctx: s, - StartTime: time.Now(), - } var commitDetail *execdetails.CommitDetails ctx = context.WithValue(ctx, execdetails.CommitDetailCtxKey, &commitDetail) err := s.doCommitWithRetry(ctx) if commitDetail != nil { s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail) } - stmt.LogSlowQuery(s.sessionVars.TxnCtx.StartTS, err == nil) s.sessionVars.TxnCtx.Cleanup() s.recordTransactionCounter(err) return err @@ -996,8 +991,9 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec sessionExecuteParseDurationGeneral.Observe(time.Since(startTS).Seconds()) } + var tempStmtNodes []ast.StmtNode compiler := executor.Compiler{Ctx: s} - for _, stmtNode := range stmtNodes { + for idx, stmtNode := range stmtNodes { s.PrepareTxnCtx(ctx) // Step2: Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). @@ -1008,11 +1004,22 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec } stmt, err := compiler.Compile(ctx, stmtNode) if err != nil { - s.rollbackOnError(ctx) - logutil.Logger(ctx).Warn("compile sql error", - zap.Error(err), - zap.String("sql", sql)) - return nil, err + if tempStmtNodes == nil { + tempStmtNodes, warns, err = s.ParseSQL(ctx, sql, charsetInfo, collation) + if err != nil || warns != nil { + //just skip errcheck, because parse will not return an error. + } + } + stmtNode = tempStmtNodes[idx] + stmt, err = compiler.SkipBindCompile(ctx, stmtNode) + if err != nil { + s.rollbackOnError(ctx) + logutil.Logger(ctx).Warn("compile sql error", + zap.Error(err), + zap.String("sql", sql)) + return nil, err + } + s.handleInvalidBindRecord(ctx, stmtNode) } if isInternal { sessionExecuteCompileDurationInternal.Observe(time.Since(startTS).Seconds()) @@ -1038,6 +1045,59 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec return recordSets, nil } +func (s *session) handleInvalidBindRecord(ctx context.Context, stmtNode ast.StmtNode) { + var normdOrigSQL string + switch x := stmtNode.(type) { + case *ast.ExplainStmt: + switch x.Stmt.(type) { + case *ast.SelectStmt: + normalizeExplainSQL := parser.Normalize(x.Text()) + idx := strings.Index(normalizeExplainSQL, "select") + normdOrigSQL = normalizeExplainSQL[idx:] + default: + return + } + case *ast.SelectStmt: + normdOrigSQL = parser.Normalize(x.Text()) + default: + return + } + sessionHandle := s.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindMeta := sessionHandle.GetBindRecord(normdOrigSQL, s.GetSessionVars().CurrentDB) + if bindMeta != nil { + bindMeta.Status = bindinfo.Invalid + return + } + + globalHandle := domain.GetDomain(s).BindHandle() + bindMeta = globalHandle.GetBindRecord(normdOrigSQL, s.GetSessionVars().CurrentDB) + if bindMeta == nil { + bindMeta = globalHandle.GetBindRecord(normdOrigSQL, "") + } + if bindMeta != nil { + record := &bindinfo.BindRecord{ + OriginalSQL: bindMeta.OriginalSQL, + BindSQL: bindMeta.BindSQL, + Db: s.GetSessionVars().CurrentDB, + Charset: bindMeta.Charset, + Collation: bindMeta.Collation, + Status: bindinfo.Invalid, + } + + err := sessionHandle.AddBindRecord(record) + if err != nil { + logutil.Logger(ctx).Warn("handleInvalidBindRecord failed", zap.Error(err)) + } + + globalHandle := domain.GetDomain(s).BindHandle() + dropBindRecord := &bindinfo.BindRecord{ + OriginalSQL: bindMeta.OriginalSQL, + Db: bindMeta.Db, + } + globalHandle.AddDropInvalidBindTask(dropBindRecord) + } +} + // rollbackOnError makes sure the next statement starts a new transaction with the latest InfoSchema. func (s *session) rollbackOnError(ctx context.Context) { if !s.sessionVars.InTxn() { @@ -1311,6 +1371,9 @@ func CreateSession(store kv.Storage) (Session, error) { Handle: do.PrivilegeHandle(), } privilege.BindPrivilegeManager(s, pm) + + sessionBindHandle := bindinfo.NewSessionBindHandle(s.parser) + s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) // 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() { @@ -1460,7 +1523,7 @@ func createSession(store kv.Storage) (*session, error) { } if plannercore.PreparedPlanCacheEnabled() { s.preparedPlanCache = kvcache.NewSimpleLRUCache(plannercore.PreparedPlanCacheCapacity, - plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) + plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) domain.BindDomain(s, dom) @@ -1483,7 +1546,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er } if plannercore.PreparedPlanCacheEnabled() { s.preparedPlanCache = kvcache.NewSimpleLRUCache(plannercore.PreparedPlanCacheCapacity, - plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) + plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) domain.BindDomain(s, dom) @@ -1495,7 +1558,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 29 + currentBootstrapVersion = 30 ) func getStoreBootstrapVersion(store kv.Storage) int64 { diff --git a/session/tidb.go b/session/tidb.go index 8b8cb6c34ed0b..2be35eb4165b3 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -196,16 +196,21 @@ func checkStmtLimit(ctx context.Context, sctx sessionctx.Context, se *session, s } // runStmt executes the sqlexec.Statement and commit or rollback the current transaction. -func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (sqlexec.RecordSet, error) { +func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (rs sqlexec.RecordSet, err error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.runStmt", opentracing.ChildOf(span.Context())) span1.LogKV("sql", s.OriginText()) defer span1.Finish() } - - var err error - var rs sqlexec.RecordSet se := sctx.(*session) + defer func() { + // If it is not a select statement, we record its slow log here, + // then it could include the transaction commit time. + if rs == nil { + s.(*executor.ExecStmt).LogSlowQuery(se.GetSessionVars().TxnCtx.StartTS, err != nil) + } + }() + err = se.checkTxnAborted(s) if err != nil { return nil, err diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index c9e46d7b30ecc..81a58c6243489 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -347,6 +347,10 @@ type SessionVars struct { // CommandValue indicates which command current session is doing. CommandValue uint32 + // TIDBOptJoinOrderAlgoThreshold defines the minimal number of join nodes + // to use the greedy join reorder algorithm. + TiDBOptJoinReorderThreshold int + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. SlowQueryFile string @@ -377,30 +381,31 @@ type ConnectionInfo struct { // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]string), - systems: make(map[string]string), - PreparedStmts: make(map[uint32]*ast.Prepared), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - KVVars: kv.NewVariables(), - RetryInfo: &RetryInfo{}, - ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - StrictSQLMode: true, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - EnableRadixJoin: false, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + Users: make(map[string]string), + systems: make(map[string]string), + PreparedStmts: make(map[uint32]*ast.Prepared), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + KVVars: kv.NewVariables(), + RetryInfo: &RetryInfo{}, + ActiveRoles: make([]*auth.RoleIdentity, 0, 10), + StrictSQLMode: true, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + EnableRadixJoin: false, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -754,6 +759,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) + case TiDBOptJoinReorderThreshold: + s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUTF8: config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) case TiDBSlowQueryFile: @@ -938,7 +945,7 @@ const ( // SlowLogFormat uses for formatting slow log. // The slow log output is like below: -// # Time: 2019-02-12-19:33:56.571953 +0800 +// # Time: 2019-04-28T15:24:04.309074+08:00 // # Txn_start_ts: 406315658548871171 // # User: root@127.0.0.1 // # Conn_ID: 6 diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 38b0f6754cab6..2a88b23393b71 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -689,6 +689,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, + {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, {ScopeSession, TiDBSlowQueryFile, ""}, {ScopeSession, TiDBWaitTableSplitFinish, BoolToIntStr(DefTiDBWaitTableSplitFinish)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d40dacb2318f3..f16aec3d5cc4e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -250,6 +250,10 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + // TIDBOptJoinReorderThreshold defines the threshold less than which + // we'll choose a rather time consuming algorithm to calculate the join order. + TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. TiDBSlowQueryFile = "tidb_slow_query_file" @@ -309,6 +313,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = false + DefTiDBOptJoinReorderThreshold = 0 DefTiDBDDLSlowOprThreshold = 300 DefTiDBUseFastAnalyze = false DefTiDBSkipIsolationLevelCheck = false diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 6783975b1b1ef..a9e49e323daae 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -485,6 +485,14 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, errors.Errorf("tidb_max_chunk_size(%d) cannot be smaller than %d", v, maxChunkSizeLowerBound) } return value, nil + case TiDBOptJoinReorderThreshold: + v, err := strconv.Atoi(value) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } + if v < 0 || v >= 64 { + return value, errors.Errorf("tidb_join_order_algo_threshold(%d) cannot be smaller than 0 or larger than 63", v) + } } return value, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 2b4c04992a4a9..4875582fd39e0 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -79,6 +79,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) c.Assert(vars.EnableRadixJoin, Equals, DefTiDBUseRadixJoin) c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) + c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) c.Assert(vars.EnableFastAnalyze, Equals, DefTiDBUseFastAnalyze) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) @@ -254,6 +255,14 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "on") c.Assert(v.EnableTablePartition, Equals, "on") + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) + err = SetSessionSystemVar(v, TiDBOptJoinReorderThreshold, types.NewIntDatum(5)) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBOptJoinReorderThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, "5") + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) + err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("1")) c.Assert(err, IsNil) val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) diff --git a/statistics/analyze_jobs.go b/statistics/analyze_jobs.go index b6910aa4a77f0..ff1fedfcf6464 100644 --- a/statistics/analyze_jobs.go +++ b/statistics/analyze_jobs.go @@ -63,7 +63,7 @@ func MoveToHistory(job *AnalyzeJob) { analyzeStatus.Lock() delete(analyzeStatus.jobs, job) analyzeStatus.history = append(analyzeStatus.history, job) - numJobs := len(analyzeStatus.jobs) + numJobs := len(analyzeStatus.history) if numJobs > numMaxHistoryJobs { analyzeStatus.history = analyzeStatus.history[numJobs-numMaxHistoryJobs:] } diff --git a/statistics/analyze_jobs_test.go b/statistics/analyze_jobs_test.go new file mode 100644 index 0000000000000..0c91672ea9bc6 --- /dev/null +++ b/statistics/analyze_jobs_test.go @@ -0,0 +1,34 @@ +// 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 statistics + +import ( + . "github.com/pingcap/check" +) + +func (s *testStatisticsSuite) TestMoveToHistory(c *C) { + numJobs := numMaxHistoryJobs*2 + 1 + jobs := make([]*AnalyzeJob, 0, numJobs) + for i := 0; i < numJobs; i++ { + job := &AnalyzeJob{} + AddNewAnalyzeJob(job) + jobs = append(jobs, job) + } + MoveToHistory(jobs[0]) + c.Assert(len(GetAllAnalyzeJobs()), Equals, numJobs) + for i := 1; i < numJobs; i++ { + MoveToHistory(jobs[i]) + } + c.Assert(len(GetAllAnalyzeJobs()), Equals, numMaxHistoryJobs) +} diff --git a/statistics/builder.go b/statistics/builder.go index 9ede4ad58cb0b..c675656135557 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -93,15 +93,20 @@ func (b *SortedBuilder) Iterate(data types.Datum) error { return nil } -// BuildColumn builds histogram from samples for column. -func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) { - count := collector.Count - ndv := collector.FMSketch.NDV() +// BuildColumnHist build a histogram for a column. +// numBuckets: number of buckets for the histogram. +// id: the id of the table. +// collector: the collector of samples. +// tp: the FieldType for the column. +// count: represents the row count for the column. +// ndv: represents the number of distinct values for the column. +// nullCount: represents the number of null values for the column. +func BuildColumnHist(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType, count int64, ndv int64, nullCount int64) (*Histogram, error) { if ndv > count { ndv = count } if count == 0 || len(collector.Samples) == 0 { - return NewHistogram(id, ndv, collector.NullCount, 0, tp, 0, collector.TotalSize), nil + return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil } sc := ctx.GetSessionVars().StmtCtx samples := collector.Samples @@ -109,7 +114,7 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample if err != nil { return nil, err } - hg := NewHistogram(id, ndv, collector.NullCount, 0, tp, int(numBuckets), collector.TotalSize) + hg := NewHistogram(id, ndv, nullCount, 0, tp, int(numBuckets), collector.TotalSize) sampleNum := int64(len(samples)) // As we use samples to build the histogram, the bucket number and repeat should multiply a factor. @@ -174,3 +179,8 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample hg.Correlation = (itemsCount*corrXYSum - corrXSum*corrXSum) / (itemsCount*corrX2Sum - corrXSum*corrXSum) return hg, nil } + +// BuildColumn builds histogram from samples for column. +func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) { + return BuildColumnHist(ctx, numBuckets, id, collector, tp, collector.Count, collector.FMSketch.NDV(), collector.NullCount) +} diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 6961d4733880e..6c58fe47abab9 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -15,6 +15,7 @@ package statistics import ( "bytes" + "fmt" "math" "sort" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/spaolacci/murmur3" ) @@ -40,15 +42,14 @@ type CMSketch struct { count uint64 // TopN is not counted in count defaultValue uint64 // In sampled data, if cmsketch returns a small value (less than avg value / 2), then this will returned. table [][]uint32 - topN map[uint64][]topNMeta + topN map[uint64][]*TopNMeta } -// topNMeta is a simple counter used by BuildTopN -type topNMeta struct { - h1 uint64 - h2 uint64 - data []byte - count uint64 +// TopNMeta is a simple counter used by BuildTopN. +type TopNMeta struct { + h2 uint64 // h2 is the second part of `murmur3.Sum128()`, it is always used with the first part `h1`. + Data []byte + Count uint64 } // NewCMSketch returns a new CM sketch. @@ -63,7 +64,6 @@ func NewCMSketch(d, w int32) *CMSketch { // topNHelper wraps some variables used when building cmsketch with top n. type topNHelper struct { sampleSize uint64 - numTop uint32 counter map[hack.MutableString]uint64 sorted []uint64 onlyOnceItems uint64 @@ -106,11 +106,11 @@ func newTopNHelper(sample [][]byte, numTop uint32) *topNHelper { sumTopN += sorted[i] } - return &topNHelper{uint64(len(sample)), numTop, counter, sorted, onlyOnceItems, sumTopN, last} + return &topNHelper{uint64(len(sample)), counter, sorted, onlyOnceItems, sumTopN, last} } -// NewCMSketchWithTopN returns a new CM sketch with TopN elements. -func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uint64) *CMSketch { +// NewCMSketchWithTopN returns a new CM sketch with TopN elements, the estimate NDV and the scale ratio. +func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uint64) (*CMSketch, uint64, uint64) { helper := newTopNHelper(sample, numTop) // rowCount is not a accurate value when fast analyzing // In some cases, if user triggers fast analyze when rowCount is close to sampleSize, unexpected bahavior might happen. @@ -118,24 +118,22 @@ func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount ui estimateNDV, scaleRatio := calculateEstimateNDV(helper, rowCount) c := buildCMSWithTopN(helper, d, w, scaleRatio) c.calculateDefaultVal(helper, estimateNDV, scaleRatio, rowCount) - return c + return c, estimateNDV, scaleRatio } func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64) (c *CMSketch) { - c, helper.sumTopN, helper.numTop = NewCMSketch(d, w), 0, 0 + c = NewCMSketch(d, w) enableTopN := helper.sampleSize/topNThreshold <= helper.sumTopN if enableTopN { - c.topN = make(map[uint64][]topNMeta) + c.topN = make(map[uint64][]*TopNMeta) } for counterKey, cnt := range helper.counter { data, scaledCount := hack.Slice(string(counterKey)), cnt*scaleRatio if enableTopN && cnt >= helper.lastVal { h1, h2 := murmur3.Sum128(data) - c.topN[h1] = append(c.topN[h1], topNMeta{h1, h2, data, scaledCount}) - helper.sumTopN += scaledCount - helper.numTop++ + c.topN[h1] = append(c.topN[h1], &TopNMeta{h2, data, scaledCount}) } else { - c.updateBytesWithDelta(data, scaledCount) + c.insertBytesByCount(data, scaledCount) } } return @@ -143,11 +141,7 @@ func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64) (c *CMS func (c *CMSketch) calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRatio, rowCount uint64) { sampleNDV := uint64(len(helper.sorted)) - if rowCount <= helper.sumTopN { - c.defaultValue = 1 - } else if estimateNDV <= uint64(helper.numTop) { - c.defaultValue = 1 - } else if estimateNDV+helper.onlyOnceItems <= uint64(sampleNDV) { + if rowCount <= (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio { c.defaultValue = 1 } else { estimateRemainingCount := rowCount - (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio @@ -155,17 +149,25 @@ func (c *CMSketch) calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRat } } +func (c *CMSketch) findTopNMeta(h1, h2 uint64, d []byte) *TopNMeta { + for _, meta := range c.topN[h1] { + if meta.h2 == h2 && bytes.Equal(d, meta.Data) { + return meta + } + } + return nil +} + // queryAddTopN TopN adds count to CMSketch.topN if exists, and returns the count of such elements after insert. // If such elements does not in topn elements, nothing will happen and false will be returned. func (c *CMSketch) updateTopNWithDelta(h1, h2 uint64, d []byte, delta uint64) bool { if c.topN == nil { return false } - for _, cnt := range c.topN[h1] { - if cnt.h2 == h2 && bytes.Equal(d, cnt.data) { - cnt.count += delta - return true - } + meta := c.findTopNMeta(h1, h2, d) + if meta != nil { + meta.Count += delta + return true } return false } @@ -174,38 +176,47 @@ func (c *CMSketch) queryTopN(h1, h2 uint64, d []byte) (uint64, bool) { if c.topN == nil { return 0, false } - for _, cnt := range c.topN[h1] { - if cnt.h2 == h2 && bytes.Equal(d, cnt.data) { - return cnt.count, true - } + meta := c.findTopNMeta(h1, h2, d) + if meta != nil { + return meta.Count, true } return 0, false } // InsertBytes inserts the bytes value into the CM Sketch. func (c *CMSketch) InsertBytes(bytes []byte) { - c.updateBytesWithDelta(bytes, 1) + c.insertBytesByCount(bytes, 1) } -// updateBytesWithDelta adds the bytes value into the CM Sketch by delta. -func (c *CMSketch) updateBytesWithDelta(bytes []byte, delta uint64) { +// insertBytesByCount adds the bytes value into the TopN (if value already in TopN) or CM Sketch by delta, this does not updates c.defaultValue. +func (c *CMSketch) insertBytesByCount(bytes []byte, count uint64) { h1, h2 := murmur3.Sum128(bytes) - if c.updateTopNWithDelta(h1, h2, bytes, delta) { + if c.updateTopNWithDelta(h1, h2, bytes, count) { return } - c.count += delta + c.count += count for i := range c.table { j := (h1 + h2*uint64(i)) % uint64(c.width) - c.table[i][j] += uint32(delta) + c.table[i][j] += uint32(count) } } func (c *CMSketch) considerDefVal(cnt uint64) bool { - return cnt < 2*(c.count/uint64(c.width)) && c.defaultValue > 0 + return (cnt == 0 || (cnt > c.defaultValue && cnt < 2*(c.count/uint64(c.width)))) && c.defaultValue > 0 +} + +// updateValueBytes updates value of d to count. +func (c *CMSketch) updateValueBytes(d []byte, count uint64) { + h1, h2 := murmur3.Sum128(d) + if oriCount, ok := c.queryTopN(h1, h2, d); ok { + deltaCount := count - oriCount + c.updateTopNWithDelta(h1, h2, d, deltaCount) + } + c.setValue(h1, h2, count) } -// setValue sets the count for value that hashed into (h1, h2). -func (c *CMSketch) setValue(h1, h2 uint64, count uint32) { +// setValue sets the count for value that hashed into (h1, h2), and update defaultValue if necessary. +func (c *CMSketch) setValue(h1, h2 uint64, count uint64) { oriCount := c.queryHashValue(h1, h2) if c.considerDefVal(oriCount) { // We should update c.defaultValue if we used c.defaultValue when getting the estimate count. @@ -217,9 +228,9 @@ func (c *CMSketch) setValue(h1, h2 uint64, count uint32) { } } - c.count += uint64(count) - oriCount + c.count += count - oriCount // let it overflow naturally - deltaCount := count - uint32(oriCount) + deltaCount := uint32(count) - uint32(oriCount) for i := range c.table { j := (h1 + h2*uint64(i)) % uint64(c.width) c.table[i][j] = c.table[i][j] + deltaCount @@ -288,7 +299,6 @@ func (c *CMSketch) MergeCMSketch(rc *CMSketch) error { } // CMSketchToProto converts CMSketch to its protobuf representation. -// TODO: Encode/Decode cmsketch with Top-N func CMSketchToProto(c *CMSketch) *tipb.CMSketch { protoSketch := &tipb.CMSketch{Rows: make([]*tipb.CMSketchRow, c.depth)} for i := range c.table { @@ -297,11 +307,16 @@ func CMSketchToProto(c *CMSketch) *tipb.CMSketch { protoSketch.Rows[i].Counters[j] = c.table[i][j] } } + for _, dataSlice := range c.topN { + for _, dataMeta := range dataSlice { + protoSketch.TopN = append(protoSketch.TopN, &tipb.CMSketchTopN{Data: dataMeta.Data, Count: dataMeta.Count}) + } + } + protoSketch.DefaultValue = c.defaultValue return protoSketch } // CMSketchFromProto converts CMSketch from its protobuf representation. -// TODO: Encode/Decode cmsketch with Top-N func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch { if protoSketch == nil { return nil @@ -314,20 +329,32 @@ func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch { c.count = c.count + uint64(counter) } } + if len(protoSketch.TopN) == 0 { + return c + } + c.defaultValue = protoSketch.DefaultValue + c.topN = make(map[uint64][]*TopNMeta) + for _, e := range protoSketch.TopN { + h1, h2 := murmur3.Sum128(e.Data) + c.topN[h1] = append(c.topN[h1], &TopNMeta{h2, e.Data, e.Count}) + } return c } -// EncodeCMSketch encodes the given CMSketch to byte slice. -func EncodeCMSketch(c *CMSketch) ([]byte, error) { - if c == nil || c.count == 0 { +// EncodeCMSketchWithoutTopN encodes the given CMSketch to byte slice. +// Note that it does not include the topN. +func EncodeCMSketchWithoutTopN(c *CMSketch) ([]byte, error) { + if c == nil { return nil, nil } p := CMSketchToProto(c) - return p.Marshal() + p.TopN = nil + protoData, err := p.Marshal() + return protoData, err } -// DecodeCMSketch decode a CMSketch from the given byte slice. -func DecodeCMSketch(data []byte) (*CMSketch, error) { +// decodeCMSketch decode a CMSketch from the given byte slice. +func decodeCMSketch(data []byte, topN []*TopNMeta) (*CMSketch, error) { if data == nil { return nil, nil } @@ -336,12 +363,29 @@ func DecodeCMSketch(data []byte) (*CMSketch, error) { if err != nil { return nil, errors.Trace(err) } - if len(p.Rows) == 0 { + if len(p.Rows) == 0 && len(topN) == 0 { return nil, nil } + for _, meta := range topN { + p.TopN = append(p.TopN, &tipb.CMSketchTopN{Data: meta.Data, Count: meta.Count}) + } return CMSketchFromProto(p), nil } +// LoadCMSketchWithTopN loads the CM sketch with topN from storage. +func LoadCMSketchWithTopN(exec sqlexec.RestrictedSQLExecutor, tableID, isIndex, histID int64, cms []byte) (*CMSketch, error) { + sql := fmt.Sprintf("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, histID) + topNRows, _, err := exec.ExecRestrictedSQL(nil, sql) + if err != nil { + return nil, err + } + topN := make([]*TopNMeta, 0, len(topNRows)) + for _, row := range topNRows { + topN = append(topN, &TopNMeta{Data: row.GetBytes(0), Count: row.GetUint64(1)}) + } + return decodeCMSketch(cms, topN) +} + // TotalCount returns the count, it is only used for test. func (c *CMSketch) TotalCount() uint64 { return c.count @@ -352,7 +396,7 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { if c == nil || rc == nil { return c == nil && rc == nil } - if c.width != rc.width || c.depth != rc.depth || c.count != rc.count { + if c.width != rc.width || c.depth != rc.depth || c.count != rc.count || c.defaultValue != rc.defaultValue { return false } for i := range c.table { @@ -362,6 +406,20 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { } } } + if len(c.topN) != len(rc.topN) { + return false + } + for h1, topNData := range c.topN { + if len(topNData) != len(rc.topN[h1]) { + return false + } + for _, val := range topNData { + meta := rc.findTopNMeta(h1, val.h2, val.Data) + if meta == nil || meta.Count != val.Count { + return false + } + } + } return true } @@ -375,5 +433,35 @@ func (c *CMSketch) Copy() *CMSketch { tbl[i] = make([]uint32, c.width) copy(tbl[i], c.table[i]) } - return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl} + var topN map[uint64][]*TopNMeta + if c.topN != nil { + topN = make(map[uint64][]*TopNMeta) + for h1, vals := range c.topN { + newVals := make([]*TopNMeta, 0, len(vals)) + for _, val := range vals { + newVal := TopNMeta{h2: val.h2, Count: val.Count, Data: make([]byte, len(val.Data))} + copy(newVal.Data, val.Data) + newVals = append(newVals, &newVal) + } + topN[h1] = newVals + } + } + return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl, defaultValue: c.defaultValue, topN: topN} +} + +// TopN gets all the topN meta. +func (c *CMSketch) TopN() []*TopNMeta { + if c == nil { + return nil + } + topN := make([]*TopNMeta, 0, len(c.topN)) + for _, meta := range c.topN { + topN = append(topN, meta...) + } + return topN +} + +// GetWidthAndDepth returns the width and depth of CM Sketch. +func (c *CMSketch) GetWidthAndDepth() (int32, int32) { + return c.width, c.depth } diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index f95e0b12e2fb4..44ee8a57dfed6 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -14,6 +14,7 @@ package statistics import ( + "fmt" "math" "math/rand" "time" @@ -23,6 +24,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/spaolacci/murmur3" ) func (c *CMSketch) insert(val *types.Datum) error { @@ -43,7 +45,8 @@ func prepareCMSWithTopN(d, w int32, vals []*types.Datum, n uint32, total uint64) } data = append(data, bytes) } - return NewCMSketchWithTopN(d, w, data, n, total), nil + cms, _, _ := NewCMSketchWithTopN(d, w, data, n, total) + return cms, nil } func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64) (*CMSketch, map[int64]uint32, error) { @@ -147,10 +150,10 @@ func (s *testStatisticsSuite) TestCMSketchCoding(c *C) { lSketch.table[i][j] = math.MaxUint32 } } - bytes, err := EncodeCMSketch(lSketch) + bytes, err := EncodeCMSketchWithoutTopN(lSketch) c.Assert(err, IsNil) - c.Assert(len(bytes), Equals, 61455) - rSketch, err := DecodeCMSketch(bytes) + c.Assert(len(bytes), Equals, 61457) + rSketch, err := decodeCMSketch(bytes, nil) c.Assert(err, IsNil) c.Assert(lSketch.Equal(rSketch), IsTrue) } @@ -212,3 +215,26 @@ func (s *testStatisticsSuite) TestCMSketchTopNUniqueData(c *C) { c.Check(avg, Equals, uint64(0)) c.Check(len(cms.topN), Equals, 0) } + +func (s *testStatisticsSuite) TestCMSketchCodingTopN(c *C) { + lSketch := NewCMSketch(5, 2048) + lSketch.count = 2048 * (math.MaxUint32) + for i := range lSketch.table { + for j := range lSketch.table[i] { + lSketch.table[i][j] = math.MaxUint32 + } + } + lSketch.topN = make(map[uint64][]*TopNMeta) + for i := 0; i < 20; i++ { + tString := []byte(fmt.Sprintf("%20000d", i)) + h1, h2 := murmur3.Sum128(tString) + lSketch.topN[h1] = []*TopNMeta{{h2, tString, math.MaxUint64}} + } + + bytes, err := EncodeCMSketchWithoutTopN(lSketch) + c.Assert(err, IsNil) + c.Assert(len(bytes), Equals, 61457) + rSketch, err := decodeCMSketch(bytes, lSketch.TopN()) + c.Assert(err, IsNil) + c.Assert(lSketch.Equal(rSketch), IsTrue) +} diff --git a/statistics/estimate.go b/statistics/estimate.go index 0878cf1e36298..2112a8fc3767c 100644 --- a/statistics/estimate.go +++ b/statistics/estimate.go @@ -18,8 +18,6 @@ import ( ) // calculateEstimateNDV calculates the estimate ndv of a sampled data from a multisize with size total. -// count[i] stores the count of the i-th element. -// onlyOnceItems is the number of elements that occurred only once. func calculateEstimateNDV(h *topNHelper, rowCount uint64) (ndv uint64, scaleRatio uint64) { sampleSize, sampleNDV, onlyOnceItems := h.sampleSize, uint64(len(h.sorted)), h.onlyOnceItems scaleRatio = rowCount / sampleSize diff --git a/statistics/feedback.go b/statistics/feedback.go index 32e52c6fab27d..95abf21dfc0ce 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/spaolacci/murmur3" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -83,7 +84,7 @@ var ( // MaxNumberOfRanges is the max number of ranges before split to collect feedback. MaxNumberOfRanges = 20 // FeedbackProbability is the probability to collect the feedback. - FeedbackProbability = 0.0 + FeedbackProbability = atomic.NewFloat64(0) ) // CalcErrorRate calculates the error rate the current QueryFeedback. @@ -106,7 +107,7 @@ func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { if q.Hist == nil || q.Hist.Len() == 0 { return false } - if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability { + if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability.Load() { return false } return true @@ -609,8 +610,7 @@ func UpdateCMSketch(c *CMSketch, eqFeedbacks []Feedback) *CMSketch { } newCMSketch := c.Copy() for _, fb := range eqFeedbacks { - h1, h2 := murmur3.Sum128(fb.Lower.GetBytes()) - newCMSketch.setValue(h1, h2, uint32(fb.Count)) + newCMSketch.updateValueBytes(fb.Lower.GetBytes(), uint64(fb.Count)) } return newCMSketch } @@ -728,7 +728,8 @@ func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch) { // decode the index point feedback, just set value count in CM Sketch start := len(pb.IndexRanges) / 2 for i := 0; i < len(pb.HashValues); i += 2 { - c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint32(pb.Counts[start+i/2])) + // TODO: update using raw bytes instead of hash values. + c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint64(pb.Counts[start+i/2])) } } } diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 42a19b21d0499..a461bc6d6911d 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -103,7 +103,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables Stat if idxInfo == nil { continue } - cms, err := statistics.DecodeCMSketch(row.GetBytes(6)) + cms, err := statistics.LoadCMSketchWithTopN(h.restrictedExec, row.GetInt64(0), row.GetInt64(1), row.GetInt64(2), row.GetBytes(6)) if err != nil { cms = nil terror.Log(errors.Trace(err)) diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index a56e1163de907..a1371e8e7f925 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -116,3 +116,45 @@ func (s *testStatsSuite) TestDumpAlteredTable(c *C) { _, err = h.DumpStatsToJSON("test", table.Meta()) c.Assert(err, IsNil) } + +func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { + // Just test if we can store and recover the Top N elements stored in database. + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("create table t(a int)") + testKit.MustExec("insert into t values (1),(3),(4),(2),(5)") + testKit.MustExec("analyze table t") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + h.Update(is) + + // Insert 30 fake data + fakeData := make([][]byte, 0, 30) + for i := 0; i < 30; i++ { + fakeData = append(fakeData, []byte(fmt.Sprintf("%01024d", i))) + } + cms, _, _ := statistics.NewCMSketchWithTopN(5, 2048, fakeData, 20, 100) + + stat := h.GetTableStats(tableInfo) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, 1) + c.Assert(err, IsNil) + c.Assert(h.Update(is), IsNil) + + stat = h.GetTableStats(tableInfo) + cmsFromStore := stat.Columns[tableInfo.Columns[0].ID].CMSketch + c.Assert(cmsFromStore, NotNil) + c.Check(cms.Equal(cmsFromStore), IsTrue) + + jsonTable, err := h.DumpStatsToJSON("test", tableInfo) + c.Check(err, IsNil) + err = h.LoadStatsFromJSON(is, jsonTable) + c.Check(err, IsNil) + stat = h.GetTableStats(tableInfo) + cmsFromJSON := stat.Columns[tableInfo.Columns[0].ID].CMSketch.Copy() + c.Check(cms.Equal(cmsFromJSON), IsTrue) +} diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index bbca79a972013..0cc79e89667e7 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -122,6 +122,11 @@ func (h *Handle) deleteHistStatsFromKV(physicalID int64, histID int64, isIndex i if err != nil { return } + // delete top n data + _, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_top_n where table_id = %d and hist_id = %d and is_index = %d", physicalID, histID, isIndex)) + if err != nil { + return + } // delete all buckets _, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d and hist_id = %d and is_index = %d", physicalID, histID, isIndex)) return diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 7545d22e35cdf..8432fdfba2535 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" + atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -89,7 +90,7 @@ func (h *Handle) Clear() { } // MaxQueryFeedbackCount is the max number of feedback that cache in memory. -var MaxQueryFeedbackCount = 1 << 10 +var MaxQueryFeedbackCount = atomic2.NewInt64(1 << 10) // NewHandle creates a Handle for update stats. func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { @@ -98,7 +99,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, globalMap: make(tableDeltaMap), Lease: lease, - feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount), + feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount.Load()), } // It is safe to use it concurrently because the exec won't touch the ctx. if exec, ok := ctx.(sqlexec.RestrictedSQLExecutor); ok { @@ -325,7 +326,7 @@ func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*stati if len(rows) == 0 { return nil, nil } - return statistics.DecodeCMSketch(rows[0].GetBytes(0)) + return statistics.LoadCMSketchWithTopN(h.restrictedExec, tblID, isIndex, histID, rows[0].GetBytes(0)) } func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo) error { @@ -524,10 +525,23 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg if err != nil { return } - data, err := statistics.EncodeCMSketch(cms) + data, err := statistics.EncodeCMSketchWithoutTopN(cms) if err != nil { return } + // Delete outdated data + deleteOutdatedTopNSQL := fmt.Sprintf("delete from mysql.stats_top_n where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, hg.ID) + _, err = exec.Execute(ctx, deleteOutdatedTopNSQL) + if err != nil { + return + } + for _, meta := range cms.TopN() { + insertSQL := fmt.Sprintf("insert into mysql.stats_top_n (table_id, is_index, hist_id, value, count) values (%d, %d, %d, X'%X', %d)", tableID, isIndex, hg.ID, meta.Data, meta.Count) + _, err = exec.Execute(ctx, insertSQL) + if err != nil { + return + } + } flag := 0 if isAnalyzed == 1 { flag = statistics.AnalyzeFlag diff --git a/statistics/handle/update.go b/statistics/handle/update.go index bfff6364aece3..24ad2fe7e1c85 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -160,7 +160,7 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi func mergeQueryFeedback(lq []*statistics.QueryFeedback, rq []*statistics.QueryFeedback) []*statistics.QueryFeedback { for _, q := range rq { - if len(lq) >= MaxQueryFeedbackCount { + if len(lq) >= int(MaxQueryFeedbackCount.Load()) { break } lq = append(lq, q) @@ -198,7 +198,7 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand defer s.Unlock() isIndex := q.Tp == statistics.IndexType s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) - if len(s.feedback) < MaxQueryFeedbackCount { + if len(s.feedback) < int(MaxQueryFeedbackCount.Load()) { s.feedback = append(s.feedback, q) } return nil diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 8748e7857b3af..f91b9c77587d6 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -545,7 +545,7 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -615,7 +615,7 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -733,7 +733,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) tests := []struct { sql string hist string @@ -799,7 +799,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { } // Test collect feedback by probability. - statistics.FeedbackProbability = 0 + statistics.FeedbackProbability.Store(0) statistics.MaxNumberOfRanges = oriNumber for _, t := range tests { testKit.MustQuery(t.sql) @@ -809,7 +809,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { } // Test that after drop stats, the feedback won't cause panic. - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) for _, t := range tests { testKit.MustQuery(t.sql) } @@ -845,7 +845,7 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { statistics.FeedbackProbability = oriProbability }() h := s.do.StatsHandle() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) tests := []struct { sql string hist string @@ -969,7 +969,7 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1022,7 +1022,7 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1094,7 +1094,7 @@ func (s *testStatsSuite) TestLogDetailedInfo(c *C) { s.do.StatsHandle().Lease = oriLease log.SetLevel(oriLevel) }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) handle.MinLogScanCount = 0 handle.MinLogErrorRate = 0 s.do.StatsHandle().Lease = 1 @@ -1264,7 +1264,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), index idx_ab(a,b), index idx_ac(a,c), index idx_b(b))") @@ -1341,7 +1341,7 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") @@ -1410,7 +1410,7 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") @@ -1479,7 +1479,7 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") diff --git a/statistics/histogram.go b/statistics/histogram.go index 2642ccd6dd520..31b70eabecd79 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" + "github.com/spaolacci/murmur3" "go.uber.org/zap" ) @@ -565,6 +566,28 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { chunk.Compare(hg.Bounds.GetRow(hg.Bounds.NumRows()-1), 0, &val) < 0 } +// ContainsFeedback checks if the histogram contains feedback updates. +// We can test it from the `repeat` field because only feedback will update it to 0. +func (hg *Histogram) ContainsFeedback() bool { + for _, bkt := range hg.Buckets { + if bkt.Repeat == 0 { + return true + } + } + return false +} + +// Copy deep copies the histogram. +func (hg *Histogram) Copy() *Histogram { + newHist := *hg + newHist.Bounds = hg.Bounds.CopyConstruct() + newHist.Buckets = make([]Bucket, 0, len(hg.Buckets)) + for _, bkt := range hg.Buckets { + newHist.Buckets = append(newHist.Buckets, bkt) + } + return &newHist +} + // ErrorRate is the error rate of estimate row count by bucket and cm sketch. type ErrorRate struct { ErrorTotal float64 @@ -967,6 +990,28 @@ func (idx *Index) outOfRange(val types.Datum) bool { return !withInLowBoundOrPrefixMatch || !withInHighBound } +// RemoveUpperBound removes the upper bound the index stats. +// It is used when merge stats for incremental analyze. +func (idx *Index) RemoveUpperBound(sc *stmtctx.StatementContext, values []types.Datum) (*Histogram, *CMSketch, error) { + hist, cms := idx.Histogram.Copy(), idx.CMSketch.Copy() + hist.Buckets[hist.Len()-1].Count -= hist.Buckets[hist.Len()-1].Repeat + hist.Buckets[hist.Len()-1].Repeat = 0 + if cms == nil { + return hist, nil, nil + } + var data []byte + var err error + for _, val := range values { + data, err = codec.EncodeKey(sc, data, val) + if err != nil { + return nil, nil, err + } + h1, h2 := murmur3.Sum128(data) + cms.setValue(h1, h2, 0) + } + return hist, cms, nil +} + // matchPrefix checks whether ad is the prefix of value func matchPrefix(row chunk.Row, colIdx int, ad *types.Datum) bool { switch ad.Kind() { diff --git a/statistics/sample.go b/statistics/sample.go index 6805943046c8f..31b1f712b1719 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -36,6 +36,9 @@ type SampleItem struct { // Ordinal is original position of this item in SampleCollector before sorting. This // is used for computing correlation. Ordinal int + // RowID is the row id of the sample in its key. + // This property is used to calculate Ordinal in fast analyze. + RowID int64 } // SortSampleItems sorts a slice of SampleItem. @@ -173,6 +176,14 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e return nil } +// CalcTotalSize is to calculate total size based on samples. +func (c *SampleCollector) CalcTotalSize() { + c.TotalSize = 0 + for _, item := range c.Samples { + c.TotalSize += int64(len(item.Value.GetBytes())) + } +} + // SampleBuilder is used to build samples for columns. // Also, if primary key is handle, it will directly build histogram for it. type SampleBuilder struct { diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 60cd6d80bff98..fef5c54907d25 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -266,7 +267,7 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran switch rangeType { case ranger.ColumnRangeType: accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].UniqueID) - ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType) + ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength) case ranger.IndexRangeType: var res *ranger.DetachRangeResult res, err = ranger.DetachCondAndBuildRangeForIndex(ctx, exprs, cols, lengths) diff --git a/statistics/table.go b/statistics/table.go index ce666369d845f..9fc6964020fe3 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" + "go.uber.org/atomic" ) const ( @@ -160,11 +161,11 @@ func (n *neededColumnMap) Delete(col tableColumnID) { // RatioOfPseudoEstimate means if modifyCount / statsTblCount is greater than this ratio, we think the stats is invalid // and use pseudo estimation. -var RatioOfPseudoEstimate = 0.7 +var RatioOfPseudoEstimate = atomic.NewFloat64(0.7) // IsOutdated returns true if the table stats is outdated. func (t *Table) IsOutdated() bool { - if t.Count > 0 && float64(t.ModifyCount)/float64(t.Count) > RatioOfPseudoEstimate { + if t.Count > 0 && float64(t.ModifyCount)/float64(t.Count) > RatioOfPseudoEstimate.Load() { return true } return false diff --git a/store/helper/helper.go b/store/helper/helper.go index 392a67404704b..6e0e250755778 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/pdapi" "go.uber.org/zap" ) @@ -365,3 +366,157 @@ func (r *RegionFrameRange) GetIndexFrame(tableID, indexID int64, dbName, tableNa } return nil } + +// RegionPeer stores information of one peer. +type RegionPeer struct { + ID int64 `json:"id"` + StoreID int64 `json:"store_id"` + IsLearner bool `json:"is_learner"` +} + +// RegionEpoch stores the information about its epoch. +type RegionEpoch struct { + ConfVer int64 `json:"conf_ver"` + Version int64 `json:"version"` +} + +// RegionPeerStat stores one field `DownSec` which indicates how long it's down than `RegionPeer`. +type RegionPeerStat struct { + RegionPeer + DownSec int64 `json:"down_seconds"` +} + +// RegionInfo stores the information of one region. +type RegionInfo struct { + ID int64 `json:"id"` + StartKey string `json:"start_key"` + EndKey string `json:"end_key"` + Epoch RegionEpoch `json:"epoch"` + Peers []RegionPeer `json:"peers"` + Leader RegionPeer `json:"leader"` + DownPeers []RegionPeerStat `json:"down_peers"` + PendingPeers []RegionPeer `json:"pending_peers"` + WrittenBytes int64 `json:"written_bytes"` + ReadBytes int64 `json:"read_bytes"` + ApproximateSize int64 `json:"approximate_size"` + ApproximateKeys int64 `json:"approximate_keys"` +} + +// RegionsInfo stores the information of regions. +type RegionsInfo struct { + Count int64 `json:"count"` + Regions []RegionInfo `json:"regions"` +} + +// GetRegionsInfo gets the region information of current store by using PD's api. +func (h *Helper) GetRegionsInfo() (*RegionsInfo, error) { + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.WithStack(errors.New("not implemented")) + } + pdHosts := etcd.EtcdAddrs() + if len(pdHosts) == 0 { + return nil, errors.New("pd unavailable") + } + req, err := http.NewRequest("GET", protocol+pdHosts[0]+pdapi.Regions, nil) + if err != nil { + return nil, errors.Trace(err) + } + timeout, cancelFunc := context.WithTimeout(context.Background(), 5*time.Second) + resp, err := http.DefaultClient.Do(req.WithContext(timeout)) + defer cancelFunc() + if err != nil { + return nil, errors.Trace(err) + } + defer func() { + err = resp.Body.Close() + if err != nil { + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) + } + }() + var regionsInfo RegionsInfo + err = json.NewDecoder(resp.Body).Decode(®ionsInfo) + if err != nil { + return nil, errors.Trace(err) + } + return ®ionsInfo, nil +} + +// StoresStat stores all information get from PD's api. +type StoresStat struct { + Count int `json:"count"` + Stores []StoreStat `json:"stores"` +} + +// StoreStat stores information of one store. +type StoreStat struct { + Store StoreBaseStat `json:"store"` + Status StoreDetailStat `json:"status"` +} + +// StoreBaseStat stores the basic information of one store. +type StoreBaseStat struct { + ID int64 `json:"id"` + Address string `json:"address"` + State int64 `json:"state"` + StateName string `json:"state_name"` + Version string `json:"version"` + Labels []StoreLabel `json:"labels"` +} + +// StoreLabel stores the information of one store label. +type StoreLabel struct { + Key string `json:"key"` + Value string `json:"value"` +} + +// StoreDetailStat stores the detail information of one store. +type StoreDetailStat struct { + Capacity string `json:"capacity"` + Available string `json:"available"` + LeaderCount int64 `json:"leader_count"` + LeaderWeight int64 `json:"leader_weight"` + LeaderScore int64 `json:"leader_score"` + LeaderSize int64 `json:"leader_size"` + RegionCount int64 `json:"region_count"` + RegionWeight int64 `json:"region_weight"` + RegionScore int64 `json:"region_score"` + RegionSize int64 `json:"region_size"` + StartTs time.Time `json:"start_ts"` + LastHeartbeatTs time.Time `json:"last_heartbeat_ts"` + Uptime string `json:"uptime"` +} + +// GetStoresStat gets the TiKV store information by accessing PD's api. +func (h *Helper) GetStoresStat() (*StoresStat, error) { + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.WithStack(errors.New("not implemented")) + } + pdHosts := etcd.EtcdAddrs() + if len(pdHosts) == 0 { + return nil, errors.New("pd unavailable") + } + req, err := http.NewRequest("GET", protocol+pdHosts[0]+pdapi.Stores, nil) + if err != nil { + return nil, errors.Trace(err) + } + timeout, cancelFunc := context.WithTimeout(context.Background(), 50*time.Millisecond) + resp, err := http.DefaultClient.Do(req.WithContext(timeout)) + defer cancelFunc() + if err != nil { + return nil, errors.Trace(err) + } + defer func() { + err = resp.Body.Close() + if err != nil { + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) + } + }() + var storesStat StoresStat + err = json.NewDecoder(resp.Body).Decode(&storesStat) + if err != nil { + return nil, errors.Trace(err) + } + return &storesStat, nil +} diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index db98ede717a00..30879e50c278e 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -82,9 +82,33 @@ func (s *HelperTestSuite) TestHotRegion(c *C) { c.Assert(fmt.Sprintf("%v", regionMetric), Equals, "map[1:{100 1 0}]") } +func (s *HelperTestSuite) TestTiKVRegionsInfo(c *C) { + h := helper.Helper{ + Store: s.store, + RegionCache: s.store.GetRegionCache(), + } + regionsInfo, err := h.GetRegionsInfo() + c.Assert(err, IsNil, Commentf("err: %+v", err)) + c.Assert(fmt.Sprintf("%v", regionsInfo), Equals, "&{1 [{1 test testtest {1 1} [{2 1 false}] {2 1 false} [] [] 100 1000 500 200}]}") +} + +func (s *HelperTestSuite) TestTiKVStoresStat(c *C) { + h := helper.Helper{ + Store: s.store, + RegionCache: s.store.GetRegionCache(), + } + stat, err := h.GetStoresStat() + c.Assert(err, IsNil, Commentf("err: %+v", err)) + data, err := json.Marshal(stat) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%s", data), Equals, "{\"count\":1,\"stores\":[{\"store\":{\"id\":1,\"address\":\"127.0.0.1:20160\",\"state\":0,\"state_name\":\"Up\",\"version\":\"3.0.0-beta\",\"labels\":[{\"key\":\"test\",\"value\":\"test\"}]},\"status\":{\"capacity\":\"60 GiB\",\"available\":\"100 GiB\",\"leader_count\":10,\"leader_weight\":1,\"leader_score\":1000,\"leader_size\":1000,\"region_count\":200,\"region_weight\":1,\"region_score\":1000,\"region_size\":1000,\"start_ts\":\"2019-04-23T19:30:30+08:00\",\"last_heartbeat_ts\":\"2019-04-23T19:31:30+08:00\",\"uptime\":\"1h30m\"}}]}") +} + func (s *HelperTestSuite) mockPDHTTPServer(c *C) { router := mux.NewRouter() - router.HandleFunc("/pd/api/v1/hotspot/regions/read", s.mockHotRegionResponse) + router.HandleFunc(pdapi.HotRead, s.mockHotRegionResponse) + router.HandleFunc(pdapi.Regions, s.mockTiKVRegionsInfoResponse) + router.HandleFunc(pdapi.Stores, s.mockStoreStatResponse) serverMux := http.NewServeMux() serverMux.Handle("/", router) server := &http.Server{Addr: "127.0.0.1:10100", Handler: serverMux} @@ -118,3 +142,104 @@ func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http } } + +func (s *HelperTestSuite) mockTiKVRegionsInfoResponse(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + resp := helper.RegionsInfo{ + Count: 1, + Regions: []helper.RegionInfo{ + { + ID: 1, + StartKey: "test", + EndKey: "testtest", + Epoch: helper.RegionEpoch{ + ConfVer: 1, + Version: 1, + }, + Peers: []helper.RegionPeer{ + { + ID: 2, + StoreID: 1, + IsLearner: false, + }, + }, + Leader: helper.RegionPeer{ + ID: 2, + StoreID: 1, + IsLearner: false, + }, + DownPeers: nil, + PendingPeers: nil, + WrittenBytes: 100, + ReadBytes: 1000, + ApproximateKeys: 200, + ApproximateSize: 500, + }, + }, + } + data, err := json.MarshalIndent(resp, "", " ") + if err != nil { + log.Panic("json marshal failed", zap.Error(err)) + } + _, err = w.Write(data) + if err != nil { + log.Panic("write http response failed", zap.Error(err)) + } +} + +func (s *HelperTestSuite) mockStoreStatResponse(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + startTs, err := time.Parse(time.RFC3339, "2019-04-23T19:30:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + lastHeartbeatTs, err := time.Parse(time.RFC3339, "2019-04-23T19:31:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + storesStat := helper.StoresStat{ + Count: 1, + Stores: []helper.StoreStat{ + { + Store: helper.StoreBaseStat{ + ID: 1, + Address: "127.0.0.1:20160", + State: 0, + StateName: "Up", + Version: "3.0.0-beta", + Labels: []helper.StoreLabel{ + { + Key: "test", + Value: "test", + }, + }, + }, + Status: helper.StoreDetailStat{ + Capacity: "60 GiB", + Available: "100 GiB", + LeaderCount: 10, + LeaderWeight: 1, + LeaderScore: 1000, + LeaderSize: 1000, + RegionCount: 200, + RegionWeight: 1, + RegionScore: 1000, + RegionSize: 1000, + StartTs: startTs, + LastHeartbeatTs: lastHeartbeatTs, + Uptime: "1h30m", + }, + }, + }, + } + data, err := json.MarshalIndent(storesStat, "", " ") + if err != nil { + log.Panic("json marshal failed", zap.Error(err)) + } + _, err = w.Write(data) + if err != nil { + log.Panic("write http response failed", zap.Error(err)) + } +} diff --git a/tidb-server/main.go b/tidb-server/main.go index 73ee8610c0fca..98c9463b901df 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -20,7 +20,6 @@ import ( "os" "runtime" "strconv" - "strings" "sync/atomic" "time" @@ -29,7 +28,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - pd "github.com/pingcap/pd/client" + "github.com/pingcap/pd/client" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -148,7 +147,10 @@ func main() { registerMetrics() configWarning := loadConfig() overrideConfig() - validateConfig() + if err := cfg.Valid(); err != nil { + fmt.Fprintln(os.Stderr, "invalid config", err) + os.Exit(1) + } if *configCheck { fmt.Println("config check successful") os.Exit(0) @@ -289,10 +291,6 @@ func parseDuration(lease string) time.Duration { return dur } -func hasRootPrivilege() bool { - return os.Geteuid() == 0 -} - func flagBoolean(name string, defaultVal bool, usage string) *bool { if !defaultVal { // Fix #4125, golang do not print default false value in usage, so we append it. @@ -305,6 +303,9 @@ func flagBoolean(name string, defaultVal bool, usage string) *bool { func loadConfig() string { cfg = config.GetGlobalConfig() if *configPath != "" { + // Not all config items are supported now. + config.SetConfReloader(*configPath, reloadConfig, hotReloadConfigItems...) + err := cfg.Load(*configPath) // This block is to accommodate an interim situation where strict config checking // is not the default behavior of TiDB. The warning message must be deferred until @@ -318,6 +319,37 @@ func loadConfig() string { return "" } +// hotReloadConfigItems lists all config items which support hot-reload. +var hotReloadConfigItems = []string{"Performance.MaxProcs", "Performance.MaxMemory", "Performance.CrossJoin", + "Performance.FeedbackProbability", "Performance.QueryFeedbackLimit", "Performance.PseudoEstimateRatio", + "OOMAction", "MemQuotaQuery"} + +func reloadConfig(nc, c *config.Config) { + // Just a part of config items need to be reload explicitly. + // Some of them like OOMAction are always used by getting from global config directly + // like config.GetGlobalConfig().OOMAction. + // These config items will become available naturally after the global config pointer + // is updated in function ReloadGlobalConfig. + if nc.Performance.MaxProcs != c.Performance.MaxProcs { + runtime.GOMAXPROCS(int(nc.Performance.MaxProcs)) + } + if nc.Performance.MaxMemory != c.Performance.MaxMemory { + plannercore.PreparedPlanCacheMaxMemory.Store(nc.Performance.MaxMemory) + } + if nc.Performance.CrossJoin != c.Performance.CrossJoin { + plannercore.AllowCartesianProduct.Store(nc.Performance.CrossJoin) + } + if nc.Performance.FeedbackProbability != c.Performance.FeedbackProbability { + statistics.FeedbackProbability.Store(nc.Performance.FeedbackProbability) + } + if nc.Performance.QueryFeedbackLimit != c.Performance.QueryFeedbackLimit { + handle.MaxQueryFeedbackCount.Store(int64(nc.Performance.QueryFeedbackLimit)) + } + if nc.Performance.PseudoEstimateRatio != c.Performance.PseudoEstimateRatio { + statistics.RatioOfPseudoEstimate.Store(nc.Performance.PseudoEstimateRatio) + } +} + func overrideConfig() { actualFlags := make(map[string]bool) flag.Visit(func(f *flag.Flag) { @@ -410,53 +442,6 @@ func overrideConfig() { } } -func validateConfig() { - if cfg.Security.SkipGrantTable && !hasRootPrivilege() { - log.Error("TiDB run with skip-grant-table need root privilege.") - os.Exit(-1) - } - if _, ok := config.ValidStorage[cfg.Store]; !ok { - nameList := make([]string, 0, len(config.ValidStorage)) - for k, v := range config.ValidStorage { - if v { - nameList = append(nameList, k) - } - } - log.Error("validate config", zap.Strings("valid storages", nameList)) - os.Exit(-1) - } - if cfg.Store == "mocktikv" && !cfg.RunDDL { - log.Error("can't disable DDL on mocktikv") - os.Exit(-1) - } - if cfg.Log.File.MaxSize > config.MaxLogFileSize { - log.Error("validate config", zap.Int("log max-size should not be larger than", config.MaxLogFileSize)) - os.Exit(-1) - } - cfg.OOMAction = strings.ToLower(cfg.OOMAction) - - // lower_case_table_names is allowed to be 0, 1, 2 - if cfg.LowerCaseTableNames < 0 || cfg.LowerCaseTableNames > 2 { - log.Error("lower-case-table-names should be 0 or 1 or 2.") - os.Exit(-1) - } - - if cfg.TxnLocalLatches.Enabled && cfg.TxnLocalLatches.Capacity == 0 { - log.Error("txn-local-latches.capacity can not be 0") - os.Exit(-1) - } - - // For tikvclient. - if cfg.TiKVClient.GrpcConnectionCount == 0 { - log.Error("grpc-connection-count should be greater than 0") - os.Exit(-1) - } - if cfg.TiKVClient.MaxTxnTimeUse == 0 { - log.Error("max-txn-time-use should be greater than 0") - os.Exit(-1) - } -} - func setGlobalVars() { ddlLeaseDuration := parseDuration(cfg.Lease) session.SetSchemaLease(ddlLeaseDuration) @@ -464,14 +449,14 @@ func setGlobalVars() { statsLeaseDuration := parseDuration(cfg.Performance.StatsLease) session.SetStatsLease(statsLeaseDuration) domain.RunAutoAnalyze = cfg.Performance.RunAutoAnalyze - statistics.FeedbackProbability = cfg.Performance.FeedbackProbability - handle.MaxQueryFeedbackCount = int(cfg.Performance.QueryFeedbackLimit) - statistics.RatioOfPseudoEstimate = cfg.Performance.PseudoEstimateRatio + statistics.FeedbackProbability.Store(cfg.Performance.FeedbackProbability) + handle.MaxQueryFeedbackCount.Store(int64(cfg.Performance.QueryFeedbackLimit)) + statistics.RatioOfPseudoEstimate.Store(cfg.Performance.PseudoEstimateRatio) ddl.RunWorker = cfg.RunDDL if cfg.SplitTable { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) } - plannercore.AllowCartesianProduct = cfg.Performance.CrossJoin + plannercore.AllowCartesianProduct.Store(cfg.Performance.CrossJoin) privileges.SkipWithGrant = cfg.Security.SkipGrantTable priority := mysql.Str2Priority(cfg.Performance.ForcePriority) @@ -495,11 +480,11 @@ func setGlobalVars() { if plannercore.PreparedPlanCacheMemoryGuardRatio < 0.0 || plannercore.PreparedPlanCacheMemoryGuardRatio > 1.0 { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 } - plannercore.PreparedPlanCacheMaxMemory = cfg.Performance.MaxMemory + plannercore.PreparedPlanCacheMaxMemory.Store(cfg.Performance.MaxMemory) total, err := memory.MemTotal() terror.MustNil(err) - if plannercore.PreparedPlanCacheMaxMemory > total || plannercore.PreparedPlanCacheMaxMemory <= 0 { - plannercore.PreparedPlanCacheMaxMemory = total + if plannercore.PreparedPlanCacheMaxMemory.Load() > total || plannercore.PreparedPlanCacheMaxMemory.Load() <= 0 { + plannercore.PreparedPlanCacheMaxMemory.Store(total) } } diff --git a/util/logutil/log.go b/util/logutil/log.go index 691b662eea63f..40061f20be11a 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -22,6 +22,7 @@ import ( "runtime" "sort" "strings" + "time" "github.com/pingcap/errors" zaplog "github.com/pingcap/log" @@ -208,7 +209,9 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { const ( // SlowLogTimeFormat is the time format for slow log. - SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" + SlowLogTimeFormat = time.RFC3339Nano + // OldSlowLogTimeFormat is the first version of the the time format for slow log, This is use for compatibility. + OldSlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" ) type slowLogFormatter struct{} diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 1cf4d935ff4d0..436784f627245 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -13,8 +13,10 @@ package pdapi -// HotRead / HotWrite is the pd apis to get the corresponding hot region information. +// The following constants are the APIs of PD server. const ( HotRead = "/pd/api/v1/hotspot/regions/read" - HotWrite = "/pd/api/v1/hotspot/regions/read" + HotWrite = "/pd/api/v1/hotspot/regions/write" + Regions = "/pd/api/v1/regions" + Stores = "/pd/api/v1/stores" ) diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 6aabbfda3f841..671ea328a0c5b 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -146,7 +146,7 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex ) res := &DetachRangeResult{} - accessConds, filterConds, newConditions, emptyRange := extractEqAndInCondition(sctx, conditions, cols, lengths) + accessConds, filterConds, newConditions, emptyRange := ExtractEqAndInCondition(sctx, conditions, cols, lengths) if emptyRange { return res, nil } @@ -156,8 +156,6 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex break } } - // We should remove all accessConds, so that they will not be added to filter conditions. - newConditions = removeAccessConditions(newConditions, accessConds) eqOrInCount := len(accessConds) res.EqCondCount = eqCount res.EqOrInCount = eqOrInCount @@ -197,7 +195,13 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex return res, err } -func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, +// ExtractEqAndInCondition will split the given condition into three parts by the information of index columns and their lengths. +// accesses: The condition will be used to build range. +// filters: filters is the part that some access conditions need to be evaluate again since it's only the prefix part of char column. +// newConditions: We'll simplify the given conditions if there're multiple in conditions or eq conditions on the same column. +// e.g. if there're a in (1, 2, 3) and a in (2, 3, 4). This two will be combined to a in (2, 3) and pushed to newConditions. +// bool: indicate whether there's nil range when merging eq and in conditions. +func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int) ([]expression.Expression, []expression.Expression, []expression.Expression, bool) { var filters []expression.Expression rb := builder{sc: sctx.GetSessionVars().StmtCtx} @@ -246,6 +250,8 @@ func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex filters = append(filters, cond) } } + // We should remove all accessConds, so that they will not be added to filter conditions. + newConditions = removeAccessConditions(newConditions, accesses) return accesses, filters, newConditions, false } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 5722c7a17e43e..015f6ea447eb7 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -243,7 +243,7 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []point, tp *t } // buildColumnRange builds range from CNF conditions. -func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, tableRange bool) (ranges []*Range, err error) { +func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, tableRange bool, colLen int) (ranges []*Range, err error) { rb := builder{sc: sc} rangePoints := fullRange for _, cond := range accessConditions { @@ -261,20 +261,30 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat if err != nil { return nil, errors.Trace(err) } + if colLen != types.UnspecifiedLength { + for _, ran := range ranges { + if fixRangeDatum(&ran.LowVal[0], colLen, tp) { + ran.LowExclude = false + } + if fixRangeDatum(&ran.HighVal[0], colLen, tp) { + ran.HighExclude = false + } + } + } return ranges, nil } // BuildTableRange builds range of PK column for PhysicalTableScan. func BuildTableRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { - return buildColumnRange(accessConditions, sc, tp, true) + return buildColumnRange(accessConditions, sc, tp, true, types.UnspecifiedLength) } // BuildColumnRange builds range from access conditions for general columns. -func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { +func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, colLen int) ([]*Range, error) { if len(conds) == 0 { return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}, nil } - return buildColumnRange(conds, sc, tp, false) + return buildColumnRange(conds, sc, tp, false, colLen) } // buildCNFIndexRange builds the range for index where the top layer is CNF. @@ -485,7 +495,7 @@ func newFieldType(tp *types.FieldType) *types.FieldType { // 1. 'expr' must be either 'EQUAL' or 'IN' function. // 2. 'points' should not be empty. func points2EqOrInCond(ctx sessionctx.Context, points []point, expr expression.Expression) expression.Expression { - // len(points) cannot be 0 here, since we impose early termination in extractEqAndInCondition + // len(points) cannot be 0 here, since we impose early termination in ExtractEqAndInCondition sf, _ := expr.(*expression.ScalarFunction) // Constant and Column args should have same RetType, simply get from first arg retType := sf.GetArgs()[0].GetType() diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index c967f70178f62..5d17fdb91e31e 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -956,7 +957,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(col, NotNil) conds = ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID) c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr)) - result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType) + result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType, types.UnspecifiedLength) c.Assert(err, IsNil) got := fmt.Sprintf("%v", result) c.Assert(got, Equals, tt.resultStr, Commentf("different for expr %s, col: %v", tt.exprStr, col)) diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 2eaf3302cbebb..d9e28ec531194 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -208,7 +208,12 @@ func (tk *TestKit) ExecToErr(sql string, args ...interface{}) error { // ResultSetToResult converts sqlexec.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. func (tk *TestKit) ResultSetToResult(rs sqlexec.RecordSet, comment check.CommentInterface) *Result { - rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) + return tk.ResultSetToResultWithCtx(context.Background(), rs, comment) +} + +// ResultSetToResultWithCtx converts sqlexec.RecordSet to testkit.Result. +func (tk *TestKit) ResultSetToResultWithCtx(ctx context.Context, rs sqlexec.RecordSet, comment check.CommentInterface) *Result { + rows, err := session.GetRows4Test(ctx, tk.Se, rs) tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) err = rs.Close() tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment)