Skip to content

Commit

Permalink
tests: fix some tests failed when system_time_zone != Asia/Shanghai (
Browse files Browse the repository at this point in the history
  • Loading branch information
Defined2014 authored Dec 18, 2023
1 parent 8ed0d47 commit 5167654
Show file tree
Hide file tree
Showing 10 changed files with 48 additions and 32 deletions.
3 changes: 2 additions & 1 deletion pkg/ddl/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/variable"
statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/filter"
"github.com/pingcap/tidb/pkg/util/gcutil"
"github.com/pingcap/tidb/pkg/util/logutil"
Expand Down Expand Up @@ -251,7 +252,7 @@ func checkAndSetFlashbackClusterInfo(se sessionctx.Context, d *ddlCtx, t *meta.M
return errors.Trace(err)
}

flashbackTSString := oracle.GetTimeFromTS(flashbackTS).String()
flashbackTSString := oracle.GetTimeFromTS(flashbackTS).Format(types.TimeFSPFormat)

// Check if there is an upgrade during [flashbackTS, now)
sql := fmt.Sprintf("select VARIABLE_VALUE from mysql.tidb as of timestamp '%s' where VARIABLE_NAME='tidb_server_version'", flashbackTSString)
Expand Down
13 changes: 7 additions & 6 deletions pkg/ddl/cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -101,7 +102,7 @@ func TestFlashbackCloseAndResetPDSchedule(t *testing.T) {
ts, err := tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{})
require.NoError(t, err)

tk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)), errno.ErrCancelledDDLJob)
tk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)), errno.ErrCancelledDDLJob)
dom.DDL().SetHook(originHook)

finishValue, err := infosync.GetPDScheduleConfig(context.Background())
Expand Down Expand Up @@ -142,7 +143,7 @@ func TestAddDDLDuringFlashback(t *testing.T) {
}
}
dom.DDL().SetHook(hook)
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

dom.DDL().SetHook(originHook)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockFlashbackTest"))
Expand Down Expand Up @@ -193,7 +194,7 @@ func TestGlobalVariablesOnFlashback(t *testing.T) {
tk.MustExec("set global tidb_super_read_only = off")
tk.MustExec("set global tidb_ttl_job_enable = on")

tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

rs, err := tk.Exec("show variables like 'tidb_super_read_only'")
require.NoError(t, err)
Expand All @@ -212,7 +213,7 @@ func TestGlobalVariablesOnFlashback(t *testing.T) {

ts, err = tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{})
require.NoError(t, err)
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))
rs, err = tk.Exec("show variables like 'tidb_super_read_only'")
require.NoError(t, err)
require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On)
Expand Down Expand Up @@ -252,7 +253,7 @@ func TestCancelFlashbackCluster(t *testing.T) {
})
dom.DDL().SetHook(hook)
tk.MustExec("set global tidb_ttl_job_enable = on")
tk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)), errno.ErrCancelledDDLJob)
tk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)), errno.ErrCancelledDDLJob)
hook.MustCancelDone(t)

rs, err := tk.Exec("show variables like 'tidb_ttl_job_enable'")
Expand All @@ -264,7 +265,7 @@ func TestCancelFlashbackCluster(t *testing.T) {
return job.SchemaState == model.StateWriteReorganization
})
dom.DDL().SetHook(hook)
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))
hook.MustCancelFailed(t)

rs, err = tk.Exec("show variables like 'tidb_ttl_job_enable'")
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/tests/tiflash/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_test(
"//pkg/testkit",
"//pkg/testkit/external",
"//pkg/testkit/testsetup",
"//pkg/types",
"//pkg/util",
"//pkg/util/logutil",
"//pkg/util/sqlkiller",
Expand Down
5 changes: 3 additions & 2 deletions pkg/ddl/tests/tiflash/ddl_tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/pingcap/tidb/pkg/tablecodec"
"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"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/sqlkiller"
Expand Down Expand Up @@ -471,8 +472,8 @@ func TestTiFlashFlashbackCluster(t *testing.T) {
}()

errorMsg := fmt.Sprintf("[ddl:-1]Detected unsupported DDL job type(%s) during [%s, now), can't do flashback",
model.ActionSetTiFlashReplica.String(), oracle.GetTimeFromTS(ts).String())
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)), errorMsg)
model.ActionSetTiFlashReplica.String(), oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat))
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)), errorMsg)

