Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/master' into expression_index_…
Browse files Browse the repository at this point in the history
…crate_index
  • Loading branch information
wjhuang2016 committed Jan 2, 2020
2 parents 6fcbd90 + 81a6eb0 commit 1abd378
Show file tree
Hide file tree
Showing 80 changed files with 1,479 additions and 736 deletions.
13 changes: 13 additions & 0 deletions Dockerfile
Original file line number Diff line number Diff line change
@@ -1,3 +1,16 @@
# 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.

# Builder image
FROM golang:1.13-alpine as builder

Expand Down
13 changes: 13 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
@@ -1,3 +1,16 @@
# 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.

PROJECT=tidb
GOPATH ?= $(shell go env GOPATH)
P=8
Expand Down
34 changes: 30 additions & 4 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,9 @@ func (s *testSuite) TestBindParse(c *C) {
c.Check(bind.Collation, Equals, "utf8mb4_bin")
c.Check(bind.CreateTime, NotNil)
c.Check(bind.UpdateTime, NotNil)
dur, err := bind.SinceUpdateTime()
c.Assert(err, IsNil)
c.Assert(int64(dur), GreaterEqual, int64(0))

// Test fields with quotes or slashes.
sql = `CREATE GLOBAL BINDING FOR select * from t where i BETWEEN "a" and "b" USING select * from t use index(index_t) where i BETWEEN "a\nb\rc\td\0e" and 'x'`
Expand Down Expand Up @@ -476,13 +479,13 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) {
tk.MustExec("create table t(a int)")
s.domain.BindHandle().CaptureBaselines()
tk.MustQuery("show global bindings").Check(testkit.Rows())
tk.MustExec("select * from t")
tk.MustExec("select * from t")
tk.MustExec("select count(*) from t where a > 10")
tk.MustExec("select count(*) from t where a > 10")
tk.MustExec("admin capture bindings")
rows := tk.MustQuery("show global bindings").Rows()
c.Assert(len(rows), Equals, 1)
c.Assert(rows[0][0], Equals, "select * from t")
c.Assert(rows[0][1], Equals, "select /*+ USE_INDEX(@`sel_1` `test`.`t` )*/ * from t")
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")
}

func (s *testSuite) TestUseMultiplyBindings(c *C) {
Expand Down Expand Up @@ -603,3 +606,26 @@ func (s *testSuite) TestDefaultSessionVars(c *C) {
"tidb_evolve_plan_baselines off",
"tidb_use_plan_baselines on"))
}

func (s *testSuite) TestDefaultDB(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec("create global binding for select * from test.t using select * from test.t use index(idx)")
tk.MustExec("use mysql")
tk.MustQuery("select * from test.t")
// Even in another database, we could still use the bindings.
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx")
tk.MustExec("drop global binding for select * from test.t")
tk.MustQuery("show global bindings").Check(testkit.Rows())

tk.MustExec("use test")
tk.MustExec("create session binding for select * from test.t using select * from test.t use index(idx)")
tk.MustExec("use mysql")
tk.MustQuery("select * from test.t")
// Even in another database, we could still use the bindings.
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx")
tk.MustExec("drop session binding for select * from test.t")
tk.MustQuery("show session bindings").Check(testkit.Rows())
}
10 changes: 10 additions & 0 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package bindinfo

import (
"context"
"time"
"unsafe"

"github.com/pingcap/parser"
Expand Down Expand Up @@ -64,6 +65,15 @@ func (b *Binding) isSame(rb *Binding) bool {
return b.BindSQL == rb.BindSQL
}

// SinceUpdateTime returns the duration since last update time. Export for test.
func (b *Binding) SinceUpdateTime() (time.Duration, error) {
updateTime, err := b.UpdateTime.Time.GoTime(time.Local)
if err != nil {
return 0, err
}
return time.Since(updateTime), nil
}

// cache is a k-v map, key is original sql, value is a slice of BindRecord.
type cache map[string][]*BindRecord

Expand Down
73 changes: 61 additions & 12 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (

"github.com/pingcap/parser"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/format"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/expression"
Expand All @@ -34,6 +35,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/types"
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -231,7 +233,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.Time{
Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())),
Type: mysql.TypeDatetime,
Type: mysql.TypeTimestamp,
Fsp: 3,
}
record.Bindings[i].UpdateTime = record.Bindings[0].CreateTime
Expand Down Expand Up @@ -284,7 +286,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e

