Skip to content

Commit

Permalink
Merge branch 'master' into dev/add_schema_ver_in_cop
Browse files Browse the repository at this point in the history
  • Loading branch information
lzmhhh123 authored Jan 7, 2020
2 parents 4a65302 + 1f103e6 commit 96296a1
Show file tree
Hide file tree
Showing 124 changed files with 2,192 additions and 540 deletions.
40 changes: 39 additions & 1 deletion bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ func (s *testSuite) cleanBindingEnv(tk *testkit.TestKit) {

func (s *testSuite) TestBindParse(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("create table t(i int)")
tk.MustExec("create index index_t on t(i)")
Expand Down Expand Up @@ -485,7 +486,7 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) {
rows := tk.MustQuery("show global bindings").Rows()
c.Assert(len(rows), Equals, 1)
c.Assert(rows[0][0], Equals, "select count ( ? ) from t where a > ?")
c.Assert(rows[0][1], Equals, "SELECT /*+ USE_INDEX(@`sel_1` `test`.`t` ), STREAM_AGG(@`sel_1`)*/ COUNT(1) FROM `test`.`t` WHERE `a`>10")
c.Assert(rows[0][1], Equals, "SELECT /*+ USE_INDEX(@`sel_1` `test`.`t` ), STREAM_AGG(@`sel_1`)*/ COUNT(1) FROM `t` WHERE `a`>10")
}

func (s *testSuite) TestUseMultiplyBindings(c *C) {
Expand Down Expand Up @@ -607,6 +608,31 @@ func (s *testSuite) TestDefaultSessionVars(c *C) {
"tidb_use_plan_baselines on"))
}

func (s *testSuite) TestDuplicateBindings(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec("create global binding for select * from t using select * from t use index(idx);")
rows := tk.MustQuery("show global bindings").Rows()
c.Assert(len(rows), Equals, 1)
createTime := rows[0][4]
tk.MustExec("create global binding for select * from t using select * from t use index(idx);")
rows = tk.MustQuery("show global bindings").Rows()
c.Assert(len(rows), Equals, 1)
c.Assert(createTime, Equals, rows[0][4])

tk.MustExec("create session binding for select * from t using select * from t use index(idx);")
rows = tk.MustQuery("show session bindings").Rows()
c.Assert(len(rows), Equals, 1)
createTime = rows[0][4]
tk.MustExec("create session binding for select * from t using select * from t use index(idx);")
rows = tk.MustQuery("show session bindings").Rows()
c.Assert(len(rows), Equals, 1)
c.Assert(createTime, Equals, rows[0][4])
}

func (s *testSuite) TestDefaultDB(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
Expand All @@ -629,3 +655,15 @@ func (s *testSuite) TestDefaultDB(c *C) {
tk.MustExec("drop session binding for select * from test.t")
tk.MustQuery("show session bindings").Check(testkit.Rows())
}

func (s *testSuite) TestOutdatedInfoSchema(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec("create global binding for select * from t using select * from t use index(idx)")
c.Assert(s.domain.BindHandle().Update(false), IsNil)
tk.MustExec("truncate table mysql.bind_info")
tk.MustExec("create global binding for select * from t using select * from t use index(idx)")
}
6 changes: 2 additions & 4 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,10 @@
package bindinfo

import (
"context"
"time"
"unsafe"

"github.com/pingcap/parser"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -105,7 +103,7 @@ func (br *BindRecord) FindBinding(hint string) *Binding {
return nil
}

func (br *BindRecord) prepareHints(sctx sessionctx.Context, is infoschema.InfoSchema) error {
func (br *BindRecord) prepareHints(sctx sessionctx.Context) error {
p := parser.New()
for i, bind := range br.Bindings {
if bind.Hint != nil || bind.id != "" || bind.Status == deleted {
Expand All @@ -115,7 +113,7 @@ func (br *BindRecord) prepareHints(sctx sessionctx.Context, is infoschema.InfoSc
if err != nil {
return err
}
hints, err := GenHintsFromSQL(context.TODO(), sctx, stmtNode, is)
hints, err := getHintsForSQL(sctx, bind.BindSQL)
if err != nil {
return err
}
Expand Down
78 changes: 43 additions & 35 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
Expand Down Expand Up @@ -110,9 +109,9 @@ func NewBindHandle(ctx sessionctx.Context) *BindHandle {
}
handle.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate))
handle.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error {
// We do not need the first two parameters because they are only use to generate hint,
// We do not need the first parameter because it is only use to generate hint,
// and we already have the hint.
return handle.AddBindRecord(nil, nil, record)
return handle.AddBindRecord(nil, record)
}
return handle
}
Expand Down Expand Up @@ -170,14 +169,14 @@ func (h *BindHandle) Update(fullLoad bool) (err error) {
}

// AddBindRecord adds a BindRecord to the storage and BindRecord to the cache.
func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSchema, record *BindRecord) (err error) {
err = record.prepareHints(sctx, is)
func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) (err error) {
err = record.prepareHints(sctx)
if err != nil {
return err
}

br := h.GetBindRecord(parser.DigestNormalized(record.OriginalSQL), record.OriginalSQL, record.Db)
var duplicateBinding string
var duplicateBinding *Binding
if br != nil {
binding := br.FindBinding(record.Bindings[0].id)
if binding != nil {
Expand All @@ -186,7 +185,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc
return nil
}
// Otherwise, we need to remove it before insert.
duplicateBinding = binding.BindSQL
duplicateBinding = binding
}
}

Expand Down Expand Up @@ -219,20 +218,26 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc
h.bindInfo.Unlock()
}()

if duplicateBinding != "" {
_, err = exec.Execute(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db, duplicateBinding))
txn, err1 := h.sctx.Context.Txn(true)
if err1 != nil {
return err1
}

if duplicateBinding != nil {
_, err = exec.Execute(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db, duplicateBinding.BindSQL))
if err != nil {
return err
}
}

txn, err1 := h.sctx.Context.Txn(true)
if err1 != nil {
return err1
}
now := types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3)
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3)
record.Bindings[i].UpdateTime = record.Bindings[0].CreateTime
if duplicateBinding != nil {
record.Bindings[i].CreateTime = duplicateBinding.CreateTime
} else {
record.Bindings[i].CreateTime = now
}
record.Bindings[i].UpdateTime = now

