Skip to content

Commit 2e19b63

Browse files
MyonKemintaekexium
authored andcommitted
*: Update client-go and verify all read ts (pingcap#58054)
ref pingcap#57786
1 parent 3029ea6 commit 2e19b63

File tree

18 files changed

+174
-155
lines changed

18 files changed

+174
-155
lines changed

pkg/ddl/column_change_test.go

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,6 @@ import (
3535
"github.com/pingcap/tidb/pkg/testkit"
3636
"github.com/pingcap/tidb/pkg/testkit/external"
3737
"github.com/pingcap/tidb/pkg/types"
38-
"github.com/pingcap/tidb/pkg/util/mock"
3938
"github.com/stretchr/testify/require"
4039
)
4140

@@ -51,7 +50,7 @@ func TestColumnAdd(t *testing.T) {
5150
d := dom.DDL()
5251
tc := &callback.TestDDLCallback{Do: dom}
5352

54-
ct := testNewContext(store)
53+
ct := testNewContext(t, store)
5554
// set up hook
5655
var (
5756
deleteOnlyTable table.Table
@@ -127,7 +126,7 @@ func TestColumnAdd(t *testing.T) {
127126
return
128127
}
129128
first = false
130-
sess := testNewContext(store)
129+
sess := testNewContext(t, store)
131130
err := sessiontxn.NewTxn(context.Background(), sess)
132131
require.NoError(t, err)
133132
_, err = writeOnlyTable.AddRecord(sess, types.MakeDatums(10, 10))
@@ -431,10 +430,8 @@ func testCheckJobDone(t *testing.T, store kv.Storage, jobID int64, isAdd bool) {
431430
}
432431
}
433432

434-
func testNewContext(store kv.Storage) sessionctx.Context {
435-
ctx := mock.NewContext()
436-
ctx.Store = store
437-
return ctx
433+
func testNewContext(t *testing.T, store kv.Storage) sessionctx.Context {
434+
return testkit.NewSession(t, store)
438435
}
439436

440437
func TestIssue40135(t *testing.T) {

pkg/ddl/column_test.go

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -167,7 +167,7 @@ func TestColumnBasic(t *testing.T) {
167167
tk.MustExec(fmt.Sprintf("insert into t1 values(%d, %d, %d)", i, 10*i, 100*i))
168168
}
169169

170-
ctx := testNewContext(store)
170+
ctx := testNewContext(t, store)
171171
err := sessiontxn.NewTxn(context.Background(), ctx)
172172
require.NoError(t, err)
173173

@@ -611,7 +611,7 @@ func checkPublicColumn(t *testing.T, ctx sessionctx.Context, tableID int64, newC
611611
}
612612

613613
func checkAddColumn(t *testing.T, state model.SchemaState, tableID int64, handle kv.Handle, newCol *table.Column, oldRow []types.Datum, columnValue interface{}, dom *domain.Domain, store kv.Storage, columnCnt int) {
614-
ctx := testNewContext(store)
614+
ctx := testNewContext(t, store)
615615
switch state {
616616
case model.StateNone:
617617
checkNoneColumn(t, ctx, tableID, handle, newCol, columnValue, dom)
@@ -655,7 +655,7 @@ func TestAddColumn(t *testing.T) {
655655
tableID = int64(tableIDi)
656656
tbl := testGetTable(t, dom, tableID)
657657

658-
ctx := testNewContext(store)
658+
ctx := testNewContext(t, store)
659659
err := sessiontxn.NewTxn(context.Background(), ctx)
660660
require.NoError(t, err)
661661
oldRow := types.MakeDatums(int64(1), int64(2), int64(3))
@@ -728,7 +728,7 @@ func TestAddColumns(t *testing.T) {
728728
tableID = int64(tableIDi)
729729
tbl := testGetTable(t, dom, tableID)
730730

731-
ctx := testNewContext(store)
731+
ctx := testNewContext(t, store)
732732
err := sessiontxn.NewTxn(context.Background(), ctx)
733733
require.NoError(t, err)
734734
oldRow := types.MakeDatums(int64(1), int64(2), int64(3))
@@ -791,7 +791,7 @@ func TestDropColumnInColumnTest(t *testing.T) {
791791
tableID = int64(tableIDi)
792792
tbl := testGetTable(t, dom, tableID)
793793

794-
ctx := testNewContext(store)
794+
ctx := testNewContext(t, store)
795795
colName := "c4"
796796
defaultColValue := int64(4)
797797
row := types.MakeDatums(int64(1), int64(2), int64(3))
@@ -852,7 +852,7 @@ func TestDropColumns(t *testing.T) {
852852
tableID = int64(tableIDi)
853853
tbl := testGetTable(t, dom, tableID)
854854

855-
ctx := testNewContext(store)
855+
ctx := testNewContext(t, store)
856856
err := sessiontxn.NewTxn(context.Background(), ctx)
857857
require.NoError(t, err)
858858

pkg/ddl/db_integration_test.go

Lines changed: 93 additions & 114 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ import (
1818
"bytes"
1919
"context"
2020
"fmt"
21+
"github.com/pingcap/tidb/pkg/planner/core"
2122
"math"
2223
"strconv"
2324
"strings"
@@ -41,7 +42,6 @@ import (
4142
"github.com/pingcap/tidb/pkg/parser/model"
4243
"github.com/pingcap/tidb/pkg/parser/mysql"
4344
"github.com/pingcap/tidb/pkg/parser/terror"
44-
"github.com/pingcap/tidb/pkg/planner/core"
4545
"github.com/pingcap/tidb/pkg/session"
4646
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
4747
"github.com/pingcap/tidb/pkg/sessionctx/variable"
@@ -1610,6 +1610,62 @@ func TestDefaultColumnWithRand(t *testing.T) {
16101610
tk.MustGetErrCode("CREATE TABLE t3 (c int, c1 int default a_function_not_supported_yet());", errno.ErrDefValGeneratedNamedFunctionIsNotAllowed)
16111611
}
16121612

1613+
// TestDefaultValueAsExpressions is used for tests that are inconvenient to place in the pkg/tests directory.
1614+
func TestDefaultValueAsExpressions(t *testing.T) {
1615+
store := testkit.CreateMockStoreWithSchemaLease(t, testLease)
1616+
tk := testkit.NewTestKit(t, store)
1617+
tk.MustExec("use test")
1618+
tk.MustExec("drop table if exists t, t1, t2")
1619+
1620+
// date_format
1621+
tk.MustExec("create table t6 (c int(10), c1 int default (date_format(now(),'%Y-%m-%d %H:%i:%s')))")
1622+
tk.MustExec("create table t7 (c int(10), c1 date default (date_format(now(),'%Y-%m')))")
1623+
// Error message like: Error 1292 (22007): Truncated incorrect DOUBLE value: '2024-03-05 16:37:25'.
1624+
tk.MustGetErrCode("insert into t6(c) values (1)", errno.ErrTruncatedWrongValue)
1625+
tk.MustGetErrCode("insert into t7(c) values (1)", errno.ErrTruncatedWrongValue)
1626+
1627+
// user
1628+
tk.MustExec("create table t (c int(10), c1 varchar(256) default (upper(substring_index(user(),'@',1))));")
1629+
tk.Session().GetSessionVars().User = &auth.UserIdentity{Username: "root", Hostname: "localhost"}
1630+
tk.MustExec("insert into t(c) values (1),(2),(3)")
1631+
tk.Session().GetSessionVars().User = &auth.UserIdentity{Username: "xyz", Hostname: "localhost"}
1632+
tk.MustExec("insert into t(c) values (4),(5),(6)")
1633+
tk.MustExec("insert into t values (7, default)")
1634+
rows := tk.MustQuery("SELECT c1 from t order by c").Rows()
1635+
for i, row := range rows {
1636+
d, ok := row[0].(string)
1637+
require.True(t, ok)
1638+
if i < 3 {
1639+
require.Equal(t, "ROOT", d)
1640+
} else {
1641+
require.Equal(t, "XYZ", d)
1642+
}
1643+
}
1644+
1645+
// replace
1646+
tk.MustExec("create table t1 (c int(10), c1 int default (REPLACE(UPPER(UUID()), '-', '')))")
1647+
// Different UUID values will result in different error code.
1648+
_, err := tk.Exec("insert into t1(c) values (1)")
1649+
originErr := errors.Cause(err)
1650+
tErr, ok := originErr.(*terror.Error)
1651+
require.Truef(t, ok, "expect type 'terror.Error', but obtain '%T': %v", originErr, originErr)
1652+
sqlErr := terror.ToSQLError(tErr)
1653+
if int(sqlErr.Code) != errno.ErrTruncatedWrongValue {
1654+
require.Equal(t, errno.ErrDataOutOfRange, int(sqlErr.Code))
1655+
}
1656+
// test modify column
1657+
// The error message has UUID, so put this test here.
1658+
tk.MustExec("create table t2(c int(10), c1 varchar(256) default (REPLACE(UPPER(UUID()), '-', '')), index idx(c1));")
1659+
tk.MustExec("insert into t2(c) values (1),(2),(3);")
1660+
tk.MustGetErrCode("alter table t2 modify column c1 varchar(30) default 'xx';", errno.WarnDataTruncated)
1661+
// test add column for enum
1662+
nowStr := time.Now().Format("2006-01")
1663+
sql := fmt.Sprintf("alter table t2 add column c3 enum('%v','n')", nowStr) + " default (date_format(now(),'%Y-%m'))"
1664+
tk.MustExec(sql)
1665+
tk.MustExec("insert into t2(c) values (4);")
1666+
tk.MustQuery("select c3 from t2").Check(testkit.Rows(nowStr, nowStr, nowStr, nowStr))
1667+
}
1668+
16131669
func TestChangingDBCharset(t *testing.T) {
16141670
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())
16151671

@@ -1821,8 +1877,6 @@ func TestParserIssue284(t *testing.T) {
18211877

18221878
func TestAddExpressionIndex(t *testing.T) {
18231879
config.UpdateGlobal(func(conf *config.Config) {
1824-
// Test for table lock.
1825-
conf.EnableTableLock = true
18261880
conf.Instance.SlowThreshold = 10000
18271881
conf.TiKVClient.AsyncCommit.SafeWindow = 0
18281882
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
@@ -1901,60 +1955,6 @@ func TestAddExpressionIndex(t *testing.T) {
19011955
})
19021956
}
19031957

1904-
func TestCreateExpressionIndexError(t *testing.T) {
1905-
config.UpdateGlobal(func(conf *config.Config) {
1906-
// Test for table lock.
1907-
conf.EnableTableLock = true
1908-
conf.Instance.SlowThreshold = 10000
1909-
conf.TiKVClient.AsyncCommit.SafeWindow = 0
1910-
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
1911-
conf.Experimental.AllowsExpressionIndex = true
1912-
})
1913-
store := testkit.CreateMockStore(t)
1914-
tk := testkit.NewTestKit(t, store)
1915-
tk.MustExec("use test")
1916-
tk.MustExec("drop table if exists t;")
1917-
tk.MustExec("create table t (a int, b real);")
1918-
tk.MustGetErrCode("alter table t add primary key ((a+b)) nonclustered;", errno.ErrFunctionalIndexPrimaryKey)
1919-
1920-
tk.MustGetErrCode("create table t(a int, index((cast(a as JSON))))", errno.ErrFunctionalIndexOnJSONOrGeometryFunction)
1921-
1922-
// Test for error
1923-
tk.MustExec("drop table if exists t;")
1924-
tk.MustExec("create table t (a int, b real);")
1925-
tk.MustGetErrCode("alter table t add primary key ((a+b)) nonclustered;", errno.ErrFunctionalIndexPrimaryKey)
1926-
tk.MustGetErrCode("alter table t add index ((rand()));", errno.ErrFunctionalIndexFunctionIsNotAllowed)
1927-
tk.MustGetErrCode("alter table t add index ((now()+1));", errno.ErrFunctionalIndexFunctionIsNotAllowed)
1928-
1929-
tk.MustExec("alter table t add column (_V$_idx_0 int);")
1930-
tk.MustGetErrCode("alter table t add index idx((a+1));", errno.ErrDupFieldName)
1931-
tk.MustExec("alter table t drop column _V$_idx_0;")
1932-
tk.MustExec("alter table t add index idx((a+1));")
1933-
tk.MustGetErrCode("alter table t add column (_V$_idx_0 int);", errno.ErrDupFieldName)
1934-
tk.MustExec("alter table t drop index idx;")
1935-
tk.MustExec("alter table t add column (_V$_idx_0 int);")
1936-
1937-
tk.MustExec("alter table t add column (_V$_expression_index_0 int);")
1938-
tk.MustGetErrCode("alter table t add index ((a+1));", errno.ErrDupFieldName)
1939-
tk.MustExec("alter table t drop column _V$_expression_index_0;")
1940-
tk.MustExec("alter table t add index ((a+1));")
1941-
tk.MustGetErrCode("alter table t drop column _V$_expression_index_0;", errno.ErrCantDropFieldOrKey)
1942-
tk.MustGetErrCode("alter table t add column e int as (_V$_expression_index_0 + 1);", errno.ErrBadField)
1943-
1944-
// NOTE (#18150): In creating expression index, row value is not allowed.
1945-
tk.MustExec("drop table if exists t;")
1946-
tk.MustGetErrCode("create table t (j json, key k (((j,j))))", errno.ErrFunctionalIndexRowValueIsNotAllowed)
1947-
tk.MustExec("create table t (j json, key k ((j+1),(j+1)))")
1948-
1949-
tk.MustGetErrCode("create table t1 (col1 int, index ((concat(''))));", errno.ErrWrongKeyColumnFunctionalIndex)
1950-
tk.MustGetErrCode("CREATE TABLE t1 (col1 INT, PRIMARY KEY ((ABS(col1))) NONCLUSTERED);", errno.ErrFunctionalIndexPrimaryKey)
1951-
1952-
// For issue 26349
1953-
tk.MustExec("drop table if exists t;")
1954-
tk.MustExec("create table t(id char(10) primary key, short_name char(10), name char(10), key n((upper(`name`))));")
1955-
tk.MustExec("update t t1 set t1.short_name='a' where t1.id='1';")
1956-
}
1957-
19581958
func queryIndexOnTable(dbName, tableName string) string {
19591959
return fmt.Sprintf("select distinct index_name, is_visible from information_schema.statistics where table_schema = '%s' and table_name = '%s' order by index_name", dbName, tableName)
19601960
}
@@ -2353,20 +2353,6 @@ func TestEnumAndSetDefaultValue(t *testing.T) {
23532353
require.Equal(t, "a", tbl.Meta().Columns[1].DefaultValue)
23542354
}
23552355

2356-
func TestStrictDoubleTypeCheck(t *testing.T) {
2357-
store := testkit.CreateMockStore(t)
2358-
tk := testkit.NewTestKit(t, store)
2359-
tk.MustExec("use test")
2360-
tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'")
2361-
sql := "create table double_type_check(id int, c double(10));"
2362-
_, err := tk.Exec(sql)
2363-
require.Error(t, err)
2364-
require.Equal(t, "[parser:1149]You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use", err.Error())
2365-
tk.MustExec("set @@tidb_enable_strict_double_type_check = 'OFF'")
2366-
defer tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'")
2367-
tk.MustExec(sql)
2368-
}
2369-
23702356
func TestDuplicateErrorMessage(t *testing.T) {
23712357
defer collate.SetNewCollationEnabledForTest(true)
23722358
store := testkit.CreateMockStore(t)
@@ -2388,10 +2374,7 @@ func TestDuplicateErrorMessage(t *testing.T) {
23882374
for _, newCollate := range []bool{false, true} {
23892375
collate.SetNewCollationEnabledForTest(newCollate)
23902376
for _, globalIndex := range []bool{false, true} {
2391-
restoreConfig := config.RestoreFunc()
2392-
config.UpdateGlobal(func(conf *config.Config) {
2393-
conf.EnableGlobalIndex = globalIndex
2394-
})
2377+
tk.MustExec(fmt.Sprintf("set tidb_enable_global_index=%t", globalIndex))
23952378
for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} {
23962379
tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex
23972380
for _, t := range tests {
@@ -2418,7 +2401,7 @@ func TestDuplicateErrorMessage(t *testing.T) {
24182401
fmt.Sprintf("[kv:1062]Duplicate entry '1-%s' for key 't.t_idx'", strings.Join(fields, "-")))
24192402
}
24202403
}
2421-
restoreConfig()
2404+
tk.MustExec("set tidb_enable_global_index=default")
24222405
}
24232406
}
24242407
}
@@ -2673,8 +2656,6 @@ func TestAvoidCreateViewOnLocalTemporaryTable(t *testing.T) {
26732656

26742657
func TestDropTemporaryTable(t *testing.T) {
26752658
config.UpdateGlobal(func(conf *config.Config) {
2676-
// Test for table lock.
2677-
conf.EnableTableLock = true
26782659
conf.Instance.SlowThreshold = 10000
26792660
conf.TiKVClient.AsyncCommit.SafeWindow = 0
26802661
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
@@ -2940,42 +2921,6 @@ func TestIssue29282(t *testing.T) {
29402921
}
29412922
}
29422923

2943-
// See https://github.com/pingcap/tidb/issues/35644
2944-
func TestCreateTempTableInTxn(t *testing.T) {
2945-
store := testkit.CreateMockStore(t)
2946-
tk := testkit.NewTestKit(t, store)
2947-
tk.MustExec("use test")
2948-
tk.MustExec("begin")
2949-
// new created temporary table should be visible
2950-
tk.MustExec("create temporary table t1(id int primary key, v int)")
2951-
tk.MustQuery("select * from t1").Check(testkit.Rows())
2952-
// new inserted data should be visible
2953-
tk.MustExec("insert into t1 values(123, 456)")
2954-
tk.MustQuery("select * from t1 where id=123").Check(testkit.Rows("123 456"))
2955-
// truncate table will clear data but table still visible
2956-
tk.MustExec("truncate table t1")
2957-
tk.MustQuery("select * from t1 where id=123").Check(testkit.Rows())
2958-
tk.MustExec("commit")
2959-
2960-
tk1 := testkit.NewTestKit(t, store)
2961-
tk1.MustExec("use test")
2962-
tk1.MustExec("create table tt(id int)")
2963-
tk1.MustExec("begin")
2964-
tk1.MustExec("create temporary table t1(id int)")
2965-
tk1.MustExec("insert into tt select * from t1")
2966-
tk1.MustExec("drop table tt")
2967-
2968-
tk2 := testkit.NewTestKit(t, store)
2969-
tk2.MustExec("use test")
2970-
tk2.MustExec("create table t2(id int primary key, v int)")
2971-
tk2.MustExec("insert into t2 values(234, 567)")
2972-
tk2.MustExec("begin")
2973-
// create a new temporary table with the same name will override physical table
2974-
tk2.MustExec("create temporary table t2(id int primary key, v int)")
2975-
tk2.MustQuery("select * from t2 where id=234").Check(testkit.Rows())
2976-
tk2.MustExec("commit")
2977-
}
2978-
29792924
// See https://github.com/pingcap/tidb/issues/29327
29802925
func TestEnumDefaultValue(t *testing.T) {
29812926
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())
@@ -3071,6 +3016,12 @@ func TestDefaultCollationForUTF8MB4(t *testing.T) {
30713016
"dby CREATE DATABASE `dby` /*!40100 DEFAULT CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci */"))
30723017
}
30733018

3019+
func TestOptimizeTable(t *testing.T) {
3020+
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())
3021+
tk := testkit.NewTestKit(t, store)
3022+
tk.MustGetErrMsg("optimize table t", "[ddl:8200]OPTIMIZE TABLE is not supported")
3023+
}
3024+
30743025
func TestIssue52680(t *testing.T) {
30753026
store, dom := testkit.CreateMockStoreAndDomain(t)
30763027
tk := testkit.NewTestKit(t, store)
@@ -3135,3 +3086,31 @@ func TestIssue52680(t *testing.T) {
31353086
tk.MustExec("insert into issue52680 values(default);")
31363087
tk.MustQuery("select * from issue52680").Check(testkit.Rows("1", "2", "3"))
31373088
}
3089+
3090+
func TestCreateIndexWithChangeMaxIndexLength(t *testing.T) {
3091+
store, dom := testkit.CreateMockStoreAndDomain(t)
3092+
originCfg := config.GetGlobalConfig()
3093+
defer func() {
3094+
config.StoreGlobalConfig(originCfg)
3095+
}()
3096+
3097+
originHook := dom.DDL().GetHook()
3098+
defer dom.DDL().SetHook(originHook)
3099+
hook := &callback.TestDDLCallback{Do: dom}
3100+
hook.OnJobRunBeforeExported = func(job *model.Job) {
3101+
if job.Type != model.ActionAddIndex {
3102+
return
3103+
}
3104+
if job.SchemaState == model.StateNone {
3105+
newCfg := *originCfg
3106+
newCfg.MaxIndexLength = 1000
3107+
config.StoreGlobalConfig(&newCfg)
3108+
}
3109+
}
3110+
dom.DDL().SetHook(hook)
3111+
3112+
tk := testkit.NewTestKit(t, store)
3113+
tk.MustExec("use test;")
3114+
tk.MustExec("create table t(id int, a json DEFAULT NULL, b varchar(2) DEFAULT NULL);")
3115+
tk.MustGetErrMsg("CREATE INDEX idx_test on t ((cast(a as char(2000) array)),b);", "[ddl:1071]Specified key was too long (2000 bytes); max key length is 1000 bytes")
3116+
}

pkg/ddl/ddl_worker_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -51,15 +51,15 @@ func TestInvalidDDLJob(t *testing.T) {
5151
BinlogInfo: &model.HistoryInfo{},
5252
Args: []interface{}{},
5353
}
54-
ctx := testNewContext(store)
54+
ctx := testNewContext(t, store)
5555
ctx.SetValue(sessionctx.QueryString, "skip")
5656
err := dom.DDL().DoDDLJob(ctx, job)
5757
require.Equal(t, err.Error(), "[ddl:8204]invalid ddl job type: none")
5858
}
5959

6060
func TestAddBatchJobError(t *testing.T) {
6161
store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease)
62-
ctx := testNewContext(store)
62+
ctx := testNewContext(t, store)
6363

6464
require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/mockAddBatchDDLJobsErr", `return(true)`))
6565
// Test the job runner should not hang forever.

0 commit comments

Comments
 (0)