Skip to content

Commit

Permalink
Merge branch 'master' into wait_timeout
Browse files Browse the repository at this point in the history
  • Loading branch information
hhu-cc authored Nov 28, 2018
2 parents 753c9ed + e5dc251 commit 3b43dd9
Show file tree
Hide file tree
Showing 42 changed files with 746 additions and 91 deletions.
6 changes: 6 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,12 @@ func (d *ddl) genGlobalID() (int64, error) {
var globalID int64
err := kv.RunInNewTxn(d.store, true, func(txn kv.Transaction) error {
var err error

// gofail: var mockGenGlobalIDFail bool
// if mockGenGlobalIDFail {
// return errors.New("gofail genGlobalID error")
// }

globalID, err = meta.NewMeta(txn).GenGlobalID()
return errors.Trace(err)
})
Expand Down
61 changes: 55 additions & 6 deletions ddl/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package ddl_test

import (
"fmt"
"math/rand"
"time"

gofail "github.com/etcd-io/gofail/runtime"
Expand Down Expand Up @@ -91,8 +92,9 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) {
c.Assert(row.Len(), Equals, 1)
c.Assert(row.GetInt64(0), DeepEquals, int64(1))
c.Assert(rs[0].Close(), IsNil)
// Reload schema.
s.dom.ResetHandle(s.store)
// Execute ddl statement reload schema.
_, err = s.se.Execute(context.Background(), "alter table t comment 'test1'")
c.Assert(err, IsNil)
err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil)
c.Assert(err, IsNil)
s.se, err = session.CreateSession4Test(s.store)
Expand All @@ -104,8 +106,8 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) {
c.Assert(err, IsNil)

// test for renaming table
gofail.Enable("github.com/pingcap/tidb/ddl/errRenameTable", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/errRenameTable")
gofail.Enable("github.com/pingcap/tidb/ddl/renameTableErr", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/renameTableErr")
_, err = s.se.Execute(context.Background(), "create table tx(a int)")
c.Assert(err, IsNil)
_, err = s.se.Execute(context.Background(), "insert into tx values(1)")
Expand All @@ -123,8 +125,9 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) {
c.Assert(row.Len(), Equals, 1)
c.Assert(row.GetInt64(0), DeepEquals, int64(1))
c.Assert(rs[0].Close(), IsNil)
// Reload schema.
s.dom.ResetHandle(s.store)
// Execute ddl statement reload schema.
_, err = s.se.Execute(context.Background(), "alter table tx comment 'tx'")
c.Assert(err, IsNil)
err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil)
c.Assert(err, IsNil)
s.se, err = session.CreateSession4Test(s.store)
Expand Down Expand Up @@ -197,3 +200,49 @@ func (s *testDBSuite) TestAddIndexFailed(c *C) {
tk.MustExec("admin check index t idx_b")
tk.MustExec("admin check table t")
}

func (s *testDBSuite) TestGenGlobalIDFail(c *C) {
defer gofail.Disable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail")
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists gen_global_id_fail")
tk.MustExec("use gen_global_id_fail")

sql1 := "create table t1(a bigint PRIMARY KEY, b int)"
sql2 := `create table t2(a bigint PRIMARY KEY, b int) partition by range (a) (
partition p0 values less than (3440),
partition p1 values less than (61440),
partition p2 values less than (122880),
partition p3 values less than maxvalue)`
sql3 := `truncate table t1`
sql4 := `truncate table t2`

testcases := []struct {
sql string
table string
mockErr bool
}{
{sql1, "t1", true},
{sql2, "t2", true},
{sql1, "t1", false},
{sql2, "t2", false},
{sql3, "t1", true},
{sql4, "t2", true},
{sql3, "t1", false},
{sql4, "t2", false},
}

for idx, test := range testcases {
if test.mockErr {
gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(true)`)
_, err := tk.Exec(test.sql)
c.Assert(err, NotNil, Commentf("the %dth test case '%s' fail", idx, test.sql))
} else {
gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(false)`)
tk.MustExec(test.sql)
tk.MustExec(fmt.Sprintf("insert into %s values (%d, 42)", test.table, rand.Intn(65536)))
tk.MustExec(fmt.Sprintf("admin check table %s", test.table))
}
}
tk.MustExec("admin check table t1")
tk.MustExec("admin check table t2")
}
69 changes: 61 additions & 8 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ type Domain struct {
info *InfoSyncer
m sync.Mutex
SchemaValidator SchemaValidator
sysSessionPool *pools.ResourcePool
sysSessionPool *sessionPool
exit chan struct{}
etcdClient *clientv3.Client
wg sync.WaitGroup
Expand Down Expand Up @@ -526,18 +526,13 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio
store: store,
SchemaValidator: NewSchemaValidator(ddlLease),
exit: make(chan struct{}),
sysSessionPool: pools.NewResourcePool(factory, capacity, capacity, resourceIdleTimeout),
sysSessionPool: newSessionPool(capacity, factory),
statsLease: statsLease,
infoHandle: infoschema.NewHandle(store),
slowQuery: newTopNSlowQueries(30, time.Hour*24*7, 500),
}
}

// ResetHandle resets the domain's infoschema handle. It is used for testing.
func (do *Domain) ResetHandle(store kv.Storage) {
do.infoHandle = infoschema.NewHandle(store)
}