require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockFlashbackTest"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS"))
Expand Down
9 changes: 5 additions & 4 deletions pkg/executor/recover_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/pingcap/tidb/pkg/util/gcutil"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -376,15 +377,15 @@ func TestRecoverClusterMeetError(t *testing.T) {
nowTS, err := tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{})
require.NoError(t, err)
tk.MustExec("truncate table mysql.stats_meta")
errorMsg := fmt.Sprintf("[ddl:-1]Detected modified system table during [%s, now), can't do flashback", oracle.GetTimeFromTS(nowTS).String())
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(nowTS)), errorMsg)
errorMsg := fmt.Sprintf("[ddl:-1]Detected modified system table during [%s, now), can't do flashback", oracle.GetTimeFromTS(nowTS).Format(types.TimeFSPFormat))
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(nowTS).Format(types.TimeFSPFormat)), errorMsg)

// update tidb_server_version
nowTS, err = tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{})
require.NoError(t, err)
tk.MustExec("update mysql.tidb set VARIABLE_VALUE=VARIABLE_VALUE+1 where VARIABLE_NAME='tidb_server_version'")
errorMsg = fmt.Sprintf("[ddl:-1]Detected TiDB upgrade during [%s, now), can't do flashback", oracle.GetTimeFromTS(nowTS).String())
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(nowTS)), errorMsg)
errorMsg = fmt.Sprintf("[ddl:-1]Detected TiDB upgrade during [%s, now), can't do flashback", oracle.GetTimeFromTS(nowTS).Format(types.TimeFSPFormat))
tk.MustGetErrMsg(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(nowTS).Format(types.TimeFSPFormat)), errorMsg)

