diff --git a/bindinfo/bind_serial_test.go b/bindinfo/bind_serial_test.go index 49ca023a1213b..2c242edb197ab 100644 --- a/bindinfo/bind_serial_test.go +++ b/bindinfo/bind_serial_test.go @@ -826,11 +826,11 @@ func TestForbidEvolvePlanBaseLinesBeforeGA(t *testing.T) { err := tk.ExecToErr("set @@tidb_evolve_plan_baselines=0") require.Equal(t, nil, err) err = tk.ExecToErr("set @@TiDB_Evolve_pLan_baselines=1") - require.Regexp(t, "Cannot enable baseline evolution feature, it is not generally available now", err) + require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") err = tk.ExecToErr("set @@TiDB_Evolve_pLan_baselines=oN") - require.Regexp(t, "Cannot enable baseline evolution feature, it is not generally available now", err) + require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") err = tk.ExecToErr("admin evolve bindings") - require.Regexp(t, "Cannot enable baseline evolution feature, it is not generally available now", err) + require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") } func TestExplainTableStmts(t *testing.T) { @@ -861,7 +861,8 @@ func TestSPMWithoutUseDatabase(t *testing.T) { tk.MustExec("create global binding for select * from t using select * from t force index(a)") err := tk1.ExecToErr("select * from t") - require.Regexp(t, ".*No database selected", err) + require.Error(t, err) + require.Regexp(t, "No database selected$", err) tk1.MustQuery(`select @@last_plan_from_binding;`).Check(testkit.Rows("0")) require.True(t, tk1.MustUseIndex("select * from test.t", "a")) tk1.MustExec("select * from test.t") diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 79edc3403be68..b675b062793c6 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -180,14 +180,9 @@ func (bc *Client) SetStorage(ctx context.Context, backend *backuppb.StorageBacke "there may be some backup files in the path already, "+ "please specify a correct backup directory!", bc.storage.URI()+"/"+metautil.MetaFile) } - exist, err = bc.storage.FileExists(ctx, metautil.LockFile) + err = CheckBackupStorageIsLocked(ctx, bc.storage) if err != nil { - return errors.Annotatef(err, "error occurred when checking %s file", metautil.LockFile) - } - if exist { - return errors.Annotatef(berrors.ErrInvalidArgument, "backup lock file exists in %v, "+ - "there may be some backup files in the path already, "+ - "please specify a correct backup directory!", bc.storage.URI()+"/"+metautil.LockFile) + return err } bc.backend = backend return nil @@ -198,6 +193,29 @@ func (bc *Client) GetClusterID() uint64 { return bc.clusterID } +// CheckBackupStorageIsLocked checks whether backups is locked. +// which means we found other backup progress already write +// some data files into the same backup directory or cloud prefix. +func CheckBackupStorageIsLocked(ctx context.Context, s storage.ExternalStorage) error { + exist, err := s.FileExists(ctx, metautil.LockFile) + if err != nil { + return errors.Annotatef(err, "error occurred when checking %s file", metautil.LockFile) + } + if exist { + err = s.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + // should return error to break the walkDir when found lock file and other .sst files. + if strings.HasSuffix(path, ".sst") { + return errors.Annotatef(berrors.ErrInvalidArgument, "backup lock file and sst file exist in %v, "+ + "there are some backup files in the path already, "+ + "please specify a correct backup directory!", s.URI()+"/"+metautil.LockFile) + } + return nil + }) + return err + } + return nil +} + // BuildTableRanges returns the key ranges encompassing the entire table, // and its partitions if exists. func BuildTableRanges(tbl *model.TableInfo) ([]kv.KeyRange, error) { @@ -719,7 +737,7 @@ func OnBackupResponse( if lockErr := v.KvError.Locked; lockErr != nil { // Try to resolve lock. log.Warn("backup occur kv error", zap.Reflect("error", v)) - msBeforeExpired, _, err1 := lockResolver.ResolveLocks( + msBeforeExpired, err1 := lockResolver.ResolveLocks( bo, backupTS, []*txnlock.Lock{txnlock.NewLock(lockErr)}) if err1 != nil { return nil, 0, errors.Trace(err1) diff --git a/br/pkg/backup/client_test.go b/br/pkg/backup/client_test.go index 3c3688f79bc9f..e46d832bae3ee 100644 --- a/br/pkg/backup/client_test.go +++ b/br/pkg/backup/client_test.go @@ -70,6 +70,13 @@ func (r *testBackup) SetUpSuite(c *C) { } +func (r *testBackup) resetStorage(c *C) { + var err error + base := c.MkDir() + r.storage, err = storage.NewLocalStorage(base) + c.Assert(err, IsNil) +} + func (r *testBackup) TestGetTS(c *C) { var ( err error @@ -335,3 +342,30 @@ func (r *testBackup) TestskipUnsupportedDDLJob(c *C) { c.Assert(err, IsNil) c.Assert(len(allDDLJobs), Equals, 8) } + +func (r *testBackup) TestCheckBackupIsLocked(c *C) { + ctx := context.Background() + + r.resetStorage(c) + // check passed with an empty storage + err := backup.CheckBackupStorageIsLocked(ctx, r.storage) + c.Assert(err, IsNil) + + // check passed with only a lock file + err = r.storage.WriteFile(ctx, metautil.LockFile, nil) + c.Assert(err, IsNil) + err = backup.CheckBackupStorageIsLocked(ctx, r.storage) + c.Assert(err, IsNil) + + // check passed with a lock file and other non-sst files. + err = r.storage.WriteFile(ctx, "1.txt", nil) + c.Assert(err, IsNil) + err = backup.CheckBackupStorageIsLocked(ctx, r.storage) + c.Assert(err, IsNil) + + // check failed + err = r.storage.WriteFile(ctx, "1.sst", nil) + c.Assert(err, IsNil) + err = backup.CheckBackupStorageIsLocked(ctx, r.storage) + c.Assert(err, ErrorMatches, "backup lock file and sst file exist in(.+)") +} diff --git a/br/pkg/conn/conn_test.go b/br/pkg/conn/conn_test.go index 2f77803fc3f78..4de0e96c2f940 100644 --- a/br/pkg/conn/conn_test.go +++ b/br/pkg/conn/conn_test.go @@ -201,7 +201,7 @@ func TestGetAllTiKVStores(t *testing.T) { {Id: 2, Labels: []*metapb.StoreLabel{{Key: "engine", Value: "tiflash"}}}, }, storeBehavior: ErrorOnTiFlash, - expectedError: "cannot restore to a cluster with active TiFlash stores.*", + expectedError: "^cannot restore to a cluster with active TiFlash stores", }, { stores: []*metapb.Store{ @@ -225,7 +225,7 @@ func TestGetAllTiKVStores(t *testing.T) { {Id: 6, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tiflash"}, {Key: "engine", Value: "tikv"}}}, }, storeBehavior: ErrorOnTiFlash, - expectedError: "cannot restore to a cluster with active TiFlash stores.*", + expectedError: "^cannot restore to a cluster with active TiFlash stores", }, { stores: []*metapb.Store{ @@ -267,9 +267,9 @@ func TestGetConnOnCanceledContext(t *testing.T) { _, err := mgr.GetBackupClient(ctx, 42) require.Error(t, err) - require.Regexp(t, ".*context canceled.*", err.Error()) + require.Contains(t, err.Error(), "context canceled") _, err = mgr.ResetBackupClient(ctx, 42) require.Error(t, err) - require.Regexp(t, ".*context canceled.*", err.Error()) + require.Contains(t, err.Error(), "context canceled") } diff --git a/br/pkg/gluetikv/glue_test.go b/br/pkg/gluetikv/glue_test.go index 220350f8c0dd7..ab08d5f99f673 100644 --- a/br/pkg/gluetikv/glue_test.go +++ b/br/pkg/gluetikv/glue_test.go @@ -22,5 +22,5 @@ import ( func TestGetVersion(t *testing.T) { g := Glue{} - require.Regexp(t, "BR(.|\n)*Release Version(.|\n)*Git Commit Hash(.|\n)*", g.GetVersion()) + require.Regexp(t, "^BR(.|\n)*Release Version(.|\n)*Git Commit Hash(.|\n)*$", g.GetVersion()) } diff --git a/br/pkg/lightning/backend/backend_test.go b/br/pkg/lightning/backend/backend_test.go index 80ee77dff9b7c..0b347994192c4 100644 --- a/br/pkg/lightning/backend/backend_test.go +++ b/br/pkg/lightning/backend/backend_test.go @@ -215,7 +215,7 @@ func TestWriteEngineFailed(t *testing.T) { require.NoError(t, err) err = writer.WriteRows(ctx, nil, rows) require.Error(t, err) - require.Regexp(t, "fake unrecoverable write error.*", err.Error()) + require.Regexp(t, "^fake unrecoverable write error", err.Error()) _, err = writer.Close(ctx) require.NoError(t, err) } @@ -243,7 +243,7 @@ func TestWriteBatchSendFailedWithRetry(t *testing.T) { require.NoError(t, err) err = writer.WriteRows(ctx, nil, rows) require.Error(t, err) - require.Regexp(t, ".*fake recoverable write batch error", err.Error()) + require.Regexp(t, "fake recoverable write batch error$", err.Error()) _, err = writer.Close(ctx) require.NoError(t, err) } @@ -264,7 +264,7 @@ func TestImportFailedNoRetry(t *testing.T) { require.NoError(t, err) err = closedEngine.Import(ctx, 1) require.Error(t, err) - require.Regexp(t, "fake unrecoverable import error.*", err.Error()) + require.Regexp(t, "^fake unrecoverable import error", err.Error()) } func TestImportFailedWithRetry(t *testing.T) { @@ -285,7 +285,7 @@ func TestImportFailedWithRetry(t *testing.T) { require.NoError(t, err) err = closedEngine.Import(ctx, 1) require.Error(t, err) - require.Regexp(t, ".*fake recoverable import error", err.Error()) + require.Regexp(t, "fake recoverable import error$", err.Error()) } func TestImportFailedRecovered(t *testing.T) { diff --git a/br/pkg/lightning/backend/tidb/tidb_test.go b/br/pkg/lightning/backend/tidb/tidb_test.go index 118344a0ef3bf..e6f14def964d2 100644 --- a/br/pkg/lightning/backend/tidb/tidb_test.go +++ b/br/pkg/lightning/backend/tidb/tidb_test.go @@ -245,7 +245,7 @@ func testStrictMode(t *testing.T) { types.NewStringDatum("\xff\xff\xff\xff"), }, 1, []int{0, -1, -1}, "5.csv", 0) require.Error(t, err) - require.Regexp(t, `.*incorrect utf8 value .* for column s0`, err.Error()) + require.Regexp(t, `incorrect utf8 value .* for column s0$`, err.Error()) // oepn a new encode because column count changed. encoder, err = bk.NewEncoder(tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) @@ -255,7 +255,7 @@ func testStrictMode(t *testing.T) { types.NewStringDatum("非 ASCII 字符串"), }, 1, []int{0, 1, -1}, "6.csv", 0) require.Error(t, err) - require.Regexp(t, ".*incorrect ascii value .* for column s1", err.Error()) + require.Regexp(t, "incorrect ascii value .* for column s1$", err.Error()) } func TestFetchRemoteTableModels_3_x(t *testing.T) { diff --git a/br/pkg/lightning/lightning_serial_test.go b/br/pkg/lightning/lightning_serial_test.go index 118c76df13364..bd268c3d3c39f 100644 --- a/br/pkg/lightning/lightning_serial_test.go +++ b/br/pkg/lightning/lightning_serial_test.go @@ -56,7 +56,7 @@ func TestRun(t *testing.T) { require.NoError(t, err) err = lightning.RunOnce(context.Background(), cfg, nil) require.Error(t, err) - require.Regexp(t, ".*mydumper dir does not exist", err.Error()) + require.Regexp(t, "mydumper dir does not exist$", err.Error()) path, _ := filepath.Abs(".") ctx := context.Background() diff --git a/br/pkg/lightning/lightning_server_serial_test.go b/br/pkg/lightning/lightning_server_serial_test.go index e8a9d86ebbdec..099299f7f7d4e 100644 --- a/br/pkg/lightning/lightning_server_serial_test.go +++ b/br/pkg/lightning/lightning_server_serial_test.go @@ -92,7 +92,7 @@ func TestRunServer(t *testing.T) { resp, err = http.DefaultClient.Do(req) require.NoError(t, err) require.Equal(t, http.StatusMethodNotAllowed, resp.StatusCode) - require.Regexp(t, ".*"+http.MethodPost+".*", resp.Header.Get("Allow")) + require.Contains(t, resp.Header.Get("Allow"), http.MethodPost) require.NoError(t, resp.Body.Close()) resp, err = http.Post(url, "application/toml", strings.NewReader("????")) @@ -101,7 +101,7 @@ func TestRunServer(t *testing.T) { err = json.NewDecoder(resp.Body).Decode(&data) require.NoError(t, err) require.Contains(t, data, "error") - require.Regexp(t, "cannot parse task.*", data["error"]) + require.Regexp(t, "^cannot parse task", data["error"]) require.NoError(t, resp.Body.Close()) resp, err = http.Post(url, "application/toml", strings.NewReader("[mydumper.csv]\nseparator = 'fooo'\ndelimiter= 'foo'")) @@ -110,7 +110,7 @@ func TestRunServer(t *testing.T) { err = json.NewDecoder(resp.Body).Decode(&data) require.NoError(t, err) require.Contains(t, data, "error") - require.Regexp(t, "invalid task configuration:.*", data["error"]) + require.Regexp(t, "^invalid task configuration:", data["error"]) require.NoError(t, resp.Body.Close()) for i := 0; i < 20; i++ { diff --git a/br/pkg/metautil/metafile_test.go b/br/pkg/metautil/metafile_test.go index 6517d0ea9174d..2553e862594f6 100644 --- a/br/pkg/metautil/metafile_test.go +++ b/br/pkg/metautil/metafile_test.go @@ -5,7 +5,6 @@ package metautil import ( "context" "crypto/sha256" - "regexp" "testing" "github.com/golang/mock/gomock" @@ -87,7 +86,8 @@ func TestWalkMetaFileInvalid(t *testing.T) { collect := func(m *backuppb.MetaFile) { panic("unreachable") } err := walkLeafMetaFile(ctx, mockStorage, root, &cipher, collect) - require.Regexp(t, regexp.MustCompile(".*ErrInvalidMetaFile.*"), err) + require.Error(t, err) + require.Contains(t, err.Error(), "ErrInvalidMetaFile") } func TestWalkMetaFile(t *testing.T) { diff --git a/br/pkg/pdutil/pd_serial_test.go b/br/pkg/pdutil/pd_serial_test.go index b72772dd7636f..c76e49cd70c61 100644 --- a/br/pkg/pdutil/pd_serial_test.go +++ b/br/pkg/pdutil/pd_serial_test.go @@ -55,7 +55,7 @@ func TestScheduler(t *testing.T) { } _, err = pdController.pauseSchedulersAndConfigWith(ctx, []string{}, cfg, mock) require.Error(t, err) - require.Regexp(t, "^failed to update PD.*", err.Error()) + require.Regexp(t, "^failed to update PD", err.Error()) go func() { <-schedulerPauseCh }() diff --git a/br/pkg/trace/tracing_serial_test.go b/br/pkg/trace/tracing_serial_test.go index 89adbdc7783c4..6ddaf3aac1a84 100644 --- a/br/pkg/trace/tracing_serial_test.go +++ b/br/pkg/trace/tracing_serial_test.go @@ -48,5 +48,5 @@ func TestSpan(t *testing.T) { // possible result: // "jobA 22:02:02.545296 20.621764ms\n" // " └─jobB 22:02:02.545297 10.293444ms\n" - require.Regexp(t, `jobA.*2[0-9]\.[0-9]+ms\n └─jobB.*1[0-9]\.[0-9]+ms\n`, s) + require.Regexp(t, `^jobA.*2[0-9]\.[0-9]+ms\n └─jobB.*1[0-9]\.[0-9]+ms\n$`, s) } diff --git a/br/pkg/utils/key_test.go b/br/pkg/utils/key_test.go index 6bb1c00f55bab..56b88a4720f21 100644 --- a/br/pkg/utils/key_test.go +++ b/br/pkg/utils/key_test.go @@ -89,7 +89,7 @@ func TestParseKey(t *testing.T) { for _, tt := range testNotSupportKey { _, err := ParseKey("notSupport", tt.any) require.Error(t, err) - require.Regexp(t, "unknown format.*", err.Error()) + require.Regexp(t, "^unknown format", err.Error()) } } diff --git a/br/pkg/utils/safe_point_test.go b/br/pkg/utils/safe_point_test.go index f80f7fff4c9dc..e77c17c3a23d9 100644 --- a/br/pkg/utils/safe_point_test.go +++ b/br/pkg/utils/safe_point_test.go @@ -32,7 +32,7 @@ func TestCheckGCSafepoint(t *testing.T) { { err := utils.CheckGCSafePoint(ctx, pdClient, 0) require.Error(t, err) - require.Regexp(t, ".*GC safepoint 2333 exceed TS 0.*", err.Error()) + require.Contains(t, err.Error(), "GC safepoint 2333 exceed TS 0") } } diff --git a/br/pkg/version/build/info_test.go b/br/pkg/version/build/info_test.go index 4714d21e78232..53c2e6fbf9bb4 100644 --- a/br/pkg/version/build/info_test.go +++ b/br/pkg/version/build/info_test.go @@ -14,11 +14,11 @@ func TestInfo(t *testing.T) { info := Info() lines := strings.Split(info, "\n") - require.Regexp(t, "Release Version.*", lines[0]) - require.Regexp(t, "Git Commit Hash.*", lines[1]) - require.Regexp(t, "Git Branch.*", lines[2]) - require.Regexp(t, "Go Version.*", lines[3]) - require.Regexp(t, "UTC Build Time.*", lines[4]) + require.Regexp(t, "^Release Version", lines[0]) + require.Regexp(t, "^Git Commit Hash", lines[1]) + require.Regexp(t, "^Git Branch", lines[2]) + require.Regexp(t, "^Go Version", lines[3]) + require.Regexp(t, "^UTC Build Time", lines[4]) } func TestLogInfo(t *testing.T) { diff --git a/br/pkg/version/version_test.go b/br/pkg/version/version_test.go index 128d5bd263874..d976b7cd7589f 100644 --- a/br/pkg/version/version_test.go +++ b/br/pkg/version/version_test.go @@ -52,7 +52,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, `incompatible.*version v4.0.0-rc.1, try update it to 4.0.0.*`, err.Error()) + require.Regexp(t, `^incompatible.*version v4.0.0-rc.1, try update it to 4.0.0`, err.Error()) } { @@ -62,7 +62,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, `incompatible.*version v3.1.0-beta.1, try update it to 3.1.0.*`, err.Error()) + require.Regexp(t, `^incompatible.*version v3.1.0-beta.1, try update it to 3.1.0`, err.Error()) } { @@ -72,7 +72,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, `incompatible.*version v3.0.15, try update it to 3.1.0.*`, err.Error()) + require.Regexp(t, `^incompatible.*version v3.0.15, try update it to 3.1.0`, err.Error()) } { @@ -92,7 +92,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, ".*TiKV .* don't support BR, please upgrade cluster .*", err.Error()) + require.Regexp(t, "TiKV .* don't support BR, please upgrade cluster ", err.Error()) } { @@ -103,7 +103,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, "TiKV .* mismatch, please .*", err.Error()) + require.Regexp(t, "^TiKV .* mismatch, please ", err.Error()) } { @@ -114,7 +114,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, "TiKV .* major version mismatch, please .*", err.Error()) + require.Regexp(t, "^TiKV .* major version mismatch, please ", err.Error()) } { @@ -125,7 +125,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.Error(t, err) - require.Regexp(t, "TiKV .* mismatch, please .*", err.Error()) + require.Regexp(t, "^TiKV .* mismatch, please ", err.Error()) } { @@ -246,11 +246,11 @@ func TestExtractTiDBVersion(t *testing.T) { _, err = ExtractTiDBVersion("") require.Error(t, err) - require.Regexp(t, "not a valid TiDB version.*", err.Error()) + require.Regexp(t, "^not a valid TiDB version", err.Error()) _, err = ExtractTiDBVersion("8.0.12") require.Error(t, err) - require.Regexp(t, "not a valid TiDB version.*", err.Error()) + require.Regexp(t, "^not a valid TiDB version", err.Error()) _, err = ExtractTiDBVersion("not-a-valid-version") require.Error(t, err) @@ -264,15 +264,15 @@ func TestCheckVersion(t *testing.T) { err = CheckVersion("TiNB", *semver.New("2.1.0"), *semver.New("2.3.5"), *semver.New("3.0.0")) require.Error(t, err) - require.Regexp(t, "TiNB version too old.*", err.Error()) + require.Regexp(t, "^TiNB version too old", err.Error()) err = CheckVersion("TiNB", *semver.New("3.1.0"), *semver.New("2.3.5"), *semver.New("3.0.0")) require.Error(t, err) - require.Regexp(t, "TiNB version too new.*", err.Error()) + require.Regexp(t, "^TiNB version too new", err.Error()) err = CheckVersion("TiNB", *semver.New("3.0.0-beta"), *semver.New("2.3.5"), *semver.New("3.0.0")) require.Error(t, err) - require.Regexp(t, "TiNB version too new.*", err.Error()) + require.Regexp(t, "^TiNB version too new", err.Error()) } func versionEqualCheck(source *semver.Version, target *semver.Version) (result bool) { @@ -398,6 +398,6 @@ Check Table Before Drop: false` _, err = FetchVersion(ctx, db) require.Error(t, err) - require.Regexp(t, ".*mock failure", err.Error()) + require.Regexp(t, "mock failure$", err.Error()) } diff --git a/cmd/ddltest/ddl_serial_test.go b/cmd/ddltest/ddl_serial_test.go index d5dcda20a7529..a92ef55447d53 100644 --- a/cmd/ddltest/ddl_serial_test.go +++ b/cmd/ddltest/ddl_serial_test.go @@ -33,7 +33,6 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/log" - zaplog "github.com/pingcap/log" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -48,7 +47,6 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/logutil" "github.com/stretchr/testify/require" "go.uber.org/zap" goctx "golang.org/x/net/context" @@ -91,11 +89,9 @@ type ddlSuite struct { } func createDDLSuite(t *testing.T) (s *ddlSuite) { + var err error s = new(ddlSuite) - err := logutil.InitLogger(&logutil.LogConfig{Config: zaplog.Config{Level: *logLevel}}) - require.NoError(t, err) - s.quit = make(chan struct{}) s.store, err = store.New(fmt.Sprintf("tikv://%s%s", *etcd, *tikvPath)) diff --git a/cmd/ddltest/main_test.go b/cmd/ddltest/main_test.go new file mode 100644 index 0000000000000..60acefcf2eaff --- /dev/null +++ b/cmd/ddltest/main_test.go @@ -0,0 +1,36 @@ +// 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 ddltest + +import ( + "fmt" + "os" + "testing" + + zaplog "github.com/pingcap/log" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + err := logutil.InitLogger(&logutil.LogConfig{Config: zaplog.Config{Level: *logLevel}}) + if err != nil { + fmt.Fprint(os.Stderr, err.Error()) + os.Exit(1) + } + goleak.VerifyTestMain(m) +} diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 60e8f8b8f67cc..964fda974ca42 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1537,7 +1537,7 @@ func (s *testSerialDBSuite1) TestCreateSecondaryIndexInCluster(c *C) { tk.MustExec("use test") // test create table with non-unique key - tk.MustGetErrCode(` + tk.MustExec(` CREATE TABLE t ( c01 varchar(255) NOT NULL, c02 varchar(255) NOT NULL, @@ -1547,7 +1547,8 @@ CREATE TABLE t ( c06 varchar(255) DEFAULT NULL, PRIMARY KEY (c01,c02,c03) clustered, KEY c04 (c04) -)`, errno.ErrTooLongKey) +)`) + tk.MustExec("drop table t") // test create long clustered primary key. tk.MustGetErrCode(` @@ -1587,7 +1588,7 @@ CREATE TABLE t ( PRIMARY KEY (c01,c02) clustered )`) tk.MustExec("create index idx1 on t(c03)") - tk.MustGetErrCode("create index idx2 on t(c03, c04)", errno.ErrTooLongKey) + tk.MustExec("create index idx2 on t(c03, c04)") tk.MustExec("create unique index uk2 on t(c03, c04)") tk.MustExec("drop table t") @@ -1606,9 +1607,9 @@ CREATE TABLE t ( )`) tk.MustExec("alter table t change c03 c10 varchar(256) default null") tk.MustGetErrCode("alter table t change c10 c100 varchar(1024) default null", errno.ErrTooLongKey) - tk.MustGetErrCode("alter table t modify c10 varchar(600) default null", errno.ErrTooLongKey) + tk.MustExec("alter table t modify c10 varchar(600) default null") tk.MustExec("alter table t modify c06 varchar(600) default null") - tk.MustGetErrCode("alter table t modify c01 varchar(510)", errno.ErrTooLongKey) + tk.MustExec("alter table t modify c01 varchar(510)") tk.MustExec("create table t2 like t") } diff --git a/ddl/ddl_algorithm_test.go b/ddl/ddl_algorithm_serial_test.go similarity index 89% rename from ddl/ddl_algorithm_test.go rename to ddl/ddl_algorithm_serial_test.go index 444eddc41efe5..be9c87af3a163 100644 --- a/ddl/ddl_algorithm_test.go +++ b/ddl/ddl_algorithm_serial_test.go @@ -15,27 +15,25 @@ package ddl_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser/ast" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testDDLAlgorithmSuite{}) - var ( allAlgorithm = []ast.AlgorithmType{ast.AlgorithmTypeCopy, ast.AlgorithmTypeInplace, ast.AlgorithmTypeInstant} ) -type testDDLAlgorithmSuite struct{} - type testCase struct { alterSpec ast.AlterTableSpec supportedAlgorithm []ast.AlgorithmType expectedAlgorithm []ast.AlgorithmType } -func (s *testDDLAlgorithmSuite) TestFindAlterAlgorithm(c *C) { +func TestFindAlterAlgorithm(t *testing.T) { supportedInstantAlgorithms := []ast.AlgorithmType{ast.AlgorithmTypeDefault, ast.AlgorithmTypeCopy, ast.AlgorithmTypeInplace, ast.AlgorithmTypeInstant} expectedInstantAlgorithms := []ast.AlgorithmType{ast.AlgorithmTypeInstant, ast.AlgorithmTypeInstant, ast.AlgorithmTypeInstant, ast.AlgorithmTypeInstant} @@ -77,11 +75,11 @@ func (s *testDDLAlgorithmSuite) TestFindAlterAlgorithm(c *C) { } for _, tc := range testCases { - runAlterAlgorithmTestCases(c, &tc) + runAlterAlgorithmTestCases(t, &tc) } } -func runAlterAlgorithmTestCases(c *C, tc *testCase) { +func runAlterAlgorithmTestCases(t *testing.T, tc *testCase) { unsupported := make([]ast.AlgorithmType, 0, len(allAlgorithm)) Loop: for _, alm := range allAlgorithm { @@ -101,16 +99,16 @@ Loop: for i, alm := range tc.supportedAlgorithm { algorithm, err = ddl.ResolveAlterAlgorithm(&tc.alterSpec, alm) if err != nil { - c.Assert(ddl.ErrAlterOperationNotSupported.Equal(err), IsTrue) + require.True(t, ddl.ErrAlterOperationNotSupported.Equal(err)) } - c.Assert(algorithm, Equals, tc.expectedAlgorithm[i]) + require.Equal(t, tc.expectedAlgorithm[i], algorithm) } // Test unsupported. for _, alm := range unsupported { algorithm, err = ddl.ResolveAlterAlgorithm(&tc.alterSpec, alm) - c.Assert(algorithm, Equals, ast.AlgorithmTypeDefault) - c.Assert(err, NotNil, Commentf("Tp:%v, alm:%s", tc.alterSpec.Tp, alm)) - c.Assert(ddl.ErrAlterOperationNotSupported.Equal(err), IsTrue) + require.Equal(t, ast.AlgorithmTypeDefault, algorithm) + require.Error(t, err) + require.True(t, ddl.ErrAlterOperationNotSupported.Equal(err)) } } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index dab631c06e981..9e7c044ed2855 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1631,27 +1631,7 @@ func buildTableInfo( idxInfo.ID = allocateIndexID(tbInfo) tbInfo.Indices = append(tbInfo.Indices, idxInfo) } - if tbInfo.IsCommonHandle { - // Ensure tblInfo's each non-unique secondary-index's len + primary-key's len <= MaxIndexLength for clustered index table. - var pkLen, idxLen int - pkLen, err = indexColumnsLen(tbInfo.Columns, tables.FindPrimaryIndex(tbInfo).Columns) - if err != nil { - return - } - for _, idx := range tbInfo.Indices { - if idx.Unique { - // Only need check for non-unique secondary-index. - continue - } - idxLen, err = indexColumnsLen(tbInfo.Columns, idx.Columns) - if err != nil { - return - } - if pkLen+idxLen > config.GetGlobalConfig().MaxIndexLength { - return nil, errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength) - } - } - } + return } @@ -4272,9 +4252,6 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Contex // Index has a max-prefix-length constraint. eg: a varchar(100), index idx(a), modifying column a to a varchar(4000) // will cause index idx to break the max-prefix-length constraint. // -// For clustered index: -// Change column in pk need recheck all non-unique index, new pk len + index len < maxIndexLength. -// Change column in secondary only need related index, pk len + new index len < maxIndexLength. func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol *model.ColumnInfo) error { columns := make([]*model.ColumnInfo, 0, len(tbInfo.Columns)) columns = append(columns, tbInfo.Columns...) @@ -4289,40 +4266,31 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol } pkIndex := tables.FindPrimaryIndex(tbInfo) - var clusteredPkLen int - if tbInfo.IsCommonHandle { - var err error - clusteredPkLen, err = indexColumnsLen(columns, pkIndex.Columns) - if err != nil { - return err - } - } - checkOneIndex := func(indexInfo *model.IndexInfo, pkLenAppendToKey int, skipCheckIfNotModify bool) (modified bool, err error) { + checkOneIndex := func(indexInfo *model.IndexInfo) (err error) { + var modified bool for _, col := range indexInfo.Columns { if col.Name.L == originalCol.Name.L { modified = true break } } - if skipCheckIfNotModify && !modified { + if !modified { return } err = checkIndexInModifiableColumns(columns, indexInfo.Columns) if err != nil { return } - err = checkIndexPrefixLength(columns, indexInfo.Columns, pkLenAppendToKey) + err = checkIndexPrefixLength(columns, indexInfo.Columns) return } - // Check primary key first and get "does primary key's column has be modified?" info. - var ( - pkModified bool - err error - ) + // Check primary key first. + var err error + if pkIndex != nil { - pkModified, err = checkOneIndex(pkIndex, 0, true) + err = checkOneIndex(pkIndex) if err != nil { return err } @@ -4333,12 +4301,9 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol if indexInfo.Primary { continue } - var pkLenAppendToKey int - if !indexInfo.Unique { - pkLenAppendToKey = clusteredPkLen - } - - _, err = checkOneIndex(indexInfo, pkLenAppendToKey, !tbInfo.IsCommonHandle || !pkModified) + // the second param should always be set to true, check index length only if it was modified + // checkOneIndex needs one param only. + err = checkOneIndex(indexInfo) if err != nil { return err } @@ -5481,22 +5446,6 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde return errors.Trace(err) } - if !unique && tblInfo.IsCommonHandle { - // Ensure new created non-unique secondary-index's len + primary-key's len <= MaxIndexLength in clustered index table. - var pkLen, idxLen int - pkLen, err = indexColumnsLen(tblInfo.Columns, tables.FindPrimaryIndex(tblInfo).Columns) - if err != nil { - return err - } - idxLen, err = indexColumnsLen(finalColumns, indexColumns) - if err != nil { - return err - } - if pkLen+idxLen > config.GetGlobalConfig().MaxIndexLength { - return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength) - } - } - global := false if unique && tblInfo.GetPartitionInfo() != nil { ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo) diff --git a/ddl/failtest/fail_db_serial_test.go b/ddl/failtest/fail_db_serial_test.go index 9b5e70f12ca57..1c509ee121e68 100644 --- a/ddl/failtest/fail_db_serial_test.go +++ b/ddl/failtest/fail_db_serial_test.go @@ -568,6 +568,6 @@ func TestPartitionAddPanic(t *testing.T) { _, err := tk.Exec(`alter table t add partition (partition p1 values less than (20));`) require.Error(t, err) result := tk.MustQuery("show create table t").Rows()[0][1] - require.Regexp(t, `(?s).*PARTITION .p0. VALUES LESS THAN \(10\).*`, result) - require.NotRegexp(t, `(?s).*PARTITION .p0. VALUES LESS THAN \(20\).*`, result) + require.Regexp(t, `PARTITION .p0. VALUES LESS THAN \(10\)`, result) + require.NotRegexp(t, `PARTITION .p0. VALUES LESS THAN \(20\)`, result) } diff --git a/ddl/index.go b/ddl/index.go index c93d2602a2cc2..957f2807138cb 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -110,12 +110,12 @@ func checkPKOnGeneratedColumn(tblInfo *model.TableInfo, indexPartSpecifications return lastCol, nil } -func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn, pkLenAppendToKey int) error { +func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn) error { idxLen, err := indexColumnsLen(columns, idxColumns) if err != nil { return err } - if idxLen+pkLenAppendToKey > config.GetGlobalConfig().MaxIndexLength { + if idxLen > config.GetGlobalConfig().MaxIndexLength { return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength) } return nil diff --git a/ddl/options_test.go b/ddl/options_test.go index b4b796cc6726f..a7d5e138c474a 100644 --- a/ddl/options_test.go +++ b/ddl/options_test.go @@ -15,22 +15,21 @@ package ddl_test import ( + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3" ) -type ddlOptionsSuite struct{} +func TestOptions(t *testing.T) { + t.Parallel() -var _ = Suite(&ddlOptionsSuite{}) - -func (s *ddlOptionsSuite) TestOptions(c *C) { client, err := clientv3.NewFromURL("test") - c.Assert(err, IsNil) + require.NoError(t, err) callback := &ddl.BaseCallback{} lease := time.Second * 3 store := &mock.Store{} @@ -49,9 +48,9 @@ func (s *ddlOptionsSuite) TestOptions(c *C) { o(opt) } - c.Assert(opt.EtcdCli, Equals, client) - c.Assert(opt.Hook, Equals, callback) - c.Assert(opt.Lease, Equals, lease) - c.Assert(opt.Store, Equals, store) - c.Assert(opt.InfoCache, Equals, infoHandle) + require.Equal(t, client, opt.EtcdCli) + require.Equal(t, callback, opt.Hook) + require.Equal(t, lease, opt.Lease) + require.Equal(t, store, opt.Store) + require.Equal(t, infoHandle, opt.InfoCache) } diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 53044feb37c9d..44ff8d237f413 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -721,6 +721,8 @@ func encodeIndexKey(sc *stmtctx.StatementContext, ran *ranger.Range) ([]byte, [] } } + // NOTE: this is a hard-code operation to avoid wrong results when accessing unique index with NULL; + // Please see https://github.com/pingcap/tidb/issues/29650 for more details if hasNull { // Append 0 to make unique-key range [null, null] to be a scan rather than point-get. high = kv.Key(high).Next() diff --git a/domain/infosync/info.go b/domain/infosync/info.go index b159100f0f92e..29b71e8b9f276 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -343,8 +343,9 @@ func doRequest(ctx context.Context, addrs []string, route, method string, body i if body != nil { req.Header.Set("Content-Type", "application/json") } - + start := time.Now() res, err = doRequestWithFailpoint(req) + metrics.PDApiExecutionHistogram.WithLabelValues("placement").Observe(time.Since(start).Seconds()) if err == nil { bodyBytes, err := io.ReadAll(res.Body) if err != nil { diff --git a/dumpling/export/config_test.go b/dumpling/export/config_test.go index 89f0fde5b3d4b..fd6ed05885384 100644 --- a/dumpling/export/config_test.go +++ b/dumpling/export/config_test.go @@ -16,7 +16,7 @@ func TestCreateExternalStorage(t *testing.T) { mockConfig := defaultConfigForTest(t) loc, err := mockConfig.createExternalStorage(tcontext.Background()) require.NoError(t, err) - require.Regexp(t, "file:.*", loc.URI()) + require.Regexp(t, "^file:", loc.URI()) } func TestMatchMysqlBugVersion(t *testing.T) { diff --git a/dumpling/export/prepare_test.go b/dumpling/export/prepare_test.go index 231eac0aab266..4315024148518 100644 --- a/dumpling/export/prepare_test.go +++ b/dumpling/export/prepare_test.go @@ -287,7 +287,7 @@ func TestConfigValidation(t *testing.T) { conf.FileType = FileFormatSQLTextString err := adjustFileFormat(conf) require.Error(t, err) - require.Regexp(t, ".*please unset --filetype or set it to 'csv'.*", err.Error()) + require.Contains(t, err.Error(), "please unset --filetype or set it to 'csv'") conf.FileType = FileFormatCSVString require.NoError(t, adjustFileFormat(conf)) diff --git a/executor/executor.go b/executor/executor.go index 209262ac0d4db..eee07f8774ed0 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -983,7 +983,12 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc } if mutation := req.Mutations[0]; mutation != nil { label := resourcegrouptag.GetResourceGroupLabelByKey(mutation.Key) - return seVars.StmtCtx.GetResourceGroupTagByLabel(label) + normalized, digest := seVars.StmtCtx.SQLDigest() + if len(normalized) == 0 { + return nil + } + _, planDigest := seVars.StmtCtx.GetPlanDigest() + return resourcegrouptag.EncodeResourceGroupTag(digest, planDigest, label) } return nil } diff --git a/executor/insert_common.go b/executor/insert_common.go index e16253a7756bc..9c7adbb3c4d1a 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -682,6 +682,11 @@ func setDatumAutoIDAndCast(ctx sessionctx.Context, d *types.Datum, id int64, col d.SetAutoID(id, col.Flag) var err error *d, err = table.CastValue(ctx, *d, col.ToInfo(), false, false) + if err == nil && d.GetInt64() < id { + // Auto ID is out of range, the truncated ID is possible to duplicate with an existing ID. + // To prevent updating unrelated rows in the REPLACE statement, it is better to throw an error. + return autoid.ErrAutoincReadFailed + } return err } diff --git a/executor/insert_test.go b/executor/insert_test.go index 6a2245abb96e3..ab41475d3e6fa 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1820,3 +1820,18 @@ func (s *testAutoRandomSuite) TestInsertIssue29892(c *C) { c.Assert(err, NotNil) c.Assert(strings.Contains(err.Error(), "Duplicate entry"), Equals, true) } + +// https://github.com/pingcap/tidb/issues/29483. +func (s *testSuite13) TestReplaceAllocatingAutoID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("drop database if exists replace_auto_id;") + tk.MustExec("create database replace_auto_id;") + tk.MustExec(`use replace_auto_id;`) + + tk.MustExec("SET sql_mode='NO_ENGINE_SUBSTITUTION';") + tk.MustExec("DROP TABLE IF EXISTS t1;") + tk.MustExec("CREATE TABLE t1 (a tinyint not null auto_increment primary key, b char(20));") + tk.MustExec("INSERT INTO t1 VALUES (127,'maxvalue');") + // Note that this error is different from MySQL's duplicated primary key error. + tk.MustGetErrCode("REPLACE INTO t1 VALUES (0,'newmaxvalue');", errno.ErrAutoincReadFailed) +} diff --git a/executor/load_data.go b/executor/load_data.go index 03539a4f8c9a9..1202675ebbce0 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -419,9 +419,9 @@ func (e *LoadDataInfo) isInQuoter(bs []byte) bool { return inQuoter } -// indexOfTerminator return index of terminator, if not, return -1. +// IndexOfTerminator return index of terminator, if not, return -1. // normally, the field terminator and line terminator is short, so we just use brute force algorithm. -func (e *LoadDataInfo) indexOfTerminator(bs []byte, isInQuoter bool) int { +func (e *LoadDataInfo) IndexOfTerminator(bs []byte, inQuoter bool) int { fieldTerm := []byte(e.FieldsInfo.Terminated) fieldTermLen := len(fieldTerm) lineTerm := []byte(e.LinesInfo.Terminated) @@ -459,13 +459,10 @@ func (e *LoadDataInfo) indexOfTerminator(bs []byte, isInQuoter bool) int { } } atFieldStart := true - inQuoter := false loop: for i := 0; i < len(bs); i++ { if atFieldStart && bs[i] == e.FieldsInfo.Enclosed { - if !isInQuoter { - inQuoter = true - } + inQuoter = !inQuoter atFieldStart = false continue } @@ -525,7 +522,7 @@ func (e *LoadDataInfo) getLine(prevData, curData []byte, ignore bool) ([]byte, [ if ignore { endIdx = strings.Index(string(hack.String(curData[curStartIdx:])), e.LinesInfo.Terminated) } else { - endIdx = e.indexOfTerminator(curData[curStartIdx:], inquotor) + endIdx = e.IndexOfTerminator(curData[curStartIdx:], inquotor) } } if endIdx == -1 { @@ -539,7 +536,7 @@ func (e *LoadDataInfo) getLine(prevData, curData []byte, ignore bool) ([]byte, [ if ignore { endIdx = strings.Index(string(hack.String(curData[startingLen:])), e.LinesInfo.Terminated) } else { - endIdx = e.indexOfTerminator(curData[startingLen:], inquotor) + endIdx = e.IndexOfTerminator(curData[startingLen:], inquotor) } if endIdx != -1 { nextDataIdx := startingLen + endIdx + terminatedLen @@ -560,7 +557,7 @@ func (e *LoadDataInfo) getLine(prevData, curData []byte, ignore bool) ([]byte, [ if ignore { endIdx = strings.Index(string(hack.String(prevData[startingLen:])), e.LinesInfo.Terminated) } else { - endIdx = e.indexOfTerminator(prevData[startingLen:], inquotor) + endIdx = e.IndexOfTerminator(prevData[startingLen:], inquotor) } if endIdx >= prevLen { return prevData[startingLen : startingLen+endIdx], curData[nextDataIdx:], true diff --git a/executor/prepared_serial_test.go b/executor/prepared_serial_test.go index dde9ae973b7e9..8315e3dadc718 100644 --- a/executor/prepared_serial_test.go +++ b/executor/prepared_serial_test.go @@ -105,10 +105,10 @@ func TestPreparePlanCache4Blacklist(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "TopN") res = tk.MustQuery("explain format = 'brief' select min(a) from t") - require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "TopN") tk.MustExec("INSERT INTO mysql.opt_rule_blacklist VALUES('max_min_eliminate');") tk.MustExec("ADMIN reload opt_rule_blacklist;") @@ -121,10 +121,10 @@ func TestPreparePlanCache4Blacklist(t *testing.T) { tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) // Plans that have been cached will not be affected by the blacklist. - require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "TopN") res = tk.MustQuery("explain format = 'brief' select min(a) from t") - require.Regexp(t, ".*StreamAgg.*", res.Rows()[0][0]) + require.Contains(t, res.Rows()[0][0], "StreamAgg") // test the blacklist of Expression Pushdown tk.MustExec("drop table if exists t;") @@ -136,7 +136,7 @@ func TestPreparePlanCache4Blacklist(t *testing.T) { tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) require.Equal(t, 3, len(res.Rows())) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "Selection") require.Equal(t, "gt(test.t.a, 2), lt(test.t.a, 2)", res.Rows()[1][4]) res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") @@ -155,14 +155,14 @@ func TestPreparePlanCache4Blacklist(t *testing.T) { res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) // The expressions can still be pushed down to tikv. require.Equal(t, 3, len(res.Rows())) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "Selection") require.Equal(t, "gt(test.t.a, 2), lt(test.t.a, 2)", res.Rows()[1][4]) res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") require.Equal(t, 4, len(res.Rows())) - require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) + require.Contains(t, res.Rows()[0][0], "Selection") require.Equal(t, "lt(test.t.a, 2)", res.Rows()[0][4]) - require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) + require.Contains(t, res.Rows()[2][0], "Selection") require.Equal(t, "gt(test.t.a, 2)", res.Rows()[2][4]) tk.MustExec("DELETE FROM mysql.expr_pushdown_blacklist;") @@ -1020,7 +1020,7 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) require.Equal(t, 4, len(res.Rows())) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) + require.Contains(t, res.Rows()[0][0], "IndexMerge") tk.MustExec("set @@tidb_enable_index_merge = 0;") tk.MustExec("execute stmt;") @@ -1029,7 +1029,7 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) require.Equal(t, 4, len(res.Rows())) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) + require.Contains(t, res.Rows()[0][0], "IndexMerge") tk.MustExec("execute stmt;") tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) @@ -1046,8 +1046,8 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) - require.Regexp(t, ".*Concurrency.*", res.Rows()[1][5]) + require.Contains(t, res.Rows()[1][0], "Apply") + require.Contains(t, res.Rows()[1][5], "Concurrency") tk.MustExec("set tidb_enable_parallel_apply=false") tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) @@ -1055,7 +1055,7 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "Apply") executionInfo := fmt.Sprintf("%v", res.Rows()[1][4]) // Do not use the parallel apply. require.False(t, strings.Contains(executionInfo, "Concurrency")) @@ -1076,8 +1076,8 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) - require.Regexp(t, ".*cache:ON.*", res.Rows()[1][5]) + require.Contains(t, res.Rows()[1][0], "Apply") + require.Contains(t, res.Rows()[1][5], "cache:ON") tk.MustExec("set tidb_mem_quota_apply_cache=0") tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) @@ -1085,7 +1085,7 @@ func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + require.Contains(t, res.Rows()[1][0], "Apply") executionInfo = fmt.Sprintf("%v", res.Rows()[1][5]) // Do not use the apply cache. require.True(t, strings.Contains(executionInfo, "cache:OFF")) diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index de55cfa595aa4..16370b8a5ed2a 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -258,7 +258,7 @@ func TestShowStatusSnapshot(t *testing.T) { tk.MustQuery("show table status;").Check(testkit.Rows()) tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'") result := tk.MustQuery("show table status;") - require.Regexp(t, "t", result.Rows()[0][0]) + require.Equal(t, "t", result.Rows()[0][0]) } func TestShowStatsExtended(t *testing.T) { diff --git a/executor/write_test.go b/executor/write_test.go index 879c8c76c035f..d30c1f19ba8d2 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" @@ -2129,6 +2130,27 @@ func TestLoadDataEscape(t *testing.T) { checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } +func TestLoadDataWithLongContent(t *testing.T) { + e := &executor.LoadDataInfo{ + FieldsInfo: &ast.FieldsClause{Terminated: ",", Escaped: '\\', Enclosed: '"'}, + LinesInfo: &ast.LinesClause{Terminated: "\n"}, + } + tests := []struct { + content string + inQuoter bool + expectedIndex int + }{ + {"123,123\n123,123", false, 7}, + {"123123\\n123123", false, -1}, + {"123123\n123123", true, -1}, + {"123123\n123123\"\n", true, 14}, + } + + for _, tt := range tests { + require.Equal(t, tt.expectedIndex, e.IndexOfTerminator([]byte(tt.content), tt.inQuoter)) + } +} + // TestLoadDataSpecifiedColumns reuse TestLoadDataEscape's test case :-) func TestLoadDataSpecifiedColumns(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" diff --git a/expression/builtin_arithmetic_test.go b/expression/builtin_arithmetic_test.go index 1690b5235471d..1848e42c31ef9 100644 --- a/expression/builtin_arithmetic_test.go +++ b/expression/builtin_arithmetic_test.go @@ -285,11 +285,11 @@ func TestArithmeticMultiply(t *testing.T) { }, { args: []interface{}{int64(-1), int64(math.MinInt64)}, - expect: []interface{}{nil, ".*BIGINT value is out of range in '\\(-1 \\* -9223372036854775808\\)'"}, + expect: []interface{}{nil, "BIGINT value is out of range in '\\(-1 \\* -9223372036854775808\\)'$"}, }, { args: []interface{}{int64(math.MinInt64), int64(-1)}, - expect: []interface{}{nil, ".*BIGINT value is out of range in '\\(-9223372036854775808 \\* -1\\)'"}, + expect: []interface{}{nil, "BIGINT value is out of range in '\\(-9223372036854775808 \\* -1\\)'$"}, }, { args: []interface{}{uint64(11), uint64(11)}, @@ -322,6 +322,7 @@ func TestArithmeticMultiply(t *testing.T) { require.NoError(t, err) trequire.DatumEqual(t, types.NewDatum(tc.expect[0]), val) } else { + require.Error(t, err) require.Regexp(t, tc.expect[1], err.Error()) } } @@ -507,6 +508,7 @@ func TestArithmeticIntDivide(t *testing.T) { require.NoError(t, err) trequire.DatumEqual(t, types.NewDatum(tc.expect[0]), val) } else { + require.Error(t, err) require.Regexp(t, tc.expect[1], err.Error()) } } diff --git a/expression/builtin_info_test.go b/expression/builtin_info_test.go index 72b3cb0b694f2..b086d4c26ba0a 100644 --- a/expression/builtin_info_test.go +++ b/expression/builtin_info_test.go @@ -202,7 +202,8 @@ func TestCharset(t *testing.T) { fc := funcs[ast.Charset] f, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(nil))) require.Nil(t, f) - require.Regexp(t, ".*FUNCTION CHARSET does not exist", err.Error()) + require.Error(t, err) + require.Regexp(t, "FUNCTION CHARSET does not exist$", err.Error()) } func TestCoercibility(t *testing.T) { diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index ab50526faef6f..bdc6541c7a366 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -119,7 +119,8 @@ func TestJSONUnquote(t *testing.T) { require.Equal(t, tt.Result, d.GetString()) require.NoError(t, err) } else { - require.Regexp(t, ".*The document root must not be followed by other values.*", err.Error()) + require.Error(t, err) + require.Contains(t, err.Error(), "The document root must not be followed by other values") } } } diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index 58263af992365..7b2471a208e02 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -180,7 +180,7 @@ func TestValues(t *testing.T) { fc := &valuesFunctionClass{baseFunctionClass{ast.Values, 0, 0}, 1, types.NewFieldType(mysql.TypeVarchar)} _, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(""))) require.Error(t, err) - require.Regexp(t, ".*Incorrect parameter count in the call to native function 'values'", err.Error()) + require.Regexp(t, "Incorrect parameter count in the call to native function 'values'$", err.Error()) sig, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums())) require.NoError(t, err) @@ -192,7 +192,7 @@ func TestValues(t *testing.T) { ctx.GetSessionVars().CurrInsertValues = chunk.MutRowFromDatums(types.MakeDatums("1")).ToRow() ret, err = evalBuiltinFunc(sig, chunk.Row{}) require.Error(t, err) - require.Regexp(t, "Session current insert values len.*", err.Error()) + require.Regexp(t, "^Session current insert values len", err.Error()) currInsertValues := types.MakeDatums("1", "2") ctx.GetSessionVars().CurrInsertValues = chunk.MutRowFromDatums(currInsertValues).ToRow() diff --git a/go.mod b/go.mod index b0216c411f6e3..145eb54825ce9 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index ccf7a5cb71fa6..f2e3eae89d719 100644 --- a/go.sum +++ b/go.sum @@ -581,7 +581,6 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210819164333-bd5706b9d9f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20211109071446-a8b4d34474bc/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20211122024046-03abd340988f h1:hjInxK1Ie6CYx7Jy2pYnBdEnWI8jIfr423l9Yh6LRy8= github.com/pingcap/kvproto v0.0.0-20211122024046-03abd340988f/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= @@ -711,8 +710,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f h1:UyJjp3wGIjf1edGiQiIdAtL5QFqaqR4+s3LDwUZU7NY= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f/go.mod h1:BEAS0vXm5BorlF/HTndqGwcGDvaiwe7B7BkfgwwZMJ4= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5 h1:c0zWUeB8aog6TceRyFEjA2fYUJka8RNGBZWU5l7XF9g= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= diff --git a/infoschema/tables_serial_test.go b/infoschema/tables_serial_test.go index 15a8793b0a68d..71aee026f6afd 100644 --- a/infoschema/tables_serial_test.go +++ b/infoschema/tables_serial_test.go @@ -759,7 +759,7 @@ func TestStmtSummaryTable(t *testing.T) { rows := tk.MustQuery("select tidb_decode_plan('" + p1 + "');").Rows() require.Equal(t, 1, len(rows)) require.Equal(t, 1, len(rows[0])) - require.Regexp(t, ".*\n.*Point_Get.*table.tidb, index.PRIMARY.VARIABLE_NAME.*", rows[0][0]) + require.Regexp(t, "\n.*Point_Get.*table.tidb, index.PRIMARY.VARIABLE_NAME", rows[0][0]) sql = "select table_names from information_schema.statements_summary " + "where digest_text like 'select `variable_value`%' and `schema_name`='test'" diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 0a12c363b0ea1..2776e7625570f 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -8051,7 +8051,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999-{{type}}", @@ -8059,14 +8059,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90-{{type}}", diff --git a/metrics/metrics.go b/metrics/metrics.go index 772663f530575..f5c1faf5bb3aa 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -157,6 +157,7 @@ func RegisterMetrics() { prometheus.MustRegister(TopSQLIgnoredCounter) prometheus.MustRegister(TopSQLReportDurationHistogram) prometheus.MustRegister(TopSQLReportDataHistogram) + prometheus.MustRegister(PDApiExecutionHistogram) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/metrics/server.go b/metrics/server.go index 3bc764f6f9afa..68f8e8b3abf86 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -229,6 +229,15 @@ var ( Name: "tiflash_query_total", Help: "Counter of TiFlash queries.", }, []string{LblType, LblResult}) + + PDApiExecutionHistogram = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "pd_api_execution_duration_seconds", + Help: "Bucketed histogram of all pd api execution time (s)", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s + }, []string{LblType}) ) // ExecuteErrorToLabel converts an execute error to label. diff --git a/parser/hintparser_test.go b/parser/hintparser_test.go index 8e9da6c6ad8fd..8ac22ec272123 100644 --- a/parser/hintparser_test.go +++ b/parser/hintparser_test.go @@ -35,7 +35,7 @@ func TestParseHint(t *testing.T) { }{ { input: "", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "MEMORY_QUOTA(8 MB) MEMORY_QUOTA(6 GB)", @@ -86,39 +86,39 @@ func TestParseHint(t *testing.T) { }, { input: "QB_NAME(1)", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "QB_NAME('string literal')", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "QB_NAME(many identifiers)", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "QB_NAME(@qb1)", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "QB_NAME(b'10')", errs: []string{ - `.*Cannot use bit-value literal.*`, - `.*Optimizer hint syntax error at line 1 .*`, + `Cannot use bit-value literal`, + `Optimizer hint syntax error at line 1 `, }, }, { input: "QB_NAME(x'1a')", errs: []string{ - `.*Cannot use hexadecimal literal.*`, - `.*Optimizer hint syntax error at line 1 .*`, + `Cannot use hexadecimal literal`, + `Optimizer hint syntax error at line 1 `, }, }, { input: "JOIN_FIXED_ORDER() BKA()", errs: []string{ - `.*Optimizer hint JOIN_FIXED_ORDER is not supported.*`, - `.*Optimizer hint BKA is not supported.*`, + `Optimizer hint JOIN_FIXED_ORDER is not supported`, + `Optimizer hint BKA is not supported`, }, }, { @@ -298,36 +298,36 @@ func TestParseHint(t *testing.T) { }, { input: "unknown_hint()", - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "set_var(timestamp = 1.5)", errs: []string{ - `.*Cannot use decimal number.*`, - `.*Optimizer hint syntax error at line 1 .*`, + `Cannot use decimal number`, + `Optimizer hint syntax error at line 1 `, }, }, { input: "set_var(timestamp = _utf8mb4'1234')", // Optimizer hint doesn't recognize _charset'strings'. - errs: []string{`.*Optimizer hint syntax error at line 1 .*`}, + errs: []string{`Optimizer hint syntax error at line 1 `}, }, { input: "set_var(timestamp = 9999999999999999999999999999999999999)", errs: []string{ - `.*integer value is out of range.*`, - `.*Optimizer hint syntax error at line 1 .*`, + `integer value is out of range`, + `Optimizer hint syntax error at line 1 `, }, }, { input: "time_range('2020-02-20 12:12:12',456)", errs: []string{ - `.*Optimizer hint syntax error at line 1 .*`, + `Optimizer hint syntax error at line 1 `, }, }, { input: "time_range(456,'2020-02-20 12:12:12')", errs: []string{ - `.*Optimizer hint syntax error at line 1 .*`, + `Optimizer hint syntax error at line 1 `, }, }, { @@ -349,7 +349,7 @@ func TestParseHint(t *testing.T) { require.Lenf(t, errs, len(tc.errs), "input = %s,\n... errs = %q", tc.input, errs) for i, err := range errs { require.Errorf(t, err, "input = %s, i = %d", tc.input, i) - require.Regexpf(t, tc.errs[i], err, "input = %s, i = %d", tc.input, i) + require.Containsf(t, err.Error(), tc.errs[i], "input = %s, i = %d", tc.input, i) } require.Equalf(t, tc.output, output, "input = %s,\n... output = %q", tc.input, output) } diff --git a/parser/model/model_test.go b/parser/model/model_test.go index 8439c1ce0fcdf..190932628c391 100644 --- a/parser/model/model_test.go +++ b/parser/model/model_test.go @@ -366,7 +366,7 @@ func TestDefaultValue(t *testing.T) { err = newBitCol.SetDefaultValue(1) // Only string type is allowed in BIT column. require.Error(t, err) - require.Regexp(t, ".*Invalid default value.*", err.Error()) + require.Contains(t, err.Error(), "Invalid default value") require.Equal(t, 1, newBitCol.GetDefaultValue()) err = newBitCol.SetDefaultValue(randBitStr) require.NoError(t, err) diff --git a/parser/parser_test.go b/parser/parser_test.go index 89e26b3edb557..6808ce18dc904 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -3464,13 +3464,13 @@ func TestHintError(t *testing.T) { stmt, warns, err := p.Parse("select /*+ tidb_unknown(T1,t2) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") require.NoError(t, err) require.Len(t, warns, 1) - require.Regexp(t, `.*Optimizer hint syntax error at line 1 column 23 near "tidb_unknown\(T1,t2\) \*/" `, warns[0].Error()) + require.Regexp(t, `Optimizer hint syntax error at line 1 column 23 near "tidb_unknown\(T1,t2\) \*/" $`, warns[0].Error()) require.Len(t, stmt[0].(*ast.SelectStmt).TableHints, 0) stmt, warns, err = p.Parse("select /*+ TIDB_INLJ(t1, T2) tidb_unknow(T1,t2, 1) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") require.Len(t, stmt[0].(*ast.SelectStmt).TableHints, 0) require.NoError(t, err) require.Len(t, warns, 1) - require.Regexp(t, `.*Optimizer hint syntax error at line 1 column 40 near "tidb_unknow\(T1,t2, 1\) \*/" `, warns[0].Error()) + require.Regexp(t, `Optimizer hint syntax error at line 1 column 40 near "tidb_unknow\(T1,t2, 1\) \*/" $`, warns[0].Error()) _, _, err = p.Parse("select c1, c2 from /*+ tidb_unknow(T1,t2) */ t1, t2 where t1.c1 = t2.c1", "", "") require.NoError(t, err) // Hints are ignored after the "FROM" keyword! _, _, err = p.Parse("select1 /*+ TIDB_INLJ(t1, T2) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") @@ -3490,7 +3490,7 @@ func TestHintError(t *testing.T) { _, warns, err = p.Parse("SELECT id FROM tbl WHERE id = 0 FOR UPDATE /*+ xyz */", "", "") require.NoError(t, err) require.Len(t, warns, 1) - require.Regexp(t, `.*near '/\*\+' at line 1`, warns[0].Error()) + require.Regexp(t, `near '/\*\+' at line 1$`, warns[0].Error()) _, warns, err = p.Parse("create global binding for select /*+ max_execution_time(1) */ 1 using select /*+ max_execution_time(1) */ 1;\n", "", "") require.NoError(t, err) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 3cf7338b6940b..9af659ef6c972 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1501,7 +1501,7 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo return indexScan.IsPointGetByUniqueKey(ctx), nil case *PhysicalTableReader: tableScan := v.TablePlans[0].(*PhysicalTableScan) - isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx) + isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPointNonNullable(ctx) if !isPointRange { return false, nil } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5e44bc2111870..c23614e7c5935 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -843,7 +843,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if canConvertPointGet { allRangeIsPoint := true for _, ran := range path.Ranges { - if !ran.IsPoint(ds.ctx) { + if !ran.IsPointNonNullable(ds.ctx) { + // unique indexes can have duplicated NULL rows so we cannot use PointGet if there is NULL allRangeIsPoint = false break } diff --git a/planner/core/integration_partition_test.go b/planner/core/integration_partition_test.go index f61912b0a4df5..33e4fb214ba81 100644 --- a/planner/core/integration_partition_test.go +++ b/planner/core/integration_partition_test.go @@ -79,10 +79,12 @@ func TestListColVariousTypes(t *testing.T) { tk.MustExec(`create table tstring (a varchar(32)) partition by list columns(a) (partition p0 values in ('a', 'b'), partition p1 values in ('c', 'd'))`) err := tk.ExecToErr(`create table tdouble (a double) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) - require.Regexp(t, ".*not allowed.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "not allowed") err = tk.ExecToErr(`create table tdecimal (a decimal(30, 10)) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) - require.Regexp(t, ".*not allowed.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "not allowed") tk.MustExec(`insert into tint values (0), (1), (2), (3)`) tk.MustExec(`insert into tdate values ('2000-01-01'), ('2000-01-02'), ('2000-01-03'), ('2000-01-04')`) @@ -330,10 +332,12 @@ func TestListPartitionDML(t *testing.T) { tk.MustExec("insert into tlist partition(p0, p1) values (2), (3), (8), (9)") err := tk.ExecToErr("insert into tlist partition(p0) values (9)") - require.Regexp(t, ".*Found a row not matching the given partition set.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Found a row not matching the given partition set") err = tk.ExecToErr("insert into tlist partition(p3) values (20)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") tk.MustExec("update tlist partition(p0) set a=a+1") tk.MustQuery("select a from tlist order by a").Check(testkit.Rows("1", "2", "3", "4", "8", "9")) @@ -353,10 +357,12 @@ func TestListPartitionDML(t *testing.T) { tk.MustExec("insert into tcollist partition(p0, p1) values (2), (3), (8), (9)") err = tk.ExecToErr("insert into tcollist partition(p0) values (9)") - require.Regexp(t, ".*Found a row not matching the given partition set.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Found a row not matching the given partition set") err = tk.ExecToErr("insert into tcollist partition(p3) values (20)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") tk.MustExec("update tcollist partition(p0) set a=a+1") tk.MustQuery("select a from tcollist order by a").Check(testkit.Rows("1", "2", "3", "4", "8", "9")) @@ -384,18 +390,22 @@ func TestListPartitionCreation(t *testing.T) { tk.MustExec("create table tuk1 (a int, b int, unique key(a)) partition by list (a) (partition p0 values in (0))") err := tk.ExecToErr("create table tuk2 (a int, b int, unique key(a)) partition by list (b) (partition p0 values in (0))") - require.Regexp(t, ".*UNIQUE INDEX must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "UNIQUE INDEX must include all columns") err = tk.ExecToErr("create table tuk2 (a int, b int, unique key(a), unique key(b)) partition by list (a) (partition p0 values in (0))") - require.Regexp(t, ".*UNIQUE INDEX must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "UNIQUE INDEX must include all columns") tk.MustExec("create table tcoluk1 (a int, b int, unique key(a)) partition by list columns(a) (partition p0 values in (0))") err = tk.ExecToErr("create table tcoluk2 (a int, b int, unique key(a)) partition by list columns(b) (partition p0 values in (0))") - require.Regexp(t, ".*UNIQUE INDEX must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "UNIQUE INDEX must include all columns") err = tk.ExecToErr("create table tcoluk2 (a int, b int, unique key(a), unique key(b)) partition by list columns(a) (partition p0 values in (0))") - require.Regexp(t, ".*UNIQUE INDEX must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "UNIQUE INDEX must include all columns") // with PK tk.MustExec("create table tpk1 (a int, b int, primary key(a)) partition by list (a) (partition p0 values in (0))") @@ -417,13 +427,16 @@ func TestListPartitionCreation(t *testing.T) { tk.MustExec("create table texp3 (a int, b int) partition by list(a*b) (partition p0 values in (0))") err = tk.ExecToErr("create table texp4 (a int, b int) partition by list(a|b) (partition p0 values in (0))") - require.Regexp(t, ".*This partition function is not allowed.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "This partition function is not allowed") err = tk.ExecToErr("create table texp4 (a int, b int) partition by list(a^b) (partition p0 values in (0))") - require.Regexp(t, ".*This partition function is not allowed.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "This partition function is not allowed") err = tk.ExecToErr("create table texp4 (a int, b int) partition by list(a&b) (partition p0 values in (0))") - require.Regexp(t, ".*This partition function is not allowed.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "This partition function is not allowed") } func TestListPartitionDDL(t *testing.T) { @@ -440,20 +453,24 @@ func TestListPartitionDDL(t *testing.T) { // index tk.MustExec(`create table tlist (a int, b int) partition by list (a) (partition p0 values in (0))`) err := tk.ExecToErr(`alter table tlist add primary key (b)`) // add pk - require.Regexp(t, ".*must include all.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all") tk.MustExec(`alter table tlist add primary key (a)`) err = tk.ExecToErr(`alter table tlist add unique key (b)`) // add uk - require.Regexp(t, ".*must include all.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all") tk.MustExec(`alter table tlist add key (b)`) // add index tk.MustExec(`alter table tlist rename index b to bb`) tk.MustExec(`alter table tlist drop index bb`) tk.MustExec(`create table tcollist (a int, b int) partition by list columns (a) (partition p0 values in (0))`) err = tk.ExecToErr(`alter table tcollist add primary key (b)`) // add pk - require.Regexp(t, ".*must include all.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all") tk.MustExec(`alter table tcollist add primary key (a)`) err = tk.ExecToErr(`alter table tcollist add unique key (b)`) // add uk - require.Regexp(t, ".*must include all.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all") tk.MustExec(`alter table tcollist add key (b)`) // add index tk.MustExec(`alter table tcollist rename index b to bb`) tk.MustExec(`alter table tcollist drop index bb`) @@ -520,26 +537,32 @@ func TestListPartitionOperations(t *testing.T) { tk.MustExec("alter table tlist drop partition p0") tk.MustQuery("select * from tlist").Sort().Check(testkit.Rows("10", "15", "5")) err := tk.ExecToErr("select * from tlist partition (p0)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") tk.MustExec("alter table tlist drop partition p1, p2") tk.MustQuery("select * from tlist").Sort().Check(testkit.Rows("15")) err = tk.ExecToErr("select * from tlist partition (p1)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") err = tk.ExecToErr("alter table tlist drop partition p3") - require.Regexp(t, ".*Cannot remove all partitions.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Cannot remove all partitions") tk.MustExec("insert into tcollist values (0), (5), (10)") tk.MustQuery("select * from tcollist").Sort().Check(testkit.Rows("0", "10", "15", "5")) tk.MustExec("alter table tcollist drop partition p0") tk.MustQuery("select * from tcollist").Sort().Check(testkit.Rows("10", "15", "5")) err = tk.ExecToErr("select * from tcollist partition (p0)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") tk.MustExec("alter table tcollist drop partition p1, p2") tk.MustQuery("select * from tcollist").Sort().Check(testkit.Rows("15")) err = tk.ExecToErr("select * from tcollist partition (p1)") - require.Regexp(t, ".*Unknown partition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown partition") err = tk.ExecToErr("alter table tcollist drop partition p3") - require.Regexp(t, ".*Cannot remove all partitions.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Cannot remove all partitions") // add partition tk.MustExec("alter table tlist add partition (partition p0 values in (0, 1, 2, 3, 4))") @@ -547,14 +570,16 @@ func TestListPartitionOperations(t *testing.T) { tk.MustExec("insert into tlist values (0), (5), (10)") tk.MustQuery("select * from tlist").Sort().Check(testkit.Rows("0", "10", "15", "5")) err = tk.ExecToErr("alter table tlist add partition (partition pxxx values in (4))") - require.Regexp(t, ".*Multiple definition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Multiple definition") tk.MustExec("alter table tcollist add partition (partition p0 values in (0, 1, 2, 3, 4))") tk.MustExec("alter table tcollist add partition (partition p1 values in (5, 6, 7, 8, 9), partition p2 values in (10, 11, 12, 13, 14))") tk.MustExec("insert into tcollist values (0), (5), (10)") tk.MustQuery("select * from tcollist").Sort().Check(testkit.Rows("0", "10", "15", "5")) err = tk.ExecToErr("alter table tcollist add partition (partition pxxx values in (4))") - require.Regexp(t, ".*Multiple definition.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Multiple definition") } func TestListPartitionPrivilege(t *testing.T) { @@ -584,13 +609,17 @@ func TestListPartitionPrivilege(t *testing.T) { tk1.SetSession(se) tk1.MustExec(`use list_partition_pri`) err = tk1.ExecToErr(`alter table tlist truncate partition p0`) - require.Regexp(t, ".*denied.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "denied") err = tk1.ExecToErr(`alter table tlist drop partition p0`) - require.Regexp(t, ".*denied.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "denied") err = tk1.ExecToErr(`alter table tlist add partition (partition p2 values in (2))`) - require.Regexp(t, ".*denied.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "denied") err = tk1.ExecToErr(`insert into tlist values (1)`) - require.Regexp(t, ".*denied.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "denied") } func TestListPartitionShardBits(t *testing.T) { @@ -719,7 +748,8 @@ func TestListPartitionAutoIncre(t *testing.T) { partition p0 values in (0, 1, 2, 3, 4), partition p1 values in (5, 6, 7, 8, 9), partition p2 values in (10, 11, 12, 13, 14))`) - require.Regexp(t, ".*it must be defined as a key.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "it must be defined as a key") tk.MustExec(`create table tlist (a int, b int AUTO_INCREMENT, key(b)) partition by list (a) ( partition p0 values in (0, 1, 2, 3, 4), @@ -735,7 +765,8 @@ func TestListPartitionAutoIncre(t *testing.T) { partition p0 values in (0, 1, 2, 3, 4), partition p1 values in (5, 6, 7, 8, 9), partition p2 values in (10, 11, 12, 13, 14))`) - require.Regexp(t, ".*it must be defined as a key.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "it must be defined as a key") tk.MustExec(`create table tcollist (a int, b int AUTO_INCREMENT, key(b)) partition by list (a) ( partition p0 values in (0, 1, 2, 3, 4), @@ -763,7 +794,8 @@ func TestListPartitionAutoRandom(t *testing.T) { partition p0 values in (0, 1, 2, 3, 4), partition p1 values in (5, 6, 7, 8, 9), partition p2 values in (10, 11, 12, 13, 14))`) - require.Regexp(t, ".*Invalid auto random.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Invalid auto random") tk.MustExec(`create table tlist (a bigint auto_random, primary key(a)) partition by list (a) ( partition p0 values in (0, 1, 2, 3, 4), @@ -774,7 +806,8 @@ func TestListPartitionAutoRandom(t *testing.T) { partition p0 values in (0, 1, 2, 3, 4), partition p1 values in (5, 6, 7, 8, 9), partition p2 values in (10, 11, 12, 13, 14))`) - require.Regexp(t, ".*Invalid auto random.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Invalid auto random") tk.MustExec(`create table tcollist (a bigint auto_random, primary key(a)) partition by list columns (a) ( partition p0 values in (0, 1, 2, 3, 4), @@ -841,9 +874,11 @@ func TestListPartitionTempTable(t *testing.T) { tk.MustExec("drop table if exists tlist") tk.MustExec(`set tidb_enable_list_partition = 1`) err := tk.ExecToErr("create global temporary table t(a int, b int) partition by list(a) (partition p0 values in (0)) on commit delete rows") - require.Regexp(t, ".*Cannot create temporary table with partitions.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Cannot create temporary table with partitions") err = tk.ExecToErr("create global temporary table t(a int, b int) partition by list columns (a) (partition p0 values in (0)) on commit delete rows") - require.Regexp(t, ".*Cannot create temporary table with partitions.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "Cannot create temporary table with partitions") } func TestListPartitionAlterPK(t *testing.T) { @@ -863,7 +898,8 @@ func TestListPartitionAlterPK(t *testing.T) { tk.MustExec(`alter table tlist add primary key(a)`) tk.MustExec(`alter table tlist drop primary key`) err := tk.ExecToErr(`alter table tlist add primary key(b)`) - require.Regexp(t, ".*must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all columns") tk.MustExec(`create table tcollist (a int, b int) partition by list columns (a) ( partition p0 values in (0, 1, 2, 3, 4), @@ -872,7 +908,8 @@ func TestListPartitionAlterPK(t *testing.T) { tk.MustExec(`alter table tcollist add primary key(a)`) tk.MustExec(`alter table tcollist drop primary key`) err = tk.ExecToErr(`alter table tcollist add primary key(b)`) - require.Regexp(t, ".*must include all columns.*", err) + require.Error(t, err) + require.Contains(t, err.Error(), "must include all columns") } func TestListPartitionRandomTransaction(t *testing.T) { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 351a20ba4c552..e58579c5bcb58 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4713,6 +4713,12 @@ func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { func (s *testIntegrationSuite) TestIssue27797(c *C) { tk := testkit.NewTestKit(c, s.store) + origin := tk.MustQuery("SELECT @@session.tidb_partition_prune_mode") + originStr := origin.Rows()[0][0].(string) + defer func() { + tk.MustExec("set @@session.tidb_partition_prune_mode = '" + originStr + "'") + }() + tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") tk.MustExec("use test") tk.MustExec("drop table if exists t27797") tk.MustExec("create table t27797(a int, b int, c int, d int) " + @@ -4835,6 +4841,56 @@ func (s *testIntegrationSerialSuite) TestRejectSortForMPP(c *C) { } } +func (s *testIntegrationSerialSuite) TestRegardNULLAsPoint(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists tpk") + tk.MustExec(`create table tuk (a int, b int, c int, unique key (a, b, c))`) + tk.MustExec(`create table tik (a int, b int, c int, key (a, b, c))`) + for _, va := range []string{"NULL", "1"} { + for _, vb := range []string{"NULL", "1"} { + for _, vc := range []string{"NULL", "1"} { + tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) + tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) + if va == "1" && vb == "1" && vc == "1" { + continue + } + // duplicated NULL rows + tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) + tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) + } + } + } + + var input []string + var output []struct { + SQL string + PlanEnabled []string + PlanDisabled []string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) + output[i].PlanEnabled = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) + output[i].PlanDisabled = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + }) + tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanEnabled...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanDisabled...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} + func (s *testIntegrationSuite) TestIssues29711(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -4897,3 +4953,32 @@ func (s *testIntegrationSuite) TestIssue30094(c *C) { " └─TableFullScan 10000.00 cop[tikv] table:t30094 keep order:false, stats:pseudo", )) } + +func (s *testIntegrationSuite) TestIssue29705(c *C) { + tk := testkit.NewTestKit(c, s.store) + origin := tk.MustQuery("SELECT @@session.tidb_partition_prune_mode") + originStr := origin.Rows()[0][0].(string) + defer func() { + tk.MustExec("set @@session.tidb_partition_prune_mode = '" + originStr + "'") + }() + tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(id int) partition by hash(id) partitions 4;") + tk.MustExec("insert into t values(1);") + result := tk.MustQuery("SELECT COUNT(1) FROM ( SELECT COUNT(1) FROM t b GROUP BY id) a;") + result.Check(testkit.Rows("1")) +} + +func (s *testIntegrationSerialSuite) TestIssue30271(c *C) { + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10), b char(10), c char(10), index (a, b, c)) collate utf8mb4_bin;") + tk.MustExec("insert into t values ('b', 'a', '1'), ('b', 'A', '2'), ('c', 'a', '3');") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustQuery("select * from t where (a>'a' and b='a') or (b = 'A' and a < 'd') order by a,c;").Check(testkit.Rows("b a 1", "b A 2", "c a 3")) + +} diff --git a/planner/core/logical_plan_trace_test.go b/planner/core/logical_plan_trace_test.go index 4ed6960409279..be97ff1624d46 100644 --- a/planner/core/logical_plan_trace_test.go +++ b/planner/core/logical_plan_trace_test.go @@ -101,6 +101,17 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { }, }, }, + { + sql: "select 1+num from (select 1+a as num from t) t1;", + flags: []uint64{flagEliminateProjection}, + assertRuleName: "projection_eliminate", + assertRuleSteps: []assertTraceStep{ + { + assertAction: "Proj[2] is eliminated, Proj[3]'s expressions changed into[plus(1, plus(1, test.t.a))]", + assertReason: "Proj[3]'s child proj[2] is redundant", + }, + }, + }, } for i, tc := range tt { diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index c2cefeeb405e4..ec8e9e92a9556 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -93,6 +93,7 @@ func (s *testPartitionPruneSuit) TestListPartitionPruner(c *C) { tk.MustExec("use test_partition") tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk.MustExec("create table t1 (id int, a int, b int ) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") tk.MustExec("create table t2 (a int, id int, b int) partition by list (a*3 + b - 2*a - b) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") tk.MustExec("create table t3 (b int, id int, a int) partition by list columns (a) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") @@ -169,6 +170,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { // tk1 use to test partition table with index. tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists test_partition_1;") + tk1.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk1.MustExec("create database test_partition_1") tk1.MustExec("use test_partition_1") tk1.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -180,6 +182,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { // tk2 use to compare the result with normal table. tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("drop database if exists test_partition_2;") + tk2.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk2.MustExec("create database test_partition_2") tk2.MustExec("use test_partition_2") tk2.MustExec("create table t1 (id int, a int, b int)") diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index f9cdca27dc42f..a4d8945121519 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1201,7 +1201,7 @@ func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sctx sessionctx.Context) bool return len(p.Ranges) == 1 && p.Index.Unique && len(p.Ranges[0].LowVal) == len(p.Index.Columns) && - p.Ranges[0].IsPoint(sctx) + p.Ranges[0].IsPointNonNullable(sctx) } // PhysicalSelection represents a filter. diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 6e8c276506f75..27056a8937c20 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -212,6 +212,18 @@ func checkCoverIndex(idx *model.IndexInfo, ranges []*ranger.Range) bool { if len(rg.LowVal) != len(idx.Columns) { return false } + for _, v := range rg.LowVal { + if v.IsNull() { + // a unique index may have duplicated rows with NULLs, so we cannot set the unique attribute to true when the range has NULL + // please see https://github.com/pingcap/tidb/issues/29650 for more details + return false + } + } + for _, v := range rg.HighVal { + if v.IsNull() { + return false + } + } } return true } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 77c727f4fb5ce..1d144c72807f9 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -241,6 +241,23 @@ func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column) erro p.schema.Columns = append(p.schema.Columns[:i], p.schema.Columns[i+1:]...) } } + // It's possible that the child operator adds extra columns to the schema. + // Currently, (*LogicalAggregation).PruneColumns() might do this. + // But we don't need such columns, so we add an extra Projection to prune this column when this happened. + for i, child := range p.Children() { + if p.schema.Len() < child.Schema().Len() { + schema := p.schema.Clone() + exprs := make([]expression.Expression, len(p.schema.Columns)) + for j, col := range schema.Columns { + exprs[j] = col + } + proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(p.ctx, p.blockOffset) + proj.SetSchema(schema) + + proj.SetChildren(child) + p.children[i] = proj + } + } } return nil } diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index f641ceb479cbf..f16b9695d5dbb 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -15,7 +15,9 @@ package core import ( + "bytes" "context" + "fmt" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -146,12 +148,12 @@ type projectionEliminator struct { // optimize implements the logicalOptRule interface. func (pe *projectionEliminator) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { - root := pe.eliminate(lp, make(map[string]*expression.Column), false) + root := pe.eliminate(lp, make(map[string]*expression.Column), false, opt) return root, nil } // eliminate eliminates the redundant projection in a logical plan. -func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*expression.Column, canEliminate bool) LogicalPlan { +func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*expression.Column, canEliminate bool, opt *logicalOptimizeOp) LogicalPlan { proj, isProj := p.(*LogicalProjection) childFlag := canEliminate if _, isUnion := p.(*LogicalUnionAll); isUnion { @@ -162,7 +164,7 @@ func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*exp childFlag = true } for i, child := range p.Children() { - p.Children()[i] = pe.eliminate(child, replace, childFlag) + p.Children()[i] = pe.eliminate(child, replace, childFlag, opt) } switch x := p.(type) { @@ -186,6 +188,7 @@ func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*exp proj.Exprs[i] = foldedExpr } p.Children()[0] = child.Children()[0] + appendProjEliminateTraceStep(proj, child, opt) } } @@ -292,3 +295,20 @@ func (p *LogicalWindow) replaceExprColumns(replace map[string]*expression.Column func (*projectionEliminator) name() string { return "projection_eliminate" } + +func appendProjEliminateTraceStep(parent, child *LogicalProjection, opt *logicalOptimizeOp) { + action := func() string { + buffer := bytes.NewBufferString( + fmt.Sprintf("Proj[%v] is eliminated, Proj[%v]'s expressions changed into[", child.ID(), parent.ID())) + for i, expr := range parent.Exprs { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(expr.String()) + } + buffer.WriteString("]") + return buffer.String() + }() + reason := fmt.Sprintf("Proj[%v]'s child proj[%v] is redundant", parent.ID(), child.ID()) + opt.appendStepToCurrent(child.ID(), child.TP(), reason, action) +} diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 7cbe788e3e58f..aece551a5b991 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -29,6 +29,25 @@ ] }, + { + "name": "TestRegardNULLAsPoint", + "cases": [ + "select * from tuk where a<=>null and b=1", + "select * from tik where a<=>null and b=1", + "select * from tuk where a<=>null and b>0 and b<2", + "select * from tik where a<=>null and b>0 and b<2", + "select * from tuk where a<=>null and b>=1 and b<2", + "select * from tik where a<=>null and b>=1 and b<2", + "select * from tuk where a<=>null and b=1 and c=1", + "select * from tik where a<=>null and b=1 and c=1", + "select * from tuk where a=1 and b<=>null and c=1", + "select * from tik where a=1 and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c=1", + "select * from tik where a<=>null and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c<=>null", + "select * from tik where a<=>null and b<=>null and c<=>null" + ] + }, { "name": "TestPushDownToTiFlashWithKeepOrder", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 3fc9530ce6942..cda227a5bbe00 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -220,6 +220,247 @@ } ] }, + { + "Name": "TestRegardNULLAsPoint", + "Cases": [ + { + "SQL": "select * from tuk where a<=>null and b=1", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.b, 1), eq(test.tuk.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.b, 1), eq(test.tik.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1)", + " └─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tik where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1)", + " └─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1), nulleq(test.tuk.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1), nulleq(test.tik.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tuk.b, NULL), nulleq(test.tuk.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tik.b, NULL), nulleq(test.tik.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + } + ] + }, { "Name": "TestPushDownToTiFlashWithKeepOrder", "Cases": [ diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 1cd95fe8f2658..c4cee0cbf1ce2 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1134,10 +1134,10 @@ func TestConfigPrivilege(t *testing.T) { require.True(t, se.Auth(&auth.UserIdentity{Username: "tcd2", Hostname: "localhost", AuthHostname: "tcd2", AuthUsername: "%"}, nil, nil)) _, err := se.ExecuteInternal(context.Background(), `SHOW CONFIG`) require.Error(t, err) - require.Regexp(t, ".*you need \\(at least one of\\) the CONFIG privilege\\(s\\) for this operation", err.Error()) + require.Regexp(t, "you need \\(at least one of\\) the CONFIG privilege\\(s\\) for this operation$", err.Error()) _, err = se.ExecuteInternal(context.Background(), `SET CONFIG TIKV testkey="testval"`) require.Error(t, err) - require.Regexp(t, ".*you need \\(at least one of\\) the CONFIG privilege\\(s\\) for this operation", err.Error()) + require.Regexp(t, "you need \\(at least one of\\) the CONFIG privilege\\(s\\) for this operation$", err.Error()) mustExec(t, se, `DROP USER tcd1, tcd2`) } @@ -1980,11 +1980,11 @@ func TestRenameUser(t *testing.T) { // Check privileges (need CREATE USER) _, err := se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") require.Error(t, err) - require.Regexp(t, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation", err.Error()) + require.Regexp(t, "Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation$", err.Error()) mustExec(t, rootSe, "GRANT UPDATE ON mysql.user TO 'ru1'@'localhost'") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") require.Error(t, err) - require.Regexp(t, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation", err.Error()) + require.Regexp(t, "Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation$", err.Error()) mustExec(t, rootSe, "GRANT CREATE USER ON *.* TO 'ru1'@'localhost'") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") require.NoError(t, err) @@ -1997,19 +1997,19 @@ func TestRenameUser(t *testing.T) { // Including negative tests, i.e. non existing from user and existing to user _, err = rootSe.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru1@localhost") require.Error(t, err) - require.Regexp(t, ".*Operation RENAME USER failed for ru3@%.*", err.Error()) + require.Contains(t, err.Error(), "Operation RENAME USER failed for ru3@%") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru4 TO ru5@localhost") require.Error(t, err) - require.Regexp(t, ".*Operation RENAME USER failed for ru4@%.*", err.Error()) + require.Contains(t, err.Error(), "Operation RENAME USER failed for ru4@%") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru3") require.Error(t, err) - require.Regexp(t, ".*Operation RENAME USER failed for ru3@%.*", err.Error()) + require.Contains(t, err.Error(), "Operation RENAME USER failed for ru3@%") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru4 TO ru7") require.Error(t, err) - require.Regexp(t, ".*Operation RENAME USER failed for ru4@%.*", err.Error()) + require.Contains(t, err.Error(), "Operation RENAME USER failed for ru4@%") _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru6@localhost TO ru1@localhost") require.Error(t, err) - require.Regexp(t, ".*Operation RENAME USER failed for ru6@localhost.*", err.Error()) + require.Contains(t, err.Error(), "Operation RENAME USER failed for ru6@localhost") // Test multi rename, this is a full swap of ru3 and ru6, i.e. need to read its previous state in the same transaction. _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru3' TO 'ru3_tmp', ru6@localhost TO ru3, 'ru3_tmp' to ru6@localhost") diff --git a/server/server_test.go b/server/server_test.go index f9d22e866458a..e623af576b1de 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -1418,7 +1418,7 @@ func (cli *testServerClient) runTestExplainForConn(t *testing.T) { row := make([]string, 9) err = rows.Scan(&row[0], &row[1], &row[2], &row[3], &row[4], &row[5], &row[6], &row[7], &row[8]) require.NoError(t, err) - require.Regexp(t, "Point_Get_1,1.00,1,root,table:t,time.*loop.*handle:1.*", strings.Join(row, ",")) + require.Regexp(t, "^Point_Get_1,1.00,1,root,table:t,time.*loop.*handle:1", strings.Join(row, ",")) require.NoError(t, rows.Close()) }) } @@ -1894,7 +1894,7 @@ func getStmtCnt(content string) (stmtCnt map[string]int) { const retryTime = 100 -func (cli *testServerClient) waitUntilServerOnline() { +func (cli *testServerClient) waitUntilServerCanConnect() { // connect server retry := 0 for ; retry < retryTime; retry++ { @@ -1911,8 +1911,14 @@ func (cli *testServerClient) waitUntilServerOnline() { if retry == retryTime { log.Fatal("failed to connect DB in every 10 ms", zap.Int("retryTime", retryTime)) } +} - for retry = 0; retry < retryTime; retry++ { +func (cli *testServerClient) waitUntilServerOnline() { + // connect server + cli.waitUntilServerCanConnect() + + retry := 0 + for ; retry < retryTime; retry++ { // fetch http status resp, err := cli.fetchStatus("/status") if err == nil { diff --git a/server/tidb_test.go b/server/tidb_test.go index 2536752d3b0fe..c76ad1957a766 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -488,7 +488,7 @@ func TestSocketAndIp(t *testing.T) { err := server.Run() require.NoError(t, err) }() - time.Sleep(time.Millisecond * 100) + cli.waitUntilServerCanConnect() defer server.Close() // Test with Socket connection + Setup user1@% for all host access @@ -689,17 +689,19 @@ func TestOnlySocket(t *testing.T) { db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "root" config.DBName = "test" - config.Addr = "127.0.0.1" })) - require.NoErrorf(t, err, "Connect succeeded when not configured!?!") - defer db.Close() + require.NoErrorf(t, err, "Open failed") + err = db.Ping() + require.Errorf(t, err, "Connect succeeded when not configured!?!") + db.Close() db, err = sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "user1" config.DBName = "test" - config.Addr = "127.0.0.1" })) - require.NoErrorf(t, err, "Connect succeeded when not configured!?!") - defer db.Close() + require.NoErrorf(t, err, "Open failed") + err = db.Ping() + require.Errorf(t, err, "Connect succeeded when not configured!?!") + db.Close() // Test with unix domain socket file connection with all hosts cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" @@ -1246,7 +1248,8 @@ func TestGracefulShutdown(t *testing.T) { // nolint: bodyclose _, err = cli.fetchStatus("/status") // status is gone - require.Regexp(t, ".*connect: connection refused", err.Error()) + require.Error(t, err) + require.Regexp(t, "connect: connection refused$", err.Error()) } func TestPessimisticInsertSelectForUpdate(t *testing.T) { @@ -1564,7 +1567,7 @@ func TestTopSQLAgent(t *testing.T) { for _, r := range records { sqlMeta, exist := agentServer.GetSQLMetaByDigestBlocking(r.SqlDigest, time.Second) require.True(t, exist) - require.Regexp(t, "select.*from.*join.*", sqlMeta.NormalizedSql) + require.Regexp(t, "^select.*from.*join", sqlMeta.NormalizedSql) if len(r.PlanDigest) == 0 { continue } @@ -1572,7 +1575,7 @@ func TestTopSQLAgent(t *testing.T) { require.True(t, exist) plan = strings.Replace(plan, "\n", " ", -1) plan = strings.Replace(plan, "\t", " ", -1) - require.Regexp(t, ".*Join.*Select.*", plan) + require.Regexp(t, "Join.*Select", plan) } } runWorkload := func(start, end int) context.CancelFunc { @@ -1677,3 +1680,103 @@ func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, t *testing.T, fn fu fn(db) } } + +func TestLocalhostClientMapping(t *testing.T) { + t.Parallel() + osTempDir := os.TempDir() + tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") + require.NoError(t, err) + socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK + defer os.RemoveAll(tempDir) + + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = socketFile + cfg.Port = cli.port + cfg.Status.ReportStatus = false + + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + defer server.Close() + cli.waitUntilServerCanConnect() + + cli.port = getPortFromTCPAddr(server.listener.Addr()) + // Create a db connection for root + db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { + config.User = "root" + config.Net = "unix" + config.DBName = "test" + config.Addr = socketFile + })) + require.NoErrorf(t, err, "Open failed") + err = db.Ping() + require.NoErrorf(t, err, "Ping failed") + defer db.Close() + dbt := testkit.NewDBTestKit(t, db) + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + + dbt.MustExec("CREATE USER 'localhostuser'@'localhost'") + dbt.MustExec("CREATE USER 'localhostuser'@'%'") + defer func() { + dbt.MustExec("DROP USER IF EXISTS 'localhostuser'@'%'") + dbt.MustExec("DROP USER IF EXISTS 'localhostuser'@'localhost'") + dbt.MustExec("DROP USER IF EXISTS 'localhostuser'@'127.0.0.1'") + }() + + dbt.MustExec("GRANT SELECT ON test.* TO 'localhostuser'@'%'") + dbt.MustExec("GRANT SELECT,UPDATE ON test.* TO 'localhostuser'@'localhost'") + + // Test with loopback interface - Should get access to localhostuser@localhost! + cli.runTests(t, func(config *mysql.Config) { + config.User = "localhostuser" + config.DBName = "test" + }, + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + // NOTICE: this is not compatible with MySQL! (MySQL would report localhostuser@localhost also for 127.0.0.1) + cli.checkRows(t, rows, "localhostuser@127.0.0.1") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'localhostuser'@'localhost'\nGRANT SELECT,UPDATE ON test.* TO 'localhostuser'@'localhost'") + }) + + dbt.MustExec("DROP USER IF EXISTS 'localhostuser'@'localhost'") + dbt.MustExec("CREATE USER 'localhostuser'@'127.0.0.1'") + dbt.MustExec("GRANT SELECT,UPDATE ON test.* TO 'localhostuser'@'127.0.0.1'") + // Test with unix domain socket file connection - Should get access to '%' + cli.runTests(t, func(config *mysql.Config) { + config.Net = "unix" + config.Addr = socketFile + config.User = "localhostuser" + config.DBName = "test" + }, + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "localhostuser@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'localhostuser'@'%'\nGRANT SELECT ON test.* TO 'localhostuser'@'%'") + }) + + // Test if only localhost exists + dbt.MustQuery("DROP USER 'localhostuser'@'%'") + dbSocket, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { + config.User = "localhostuser" + config.Net = "unix" + config.DBName = "test" + config.Addr = socketFile + })) + require.NoErrorf(t, err, "Open failed") + defer dbSocket.Close() + err = dbSocket.Ping() + require.Errorf(t, err, "Connection successful without matching host for unix domain socket!") +} diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 5d896000d5def..37fbe907f2b10 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/tracing" - "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" atomic2 "go.uber.org/atomic" @@ -289,27 +288,20 @@ func (sc *StatementContext) GetPlanDigest() (normalized string, planDigest *pars // GetResourceGroupTagger returns the implementation of tikvrpc.ResourceGroupTagger related to self. func (sc *StatementContext) GetResourceGroupTagger() tikvrpc.ResourceGroupTagger { + normalized, digest := sc.SQLDigest() + planDigest := sc.planDigest return func(req *tikvrpc.Request) { if req == nil { return } - req.ResourceGroupTag = sc.GetResourceGroupTagByLabel( + if len(normalized) == 0 { + return + } + req.ResourceGroupTag = resourcegrouptag.EncodeResourceGroupTag(digest, planDigest, resourcegrouptag.GetResourceGroupLabelByKey(resourcegrouptag.GetFirstKeyFromRequest(req))) } } -// GetResourceGroupTagByLabel gets the resource group of the statement based on the label. -func (sc *StatementContext) GetResourceGroupTagByLabel(label tipb.ResourceGroupTagLabel) []byte { - if sc == nil { - return nil - } - normalized, sqlDigest := sc.SQLDigest() - if len(normalized) == 0 { - return nil - } - return resourcegrouptag.EncodeResourceGroupTag(sqlDigest, sc.planDigest, label) -} - // SetPlanDigest sets the normalized plan and plan digest. func (sc *StatementContext) SetPlanDigest(normalized string, planDigest *parser.Digest) { if planDigest != nil { diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index acfc9c00866d1..f5bf2cca866be 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/util" ) @@ -91,25 +90,3 @@ func TestStatementContextPushDownFLags(t *testing.T) { require.Equal(t, tt.out, got) } } - -func TestGetResourceGroupTagByLabel(t *testing.T) { - ctx := stmtctx.StatementContext{OriginalSQL: "SELECT * FROM t"} - tagRow := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow) - tagIndex := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex) - tagUnknown := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) - tagRow2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow) - tagIndex2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex) - tagUnknown2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) - require.NotEmpty(t, tagRow) - require.NotEmpty(t, tagIndex) - require.NotEmpty(t, tagUnknown) - require.NotEmpty(t, tagRow2) - require.NotEmpty(t, tagIndex2) - require.NotEmpty(t, tagUnknown2) - require.Equal(t, &tagRow, &tagRow2) // mem addr - require.Equal(t, &tagIndex, &tagIndex2) - require.Equal(t, &tagUnknown, &tagUnknown2) - require.NotEqual(t, &tagRow, &tagIndex) - require.NotEqual(t, &tagRow, &tagUnknown) - require.NotEqual(t, &tagIndex, &tagUnknown) -} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 092948a5fac48..057ba9a6dcdff 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -936,6 +936,9 @@ type SessionVars struct { // EnablePseudoForOutdatedStats if using pseudo for outdated stats EnablePseudoForOutdatedStats bool + // RegardNULLAsPoint if regard NULL as Point + RegardNULLAsPoint bool + // LocalTemporaryTables is *infoschema.LocalTemporaryTables, use interface to avoid circle dependency. // It's nil if there is no local temporary table. LocalTemporaryTables interface{} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index f76a9ff9f0c98..5a47319c6580f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1857,6 +1857,10 @@ var defaultSysVars = []*SysVar{ s.EnablePseudoForOutdatedStats = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBRegardNULLAsPoint, Value: BoolToOnOff(DefTiDBRegardNULLAsPoint), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.RegardNULLAsPoint = TiDBOptOn(val) + return nil + }}, {Scope: ScopeNone, Name: "version_compile_os", Value: runtime.GOOS}, {Scope: ScopeNone, Name: "version_compile_machine", Value: runtime.GOARCH}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 2a5e98ece6dd5..2e38db2699969 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -598,6 +598,9 @@ const ( // TiDBEnablePseudoForOutdatedStats indicates whether use pseudo for outdated stats TiDBEnablePseudoForOutdatedStats = "tidb_enable_pseudo_for_outdated_stats" + // TiDBRegardNULLAsPoint indicates whether regard NULL as point when optimizing + TiDBRegardNULLAsPoint = "tidb_regard_null_as_point" + // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" ) @@ -771,6 +774,7 @@ const ( DefTiDBEnableTSOFollowerProxy = false DefTiDBEnableOrderedResultMode = false DefTiDBEnablePseudoForOutdatedStats = true + DefTiDBRegardNULLAsPoint = true DefEnablePlacementCheck = true DefTimestamp = "0" ) diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index a388ecf50591f..e746d1407b305 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -467,7 +467,8 @@ func TestVarsutil(t *testing.T) { require.NoError(t, err) require.Equal(t, "10", val) err = SetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount, "a") - require.Regexp(t, ".*Incorrect argument type to variable 'tidb_stmt_summary_max_stmt_count'", err) + require.Error(t, err) + require.Regexp(t, "Incorrect argument type to variable 'tidb_stmt_summary_max_stmt_count'$", err) err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "10") require.NoError(t, err) @@ -475,16 +476,20 @@ func TestVarsutil(t *testing.T) { require.NoError(t, err) require.Equal(t, "10", val) err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "a") - require.Regexp(t, ".*Incorrect argument type to variable 'tidb_stmt_summary_max_sql_length'", err.Error()) + require.Error(t, err) + require.Regexp(t, "Incorrect argument type to variable 'tidb_stmt_summary_max_sql_length'$", err.Error()) err = SetSessionSystemVar(v, TiDBFoundInPlanCache, "1") - require.Regexp(t, ".*]Variable 'last_plan_from_cache' is a read only variable", err.Error()) + require.Error(t, err) + require.Regexp(t, "]Variable 'last_plan_from_cache' is a read only variable$", err.Error()) err = SetSessionSystemVar(v, TiDBFoundInBinding, "1") - require.Regexp(t, ".*]Variable 'last_plan_from_binding' is a read only variable", err.Error()) + require.Error(t, err) + require.Regexp(t, "]Variable 'last_plan_from_binding' is a read only variable$", err.Error()) err = SetSessionSystemVar(v, "UnknownVariable", "on") - require.Regexp(t, ".*]Unknown system variable 'UnknownVariable'", err.Error()) + require.Error(t, err) + require.Regexp(t, "]Unknown system variable 'UnknownVariable'$", err.Error()) // reset warnings v.StmtCtx.TruncateWarnings(0) @@ -493,7 +498,8 @@ func TestVarsutil(t *testing.T) { err = SetSessionSystemVar(v, TiDBAnalyzeVersion, "4") require.NoError(t, err) // converts to max value warn := v.StmtCtx.GetWarnings()[0] - require.Regexp(t, ".*Truncated incorrect tidb_analyze_version value", warn.Err.Error()) + require.Error(t, warn.Err) + require.Contains(t, warn.Err.Error(), "Truncated incorrect tidb_analyze_version value") } func TestValidate(t *testing.T) { diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 50ec15dd01176..25ed965ea15e3 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -306,7 +306,8 @@ type copIterator struct { // when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice. closed uint32 - resolvedLocks util.TSSet + resolvedLocks util.TSSet + committedLocks util.TSSet actionOnExceed *rateLimitAction } @@ -444,7 +445,7 @@ func (it *copIterator) open(ctx context.Context, enabledRateLimitAction bool) { respChan: it.respChan, finishCh: it.finishCh, vars: it.vars, - kvclient: txnsnapshot.NewClientHelper(it.store.store, &it.resolvedLocks, false), + kvclient: txnsnapshot.NewClientHelper(it.store.store, &it.resolvedLocks, &it.committedLocks, false), memTracker: it.memTracker, replicaReadSeed: it.replicaReadSeed, actionOnExceed: it.actionOnExceed, diff --git a/store/helper/helper.go b/store/helper/helper.go index 3b8e122071959..125052d10cd75 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/tablecodec" @@ -767,10 +768,12 @@ func (h *Helper) requestPD(method, uri string, body io.Reader, res interface{}) if err != nil { return err } + start := time.Now() resp, err := util.InternalHTTPClient().Do(req) if err != nil { return errors.Trace(err) } + metrics.PDApiExecutionHistogram.WithLabelValues("common").Observe(time.Since(start).Seconds()) defer func() { err = resp.Body.Close() diff --git a/tidb-server/main.go b/tidb-server/main.go index 187c3d15fa6a1..f53b1f26f0e88 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -215,11 +215,6 @@ func main() { syncLog() } -func exit() { - syncLog() - os.Exit(0) -} - func syncLog() { if err := log.Sync(); err != nil { // Don't complain about /dev/stdout as Fsync will return EINVAL. @@ -259,7 +254,7 @@ func setCPUAffinity() { c, err := strconv.Atoi(af) if err != nil { fmt.Fprintf(os.Stderr, "wrong affinity cpu config: %s", *affinityCPU) - exit() + os.Exit(1) } cpu = append(cpu, c) } @@ -267,7 +262,7 @@ func setCPUAffinity() { err := linux.SetAffinity(cpu) if err != nil { fmt.Fprintf(os.Stderr, "set cpu affinity failure: %v", err) - exit() + os.Exit(1) } runtime.GOMAXPROCS(len(cpu)) metrics.MaxProcs.Set(float64(runtime.GOMAXPROCS(0))) diff --git a/types/binary_literal_test.go b/types/binary_literal_test.go index 81b9fde6767a5..bf7cca07266f4 100644 --- a/types/binary_literal_test.go +++ b/types/binary_literal_test.go @@ -111,6 +111,7 @@ func TestBinaryLiteral(t *testing.T) { t.Parallel() b, err := ParseBitStr("") require.Nil(t, b) + require.Error(t, err) require.Contains(t, err.Error(), "invalid empty ") }) @@ -151,6 +152,7 @@ func TestBinaryLiteral(t *testing.T) { t.Parallel() b, err := ParseBitStr("") require.Nil(t, b) + require.Error(t, err) require.Contains(t, err.Error(), "invalid empty ") }) diff --git a/types/datum.go b/types/datum.go index efaef65675627..f81c3c7a94d14 100644 --- a/types/datum.go +++ b/types/datum.go @@ -2180,7 +2180,8 @@ func (ds *datumsSorter) Len() int { } func (ds *datumsSorter) Less(i, j int) bool { - cmp, err := ds.datums[i].CompareDatum(ds.sc, &ds.datums[j]) + // TODO: set collation explicitly when rewrites feedback. + cmp, err := ds.datums[i].Compare(ds.sc, &ds.datums[j], collate.GetCollator(ds.datums[i].Collation())) if err != nil { ds.err = errors.Trace(err) return true @@ -2362,7 +2363,7 @@ func ChangeReverseResultByUpperLowerBound( resRetType.Decimal = int(res.GetMysqlDecimal().GetDigitsInt()) } bound := getDatumBound(&resRetType, rType) - cmp, err := d.CompareDatum(sc, &bound) + cmp, err := d.Compare(sc, &bound, collate.GetCollator(resRetType.Collate)) if err != nil { return d, err } diff --git a/types/datum_test.go b/types/datum_test.go index 2a36f21d4cc7e..fd433d1e3fb19 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -167,7 +167,7 @@ func TestConvertToFloat(t *testing.T) { {NewDatum([]byte("12345.678")), mysql.TypeDouble, "", 12345.678, 12345.678}, {NewDatum(int64(12345)), mysql.TypeDouble, "", 12345, 12345}, {NewDatum(uint64(123456)), mysql.TypeDouble, "", 123456, 123456}, - {NewDatum(byte(123)), mysql.TypeDouble, "cannot convert .*", 0, 0}, + {NewDatum(byte(123)), mysql.TypeDouble, "cannot convert ", 0, 0}, {NewDatum(math.NaN()), mysql.TypeDouble, "constant .* overflows double", 0, 0}, {NewDatum(math.Inf(-1)), mysql.TypeDouble, "constant .* overflows double", math.Inf(-1), float32(math.Inf(-1))}, {NewDatum(math.Inf(1)), mysql.TypeDouble, "constant .* overflows double", math.Inf(1), float32(math.Inf(1))}, @@ -182,7 +182,8 @@ func TestConvertToFloat(t *testing.T) { if testCase.errMsg == "" { require.NoError(t, err) } else { - require.Regexp(t, testCase.errMsg, err) + require.Error(t, err) + require.Regexp(t, testCase.errMsg, err.Error()) } require.Equal(t, testCase.r32, converted.GetFloat32()) if testCase.tp == mysql.TypeDouble { @@ -476,7 +477,7 @@ func TestChangeReverseResultByUpperLowerBound(t *testing.T) { reverseRes, err := ChangeReverseResultByUpperLowerBound(sc, test.retType, test.a, test.roundType) require.NoError(t, err) var cmp int - cmp, err = reverseRes.CompareDatum(sc, &test.res) + cmp, err = reverseRes.Compare(sc, &test.res, collate.GetBinaryCollator()) require.NoError(t, err) require.Equalf(t, 0, cmp, "%dth got:%#v, expect:%#v", ith, reverseRes, test.res) } diff --git a/types/fsp_test.go b/types/fsp_test.go index f03b421c0f745..7f5761ba7bd42 100644 --- a/types/fsp_test.go +++ b/types/fsp_test.go @@ -73,13 +73,13 @@ func TestParseFrac(t *testing.T) { require.Equal(t, 0, obtained) require.False(t, overflow) require.Error(t, err) - require.Regexp(t, "Invalid fsp .*", err.Error()) + require.Regexp(t, "^Invalid fsp ", err.Error()) obtained, overflow, err = ParseFrac("NotNum", MaxFsp) require.Equal(t, 0, obtained) require.False(t, overflow) require.Error(t, err) - require.Regexp(t, "strconv.ParseInt:.*", err.Error()) + require.Regexp(t, "^strconv.ParseInt:", err.Error()) obtained, overflow, err = ParseFrac("1235", 6) require.Equal(t, 123500, obtained) diff --git a/types/json/binary_test.go b/types/json/binary_test.go index 2d7fe58ee4eb8..e54ed71398d45 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -461,12 +461,12 @@ func TestParseBinaryFromString(t *testing.T) { obj, err := ParseBinaryFromString("") require.Error(t, err) require.Equal(t, "", obj.String()) - require.Regexp(t, ".*The document is empty.*", err.Error()) + require.Contains(t, err.Error(), "The document is empty") obj, err = ParseBinaryFromString(`"a""`) require.Error(t, err) require.Equal(t, "", obj.String()) - require.Regexp(t, ".*The document root must not be followed by other values\\..*", err.Error()) + require.Contains(t, err.Error(), "The document root must not be followed by other values.") } func TestCreateBinary(t *testing.T) { @@ -493,7 +493,7 @@ func TestCreateBinary(t *testing.T) { func() { defer func() { r := recover() - require.Regexp(t, "unknown type:.*", r) + require.Regexp(t, "^unknown type:", r) }() bj = CreateBinary(int8(123)) require.Equal(t, bj.TypeCode, bj.TypeCode) diff --git a/types/time_test.go b/types/time_test.go index d061518d56eec..fc914034acc73 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -1541,7 +1541,8 @@ func TestExtractDatetimeNum(t *testing.T) { res, err = types.ExtractDatetimeNum(&in, "TEST_ERROR") require.Equal(t, int64(0), res) - require.Regexp(t, "invalid unit.*", err) + require.Error(t, err) + require.Regexp(t, "^invalid unit", err) in = types.NewTime(types.FromDate(0000, 00, 00, 00, 00, 00, 0000), mysql.TypeTimestamp, types.DefaultFsp) @@ -1627,7 +1628,8 @@ func TestExtractDurationNum(t *testing.T) { } res, err := types.ExtractDurationNum(&in, "TEST_ERROR") require.Equal(t, int64(0), res) - require.Regexp(t, "invalid unit.*", err) + require.Error(t, err) + require.Regexp(t, "^invalid unit", err) } } diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 1341317649c22..07fb1be85e17f 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -210,7 +210,7 @@ func TestCancelJobs(t *testing.T) { errs, err = CancelJobs(txn, []int64{-1}) require.NoError(t, err) require.Error(t, errs[0]) - require.Regexp(t, ".*DDL Job:-1 not found", errs[0].Error()) + require.Regexp(t, "DDL Job:-1 not found$", errs[0].Error()) // test cancel finish job. job := &model.Job{ @@ -224,7 +224,7 @@ func TestCancelJobs(t *testing.T) { errs, err = CancelJobs(txn, []int64{100}) require.NoError(t, err) require.Error(t, errs[0]) - require.Regexp(t, ".*This job:100 is finished, so can't be cancelled", errs[0].Error()) + require.Regexp(t, "This job:100 is finished, so can't be cancelled$", errs[0].Error()) // test can't cancelable job. job.Type = model.ActionDropIndex @@ -236,7 +236,7 @@ func TestCancelJobs(t *testing.T) { errs, err = CancelJobs(txn, []int64{101}) require.NoError(t, err) require.Error(t, errs[0]) - require.Regexp(t, ".*This job:101 is almost finished, can't be cancelled now", errs[0].Error()) + require.Regexp(t, "This job:101 is almost finished, can't be cancelled now$", errs[0].Error()) // When both types of jobs exist in the DDL queue, // we first cancel the job with a larger ID. diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index fef959a0c9c8b..a1480c9745ed2 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -945,12 +945,12 @@ func TestCutOneError(t *testing.T) { var b []byte _, _, err := CutOne(b) require.Error(t, err) - require.Regexp(t, "invalid encoded key", err.Error()) + require.EqualError(t, err, "invalid encoded key") b = []byte{4 /* codec.uintFlag */, 0, 0, 0} _, _, err = CutOne(b) require.Error(t, err) - require.Regexp(t, "invalid encoded key.*", err.Error()) + require.Regexp(t, "^invalid encoded key", err.Error()) } func TestSetRawValues(t *testing.T) { diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index f539f759c1675..7b422d10243cc 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -98,7 +98,7 @@ func detachColumnDNFConditions(sctx sessionctx.Context, conditions []expression. // in function which is `column in (constant list)`. // If so, it will return the offset of this column in the slice, otherwise return -1 for not found. // Since combining `x >= 2` and `x <= 2` can lead to an eq condition `x = 2`, we take le/ge/lt/gt into consideration. -func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression.Column) int { +func getPotentialEqOrInColOffset(sctx sessionctx.Context, expr expression.Expression, cols []*expression.Column) int { f, ok := expr.(*expression.ScalarFunction) if !ok { return -1 @@ -109,7 +109,7 @@ func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression. dnfItems := expression.FlattenDNFConditions(f) offset := int(-1) for _, dnfItem := range dnfItems { - curOffset := getPotentialEqOrInColOffset(dnfItem, cols) + curOffset := getPotentialEqOrInColOffset(sctx, dnfItem, cols) if curOffset == -1 { return -1 } @@ -129,7 +129,7 @@ func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression. } if constVal, ok := f.GetArgs()[1].(*expression.Constant); ok { val, err := constVal.Eval(chunk.Row{}) - if err != nil || val.IsNull() { + if err != nil || (!sctx.GetSessionVars().RegardNULLAsPoint && val.IsNull()) { // treat col<=>null as range scan instead of point get to avoid incorrect results // when nullable unique index has multiple matches for filter x is null return -1 @@ -151,7 +151,7 @@ func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression. } if constVal, ok := f.GetArgs()[0].(*expression.Constant); ok { val, err := constVal.Eval(chunk.Row{}) - if err != nil || val.IsNull() { + if err != nil || (!sctx.GetSessionVars().RegardNULLAsPoint && val.IsNull()) { return -1 } for i, col := range cols { @@ -517,7 +517,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex columnValues := make([]*valueInfo, len(cols)) offsets := make([]int, len(conditions)) for i, cond := range conditions { - offset := getPotentialEqOrInColOffset(cond, cols) + offset := getPotentialEqOrInColOffset(sctx, cond, cols) offsets[i] = offset if offset == -1 { continue @@ -715,7 +715,8 @@ func isSameValue(sc *stmtctx.StatementContext, lhs, rhs *valueInfo) (bool, error if lhs == nil || rhs == nil || lhs.mutable || rhs.mutable || lhs.value.Kind() != rhs.value.Kind() { return false, nil } - cmp, err := lhs.value.CompareDatum(sc, rhs.value) + // binary collator may not the best choice, but it can make sure the result is correct. + cmp, err := lhs.value.Compare(sc, rhs.value, collate.GetBinaryCollator()) if err != nil { return false, err } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index f0aaa9b09ee7b..d18bf80849529 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -961,6 +961,7 @@ func TestCompIndexDNFMatch(t *testing.T) { require.NoError(t, err) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") + testKit.MustExec(`set @@session.tidb_regard_null_as_point=false`) testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a int, b int, c int, key(a,b,c));") testKit.MustExec("insert into t values(1,2,2)") diff --git a/util/ranger/types.go b/util/ranger/types.go index 8adb8f9389485..c950419cf217f 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -82,7 +82,7 @@ func (ran *Range) Clone() *Range { // IsPoint returns if the range is a point. func (ran *Range) IsPoint(sctx sessionctx.Context) bool { - return ran.isPoint(sctx, false) + return ran.isPoint(sctx, sctx.GetSessionVars().RegardNULLAsPoint) } func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool { @@ -112,6 +112,11 @@ func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool return !ran.LowExclude && !ran.HighExclude } +// IsPointNonNullable returns if the range is a point without NULL. +func (ran *Range) IsPointNonNullable(sctx sessionctx.Context) bool { + return ran.isPoint(sctx, false) +} + // IsPointNullable returns if the range is a point. func (ran *Range) IsPointNullable(sctx sessionctx.Context) bool { return ran.isPoint(sctx, true) diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index cb743418d1692..26abf03eb6e95 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -130,7 +130,7 @@ func TestRowDecoder(t *testing.T) { for i, col := range cols[:len(cols)-1] { v, ok := r[col.ID] if ok { - equal, err1 := v.CompareDatum(sc, &row.output[i]) + equal, err1 := v.Compare(sc, &row.output[i], collate.GetBinaryCollator()) require.Nil(t, err1) require.Equal(t, 0, equal) } else { @@ -144,7 +144,7 @@ func TestRowDecoder(t *testing.T) { for k, v := range r2 { v1, ok := r[k] require.True(t, ok) - equal, err1 := v.CompareDatum(sc, &v1) + equal, err1 := v.Compare(sc, &v1, collate.GetBinaryCollator()) require.Nil(t, err1) require.Equal(t, 0, equal) } diff --git a/util/sqlexec/utils_test.go b/util/sqlexec/utils_test.go index 16826e57149bc..7cc2b76d4c38e 100644 --- a/util/sqlexec/utils_test.go +++ b/util/sqlexec/utils_test.go @@ -143,7 +143,7 @@ func TestEscapeSQL(t *testing.T) { name: "%? missing arguments", input: "select %? from %?", params: []interface{}{4}, - err: "missing arguments.*", + err: "^missing arguments", }, { name: "nil", @@ -339,7 +339,7 @@ func TestEscapeSQL(t *testing.T) { name: "identifier, wrong arg", input: "use %n", params: []interface{}{3}, - err: "expect a string identifier.*", + err: "^expect a string identifier", }, { name: "identifier",