Skip to content

Commit

Permalink
Merge branch 'master' of github.com:pingcap/tidb into expression_inde…
Browse files Browse the repository at this point in the history
…x_crate_index
  • Loading branch information
wjhuang2016 committed Jan 3, 2020
2 parents 1a40266 + 324a468 commit 0fbfd50
Show file tree
Hide file tree
Showing 56 changed files with 1,058 additions and 200 deletions.
15 changes: 14 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 @@ -629,3 +630,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
60 changes: 32 additions & 28 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,8 +169,8 @@ 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
}
Expand Down Expand Up @@ -219,17 +218,18 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc
h.bindInfo.Unlock()
}()

txn, err1 := h.sctx.Context.Txn(true)
if err1 != nil {
return err1
}

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

txn, err1 := h.sctx.Context.Txn(true)
if err1 != nil {
return err1
}
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
Expand Down Expand Up @@ -387,13 +387,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 +513,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 +528,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 +548,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 @@ -789,7 +793,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
5 changes: 2 additions & 3 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,13 +47,13 @@ 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 {
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
}

err := record.prepareHints(sctx, is)
err := record.prepareHints(sctx)
// update the BindMeta to the cache.
if err == nil {
h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record)
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
55 changes: 55 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1861,6 +1861,61 @@ func (s *testDBSuite4) TestChangeColumn(c *C) {
s.tk.MustExec("drop table t3")
}

func (s *testDBSuite5) TestRenameColumn(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use " + s.schemaName)

assertColNames := func(tableName string, colNames ...string) {
cols := s.testGetTable(c, tableName).Cols()
c.Assert(len(cols), Equals, len(colNames), Commentf("number of columns mismatch"))
for i := range cols {
c.Assert(cols[i].Name.L, Equals, strings.ToLower(colNames[i]))
}
}

s.mustExec(c, "create table test_rename_column (id int not null primary key auto_increment, col1 int)")
s.mustExec(c, "alter table test_rename_column rename column col1 to col1")
assertColNames("test_rename_column", "id", "col1")
s.mustExec(c, "alter table test_rename_column rename column col1 to col2")
assertColNames("test_rename_column", "id", "col2")

// Test renaming non-exist columns.
s.tk.MustGetErrCode("alter table test_rename_column rename column non_exist_col to col3", mysql.ErrBadField)

// Test renaming to an exist column.
s.tk.MustGetErrCode("alter table test_rename_column rename column col2 to id", mysql.ErrDupFieldName)

// Test renaming the column with foreign key.
s.tk.MustExec("drop table test_rename_column")
s.tk.MustExec("create table test_rename_column_base (base int)")
s.tk.MustExec("create table test_rename_column (col int, foreign key (col) references test_rename_column_base(base))")

s.tk.MustGetErrCode("alter table test_rename_column rename column col to col1", mysql.ErrFKIncompatibleColumns)

s.tk.MustExec("drop table test_rename_column_base")

// Test renaming generated columns.
s.tk.MustExec("drop table test_rename_column")
s.tk.MustExec("create table test_rename_column (id int, col1 int generated always as (id + 1))")

s.mustExec(c, "alter table test_rename_column rename column col1 to col2")
assertColNames("test_rename_column", "id", "col2")
s.mustExec(c, "alter table test_rename_column rename column col2 to col1")
assertColNames("test_rename_column", "id", "col1")
s.tk.MustGetErrCode("alter table test_rename_column rename column id to id1", mysql.ErrBadField)

// Test renaming view columns.
s.tk.MustExec("drop table test_rename_column")
s.mustExec(c, "create table test_rename_column (id int, col1 int)")
s.mustExec(c, "create view test_rename_column_view as select * from test_rename_column")

s.mustExec(c, "alter table test_rename_column rename column col1 to col2")
s.tk.MustGetErrCode("select * from test_rename_column_view", mysql.ErrViewInvalid)

s.mustExec(c, "drop view test_rename_column_view")
s.tk.MustExec("drop table test_rename_column")
}

func (s *testDBSuite) mustExec(c *C, query string, args ...interface{}) {
s.tk.MustExec(query, args...)
}
Expand Down
58 changes: 58 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -2025,6 +2025,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = d.ModifyColumn(ctx, ident, spec)
case ast.AlterTableChangeColumn:
err = d.ChangeColumn(ctx, ident, spec)
case ast.AlterTableRenameColumn:
err = d.RenameColumn(ctx, ident, spec)
case ast.AlterTableAlterColumn:
err = d.AlterColumn(ctx, ident, spec)
case ast.AlterTableRenameTable:
Expand Down Expand Up @@ -2940,6 +2942,62 @@ func (d *ddl) ChangeColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al
return errors.Trace(err)
}

// RenameColumn renames an existing column.
func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
oldColName := spec.OldColumnName.Name
newColName := spec.NewColumnName.Name
if oldColName.L == newColName.L {
return nil
}

schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ident)
if err != nil {
return errors.Trace(err)
}

oldCol := table.FindCol(tbl.VisibleCols(), oldColName.L)
if oldCol == nil {
return infoschema.ErrColumnNotExists.GenWithStackByArgs(oldColName, ident.Name)
}

allCols := tbl.Cols()
colWithNewNameAlreadyExist := table.FindCol(allCols, newColName.L) != nil
if colWithNewNameAlreadyExist {
return infoschema.ErrColumnExists.GenWithStackByArgs(newColName)
}

if fkInfo := getColumnForeignKeyInfo(oldColName.L, tbl.Meta().ForeignKeys); fkInfo != nil {
return errFKIncompatibleColumns.GenWithStackByArgs(oldColName, fkInfo.Name)
}

// Check generated expression.
for _, col := range allCols {
if col.GeneratedExpr == nil {
continue
}
dependedColNames := findColumnNamesInExpr(col.GeneratedExpr)
for _, name := range dependedColNames {
if name.Name.L == oldColName.L {
return ErrBadField.GenWithStackByArgs(oldColName.O, "generated column function")
}
}
}

newCol := oldCol.Clone()
newCol.Name = newColName
job := &model.Job{
SchemaID: schema.ID,
TableID: tbl.Meta().ID,
SchemaName: schema.Name.L,
Type: model.ActionModifyColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{&newCol, oldColName, spec.Position, 0},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

// ModifyColumn does modification on an existing column, currently we only support limited kind of changes
// that do not need to change or check data on the table.
func (d *ddl) ModifyColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
Expand Down
6 changes: 3 additions & 3 deletions ddl/util/syncer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ import (
"go.etcd.io/etcd/integration"
"go.etcd.io/etcd/mvcc/mvccpb"
goctx "golang.org/x/net/context"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -230,7 +230,7 @@ func TestSyncerSimple(t *testing.T) {
}

func isTimeoutError(err error) bool {
if terror.ErrorEqual(err, goctx.DeadlineExceeded) || grpc.Code(errors.Cause(err)) == codes.DeadlineExceeded ||
if terror.ErrorEqual(err, goctx.DeadlineExceeded) || status.Code(errors.Cause(err)) == codes.DeadlineExceeded ||
terror.ErrorEqual(err, etcdserver.ErrTimeout) {
return true
}
Expand All @@ -250,7 +250,7 @@ func checkRespKV(t *testing.T, kvCount int, key, val string,
if string(kv.Key) != key {
t.Fatalf("key resp %s, exported %s", kv.Key, key)
}
if val != val {
if string(kv.Value) != val {
t.Fatalf("val resp %s, exported %s", kv.Value, val)
}
}
Loading

0 comments on commit 0fbfd50

Please sign in to comment.