// insert the BindRecord to the storage.
_, err = exec.Execute(context.TODO(), h.insertBindInfoSQL(record.OriginalSQL, record.Db, record.Bindings[i]))
Expand Down Expand Up @@ -387,13 +392,8 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) {
hash := parser.DigestNormalized(bindRecord.OriginalSQL)
h.sctx.Lock()
defer h.sctx.Unlock()
err := h.sctx.RefreshTxnCtx(context.TODO())
if err != nil {
return "", nil, err
}
h.sctx.GetSessionVars().StmtCtx.TimeZone = h.sctx.GetSessionVars().TimeZone
h.sctx.GetSessionVars().CurrentDB = bindRecord.Db
err = bindRecord.prepareHints(h.sctx.Context, h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema))
err := bindRecord.prepareHints(h.sctx.Context)
return hash, bindRecord, err
}

Expand Down Expand Up @@ -518,10 +518,6 @@ func (h *BindHandle) logicalDeleteBindInfoSQL(originalSQL, db string, updateTs t
return sql + fmt.Sprintf(` and bind_sql = %s`, expression.Quote(bindingSQL))
}

// GenHintsFromSQL is used to generate hints from SQL.
// It is used to avoid the circle dependence with planner package.
var GenHintsFromSQL func(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (string, error)

// CaptureBaselines is used to automatically capture plan baselines.
func (h *BindHandle) CaptureBaselines() {
parser4Capture := parser.New()
Expand All @@ -537,12 +533,8 @@ func (h *BindHandle) CaptureBaselines() {
continue
}
h.sctx.Lock()
err = h.sctx.RefreshTxnCtx(context.TODO())
var hints string
if err == nil {
h.sctx.GetSessionVars().CurrentDB = schemas[i]
hints, err = GenHintsFromSQL(context.TODO(), h.sctx.Context, stmt, h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema))
}
h.sctx.GetSessionVars().CurrentDB = schemas[i]
hints, err := getHintsForSQL(h.sctx.Context, sqls[i])
h.sctx.Unlock()
if err != nil {
logutil.BgLogger().Info("generate hints failed", zap.String("SQL", sqls[i]), zap.Error(err))
Expand All @@ -561,14 +553,31 @@ func (h *BindHandle) CaptureBaselines() {
Charset: charset,
Collation: collation,
}
// We don't need to pass the `sctx` and `is` because they are used to generate hints and we already filled hints in.
err = h.AddBindRecord(nil, nil, &BindRecord{OriginalSQL: normalizedSQL, Db: schemas[i], Bindings: []Binding{binding}})
// We don't need to pass the `sctx` because they are used to generate hints and we already filled hints in.
err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: schemas[i], Bindings: []Binding{binding}})
if err != nil {
logutil.BgLogger().Info("capture baseline failed", zap.String("SQL", sqls[i]), zap.Error(err))
}
}
}

func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
oriVals := sctx.GetSessionVars().UsePlanBaselines
sctx.GetSessionVars().UsePlanBaselines = false
recordSets, err := sctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql))
sctx.GetSessionVars().UsePlanBaselines = oriVals
defer terror.Log(recordSets[0].Close())
if err != nil {
return "", err
}
chk := recordSets[0].NewChunk()
err = recordSets[0].Next(context.TODO(), chk)
if err != nil {
return "", err
}
return chk.GetRow(0).GetString(0), nil
}

