From 4dc385ce8b7a3b194462e76670f89483de5884b7 Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 23 Dec 2021 20:01:47 +0800 Subject: [PATCH] test: merge serial tests in ddl, infoschema, session, store, table, telemetry and types (#30874) --- ddl/db_cache_serial_test.go | 97 ----- ddl/db_cache_test.go | 70 ++++ ...m_serial_test.go => ddl_algorithm_test.go} | 0 ...fail_db_serial_test.go => fail_db_test.go} | 0 .../{syncer_serial_test.go => syncer_test.go} | 0 ..._serial_test.go => cluster_tables_test.go} | 0 infoschema/perfschema/tables_serial_test.go | 159 -------- infoschema/perfschema/tables_test.go | 136 +++++++ .../{tables_serial_test.go => tables_test.go} | 0 ...strap_serial_test.go => bootstrap_test.go} | 0 session/clustered_index_serial_test.go | 341 ------------------ session/clustered_index_test.go | 318 ++++++++++++++++ ...test.go => index_usage_sync_lease_test.go} | 0 ..._serial_test.go => schema_amender_test.go} | 0 ...rial_test.go => batch_coprocessor_test.go} | 0 .../{config_serial_test.go => config_test.go} | 0 ...erial_test.go => snap_interceptor_test.go} | 0 ...l_fail_serial_test.go => sql_fail_test.go} | 0 .../{txn_serial_test.go => txn_test.go} | 0 ...orker_serial_test.go => gc_worker_test.go} | 0 table/tables/index_serial_test.go | 324 ----------------- table/tables/index_test.go | 295 ++++++++++++++- ...dow_serial_test.go => data_window_test.go} | 0 telemetry/telemetry_serial_test.go | 54 --- telemetry/telemetry_test.go | 26 ++ types/{enum_serial_test.go => enum_test.go} | 0 types/mydecimal_serial_test.go | 199 ---------- types/mydecimal_test.go | 178 +++++++++ types/{set_serial_test.go => set_test.go} | 0 29 files changed, 1022 insertions(+), 1175 deletions(-) delete mode 100644 ddl/db_cache_serial_test.go rename ddl/{ddl_algorithm_serial_test.go => ddl_algorithm_test.go} (100%) rename ddl/failtest/{fail_db_serial_test.go => fail_db_test.go} (100%) rename ddl/util/{syncer_serial_test.go => syncer_test.go} (100%) rename infoschema/{cluster_tables_serial_test.go => cluster_tables_test.go} (100%) delete mode 100644 infoschema/perfschema/tables_serial_test.go rename infoschema/{tables_serial_test.go => tables_test.go} (100%) rename session/{bootstrap_serial_test.go => bootstrap_test.go} (100%) delete mode 100644 session/clustered_index_serial_test.go rename session/{index_usage_sync_lease_serial_test.go => index_usage_sync_lease_test.go} (100%) rename session/{schema_amender_serial_test.go => schema_amender_test.go} (100%) rename store/{batch_coprocessor_serial_test.go => batch_coprocessor_test.go} (100%) rename store/driver/{config_serial_test.go => config_test.go} (100%) rename store/driver/{snap_interceptor_serial_test.go => snap_interceptor_test.go} (100%) rename store/driver/{sql_fail_serial_test.go => sql_fail_test.go} (100%) rename store/driver/{txn_serial_test.go => txn_test.go} (100%) rename store/gcworker/{gc_worker_serial_test.go => gc_worker_test.go} (100%) delete mode 100644 table/tables/index_serial_test.go rename telemetry/{data_window_serial_test.go => data_window_test.go} (100%) delete mode 100644 telemetry/telemetry_serial_test.go rename types/{enum_serial_test.go => enum_test.go} (100%) delete mode 100644 types/mydecimal_serial_test.go rename types/{set_serial_test.go => set_test.go} (100%) diff --git a/ddl/db_cache_serial_test.go b/ddl/db_cache_serial_test.go deleted file mode 100644 index e024cef522682..0000000000000 --- a/ddl/db_cache_serial_test.go +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2021 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 ddl_test - -import ( - "testing" - "time" - - "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" -) - -func TestAlterTableCache(t *testing.T) { - store, err := mockstore.NewMockStore() - require.NoError(t, err) - session.SetSchemaLease(600 * time.Millisecond) - session.DisableStats4Test() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - - dom.SetStatsUpdating(true) - - clean := func() { - dom.Close() - err := store.Close() - require.NoError(t, err) - } - defer clean() - tk := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk2.MustExec("use test") - /* Test of cache table */ - tk.MustExec("create table t1 ( n int auto_increment primary key)") - tk.MustGetErrCode("alter table t1 ca", errno.ErrParse) - tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable) - tk.MustExec("alter table t1 cache") - checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable) - tk.MustExec("drop table if exists t1") - /*Test can't skip schema checker*/ - tk.MustExec("drop table if exists t1,t2") - tk.MustExec("CREATE TABLE t1 (a int)") - tk.MustExec("CREATE TABLE t2 (a int)") - tk.MustExec("begin") - tk.MustExec("insert into t1 set a=1;") - tk2.MustExec("alter table t1 cache;") - _, err = tk.Exec("commit") - require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err)) - /* Test can skip schema checker */ - tk.MustExec("begin") - tk.MustExec("drop table if exists t1") - tk.MustExec("CREATE TABLE t1 (a int)") - tk.MustExec("insert into t1 set a=2;") - tk2.MustExec("alter table t2 cache") - tk.MustExec("commit") - // Test if a table is not exists - tk.MustExec("drop table if exists t") - tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable) - tk.MustExec("create table t (a int)") - tk.MustExec("alter table t cache") - // Multiple alter cache is okay - tk.MustExec("alter table t cache") - tk.MustExec("alter table t cache") - // Test a temporary table - tk.MustExec("drop table if exists t") - tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)") - tk.MustExec("drop table if exists tmp1") - // local temporary table alter is not supported - tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation) - // test global temporary table - tk.MustExec("create global temporary table tmp1 " + - "(id int not null primary key, code int not null, value int default null, unique key code(code))" + - "on commit delete rows") - tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error()) - -} diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go index 120c0c687e049..506b8d7089fe6 100644 --- a/ddl/db_cache_test.go +++ b/ddl/db_cache_test.go @@ -16,12 +16,16 @@ package ddl_test import ( "testing" + "time" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -133,3 +137,69 @@ func TestIndexOnCacheTable(t *testing.T) { tk.MustExec("alter table cache_index_1 cache") tk.MustGetErrCode("alter table cache_index_1 drop index i1, drop index i2;", errno.ErrOptOnCacheTable) } + +func TestAlterTableCache(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + session.SetSchemaLease(600 * time.Millisecond) + session.DisableStats4Test() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + + dom.SetStatsUpdating(true) + + clean := func() { + dom.Close() + err := store.Close() + require.NoError(t, err) + } + defer clean() + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk2.MustExec("use test") + /* Test of cache table */ + tk.MustExec("create table t1 ( n int auto_increment primary key)") + tk.MustGetErrCode("alter table t1 ca", errno.ErrParse) + tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable) + tk.MustExec("alter table t1 cache") + checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable) + tk.MustExec("drop table if exists t1") + /*Test can't skip schema checker*/ + tk.MustExec("drop table if exists t1,t2") + tk.MustExec("CREATE TABLE t1 (a int)") + tk.MustExec("CREATE TABLE t2 (a int)") + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1;") + tk2.MustExec("alter table t1 cache;") + _, err = tk.Exec("commit") + require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err)) + /* Test can skip schema checker */ + tk.MustExec("begin") + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE t1 (a int)") + tk.MustExec("insert into t1 set a=2;") + tk2.MustExec("alter table t2 cache") + tk.MustExec("commit") + // Test if a table is not exists + tk.MustExec("drop table if exists t") + tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable) + tk.MustExec("create table t (a int)") + tk.MustExec("alter table t cache") + // Multiple alter cache is okay + tk.MustExec("alter table t cache") + tk.MustExec("alter table t cache") + // Test a temporary table + tk.MustExec("drop table if exists t") + tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)") + tk.MustExec("drop table if exists tmp1") + // local temporary table alter is not supported + tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation) + // test global temporary table + tk.MustExec("create global temporary table tmp1 " + + "(id int not null primary key, code int not null, value int default null, unique key code(code))" + + "on commit delete rows") + tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error()) +} diff --git a/ddl/ddl_algorithm_serial_test.go b/ddl/ddl_algorithm_test.go similarity index 100% rename from ddl/ddl_algorithm_serial_test.go rename to ddl/ddl_algorithm_test.go diff --git a/ddl/failtest/fail_db_serial_test.go b/ddl/failtest/fail_db_test.go similarity index 100% rename from ddl/failtest/fail_db_serial_test.go rename to ddl/failtest/fail_db_test.go diff --git a/ddl/util/syncer_serial_test.go b/ddl/util/syncer_test.go similarity index 100% rename from ddl/util/syncer_serial_test.go rename to ddl/util/syncer_test.go diff --git a/infoschema/cluster_tables_serial_test.go b/infoschema/cluster_tables_test.go similarity index 100% rename from infoschema/cluster_tables_serial_test.go rename to infoschema/cluster_tables_test.go diff --git a/infoschema/perfschema/tables_serial_test.go b/infoschema/perfschema/tables_serial_test.go deleted file mode 100644 index f8126a57b2b7c..0000000000000 --- a/infoschema/perfschema/tables_serial_test.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2021 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 perfschema_test - -import ( - "fmt" - "io" - "net/http" - "net/http/httptest" - "os" - "path/filepath" - "runtime/pprof" - "strings" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" -) - -func TestTiKVProfileCPU(t *testing.T) { - store, clean := newMockStore(t) - defer clean() - - router := http.NewServeMux() - mockServer := httptest.NewServer(router) - mockAddr := strings.TrimPrefix(mockServer.URL, "http://") - defer mockServer.Close() - - // mock tikv profile - copyHandler := func(filename string) http.HandlerFunc { - return func(w http.ResponseWriter, _ *http.Request) { - file, err := os.Open(filepath.Join(currentSourceDir(), filename)) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - defer func() { terror.Log(file.Close()) }() - _, err = io.Copy(w, file) - terror.Log(err) - } - } - router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) - - // failpoint setting - servers := []string{ - strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), - strings.Join([]string{"pd", mockAddr, mockAddr}, ","), - } - fpExpr := strings.Join(servers, ";") - fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" - require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) - defer func() { require.NoError(t, failpoint.Disable(fpName)) }() - - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use performance_schema") - result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") - - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - result.Check(testkit.Rows( - "root 100% 100%", - "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", - "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", - "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", - "│ ├─Unknown 3.57% 22.22%", - "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", - "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", - "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", - "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", - "│ └─rallocx 1.79% 100%", - "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", - "│ └─sdallocx 1.79% 100%", - "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", - "│ └─mallocx 0.89% 100%", - "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", - "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", - "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", - "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", - "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", - "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", - "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", - "└─::next::he129c78b3deb639d 0.89% 0.89%", - " └─Unknown 0.89% 100%")) - - // We can use current processe profile to mock profile of PD because the PD has the - // same way of retrieving profile with TiDB. And the purpose of this test case is used - // to make sure all profile HTTP API have been accessed. - accessed := map[string]struct{}{} - handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { - debugLevel := 0 - if len(debug) > 0 { - debugLevel = debug[0] - } - return func(w http.ResponseWriter, _ *http.Request) { - profile := pprof.Lookup(name) - if profile == nil { - http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) - return - } - if err := profile.WriteTo(w, debugLevel); err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - accessed[name] = struct{}{} - } - } - - // mock PD profile - router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) - router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) - router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) - router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) - router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) - router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) - - tk.MustQuery("select * from pd_profile_cpu where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_memory where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_mutex where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_allocs where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_block where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_goroutines") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) -} diff --git a/infoschema/perfschema/tables_test.go b/infoschema/perfschema/tables_test.go index be739a5b6af48..057ba404efec7 100644 --- a/infoschema/perfschema/tables_test.go +++ b/infoschema/perfschema/tables_test.go @@ -15,12 +15,21 @@ package perfschema_test import ( + "fmt" + "io" + "net/http" + "net/http/httptest" + "os" "path/filepath" "runtime" + "runtime/pprof" + "strings" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema/perfschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" @@ -44,6 +53,133 @@ func TestPerfSchemaTables(t *testing.T) { tk.MustQuery("select * from events_stages_history_long").Check(testkit.Rows()) } +func TestTiKVProfileCPU(t *testing.T) { + store, clean := newMockStore(t) + defer clean() + + router := http.NewServeMux() + mockServer := httptest.NewServer(router) + mockAddr := strings.TrimPrefix(mockServer.URL, "http://") + defer mockServer.Close() + + // mock tikv profile + copyHandler := func(filename string) http.HandlerFunc { + return func(w http.ResponseWriter, _ *http.Request) { + file, err := os.Open(filepath.Join(currentSourceDir(), filename)) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + defer func() { terror.Log(file.Close()) }() + _, err = io.Copy(w, file) + terror.Log(err) + } + } + router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) + + // failpoint setting + servers := []string{ + strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), + strings.Join([]string{"pd", mockAddr, mockAddr}, ","), + } + fpExpr := strings.Join(servers, ";") + fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" + require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) + defer func() { require.NoError(t, failpoint.Disable(fpName)) }() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use performance_schema") + result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") + + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + result.Check(testkit.Rows( + "root 100% 100%", + "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", + "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", + "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", + "│ ├─Unknown 3.57% 22.22%", + "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", + "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", + "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", + "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", + "│ └─rallocx 1.79% 100%", + "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", + "│ └─sdallocx 1.79% 100%", + "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", + "│ └─mallocx 0.89% 100%", + "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", + "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", + "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", + "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", + "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", + "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", + "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", + "└─::next::he129c78b3deb639d 0.89% 0.89%", + " └─Unknown 0.89% 100%")) + + // We can use current processe profile to mock profile of PD because the PD has the + // same way of retrieving profile with TiDB. And the purpose of this test case is used + // to make sure all profile HTTP API have been accessed. + accessed := map[string]struct{}{} + handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { + debugLevel := 0 + if len(debug) > 0 { + debugLevel = debug[0] + } + return func(w http.ResponseWriter, _ *http.Request) { + profile := pprof.Lookup(name) + if profile == nil { + http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) + return + } + if err := profile.WriteTo(w, debugLevel); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + accessed[name] = struct{}{} + } + } + + // mock PD profile + router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) + router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) + router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) + router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) + router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) + router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) + + tk.MustQuery("select * from pd_profile_cpu where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_memory where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_mutex where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_allocs where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_block where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_goroutines") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) +} + func newMockStore(t *testing.T) (store kv.Storage, clean func()) { var err error store, err = mockstore.NewMockStore() diff --git a/infoschema/tables_serial_test.go b/infoschema/tables_test.go similarity index 100% rename from infoschema/tables_serial_test.go rename to infoschema/tables_test.go diff --git a/session/bootstrap_serial_test.go b/session/bootstrap_test.go similarity index 100% rename from session/bootstrap_serial_test.go rename to session/bootstrap_test.go diff --git a/session/clustered_index_serial_test.go b/session/clustered_index_serial_test.go deleted file mode 100644 index 056c525944ac0..0000000000000 --- a/session/clustered_index_serial_test.go +++ /dev/null @@ -1,341 +0,0 @@ -// Copyright 2021 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 session_test - -import ( - "fmt" - "math/rand" - "strings" - "testing" - - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/israce" -) - -func TestCreateClusteredTable(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := createTestKit(t, store) - tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'off';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'on';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") - defer config.RestoreFunc()() - config.UpdateGlobal(func(conf *config.Config) { - conf.AlterPrimaryKey = true - }) - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) -} - -// Test for union scan in prefixed clustered index table. -// See https://github.com/pingcap/tidb/issues/22069. -func TestClusteredUnionScanOnPrefixingPrimaryKey(t *testing.T) { - originCollate := collate.NewCollationEnabled() - collate.SetNewCollationEnabledForTest(false) - defer collate.SetNewCollationEnabledForTest(originCollate) - store, clean := testkit.CreateMockStore(t) - defer clean() - tk := createTestKit(t, store) - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (col_1 varchar(255), col_2 tinyint, primary key idx_1 (col_1(1)));") - tk.MustExec("insert into t values ('aaaaa', -38);") - tk.MustExec("insert into t values ('bbbbb', -48);") - - tk.MustExec("begin PESSIMISTIC;") - tk.MustExec("update t set col_2 = 47 where col_1 in ('aaaaa') order by col_1,col_2;") - tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) - tk.MustGetErrCode("insert into t values ('bb', 0);", errno.ErrDupEntry) - tk.MustGetErrCode("insert into t values ('aa', 0);", errno.ErrDupEntry) - tk.MustExec("commit;") - tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) - tk.MustExec("admin check table t;") -} - -// https://github.com/pingcap/tidb/issues/22453 -func TestClusteredIndexSplitAndAddIndex2(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := createTestKit(t, store) - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int, b enum('Alice'), c int, primary key (c, b));") - tk.MustExec("insert into t values (-1,'Alice',100);") - tk.MustExec("insert into t values (-1,'Alice',7000);") - tk.MustQuery("split table t between (0,'Alice') and (10000,'Alice') regions 2;").Check(testkit.Rows("1 1")) - tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3;") - tk.MustExec("alter table t add index idx (c);") - tk.MustExec("admin check table t;") -} - -func TestClusteredIndexSyntax(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - const showPKType = `select tidb_pk_type from information_schema.tables where table_schema = 'test' and table_name = 't';` - const nonClustered, clustered = `NONCLUSTERED`, `CLUSTERED` - assertPkType := func(sql string, pkType string) { - tk.MustExec("drop table if exists t;") - tk.MustExec(sql) - tk.MustQuery(showPKType).Check(testkit.Rows(pkType)) - } - - // Test single integer column as the primary key. - clusteredDefault := clustered - assertPkType("create table t (a int primary key, b int);", clusteredDefault) - assertPkType("create table t (a int, b int, primary key(a) clustered);", clustered) - assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] clustered */);", clustered) - assertPkType("create table t (a int, b int, primary key(a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] nonclustered */);", nonClustered) - - // Test for clustered index. - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - assertPkType("create table t (a int, b varchar(255), primary key(b, a));", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - assertPkType("create table t (a int, b varchar(255), primary key(b, a));", clusteredDefault) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] nonclustered */);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] clustered */);", clustered) - - tk.MustGetErrCode("create table t (a varchar(255) unique key clustered);", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) reference t1(a) clustered);", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) clustered reference t1(a));", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255) clustered);", errno.ErrParse) - - errMsg := "[ddl:8200]CLUSTERED/NONCLUSTERED keyword is only supported for primary key" - tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), key(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), key(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), index(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), index(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) clustered)", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) nonclustered)", errMsg) -} - -func TestPrefixClusteredIndexAddIndexAndRecover(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test;") - tk1.MustExec("drop table if exists t;") - defer func() { - tk1.MustExec("drop table if exists t;") - }() - - tk1.MustExec("create table t(a char(3), b char(3), primary key(a(1)) clustered)") - tk1.MustExec("insert into t values ('aaa', 'bbb')") - tk1.MustExec("alter table t add index idx(b)") - tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) - tk1.MustExec("admin check table t") - tk1.MustExec("admin recover index t idx") - tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) - tk1.MustExec("admin check table t") -} - -func TestPartitionTable(t *testing.T) { - if israce.RaceEnabled { - t.Skip("exhaustive types test, skip race test") - } - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database test_view") - tk.MustExec("use test_view") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - - tk.MustExec(`create table thash (a int, b int, c varchar(32), primary key(a, b) clustered) partition by hash(a) partitions 4`) - tk.MustExec(`create table trange (a int, b int, c varchar(32), primary key(a, b) clustered) partition by range columns(a) ( - partition p0 values less than (3000), - partition p1 values less than (6000), - partition p2 values less than (9000), - partition p3 values less than (10000))`) - tk.MustExec(`create table tnormal (a int, b int, c varchar(32), primary key(a, b))`) - - vals := make([]string, 0, 4000) - existedPK := make(map[string]struct{}, 4000) - for i := 0; i < 4000; { - a := rand.Intn(10000) - b := rand.Intn(10000) - pk := fmt.Sprintf("%v, %v", a, b) - if _, ok := existedPK[pk]; ok { - continue - } - existedPK[pk] = struct{}{} - i++ - vals = append(vals, fmt.Sprintf(`(%v, %v, '%v')`, a, b, rand.Intn(10000))) - } - - tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) - tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) - tk.MustExec("insert into tnormal values " + strings.Join(vals, ", ")) - - for i := 0; i < 200; i++ { - cond := fmt.Sprintf("where a in (%v, %v, %v) and b < %v", rand.Intn(10000), rand.Intn(10000), rand.Intn(10000), rand.Intn(10000)) - result := tk.MustQuery("select * from tnormal " + cond).Sort().Rows() - tk.MustQuery("select * from thash use index(primary) " + cond).Sort().Check(result) - tk.MustQuery("select * from trange use index(primary) " + cond).Sort().Check(result) - } -} - -// https://github.com/pingcap/tidb/issues/23106 -func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered) collate utf8mb4_general_ci;") - tk.MustExec("insert into t values (1, 'asd', 1), (1, 'dsa', 1);") - tk.MustGetErrCode("alter table t add unique index t_idx(id1, a1);", errno.ErrDupEntry) - - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered, unique key t_idx(id1, a1)) collate utf8mb4_general_ci;") - tk.MustExec("begin;") - tk.MustExec("insert into t values (1, 'asd', 1);") - tk.MustQuery("select * from t use index (t_idx);").Check(testkit.Rows("1 asd 1")) - tk.MustExec("commit;") - tk.MustExec("admin check table t;") - tk.MustExec("drop table t;") -} - -// https://github.com/pingcap/tidb/issues/23178 -func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table t (a text collate utf8mb4_general_ci not null, b int(11) not null, " + - "primary key (a(10), b) clustered, key idx(a(2)) ) default charset=utf8mb4 collate=utf8mb4_bin;") - tk.MustExec("insert into t values ('aaa', 2);") - // Key-value content: sk = sortKey, p = prefixed - // row record: sk(aaa), 2 -> aaa - // index record: sk(p(aa)), {sk(aaa), 2} -> restore data(aaa) - tk.MustExec("admin check table t;") - tk.MustExec("drop table t;") -} - -func TestClusteredIndexNewCollationWithOldRowFormat(t *testing.T) { - // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) - // but unistore doesn't support old row format. - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.Session().GetSessionVars().RowEncoder.Enable = false - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") - tk.MustExec("insert into t2 select 'aBc'") - tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) -} diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index b993bd3405a6d..32de5a4348632 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -15,14 +15,20 @@ package session_test import ( + "fmt" + "math/rand" + "strings" "testing" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/stretchr/testify/require" ) @@ -421,3 +427,315 @@ func TestClusteredIndexSelectWhereInNull(t *testing.T) { tk.MustExec("create table t (a datetime, b bigint, primary key (a));") tk.MustQuery("select * from t where a in (null);").Check(testkit.Rows( /* empty result */ )) } + +func TestCreateClusteredTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := createTestKit(t, store) + tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'off';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'on';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.AlterPrimaryKey = true + }) + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) +} + +// Test for union scan in prefixed clustered index table. +// See https://github.com/pingcap/tidb/issues/22069. +func TestClusteredUnionScanOnPrefixingPrimaryKey(t *testing.T) { + originCollate := collate.NewCollationEnabled() + collate.SetNewCollationEnabledForTest(false) + defer collate.SetNewCollationEnabledForTest(originCollate) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createTestKit(t, store) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (col_1 varchar(255), col_2 tinyint, primary key idx_1 (col_1(1)));") + tk.MustExec("insert into t values ('aaaaa', -38);") + tk.MustExec("insert into t values ('bbbbb', -48);") + + tk.MustExec("begin PESSIMISTIC;") + tk.MustExec("update t set col_2 = 47 where col_1 in ('aaaaa') order by col_1,col_2;") + tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) + tk.MustGetErrCode("insert into t values ('bb', 0);", errno.ErrDupEntry) + tk.MustGetErrCode("insert into t values ('aa', 0);", errno.ErrDupEntry) + tk.MustExec("commit;") + tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) + tk.MustExec("admin check table t;") +} + +// https://github.com/pingcap/tidb/issues/22453 +func TestClusteredIndexSplitAndAddIndex2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := createTestKit(t, store) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b enum('Alice'), c int, primary key (c, b));") + tk.MustExec("insert into t values (-1,'Alice',100);") + tk.MustExec("insert into t values (-1,'Alice',7000);") + tk.MustQuery("split table t between (0,'Alice') and (10000,'Alice') regions 2;").Check(testkit.Rows("1 1")) + tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3;") + tk.MustExec("alter table t add index idx (c);") + tk.MustExec("admin check table t;") +} + +func TestClusteredIndexSyntax(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + const showPKType = `select tidb_pk_type from information_schema.tables where table_schema = 'test' and table_name = 't';` + const nonClustered, clustered = `NONCLUSTERED`, `CLUSTERED` + assertPkType := func(sql string, pkType string) { + tk.MustExec("drop table if exists t;") + tk.MustExec(sql) + tk.MustQuery(showPKType).Check(testkit.Rows(pkType)) + } + + // Test single integer column as the primary key. + clusteredDefault := clustered + assertPkType("create table t (a int primary key, b int);", clusteredDefault) + assertPkType("create table t (a int, b int, primary key(a) clustered);", clustered) + assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] clustered */);", clustered) + assertPkType("create table t (a int, b int, primary key(a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] nonclustered */);", nonClustered) + + // Test for clustered index. + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + assertPkType("create table t (a int, b varchar(255), primary key(b, a));", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + assertPkType("create table t (a int, b varchar(255), primary key(b, a));", clusteredDefault) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] nonclustered */);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] clustered */);", clustered) + + tk.MustGetErrCode("create table t (a varchar(255) unique key clustered);", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) reference t1(a) clustered);", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) clustered reference t1(a));", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255) clustered);", errno.ErrParse) + + errMsg := "[ddl:8200]CLUSTERED/NONCLUSTERED keyword is only supported for primary key" + tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), key(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), key(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), index(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), index(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) clustered)", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) nonclustered)", errMsg) +} + +func TestPrefixClusteredIndexAddIndexAndRecover(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test;") + tk1.MustExec("drop table if exists t;") + defer func() { + tk1.MustExec("drop table if exists t;") + }() + + tk1.MustExec("create table t(a char(3), b char(3), primary key(a(1)) clustered)") + tk1.MustExec("insert into t values ('aaa', 'bbb')") + tk1.MustExec("alter table t add index idx(b)") + tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) + tk1.MustExec("admin check table t") + tk1.MustExec("admin recover index t idx") + tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) + tk1.MustExec("admin check table t") +} + +func TestPartitionTable(t *testing.T) { + if israce.RaceEnabled { + t.Skip("exhaustive types test, skip race test") + } + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database test_view") + tk.MustExec("use test_view") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash (a int, b int, c varchar(32), primary key(a, b) clustered) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, c varchar(32), primary key(a, b) clustered) partition by range columns(a) ( + partition p0 values less than (3000), + partition p1 values less than (6000), + partition p2 values less than (9000), + partition p3 values less than (10000))`) + tk.MustExec(`create table tnormal (a int, b int, c varchar(32), primary key(a, b))`) + + vals := make([]string, 0, 4000) + existedPK := make(map[string]struct{}, 4000) + for i := 0; i < 4000; { + a := rand.Intn(10000) + b := rand.Intn(10000) + pk := fmt.Sprintf("%v, %v", a, b) + if _, ok := existedPK[pk]; ok { + continue + } + existedPK[pk] = struct{}{} + i++ + vals = append(vals, fmt.Sprintf(`(%v, %v, '%v')`, a, b, rand.Intn(10000))) + } + + tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) + tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) + tk.MustExec("insert into tnormal values " + strings.Join(vals, ", ")) + + for i := 0; i < 200; i++ { + cond := fmt.Sprintf("where a in (%v, %v, %v) and b < %v", rand.Intn(10000), rand.Intn(10000), rand.Intn(10000), rand.Intn(10000)) + result := tk.MustQuery("select * from tnormal " + cond).Sort().Rows() + tk.MustQuery("select * from thash use index(primary) " + cond).Sort().Check(result) + tk.MustQuery("select * from trange use index(primary) " + cond).Sort().Check(result) + } +} + +// https://github.com/pingcap/tidb/issues/23106 +func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered) collate utf8mb4_general_ci;") + tk.MustExec("insert into t values (1, 'asd', 1), (1, 'dsa', 1);") + tk.MustGetErrCode("alter table t add unique index t_idx(id1, a1);", errno.ErrDupEntry) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered, unique key t_idx(id1, a1)) collate utf8mb4_general_ci;") + tk.MustExec("begin;") + tk.MustExec("insert into t values (1, 'asd', 1);") + tk.MustQuery("select * from t use index (t_idx);").Check(testkit.Rows("1 asd 1")) + tk.MustExec("commit;") + tk.MustExec("admin check table t;") + tk.MustExec("drop table t;") +} + +// https://github.com/pingcap/tidb/issues/23178 +func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table t (a text collate utf8mb4_general_ci not null, b int(11) not null, " + + "primary key (a(10), b) clustered, key idx(a(2)) ) default charset=utf8mb4 collate=utf8mb4_bin;") + tk.MustExec("insert into t values ('aaa', 2);") + // Key-value content: sk = sortKey, p = prefixed + // row record: sk(aaa), 2 -> aaa + // index record: sk(p(aa)), {sk(aaa), 2} -> restore data(aaa) + tk.MustExec("admin check table t;") + tk.MustExec("drop table t;") +} + +func TestClusteredIndexNewCollationWithOldRowFormat(t *testing.T) { + // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) + // but unistore doesn't support old row format. + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().RowEncoder.Enable = false + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") + tk.MustExec("insert into t2 select 'aBc'") + tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) +} diff --git a/session/index_usage_sync_lease_serial_test.go b/session/index_usage_sync_lease_test.go similarity index 100% rename from session/index_usage_sync_lease_serial_test.go rename to session/index_usage_sync_lease_test.go diff --git a/session/schema_amender_serial_test.go b/session/schema_amender_test.go similarity index 100% rename from session/schema_amender_serial_test.go rename to session/schema_amender_test.go diff --git a/store/batch_coprocessor_serial_test.go b/store/batch_coprocessor_test.go similarity index 100% rename from store/batch_coprocessor_serial_test.go rename to store/batch_coprocessor_test.go diff --git a/store/driver/config_serial_test.go b/store/driver/config_test.go similarity index 100% rename from store/driver/config_serial_test.go rename to store/driver/config_test.go diff --git a/store/driver/snap_interceptor_serial_test.go b/store/driver/snap_interceptor_test.go similarity index 100% rename from store/driver/snap_interceptor_serial_test.go rename to store/driver/snap_interceptor_test.go diff --git a/store/driver/sql_fail_serial_test.go b/store/driver/sql_fail_test.go similarity index 100% rename from store/driver/sql_fail_serial_test.go rename to store/driver/sql_fail_test.go diff --git a/store/driver/txn_serial_test.go b/store/driver/txn_test.go similarity index 100% rename from store/driver/txn_serial_test.go rename to store/driver/txn_test.go diff --git a/store/gcworker/gc_worker_serial_test.go b/store/gcworker/gc_worker_test.go similarity index 100% rename from store/gcworker/gc_worker_serial_test.go rename to store/gcworker/gc_worker_test.go diff --git a/table/tables/index_serial_test.go b/table/tables/index_serial_test.go deleted file mode 100644 index 16784c16d0c7e..0000000000000 --- a/table/tables/index_serial_test.go +++ /dev/null @@ -1,324 +0,0 @@ -// Copyright 2021 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 tables_test - -import ( - "context" - "io" - "testing" - "time" - - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/rowcodec" - "github.com/stretchr/testify/require" -) - -func TestIndex(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test ununiq index. - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - values := types.MakeDatums(1, 2) - mockCtx := mock.NewContext() - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - it, err := index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err := it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) - require.NoError(t, err) - require.False(t, exist) - - exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.True(t, exist) - - err = index.Delete(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0), nil) - require.NoError(t, err) - - _, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, hit, err := index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - err = index.Drop(txn) - require.NoError(t, err) - - it, hit, err = index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - tblInfo = &model.TableInfo{ - ID: 2, - Indices: []*model.IndexInfo{ - { - ID: 3, - Name: model.NewCIStr("test"), - Unique: true, - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index = tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test uniq index. - txn, err = store.Begin() - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2), nil) - require.NotNil(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) - require.NotNil(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - _, err = index.FetchValues(make([]types.Datum, 0), nil) - require.NotNil(t, err) - - txn, err = store.Begin() - require.NoError(t, err) - - // Test the function of Next when the value of unique key is nil. - values2 := types.MakeDatums(nil, nil) - _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2), nil) - require.NoError(t, err) - it, err = index.SeekFirst(txn) - require.NoError(t, err) - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, nil, getValues[0].GetInterface()) - require.Equal(t, nil, getValues[1].GetInterface()) - require.Equal(t, int64(2), h.IntValue()) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) -} - -func TestCombineIndexSeek(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 1}, - {Offset: 2}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {Offset: 0}, - {Offset: 1}, - {Offset: 2}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - mockCtx := mock.NewContext() - values := types.MakeDatums("abc", "def") - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil)) - require.NoError(t, err) - defer iter.Close() - require.False(t, hit) - _, h, err := iter.Next() - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) -} - -func TestMultiColumnCommonHandle(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tblInfo := buildTableInfo(t, "create table t (a int, b int, u varchar(64) unique, nu varchar(64), primary key (a, b), index nu (nu))") - var idxUnique, idxNonUnique table.Index - for _, idxInfo := range tblInfo.Indices { - idx := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - if idxInfo.Name.L == "u" { - idxUnique = idx - } else if idxInfo.Name.L == "nu" { - idxNonUnique = idx - } - } - var a, b *model.ColumnInfo - for _, col := range tblInfo.Columns { - if col.Name.String() == "a" { - a = col - } else if col.Name.String() == "b" { - b = col - } - } - require.NotNil(t, a) - require.NotNil(t, b) - - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - mockCtx := mock.NewContext() - sc := mockCtx.GetSessionVars().StmtCtx - // create index for "insert t values (3, 2, "abc", "abc") - idxColVals := types.MakeDatums("abc") - handleColVals := types.MakeDatums(3, 2) - encodedHandle, err := codec.EncodeKey(sc, nil, handleColVals...) - require.NoError(t, err) - commonHandle, err := kv.NewCommonHandle(encodedHandle) - require.NoError(t, err) - _ = idxNonUnique - for _, idx := range []table.Index{idxUnique, idxNonUnique} { - key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) - require.NoError(t, err) - _, err = idx.Create(mockCtx, txn, idxColVals, commonHandle, nil) - require.NoError(t, err) - val, err := txn.Get(context.Background(), key) - require.NoError(t, err) - colInfo := tables.BuildRowcodecColInfoForIndexColumns(idx.Meta(), tblInfo) - colInfo = append(colInfo, rowcodec.ColInfo{ - ID: a.ID, - IsPKHandle: false, - Ft: rowcodec.FieldTypeFromModelColumn(a), - }) - colInfo = append(colInfo, rowcodec.ColInfo{ - ID: b.ID, - IsPKHandle: false, - Ft: rowcodec.FieldTypeFromModelColumn(b), - }) - colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, colInfo) - require.NoError(t, err) - require.Len(t, colVals, 3) - _, d, err := codec.DecodeOne(colVals[0]) - require.NoError(t, err) - require.Equal(t, "abc", d.GetString()) - _, d, err = codec.DecodeOne(colVals[1]) - require.NoError(t, err) - require.Equal(t, int64(3), d.GetInt64()) - _, d, err = codec.DecodeOne(colVals[2]) - require.NoError(t, err) - require.Equal(t, int64(2), d.GetInt64()) - handle, err := tablecodec.DecodeIndexHandle(key, val, 1) - require.NoError(t, err) - require.False(t, handle.IsInt()) - require.Equal(t, commonHandle.Encoded(), handle.Encoded()) - } -} diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 5678ce8b39b18..3e70c85365bd8 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2021 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,23 +16,316 @@ package tables_test import ( "context" + "io" "testing" + "time" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/rowcodec" "github.com/stretchr/testify/require" ) +func TestIndex(t *testing.T) { + tblInfo := &model.TableInfo{ + ID: 1, + Indices: []*model.IndexInfo{ + { + ID: 2, + Name: model.NewCIStr("test"), + Columns: []*model.IndexColumn{ + {Offset: 0}, + {Offset: 1}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, + }, + } + index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + // Test ununiq index. + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + + values := types.MakeDatums(1, 2) + mockCtx := mock.NewContext() + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + it, err := index.SeekFirst(txn) + require.NoError(t, err) + + getValues, h, err := it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, int64(1), getValues[0].GetInt64()) + require.Equal(t, int64(2), getValues[1].GetInt64()) + require.Equal(t, int64(1), h.IntValue()) + it.Close() + sc := &stmtctx.StatementContext{TimeZone: time.Local} + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) + require.NoError(t, err) + require.False(t, exist) + + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + require.True(t, exist) + + err = index.Delete(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0), nil) + require.NoError(t, err) + + _, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, hit, err := index.Seek(sc, txn, values) + require.NoError(t, err) + require.False(t, hit) + + err = index.Drop(txn) + require.NoError(t, err) + + it, hit, err = index.Seek(sc, txn, values) + require.NoError(t, err) + require.False(t, hit) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + err = txn.Commit(context.Background()) + require.NoError(t, err) + + tblInfo = &model.TableInfo{ + ID: 2, + Indices: []*model.IndexInfo{ + { + ID: 3, + Name: model.NewCIStr("test"), + Unique: true, + Columns: []*model.IndexColumn{ + {Offset: 0}, + {Offset: 1}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, + }, + } + index = tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + // Test uniq index. + txn, err = store.Begin() + require.NoError(t, err) + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2), nil) + require.NotNil(t, err) + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + getValues, h, err = it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, int64(1), getValues[0].GetInt64()) + require.Equal(t, int64(2), getValues[1].GetInt64()) + require.Equal(t, int64(1), h.IntValue()) + it.Close() + + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + require.Equal(t, int64(1), h.IntValue()) + require.True(t, exist) + + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) + require.NotNil(t, err) + require.Equal(t, int64(1), h.IntValue()) + require.True(t, exist) + + err = txn.Commit(context.Background()) + require.NoError(t, err) + + _, err = index.FetchValues(make([]types.Datum, 0), nil) + require.NotNil(t, err) + + txn, err = store.Begin() + require.NoError(t, err) + + // Test the function of Next when the value of unique key is nil. + values2 := types.MakeDatums(nil, nil) + _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2), nil) + require.NoError(t, err) + it, err = index.SeekFirst(txn) + require.NoError(t, err) + getValues, h, err = it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, nil, getValues[0].GetInterface()) + require.Equal(t, nil, getValues[1].GetInterface()) + require.Equal(t, int64(2), h.IntValue()) + it.Close() + + err = txn.Commit(context.Background()) + require.NoError(t, err) +} + +func TestCombineIndexSeek(t *testing.T) { + tblInfo := &model.TableInfo{ + ID: 1, + Indices: []*model.IndexInfo{ + { + ID: 2, + Name: model.NewCIStr("test"), + Columns: []*model.IndexColumn{ + {Offset: 1}, + {Offset: 2}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {Offset: 0}, + {Offset: 1}, + {Offset: 2}, + }, + } + index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + + mockCtx := mock.NewContext() + values := types.MakeDatums("abc", "def") + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + sc := &stmtctx.StatementContext{TimeZone: time.Local} + iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil)) + require.NoError(t, err) + defer iter.Close() + require.False(t, hit) + _, h, err := iter.Next() + require.NoError(t, err) + require.Equal(t, int64(1), h.IntValue()) +} + +func TestMultiColumnCommonHandle(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tblInfo := buildTableInfo(t, "create table t (a int, b int, u varchar(64) unique, nu varchar(64), primary key (a, b), index nu (nu))") + var idxUnique, idxNonUnique table.Index + for _, idxInfo := range tblInfo.Indices { + idx := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + if idxInfo.Name.L == "u" { + idxUnique = idx + } else if idxInfo.Name.L == "nu" { + idxNonUnique = idx + } + } + var a, b *model.ColumnInfo + for _, col := range tblInfo.Columns { + if col.Name.String() == "a" { + a = col + } else if col.Name.String() == "b" { + b = col + } + } + require.NotNil(t, a) + require.NotNil(t, b) + + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + mockCtx := mock.NewContext() + sc := mockCtx.GetSessionVars().StmtCtx + // create index for "insert t values (3, 2, "abc", "abc") + idxColVals := types.MakeDatums("abc") + handleColVals := types.MakeDatums(3, 2) + encodedHandle, err := codec.EncodeKey(sc, nil, handleColVals...) + require.NoError(t, err) + commonHandle, err := kv.NewCommonHandle(encodedHandle) + require.NoError(t, err) + _ = idxNonUnique + for _, idx := range []table.Index{idxUnique, idxNonUnique} { + key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) + require.NoError(t, err) + _, err = idx.Create(mockCtx, txn, idxColVals, commonHandle, nil) + require.NoError(t, err) + val, err := txn.Get(context.Background(), key) + require.NoError(t, err) + colInfo := tables.BuildRowcodecColInfoForIndexColumns(idx.Meta(), tblInfo) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: a.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(a), + }) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: b.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(b), + }) + colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, colInfo) + require.NoError(t, err) + require.Len(t, colVals, 3) + _, d, err := codec.DecodeOne(colVals[0]) + require.NoError(t, err) + require.Equal(t, "abc", d.GetString()) + _, d, err = codec.DecodeOne(colVals[1]) + require.NoError(t, err) + require.Equal(t, int64(3), d.GetInt64()) + _, d, err = codec.DecodeOne(colVals[2]) + require.NoError(t, err) + require.Equal(t, int64(2), d.GetInt64()) + handle, err := tablecodec.DecodeIndexHandle(key, val, 1) + require.NoError(t, err) + require.False(t, handle.IsInt()) + require.Equal(t, commonHandle.Encoded(), handle.Encoded()) + } +} + func TestSingleColumnCommonHandle(t *testing.T) { tblInfo := buildTableInfo(t, "create table t (a varchar(255) primary key, u int unique, nu int, index nu (nu))") var idxUnique, idxNonUnique table.Index diff --git a/telemetry/data_window_serial_test.go b/telemetry/data_window_test.go similarity index 100% rename from telemetry/data_window_serial_test.go rename to telemetry/data_window_test.go diff --git a/telemetry/telemetry_serial_test.go b/telemetry/telemetry_serial_test.go deleted file mode 100644 index c464d46c4c6de..0000000000000 --- a/telemetry/telemetry_serial_test.go +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2021 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 telemetry_test - -import ( - "runtime" - "testing" - - "github.com/Jeffail/gabs/v2" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/telemetry" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" -) - -func TestReport(t *testing.T) { - if runtime.GOOS == "windows" { - t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") - } - - etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) - defer etcdCluster.Terminate(t) - store, clean := testkit.CreateMockStore(t) - defer clean() - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - defer se.Close() - - config.GetGlobalConfig().EnableTelemetry = false - require.NoError(t, telemetry.ReportUsageData(se, etcdCluster.RandClient())) - - status, err := telemetry.GetTelemetryStatus(etcdCluster.RandClient()) - require.NoError(t, err) - - jsonParsed, err := gabs.ParseJSON([]byte(status)) - require.NoError(t, err) - require.True(t, jsonParsed.Path("is_error").Data().(bool)) - require.Equal(t, "telemetry is disabled", jsonParsed.Path("error_msg").Data().(string)) - require.False(t, jsonParsed.Path("is_request_sent").Data().(bool)) -} diff --git a/telemetry/telemetry_test.go b/telemetry/telemetry_test.go index c7f3cef8d92a8..56fc2da76852a 100644 --- a/telemetry/telemetry_test.go +++ b/telemetry/telemetry_test.go @@ -101,3 +101,29 @@ func TestPreview(t *testing.T) { require.NoError(t, err) require.Equal(t, "", r) } + +func TestReport(t *testing.T) { + if runtime.GOOS == "windows" { + t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") + } + + etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer etcdCluster.Terminate(t) + store, clean := testkit.CreateMockStore(t) + defer clean() + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + defer se.Close() + + config.GetGlobalConfig().EnableTelemetry = false + require.NoError(t, telemetry.ReportUsageData(se, etcdCluster.RandClient())) + + status, err := telemetry.GetTelemetryStatus(etcdCluster.RandClient()) + require.NoError(t, err) + + jsonParsed, err := gabs.ParseJSON([]byte(status)) + require.NoError(t, err) + require.True(t, jsonParsed.Path("is_error").Data().(bool)) + require.Equal(t, "telemetry is disabled", jsonParsed.Path("error_msg").Data().(string)) + require.False(t, jsonParsed.Path("is_request_sent").Data().(bool)) +} diff --git a/types/enum_serial_test.go b/types/enum_test.go similarity index 100% rename from types/enum_serial_test.go rename to types/enum_test.go diff --git a/types/mydecimal_serial_test.go b/types/mydecimal_serial_test.go deleted file mode 100644 index cd72ef7ce131a..0000000000000 --- a/types/mydecimal_serial_test.go +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright 2021 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 types - -import ( - "fmt" - "testing" - - "github.com/stretchr/testify/require" -) - -// this test will change global variable `wordBufLen`, so it must run in serial -func TestShiftMyDecimal(t *testing.T) { - type tcase struct { - input string - shift int - output string - err error - } - - var dotest = func(tests []tcase) { - for _, test := range tests { - t.Run(fmt.Sprintf("%v (shift: %v, wordBufLen: %v)", test.input, test.shift, wordBufLen), func(t *testing.T) { - var dec MyDecimal - require.NoError(t, dec.FromString([]byte(test.input))) - require.Equal(t, test.err, dec.Shift(test.shift)) - require.Equal(t, test.output, string(dec.ToString())) - }) - } - } - - wordBufLen = maxWordBufLen - tests := []tcase{ - {"123.123", 1, "1231.23", nil}, - {"123457189.123123456789000", 1, "1234571891.23123456789", nil}, - {"123457189.123123456789000", 8, "12345718912312345.6789", nil}, - {"123457189.123123456789000", 9, "123457189123123456.789", nil}, - {"123457189.123123456789000", 10, "1234571891231234567.89", nil}, - {"123457189.123123456789000", 17, "12345718912312345678900000", nil}, - {"123457189.123123456789000", 18, "123457189123123456789000000", nil}, - {"123457189.123123456789000", 19, "1234571891231234567890000000", nil}, - {"123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, - {"123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, - {"123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, - {"000000000000000000000000123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, - {"00000000123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, - {"00000000000000000123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, - {"123", 1, "1230", nil}, - {"123", 10, "1230000000000", nil}, - {".123", 1, "1.23", nil}, - {".123", 10, "1230000000", nil}, - {".123", 14, "12300000000000", nil}, - {"000.000", 1000, "0", nil}, - {"000.", 1000, "0", nil}, - {".000", 1000, "0", nil}, - {"1", 1000, "1", ErrOverflow}, - {"123.123", -1, "12.3123", nil}, - {"123987654321.123456789000", -1, "12398765432.1123456789", nil}, - {"123987654321.123456789000", -2, "1239876543.21123456789", nil}, - {"123987654321.123456789000", -3, "123987654.321123456789", nil}, - {"123987654321.123456789000", -8, "1239.87654321123456789", nil}, - {"123987654321.123456789000", -9, "123.987654321123456789", nil}, - {"123987654321.123456789000", -10, "12.3987654321123456789", nil}, - {"123987654321.123456789000", -11, "1.23987654321123456789", nil}, - {"123987654321.123456789000", -12, "0.123987654321123456789", nil}, - {"123987654321.123456789000", -13, "0.0123987654321123456789", nil}, - {"123987654321.123456789000", -14, "0.00123987654321123456789", nil}, - {"00000087654321.123456789000", -14, "0.00000087654321123456789", nil}, - } - dotest(tests) - - wordBufLen = 2 - tests = []tcase{ - {"123.123", -2, "1.23123", nil}, - {"123.123", -3, "0.123123", nil}, - {"123.123", -6, "0.000123123", nil}, - {"123.123", -7, "0.0000123123", nil}, - {"123.123", -15, "0.000000000000123123", nil}, - {"123.123", -16, "0.000000000000012312", ErrTruncated}, - {"123.123", -17, "0.000000000000001231", ErrTruncated}, - {"123.123", -18, "0.000000000000000123", ErrTruncated}, - {"123.123", -19, "0.000000000000000012", ErrTruncated}, - {"123.123", -20, "0.000000000000000001", ErrTruncated}, - {"123.123", -21, "0", ErrTruncated}, - {".000000000123", -1, "0.0000000000123", nil}, - {".000000000123", -6, "0.000000000000000123", nil}, - {".000000000123", -7, "0.000000000000000012", ErrTruncated}, - {".000000000123", -8, "0.000000000000000001", ErrTruncated}, - {".000000000123", -9, "0", ErrTruncated}, - {".000000000123", 1, "0.00000000123", nil}, - {".000000000123", 8, "0.0123", nil}, - {".000000000123", 9, "0.123", nil}, - {".000000000123", 10, "1.23", nil}, - {".000000000123", 17, "12300000", nil}, - {".000000000123", 18, "123000000", nil}, - {".000000000123", 19, "1230000000", nil}, - {".000000000123", 20, "12300000000", nil}, - {".000000000123", 21, "123000000000", nil}, - {".000000000123", 22, "1230000000000", nil}, - {".000000000123", 23, "12300000000000", nil}, - {".000000000123", 24, "123000000000000", nil}, - {".000000000123", 25, "1230000000000000", nil}, - {".000000000123", 26, "12300000000000000", nil}, - {".000000000123", 27, "123000000000000000", nil}, - {".000000000123", 28, "0.000000000123", ErrOverflow}, - {"123456789.987654321", -1, "12345678.998765432", ErrTruncated}, - {"123456789.987654321", -2, "1234567.899876543", ErrTruncated}, - {"123456789.987654321", -8, "1.234567900", ErrTruncated}, - {"123456789.987654321", -9, "0.123456789987654321", nil}, - {"123456789.987654321", -10, "0.012345678998765432", ErrTruncated}, - {"123456789.987654321", -17, "0.000000001234567900", ErrTruncated}, - {"123456789.987654321", -18, "0.000000000123456790", ErrTruncated}, - {"123456789.987654321", -19, "0.000000000012345679", ErrTruncated}, - {"123456789.987654321", -26, "0.000000000000000001", ErrTruncated}, - {"123456789.987654321", -27, "0", ErrTruncated}, - {"123456789.987654321", 1, "1234567900", ErrTruncated}, - {"123456789.987654321", 2, "12345678999", ErrTruncated}, - {"123456789.987654321", 4, "1234567899877", ErrTruncated}, - {"123456789.987654321", 8, "12345678998765432", ErrTruncated}, - {"123456789.987654321", 9, "123456789987654321", nil}, - {"123456789.987654321", 10, "123456789.987654321", ErrOverflow}, - {"123456789.987654321", 0, "123456789.987654321", nil}, - } - dotest(tests) - - // reset - wordBufLen = maxWordBufLen -} - -// this test will change global variable `wordBufLen`, so it must run in serial -func TestFromStringMyDecimal(t *testing.T) { - type tcase struct { - input string - output string - err error - } - - var dotest = func(tests []tcase) { - for _, test := range tests { - t.Run(fmt.Sprintf("%v (wordBufLen: %v)", test.input, wordBufLen), func(t *testing.T) { - var dec MyDecimal - require.Equal(t, test.err, dec.FromString([]byte(test.input))) - require.Equal(t, test.output, string(dec.ToString())) - }) - } - } - - wordBufLen = maxWordBufLen - tests := []tcase{ - {"12345", "12345", nil}, - {"12345.", "12345", nil}, - {"123.45.", "123.45", ErrTruncated}, - {"-123.45.", "-123.45", ErrTruncated}, - {".00012345000098765", "0.00012345000098765", nil}, - {".12345000098765", "0.12345000098765", nil}, - {"-.000000012345000098765", "-0.000000012345000098765", nil}, - {"1234500009876.5", "1234500009876.5", nil}, - {"123E5", "12300000", nil}, - {"123E-2", "1.23", nil}, - {"1e1073741823", "999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, - {"-1e1073741823", "-999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, - {"1e18446744073709551620", "0", ErrBadNumber}, - {"1e", "1", ErrTruncated}, - {"1e001", "10", nil}, - {"1e00", "1", nil}, - {"1eabc", "1", ErrTruncated}, - {"1e 1dddd ", "10", ErrTruncated}, - {"1e - 1", "1", ErrTruncated}, - {"1e -1", "0.1", nil}, - {"0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", "0.000000000000000000000000000000000000000000000000000000000000000000000000", ErrTruncated}, - {"1asf", "1", ErrTruncated}, - {"1.1.1.1.1", "1.1", ErrTruncated}, - {"1 1", "1", ErrTruncated}, - {"1 ", "1", nil}, - } - dotest(tests) - - wordBufLen = 1 - tests = []tcase{ - {"123450000098765", "98765", ErrOverflow}, - {"123450.000098765", "123450", ErrTruncated}, - } - dotest(tests) - - // reset - wordBufLen = maxWordBufLen -} diff --git a/types/mydecimal_test.go b/types/mydecimal_test.go index 61e86ffb1dc7a..13da9284ad395 100644 --- a/types/mydecimal_test.go +++ b/types/mydecimal_test.go @@ -15,6 +15,7 @@ package types import ( + "fmt" "strconv" "strings" "testing" @@ -814,3 +815,180 @@ func TestReset(t *testing.T) { require.NoError(t, DecimalAdd(&x2, &y2, &z1)) require.Equal(t, z2, z1) } + +// this test will change global variable `wordBufLen`, so it must run in serial +func TestShiftMyDecimal(t *testing.T) { + type tcase struct { + input string + shift int + output string + err error + } + + var dotest = func(tests []tcase) { + for _, test := range tests { + t.Run(fmt.Sprintf("%v (shift: %v, wordBufLen: %v)", test.input, test.shift, wordBufLen), func(t *testing.T) { + var dec MyDecimal + require.NoError(t, dec.FromString([]byte(test.input))) + require.Equal(t, test.err, dec.Shift(test.shift)) + require.Equal(t, test.output, string(dec.ToString())) + }) + } + } + + wordBufLen = maxWordBufLen + tests := []tcase{ + {"123.123", 1, "1231.23", nil}, + {"123457189.123123456789000", 1, "1234571891.23123456789", nil}, + {"123457189.123123456789000", 8, "12345718912312345.6789", nil}, + {"123457189.123123456789000", 9, "123457189123123456.789", nil}, + {"123457189.123123456789000", 10, "1234571891231234567.89", nil}, + {"123457189.123123456789000", 17, "12345718912312345678900000", nil}, + {"123457189.123123456789000", 18, "123457189123123456789000000", nil}, + {"123457189.123123456789000", 19, "1234571891231234567890000000", nil}, + {"123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, + {"123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, + {"123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, + {"000000000000000000000000123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, + {"00000000123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, + {"00000000000000000123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, + {"123", 1, "1230", nil}, + {"123", 10, "1230000000000", nil}, + {".123", 1, "1.23", nil}, + {".123", 10, "1230000000", nil}, + {".123", 14, "12300000000000", nil}, + {"000.000", 1000, "0", nil}, + {"000.", 1000, "0", nil}, + {".000", 1000, "0", nil}, + {"1", 1000, "1", ErrOverflow}, + {"123.123", -1, "12.3123", nil}, + {"123987654321.123456789000", -1, "12398765432.1123456789", nil}, + {"123987654321.123456789000", -2, "1239876543.21123456789", nil}, + {"123987654321.123456789000", -3, "123987654.321123456789", nil}, + {"123987654321.123456789000", -8, "1239.87654321123456789", nil}, + {"123987654321.123456789000", -9, "123.987654321123456789", nil}, + {"123987654321.123456789000", -10, "12.3987654321123456789", nil}, + {"123987654321.123456789000", -11, "1.23987654321123456789", nil}, + {"123987654321.123456789000", -12, "0.123987654321123456789", nil}, + {"123987654321.123456789000", -13, "0.0123987654321123456789", nil}, + {"123987654321.123456789000", -14, "0.00123987654321123456789", nil}, + {"00000087654321.123456789000", -14, "0.00000087654321123456789", nil}, + } + dotest(tests) + + wordBufLen = 2 + tests = []tcase{ + {"123.123", -2, "1.23123", nil}, + {"123.123", -3, "0.123123", nil}, + {"123.123", -6, "0.000123123", nil}, + {"123.123", -7, "0.0000123123", nil}, + {"123.123", -15, "0.000000000000123123", nil}, + {"123.123", -16, "0.000000000000012312", ErrTruncated}, + {"123.123", -17, "0.000000000000001231", ErrTruncated}, + {"123.123", -18, "0.000000000000000123", ErrTruncated}, + {"123.123", -19, "0.000000000000000012", ErrTruncated}, + {"123.123", -20, "0.000000000000000001", ErrTruncated}, + {"123.123", -21, "0", ErrTruncated}, + {".000000000123", -1, "0.0000000000123", nil}, + {".000000000123", -6, "0.000000000000000123", nil}, + {".000000000123", -7, "0.000000000000000012", ErrTruncated}, + {".000000000123", -8, "0.000000000000000001", ErrTruncated}, + {".000000000123", -9, "0", ErrTruncated}, + {".000000000123", 1, "0.00000000123", nil}, + {".000000000123", 8, "0.0123", nil}, + {".000000000123", 9, "0.123", nil}, + {".000000000123", 10, "1.23", nil}, + {".000000000123", 17, "12300000", nil}, + {".000000000123", 18, "123000000", nil}, + {".000000000123", 19, "1230000000", nil}, + {".000000000123", 20, "12300000000", nil}, + {".000000000123", 21, "123000000000", nil}, + {".000000000123", 22, "1230000000000", nil}, + {".000000000123", 23, "12300000000000", nil}, + {".000000000123", 24, "123000000000000", nil}, + {".000000000123", 25, "1230000000000000", nil}, + {".000000000123", 26, "12300000000000000", nil}, + {".000000000123", 27, "123000000000000000", nil}, + {".000000000123", 28, "0.000000000123", ErrOverflow}, + {"123456789.987654321", -1, "12345678.998765432", ErrTruncated}, + {"123456789.987654321", -2, "1234567.899876543", ErrTruncated}, + {"123456789.987654321", -8, "1.234567900", ErrTruncated}, + {"123456789.987654321", -9, "0.123456789987654321", nil}, + {"123456789.987654321", -10, "0.012345678998765432", ErrTruncated}, + {"123456789.987654321", -17, "0.000000001234567900", ErrTruncated}, + {"123456789.987654321", -18, "0.000000000123456790", ErrTruncated}, + {"123456789.987654321", -19, "0.000000000012345679", ErrTruncated}, + {"123456789.987654321", -26, "0.000000000000000001", ErrTruncated}, + {"123456789.987654321", -27, "0", ErrTruncated}, + {"123456789.987654321", 1, "1234567900", ErrTruncated}, + {"123456789.987654321", 2, "12345678999", ErrTruncated}, + {"123456789.987654321", 4, "1234567899877", ErrTruncated}, + {"123456789.987654321", 8, "12345678998765432", ErrTruncated}, + {"123456789.987654321", 9, "123456789987654321", nil}, + {"123456789.987654321", 10, "123456789.987654321", ErrOverflow}, + {"123456789.987654321", 0, "123456789.987654321", nil}, + } + dotest(tests) + + // reset + wordBufLen = maxWordBufLen +} + +// this test will change global variable `wordBufLen`, so it must run in serial +func TestFromStringMyDecimal(t *testing.T) { + type tcase struct { + input string + output string + err error + } + + var dotest = func(tests []tcase) { + for _, test := range tests { + t.Run(fmt.Sprintf("%v (wordBufLen: %v)", test.input, wordBufLen), func(t *testing.T) { + var dec MyDecimal + require.Equal(t, test.err, dec.FromString([]byte(test.input))) + require.Equal(t, test.output, string(dec.ToString())) + }) + } + } + + wordBufLen = maxWordBufLen + tests := []tcase{ + {"12345", "12345", nil}, + {"12345.", "12345", nil}, + {"123.45.", "123.45", ErrTruncated}, + {"-123.45.", "-123.45", ErrTruncated}, + {".00012345000098765", "0.00012345000098765", nil}, + {".12345000098765", "0.12345000098765", nil}, + {"-.000000012345000098765", "-0.000000012345000098765", nil}, + {"1234500009876.5", "1234500009876.5", nil}, + {"123E5", "12300000", nil}, + {"123E-2", "1.23", nil}, + {"1e1073741823", "999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, + {"-1e1073741823", "-999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, + {"1e18446744073709551620", "0", ErrBadNumber}, + {"1e", "1", ErrTruncated}, + {"1e001", "10", nil}, + {"1e00", "1", nil}, + {"1eabc", "1", ErrTruncated}, + {"1e 1dddd ", "10", ErrTruncated}, + {"1e - 1", "1", ErrTruncated}, + {"1e -1", "0.1", nil}, + {"0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", "0.000000000000000000000000000000000000000000000000000000000000000000000000", ErrTruncated}, + {"1asf", "1", ErrTruncated}, + {"1.1.1.1.1", "1.1", ErrTruncated}, + {"1 1", "1", ErrTruncated}, + {"1 ", "1", nil}, + } + dotest(tests) + + wordBufLen = 1 + tests = []tcase{ + {"123450000098765", "98765", ErrOverflow}, + {"123450.000098765", "123450", ErrTruncated}, + } + dotest(tests) + + // reset + wordBufLen = maxWordBufLen +} diff --git a/types/set_serial_test.go b/types/set_test.go similarity index 100% rename from types/set_serial_test.go rename to types/set_test.go