// Init initializes a domain.
func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.Resource, error)) error {
if ebd, ok := do.store.(EtcdBackend); ok {
Expand Down Expand Up @@ -606,8 +601,66 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R
return nil
}

type sessionPool struct {
resources chan pools.Resource
factory pools.Factory
mu struct {
sync.RWMutex
closed bool
}
}

func newSessionPool(cap int, factory pools.Factory) *sessionPool {
return &sessionPool{
resources: make(chan pools.Resource, cap),
factory: factory,
}
}

func (p *sessionPool) Get() (resource pools.Resource, err error) {
var ok bool
select {
case resource, ok = <-p.resources:
if !ok {
err = errors.New("session pool closed")
}
default:
resource, err = p.factory()
}
return
}

func (p *sessionPool) Put(resource pools.Resource) {
p.mu.RLock()
defer p.mu.RUnlock()
if p.mu.closed {
resource.Close()
return
}

select {
case p.resources <- resource:
default:
resource.Close()
}
}
func (p *sessionPool) Close() {
p.mu.Lock()
if p.mu.closed {
p.mu.Unlock()
return
}
p.mu.closed = true
close(p.resources)
p.mu.Unlock()

for r := range p.resources {
r.Close()
}
}

// SysSessionPool returns the system session pool.
func (do *Domain) SysSessionPool() *pools.ResourcePool {
func (do *Domain) SysSessionPool() *sessionPool {
return do.sysSessionPool
}

Expand Down
6 changes: 6 additions & 0 deletions executor/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,9 @@ var (
ErrCantChangeTxCharacteristics = terror.ClassExecutor.New(mysql.ErrCantChangeTxCharacteristics, mysql.MySQLErrName[mysql.ErrCantChangeTxCharacteristics])
ErrPsManyParam = terror.ClassExecutor.New(mysql.ErrPsManyParam, mysql.MySQLErrName[mysql.ErrPsManyParam])
ErrAdminCheckTable = terror.ClassExecutor.New(mysql.ErrAdminCheckTable, mysql.MySQLErrName[mysql.ErrAdminCheckTable])
ErrDBaccessDenied = terror.ClassExecutor.New(mysql.ErrDBaccessDenied, mysql.MySQLErrName[mysql.ErrDBaccessDenied])
ErrTableaccessDenied = terror.ClassExecutor.New(mysql.ErrTableaccessDenied, mysql.MySQLErrName[mysql.ErrTableaccessDenied])
ErrBadDB = terror.ClassExecutor.New(mysql.ErrBadDB, mysql.MySQLErrName[mysql.ErrBadDB])
)

func init() {
Expand All @@ -57,6 +60,9 @@ func init() {
mysql.ErrCantChangeTxCharacteristics: mysql.ErrCantChangeTxCharacteristics,
mysql.ErrPsManyParam: mysql.ErrPsManyParam,
mysql.ErrAdminCheckTable: mysql.ErrAdminCheckTable,
mysql.ErrDBaccessDenied: mysql.ErrDBaccessDenied,
mysql.ErrTableaccessDenied: mysql.ErrTableaccessDenied,
mysql.ErrBadDB: mysql.ErrBadDB,
}
terror.ErrClassToMySQLCodes[terror.ClassExecutor] = tableMySQLErrCodes
}
6 changes: 6 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1102,6 +1102,12 @@ func (s *testSuite) TestUnion(c *C) {
for i := 0; i < 4; i++ {
tk.MustQuery("SELECT(SELECT 0 AS a FROM dual UNION SELECT 1 AS a FROM dual ORDER BY a ASC LIMIT 1) AS dev").Check(testkit.Rows("0"))
}

// #issue 8231
tk.MustExec("drop table if exists t1")
tk.MustExec("CREATE TABLE t1 (uid int(1))")
tk.MustExec("INSERT INTO t1 SELECT 150")
tk.MustQuery("SELECT 'a' UNION SELECT uid FROM t1 order by 1 desc;").Check(testkit.Rows("a", "150"))
}

func (s *testSuite) TestNeighbouringProj(c *C) {
Expand Down
14 changes: 14 additions & 0 deletions executor/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,20 @@ func (s *testSuite) TestPrepared(c *C) {
c.Assert(err, IsNil)
c.Assert(len(fields), Equals, 0)

// issue 8074
tk.MustExec("drop table if exists prepare1;")
tk.MustExec("create table prepare1 (a decimal(1))")
tk.MustExec("insert into prepare1 values(1);")
_, err = tk.Exec("prepare stmt FROM @sql1")
c.Assert(err.Error(), Equals, "line 1 column 4 near \"\" (total length 4)")
tk.MustExec("SET @sql = 'update prepare1 set a=5 where a=?';")
_, err = tk.Exec("prepare stmt FROM @sql")
c.Assert(err, IsNil)
tk.MustExec("set @var=1;")
_, err = tk.Exec("execute stmt using @var")
c.Assert(err, IsNil)
tk.MustQuery("select a from prepare1;").Check(testkit.Rows("5"))

// Coverage.
exec := &executor.ExecuteExec{}
exec.Next(ctx, nil)
Expand Down
9 changes: 9 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,6 +516,15 @@ func (s *testSuite) TestValidateSetVar(c *C) {
tk.MustExec("set @@innodb_lock_wait_timeout = 1073741825")
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect innodb_lock_wait_timeout value: '1073741825'"))

tk.MustExec("set @@global.validate_password_number_count=-1")
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect validate_password_number_count value: '-1'"))

tk.MustExec("set @@global.validate_password_length=-1")
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect validate_password_length value: '-1'"))

tk.MustExec("set @@global.validate_password_length=8")
tk.MustQuery("show warnings").Check(testkit.Rows())

_, err = tk.Exec("set @@tx_isolation=''")
c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err))

Expand Down
Loading

0 comments on commit 3b43dd9

Please sign in to comment.