Skip to content

Commit

Permalink
Merge branch 'release-4.0' into release-4.0-87a1dbd4a1d3
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Jun 3, 2020
2 parents 463cecf + 13522b9 commit bd9113b
Show file tree
Hide file tree
Showing 30 changed files with 360 additions and 90 deletions.
60 changes: 57 additions & 3 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,8 +120,9 @@ func (s *testSuite) TestBindParse(c *C) {
status := "using"
charset := "utf8mb4"
collation := "utf8mb4_bin"
sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s')`,
originSQL, bindSQL, defaultDb, status, charset, collation)
source := bindinfo.Manual
sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation,source) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s', '%s')`,
originSQL, bindSQL, defaultDb, status, charset, collation, source)
tk.MustExec(sql)
bindHandle := bindinfo.NewBindHandle(tk.Se)
err := bindHandle.Update(true)
Expand Down Expand Up @@ -822,7 +823,8 @@ func (s *testSuite) TestEvolveInvalidBindings(c *C) {
tk.MustExec("create table t(a int, b int, index idx_a(a))")
tk.MustExec("create global binding for select * from t where a > 10 using select /*+ USE_INDEX(t) */ * from t where a > 10")
// Manufacture a rejected binding by hacking mysql.bind_info.
tk.MustExec("insert into mysql.bind_info values('select * from t where a > ?', 'select /*+ USE_INDEX(t,idx_a) */ * from t where a > 10', 'test', 'rejected', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '')")
tk.MustExec("insert into mysql.bind_info values('select * from t where a > ?', 'select /*+ USE_INDEX(t,idx_a) */ * from t where a > 10', 'test', 'rejected', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" +
bindinfo.Manual + "')")
tk.MustQuery("select bind_sql, status from mysql.bind_info").Sort().Check(testkit.Rows(
"select /*+ USE_INDEX(t) */ * from t where a > 10 using",
"select /*+ USE_INDEX(t,idx_a) */ * from t where a > 10 rejected",
Expand Down Expand Up @@ -1096,3 +1098,55 @@ func (s *testSuite) TestReCreateBindAfterEvolvePlan(c *C) {
tk.MustQuery("select * from t where a >= 4 and b >= 1")
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_b")
}

func (s *testSuite) TestbindingSource(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, index idx_a(a))")

// Test Source for SQL created sql
tk.MustExec("create global binding for select * from t where a > 10 using select * from t ignore index(idx_a) where a > 10")
bindHandle := s.domain.BindHandle()
sql, hash := parser.NormalizeDigest("select * from t where a > ?")
bindData := bindHandle.GetBindRecord(hash, sql, "test")
c.Check(bindData, NotNil)
c.Check(bindData.OriginalSQL, Equals, "select * from t where a > ?")
c.Assert(len(bindData.Bindings), Equals, 1)
bind := bindData.Bindings[0]
c.Assert(bind.Source, Equals, bindinfo.Manual)

// Test Source for evolved sql
tk.MustExec("set @@tidb_evolve_plan_baselines=1")
tk.MustQuery("select * from t where a > 10")
bindHandle.SaveEvolveTasksToStore()
sql, hash = parser.NormalizeDigest("select * from t where a > ?")
bindData = bindHandle.GetBindRecord(hash, sql, "test")
c.Check(bindData, NotNil)
c.Check(bindData.OriginalSQL, Equals, "select * from t where a > ?")
c.Assert(len(bindData.Bindings), Equals, 2)
bind = bindData.Bindings[1]
c.Assert(bind.Source, Equals, bindinfo.Evolve)
tk.MustExec("set @@tidb_evolve_plan_baselines=0")

// Test Source for captured sqls
stmtsummary.StmtSummaryByDigestMap.Clear()
tk.MustExec("set @@tidb_capture_plan_baselines = on")
defer func() {
tk.MustExec("set @@tidb_capture_plan_baselines = off")
}()
tk.MustExec("use test")
c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue)
tk.MustExec("select * from t ignore index(idx_a) where a < 10")
tk.MustExec("select * from t ignore index(idx_a) where a < 10")
tk.MustExec("admin capture bindings")
bindHandle.CaptureBaselines()
sql, hash = parser.NormalizeDigest("select * from t where a < ?")
bindData = bindHandle.GetBindRecord(hash, sql, "test")
c.Check(bindData, NotNil)
c.Check(bindData.OriginalSQL, Equals, "select * from t where a < ?")
c.Assert(len(bindData.Bindings), Equals, 1)
bind = bindData.Bindings[0]
c.Assert(bind.Source, Equals, bindinfo.Capture)
}
7 changes: 7 additions & 0 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,12 @@ const (
// Rejected means that the bind has been rejected after verify process.
// We can retry it after certain time has passed.
Rejected = "rejected"
// Manual indicates the binding is created by SQL like "create binding for ...".
Manual = "manual"
// Capture indicates the binding is captured by TiDB automatically.
Capture = "capture"
// Evolve indicates the binding is evolved by TiDB from old bindings.
Evolve = "evolve"
)

// Binding stores the basic bind hint info.
Expand All @@ -47,6 +53,7 @@ type Binding struct {
Status string
CreateTime types.Time
UpdateTime types.Time
Source string
Charset string
Collation string
// Hint is the parsed hints, it is used to bind hints to stmt node.
Expand Down
7 changes: 5 additions & 2 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) {
lastUpdateTime := h.bindInfo.lastUpdateTime
h.bindInfo.Unlock()

sql := "select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation from mysql.bind_info"
sql := "select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source from mysql.bind_info"
if !fullLoad {
sql += " where update_time > \"" + lastUpdateTime.String() + "\""
}
Expand Down Expand Up @@ -462,6 +462,7 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) {
UpdateTime: row.GetTime(5),
Charset: row.GetString(6),
Collation: row.GetString(7),
Source: row.GetString(8),
}
bindRecord := &BindRecord{
OriginalSQL: row.GetString(0),
Expand Down Expand Up @@ -567,7 +568,7 @@ func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db, bindSQL string) string
}

