diff --git a/.github/workflows/integration-test-with-real-tikv.yml b/.github/workflows/integration-test-with-real-tikv.yml index 10109899a9bcc..93bcd8be3bc9f 100644 --- a/.github/workflows/integration-test-with-real-tikv.yml +++ b/.github/workflows/integration-test-with-real-tikv.yml @@ -33,3 +33,25 @@ jobs: export log_level=error make failpoint-enable go test ./tests/pessimistictest -v -with-real-tikv -timeout 20m + + realtikv-test: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + - uses: actions/setup-go@v2 + with: + go-version: 1.18 + - name: Setup TiKV cluster + run: | + # Disable pipelined pessimistic lock temporarily until tikv#11649 is resolved + echo -e "[pessimistic-txn]\npipelined = false\n" > tikv.toml + echo -e "[raftdb]\nmax-open-files = 20480\n" >> tikv.toml + echo -e "[rocksdb]\nmax-open-files = 20480\n" >> tikv.toml + curl --proto '=https' --tlsv1.2 -sSf https://tiup-mirrors.pingcap.com/install.sh | sh + $HOME/.tiup/bin/tiup playground nightly --mode tikv-slim --kv 3 --pd 3 --without-monitor --kv.config tikv.toml & + curl --retry-connrefused --retry 5 --retry-delay 5 http://127.0.0.1:2379 + - name: Run Tests + run: | + export log_level=error + make failpoint-enable + go test ./tests/realtikvtest -v -with-real-tikv -timeout 30m diff --git a/session/nontransactional_test.go b/session/nontransactional_test.go index d675a16827ed8..7fe9d827a0a33 100644 --- a/session/nontransactional_test.go +++ b/session/nontransactional_test.go @@ -28,7 +28,7 @@ import ( ) func TestNonTransactionalDeleteSharding(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -70,7 +70,7 @@ func TestNonTransactionalDeleteSharding(t *testing.T) { } func TestNonTransactionalDeleteDryRun(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -89,7 +89,7 @@ func TestNonTransactionalDeleteDryRun(t *testing.T) { } func TestNonTransactionalDeleteErrorMessage(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -124,7 +124,7 @@ func TestNonTransactionalDeleteErrorMessage(t *testing.T) { } func TestNonTransactionalDeleteSplitOnTiDBRowID(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -138,7 +138,7 @@ func TestNonTransactionalDeleteSplitOnTiDBRowID(t *testing.T) { } func TestNonTransactionalDeleteNull(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -161,7 +161,7 @@ func TestNonTransactionalDeleteNull(t *testing.T) { } func TestNonTransactionalDeleteSmallBatch(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=1024") @@ -177,7 +177,7 @@ func TestNonTransactionalDeleteSmallBatch(t *testing.T) { } func TestNonTransactionalDeleteShardOnGeneratedColumn(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -191,7 +191,7 @@ func TestNonTransactionalDeleteShardOnGeneratedColumn(t *testing.T) { } func TestNonTransactionalDeleteAutoDetectShardColumn(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -229,7 +229,7 @@ func TestNonTransactionalDeleteAutoDetectShardColumn(t *testing.T) { } func TestNonTransactionalDeleteInvisibleIndex(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -249,7 +249,7 @@ func TestNonTransactionalDeleteInvisibleIndex(t *testing.T) { } func TestNonTransactionalDeleteIgnoreSelectLimit(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -264,7 +264,7 @@ func TestNonTransactionalDeleteIgnoreSelectLimit(t *testing.T) { } func TestNonTransactionalDeleteReadStaleness(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_max_chunk_size=35") @@ -280,7 +280,7 @@ func TestNonTransactionalDeleteReadStaleness(t *testing.T) { } func TestNonTransactionalDeleteCheckConstraint(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -348,7 +348,7 @@ func TestNonTransactionalDeleteCheckConstraint(t *testing.T) { } func TestNonTransactionalDeleteOptimizerHints(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -361,7 +361,7 @@ func TestNonTransactionalDeleteOptimizerHints(t *testing.T) { } func TestNonTransactionalDeleteMultiTables(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -380,7 +380,7 @@ func TestNonTransactionalDeleteMultiTables(t *testing.T) { } func TestNonTransactionalDeleteAlias(t *testing.T) { - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -423,7 +423,7 @@ func TestNonTransactionalDeleteAlias(t *testing.T) { func TestNonTransactionalDeleteShardOnUnsupportedTypes(t *testing.T) { // When some day the test fail because such types are supported, we can update related docs and consider remove the test. - store, clean := createMockStoreAndSetup(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/session/session_legacy_test.go b/session/session_legacy_test.go index adc395ac012b4..90466835b560c 100644 --- a/session/session_legacy_test.go +++ b/session/session_legacy_test.go @@ -27,6 +27,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" @@ -44,13 +45,11 @@ import ( "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -59,10 +58,10 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tipb/go-binlog" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/txnkv/transaction" + clientv3 "go.etcd.io/etcd/client/v3" "google.golang.org/grpc" ) @@ -79,7 +78,6 @@ type testSessionSuiteBase struct { cluster testutils.Cluster store kv.Storage dom *domain.Domain - pdAddr string } type testSessionSuite struct { @@ -98,16 +96,72 @@ type testSessionSerialSuite struct { testSessionSuiteBase } +func clearTiKVStorage(store kv.Storage) error { + txn, err := store.Begin() + if err != nil { + return errors.Trace(err) + } + iter, err := txn.Iter(nil, nil) + if err != nil { + return errors.Trace(err) + } + for iter.Valid() { + if err := txn.Delete(iter.Key()); err != nil { + return errors.Trace(err) + } + if err := iter.Next(); err != nil { + return errors.Trace(err) + } + } + return txn.Commit(context.Background()) +} + +func clearEtcdStorage(ebd kv.EtcdBackend) error { + endpoints, err := ebd.EtcdAddrs() + if err != nil { + return err + } + cli, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + AutoSyncInterval: 30 * time.Second, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpc.WithBackoffMaxDelay(time.Second * 3), + }, + TLS: ebd.TLSConfig(), + }) + if err != nil { + return errors.Trace(err) + } + defer cli.Close() + + resp, err := cli.Get(context.Background(), "/tidb", clientv3.WithPrefix()) + if err != nil { + return errors.Trace(err) + } + for _, kv := range resp.Kvs { + if kv.Lease != 0 { + if _, err := cli.Revoke(context.Background(), clientv3.LeaseID(kv.Lease)); err != nil { + return errors.Trace(err) + } + } + } + _, err = cli.Delete(context.Background(), "/tidb", clientv3.WithPrefix()) + if err != nil { + return errors.Trace(err) + } + return nil +} + func (s *testSessionSuiteBase) SetUpSuite(c *C) { testleak.BeforeTest() if *withTiKV { - s.pdAddr = "127.0.0.1:2379" var d driver.TiKVDriver config.UpdateGlobal(func(conf *config.Config) { conf.TxnLocalLatches.Enabled = false }) - store, err := d.Open(fmt.Sprintf("tikv://%s?disableGC=true", s.pdAddr)) + store, err := d.Open("tikv://127.0.0.1:2379?disableGC=true") c.Assert(err, IsNil) err = clearTiKVStorage(store) c.Assert(err, IsNil) @@ -155,56 +209,6 @@ func (s *testSessionSuiteBase) TearDownTest(c *C) { } } -type mockBinlogPump struct { -} - -var _ binlog.PumpClient = &mockBinlogPump{} - -func (p *mockBinlogPump) WriteBinlog(ctx context.Context, in *binlog.WriteBinlogReq, opts ...grpc.CallOption) (*binlog.WriteBinlogResp, error) { - return &binlog.WriteBinlogResp{}, nil -} - -type mockPumpPullBinlogsClient struct { - grpc.ClientStream -} - -func (m mockPumpPullBinlogsClient) Recv() (*binlog.PullBinlogResp, error) { - return nil, nil -} - -func (p *mockBinlogPump) PullBinlogs(ctx context.Context, in *binlog.PullBinlogReq, opts ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { - return mockPumpPullBinlogsClient{mockcopr.MockGRPCClientStream()}, nil -} - -func (s *testSessionSuite) TestForCoverage(c *C) { - // Just for test coverage. - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (id int auto_increment, v int, index (id))") - tk.MustExec("insert t values ()") - tk.MustExec("insert t values ()") - tk.MustExec("insert t values ()") - - // Normal request will not cover txn.Seek. - tk.MustExec("admin check table t") - - // Cover dirty table operations in StateTxn. - tk.Se.GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&mockBinlogPump{}) - tk.MustExec("begin") - tk.MustExec("truncate table t") - tk.MustExec("insert t values ()") - tk.MustExec("delete from t where id = 2") - tk.MustExec("update t set v = 5 where id = 2") - tk.MustExec("insert t values ()") - tk.MustExec("rollback") - - c.Check(tk.Se.SetCollation(mysql.DefaultCollationID), IsNil) - - tk.MustExec("show processlist") - _, err := tk.Se.FieldList("t") - c.Check(err, IsNil) -} - func (s *testSessionSuite2) TestErrorRollback(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t_rollback") diff --git a/session/backup_restore_test.go b/tests/realtikvtest/backup_restore_test.go similarity index 96% rename from session/backup_restore_test.go rename to tests/realtikvtest/backup_restore_test.go index b8f8d8f34e0cc..8990f96714f54 100644 --- a/session/backup_restore_test.go +++ b/tests/realtikvtest/backup_restore_test.go @@ -15,7 +15,7 @@ // This file contains tests about backup restore (br) which need running with real TiKV. // Only tests under /session will be run with real TiKV, so we put them here instead of /br. -package session_test +package realtikvtest import ( "os" @@ -27,9 +27,8 @@ import ( "github.com/stretchr/testify/require" ) -// TODO move this test to BR integration tests. func TestBackupAndRestore(t *testing.T) { - if !*withTiKV { + if !*withRealTiKV { t.Skip("only run BR SQL integration test with tikv store") } diff --git a/tests/realtikvtest/binlog_test.go b/tests/realtikvtest/binlog_test.go new file mode 100644 index 0000000000000..fb80cacc1be40 --- /dev/null +++ b/tests/realtikvtest/binlog_test.go @@ -0,0 +1,85 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// This file contains tests about binlog which need running with real TiKV. +// Only tests under /session will be run with real TiKV, so we put them here. + +package realtikvtest + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/binloginfo" + "github.com/pingcap/tidb/store/mockstore/mockcopr" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tipb/go-binlog" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" +) + +type mockBinlogPump struct { +} + +var _ binlog.PumpClient = &mockBinlogPump{} + +func (p *mockBinlogPump) WriteBinlog(_ context.Context, _ *binlog.WriteBinlogReq, _ ...grpc.CallOption) (*binlog.WriteBinlogResp, error) { + return &binlog.WriteBinlogResp{}, nil +} + +func (p *mockBinlogPump) PullBinlogs(_ context.Context, _ *binlog.PullBinlogReq, _ ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { + return mockPumpPullBinlogsClient{mockcopr.MockGRPCClientStream()}, nil +} + +type mockPumpPullBinlogsClient struct { + grpc.ClientStream +} + +func (m mockPumpPullBinlogsClient) Recv() (*binlog.PullBinlogResp, error) { + return nil, nil +} + +func TestForCoverage(t *testing.T) { + // Just for test coverage. + store, clean := createMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int auto_increment, v int, index (id))") + tk.MustExec("insert t values ()") + tk.MustExec("insert t values ()") + tk.MustExec("insert t values ()") + + // Normal request will not cover txn.Seek. + tk.MustExec("admin check table t") + + // Cover dirty table operations in StateTxn. + tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&mockBinlogPump{}) + tk.MustExec("begin") + tk.MustExec("truncate table t") + tk.MustExec("insert t values ()") + tk.MustExec("delete from t where id = 2") + tk.MustExec("update t set v = 5 where id = 2") + tk.MustExec("insert t values ()") + tk.MustExec("rollback") + + require.NoError(t, tk.Session().SetCollation(mysql.DefaultCollationID)) + + tk.MustExec("show processlist") + _, err := tk.Session().FieldList("t") + require.NoError(t, err) +} diff --git a/session/isolation_test.go b/tests/realtikvtest/isolation_test.go similarity index 99% rename from session/isolation_test.go rename to tests/realtikvtest/isolation_test.go index 9e211b103de2d..fededd0adae76 100644 --- a/session/isolation_test.go +++ b/tests/realtikvtest/isolation_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package session_test +package realtikvtest import ( "testing" diff --git a/tests/realtikvtest/main_test.go b/tests/realtikvtest/main_test.go new file mode 100644 index 0000000000000..1ab62ffeb722a --- /dev/null +++ b/tests/realtikvtest/main_test.go @@ -0,0 +1,146 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package realtikvtest + +import ( + "context" + "flag" + "sync/atomic" + "testing" + "time" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/driver" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testbridge" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikv" + "github.com/tikv/client-go/v2/txnkv/transaction" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + "google.golang.org/grpc" +) + +var withRealTiKV = flag.Bool("with-real-tikv", false, "whether tests run with real TiKV") + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + flag.Parse() + session.SetSchemaLease(20 * time.Millisecond) + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + }) + tikv.EnableFailpoints() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/internal/retry.newBackoffFn.func1"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/v3.waitRetryBackoff"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), + goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), + goleak.IgnoreTopFunction("google.golang.org/grpc/internal/transport.(*controlBuffer).get"), + goleak.IgnoreTopFunction("google.golang.org/grpc/internal/transport.(*http2Client).keepalive"), + goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), + goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), + } + goleak.VerifyTestMain(m, opts...) +} + +func clearTiKVStorage(t *testing.T, store kv.Storage) { + txn, err := store.Begin() + require.NoError(t, err) + iter, err := txn.Iter(nil, nil) + require.NoError(t, err) + for iter.Valid() { + require.NoError(t, txn.Delete(iter.Key())) + require.NoError(t, iter.Next()) + } + require.NoError(t, txn.Commit(context.Background())) +} + +func clearEtcdStorage(t *testing.T, backend kv.EtcdBackend) { + endpoints, err := backend.EtcdAddrs() + require.NoError(t, err) + cli, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + AutoSyncInterval: 30 * time.Second, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpc.WithBackoffMaxDelay(time.Second * 3), + }, + TLS: backend.TLSConfig(), + }) + require.NoError(t, err) + defer func() { require.NoError(t, cli.Close()) }() + resp, err := cli.Get(context.Background(), "/tidb", clientv3.WithPrefix()) + require.NoError(t, err) + for _, entry := range resp.Kvs { + if entry.Lease != 0 { + _, err := cli.Revoke(context.Background(), clientv3.LeaseID(entry.Lease)) + require.NoError(t, err) + } + } + _, err = cli.Delete(context.Background(), "/tidb", clientv3.WithPrefix()) + require.NoError(t, err) +} + +func createMockStoreAndSetup(t *testing.T, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, func()) { + store, _, clean := createMockStoreAndDomainAndSetup(t, opts...) + return store, clean +} + +func createMockStoreAndDomainAndSetup(t *testing.T, opts ...mockstore.MockTiKVStoreOption) (kv.Storage, *domain.Domain, func()) { + // set it to 5 seconds for testing lock resolve. + atomic.StoreUint64(&transaction.ManagedLockTTL, 5000) + transaction.PrewriteMaxBackoff.Store(500) + + var store kv.Storage + var dom *domain.Domain + var err error + + if *withRealTiKV { + var d driver.TiKVDriver + config.UpdateGlobal(func(conf *config.Config) { + conf.TxnLocalLatches.Enabled = false + }) + store, err = d.Open("tikv://127.0.0.1:2379?disableGC=true") + require.NoError(t, err) + + clearTiKVStorage(t, store) + clearEtcdStorage(t, store.(kv.EtcdBackend)) + + session.ResetStoreForWithTiKVTest(store) + dom, err = session.BootstrapSession(store) + require.NoError(t, err) + + } else { + store, err = mockstore.NewMockStore(opts...) + require.NoError(t, err) + session.DisableStats4Test() + dom, err = session.BootstrapSession(store) + require.NoError(t, err) + } + + return store, dom, func() { + dom.Close() + require.NoError(t, store.Close()) + transaction.PrewriteMaxBackoff.Store(20000) + } +} diff --git a/session/session_fail_test.go b/tests/realtikvtest/session_fail_test.go similarity index 86% rename from session/session_fail_test.go rename to tests/realtikvtest/session_fail_test.go index cc9ea66ab1a44..08a0d1db8c385 100644 --- a/session/session_fail_test.go +++ b/tests/realtikvtest/session_fail_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package session_test +package realtikvtest import ( "context" @@ -21,6 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -29,7 +30,9 @@ func TestFailStatementCommitInRetry(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") tk.MustExec("begin") @@ -50,7 +53,9 @@ func TestGetTSFailDirtyState(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/mockGetTSFail", "return")) @@ -80,7 +85,9 @@ func TestGetTSFailDirtyStateInretry(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError", `return(true)`)) @@ -98,7 +105,9 @@ func TestKillFlagInBackoff(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table kill_backoff (id int)") // Inject 1 time timeout. If `Killed` is not successfully passed, it will retry and complete query. require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("timeout")->return("")`)) @@ -116,7 +125,9 @@ func TestClusterTableSendError(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("requestTiDBStoreError")`)) defer failpoint.Disable("tikvclient/tikvStoreSendReqResult") tk.MustQuery("select * from information_schema.cluster_slow_query") @@ -128,7 +139,9 @@ func TestAutoCommitNeedNotLinearizability(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() - tk := createTestKit(t, store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1;") defer tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1 (c int)`) diff --git a/session/session_test.go b/tests/realtikvtest/session_test.go similarity index 91% rename from session/session_test.go rename to tests/realtikvtest/session_test.go index 1eed88061b011..890594cbd66b3 100644 --- a/session/session_test.go +++ b/tests/realtikvtest/session_test.go @@ -12,112 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -package session_test +package realtikvtest import ( - "context" "fmt" "testing" "time" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" - clientv3 "go.etcd.io/etcd/client/v3" - "google.golang.org/grpc" ) -func clearTiKVStorage(store kv.Storage) error { - txn, err := store.Begin() - if err != nil { - return errors.Trace(err) - } - iter, err := txn.Iter(nil, nil) - if err != nil { - return errors.Trace(err) - } - for iter.Valid() { - if err := txn.Delete(iter.Key()); err != nil { - return errors.Trace(err) - } - if err := iter.Next(); err != nil { - return errors.Trace(err) - } - } - return txn.Commit(context.Background()) -} - -func clearEtcdStorage(ebd kv.EtcdBackend) error { - endpoints, err := ebd.EtcdAddrs() - if err != nil { - return err - } - cli, err := clientv3.New(clientv3.Config{ - Endpoints: endpoints, - AutoSyncInterval: 30 * time.Second, - DialTimeout: 5 * time.Second, - DialOptions: []grpc.DialOption{ - grpc.WithBackoffMaxDelay(time.Second * 3), - }, - TLS: ebd.TLSConfig(), - }) - if err != nil { - return errors.Trace(err) - } - defer cli.Close() - - resp, err := cli.Get(context.Background(), "/tidb", clientv3.WithPrefix()) - if err != nil { - return errors.Trace(err) - } - for _, kv := range resp.Kvs { - if kv.Lease != 0 { - if _, err := cli.Revoke(context.Background(), clientv3.LeaseID(kv.Lease)); err != nil { - return errors.Trace(err) - } - } - } - _, err = cli.Delete(context.Background(), "/tidb", clientv3.WithPrefix()) - if err != nil { - return errors.Trace(err) - } - return nil -} - -func createMockStoreAndSetup(t *testing.T) (kv.Storage, func()) { - store, _, clean := createMockStoreAndDomainAndSetup(t) - return store, clean -} - -func createMockStoreAndDomainAndSetup(t *testing.T) (kv.Storage, *domain.Domain, func()) { - if *withTiKV { - var d driver.TiKVDriver - config.UpdateGlobal(func(conf *config.Config) { - conf.TxnLocalLatches.Enabled = false - }) - store, err := d.Open("tikv://127.0.0.1:2379?disableGC=true") - require.NoError(t, err) - require.NoError(t, clearTiKVStorage(store)) - require.NoError(t, clearEtcdStorage(store.(kv.EtcdBackend))) - session.ResetStoreForWithTiKVTest(store) - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - - return store, dom, func() { - dom.Close() - require.NoError(t, store.Close()) - } - } - return testkit.CreateMockStoreAndDomain(t) -} - func TestSysdateIsNow(t *testing.T) { store, clean := createMockStoreAndSetup(t) defer clean() diff --git a/session/statistics_test.go b/tests/realtikvtest/statistics_test.go similarity index 99% rename from session/statistics_test.go rename to tests/realtikvtest/statistics_test.go index 83e6d3cc8f7d3..ffc94e429ec22 100644 --- a/session/statistics_test.go +++ b/tests/realtikvtest/statistics_test.go @@ -15,7 +15,7 @@ // This file contains tests about statistics which need running with real TiKV. // Only tests under /session will be run with real TiKV, so we put them here instead of /statistics. -package session_test +package realtikvtest import ( "fmt" @@ -50,8 +50,6 @@ func TestNewCollationStatsWithPrefixIndex(t *testing.T) { tk.MustExec("insert into t values('aaAAaaaAAAabbc'), ('AaAaAaAaAaAbBC'), ('AAAaabbBBbbb'), ('AAAaabbBBbbbccc'), ('aaa'), ('Aa'), ('A'), ('ab')") tk.MustExec("insert into t values('b'), ('bBb'), ('Bb'), ('bA'), ('BBBB'), ('BBBBBDDDDDdd'), ('bbbbBBBBbbBBR'), ('BBbbBBbbBBbbBBRRR')") h := dom.StatsHandle() - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - tk.MustExec("set @@session.tidb_analyze_version=1") require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) diff --git a/session/txn_state_test.go b/tests/realtikvtest/txn_state_test.go similarity index 99% rename from session/txn_state_test.go rename to tests/realtikvtest/txn_state_test.go index 6e087a166bb34..ff6b626b2fbcd 100644 --- a/session/txn_state_test.go +++ b/tests/realtikvtest/txn_state_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package session_test +package realtikvtest import ( "context"