updateTs := types.Time{
Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())),
Type: mysql.TypeDatetime,
Type: mysql.TypeTimestamp,
Fsp: 3,
}

Expand Down Expand Up @@ -554,25 +556,72 @@ func (h *BindHandle) CaptureBaselines() {
logutil.BgLogger().Info("generate hints failed", zap.String("SQL", sqls[i]), zap.Error(err))
continue
}
// We can skip simple query like point get.
if hints == "" {
bindSQL := GenerateBindSQL(context.TODO(), stmt, hints)
if bindSQL == "" {
continue
}
bindsql := strings.Replace(normalizedSQL, "select", fmt.Sprintf("select /*+ %s*/", hints), 1)
charset, collation := h.sctx.GetSessionVars().GetCharsetInfo()
binding := Binding{
BindSQL: bindsql,
Status: Using,
Hint: CollectHint(stmt),
id: hints,
BindSQL: bindSQL,
Status: Using,
Hint: CollectHint(stmt),
id: hints,
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: sqls[i], Db: schemas[i], Bindings: []Binding{binding}})
err = h.AddBindRecord(nil, 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))
}
}
}

// 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.
if planHint == "" {
return ""
}
paramChecker := &paramMarkerChecker{}
stmtNode.Accept(paramChecker)
// We need to evolve on current sql, but we cannot restore values for paramMarkers yet,
// so just ignore them now.
if paramChecker.hasParamMarker {
return ""
}
// We need to evolve plan based on the current sql, not the original sql which may have different parameters.
// So here we would remove the hint and inject the current best plan hint.
BindHint(stmtNode, &HintsSet{})
var sb strings.Builder
restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb)
err := stmtNode.Restore(restoreCtx)
if err != nil {
logutil.Logger(ctx).Warn("Restore SQL failed", zap.Error(err))
}
bindSQL := sb.String()
selectIdx := strings.Index(bindSQL, "SELECT")
// Remove possible `explain` prefix.
bindSQL = bindSQL[selectIdx:]
return strings.Replace(bindSQL, "SELECT", fmt.Sprintf("SELECT /*+ %s*/", planHint), 1)
}

type paramMarkerChecker struct {
hasParamMarker bool
}

func (e *paramMarkerChecker) Enter(in ast.Node) (ast.Node, bool) {
if _, ok := in.(*driver.ParamMarkerExpr); ok {
e.hasParamMarker = true
return in, true
}
return in, false
}

func (e *paramMarkerChecker) Leave(in ast.Node) (ast.Node, bool) {
return in, true
}

