Skip to content

Commit

Permalink
Merge branch 'release-5.1' into release-5.1-ddfc0bd8cf3a
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 committed Jul 20, 2021
2 parents 899ccfc + 858fec5 commit 4f1fafa
Show file tree
Hide file tree
Showing 113 changed files with 3,137 additions and 450 deletions.
16 changes: 16 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -2098,3 +2099,18 @@ func (s *testSuite) TestBindingWithoutCharset(c *C) {
c.Assert(rows[0][0], Equals, "select * from `test` . `t` where `a` = ?")
c.Assert(rows[0][1], Equals, "SELECT * FROM `test`.`t` WHERE `a` = 'aa'")
}

func (s *testSuite) TestTemporaryTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("set tidb_enable_global_temporary_table = true")
tk.MustExec("create global temporary table t(a int, b int, key(a), key(b)) on commit delete rows")
tk.MustExec("create table t2(a int, b int, key(a), key(b))")
tk.MustGetErrCode("create session binding for select * from t where b = 123 using select * from t ignore index(b) where b = 123;", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for insert into t select * from t2 where t2.b = 1 and t2.c > 1 using insert into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for replace into t select * from t2 where t2.b = 1 and t2.c > 1 using replace into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for update t set a = 1 where b = 1 and c > 1 using update /*+ use_index(t, c) */ t set a = 1 where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for delete from t where b = 1 and c > 1 using delete /*+ use_index(t, c) */ from t where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
}
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -653,7 +653,7 @@ var defaultConf = Config{
StmtSummary: StmtSummary{
Enable: true,
EnableInternalQuery: false,
MaxStmtCount: 200,
MaxStmtCount: 3000,
MaxSQLLength: 4096,
RefreshInterval: 1800,
HistorySize: 24,
Expand Down
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -448,7 +448,7 @@ enable = true
enable-internal-query = false

# max number of statements kept in memory.
max-stmt-count = 200
max-stmt-count = 3000

# max length of displayed normalized sql and sample sql.
max-sql-length = 4096
Expand Down
4 changes: 2 additions & 2 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1678,7 +1678,7 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol
tk1 := testkit.NewTestKit(c, s.store)
tk1.MustExec("use test")

tk.MustExec(fmt.Sprintf("set time_zone = 'UTC'"))
tk.MustExec("set time_zone = 'UTC'")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int not null, unique key(a))")
tk.MustExec("insert into t values(1, 1)")
Expand Down Expand Up @@ -1763,7 +1763,7 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol
tk1 := testkit.NewTestKit(c, s.store)
tk1.MustExec("use test")

tk.MustExec(fmt.Sprintf("set time_zone = 'UTC'"))
tk.MustExec("set time_zone = 'UTC'")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a VARCHAR(31) NULL DEFAULT 'wwrzfwzb01j6ddj', b DECIMAL(12,0) NULL DEFAULT '-729850476163')")
tk.MustExec("ALTER TABLE t ADD COLUMN x CHAR(218) NULL DEFAULT 'lkittuae'")
Expand Down
69 changes: 69 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,75 @@ func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) {
testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey)
}

func (s *testSerialDBSuite) TestWriteReorgForColumnTypeChangeOnAmendTxn(c *C) {
tk2 := testkit.NewTestKit(c, s.store)
tk2.MustExec("use test_db")
tk2.MustExec("set global tidb_enable_amend_pessimistic_txn = ON;")
defer func() {
tk2.MustExec("set global tidb_enable_amend_pessimistic_txn = OFF;")
}()

d := s.dom.DDL()
originalHook := d.GetHook()
defer d.(ddl.DDLForTest).SetHook(originalHook)
testInsertOnModifyColumn := func(sql string, startColState, commitColState model.SchemaState, retStrs []string, retErr error) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test_db")
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (c1 int, c2 int, c3 int, unique key(c1))")
tk.MustExec("insert into t1 values (20, 20, 20);")

var checkErr error
tk1 := testkit.NewTestKit(c, s.store)
hook := &ddl.TestDDLCallback{Do: s.dom}
times := 0
hook.OnJobUpdatedExported = func(job *model.Job) {
if job.Type != model.ActionModifyColumn || checkErr != nil ||
(job.SchemaState != startColState && job.SchemaState != commitColState) {
return
}

if job.SchemaState == startColState {
tk1.MustExec("use test_db")
tk1.MustExec("begin pessimistic;")
tk1.MustExec("insert into t1 values(101, 102, 103)")
return
}
if times == 0 {
_, checkErr = tk1.Exec("commit;")
}
times++
}
d.(ddl.DDLForTest).SetHook(hook)

tk.MustExec(sql)
if retErr == nil {
c.Assert(checkErr, IsNil)
} else {
c.Assert(strings.Contains(checkErr.Error(), retErr.Error()), IsTrue)
}
tk.MustQuery("select * from t1;").Check(testkit.Rows(retStrs...))

tk.MustExec("admin check table t1")
}