// GenerateBindSQL generates binding sqls from stmt node and plan hints.
func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string) string {
// If would be nil for very simple cases such as point get, we do not need to evolve for them.
Expand Down Expand Up @@ -749,7 +758,6 @@ func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan
}
terror.Call(recordSets[0].Close)
resultChan <- err
return
}

// HandleEvolvePlanTask tries to evolve one plan task.
Expand Down Expand Up @@ -789,7 +797,7 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context) error {
} else {
binding.Status = Using
}
return h.AddBindRecord(sctx, sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema), &BindRecord{OriginalSQL: originalSQL, Db: db, Bindings: []Binding{binding}})
return h.AddBindRecord(sctx, &BindRecord{OriginalSQL: originalSQL, Db: db, Bindings: []Binding{binding}})
}

// Clear resets the bind handle. It is used for test.
Expand Down
31 changes: 22 additions & 9 deletions bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (

"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -48,18 +47,32 @@ func (h *SessionHandle) appendBindRecord(hash string, meta *BindRecord) {
}

// AddBindRecord new a BindRecord with BindMeta, add it to the cache.
func (h *SessionHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSchema, record *BindRecord) error {
func (h *SessionHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) error {
err := record.prepareHints(sctx)
if err != nil {
return err
}
br := h.GetBindRecord(record.OriginalSQL, record.Db)
var duplicateBinding *Binding
if br != nil {
binding := br.FindBinding(record.Bindings[0].id)
if binding != nil {
duplicateBinding = binding
}
}
now := types.NewTime(types.FromGoTime(time.Now().In(sctx.GetSessionVars().StmtCtx.TimeZone)), mysql.TypeTimestamp, 3)
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, 3)
record.Bindings[i].UpdateTime = record.Bindings[i].CreateTime
if duplicateBinding != nil {
record.Bindings[i].CreateTime = duplicateBinding.CreateTime
} else {
record.Bindings[i].CreateTime = now
}
record.Bindings[i].UpdateTime = now
}

err := record.prepareHints(sctx, is)
// update the BindMeta to the cache.
if err == nil {
h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record)
}
return err
h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record)
return nil
}

// DropBindRecord drops a BindRecord in the cache.
Expand Down
6 changes: 3 additions & 3 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -465,9 +465,9 @@ begin;
insert tb values ('1');
explain select * from ta where a = 1;
id count task operator info
Selection_5 8000.00 root eq(cast(test.ta.a), 1)
└─TableReader_7 10000.00 root data:TableScan_6
└─TableScan_6 10000.00 cop[tikv] table:ta, range:[-inf,+inf], keep order:false, stats:pseudo
TableReader_7 8000.00 root data:Selection_6
└─Selection_6 8000.00 cop[tikv] eq(cast(test.ta.a), 1)
└─TableScan_5 10000.00 cop[tikv] table:ta, range:[-inf,+inf], keep order:false, stats:pseudo
rollback;
drop table if exists t1, t2;
create table t1(a int, b int, c int, primary key(a, b));
Expand Down
1 change: 1 addition & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,7 @@ type Performance struct {
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
PProfSQLCPU bool `toml:"pprof-sql-cpu" json:"pprof-sql-cpu"`
}

// PlanCache is the PlanCache section of the config.
Expand Down
3 changes: 3 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,9 @@ func adjustColumnInfoInDropColumn(tblInfo *model.TableInfo, offset int) {
oldCols[i].Offset = i - 1
}
oldCols[offset].Offset = len(oldCols) - 1
// For expression index, we drop hidden columns and index simultaneously.
// So we need to change the offset of expression index.
offsetChanged[offset] = len(oldCols) - 1
// Update index column offset info.
// TODO: There may be some corner cases for index column offsets, we may check this later.
for _, idx := range tblInfo.Indices {
Expand Down
10 changes: 10 additions & 0 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -773,6 +773,16 @@ func (s *testStateChangeSuite) TestParallelAlterAddIndex(c *C) {
s.testControlParallelExecSQL(c, sql1, sql2, f)
}

func (s *testStateChangeSuite) TestParallelAlterAddExpressionIndex(c *C) {
sql1 := "ALTER TABLE t add index expr_index_b((b+1));"
sql2 := "CREATE INDEX expr_index_b ON t ((c+1));"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist expr_index_b")
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}

func (s *testStateChangeSuite) TestParallelAddPrimaryKey(c *C) {
sql1 := "ALTER TABLE t add primary key index_b(b);"
sql2 := "ALTER TABLE t add primary key index_b(c);"
Expand Down
Loading

0 comments on commit 96296a1

Please sign in to comment.