Skip to content

Commit

Permalink
*: Update client-go and verify all read ts (#58054)
Browse files Browse the repository at this point in the history
ref #57786
  • Loading branch information
MyonKeminta authored and ekexium committed Jan 15, 2025
1 parent 3029ea6 commit 2e19b63
Show file tree
Hide file tree
Showing 18 changed files with 174 additions and 155 deletions.
11 changes: 4 additions & 7 deletions pkg/ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ import (
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/external"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/mock"
"github.com/stretchr/testify/require"
)

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

ct := testNewContext(store)
ct := testNewContext(t, store)
// set up hook
var (
deleteOnlyTable table.Table
Expand Down Expand Up @@ -127,7 +126,7 @@ func TestColumnAdd(t *testing.T) {
return
}
first = false
sess := testNewContext(store)
sess := testNewContext(t, store)
err := sessiontxn.NewTxn(context.Background(), sess)
require.NoError(t, err)
_, err = writeOnlyTable.AddRecord(sess, types.MakeDatums(10, 10))
Expand Down Expand Up @@ -431,10 +430,8 @@ func testCheckJobDone(t *testing.T, store kv.Storage, jobID int64, isAdd bool) {
}
}

func testNewContext(store kv.Storage) sessionctx.Context {
ctx := mock.NewContext()
ctx.Store = store
return ctx
func testNewContext(t *testing.T, store kv.Storage) sessionctx.Context {
return testkit.NewSession(t, store)
}

func TestIssue40135(t *testing.T) {
Expand Down
12 changes: 6 additions & 6 deletions pkg/ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ func TestColumnBasic(t *testing.T) {
tk.MustExec(fmt.Sprintf("insert into t1 values(%d, %d, %d)", i, 10*i, 100*i))
}

ctx := testNewContext(store)
ctx := testNewContext(t, store)
err := sessiontxn.NewTxn(context.Background(), ctx)
require.NoError(t, err)

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

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) {
ctx := testNewContext(store)
ctx := testNewContext(t, store)
switch state {
case model.StateNone:
checkNoneColumn(t, ctx, tableID, handle, newCol, columnValue, dom)
Expand Down Expand Up @@ -655,7 +655,7 @@ func TestAddColumn(t *testing.T) {
tableID = int64(tableIDi)
tbl := testGetTable(t, dom, tableID)

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

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

ctx := testNewContext(store)
ctx := testNewContext(t, store)
colName := "c4"
defaultColValue := int64(4)
row := types.MakeDatums(int64(1), int64(2), int64(3))
Expand Down Expand Up @@ -852,7 +852,7 @@ func TestDropColumns(t *testing.T) {
tableID = int64(tableIDi)
tbl := testGetTable(t, dom, tableID)

ctx := testNewContext(store)
ctx := testNewContext(t, store)
err := sessiontxn.NewTxn(context.Background(), ctx)
require.NoError(t, err)

Expand Down
207 changes: 93 additions & 114 deletions pkg/ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"bytes"
"context"
"fmt"
"github.com/pingcap/tidb/pkg/planner/core"
"math"
"strconv"
"strings"
Expand All @@ -41,7 +42,6 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
Expand Down Expand Up @@ -1610,6 +1610,62 @@ func TestDefaultColumnWithRand(t *testing.T) {
tk.MustGetErrCode("CREATE TABLE t3 (c int, c1 int default a_function_not_supported_yet());", errno.ErrDefValGeneratedNamedFunctionIsNotAllowed)
}

// TestDefaultValueAsExpressions is used for tests that are inconvenient to place in the pkg/tests directory.
func TestDefaultValueAsExpressions(t *testing.T) {
store := testkit.CreateMockStoreWithSchemaLease(t, testLease)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t, t1, t2")

// date_format
tk.MustExec("create table t6 (c int(10), c1 int default (date_format(now(),'%Y-%m-%d %H:%i:%s')))")
tk.MustExec("create table t7 (c int(10), c1 date default (date_format(now(),'%Y-%m')))")
// Error message like: Error 1292 (22007): Truncated incorrect DOUBLE value: '2024-03-05 16:37:25'.
tk.MustGetErrCode("insert into t6(c) values (1)", errno.ErrTruncatedWrongValue)
tk.MustGetErrCode("insert into t7(c) values (1)", errno.ErrTruncatedWrongValue)

// user
tk.MustExec("create table t (c int(10), c1 varchar(256) default (upper(substring_index(user(),'@',1))));")
tk.Session().GetSessionVars().User = &auth.UserIdentity{Username: "root", Hostname: "localhost"}
tk.MustExec("insert into t(c) values (1),(2),(3)")
tk.Session().GetSessionVars().User = &auth.UserIdentity{Username: "xyz", Hostname: "localhost"}
tk.MustExec("insert into t(c) values (4),(5),(6)")
tk.MustExec("insert into t values (7, default)")
rows := tk.MustQuery("SELECT c1 from t order by c").Rows()
for i, row := range rows {
d, ok := row[0].(string)
require.True(t, ok)
if i < 3 {
require.Equal(t, "ROOT", d)
} else {
require.Equal(t, "XYZ", d)
}
}

// replace
tk.MustExec("create table t1 (c int(10), c1 int default (REPLACE(UPPER(UUID()), '-', '')))")
// Different UUID values will result in different error code.
_, err := tk.Exec("insert into t1(c) values (1)")
originErr := errors.Cause(err)
tErr, ok := originErr.(*terror.Error)
require.Truef(t, ok, "expect type 'terror.Error', but obtain '%T': %v", originErr, originErr)
sqlErr := terror.ToSQLError(tErr)
if int(sqlErr.Code) != errno.ErrTruncatedWrongValue {
require.Equal(t, errno.ErrDataOutOfRange, int(sqlErr.Code))
}
// test modify column
// The error message has UUID, so put this test here.
tk.MustExec("create table t2(c int(10), c1 varchar(256) default (REPLACE(UPPER(UUID()), '-', '')), index idx(c1));")
tk.MustExec("insert into t2(c) values (1),(2),(3);")
tk.MustGetErrCode("alter table t2 modify column c1 varchar(30) default 'xx';", errno.WarnDataTruncated)
// test add column for enum
nowStr := time.Now().Format("2006-01")
sql := fmt.Sprintf("alter table t2 add column c3 enum('%v','n')", nowStr) + " default (date_format(now(),'%Y-%m'))"
tk.MustExec(sql)
tk.MustExec("insert into t2(c) values (4);")
tk.MustQuery("select c3 from t2").Check(testkit.Rows(nowStr, nowStr, nowStr, nowStr))
}

func TestChangingDBCharset(t *testing.T) {
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())

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

func TestAddExpressionIndex(t *testing.T) {
config.UpdateGlobal(func(conf *config.Config) {
// Test for table lock.
conf.EnableTableLock = true
conf.Instance.SlowThreshold = 10000
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
Expand Down Expand Up @@ -1901,60 +1955,6 @@ func TestAddExpressionIndex(t *testing.T) {
})
}

func TestCreateExpressionIndexError(t *testing.T) {
config.UpdateGlobal(func(conf *config.Config) {
// Test for table lock.
conf.EnableTableLock = true
conf.Instance.SlowThreshold = 10000
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int, b real);")
tk.MustGetErrCode("alter table t add primary key ((a+b)) nonclustered;", errno.ErrFunctionalIndexPrimaryKey)

tk.MustGetErrCode("create table t(a int, index((cast(a as JSON))))", errno.ErrFunctionalIndexOnJSONOrGeometryFunction)

// Test for error
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int, b real);")
tk.MustGetErrCode("alter table t add primary key ((a+b)) nonclustered;", errno.ErrFunctionalIndexPrimaryKey)
tk.MustGetErrCode("alter table t add index ((rand()));", errno.ErrFunctionalIndexFunctionIsNotAllowed)
tk.MustGetErrCode("alter table t add index ((now()+1));", errno.ErrFunctionalIndexFunctionIsNotAllowed)

tk.MustExec("alter table t add column (_V$_idx_0 int);")
tk.MustGetErrCode("alter table t add index idx((a+1));", errno.ErrDupFieldName)
tk.MustExec("alter table t drop column _V$_idx_0;")
tk.MustExec("alter table t add index idx((a+1));")
tk.MustGetErrCode("alter table t add column (_V$_idx_0 int);", errno.ErrDupFieldName)
tk.MustExec("alter table t drop index idx;")
tk.MustExec("alter table t add column (_V$_idx_0 int);")

tk.MustExec("alter table t add column (_V$_expression_index_0 int);")
tk.MustGetErrCode("alter table t add index ((a+1));", errno.ErrDupFieldName)
tk.MustExec("alter table t drop column _V$_expression_index_0;")
tk.MustExec("alter table t add index ((a+1));")
tk.MustGetErrCode("alter table t drop column _V$_expression_index_0;", errno.ErrCantDropFieldOrKey)
tk.MustGetErrCode("alter table t add column e int as (_V$_expression_index_0 + 1);", errno.ErrBadField)

// NOTE (#18150): In creating expression index, row value is not allowed.
tk.MustExec("drop table if exists t;")
tk.MustGetErrCode("create table t (j json, key k (((j,j))))", errno.ErrFunctionalIndexRowValueIsNotAllowed)
tk.MustExec("create table t (j json, key k ((j+1),(j+1)))")

tk.MustGetErrCode("create table t1 (col1 int, index ((concat(''))));", errno.ErrWrongKeyColumnFunctionalIndex)
tk.MustGetErrCode("CREATE TABLE t1 (col1 INT, PRIMARY KEY ((ABS(col1))) NONCLUSTERED);", errno.ErrFunctionalIndexPrimaryKey)

// For issue 26349
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t(id char(10) primary key, short_name char(10), name char(10), key n((upper(`name`))));")
tk.MustExec("update t t1 set t1.short_name='a' where t1.id='1';")
}

func queryIndexOnTable(dbName, tableName string) string {
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)
}
Expand Down Expand Up @@ -2353,20 +2353,6 @@ func TestEnumAndSetDefaultValue(t *testing.T) {
require.Equal(t, "a", tbl.Meta().Columns[1].DefaultValue)
}

func TestStrictDoubleTypeCheck(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'")
sql := "create table double_type_check(id int, c double(10));"
_, err := tk.Exec(sql)
require.Error(t, err)
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())
tk.MustExec("set @@tidb_enable_strict_double_type_check = 'OFF'")
defer tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'")
tk.MustExec(sql)
}

func TestDuplicateErrorMessage(t *testing.T) {
defer collate.SetNewCollationEnabledForTest(true)
store := testkit.CreateMockStore(t)
Expand All @@ -2388,10 +2374,7 @@ func TestDuplicateErrorMessage(t *testing.T) {
for _, newCollate := range []bool{false, true} {
collate.SetNewCollationEnabledForTest(newCollate)
for _, globalIndex := range []bool{false, true} {
restoreConfig := config.RestoreFunc()
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableGlobalIndex = globalIndex
})
tk.MustExec(fmt.Sprintf("set tidb_enable_global_index=%t", globalIndex))
for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} {
tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex
for _, t := range tests {
Expand All @@ -2418,7 +2401,7 @@ func TestDuplicateErrorMessage(t *testing.T) {
fmt.Sprintf("[kv:1062]Duplicate entry '1-%s' for key 't.t_idx'", strings.Join(fields, "-")))
}
}
restoreConfig()
tk.MustExec("set tidb_enable_global_index=default")
}
}
}
Expand Down Expand Up @@ -2673,8 +2656,6 @@ func TestAvoidCreateViewOnLocalTemporaryTable(t *testing.T) {

func TestDropTemporaryTable(t *testing.T) {
config.UpdateGlobal(func(conf *config.Config) {
// Test for table lock.
conf.EnableTableLock = true
conf.Instance.SlowThreshold = 10000
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
Expand Down Expand Up @@ -2940,42 +2921,6 @@ func TestIssue29282(t *testing.T) {
}
}

// See https://github.com/pingcap/tidb/issues/35644
func TestCreateTempTableInTxn(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("begin")
// new created temporary table should be visible
tk.MustExec("create temporary table t1(id int primary key, v int)")
tk.MustQuery("select * from t1").Check(testkit.Rows())
// new inserted data should be visible
tk.MustExec("insert into t1 values(123, 456)")
tk.MustQuery("select * from t1 where id=123").Check(testkit.Rows("123 456"))
// truncate table will clear data but table still visible
tk.MustExec("truncate table t1")
tk.MustQuery("select * from t1 where id=123").Check(testkit.Rows())
tk.MustExec("commit")

tk1 := testkit.NewTestKit(t, store)
tk1.MustExec("use test")
tk1.MustExec("create table tt(id int)")
tk1.MustExec("begin")
tk1.MustExec("create temporary table t1(id int)")
tk1.MustExec("insert into tt select * from t1")
tk1.MustExec("drop table tt")

tk2 := testkit.NewTestKit(t, store)
tk2.MustExec("use test")
tk2.MustExec("create table t2(id int primary key, v int)")
tk2.MustExec("insert into t2 values(234, 567)")
tk2.MustExec("begin")
// create a new temporary table with the same name will override physical table
tk2.MustExec("create temporary table t2(id int primary key, v int)")
tk2.MustQuery("select * from t2 where id=234").Check(testkit.Rows())
tk2.MustExec("commit")
}

// See https://github.com/pingcap/tidb/issues/29327
func TestEnumDefaultValue(t *testing.T) {
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())
Expand Down Expand Up @@ -3071,6 +3016,12 @@ func TestDefaultCollationForUTF8MB4(t *testing.T) {
"dby CREATE DATABASE `dby` /*!40100 DEFAULT CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci */"))
}

func TestOptimizeTable(t *testing.T) {
store := testkit.CreateMockStore(t, mockstore.WithDDLChecker())
tk := testkit.NewTestKit(t, store)
tk.MustGetErrMsg("optimize table t", "[ddl:8200]OPTIMIZE TABLE is not supported")
}

func TestIssue52680(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -3135,3 +3086,31 @@ func TestIssue52680(t *testing.T) {
tk.MustExec("insert into issue52680 values(default);")
tk.MustQuery("select * from issue52680").Check(testkit.Rows("1", "2", "3"))
}

func TestCreateIndexWithChangeMaxIndexLength(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
originCfg := config.GetGlobalConfig()
defer func() {
config.StoreGlobalConfig(originCfg)
}()

originHook := dom.DDL().GetHook()
defer dom.DDL().SetHook(originHook)
hook := &callback.TestDDLCallback{Do: dom}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.Type != model.ActionAddIndex {
return
}
if job.SchemaState == model.StateNone {
newCfg := *originCfg
newCfg.MaxIndexLength = 1000
config.StoreGlobalConfig(&newCfg)
}
}
dom.DDL().SetHook(hook)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("create table t(id int, a json DEFAULT NULL, b varchar(2) DEFAULT NULL);")
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")
}
4 changes: 2 additions & 2 deletions pkg/ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,15 +51,15 @@ func TestInvalidDDLJob(t *testing.T) {
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{},
}
ctx := testNewContext(store)
ctx := testNewContext(t, store)
ctx.SetValue(sessionctx.QueryString, "skip")
err := dom.DDL().DoDDLJob(ctx, job)
require.Equal(t, err.Error(), "[ddl:8204]invalid ddl job type: none")
}

func TestAddBatchJobError(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease)
ctx := testNewContext(store)
ctx := testNewContext(t, store)

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

0 comments on commit 2e19b63

Please sign in to comment.