// Testing it needs reorg data.
ddlStatement := "alter table t1 change column c2 cc smallint;"
testInsertOnModifyColumn(ddlStatement, model.StateNone, model.StateWriteReorganization, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)
testInsertOnModifyColumn(ddlStatement, model.StateDeleteOnly, model.StateWriteReorganization, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)
testInsertOnModifyColumn(ddlStatement, model.StateWriteOnly, model.StateWriteReorganization, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)
testInsertOnModifyColumn(ddlStatement, model.StateNone, model.StatePublic, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)
testInsertOnModifyColumn(ddlStatement, model.StateDeleteOnly, model.StatePublic, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)
testInsertOnModifyColumn(ddlStatement, model.StateWriteOnly, model.StatePublic, []string{"20 20 20"}, domain.ErrInfoSchemaChanged)

// Testing it needs not reorg data. This case only have two state: none, public.
ddlStatement = "alter table t1 change column c2 cc bigint;"
testInsertOnModifyColumn(ddlStatement, model.StateNone, model.StateWriteReorganization, []string{"20 20 20"}, nil)
testInsertOnModifyColumn(ddlStatement, model.StateWriteOnly, model.StateWriteReorganization, []string{"20 20 20"}, nil)
testInsertOnModifyColumn(ddlStatement, model.StateNone, model.StatePublic, []string{"20 20 20", "101 102 103"}, nil)
testInsertOnModifyColumn(ddlStatement, model.StateWriteOnly, model.StatePublic, []string{"20 20 20"}, nil)
}

func (s *testSerialDBSuite) TestAddExpressionIndexRollback(c *C) {
config.UpdateGlobal(func(conf *config.Config) {
conf.Experimental.AllowsExpressionIndex = true
Expand Down
46 changes: 28 additions & 18 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1642,12 +1642,15 @@ func checkTableInfoValid(tblInfo *model.TableInfo) error {
return checkInvisibleIndexOnPK(tblInfo)
}

func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) {
func buildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) {
// Check the referred table is a real table object.
if referTblInfo.IsSequence() || referTblInfo.IsView() {
return nil, ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE")
}
tblInfo := *referTblInfo
if err := setTemporaryType(ctx, &tblInfo, s); err != nil {
return nil, errors.Trace(err)
}
// Check non-public column and adjust column offset.
newColumns := referTblInfo.Cols()
newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices))
Expand Down Expand Up @@ -1735,22 +1738,8 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh
if err != nil {
return nil, errors.Trace(err)
}
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
if !ctx.GetSessionVars().EnableGlobalTemporaryTable {
return nil, errors.New("global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table")
}
// "create global temporary table ... on commit preserve rows"
if !s.OnCommitDelete {
return nil, errors.Trace(errUnsupportedOnCommitPreserve)
}
case ast.TemporaryLocal:
// TODO: set "tbInfo.TempTableType = model.TempTableLocal" after local temporary table is supported.
tbInfo.TempTableType = model.TempTableNone
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored"))
case ast.TemporaryNone:
tbInfo.TempTableType = model.TempTableNone
if err = setTemporaryType(ctx, tbInfo, s); err != nil {
return nil, errors.Trace(err)
}

if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil {
Expand Down Expand Up @@ -1812,7 +1801,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
// build tableInfo
var tbInfo *model.TableInfo
if s.ReferTable != nil {
tbInfo, err = buildTableInfoWithLike(ident, referTbl.Meta(), s)
tbInfo, err = buildTableInfoWithLike(ctx, ident, referTbl.Meta(), s)
} else {
tbInfo, err = buildTableInfoWithStmt(ctx, s, schema.Charset, schema.Collate)
}
Expand All @@ -1832,6 +1821,27 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
return d.CreateTableWithInfo(ctx, schema.Name, tbInfo, onExist, false /*tryRetainID*/)
}

func setTemporaryType(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) error {
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
if !ctx.GetSessionVars().EnableGlobalTemporaryTable {
return errors.New("global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table")
}
// "create global temporary table ... on commit preserve rows"
if !s.OnCommitDelete {
return errors.Trace(errUnsupportedOnCommitPreserve)
}
case ast.TemporaryLocal:
// TODO: set "tbInfo.TempTableType = model.TempTableLocal" after local temporary table is supported.
tbInfo.TempTableType = model.TempTableNone
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored"))
default:
tbInfo.TempTableType = model.TempTableNone
}
return nil
}