func (h *BindHandle) insertBindInfoSQL(orignalSQL string, db string, info Binding) string {
return fmt.Sprintf(`INSERT INTO mysql.bind_info VALUES (%s, %s, %s, %s, %s, %s, %s, %s)`,
return fmt.Sprintf(`INSERT INTO mysql.bind_info VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s)`,
expression.Quote(orignalSQL),
expression.Quote(info.BindSQL),
expression.Quote(db),
Expand All @@ -576,6 +577,7 @@ func (h *BindHandle) insertBindInfoSQL(orignalSQL string, db string, info Bindin
expression.Quote(info.UpdateTime.String()),
expression.Quote(info.Charset),
expression.Quote(info.Collation),
expression.Quote(info.Source),
)
}

Expand Down Expand Up @@ -628,6 +630,7 @@ func (h *BindHandle) CaptureBaselines() {
Status: Using,
Charset: charset,
Collation: collation,
Source: Capture,
}
// We don't need to pass the `sctx` because the BindSQL has been validated already.
err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}})
Expand Down
5 changes: 5 additions & 0 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -648,7 +648,12 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R
if ebd, ok := do.store.(tikv.EtcdBackend); ok {
if addrs := ebd.EtcdAddrs(); addrs != nil {
cfg := config.GetGlobalConfig()
// silence etcd warn log, when domain closed, it won't randomly print warn log
// see details at the issue https://github.com/pingcap/tidb/issues/15479
etcdLogCfg := zap.NewProductionConfig()
etcdLogCfg.Level = zap.NewAtomicLevelAt(zap.ErrorLevel)
cli, err := clientv3.New(clientv3.Config{
LogConfig: &etcdLogCfg,
Endpoints: addrs,
AutoSyncInterval: 30 * time.Second,
DialTimeout: 5 * time.Second,
Expand Down
6 changes: 4 additions & 2 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/set"
Expand Down Expand Up @@ -243,8 +244,9 @@ func (e *HashAggExec) Close() error {
partialConcurrency = cap(e.partialWorkers)
finalConcurrency = cap(e.finalWorkers)
}
e.runtimeStats.SetConcurrencyInfo("PartialConcurrency", partialConcurrency)
e.runtimeStats.SetConcurrencyInfo("FinalConcurrency", finalConcurrency)
partialConcurrencyInfo := execdetails.NewConcurrencyInfo("PartialConcurrency", partialConcurrency)
finalConcurrencyInfo := execdetails.NewConcurrencyInfo("FinalConcurrency", finalConcurrency)
e.runtimeStats.SetConcurrencyInfo(partialConcurrencyInfo, finalConcurrencyInfo)
}
return e.baseExecutor.Close()
}
Expand Down
1 change: 1 addition & 0 deletions executor/bind.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ func (e *SQLBindExec) createSQLBind() error {
Charset: e.charset,
Collation: e.collation,
Status: bindinfo.Using,
Source: bindinfo.Manual,
}
record := &bindinfo.BindRecord{
OriginalSQL: e.normdOrigSQL,
Expand Down
3 changes: 2 additions & 1 deletion executor/index_lookup_hash_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
Expand Down Expand Up @@ -295,7 +296,7 @@ func (e *IndexNestedLoopHashJoin) Close() error {
}
if e.runtimeStats != nil {
concurrency := cap(e.joinChkResourceCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
for i := range e.joinChkResourceCh {
close(e.joinChkResourceCh[i])
Expand Down
3 changes: 2 additions & 1 deletion executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/mvmap"
Expand Down Expand Up @@ -681,7 +682,7 @@ func (e *IndexLookUpJoin) Close() error {
e.memTracker = nil
if e.runtimeStats != nil {
concurrency := cap(e.resultCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
return e.baseExecutor.Close()
}
3 changes: 2 additions & 1 deletion executor/index_lookup_merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/stringutil"
Expand Down Expand Up @@ -686,7 +687,7 @@ func (e *IndexLookUpMergeJoin) Close() error {
e.memTracker = nil
if e.runtimeStats != nil {
concurrency := cap(e.resultCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
return e.baseExecutor.Close()
}
7 changes: 5 additions & 2 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/stringutil"
)
Expand Down Expand Up @@ -138,8 +139,10 @@ func (e *HashJoinExec) Close() error {

if e.runtimeStats != nil {
concurrency := cap(e.joiners)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String())
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
if e.rowContainer != nil {
e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String())
}
}
err := e.baseExecutor.Close()
return err
Expand Down
5 changes: 3 additions & 2 deletions executor/projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"go.uber.org/zap"
Expand Down Expand Up @@ -308,9 +309,9 @@ func (e *ProjectionExec) Close() error {
}
if e.runtimeStats != nil {
if e.isUnparallelExec() {
e.runtimeStats.SetConcurrencyInfo("Concurrency", 0)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", 0))
} else {
e.runtimeStats.SetConcurrencyInfo("Concurrency", int(e.numWorkers))
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", int(e.numWorkers)))
}
}
return e.baseExecutor.Close()
Expand Down
1 change: 1 addition & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,7 @@ func (e *ShowExec) fetchShowBind() error {
hint.UpdateTime,
hint.Charset,
hint.Collation,
hint.Source,
})
}
}
Expand Down
3 changes: 2 additions & 1 deletion executor/shuffle.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/spaolacci/murmur3"
"go.uber.org/zap"
Expand Down Expand Up @@ -144,7 +145,7 @@ func (e *ShuffleExec) Close() error {
e.executed = false

if e.runtimeStats != nil {
e.runtimeStats.SetConcurrencyInfo("ShuffleConcurrency", e.concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("ShuffleConcurrency", e.concurrency))
}

err := e.dataSource.Close()
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_string_vec_generated.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

55 changes: 53 additions & 2 deletions expression/builtin_time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,60 @@ func (s *testEvaluatorSuite) TestDate(c *C) {
Input interface{}
Expect interface{}
}{
{"2011-11-11", "2011-11-11"},
{nil, nil},
{"2011-11-11 10:10:10", "2011-11-11"},
// standard format
{"2011-12-13", "2011-12-13"},
{"2011-12-13 10:10:10", "2011-12-13"},
// alternative delimiters, any ASCII punctuation character is a valid delimiter,
// punctuation character is defined by C++ std::ispunct: any graphical character
// that is not alphanumeric.
{"2011\"12\"13", "2011-12-13"},
{"2011#12#13", "2011-12-13"},
{"2011$12$13", "2011-12-13"},
{"2011%12%13", "2011-12-13"},
{"2011&12&13", "2011-12-13"},
{"2011'12'13", "2011-12-13"},
{"2011(12(13", "2011-12-13"},
{"2011)12)13", "2011-12-13"},
{"2011*12*13", "2011-12-13"},
{"2011+12+13", "2011-12-13"},
{"2011,12,13", "2011-12-13"},
{"2011.12.13", "2011-12-13"},
{"2011/12/13", "2011-12-13"},
{"2011:12:13", "2011-12-13"},
{"2011;12;13", "2011-12-13"},
{"2011<12<13", "2011-12-13"},
{"2011=12=13", "2011-12-13"},
{"2011>12>13", "2011-12-13"},
{"2011?12?13", "2011-12-13"},
{"2011@12@13", "2011-12-13"},
{"2011[12[13", "2011-12-13"},
{"2011\\12\\13", "2011-12-13"},
{"2011]12]13", "2011-12-13"},
{"2011^12^13", "2011-12-13"},
{"2011_12_13", "2011-12-13"},
{"2011`12`13", "2011-12-13"},
{"2011{12{13", "2011-12-13"},
{"2011|12|13", "2011-12-13"},
{"2011}12}13", "2011-12-13"},
{"2011~12~13", "2011-12-13"},
// internal format (YYYYMMDD, YYYYYMMDDHHMMSS)
{"20111213", "2011-12-13"},
{"111213", "2011-12-13"},
// leading and trailing space
{" 2011-12-13", "2011-12-13"},
{"2011-12-13 ", "2011-12-13"},
{" 2011-12-13 ", "2011-12-13"},
// extra dashes
{"2011-12--13", "2011-12-13"},
{"2011--12-13", "2011-12-13"},
{"2011----12----13", "2011-12-13"},
// combinations
{" 2011----12----13 ", "2011-12-13"},
// errors
{"2011 12 13", nil},
{"2011A12A13", nil},
{"2011T12T13", nil},
}
dtblDate := tblToDtbl(tblDate)
for _, t := range dtblDate {
Expand Down
4 changes: 4 additions & 0 deletions expression/generator/string_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,11 @@ func (b *builtinField{{ .TypeName }}Sig) vecEvalInt(input *chunk.Chunk, result *
{{ if .Fixed }}
if arg0[j] == arg1[j] {
{{ else }}
{{ if eq .TypeName "String" }}
if b.ctor.Compare(buf0.GetString(j), buf1.GetString(j)) == 0 {
{{ else }}
if buf0.Get{{ .TypeName }}(j) == buf1.Get{{ .TypeName }}(j) {
{{ end }}
{{ end }}
i64s[j] = int64(i)
}
Expand Down
Loading

0 comments on commit bd9113b

Please sign in to comment.