require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockFlashbackTest"))
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/stale_txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -539,7 +539,7 @@ func TestSetTransactionReadOnlyAsOf(t *testing.T) {
}{
{
sql: `SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`,
expectedTS: 424394603102208000,
expectedTS: oracle.GoTimeToTS(time.Date(2021, 4, 21, 0, 42, 12, 0, time.Local)),
injectSafeTS: 0,
},
{
Expand Down Expand Up @@ -580,7 +580,7 @@ func TestSetTransactionReadOnlyAsOf(t *testing.T) {
require.Equal(t, "start transaction read only as of is forbidden after set transaction read only as of", err.Error())

tk.MustExec("begin")
require.Equal(t, uint64(424394603102208000), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS())
require.Equal(t, oracle.GoTimeToTS(time.Date(2021, 4, 21, 0, 42, 12, 0, time.Local)), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS())
tk.MustExec("commit")
tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/infoschema/test/clustertablestest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ go_test(
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_tipb//go-tipb",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_client_go_v2//testutils",
"@com_github_tikv_pd_client//http",
"@org_golang_google_grpc//:grpc",
Expand Down
16 changes: 12 additions & 4 deletions pkg/infoschema/test/clustertablestest/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/pingcap/tidb/pkg/util/gctuner"
"github.com/pingcap/tidb/pkg/util/memory"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
)

func newTestKitWithRoot(t *testing.T, store kv.Storage) *testkit.TestKit {
Expand Down Expand Up @@ -1214,8 +1215,12 @@ func TestTiDBTrx(t *testing.T) {
memDBTracker := memory.NewTracker(memory.LabelForMemDB, -1)
memDBTracker.Consume(19)
tk.Session().GetSessionVars().MemDBFootprint = memDBTracker

t1 := time.Date(2021, 5, 7, 4, 56, 48, 1000000, time.UTC)
t2 := time.Date(2021, 5, 20, 13, 16, 35, 778000000, time.UTC)

sm.TxnInfo[0] = &txninfo.TxnInfo{
StartTS: 424768545227014155,
StartTS: oracle.GoTimeToTS(t1),
CurrentSQLDigest: digest.String(),
State: txninfo.TxnIdle,
EntriesCount: 1,
Expand All @@ -1226,7 +1231,7 @@ func TestTiDBTrx(t *testing.T) {

blockTime2 := time.Date(2021, 05, 20, 13, 18, 30, 123456000, time.Local)
sm.TxnInfo[1] = &txninfo.TxnInfo{
StartTS: 425070846483628033,
StartTS: oracle.GoTimeToTS(t2),
CurrentSQLDigest: "",
AllSQLDigests: []string{"sql1", "sql2", digest.String()},
State: txninfo.TxnLockAcquiring,
Expand All @@ -1252,8 +1257,11 @@ func TestTiDBTrx(t *testing.T) {
ALL_SQL_DIGESTS,
RELATED_TABLE_IDS
from information_schema.TIDB_TRX`).Check(testkit.Rows(
"424768545227014155 2021-05-07 12:56:48.001000 "+digest.String()+" update `test_tidb_trx` set `i` = `i` + ? Idle <nil> 1 19 2 root test [] ",
"425070846483628033 2021-05-20 21:16:35.778000 <nil> <nil> LockWaiting 2021-05-20 13:18:30.123456 0 19 10 user1 db1 [\"sql1\",\"sql2\",\""+digest.String()+"\"] "))
"424768545227014144 "+t1.Local().Format(types.TimeFSPFormat)+" "+digest.String()+" update `test_tidb_trx` set `i` = `i` + ? Idle <nil> 1 19 2 root test [] ",
"425070846483628032 "+t2.Local().Format(types.TimeFSPFormat)+" <nil> <nil> LockWaiting "+
// `WAITING_START_TIME` will not be affected by time_zone, it is in memory and we assume that the system time zone will not change.
blockTime2.Format(types.TimeFSPFormat)+
" 0 19 10 user1 db1 [\"sql1\",\"sql2\",\""+digest.String()+"\"] "))

rows := tk.MustQuery(`select WAITING_TIME from information_schema.TIDB_TRX where WAITING_TIME is not null`)
require.Len(t, rows.Rows(), 1)
Expand Down
1 change: 1 addition & 0 deletions tests/realtikvtest/flashbacktest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_test(
"//pkg/parser/model",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/types",
"//tests/realtikvtest",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//assert",
Expand Down
27 changes: 14 additions & 13 deletions tests/realtikvtest/flashbacktest/flashback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/tests/realtikvtest"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -85,7 +86,7 @@ func TestFlashback(t *testing.T) {
fmt.Sprintf("return(%v)", injectSafeTS)))

tk.MustExec("insert t values (4), (5), (6)")
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("select max(a) from t").Rows()[0][0], "3")
Expand Down Expand Up @@ -121,7 +122,7 @@ func TestPrepareFlashbackFailed(t *testing.T) {
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/mockPrepareMeetsEpochNotMatch", `return(true)`))

tk.MustExec("insert t values (4), (5), (6)")
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("select max(a) from t").Rows()[0][0], "3")
Expand Down Expand Up @@ -169,7 +170,7 @@ func TestFlashbackAddDropIndex(t *testing.T) {
fmt.Sprintf("return(%v)", injectSafeTS)))

tk.MustExec("insert t values (4), (5), (6)")
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("select max(a) from t use index(i)").Rows()[0][0], "3")
Expand Down Expand Up @@ -214,7 +215,7 @@ func TestFlashbackAddDropModifyColumn(t *testing.T) {
fmt.Sprintf("return(%v)", injectSafeTS)))

tk.MustExec("insert t values (4, 4), (5, 5), (6, 6)")
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("show create table t").Rows()[0][1], "CREATE TABLE `t` (\n"+
Expand Down Expand Up @@ -265,7 +266,7 @@ func TestFlashbackBasicRenameDropCreateTable(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("select max(a) from t").Rows()[0][0], "3")
Expand Down Expand Up @@ -303,7 +304,7 @@ func TestFlashbackCreateDropTableWithData(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
require.Equal(t, tk.MustQuery("select count(a) from t").Rows()[0][0], "0")
Expand Down Expand Up @@ -342,7 +343,7 @@ func TestFlashbackCreateDropSchema(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table test.t")
res := tk.MustQuery("select max(a) from test.t").Rows()
Expand Down Expand Up @@ -381,7 +382,7 @@ func TestFlashbackAutoID(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
res = tk.MustQuery("select max(a) from t").Rows()
Expand Down Expand Up @@ -421,7 +422,7 @@ func TestFlashbackSequence(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

// flashback schema and skip cached values
res = tk.MustQuery("select nextval(seq)").Rows()
Expand Down Expand Up @@ -465,7 +466,7 @@ func TestFlashbackPartitionTable(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustExec("admin check table t")
res = tk.MustQuery("select max(a), min(a), count(*) from t").Rows()
Expand Down Expand Up @@ -506,7 +507,7 @@ func TestFlashbackTmpTable(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

res := tk.MustQuery("select max(a) from t").Rows()
require.Equal(t, res[0][0], "3")
Expand All @@ -522,7 +523,7 @@ func TestFlashbackTmpTable(t *testing.T) {

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS",
fmt.Sprintf("return(%v)", injectSafeTS)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))

tk.MustGetErrCode("select * from t", errno.ErrNoSuchTable)

Expand Down Expand Up @@ -572,7 +573,7 @@ func TestFlashbackInProcessErrorMsg(t *testing.T) {
}
}
dom.DDL().SetHook(hook)
tk.Exec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts)))
tk.Exec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat)))
dom.DDL().SetHook(originHook)

require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS"))
Expand Down

0 comments on commit 5167654

Please sign in to comment.