func (d *ddl) CreateTableWithInfo(
ctx sessionctx.Context,
dbName model.CIStr,
Expand Down
113 changes: 112 additions & 1 deletion ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -525,16 +525,127 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) {

tk.MustExec("drop database ctwl_db")
tk.MustExec("drop database ctwl_db1")
}

func (s *testSerialSuite) TestCreateTableWithLikeAtTemporaryMode(c *C) {
tk := testkit.NewTestKit(c, s.store)

// Test create table like at temporary mode.
tk.MustExec("set tidb_enable_global_temporary_table=true")
tk.MustExec("use test")
tk.MustExec("drop table if exists temporary_table;")
tk.MustExec("create global temporary table temporary_table (a int, b int,index(a)) on commit delete rows")
tk.MustExec("drop table if exists temporary_table_t1;")
_, err = tk.Exec("create table temporary_table_t1 like temporary_table")
_, err := tk.Exec("create table temporary_table_t1 like temporary_table")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
tk.MustExec("drop table if exists temporary_table;")

// Test create temporary table like.
// Test auto_random.
tk.MustExec("drop table if exists auto_random_table")
_, err = tk.Exec("create table auto_random_table (a bigint primary key auto_random(3), b varchar(255));")
defer tk.MustExec("drop table if exists auto_random_table")
tk.MustExec("drop table if exists auto_random_temporary_global")
_, err = tk.Exec("create global temporary table auto_random_temporary_global like auto_random_table on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random").Error())

// Test pre split regions.
tk.MustExec("drop table if exists table_pre_split")
_, err = tk.Exec("create table table_pre_split(id int) shard_row_id_bits = 2 pre_split_regions=2;")
defer tk.MustExec("drop table if exists table_pre_split")
tk.MustExec("drop table if exists temporary_table_pre_split")
_, err = tk.Exec("create global temporary table temporary_table_pre_split like table_pre_split ON COMMIT DELETE ROWS;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions").Error())

