*: run a bootstrap session before store is ready (#2481)

This commit is contained in:
tiancaiamao
2017-01-17 14:46:35 +08:00
committed by Han Fei
parent e79a460a28
commit e1ecc1059e
18 changed files with 158 additions and 95 deletions

View File

@ -28,7 +28,7 @@ import (
func (s *testSessionSuite) TestBootstrap(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "USE mysql;")
r := mustExecSQL(c, se, `select * from user;`)
@ -121,6 +121,7 @@ func (s *testSessionSuite) TestBootstrapWithError(c *C) {
store := newStore(c, s.dbNameBootstrap)
s.bootstrapWithOnlyDDLWork(store, c)
BootstrapSession(store)
se := newSession(c, store, s.dbNameBootstrap)
mustExecSQL(c, se, "USE mysql;")
r := mustExecSQL(c, se, `select * from user;`)
@ -153,7 +154,7 @@ func (s *testSessionSuite) TestBootstrapWithError(c *C) {
// Test case for upgrade
func (s *testSessionSuite) TestUpgrade(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "USE mysql;")
@ -195,6 +196,7 @@ func (s *testSessionSuite) TestUpgrade(c *C) {
c.Assert(ver, Equals, int64(0))
// Create a new session then upgrade() will run automatically.
BootstrapSession(store)
se2 := newSession(c, store, s.dbName)
r = mustExecSQL(c, se2, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`)
row, err = r.Next()

View File

@ -60,6 +60,7 @@ func (s *testDBSuite) SetUpSuite(c *C) {
s.schemaName = "test_db"
s.store, err = tidb.NewStore(tidb.EngineGoLevelDBMemory)
c.Assert(err, IsNil)
tidb.BootstrapSession(s.store)
localstore.MockRemoteStore = true
s.s, err = tidb.CreateSession(s.store)
c.Assert(err, IsNil)
@ -755,6 +756,7 @@ func (s *testDBSuite) TestUpdateMultipleTable(c *C) {
defer testleak.AfterTest(c)
store, err := tidb.NewStore("memory://update_multiple_table")
c.Assert(err, IsNil)
tidb.BootstrapSession(store)
tk := testkit.NewTestKit(c, store)
tk.MustExec("use test")
tk.MustExec("create table t1 (c1 int, c2 int)")
@ -814,6 +816,7 @@ func (s *testDBSuite) TestTruncateTable(c *C) {
defer testleak.AfterTest(c)
store, err := tidb.NewStore("memory://truncate_table")
c.Assert(err, IsNil)
tidb.BootstrapSession(store)
tk := testkit.NewTestKit(c, store)
tk.MustExec("use test")
tk.MustExec("create table t (c1 int, c2 int)")
@ -864,6 +867,7 @@ func (s *testDBSuite) TestRenameTable(c *C) {
defer testleak.AfterTest(c)
store, err := tidb.NewStore("memory://rename_table")
c.Assert(err, IsNil)
tidb.BootstrapSession(store)
s.tk = testkit.NewTestKit(c, store)
s.tk.MustExec("use test")

View File

@ -65,6 +65,8 @@ func (s *testSuite) SetUpSuite(c *C) {
c.Assert(err, IsNil)
s.store = store
}
err := tidb.BootstrapSession(s.store)
c.Assert(err, IsNil)
logLevel := os.Getenv("log_level")
log.SetLevelByString(logLevel)
executor.BaseLookupTableTaskSize = 2

View File

@ -73,6 +73,8 @@ func (p *testPerfSchemaSuit) TestInsert(c *C) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory)
c.Assert(err, IsNil)
defer store.Close()
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
se := newSession(c, store, "")
defer se.Close()
mustExec(c, se, `insert into performance_schema.setup_actors values("localhost", "nieyy", "contributor", "NO", "NO");`)
@ -108,6 +110,8 @@ func (p *testPerfSchemaSuit) TestInstrument(c *C) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory + "/test_instrument_db")
c.Assert(err, IsNil)
defer store.Close()
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
se := newSession(c, store, "test_instrument_db")
defer se.Close()
@ -122,6 +126,8 @@ func (p *testPerfSchemaSuit) TestConcurrentStatement(c *C) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory + "/test_con_stmt")
c.Assert(err, IsNil)
defer store.Close()
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
se := newSession(c, store, "test_con_stmt")
mustExec(c, se, "drop table if exists test")

View File

@ -15,7 +15,6 @@ package plan_test
import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/parser"
@ -81,7 +80,7 @@ var resolverTestCases = []resolverTestCase{
}
func (ts *testNameResolverSuite) TestNameResolver(c *C) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory)
store, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
defer store.Close()
testKit := testkit.NewTestKit(c, store)

View File

@ -14,10 +14,12 @@
package plan_test
import (
"github.com/juju/errors"
. "github.com/pingcap/check"
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan"
@ -34,7 +36,7 @@ type testTypeInferrerSuite struct {
}
func (ts *testTypeInferrerSuite) TestInferType(c *C) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory)
store, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
defer store.Close()
testKit := testkit.NewTestKit(c, store)
@ -196,7 +198,7 @@ func (ts *testTypeInferrerSuite) TestInferType(c *C) {
func (s *testTypeInferrerSuite) TestColumnInfoModified(c *C) {
defer testleak.AfterTest(c)()
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory)
store, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
defer store.Close()
testKit := testkit.NewTestKit(c, store)
@ -210,3 +212,12 @@ func (s *testTypeInferrerSuite) TestColumnInfoModified(c *C) {
col := table.FindCol(tbl.Cols(), "col1")
c.Assert(col.Tp, Equals, mysql.TypeLong)
}
func newStoreWithBootstrap() (kv.Storage, error) {
store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory)
if err != nil {
return nil, errors.Trace(err)
}
err = tidb.BootstrapSession(store)
return store, errors.Trace(err)
}

View File

@ -31,6 +31,8 @@ type testCacheSuite struct {
func (s *testCacheSuite) SetUpSuite(c *C) {
store, err := tidb.NewStore("memory://mysql")
c.Assert(err, IsNil)
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
s.store = store
}

View File

@ -274,6 +274,8 @@ func mustExec(c *C, se tidb.Session, sql string) {
func newStore(c *C, dbPath string) kv.Storage {
store, err := tidb.NewStore("memory" + "://" + dbPath)
c.Assert(err, IsNil)
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
return store
}

View File

@ -34,6 +34,8 @@ func (ts *TidbTestSuite) SetUpSuite(c *C) {
log.SetLevelByString("error")
store, err := tidb.NewStore("memory:///tmp/tidb")
c.Assert(err, IsNil)
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
ts.tidbdrv = NewTiDBDriver(store)
cfg := &Config{
Addr: ":4001",

View File

@ -702,6 +702,20 @@ func chooseMinLease(n1 time.Duration, n2 time.Duration) time.Duration {
// CreateSession creates a new session environment.
func CreateSession(store kv.Storage) (Session, error) {
s, err := createSession(store)
if err != nil {
return nil, errors.Trace(err)
}
// TODO: Add auth here
privChecker := &privileges.UserPrivileges{}
privilege.BindPrivilegeChecker(s, privChecker)
return s, nil
}
// BootstrapSession runs the first time when the TiDB server start.
func BootstrapSession(store kv.Storage) error {
ver := getStoreBootstrapVersion(store)
if ver == notBootstrapped {
runInBootstrapSession(store, bootstrap)
@ -709,7 +723,8 @@ func CreateSession(store kv.Storage) (Session, error) {
runInBootstrapSession(store, upgrade)
}
return createSession(store)
_, err := domap.Get(store)
return errors.Trace(err)
}
// runInBootstrapSession create a special session for boostrap to run.
@ -753,9 +768,6 @@ func createSession(store kv.Storage) (*session, error) {
// session implements variable.GlobalVarAccessor. Bind it to ctx.
s.sessionVars.GlobalVarsAccessor = s
// TODO: Add auth here
privChecker := &privileges.UserPrivileges{}
privilege.BindPrivilegeChecker(s, privChecker)
return s, nil
}

View File

@ -67,7 +67,7 @@ func (s *testSessionSuite) TearDownSuite(c *C) {
func (s *testSessionSuite) TestPrepare(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
// create table
mustExecSQL(c, se, s.dropTableSQL)
@ -111,7 +111,7 @@ func (s *testSessionSuite) TestPrepare(c *C) {
func (s *testSessionSuite) TestAffectedRows(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, s.dropTableSQL)
mustExecSQL(c, se, s.createTableSQL)
@ -173,7 +173,7 @@ func (s *testSessionSuite) TestAffectedRows(c *C) {
func (s *testSessionSuite) TestString(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
sessionExec(c, se, "select 1")
// here to check the panic bug in String() when txn is nil after committed.
@ -185,7 +185,7 @@ func (s *testSessionSuite) TestString(c *C) {
func (s *testSessionSuite) TestResultField(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
// create table
mustExecSQL(c, se, s.dropTableSQL)
@ -211,7 +211,7 @@ func (s *testSessionSuite) TestResultField(c *C) {
func (s *testSessionSuite) TestPrimaryKeyAutoincrement(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL, name varchar(255) UNIQUE NOT NULL, status int)")
@ -249,7 +249,7 @@ func (s *testSessionSuite) TestPrimaryKeyAutoincrement(c *C) {
func (s *testSessionSuite) TestAutoIncrementID(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -291,7 +291,7 @@ func checkAutocommit(c *C, se Session, expectStatus uint16) {
// See https://dev.mysql.com/doc/internals/en/status-flags.html
func (s *testSessionSuite) TestAutocommit(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
checkTxn(c, se, "drop table if exists t;", 0)
checkAutocommit(c, se, 2)
@ -333,7 +333,7 @@ func checkInTrans(c *C, se Session, stmt string, expectStatus uint16) {
// See https://dev.mysql.com/doc/internals/en/status-flags.html
func (s *testSessionSuite) TestInTrans(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
checkInTrans(c, se, "drop table if exists t;", 0)
checkInTrans(c, se, "create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)", 0)
@ -368,7 +368,7 @@ func (s *testSessionSuite) TestInTrans(c *C) {
// See http://dev.mysql.com/doc/refman/5.7/en/commit.html
func (s *testSessionSuite) TestRowLock(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
se2 := newSession(c, store, s.dbName)
@ -412,7 +412,7 @@ func (s *testSessionSuite) TestRowLock(c *C) {
func (s *testSessionSuite) TestIssue1118(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
c.Assert(se.(*session).txn, IsNil)
@ -475,7 +475,7 @@ func (s *testSessionSuite) TestIssue1118(c *C) {
func (s *testSessionSuite) TestIssue827(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
@ -638,7 +638,7 @@ func (s *testSessionSuite) TestIssue827(c *C) {
func (s *testSessionSuite) TestIssue996(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -672,7 +672,7 @@ func (s *testSessionSuite) TestIssue986(c *C) {
sqlText := `CREATE TABLE address (
id bigint(20) NOT NULL AUTO_INCREMENT,
PRIMARY KEY (id));`
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, sqlText)
sqlText = `insert into address values ('10')`
@ -684,7 +684,7 @@ func (s *testSessionSuite) TestIssue986(c *C) {
func (s *testSessionSuite) TestIssue1089(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
r := mustExecSQL(c, se, "select cast(0.5 as unsigned)")
@ -702,7 +702,7 @@ func (s *testSessionSuite) TestIssue1089(c *C) {
func (s *testSessionSuite) TestIssue1135(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName+"1")
@ -766,7 +766,7 @@ func (s *testSessionSuite) TestIssue1135(c *C) {
func (s *testSessionSuite) TestIssue1114(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "set @tmp = 0")
@ -814,7 +814,7 @@ func (s *testSessionSuite) TestIssue1114(c *C) {
func (s *testSessionSuite) TestSelectForUpdate(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
se2 := newSession(c, store, s.dbName)
@ -894,7 +894,7 @@ func (s *testSessionSuite) TestSelectForUpdate(c *C) {
func (s *testSessionSuite) TestRow(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
r := mustExecSQL(c, se, "select row(1, 1) in (row(1, 1))")
@ -938,7 +938,7 @@ func (s *testSessionSuite) TestRow(c *C) {
func (s *testSessionSuite) TestIndex(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "create table if not exists test_index (c1 int, c double, index(c1), index(c))")
@ -987,7 +987,7 @@ func (s *testSessionSuite) TestIndex(c *C) {
func (s *testSessionSuite) TestMySQLTypes(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
r := mustExecSQL(c, se, `select 0x01 + 1, x'4D7953514C' = "MySQL"`)
@ -1008,7 +1008,7 @@ func (s *testSessionSuite) TestMySQLTypes(c *C) {
func (s *testSessionSuite) TestExpression(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
r := mustExecSQL(c, se, `select + (1 > 0), -(1 >0), + (1 < 0), - (1 < 0)`)
@ -1029,7 +1029,7 @@ func (s *testSessionSuite) TestExpression(c *C) {
func (s *testSessionSuite) TestSelect(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "create table if not exists t (c1 int, c2 int)")
@ -1197,7 +1197,7 @@ func (s *testSessionSuite) TestSelect(c *C) {
func (s *testSessionSuite) TestSubQuery(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "create table if not exists t1 (c1 int, c2 int)")
@ -1225,7 +1225,7 @@ func (s *testSessionSuite) TestSubQuery(c *C) {
func (s *testSessionSuite) TestShow(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "set global autocommit=1")
@ -1288,7 +1288,7 @@ func (s *testSessionSuite) TestShow(c *C) {
func (s *testSessionSuite) TestTimeFunc(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
last := time.Now().Format(types.TimeFormat)
@ -1315,7 +1315,7 @@ func (s *testSessionSuite) TestTimeFunc(c *C) {
func (s *testSessionSuite) TestBit(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1334,7 +1334,7 @@ func (s *testSessionSuite) TestBit(c *C) {
func (s *testSessionSuite) TestEnum(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1364,7 +1364,7 @@ func (s *testSessionSuite) TestEnum(c *C) {
func (s *testSessionSuite) TestSet(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1399,7 +1399,7 @@ func (s *testSessionSuite) TestSet(c *C) {
func (s *testSessionSuite) TestDatabase(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
// Test database.
@ -1426,7 +1426,7 @@ func (s *testSessionSuite) TestDatabase(c *C) {
func (s *testSessionSuite) TestWhereLike(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1449,7 +1449,7 @@ func (s *testSessionSuite) TestDefaultFlenBug(c *C) {
defer testleak.AfterTest(c)()
// If set unspecified column flen to 0, it will cause bug in union.
// This test is used to prevent the bug reappear.
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "create table t1 (c double);")
@ -1470,7 +1470,7 @@ func (s *testSessionSuite) TestDefaultFlenBug(c *C) {
func (s *testSessionSuite) TestExecRestrictedSQL(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName).(*session)
r, err := se.ExecRestrictedSQL(se, "select 1;")
c.Assert(r, NotNil)
@ -1486,7 +1486,7 @@ func (s *testSessionSuite) TestExecRestrictedSQL(c *C) {
func (s *testSessionSuite) TestGroupBy(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (c1 int, c2 int)")
@ -1513,7 +1513,7 @@ func (s *testSessionSuite) TestGroupBy(c *C) {
func (s *testSessionSuite) TestOrderBy(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (c1 int, c2 int, c3 varchar(20))")
@ -1548,7 +1548,7 @@ func (s *testSessionSuite) TestOrderBy(c *C) {
func (s *testSessionSuite) TestHaving(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (c1 int, c2 int, c3 int)")
@ -1590,7 +1590,7 @@ func (s *testSessionSuite) TestHaving(c *C) {
func (s *testSessionSuite) TestResultType(c *C) {
defer testleak.AfterTest(c)()
// Testcase for https://github.com/pingcap/tidb/issues/325
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
rs := mustExecSQL(c, se, `select cast(null as char(30))`)
c.Assert(rs, NotNil)
@ -1607,7 +1607,7 @@ func (s *testSessionSuite) TestResultType(c *C) {
func (s *testSessionSuite) TestIssue461(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se1 := newSession(c, store, s.dbName)
mustExecSQL(c, se1,
`CREATE TABLE test ( id int(11) UNSIGNED NOT NULL AUTO_INCREMENT, val int UNIQUE, PRIMARY KEY (id)); `)
@ -1640,7 +1640,7 @@ func (s *testSessionSuite) TestIssue461(c *C) {
func (s *testSessionSuite) TestIssue463(c *C) {
defer testleak.AfterTest(c)()
// Testcase for https://github.com/pingcap/tidb/issues/463
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "DROP TABLE IF EXISTS test")
mustExecSQL(c, se,
@ -1688,7 +1688,7 @@ func (s *testSessionSuite) TestIssue463(c *C) {
func (s *testSessionSuite) TestIssue177(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, `drop table if exists t1;`)
mustExecSQL(c, se, `drop table if exists t2;`)
@ -1708,7 +1708,7 @@ func (s *testSessionSuite) TestIssue177(c *C) {
func (s *testSessionSuite) TestBuiltin(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
// Testcase for https://github.com/pingcap/tidb/issues/382
@ -1721,7 +1721,7 @@ func (s *testSessionSuite) TestBuiltin(c *C) {
func (s *testSessionSuite) TestFieldText(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (a int)")
@ -1750,7 +1750,7 @@ func (s *testSessionSuite) TestFieldText(c *C) {
func (s *testSessionSuite) TestIndexPointLookup(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
mustExecSQL(c, se, "create table t (a int)")
@ -1768,7 +1768,7 @@ func (s *testSessionSuite) TestIndexPointLookup(c *C) {
func (s *testSessionSuite) TestIssue454(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1785,7 +1785,7 @@ func (s *testSessionSuite) TestIssue454(c *C) {
func (s *testSessionSuite) TestIssue456(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t")
@ -1803,7 +1803,7 @@ func (s *testSessionSuite) TestIssue456(c *C) {
// For https://github.com/pingcap/tidb/issues/571
func (s *testSessionSuite) TestIssue571(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "begin")
@ -1855,7 +1855,7 @@ func (s *testSessionSuite) TestIssue571(c *C) {
func (s *testSessionSuite) TestIssue620(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t1")
@ -1875,7 +1875,7 @@ func (s *testSessionSuite) TestIssue620(c *C) {
func (s *testSessionSuite) TestRetryPreparedStmt(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
se2 := newSession(c, store, s.dbName)
@ -1906,7 +1906,7 @@ func (s *testSessionSuite) TestRetryPreparedStmt(c *C) {
func (s *testSessionSuite) TestSleep(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "select sleep(0.01);")
@ -1924,7 +1924,7 @@ func (s *testSessionSuite) TestSleep(c *C) {
func (s *testSessionSuite) TestIssue893(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t1; create table t1(id int ); insert into t1 values (1);")
mustExecMatch(c, se, "select * from t1;", [][]interface{}{{1}})
@ -1935,7 +1935,7 @@ func (s *testSessionSuite) TestIssue893(c *C) {
func (s *testSessionSuite) TestIssue1265(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
@ -1947,7 +1947,7 @@ func (s *testSessionSuite) TestIssue1265(c *C) {
func (s *testSessionSuite) TestIssue1435(c *C) {
defer testleak.AfterTest(c)()
localstore.MockRemoteStore = true
store := newStore(c, s.dbName+"issue1435")
store := newStoreWithBootstrap(c, s.dbName+"issue1435")
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
se2 := newSession(c, store, s.dbName)
@ -2046,7 +2046,7 @@ func (s *testSessionSuite) TestIssue1435(c *C) {
// Testcase for session
func (s *testSessionSuite) TestSession(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "ROLLBACK;")
@ -2058,7 +2058,7 @@ func (s *testSessionSuite) TestSession(c *C) {
func (s *testSessionSuite) TestSessionAuth(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
defer se.Close()
c.Assert(se.Auth("Any not exist username with zero password! @anyhost", []byte(""), []byte("")), IsFalse)
@ -2069,7 +2069,7 @@ func (s *testSessionSuite) TestSessionAuth(c *C) {
func (s *testSessionSuite) TestErrorRollback(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
s1 := newSession(c, store, s.dbName)
defer s1.Close()
@ -2115,7 +2115,7 @@ func (s *testSessionSuite) TestErrorRollback(c *C) {
func (s *testSessionSuite) TestMultiColumnIndex(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
mustExecSQL(c, se, "create table t (c1 int, c2 int);")
@ -2162,7 +2162,7 @@ func (s *testSessionSuite) TestMultiColumnIndex(c *C) {
func (s *testSessionSuite) TestSubstringIndexExpr(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
mustExecSQL(c, se, "create table t (c varchar(128));")
@ -2175,7 +2175,7 @@ func (s *testSessionSuite) TestSubstringIndexExpr(c *C) {
func (s *testSessionSuite) TestIndexMaxLength(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
@ -2255,7 +2255,7 @@ func (s *testSessionSuite) TestIndexMaxLength(c *C) {
func (s *testSessionSuite) TestSpecifyIndexPrefixLength(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
@ -2368,7 +2368,7 @@ func (s *testSessionSuite) TestSpecifyIndexPrefixLength(c *C) {
func (s *testSessionSuite) TestIndexColumnLength(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "drop table if exists t;")
mustExecSQL(c, se, "create table t (c1 int, c2 blob);")
@ -2399,7 +2399,7 @@ func (s *testSessionSuite) TestIgnoreForeignKey(c *C) {
CONSTRAINT FK_7rod8a71yep5vxasb0ms3osbg FOREIGN KEY (user_id) REFERENCES waimaiqa.user (id),
INDEX FK_7rod8a71yep5vxasb0ms3osbg (user_id) comment ''
) ENGINE=InnoDB AUTO_INCREMENT=30 DEFAULT CHARACTER SET utf8 COLLATE utf8_general_ci ROW_FORMAT=COMPACT COMMENT='' CHECKSUM=0 DELAY_KEY_WRITE=0;`
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, sqlText)
@ -2409,7 +2409,7 @@ func (s *testSessionSuite) TestIgnoreForeignKey(c *C) {
func (s *testSessionSuite) TestJoinSubquery(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecSQL(c, se, "CREATE TABLE table1 (id INTEGER key AUTO_INCREMENT, data VARCHAR(30))")
mustExecSQL(c, se, "CREATE TABLE table2 (id INTEGER key AUTO_INCREMENT, data VARCHAR(30), t1id INTEGER)")
@ -2431,7 +2431,7 @@ func (s *testSessionSuite) TestGlobalVarAccessor(c *C) {
varValue1 := "4194305"
varValue2 := "4194306"
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName).(*session)
// Get globalSysVar twice and get the same value
v, err := se.GetGlobalSysVar(varName)
@ -2498,7 +2498,7 @@ func (s *testSessionSuite) TestSqlLogicTestCase(c *C) {
CREATE TABLE tab1(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT)
INSERT INTO tab1 VALUES(0,26,690.51)
CREATE INDEX idx_tab1_1 on tab1 (col1)`
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecMultiSQL(c, se, initSQL)
@ -2511,7 +2511,7 @@ func (s *testSessionSuite) TestXAggregateWithIndexScan(c *C) {
drop table IF EXISTS t;
CREATE TABLE t(c INT, index cidx (c));
INSERT INTO t VALUES(1), (null), (2);`
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecMultiSQL(c, se, initSQL)
sql := "SELECT COUNT(c) FROM t WHERE c > 0;"
@ -2538,7 +2538,7 @@ func (s *testSessionSuite) TestXAggregateWithoutAggFunc(c *C) {
drop table IF EXISTS t;
CREATE TABLE t (c INT);
INSERT INTO t VALUES(1), (2), (3), (3);`
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecMultiSQL(c, se, initSQL)
sql := "SELECT 18 FROM t group by c;"
@ -2565,7 +2565,7 @@ func (s *testSessionSuite) TestSelectHaving(c *C) {
insert INTO %s VALUES (1, "hello");
insert into %s VALUES (2, "hello");`,
table, table, table, table)
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecMultiSQL(c, se, initSQL)
sql := "select id, name from select_having_test where id in (1,3) having name like 'he%';"
@ -2575,7 +2575,7 @@ func (s *testSessionSuite) TestSelectHaving(c *C) {
}
func (s *testSessionSuite) TestQueryString(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecute(se, "use "+s.dbName)
_, err := se.Execute("create table mutil1 (a int);create table multi2 (a int)")
@ -2587,7 +2587,7 @@ func (s *testSessionSuite) TestQueryString(c *C) {
// Test that the auto_increment ID does not reuse the old table's allocator.
func (s *testSessionSuite) TestTruncateAlloc(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
mustExecute(se, "use "+s.dbName)
_, err := se.Execute("create table truncate_id (a int primary key auto_increment)")
@ -2604,7 +2604,7 @@ func (s *testSessionSuite) TestTruncateAlloc(c *C) {
// Test infomation_schema.columns.
func (s *testSessionSuite) TestISColumns(c *C) {
defer testleak.AfterTest(c)()
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
sql := "select ORDINAL_POSITION from INFORMATION_SCHEMA.COLUMNS;"
mustExecSQL(c, se, sql)

View File

@ -78,6 +78,8 @@ func (s *testBinlogSuite) SetUpSuite(c *C) {
store, err := tikv.NewMockTikvStore()
c.Assert(err, IsNil)
s.store = store
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
tidb.SetSchemaLease(0)
s.unixFile = "/tmp/mock-binlog-pump"
os.Remove(s.unixFile)
@ -117,11 +119,13 @@ func (s *testBinlogSuite) TestBinlog(c *C) {
var matched bool // got matched pre DDL and commit DDL
for i := 0; i < 10; i++ {
preDDL, commitDDL := getLatestDDLBinlog(c, pump, ddlQuery)
if preDDL.DdlJobId == commitDDL.DdlJobId {
c.Assert(commitDDL.StartTs, Equals, preDDL.StartTs)
c.Assert(commitDDL.CommitTs, Greater, commitDDL.StartTs)
matched = true
break
if preDDL != nil && commitDDL != nil {
if preDDL.DdlJobId == commitDDL.DdlJobId {
c.Assert(commitDDL.StartTs, Equals, preDDL.StartTs)
c.Assert(commitDDL.CommitTs, Greater, commitDDL.StartTs)
matched = true
break
}
}
time.Sleep(time.Millisecond * 10)
}

View File

@ -29,7 +29,7 @@ type testGCWorkerSuite struct {
var _ = Suite(&testGCWorkerSuite{})
func (s *testGCWorkerSuite) SetUpTest(c *C) {
s.store = newTestStore(c)
s.store = newTestStoreWithBootstrap(c)
s.oracle = &mockOracle{}
s.store.oracle = s.oracle
gcWorker, err := NewGCWorker(s.store)

View File

@ -46,6 +46,8 @@ func (s *testStoreSuite) SetUpTest(c *C) {
store, err := newTikvStore("mock-tikv-store", pdCli, clientFactory, false)
c.Assert(err, IsNil)
s.store = store
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
}
func (s *testStoreSuite) TestParsePath(c *C) {

View File

@ -19,6 +19,7 @@ import (
"time"
. "github.com/pingcap/check"
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/util/codec"
)
@ -44,6 +45,13 @@ func newTestStore(c *C) *tikvStore {
return store.(*tikvStore)
}
func newTestStoreWithBootstrap(c *C) *tikvStore {
store := newTestStore(c)
err := tidb.BootstrapSession(store)
c.Assert(err, IsNil)
return store
}
type testTiclientSuite struct {
store *tikvStore
// prefix is prefix of each key in this test. It is used for table isolation,

View File

@ -49,6 +49,8 @@ func (ts *testSuite) SetUpSuite(c *C) {
store, err := driver.Open("memory")
c.Check(err, IsNil)
ts.store = store
err = tidb.BootstrapSession(store)
c.Assert(err, IsNil)
ts.se, err = tidb.CreateSession(ts.store)
c.Assert(err, IsNil)
}

View File

@ -111,12 +111,11 @@ func main() {
createBinlogClient()
}
// Create a session to load information schema.
se, err := tidb.CreateSession(store)
// Bootstrap a session to load information schema.
err := tidb.BootstrapSession(store)
if err != nil {
log.Fatal(errors.ErrorStack(err))
}
se.Close()
var driver server.IDriver
driver = server.NewTiDBDriver(store)

View File

@ -91,7 +91,7 @@ func checkResult(c *C, se Session, affectedRows uint64, insertID uint64) {
func (s *testMainSuite) TestConcurrent(c *C) {
dbName := "test_concurrent_db"
defer removeStore(c, dbName)
store := newStore(c, dbName)
store := newStoreWithBootstrap(c, dbName)
se := newSession(c, store, dbName)
defer store.Close()
// create db
@ -123,7 +123,7 @@ func (s *testMainSuite) TestConcurrent(c *C) {
}
func (s *testMainSuite) TestTableInfoMeta(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
defer store.Close()
@ -157,7 +157,7 @@ func (s *testMainSuite) TestTableInfoMeta(c *C) {
}
func (s *testMainSuite) TestInfoSchema(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
rs := mustExecSQL(c, se, "SELECT CHARACTER_SET_NAME FROM INFORMATION_SCHEMA.CHARACTER_SETS WHERE CHARACTER_SET_NAME = 'utf8mb4'")
row, err := rs.Next()
@ -169,7 +169,7 @@ func (s *testMainSuite) TestInfoSchema(c *C) {
}
func (s *testMainSuite) TestCaseInsensitive(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
defer store.Close()
mustExecSQL(c, se, "create table T (a text, B int)")
@ -202,7 +202,7 @@ func (s *testMainSuite) TestCaseInsensitive(c *C) {
// Testcase for delete panic
func (s *testMainSuite) TestDeletePanic(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
defer store.Close()
mustExecSQL(c, se, "create table t (c int)")
@ -214,7 +214,7 @@ func (s *testMainSuite) TestDeletePanic(c *C) {
// Testcase for arg type.
func (s *testMainSuite) TestCheckArgs(c *C) {
store := newStore(c, s.dbName)
store := newStoreWithBootstrap(c, s.dbName)
se := newSession(c, store, s.dbName)
defer store.Close()
mustExecSQL(c, se, "create table if not exists t (c datetime)")
@ -267,7 +267,7 @@ func (s *testMainSuite) TestRetryOpenStore(c *C) {
// TODO: Merge TestIssue1435 in session test.
func (s *testMainSuite) TestSchemaValidity(c *C) {
localstore.MockRemoteStore = true
store := newStore(c, s.dbName+"schema_validity")
store := newStoreWithBootstrap(c, s.dbName+"schema_validity")
se := newSession(c, store, s.dbName)
se1 := newSession(c, store, s.dbName)
se2 := newSession(c, store, s.dbName)
@ -372,6 +372,12 @@ func newStore(c *C, dbPath string) kv.Storage {
return store
}
func newStoreWithBootstrap(c *C, dbPath string) kv.Storage {
store := newStore(c, dbPath)
BootstrapSession(store)
return store
}
var testConnID uint64
func newSession(c *C, store kv.Storage, dbName string) Session {