From 383256cf90f6f49e12fa995fe7ca8fda83c500f3 Mon Sep 17 00:00:00 2001 From: djshow832 <873581766@qq.com> Date: Thu, 25 Nov 2021 21:20:52 +0800 Subject: [PATCH 1/2] add tests --- plugin/integration_test.go | 825 ++++++++++++++++++++++++++++++++----- server/mock_conn.go | 5 + 2 files changed, 719 insertions(+), 111 deletions(-) diff --git a/plugin/integration_test.go b/plugin/integration_test.go index d00e8587bba98..d657393c0dc2d 100644 --- a/plugin/integration_test.go +++ b/plugin/integration_test.go @@ -15,40 +15,727 @@ package plugin_test import ( - "bytes" "context" "fmt" "strconv" + "strings" "testing" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/plugin" - "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/testutils" ) -type testAuditLogSuite struct { - cluster testutils.Cluster - store kv.Storage - dom *domain.Domain +// Audit tests cannot run in parallel. +func TestAuditLogNormal(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + sv := server.CreateMockServer(t, store) + defer sv.Close() + conn := server.CreateMockConn(t, store, sv) + defer conn.Close() - bytes.Buffer + type normalTest struct { + sql string + text string + rows uint64 + stmtType string + dbs string + tables string + cmd string + event plugin.GeneralEvent + } + + tests := []normalTest{ + { + sql: "CREATE DATABASE mynewdatabase", + stmtType: "CreateDatabase", + dbs: "mynewdatabase", + }, + { + sql: "CREATE TABLE t1 (a INT NOT NULL)", + stmtType: "CreateTable", + dbs: "test", + tables: "t1", + }, + { + sql: "CREATE TABLE t2 LIKE t1", + stmtType: "CreateTable", + dbs: "test,test", + tables: "t2,t1", + }, + { + sql: "CREATE INDEX a ON t1 (a)", + stmtType: "CreateIndex", + dbs: "test", + tables: "t1", + }, + { + sql: "CREATE SEQUENCE seq", + stmtType: "other", + dbs: "test", + tables: "seq", + }, + { + sql: " create temporary table t3 (a int)", + stmtType: "CreateTable", + dbs: "test", + tables: "t3", + }, + { + sql: "create global temporary table t4 (a int) on commit delete rows", + stmtType: "CreateTable", + dbs: "test", + tables: "t4", + }, + { + sql: "CREATE VIEW v1 AS SELECT * FROM t1 WHERE a> 2", + stmtType: "CreateView", + dbs: "test,test", + tables: "t1,v1", + }, + { + sql: "USE test", + stmtType: "Use", + }, + { + sql: "DROP DATABASE mynewdatabase", + stmtType: "DropDatabase", + dbs: "mynewdatabase", + }, + { + sql: "SHOW CREATE SEQUENCE seq", + stmtType: "Show", + dbs: "test", + tables: "seq", + }, + { + sql: "DROP SEQUENCE seq", + stmtType: "other", + dbs: "test", + tables: "seq", + }, + { + sql: "DROP TABLE t4", + stmtType: "DropTable", + dbs: "test", + tables: "t4", + }, + { + sql: "DROP VIEW v1", + stmtType: "DropView", + dbs: "test", + tables: "v1", + }, + { + sql: "ALTER TABLE t1 ADD COLUMN c1 INT NOT NULL", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 MODIFY c1 BIGINT", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 ADD INDEX (c1)", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 ALTER INDEX c1 INVISIBLE", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 RENAME INDEX c1 TO c2", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 DROP INDEX c2", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 CHANGE c1 c2 INT", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "ALTER TABLE t1 DROP COLUMN c2", + stmtType: "AlterTable", + dbs: "test", + tables: "t1", + }, + { + sql: "CREATE SESSION BINDING FOR SELECT * FROM t1 WHERE a = 123 USING SELECT * FROM t1 IGNORE INDEX (a) WHERE a = 123", + stmtType: "CreateBinding", + }, + { + sql: "DROP SESSION BINDING FOR SELECT * FROM t1 WHERE a = 123", + stmtType: "DropBinding", + }, + //{ + // sql: "LOAD STATS '/tmp/stats.json'", + // stmtType: "other", + //}, + //{ + // sql: "DROP STATS t", + // stmtType: "other", + //}, + { + sql: "RENAME TABLE t2 TO t5", + stmtType: "other", + dbs: "test,test", + tables: "t2,t5", + }, + { + sql: "TRUNCATE t1", + stmtType: "TruncateTable", + dbs: "test", + tables: "t1", + }, + //{ + // sql: "FLASHBACK TABLE t TO t1", + // stmtType: "other", + // dbs: "test", + // tables: "t1", + //}, + //{ + // sql: "RECOVER TABLE t1", + // stmtType: "other", + // dbs: "test", + // tables: "t1,t2", + //}, + { + sql: "ALTER DATABASE test DEFAULT CHARACTER SET = utf8mb4", + stmtType: "other", + dbs: "test", + }, + { + sql: "ADMIN RELOAD opt_rule_blacklist", + stmtType: "other", + }, + //{ + // sql: "ADMIN PLUGINS ENABLE audit_test", + // stmtType: "other", + //}, + { + sql: "ADMIN FLUSH bindings", + stmtType: "other", + }, + //{ + // sql: "ADMIN REPAIR TABLE t1 CREATE TABLE (id int)", + // stmtType: "other", + // dbs: "test", + // tables: "t1", + //}, + { + sql: "ADMIN SHOW SLOW RECENT 10", + stmtType: "other", + }, + { + sql: "ADMIN SHOW DDL JOBS", + stmtType: "other", + }, + //{ + // sql: "ADMIN CANCEL DDL JOBS 1", + // stmtType: "other", + //}, + { + sql: "ADMIN CHECKSUM TABLE t1", + stmtType: "other", + //dbs: "test", + //tables: "t1", + }, + { + sql: "ADMIN CHECK TABLE t1", + stmtType: "other", + //dbs: "test", + //tables: "t1", + }, + { + sql: "ADMIN CHECK INDEX t1 a", + stmtType: "other", + //dbs: "test", + //tables: "t1", + }, + { + sql: "CREATE USER 'newuser' IDENTIFIED BY 'newuserpassword'", + stmtType: "CreateUser", + }, + { + sql: "ALTER USER 'newuser' IDENTIFIED BY 'newnewpassword'", + stmtType: "other", + }, + { + sql: "CREATE ROLE analyticsteam", + stmtType: "CreateUser", + }, + { + sql: "GRANT SELECT ON test.* TO analyticsteam", + stmtType: "Grant", + dbs: "test", + }, + { + sql: "GRANT analyticsteam TO 'newuser'", + stmtType: "other", + }, + { + sql: "SET DEFAULT ROLE analyticsteam TO newuser;", + stmtType: "other", + }, + { + sql: "REVOKE SELECT ON test.* FROM 'analyticsteam'", + stmtType: "Revoke", + dbs: "test", + }, + { + sql: "DROP ROLE analyticsteam", + stmtType: "other", + }, + { + sql: "FLUSH PRIVILEGES", + stmtType: "other", + }, + { + sql: "SET PASSWORD FOR 'newuser' = 'test'", + stmtType: "Set", + }, + //{ + // sql: "SET ROLE ALL", + // stmtType: "other", + //}, + { + sql: "DROP USER 'newuser'", + stmtType: "other", + }, + { + sql: "analyze table t1", + stmtType: "AnalyzeTable", + dbs: "test", + tables: "t1", + }, + { + sql: "SPLIT TABLE t1 BETWEEN (0) AND (1000000000) REGIONS 16", + stmtType: "other", + //dbs: "test", + //tables: "t1", + }, + //{ + // sql: "BACKUP DATABASE `test` TO '.'", + // stmtType: "other", + // dbs: "test", + //}, + //{ + // sql: "RESTORE DATABASE * FROM '.'", + // stmtType: "other", + //}, + //{ + // sql: "CHANGE DRAINER TO NODE_STATE ='paused' FOR NODE_ID 'drainer1'", + // stmtType: "other", + //}, + //{ + // sql: "CHANGE PUMP TO NODE_STATE ='paused' FOR NODE_ID 'pump1'", + // stmtType: "other", + //}, + { + sql: "BEGIN", + stmtType: "Begin", + }, + { + sql: "ROLLBACK", + stmtType: "RollBack", + }, + { + sql: "START TRANSACTION", + stmtType: "Begin", + }, + { + sql: "COMMIT", + stmtType: "Commit", + }, + //{ + // sql: "SHOW DRAINER STATUS", + // stmtType: "Show", + //}, + //{ + // sql: "SHOW PUMP STATUS", + // stmtType: "Show", + //}, + //{ + // sql: "SHOW GRANTS", + // stmtType: "Show", + //}, + { + sql: "SHOW PROCESSLIST", + stmtType: "Show", + }, + //{ + // sql: "SHOW BACKUPS", + // stmtType: "Show", + //}, + //{ + // sql: "SHOW RESTORES", + // stmtType: "Show", + //}, + { + sql: "show analyze status", + stmtType: "Show", + }, + { + sql: "SHOW SESSION BINDINGS", + stmtType: "Show", + }, + { + sql: "SHOW BUILTINS", + stmtType: "Show", + }, + { + sql: "SHOW CHARACTER SET", + stmtType: "Show", + }, + { + sql: "SHOW COLLATION", + stmtType: "Show", + }, + { + sql: "show columns from t1", + stmtType: "Show", + }, + { + sql: "show fields from t1", + stmtType: "Show", + }, + //{ + // sql: "SHOW CONFIG", + // stmtType: "Show", + //}, + { + sql: "SHOW CREATE TABLE t1", + stmtType: "Show", + dbs: "test", + tables: "t1", + }, + { + sql: "SHOW CREATE USER 'root'", + stmtType: "Show", + }, + { + sql: "SHOW DATABASES", + stmtType: "Show", + }, + { + sql: "SHOW ENGINES", + stmtType: "Show", + }, + { + sql: "SHOW ERRORS", + stmtType: "Show", + }, + { + sql: "SHOW INDEXES FROM t1", + stmtType: "Show", + }, + { + sql: "SHOW MASTER STATUS", + stmtType: "Show", + }, + { + sql: "SHOW PLUGINS", + stmtType: "Show", + }, + { + sql: "show privileges", + stmtType: "Show", + }, + { + sql: "SHOW PROFILES", + stmtType: "Show", + }, + //{ + // sql: "SHOW PUMP STATUS", + // stmtType: "Show", + //}, + { + sql: "SHOW SCHEMAS", + stmtType: "Show", + }, + { + sql: "SHOW STATS_HEALTHY", + stmtType: "Show", + dbs: "mysql", + }, + { + sql: "show stats_histograms", + stmtType: "Show", + dbs: "mysql", + }, + { + sql: "show stats_meta", + stmtType: "Show", + dbs: "mysql", + }, + { + sql: "show status", + stmtType: "Show", + }, + { + sql: "show table t1 next_row_id", + stmtType: "Show", + dbs: "test", + tables: "t1", + }, + { + sql: "show table t1 regions", + stmtType: "Show", + }, + { + sql: "SHOW TABLE STATUS LIKE 't1'", + stmtType: "Show", + }, + { + sql: "SHOW TABLES", + stmtType: "Show", + }, + { + sql: "SHOW VARIABLES", + stmtType: "Show", + }, + { + sql: "SHOW WARNINGS", + stmtType: "Show", + }, + { + sql: "SET @number = 5", + stmtType: "Set", + }, + { + sql: "SET NAMES utf8", + stmtType: "Set", + }, + { + sql: "SET CHARACTER SET utf8mb4", + stmtType: "Set", + }, + { + sql: "SET SESSION TRANSACTION ISOLATION LEVEL READ COMMITTED", + stmtType: "Set", + }, + { + sql: "SET SESSION sql_mode = 'STRICT_TRANS_TABLES,NO_AUTO_CREATE_USER'", + stmtType: "Set", + }, + { + sql: "PREPARE mystmt FROM 'SELECT ? as num FROM DUAL'", + stmtType: "Prepare", + }, + { + sql: "EXECUTE mystmt USING @number", + text: "SELECT ? as num FROM DUAL", + stmtType: "Select", + }, + { + sql: "DEALLOCATE PREPARE mystmt", + stmtType: "Deallocate", + }, + { + sql: "INSERT INTO t1 VALUES (1), (2)", + stmtType: "Insert", + dbs: "test", + tables: "t1", + rows: 2, + }, + { + sql: "DELETE FROM t1 WHERE a = 2", + stmtType: "Delete", + dbs: "test", + tables: "t1", + rows: 1, + }, + { + sql: "REPLACE INTO t1 VALUES(3)", + stmtType: "Replace", + dbs: "test", + tables: "t1", + rows: 1, + }, + { + sql: "UPDATE t1 SET a=5 WHERE a=1", + stmtType: "Update", + dbs: "test", + tables: "t1", + rows: 1, + }, + { + sql: "DO 1", + stmtType: "other", + }, + //{ + // sql: "LOAD DATA LOCAL INFILE 'data.csv' INTO TABLE t1 FIELDS TERMINATED BY ',' ENCLOSED BY '\"' LINES TERMINATED BY '\r\n' IGNORE 1 LINES (id)", + // stmtType: "LoadData", + // dbs: "test", + // tables: "t1", + //}, + { + sql: "SELECT * FROM t1", + stmtType: "Select", + dbs: "test", + tables: "t1", + }, + { + sql: "SELECT 1", + stmtType: "Select", + }, + { + sql: "TABLE t1", + stmtType: "Select", + dbs: "test", + tables: "t1", + }, + { + sql: "EXPLAIN ANALYZE SELECT * FROM t1 WHERE a = 1", + stmtType: "Explain", + //dbs: "test", + //tables: "t1", + }, + { + sql: "EXPLAIN SELECT * FROM t1", + stmtType: "Explain", + //dbs: "test", + //tables: "t1", + }, + { + sql: "EXPLAIN SELECT * FROM t1 WHERE a = 1", + stmtType: "Explain", + //dbs: "test", + //tables: "t1", + }, + { + sql: "DESC SELECT * FROM t1 WHERE a = 1", + stmtType: "Explain", + //dbs: "test", + //tables: "t1", + }, + { + sql: "DESCRIBE SELECT * FROM t1 WHERE a = 1", + stmtType: "Explain", + //dbs: "test", + //tables: "t1", + }, + { + sql: "trace format='row' select * from t1", + stmtType: "Trace", + //dbs: "test", + //tables: "t1", + }, + { + sql: "flush status", + stmtType: "other", + }, + { + sql: "FLUSH TABLES", + stmtType: "other", + }, + //{ + // sql: "KILL TIDB 2", + // stmtType: "other", + //}, + //{ + // sql: "SHUTDOWN", + // stmtType: "Shutdow", + //}, + //{ + // sql: "ALTER INSTANCE RELOAD TLS", + // stmtType: "other", + //}, + } + + testResults := make([]normalTest, 0) + dbNames := make([]string, 0) + tableNames := make([]string, 0) + onGeneralEvent := func(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { + dbNames = dbNames[:0] + tableNames = tableNames[:0] + for _, value := range sctx.StmtCtx.Tables { + dbNames = append(dbNames, value.DB) + tableNames = append(tableNames, value.Table) + } + audit := normalTest{ + text: sctx.StmtCtx.OriginalSQL, + rows: sctx.StmtCtx.AffectedRows(), + stmtType: sctx.StmtCtx.StmtType, + dbs: strings.Join(dbNames, ","), + tables: strings.Join(tableNames, ","), + cmd: cmd, + event: event, + } + testResults = append(testResults, audit) + } + loadPlugin(t, onGeneralEvent) + defer plugin.Shutdown(context.Background()) + + require.NoError(t, conn.HandleQuery(context.Background(), "use test")) + for _, test := range tests { + testResults = testResults[:0] + errMsg := fmt.Sprintf("statement: %s", test.sql) + query := append([]byte{mysql.ComQuery}, []byte(test.sql)...) + err := conn.Dispatch(context.Background(), query) + require.NoError(t, err, errMsg) + require.Equal(t, 2, len(testResults), errMsg) + result := testResults[0] + require.Equal(t, "Query", result.cmd, errMsg) + require.Equal(t, plugin.Starting, result.event, errMsg) + result = testResults[1] + if test.text == "" { + require.Equal(t, test.sql, result.text, errMsg) + } else { + require.Equal(t, test.text, result.text, errMsg) + } + require.Equal(t, test.rows, result.rows, errMsg) + require.Equal(t, test.stmtType, result.stmtType, errMsg) + require.Equal(t, test.dbs, result.dbs, errMsg) + require.Equal(t, test.tables, result.tables, errMsg) + require.Equal(t, "Query", result.cmd, errMsg) + require.Equal(t, plugin.Completed, result.event, errMsg) + } } -func (s *testAuditLogSuite) setup(t *testing.T) { - pluginName := "test_audit_log" +func loadPlugin(t *testing.T, onGeneralEvent func(context.Context, *variable.SessionVars, plugin.GeneralEvent, string)) { + ctx := context.Background() + pluginName := "audit_test" pluginVersion := uint16(1) pluginSign := pluginName + "-" + strconv.Itoa(int(pluginVersion)) - config.UpdateGlobal(func(conf *config.Config) { - conf.Plugin.Load = pluginSign - }) + cfg := plugin.Config{ + Plugins: []string{pluginSign}, + PluginDir: "", + EnvVersion: map[string]uint16{"go": 1112}, + } + + validate := func(ctx context.Context, manifest *plugin.Manifest) error { + return nil + } + onInit := func(ctx context.Context, manifest *plugin.Manifest) error { + return nil + } + onShutdown := func(ctx context.Context, manifest *plugin.Manifest) error { + return nil + } + onConnectionEvent := func(ctx context.Context, event plugin.ConnectionEvent, info *variable.ConnectionInfo) error { + return nil + } // setup load test hook. loadOne := func(p *plugin.Plugin, dir string, pluginID plugin.ID) (manifest func() *plugin.Manifest, err error) { @@ -58,106 +745,22 @@ func (s *testAuditLogSuite) setup(t *testing.T) { Kind: plugin.Audit, Name: pluginName, Version: pluginVersion, - OnInit: OnInit, - OnShutdown: OnShutdown, - Validate: Validate, + OnInit: onInit, + OnShutdown: onShutdown, + Validate: validate, }, - OnGeneralEvent: s.OnGeneralEvent, - OnConnectionEvent: OnConnectionEvent, + OnGeneralEvent: onGeneralEvent, + OnConnectionEvent: onConnectionEvent, } return plugin.ExportManifest(m) }, nil } plugin.SetTestHook(loadOne) - store, err := mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockstore.BootstrapWithSingleStore(c) - s.cluster = c - }), - ) - require.NoError(t, err) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() - - d, err := session.BootstrapSession(s.store) - require.NoError(t, err) - d.SetStatsUpdating(true) - s.dom = d -} - -func (s *testAuditLogSuite) teardown() { - s.dom.Close() - s.store.Close() -} - -func TestAuditLog(t *testing.T) { - var s testAuditLogSuite - s.setup(t) - defer s.teardown() - - var buf1 bytes.Buffer - tk := testkit.NewAsyncTestKit(t, s.store) - ctx := tk.OpenSession(context.Background(), "test") - buf1.WriteString("Use use `test`\n") // Workaround for the testing framework. - - tk.MustExec(ctx, "use test") - buf1.WriteString("Use use `test`\n") - - tk.MustExec(ctx, "create table t (id int primary key, a int, b int unique)") - buf1.WriteString("CreateTable create table `t` ( `id` int primary key , `a` int , `b` int unique )\n") - - tk.MustExec(ctx, "create view v1 as select * from t where id > 2") - buf1.WriteString("CreateView create view `v1` as select * from `t` where `id` > ?\n") - - tk.MustExec(ctx, "drop view v1") - buf1.WriteString("DropView drop view `v1`\n") - - tk.MustExec(ctx, "create session binding for select * from t where b = 123 using select * from t ignore index(b) where b = 123") - buf1.WriteString("CreateBinding create session binding for select * from `t` where `b` = ? using select * from `t` where `b` = ?\n") - - tk.MustExec(ctx, "prepare mystmt from 'select ? as num from DUAL'") - buf1.WriteString("Prepare prepare `mystmt` from ?\n") - - tk.MustExec(ctx, "set @number = 5") - buf1.WriteString("Set set @number = ?\n") - - tk.MustExec(ctx, "execute mystmt using @number") - buf1.WriteString("Select select ? as `num` from dual\n") - - tk.MustQuery(ctx, "trace format = 'row' select * from t") - buf1.WriteString("Trace trace format = ? select * from `t`\n") - - tk.MustExec(ctx, "shutdown") - buf1.WriteString("Shutdown shutdown\n") - - require.Equal(t, buf1.String(), s.Buffer.String()) -} - -func Validate(ctx context.Context, m *plugin.Manifest) error { - return nil -} - -// OnInit implements TiDB plugin's OnInit SPI. -func OnInit(ctx context.Context, manifest *plugin.Manifest) error { - return nil -} - -// OnShutdown implements TiDB plugin's OnShutdown SPI. -func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { - return nil -} - -// OnGeneralEvent implements TiDB Audit plugin's OnGeneralEvent SPI. -func (s *testAuditLogSuite) OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { - if sctx != nil { - normalized, _ := sctx.StmtCtx.SQLDigest() - fmt.Fprintln(&s.Buffer, sctx.StmtCtx.StmtType, normalized) - } -} + // trigger load. + err := plugin.Load(ctx, cfg) + require.NoErrorf(t, err, "load plugin [%s] fail, error [%s]\n", pluginSign, err) -// OnConnectionEvent implements TiDB Audit plugin's OnConnectionEvent SPI. -func OnConnectionEvent(ctx context.Context, event plugin.ConnectionEvent, info *variable.ConnectionInfo) error { - return nil + err = plugin.Init(ctx, cfg) + require.NoErrorf(t, err, "init plugin [%s] fail, error [%s]\n", pluginSign, err) } diff --git a/server/mock_conn.go b/server/mock_conn.go index 2d3c8d581466d..14e4daacf059e 100644 --- a/server/mock_conn.go +++ b/server/mock_conn.go @@ -18,6 +18,7 @@ import ( "bufio" "bytes" "context" + "flag" "testing" "github.com/pingcap/tidb/config" @@ -68,6 +69,10 @@ func (mc *mockConn) Close() { // CreateMockServer creates a mock server. func CreateMockServer(t *testing.T, store kv.Storage) *Server { + if !runInGoTest { + // If CreateMockServer is called in another package, runInGoTest is not initialized. + runInGoTest = flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil + } tidbdrv := NewTiDBDriver(store) cfg := config.NewConfig() cfg.Socket = "" From 7da580d3e215751ba0b101c4b9f3aa3f33503684 Mon Sep 17 00:00:00 2001 From: djshow832 <873581766@qq.com> Date: Fri, 26 Nov 2021 09:49:24 +0800 Subject: [PATCH 2/2] add initial settings --- plugin/integration_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/plugin/integration_test.go b/plugin/integration_test.go index d657393c0dc2d..eac32e58707df 100644 --- a/plugin/integration_test.go +++ b/plugin/integration_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/server" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -37,6 +38,8 @@ func TestAuditLogNormal(t *testing.T) { defer sv.Close() conn := server.CreateMockConn(t, store, sv) defer conn.Close() + session.DisableStats4Test() + session.SetSchemaLease(0) type normalTest struct { sql string @@ -695,6 +698,7 @@ func TestAuditLogNormal(t *testing.T) { require.NoError(t, err, errMsg) require.Equal(t, 2, len(testResults), errMsg) result := testResults[0] + // TODO: currently, result.text is wrong. require.Equal(t, "Query", result.cmd, errMsg) require.Equal(t, plugin.Starting, result.event, errMsg) result = testResults[1]