// Test shard_row_id_bits.
tk.MustExec("drop table if exists shard_row_id_table, shard_row_id_temporary_table, shard_row_id_table_plus, shard_row_id_temporary_table_plus")
_, err = tk.Exec("create table shard_row_id_table (a int) shard_row_id_bits = 5;")
_, err = tk.Exec("create global temporary table shard_row_id_temporary_table like shard_row_id_table on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error())
tk.MustExec("create table shard_row_id_table_plus (a int);")
tk.MustExec("create global temporary table shard_row_id_temporary_table_plus (a int) on commit delete rows;")
defer tk.MustExec("drop table if exists shard_row_id_table, shard_row_id_temporary_table, shard_row_id_table_plus, shard_row_id_temporary_table_plus")
_, err = tk.Exec("alter table shard_row_id_temporary_table_plus shard_row_id_bits = 4;")
c.Assert(err.Error(), Equals, ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error())

// Test partition.
tk.MustExec("drop table if exists global_partition_table;")
tk.MustExec("create table global_partition_table (a int, b int) partition by hash(a) partitions 3;")
defer tk.MustExec("drop table if exists global_partition_table;")
tk.MustGetErrCode("create global temporary table global_partition_temp_table like global_partition_table ON COMMIT DELETE ROWS;",
errno.ErrPartitionNoTemporary)
// Test virtual columns.
tk.MustExec("drop table if exists test_gv_ddl, test_gv_ddl_temp")
tk.MustExec(`create table test_gv_ddl(a int, b int as (a+8) virtual, c int as (b + 2) stored)`)
tk.MustExec(`create global temporary table test_gv_ddl_temp like test_gv_ddl on commit delete rows;`)
defer tk.MustExec("drop table if exists test_gv_ddl_temp, test_gv_ddl")
is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_gv_ddl"))
c.Assert(err, IsNil)
testCases := []struct {
generatedExprString string
generatedStored bool
}{
{"", false},
{"`a` + 8", false},
{"`b` + 2", true},
}
for i, column := range table.Meta().Columns {
c.Assert(column.GeneratedExprString, Equals, testCases[i].generatedExprString)
c.Assert(column.GeneratedStored, Equals, testCases[i].generatedStored)
}
result := tk.MustQuery(`DESC test_gv_ddl_temp`)
result.Check(testkit.Rows(`a int(11) YES <nil> `, `b int(11) YES <nil> VIRTUAL GENERATED`, `c int(11) YES <nil> STORED GENERATED`))
tk.MustExec("begin;")
tk.MustExec("insert into test_gv_ddl_temp values (1, default, default)")
tk.MustQuery("select * from test_gv_ddl_temp").Check(testkit.Rows("1 9 11"))
_, err = tk.Exec("commit")
c.Assert(err, IsNil)

// Test foreign key.
tk.MustExec("drop table if exists test_foreign_key, t1")
tk.MustExec("create table t1 (a int, b int);")
tk.MustExec("create table test_foreign_key (c int,d int,foreign key (d) references t1 (b));")
defer tk.MustExec("drop table if exists test_foreign_key, t1;")
tk.MustExec("create global temporary table test_foreign_key_temp like test_foreign_key on commit delete rows;")
is = tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_foreign_key_temp"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
c.Assert(len(tableInfo.ForeignKeys), Equals, 0)

// Issue 25613.
// Test from->normal, to->normal.
tk.MustExec("drop table if exists tb1, tb2")
tk.MustExec("create table tb1(id int);")
tk.MustExec("create table tb2 like tb1")
defer tk.MustExec("drop table if exists tb1, tb2")
tk.MustQuery("show create table tb2;").Check(testkit.Rows("tb2 CREATE TABLE `tb2` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

// Test from->normal, to->global temporary.
tk.MustExec("drop table if exists tb3, tb4")
tk.MustExec("create table tb3(id int);")
tk.MustExec("create global temporary table tb4 like tb3 on commit delete rows;")
defer tk.MustExec("drop table if exists tb3, tb4")
tk.MustQuery("show create table tb4;").Check(testkit.Rows("tb4 CREATE GLOBAL TEMPORARY TABLE `tb4` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS"))

// Test from->global temporary, to->normal.
tk.MustExec("drop table if exists tb5, tb6")
tk.MustExec("create global temporary table tb5(id int) on commit delete rows;")
_, err = tk.Exec("create table tb6 like tb5;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
defer tk.MustExec("drop table if exists tb5, tb6")

// Test from->global temporary, to->global temporary.
tk.MustExec("drop table if exists tb7, tb8")
tk.MustExec("create global temporary table tb7(id int) on commit delete rows;")
_, err = tk.Exec("create global temporary table tb8 like tb7 on commit delete rows;")
c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error())
defer tk.MustExec("drop table if exists tb7, tb8")
}

// TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started.
Expand Down
4 changes: 2 additions & 2 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,13 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/oracle"
)

// RequestBuilder is used to build a "kv.Request".
Expand Down Expand Up @@ -62,7 +62,7 @@ func (builder *RequestBuilder) Build() (*kv.Request, error) {
assertScope := val.(string)
if len(assertScope) > 0 {
if builder.IsStaleness && assertScope != builder.TxnScope {
panic("batch point get staleness option fail")
panic("request builder get staleness option fail")
}
}
})
Expand Down
2 changes: 1 addition & 1 deletion distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand All @@ -35,7 +36,6 @@ import (
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/oracle"
)

var _ = Suite(&testSuite{})
Expand Down
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -923,6 +923,7 @@ const (
ErrTxnTooLarge = 8004
ErrWriteConflictInTiDB = 8005
ErrOptOnTemporaryTable = 8006
ErrDropTableOnTemporaryTable = 8007
ErrUnsupportedReloadPlugin = 8018
ErrUnsupportedReloadPluginVar = 8019
ErrTableLocked = 8020
Expand Down
Loading

0 comments on commit 4f1fafa

Please sign in to comment.