// AddEvolvePlanTask adds the evolve plan task into memory cache. It would be flushed to store periodically.
func (h *BindHandle) AddEvolvePlanTask(originalSQL, DB string, binding Binding, planHint string) {
binding.id = planHint
Expand Down Expand Up @@ -641,13 +690,13 @@ func (h *BindHandle) getOnePendingVerifyJob() (string, string, Binding) {
if bind.Status != Rejected {
continue
}
updateTime, err := bind.UpdateTime.Time.GoTime(time.UTC)
dur, err := bind.SinceUpdateTime()
// Should not happen.
if err != nil {
continue
}
// Rejected and retry it now.
if time.Since(updateTime) > nextVerifyDuration {
if dur > nextVerifyDuration {
return bindRecord.OriginalSQL, bindRecord.Db, bind
}
}
Expand Down
2 changes: 1 addition & 1 deletion bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func (h *SessionHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.Inf
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.Time{
Time: types.FromGoTime(time.Now()),
Type: mysql.TypeDatetime,
Type: mysql.TypeTimestamp,
Fsp: 3,
}
record.Bindings[i].UpdateTime = record.Bindings[i].CreateTime
Expand Down
12 changes: 12 additions & 0 deletions checkout-pr-branch.sh
Original file line number Diff line number Diff line change
@@ -1,4 +1,16 @@
#!/usr/bin/env bash
# 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.

# This script is used to checkout a TiDB PR branch in a forked repo.
if test -z $1; then
Expand Down
12 changes: 12 additions & 0 deletions cmd/explaintest/run-tests.sh
Original file line number Diff line number Diff line change
@@ -1,4 +1,16 @@
#!/usr/bin/env bash
# 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.

TIDB_TEST_STORE_NAME=$TIDB_TEST_STORE_NAME
TIKV_PATH=$TIKV_PATH
Expand Down
3 changes: 3 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -751,6 +751,9 @@ func (c *Config) Valid() error {
if c.AlterPrimaryKey && c.Experimental.AllowAutoRandom {
return fmt.Errorf("allow-auto-random is unavailable when alter-primary-key is enabled")
}
if c.PreparedPlanCache.Capacity < 1 {
return fmt.Errorf("capacity in [prepared-plan-cache] should be at least 1")
}
return nil
}

Expand Down
2 changes: 1 addition & 1 deletion ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3266,7 +3266,7 @@ func (s *testDBSuite4) TestAddColumn2(c *C) {
c.Assert(err, IsNil)
_, err = writeOnlyTable.AddRecord(s.tk.Se, types.MakeDatums(oldRow[0].GetInt64(), 2, oldRow[2].GetInt64()), table.IsUpdate)
c.Assert(err, IsNil)
err = s.tk.Se.StmtCommit()
err = s.tk.Se.StmtCommit(nil)
c.Assert(err, IsNil)
err = s.tk.Se.CommitTxn(ctx)
c.Assert(err, IsNil)
Expand Down
7 changes: 3 additions & 4 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -866,11 +866,10 @@ func (do *Domain) globalBindHandleWorkerLoop() {
if err != nil {
logutil.BgLogger().Error("update bindinfo failed", zap.Error(err))
}
if !variable.TiDBOptOn(variable.CapturePlanBaseline.GetVal()) {
continue
}
do.bindHandle.DropInvalidBindRecord()
do.bindHandle.CaptureBaselines()
if variable.TiDBOptOn(variable.CapturePlanBaseline.GetVal()) {
do.bindHandle.CaptureBaselines()
}
do.bindHandle.SaveEvolveTasksToStore()
}
}
Expand Down
7 changes: 4 additions & 3 deletions executor/bind.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ type SQLBindExec struct {
bindSQL string
charset string
collation string
db string
isGlobal bool
bindAst ast.StmtNode
}
Expand Down Expand Up @@ -69,9 +70,9 @@ func (e *SQLBindExec) dropSQLBind() error {
}
if !e.isGlobal {
handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
return handle.DropBindRecord(e.normdOrigSQL, e.ctx.GetSessionVars().CurrentDB, bindInfo)
return handle.DropBindRecord(e.normdOrigSQL, e.db, bindInfo)
}
return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.normdOrigSQL, e.ctx.GetSessionVars().CurrentDB, bindInfo)
return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.normdOrigSQL, e.db, bindInfo)
}

func (e *SQLBindExec) createSQLBind() error {
Expand All @@ -83,7 +84,7 @@ func (e *SQLBindExec) createSQLBind() error {
}
record := &bindinfo.BindRecord{
OriginalSQL: e.normdOrigSQL,
Db: e.ctx.GetSessionVars().CurrentDB,
Db: e.db,
Bindings: []bindinfo.Binding{bindInfo},
}
if !e.isGlobal {
Expand Down
10 changes: 6 additions & 4 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -622,13 +622,14 @@ func (b *executorBuilder) buildShow(v *plannercore.PhysicalShow) Executor {
Table: v.Table,
Column: v.Column,
IndexName: v.IndexName,
User: v.User,
Roles: v.Roles,
IfNotExists: v.IfNotExists,
Flag: v.Flag,
Roles: v.Roles,
User: v.User,
is: b.is,
Full: v.Full,
IfNotExists: v.IfNotExists,
GlobalScope: v.GlobalScope,
is: b.is,
Extended: v.Extended,
}
if e.Tp == ast.ShowGrants && e.User == nil {
// The input is a "show grants" statement, fulfill the user and roles field.
Expand Down Expand Up @@ -2535,6 +2536,7 @@ func (b *executorBuilder) buildSQLBindExec(v *plannercore.SQLBindPlan) Executor
bindSQL: v.BindSQL,
charset: v.Charset,
collation: v.Collation,
db: v.Db,
isGlobal: v.IsGlobal,
bindAst: v.BindStmt,
}
Expand Down
Loading

0 comments on commit 1abd378

Please sign in to comment.