diff --git a/.github/workflows/integration-test-with-real-tikv.yml b/.github/workflows/integration-test-with-real-tikv.yml deleted file mode 100644 index 4b34b539444bd..0000000000000 --- a/.github/workflows/integration-test-with-real-tikv.yml +++ /dev/null @@ -1,44 +0,0 @@ -name: Real TiKV Tests - -on: - push: - branches: [master] - pull_request: - branches: [master] - -concurrency: - group: ${{ github.ref }}-${{ github.workflow }} - cancel-in-progress: true - -jobs: - realtikv-test: - runs-on: ubuntu-latest - - strategy: - matrix: - suite: - - brietest - - pessimistictest - - sessiontest - - statisticstest - - txntest - - steps: - - uses: actions/checkout@v2 - - uses: actions/setup-go@v2 - with: - go-version: 1.18 - - name: Setup TiKV cluster - run: | - # Disable pipelined pessimistic lock temporarily until tikv#11649 is resolved - echo -e "[pessimistic-txn]\npipelined = false\n" > tikv.toml - echo -e "[raftdb]\nmax-open-files = 20480\n" >> tikv.toml - echo -e "[rocksdb]\nmax-open-files = 20480\n" >> tikv.toml - curl --proto '=https' --tlsv1.2 -sSf https://tiup-mirrors.pingcap.com/install.sh | sh - $HOME/.tiup/bin/tiup playground nightly --mode tikv-slim --kv 3 --pd 3 --without-monitor --kv.config tikv.toml & - curl --retry-connrefused --retry 5 --retry-delay 5 http://127.0.0.1:2379 - - name: Run Tests - run: | - export log_level=error - make failpoint-enable - go test ./tests/realtikvtest/${{ matrix.suite }} -v -with-real-tikv -timeout 30m diff --git a/DEPS.bzl b/DEPS.bzl index f1e7920961fc9..65f9d5b4b657c 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -927,6 +927,14 @@ def go_deps(): sum = "h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM=", version = "v0.0.4", ) + go_repository( + name = "com_github_golangci_gofmt", + build_file_proto_mode = "disable", + importpath = "github.com/golangci/gofmt", + sum = "h1:iR3fYXUjHCR97qWS8ch1y9zPNsgXThGwjKPrYfqMPks=", + version = "v0.0.0-20190930125516-244bba706f1a", + ) + go_repository( name = "com_github_golangci_prealloc", build_file_proto_mode = "disable", diff --git a/README.md b/README.md index ca96703510e40..36669b3de6682 100644 --- a/README.md +++ b/README.md @@ -62,10 +62,6 @@ The [community repository](https://github.com/pingcap/community) hosts all infor Contributions are welcomed and greatly appreciated. All the contributors are welcomed to claim your reward by filing this [form](https://forms.pingcap.com/f/tidb-contribution-swag). See [Contribution to TiDB](https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/introduction.html) for details on typical contribution workflows. For more contributing information, click on the contributor icon above. -## Adopters - -View the current list of in-production TiDB adopters [here](https://docs.pingcap.com/tidb/stable/adopters). - ## Case studies - [English](https://pingcap.com/case-studies) diff --git a/br/pkg/errors/BUILD.bazel b/br/pkg/errors/BUILD.bazel index d07b0c723102a..fd6f45603d35c 100644 --- a/br/pkg/errors/BUILD.bazel +++ b/br/pkg/errors/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "errors", @@ -7,3 +7,13 @@ go_library( visibility = ["//visibility:public"], deps = ["@com_github_pingcap_errors//:errors"], ) + +go_test( + name = "errors_test", + srcs = ["errors_test.go"], + deps = [ + ":errors", + "@com_github_pingcap_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) diff --git a/br/pkg/errors/errors.go b/br/pkg/errors/errors.go index 67f8a5c15d4ba..7585f7bc43673 100644 --- a/br/pkg/errors/errors.go +++ b/br/pkg/errors/errors.go @@ -3,6 +3,9 @@ package errors import ( + "context" + stderrors "errors" + "github.com/pingcap/errors" ) @@ -15,6 +18,17 @@ func Is(err error, is *errors.Error) bool { return errorFound != nil } +// IsContextCanceled checks whether the is caused by context.Canceled. +// errors.Cause does not work for the error wrapped by %w in fmt.Errorf. +// So we need to call stderrors.Is to unwrap the error. +func IsContextCanceled(err error) bool { + err = errors.Cause(err) + if err == context.Canceled || err == context.DeadlineExceeded { + return true + } + return stderrors.Is(err, context.Canceled) || stderrors.Is(err, context.DeadlineExceeded) +} + // BR errors. var ( ErrUnknown = errors.Normalize("internal error", errors.RFCCodeText("BR:Common:ErrUnknown")) diff --git a/br/pkg/errors/errors_test.go b/br/pkg/errors/errors_test.go new file mode 100644 index 0000000000000..a6f4c412280cc --- /dev/null +++ b/br/pkg/errors/errors_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. + +package errors_test + +import ( + "context" + "net/url" + "testing" + + "github.com/pingcap/errors" + berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/stretchr/testify/require" +) + +func TestIsContextCanceled(t *testing.T) { + require.False(t, berrors.IsContextCanceled(nil)) + require.False(t, berrors.IsContextCanceled(errors.New("connection closed"))) + require.True(t, berrors.IsContextCanceled(context.Canceled)) + require.True(t, berrors.IsContextCanceled(context.DeadlineExceeded)) + require.True(t, berrors.IsContextCanceled(errors.Trace(context.Canceled))) + require.True(t, berrors.IsContextCanceled(errors.Trace(context.DeadlineExceeded))) + require.True(t, berrors.IsContextCanceled(&url.Error{Err: context.Canceled})) + require.True(t, berrors.IsContextCanceled(&url.Error{Err: context.DeadlineExceeded})) +} diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 76bdba192bd99..6679d8bf93b98 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1392,8 +1392,32 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi return err } + if len(ranges) > 0 && local.pdCtl.CanPauseSchedulerByKeyRange() { + subCtx, cancel := context.WithCancel(ctx) + defer cancel() + + var startKey, endKey []byte + if len(ranges[0].start) > 0 { + startKey = codec.EncodeBytes(nil, ranges[0].start) + } + if len(ranges[len(ranges)-1].end) > 0 { + endKey = codec.EncodeBytes(nil, ranges[len(ranges)-1].end) + } + done, err := local.pdCtl.PauseSchedulersByKeyRange(subCtx, startKey, endKey) + if err != nil { + return errors.Trace(err) + } + defer func() { + cancel() + <-done + }() + } + log.FromContext(ctx).Info("start import engine", zap.Stringer("uuid", engineUUID), zap.Int("ranges", len(ranges)), zap.Int64("count", lfLength), zap.Int64("size", lfTotalSize)) + + failpoint.Inject("ReadyForImportEngine", func() {}) + for { unfinishedRanges := lf.unfinishedRanges(ranges) if len(unfinishedRanges) == 0 { @@ -1480,7 +1504,6 @@ func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, t logger.Warn("[resolve-dupe] skipping resolution due to selected algorithm. this table will become inconsistent!", zap.Stringer("algorithm", algorithm)) return nil case config.DupeResAlgRemove: - break default: panic(fmt.Sprintf("[resolve-dupe] unknown resolution algorithm %v", algorithm)) } diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index 9839e3592d195..1672b5f212436 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -19,7 +19,6 @@ import ( "context" "database/sql" "math" - "regexp" "runtime" "sort" "strings" @@ -332,14 +331,7 @@ func (local *local) SplitAndScatterRegionByRanges( } startTime := time.Now() - scatterCount := 0 - for _, region := range scatterRegions { - local.waitForScatterRegion(ctx, region) - if time.Since(startTime) > split.ScatterWaitUpperInterval { - break - } - scatterCount++ - } + scatterCount, err := local.waitForScatterRegions(ctx, scatterRegions) if scatterCount == len(scatterRegions) { log.FromContext(ctx).Info("waiting for scattering regions done", zap.Int("skipped_keys", skippedKeys), @@ -349,7 +341,8 @@ func (local *local) SplitAndScatterRegionByRanges( zap.Int("skipped_keys", skippedKeys), zap.Int("scatterCount", scatterCount), zap.Int("regions", len(scatterRegions)), - zap.Duration("take", time.Since(startTime))) + zap.Duration("take", time.Since(startTime)), + zap.Error(err)) } return nil } @@ -447,28 +440,38 @@ func (local *local) waitForSplit(ctx context.Context, regionID uint64) { } } -func (local *local) waitForScatterRegion(ctx context.Context, regionInfo *split.RegionInfo) { - for i := 0; i < split.ScatterWaitMaxRetryTimes; i++ { - ok, err := local.checkScatterRegionFinishedOrReScatter(ctx, regionInfo) - if ok { - return - } - if err != nil { - if !common.IsRetryableError(err) { - log.FromContext(ctx).Warn("wait for scatter region encountered non-retryable error", logutil.Region(regionInfo.Region), zap.Error(err)) - return +func (local *local) waitForScatterRegions(ctx context.Context, regions []*split.RegionInfo) (scatterCount int, _ error) { + subCtx, cancel := context.WithTimeout(ctx, split.ScatterWaitUpperInterval) + defer cancel() + + for len(regions) > 0 { + var retryRegions []*split.RegionInfo + for _, region := range regions { + scattered, err := local.checkRegionScatteredOrReScatter(subCtx, region) + if scattered { + scatterCount++ + continue } - log.FromContext(ctx).Warn("wait for scatter region encountered error, will retry again", logutil.Region(regionInfo.Region), zap.Error(err)) + if err != nil { + if !common.IsRetryableError(err) { + log.FromContext(ctx).Warn("wait for scatter region encountered non-retryable error", logutil.Region(region.Region), zap.Error(err)) + return scatterCount, err + } + log.FromContext(ctx).Warn("wait for scatter region encountered error, will retry again", logutil.Region(region.Region), zap.Error(err)) + } + retryRegions = append(retryRegions, region) } + regions = retryRegions select { case <-time.After(time.Second): - case <-ctx.Done(): + case <-subCtx.Done(): return } } + return scatterCount, nil } -func (local *local) checkScatterRegionFinishedOrReScatter(ctx context.Context, regionInfo *split.RegionInfo) (bool, error) { +func (local *local) checkRegionScatteredOrReScatter(ctx context.Context, regionInfo *split.RegionInfo) (bool, error) { resp, err := local.splitCli.GetOperator(ctx, regionInfo.Region.GetId()) if err != nil { return false, err @@ -478,13 +481,9 @@ func (local *local) checkScatterRegionFinishedOrReScatter(ctx context.Context, r if respErr.GetType() == pdpb.ErrorType_REGION_NOT_FOUND { return true, nil } - // don't return error if region replicate not complete - // TODO: should add a new error type to avoid this check by string matching - matches, _ := regexp.MatchString("region \\d+ is not fully replicated", respErr.Message) - if matches { - return false, nil - } - return false, errors.Errorf("get operator error: %s", respErr.GetType()) + return false, errors.Errorf( + "failed to get region operator, error type: %s, error message: %s", + respErr.GetType().String(), respErr.GetMessage()) } // If the current operator of the region is not 'scatter-region', we could assume // that 'scatter-operator' has finished. diff --git a/br/pkg/lightning/common/retry.go b/br/pkg/lightning/common/retry.go index accf7423414b4..3a7757d650738 100644 --- a/br/pkg/lightning/common/retry.go +++ b/br/pkg/lightning/common/retry.go @@ -20,6 +20,7 @@ import ( "io" "net" "os" + "regexp" "syscall" "github.com/go-sql-driver/mysql" @@ -30,6 +31,8 @@ import ( "google.golang.org/grpc/status" ) +var regionNotFullyReplicatedRe = regexp.MustCompile(`region \d+ is not fully replicated`) + // IsRetryableError returns whether the error is transient (e.g. network // connection dropped) or irrecoverable (e.g. user pressing Ctrl+C). This // function returns `false` (irrecoverable) if `err == nil`. @@ -88,6 +91,9 @@ func isSingleRetryableError(err error) bool { } return false default: + if regionNotFullyReplicatedRe.MatchString(err.Error()) { + return true + } switch status.Code(err) { case codes.DeadlineExceeded, codes.NotFound, codes.AlreadyExists, codes.PermissionDenied, codes.ResourceExhausted, codes.Aborted, codes.OutOfRange, codes.Unavailable, codes.DataLoss: return true diff --git a/br/pkg/lightning/common/retry_test.go b/br/pkg/lightning/common/retry_test.go index 670004260f5a1..b707a8d1d1c5f 100644 --- a/br/pkg/lightning/common/retry_test.go +++ b/br/pkg/lightning/common/retry_test.go @@ -94,4 +94,6 @@ func TestIsRetryableError(t *testing.T) { require.False(t, IsRetryableError(multierr.Combine(context.Canceled, context.Canceled))) require.True(t, IsRetryableError(multierr.Combine(&net.DNSError{IsTimeout: true}, &net.DNSError{IsTimeout: true}))) require.False(t, IsRetryableError(multierr.Combine(context.Canceled, &net.DNSError{IsTimeout: true}))) + + require.True(t, IsRetryableError(errors.Errorf("region %d is not fully replicated", 1234))) } diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index b0ffe32fa3cd5..0066895568550 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -1156,7 +1156,7 @@ func (cfg *Config) CheckAndAdjustSecurity() error { return common.ErrInvalidConfig.GenWithStack("cannot set `tidb.tls` to 'cluster' without a [security] section") } case "false", "skip-verify", "preferred": - break + return nil default: return common.ErrInvalidConfig.GenWithStack("unsupported `tidb.tls` config %s", cfg.TiDB.TLS) } diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index 111b7c93b59b4..6e0f63f4df463 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -209,6 +209,14 @@ func (l *Lightning) goServe(statusAddr string, realAddrWriter io.Writer) error { mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + // Enable failpoint http API for testing. + failpoint.Inject("EnableTestAPI", func() { + mux.HandleFunc("/fail/", func(w http.ResponseWriter, r *http.Request) { + r.URL.Path = strings.TrimPrefix(r.URL.Path, "/fail") + new(failpoint.HttpHandler).ServeHTTP(w, r) + }) + }) + handleTasks := http.StripPrefix("/tasks", http.HandlerFunc(l.handleTask)) mux.Handle("/tasks", httpHandleWrapper(handleTasks.ServeHTTP)) mux.Handle("/tasks/", httpHandleWrapper(handleTasks.ServeHTTP)) diff --git a/br/pkg/lightning/mydump/loader.go b/br/pkg/lightning/mydump/loader.go index 30f4f14c1464c..40091c61b2d03 100644 --- a/br/pkg/lightning/mydump/loader.go +++ b/br/pkg/lightning/mydump/loader.go @@ -39,21 +39,19 @@ type MDDatabaseMeta struct { } func (m *MDDatabaseMeta) GetSchema(ctx context.Context, store storage.ExternalStorage) string { - schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) - if err != nil { - log.FromContext(ctx).Warn("failed to extract table schema", - zap.String("Path", m.SchemaFile.FileMeta.Path), - log.ShortError(err), - ) - schema = nil - } - schemaStr := strings.TrimSpace(string(schema)) - // set default if schema sql is empty - if len(schemaStr) == 0 { - schemaStr = "CREATE DATABASE IF NOT EXISTS " + common.EscapeIdentifier(m.Name) + if m.SchemaFile.FileMeta.Path != "" { + schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) + if err != nil { + log.FromContext(ctx).Warn("failed to extract table schema", + zap.String("Path", m.SchemaFile.FileMeta.Path), + log.ShortError(err), + ) + } else if schemaStr := strings.TrimSpace(string(schema)); schemaStr != "" { + return schemaStr + } } - - return schemaStr + // set default if schema sql is empty or failed to extract. + return "CREATE DATABASE IF NOT EXISTS " + common.EscapeIdentifier(m.Name) } type MDTableMeta struct { diff --git a/br/pkg/lightning/mydump/loader_test.go b/br/pkg/lightning/mydump/loader_test.go index 0f7079a1f95df..e256a685f20ee 100644 --- a/br/pkg/lightning/mydump/loader_test.go +++ b/br/pkg/lightning/mydump/loader_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" md "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/storage" filter "github.com/pingcap/tidb/util/table-filter" @@ -181,13 +182,16 @@ func TestTableInfoNotFound(t *testing.T) { loader, err := md.NewMyDumpLoader(ctx, s.cfg) require.NoError(t, err) for _, dbMeta := range loader.GetDatabases() { - dbSQL := dbMeta.GetSchema(ctx, store) + logger, buffer := log.MakeTestLogger() + logCtx := log.NewContext(ctx, logger) + dbSQL := dbMeta.GetSchema(logCtx, store) require.Equal(t, "CREATE DATABASE IF NOT EXISTS `db`", dbSQL) for _, tblMeta := range dbMeta.Tables { - sql, err := tblMeta.GetSchema(ctx, store) + sql, err := tblMeta.GetSchema(logCtx, store) require.Equal(t, "", sql) require.NoError(t, err) } + require.NotContains(t, buffer.Stripped(), "failed to extract table schema") } } diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 0af04e69feedb..b94bde8208be6 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -555,6 +555,8 @@ type taskMetaMgr interface { // need to update or any new tasks. There is at most one lightning who can execute the action function at the same time. // Note that action may be executed multiple times due to transaction retry, caller should make sure it's idempotent. CheckTasksExclusively(ctx context.Context, action func(tasks []taskMeta) ([]taskMeta, error)) error + // CanPauseSchedulerByKeyRange returns whether the scheduler can pause by the key range. + CanPauseSchedulerByKeyRange() bool CheckAndPausePdSchedulers(ctx context.Context) (pdutil.UndoFunc, error) // CheckAndFinishRestore check task meta and return whether to switch cluster to normal state and clean up the metadata // Return values: first boolean indicates whether switch back tidb cluster to normal state (restore schedulers, switch tikv to normal) @@ -867,6 +869,10 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U }, nil } +func (m *dbTaskMetaMgr) CanPauseSchedulerByKeyRange() bool { + return m.pd.CanPauseSchedulerByKeyRange() +} + // CheckAndFinishRestore check task meta and return whether to switch cluster to normal state and clean up the metadata // Return values: first boolean indicates whether switch back tidb cluster to normal state (restore schedulers, switch tikv to normal) // the second boolean indicates whether to clean up the metadata in tidb @@ -1058,6 +1064,10 @@ func (m noopTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil. }, nil } +func (m noopTaskMetaMgr) CanPauseSchedulerByKeyRange() bool { + return false +} + func (m noopTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) { return true, nil } @@ -1168,6 +1178,10 @@ func (m *singleTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdut return m.pd.RemoveSchedulers(ctx) } +func (m *singleTaskMetaMgr) CanPauseSchedulerByKeyRange() bool { + return m.pd.CanPauseSchedulerByKeyRange() +} + func (m *singleTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) { return m.initialized, nil } diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 31a48c620846a..faf774efa5c7a 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -916,7 +916,6 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string switch { case err == nil: - break case utils.MessageIsRetryableStorageError(err.Error()), common.IsContextCanceledError(err): // recoverable error, should not be recorded in checkpoint // which will prevent lightning from automatically recovering @@ -1370,42 +1369,51 @@ func (rc *Controller) restoreTables(ctx context.Context) (finalErr error) { // make split region and ingest sst more stable // because importer backend is mostly use for v3.x cluster which doesn't support these api, // so we also don't do this for import backend - finishSchedulers := func() {} + finishSchedulers := func() { + if rc.taskMgr != nil { + rc.taskMgr.Close() + } + } // if one lightning failed abnormally, and can't determine whether it needs to switch back, // we do not do switch back automatically switchBack := false cleanup := false postProgress := func() error { return nil } if rc.cfg.TikvImporter.Backend == config.BackendLocal { + var restoreFn pdutil.UndoFunc - logTask.Info("removing PD leader®ion schedulers") + if !rc.taskMgr.CanPauseSchedulerByKeyRange() { + logTask.Info("removing PD leader®ion schedulers") - restoreFn, err := rc.taskMgr.CheckAndPausePdSchedulers(ctx) - if err != nil { - return errors.Trace(err) + var err error + restoreFn, err = rc.taskMgr.CheckAndPausePdSchedulers(ctx) + if err != nil { + return errors.Trace(err) + } } finishSchedulers = func() { - if restoreFn != nil { - taskFinished := finalErr == nil - // use context.Background to make sure this restore function can still be executed even if ctx is canceled - restoreCtx := context.Background() - needSwitchBack, needCleanup, err := rc.taskMgr.CheckAndFinishRestore(restoreCtx, taskFinished) - if err != nil { - logTask.Warn("check restore pd schedulers failed", zap.Error(err)) - return - } - switchBack = needSwitchBack - if needSwitchBack { - logTask.Info("add back PD leader®ion schedulers") - if restoreE := restoreFn(restoreCtx); restoreE != nil { - logTask.Warn("failed to restore removed schedulers, you may need to restore them manually", zap.Error(restoreE)) - } + taskFinished := finalErr == nil + // use context.Background to make sure this restore function can still be executed even if ctx is canceled + restoreCtx := context.Background() + needSwitchBack, needCleanup, err := rc.taskMgr.CheckAndFinishRestore(restoreCtx, taskFinished) + if err != nil { + logTask.Warn("check restore pd schedulers failed", zap.Error(err)) + return + } + switchBack = needSwitchBack + cleanup = needCleanup + + if needSwitchBack && restoreFn != nil { + logTask.Info("add back PD leader®ion schedulers") + if restoreE := restoreFn(restoreCtx); restoreE != nil { + logTask.Warn("failed to restore removed schedulers, you may need to restore them manually", zap.Error(restoreE)) } - cleanup = needCleanup } - rc.taskMgr.Close() + if rc.taskMgr != nil { + rc.taskMgr.Close() + } } } diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index 9cf278a67d1cc..f44691b601df3 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -195,7 +195,7 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string for _, stmt := range stmts { switch node := stmt.(type) { case *ast.CreateDatabaseStmt: - node.Name = dbName + node.Name = model.NewCIStr(dbName) node.IfNotExists = true case *ast.CreateTableStmt: node.Table.Schema = model.NewCIStr(dbName) diff --git a/br/pkg/pdutil/BUILD.bazel b/br/pkg/pdutil/BUILD.bazel index f2e997d3d833c..128b4b6b31cda 100644 --- a/br/pkg/pdutil/BUILD.bazel +++ b/br/pkg/pdutil/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//util/codec", "@com_github_coreos_go_semver//semver", "@com_github_docker_go_units//:go-units", + "@com_github_google_uuid//:uuid", "@com_github_opentracing_opentracing_go//:opentracing-go", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go old mode 100644 new mode 100755 index 599eefe30afb5..1c2d8dd6754bc --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -6,6 +6,7 @@ import ( "bytes" "context" "crypto/tls" + "encoding/hex" "encoding/json" "fmt" "io" @@ -17,6 +18,7 @@ import ( "github.com/coreos/go-semver/semver" "github.com/docker/go-units" + "github.com/google/uuid" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -36,6 +38,7 @@ const ( regionCountPrefix = "pd/api/v1/stats/region" storePrefix = "pd/api/v1/store" schedulerPrefix = "pd/api/v1/schedulers" + regionLabelPrefix = "pd/api/v1/config/region-label/rule" maxMsgSize = int(128 * units.MiB) // pd.ScanRegion may return a large response scheduleConfigPrefix = "pd/api/v1/config/schedule" configPrefix = "pd/api/v1/config" @@ -94,6 +97,9 @@ var ( // see https://github.com/tikv/pd/pull/3088 pauseConfigVersion = semver.Version{Major: 4, Minor: 0, Patch: 8} + // After v6.1.0 version, we can pause schedulers by key range with TTL. + minVersionForRegionLabelTTL = semver.Version{Major: 6, Minor: 1, Patch: 0} + // Schedulers represent region/leader schedulers which can impact on performance. Schedulers = map[string]struct{}{ "balance-leader-scheduler": {}, @@ -130,9 +136,9 @@ var ( ) // pdHTTPRequest defines the interface to send a request to pd and return the result in bytes. -type pdHTTPRequest func(context.Context, string, string, *http.Client, string, io.Reader) ([]byte, error) +type pdHTTPRequest func(ctx context.Context, addr string, prefix string, cli *http.Client, method string, body io.Reader) ([]byte, error) -// pdRequest is a func to send a HTTP to pd and return the result bytes. +// pdRequest is a func to send an HTTP to pd and return the result bytes. func pdRequest( ctx context.Context, addr string, prefix string, @@ -709,6 +715,142 @@ func (p *PdController) doRemoveSchedulersWith( return removedSchedulers, err } +// RegionLabel is the label of a region. This struct is partially copied from +// https://github.com/tikv/pd/blob/783d060861cef37c38cbdcab9777fe95c17907fe/server/schedule/labeler/rules.go#L31. +type RegionLabel struct { + Key string `json:"key"` + Value string `json:"value"` + TTL string `json:"ttl,omitempty"` + StartAt string `json:"start_at,omitempty"` +} + +// LabelRule is the rule to assign labels to a region. This struct is partially copied from +// https://github.com/tikv/pd/blob/783d060861cef37c38cbdcab9777fe95c17907fe/server/schedule/labeler/rules.go#L41. +type LabelRule struct { + ID string `json:"id"` + Labels []RegionLabel `json:"labels"` + RuleType string `json:"rule_type"` + Data interface{} `json:"data"` +} + +// KeyRangeRule contains the start key and end key of the LabelRule. This struct is partially copied from +// https://github.com/tikv/pd/blob/783d060861cef37c38cbdcab9777fe95c17907fe/server/schedule/labeler/rules.go#L62. +type KeyRangeRule struct { + StartKeyHex string `json:"start_key"` // hex format start key, for marshal/unmarshal + EndKeyHex string `json:"end_key"` // hex format end key, for marshal/unmarshal +} + +// CreateOrUpdateRegionLabelRule creates or updates a region label rule. +func (p *PdController) CreateOrUpdateRegionLabelRule(ctx context.Context, rule LabelRule) error { + reqData, err := json.Marshal(&rule) + if err != nil { + panic(err) + } + var lastErr error + for i, addr := range p.addrs { + _, lastErr = pdRequest(ctx, addr, regionLabelPrefix, + p.cli, http.MethodPost, bytes.NewBuffer(reqData)) + if lastErr == nil { + return nil + } + if berrors.IsContextCanceled(lastErr) { + return errors.Trace(lastErr) + } + + if i < len(p.addrs) { + log.Warn("failed to create or update region label rule, will try next pd address", + zap.Error(lastErr), zap.String("pdAddr", addr)) + } + } + return errors.Trace(lastErr) +} + +func (p *PdController) DeleteRegionLabelRule(ctx context.Context, ruleID string) error { + var lastErr error + for i, addr := range p.addrs { + _, lastErr = pdRequest(ctx, addr, fmt.Sprintf("%s/%s", regionLabelPrefix, ruleID), + p.cli, http.MethodDelete, nil) + if lastErr == nil { + return nil + } + if berrors.IsContextCanceled(lastErr) { + return errors.Trace(lastErr) + } + + if i < len(p.addrs) { + log.Warn("failed to delete region label rule, will try next pd address", + zap.Error(lastErr), zap.String("pdAddr", addr)) + } + } + return errors.Trace(lastErr) +} + +// PauseSchedulersByKeyRange will pause schedulers for regions in the specific key range. +// This function will spawn a goroutine to keep pausing schedulers periodically until the context is done. +// The return done channel is used to notify the caller that the background goroutine is exited. +func (p *PdController) PauseSchedulersByKeyRange(ctx context.Context, startKey, endKey []byte) (done <-chan struct{}, err error) { + return p.pauseSchedulerByKeyRangeWithTTL(ctx, startKey, endKey, pauseTimeout) +} + +func (p *PdController) pauseSchedulerByKeyRangeWithTTL(ctx context.Context, startKey, endKey []byte, ttl time.Duration) (_done <-chan struct{}, err error) { + rule := LabelRule{ + ID: uuid.New().String(), + Labels: []RegionLabel{{ + Key: "schedule", + Value: "deny", + TTL: ttl.String(), + }}, + RuleType: "key-range", + // Data should be a list of KeyRangeRule when rule type is key-range. + // See https://github.com/tikv/pd/blob/783d060861cef37c38cbdcab9777fe95c17907fe/server/schedule/labeler/rules.go#L169. + Data: []KeyRangeRule{{ + StartKeyHex: hex.EncodeToString(startKey), + EndKeyHex: hex.EncodeToString(endKey), + }}, + } + done := make(chan struct{}) + if err := p.CreateOrUpdateRegionLabelRule(ctx, rule); err != nil { + close(done) + return nil, errors.Trace(err) + } + + go func() { + defer close(done) + ticker := time.NewTicker(ttl / 3) + defer ticker.Stop() + loop: + for { + select { + case <-ticker.C: + if err := p.CreateOrUpdateRegionLabelRule(ctx, rule); err != nil { + if berrors.IsContextCanceled(err) { + break loop + } + log.Warn("pause scheduler by key range failed, ignore it and wait next time pause", zap.Error(err)) + } + case <-ctx.Done(): + break loop + } + } + // Use a new context to avoid the context is canceled by the caller. + recoverCtx, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + // Set ttl to 0 to remove the rule. + rule.Labels[0].TTL = time.Duration(0).String() + if err := p.DeleteRegionLabelRule(recoverCtx, rule.ID); err != nil { + log.Warn("failed to delete region label rule, the rule will be removed after ttl expires", + zap.String("rule-id", rule.ID), zap.Duration("ttl", ttl), zap.Error(err)) + } + }() + return done, nil +} + +// CanPauseSchedulerByKeyRange returns whether the scheduler can be paused by key range. +func (p *PdController) CanPauseSchedulerByKeyRange() bool { + // We need ttl feature to ensure scheduler can recover from pause automatically. + return p.version.Compare(minVersionForRegionLabelTTL) >= 0 +} + // Close close the connection to pd. func (p *PdController) Close() { p.pdClient.Close() diff --git a/br/pkg/pdutil/pd_serial_test.go b/br/pkg/pdutil/pd_serial_test.go index 05f0d34aa2ef2..608830fe190fe 100644 --- a/br/pkg/pdutil/pd_serial_test.go +++ b/br/pkg/pdutil/pd_serial_test.go @@ -12,7 +12,9 @@ import ( "net/http" "net/http/httptest" "net/url" + "strings" "testing" + "time" "github.com/coreos/go-semver/semver" "github.com/pingcap/failpoint" @@ -231,3 +233,47 @@ func TestStoreInfo(t *testing.T) { require.Equal(t, "Tombstone", resp.Store.StateName) require.Equal(t, uint64(1024), uint64(resp.Status.Available)) } + +func TestPauseSchedulersByKeyRange(t *testing.T) { + const ttl = time.Second + + labelExpires := make(map[string]time.Time) + + httpSrv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.Method == http.MethodDelete { + ruleID := strings.TrimPrefix(r.URL.Path, "/"+regionLabelPrefix+"/") + print(ruleID) + delete(labelExpires, ruleID) + return + } + var labelRule LabelRule + err := json.NewDecoder(r.Body).Decode(&labelRule) + require.NoError(t, err) + require.Len(t, labelRule.Labels, 1) + regionLabel := labelRule.Labels[0] + require.Equal(t, "schedule", regionLabel.Key) + require.Equal(t, "deny", regionLabel.Value) + reqTTL, err := time.ParseDuration(regionLabel.TTL) + require.NoError(t, err) + if reqTTL == 0 { + delete(labelExpires, labelRule.ID) + } else { + require.Equal(t, ttl, reqTTL) + if expire, ok := labelExpires[labelRule.ID]; ok { + require.True(t, expire.After(time.Now()), "should not expire before now") + } + labelExpires[labelRule.ID] = time.Now().Add(ttl) + } + })) + defer httpSrv.Close() + + pdController := &PdController{addrs: []string{httpSrv.URL}, cli: http.DefaultClient} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + done, err := pdController.pauseSchedulerByKeyRangeWithTTL(ctx, []byte{0, 0, 0, 0}, []byte{0xff, 0xff, 0xff, 0xff}, ttl) + require.NoError(t, err) + time.Sleep(ttl * 3) + cancel() + <-done + require.Len(t, labelExpires, 0) +} diff --git a/br/tests/lightning_disable_scheduler_by_key_range/config.toml b/br/tests/lightning_disable_scheduler_by_key_range/config.toml new file mode 100644 index 0000000000000..dce628f6a61c4 --- /dev/null +++ b/br/tests/lightning_disable_scheduler_by_key_range/config.toml @@ -0,0 +1,2 @@ +[lightning] +status-addr = ":8289" diff --git a/br/tests/lightning_disable_scheduler_by_key_range/data/test-schema-create.sql b/br/tests/lightning_disable_scheduler_by_key_range/data/test-schema-create.sql new file mode 100644 index 0000000000000..14379bd68472a --- /dev/null +++ b/br/tests/lightning_disable_scheduler_by_key_range/data/test-schema-create.sql @@ -0,0 +1 @@ +CREATE DATABASE test; diff --git a/br/tests/lightning_disable_scheduler_by_key_range/data/test.t-schema.sql b/br/tests/lightning_disable_scheduler_by_key_range/data/test.t-schema.sql new file mode 100644 index 0000000000000..57a1b65732950 --- /dev/null +++ b/br/tests/lightning_disable_scheduler_by_key_range/data/test.t-schema.sql @@ -0,0 +1 @@ +CREATE TABLE t(a INT PRIMARY KEY, b int); diff --git a/br/tests/lightning_disable_scheduler_by_key_range/data/test.t.sql b/br/tests/lightning_disable_scheduler_by_key_range/data/test.t.sql new file mode 100644 index 0000000000000..30e06b42e169b --- /dev/null +++ b/br/tests/lightning_disable_scheduler_by_key_range/data/test.t.sql @@ -0,0 +1 @@ +INSERT INTO t VALUES (1,1); diff --git a/br/tests/lightning_disable_scheduler_by_key_range/run.sh b/br/tests/lightning_disable_scheduler_by_key_range/run.sh new file mode 100644 index 0000000000000..9df6067c8baf8 --- /dev/null +++ b/br/tests/lightning_disable_scheduler_by_key_range/run.sh @@ -0,0 +1,65 @@ +#!/bin/bash +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eux + +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/lightning/EnableTestAPI=return" +export GO_FAILPOINTS="${GO_FAILPOINTS};github.com/pingcap/tidb/br/pkg/lightning/backend/local/ReadyForImportEngine=sleep(10000)" + +run_lightning --backend='local' & +shpid="$!" +pid= + +ensure_lightning_is_started() { + for _ in {0..60}; do + pid=$(pstree -p "$shpid" | grep -Eo "tidb-lightning\.\([0-9]*\)" | grep -Eo "[0-9]*") || true + [ -n "$pid" ] && break + sleep 1 + done + if [ -z "$pid" ]; then + echo "lightning doesn't start successfully, please check the log" >&2 + exit 1 + fi + echo "lightning is started, pid is $pid" +} + +ready_for_import_engine() { + for _ in {0..60}; do + grep -Fq "start import engine" "$TEST_DIR"/lightning.log && return + sleep 1 + done + echo "lightning doesn't start import engine, please check the log" >&2 + exit 1 +} + +ensure_lightning_is_started +ready_for_import_engine + +run_curl "https://${PD_ADDR}/pd/api/v1/config/cluster-version" + +length=$(run_curl "https://${PD_ADDR}/pd/api/v1/config/region-label/rules" | jq 'select(.[].rule_type == "key-range") | length') +if [ "$length" != "1" ]; then + echo "region-label key-range rules should be 1, but got $length" >&2 + exit 1 +fi + +wait "$shpid" + +length=$(run_curl "https://${PD_ADDR}/pd/api/v1/config/region-label/rules" | jq 'select(.[].rule_type == "key-range") | length') +if [ -n "$length" ] && [ "$length" -ne 0 ]; then + echo "region-label key-range rules should be 0, but got $length" >&2 + exit 1 +fi diff --git a/br/tests/lightning_incremental/run.sh b/br/tests/lightning_incremental/run.sh index 4cdd5a53ec74b..a025e7bebc6f1 100644 --- a/br/tests/lightning_incremental/run.sh +++ b/br/tests/lightning_incremental/run.sh @@ -25,8 +25,6 @@ run_lightning_and_check_meta() { check_not_contains "Database: lightning_metadata" } -DB_NAME=incr - run_sql "DROP DATABASE IF EXISTS incr;" run_sql "DROP DATABASE IF EXISTS lightning_metadata;" run_lightning_and_check_meta @@ -48,8 +46,8 @@ for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do done for tbl in pk_auto_inc rowid_uk_inc; do - run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" - check_contains "group_concat(v): a,b,c" + run_sql "SELECT group_concat(v order by v) as result from incr.$tbl group by 'all';" + check_contains "result: a,b,c" done run_sql "SELECT sum(pk) from incr.uk_auto_inc;" @@ -75,8 +73,8 @@ for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do done for tbl in pk_auto_inc rowid_uk_inc; do - run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" - check_contains "group_concat(v): a,b,c,d,e,f" + run_sql "SELECT group_concat(v order by v) as result from incr.$tbl group by 'all';" + check_contains "result: a,b,c,d,e,f" done run_sql "SELECT sum(pk) from incr.uk_auto_inc;" diff --git a/build/BUILD.bazel b/build/BUILD.bazel index 4264907bb1fb9..b283c4b2e0f20 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -4,17 +4,41 @@ load("@io_bazel_rules_go//go:def.bzl", "nogo") load("//build/linter/staticcheck:def.bzl", "staticcheck_analyzers") STATICHECK_ANALYZERS = [ + "S1000", + "S1001", "S1002", + "S1003", "S1004", + "S1005", + "S1006", "S1007", + "S1008", "S1009", "S1010", + "S1011", "S1012", + "S1016", + "S1017", + "S1018", "S1019", "S1020", "S1021", + "S1023", "S1024", + "S1025", + "S1028", + "S1029", "S1030", + "S1031", + "S1032", + "S1033", + "S1034", + "S1035", + "S1036", + "S1037", + "S1038", + "S1039", + "S1040", "SA2000", "SA2001", "SA2003", @@ -83,6 +107,7 @@ nogo( "@org_golang_x_tools//go/analysis/passes/unusedresult:go_default_library", "//build/linter/durationcheck:durationcheck", "//build/linter/exportloopref:exportloopref", + "//build/linter/gofmt:gofmt", "//build/linter/ineffassign:ineffassign", "//build/linter/prealloc:prealloc", ] + staticcheck_analyzers(STATICHECK_ANALYZERS), diff --git a/build/linter/gofmt/BUILD.bazel b/build/linter/gofmt/BUILD.bazel new file mode 100644 index 0000000000000..c21e7e7e9fe40 --- /dev/null +++ b/build/linter/gofmt/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "gofmt", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/gofmt", + visibility = ["//visibility:public"], + deps = [ + "@com_github_golangci_gofmt//gofmt", + "@org_golang_x_tools//go/analysis", + ], +) diff --git a/build/linter/gofmt/analyzer.go b/build/linter/gofmt/analyzer.go new file mode 100644 index 0000000000000..05c266b0bbe20 --- /dev/null +++ b/build/linter/gofmt/analyzer.go @@ -0,0 +1,65 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gofmt + +import ( + "fmt" + "strings" + + "github.com/golangci/gofmt/gofmt" + "golang.org/x/tools/go/analysis" +) + +// Analyzer is the analyzer struct of gofmt. +var Analyzer = &analysis.Analyzer{ + Name: "gofmt", + Doc: "gofmt checks whether code was gofmt-ed" + + "this tool runs with -s option to check for code simplification", + Run: run, +} + +var needSimplify bool + +func init() { + Analyzer.Flags.BoolVar(&needSimplify, "need-simplify", true, "run gofmt with -s for code simplification") +} + +func run(pass *analysis.Pass) (any, error) { + fileNames := make([]string, 0, 10) + for _, f := range pass.Files { + pos := pass.Fset.PositionFor(f.Pos(), false) + if pos.Filename != "" && !strings.HasSuffix(pos.Filename, "failpoint_binding__.go") { + fileNames = append(fileNames, pos.Filename) + } + } + + for _, f := range fileNames { + diff, err := gofmt.Run(f, needSimplify) + if err != nil { + return nil, fmt.Errorf("could not run gofmt: %w (%s)", err, f) + } + + if diff == nil { + continue + } + + pass.Report(analysis.Diagnostic{ + Pos: 1, + Message: fmt.Sprintf("\n%s", diff), + }) + } + + return nil, nil +} diff --git a/build/nogo_config.json b/build/nogo_config.json index d0d99435f146b..8f7f4e940a597 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -104,6 +104,16 @@ ".*_generated\\.go$": "ignore generated code" } }, + "gofmt": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "/cgo/": "ignore cgo code", + "/rules_go_work-*": "ignore generated code", + ".*test_/testmain\\.go$": "ignore generated code", + ".*failpoint_binding__.go$": "ignore generated code" + } + }, "httpresponse": { "exclude_files": { "/external/": "no need to vet third party code", @@ -220,24 +230,60 @@ "parser/digester_test.go": "ignore code" } }, + "S1000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1002": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1003": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1004": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1005": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1006": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1007": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1008": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1009": { "exclude_files": { "/external/": "no need to vet third party code", @@ -250,12 +296,54 @@ ".*_generated\\.go$": "ignore generated code" } }, + "S1011": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1012": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1013": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1014": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1015": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1016": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1017": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1018": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1019": { "exclude_files": { "/external/": "no need to vet third party code", @@ -276,18 +364,122 @@ "tools/check/ut.go": "ignore code" } }, + "S1022": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1023": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/parser.go": "ignore code" + } + }, "S1024": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1025": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1026": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1027": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1028": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1029": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1030": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1031": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1032": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1033": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1034": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1035": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1036": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1037": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1038": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1039": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1040": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/parser.go": "ignore generated code" + } + }, "SA2000": { "exclude_files": { "/external/": "no need to vet third party code", diff --git a/cmd/explaintest/r/collation_agg_func_disabled.result b/cmd/explaintest/r/collation_agg_func_disabled.result index 75ba58783482b..f0297f70b094c 100644 --- a/cmd/explaintest/r/collation_agg_func_disabled.result +++ b/cmd/explaintest/r/collation_agg_func_disabled.result @@ -200,11 +200,9 @@ select min(b) from tt; min(b) B desc format='brief' select min(b collate utf8mb4_bin) from tt; -id estRows task access object operator info -StreamAgg 1.00 root funcs:min(Column#8)->Column#6 -└─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:min(cast(collation_agg_func.tt.b, enum('a','B','c')))->Column#8 - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select min(b collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select max(b) from tt; id estRows task access object operator info StreamAgg 1.00 root funcs:max(Column#8)->Column#6 @@ -215,11 +213,9 @@ select max(b) from tt; max(b) c desc format='brief' select max(b collate utf8mb4_bin) from tt; -id estRows task access object operator info -StreamAgg 1.00 root funcs:max(Column#8)->Column#6 -└─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:max(cast(collation_agg_func.tt.b, enum('a','B','c')))->Column#8 - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select max(b collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select min(c) from tt; id estRows task access object operator info HashAgg 1.00 root funcs:min(collation_agg_func.tt.c)->Column#6 @@ -229,11 +225,9 @@ select min(c) from tt; min(c) B desc format='brief' select min(c collate utf8mb4_bin) from tt; -id estRows task access object operator info -HashAgg 1.00 root funcs:min(Column#7)->Column#6 -└─Projection 10000.00 root cast(collation_agg_func.tt.c, set('a','B','c'))->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select min(c collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select max(c) from tt; id estRows task access object operator info HashAgg 1.00 root funcs:max(collation_agg_func.tt.c)->Column#6 @@ -243,11 +237,9 @@ select max(c) from tt; max(c) c desc format='brief' select max(c collate utf8mb4_bin) from tt; -id estRows task access object operator info -HashAgg 1.00 root funcs:max(Column#7)->Column#6 -└─Projection 10000.00 root cast(collation_agg_func.tt.c, set('a','B','c'))->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select max(c collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select min(d) from tt; id estRows task access object operator info StreamAgg 1.00 root funcs:min(collation_agg_func.tt.d)->Column#6 diff --git a/cmd/explaintest/r/collation_agg_func_enabled.result b/cmd/explaintest/r/collation_agg_func_enabled.result index ebc4f51ad36ce..e40627439cd88 100644 --- a/cmd/explaintest/r/collation_agg_func_enabled.result +++ b/cmd/explaintest/r/collation_agg_func_enabled.result @@ -197,11 +197,9 @@ select min(b) from tt; min(b) a desc format='brief' select min(b collate utf8mb4_bin) from tt; -id estRows task access object operator info -StreamAgg 1.00 root funcs:min(Column#8)->Column#6 -└─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:min(cast(collation_agg_func.tt.b, enum('a','B','c')))->Column#8 - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select min(b collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select max(b) from tt; id estRows task access object operator info StreamAgg 1.00 root funcs:max(Column#8)->Column#6 @@ -212,11 +210,9 @@ select max(b) from tt; max(b) c desc format='brief' select max(b collate utf8mb4_bin) from tt; -id estRows task access object operator info -StreamAgg 1.00 root funcs:max(Column#8)->Column#6 -└─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:max(cast(collation_agg_func.tt.b, enum('a','B','c')))->Column#8 - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select max(b collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select min(c) from tt; id estRows task access object operator info HashAgg 1.00 root funcs:min(collation_agg_func.tt.c)->Column#6 @@ -226,11 +222,9 @@ select min(c) from tt; min(c) a desc format='brief' select min(c collate utf8mb4_bin) from tt; -id estRows task access object operator info -HashAgg 1.00 root funcs:min(Column#7)->Column#6 -└─Projection 10000.00 root cast(collation_agg_func.tt.c, set('a','B','c'))->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select min(c collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select max(c) from tt; id estRows task access object operator info HashAgg 1.00 root funcs:max(collation_agg_func.tt.c)->Column#6 @@ -240,11 +234,9 @@ select max(c) from tt; max(c) c desc format='brief' select max(c collate utf8mb4_bin) from tt; -id estRows task access object operator info -HashAgg 1.00 root funcs:max(Column#7)->Column#6 -└─Projection 10000.00 root cast(collation_agg_func.tt.c, set('a','B','c'))->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' +select max(c collate utf8mb4_bin) from tt; +Error 1235: This version of TiDB doesn't yet support 'use collate clause for enum or set' desc format='brief' select min(d) from tt; id estRows task access object operator info StreamAgg 1.00 root funcs:min(collation_agg_func.tt.d)->Column#6 diff --git a/cmd/explaintest/t/collation_agg_func.test b/cmd/explaintest/t/collation_agg_func.test index eb7ada2209981..160116ac06c96 100644 --- a/cmd/explaintest/t/collation_agg_func.test +++ b/cmd/explaintest/t/collation_agg_func.test @@ -66,24 +66,28 @@ desc format='brief' select max(a collate utf8mb4_bin) from tt; select max(a collate utf8mb4_bin) from tt; desc format='brief' select min(b) from tt; select min(b) from tt; +--error 1235 desc format='brief' select min(b collate utf8mb4_bin) from tt; -# Fix me later. -# select min(b collate utf8mb4_bin) from tt; +--error 1235 +select min(b collate utf8mb4_bin) from tt; desc format='brief' select max(b) from tt; select max(b) from tt; +--error 1235 desc format='brief' select max(b collate utf8mb4_bin) from tt; -# Fix me later. -# select max(b collate utf8mb4_bin) from tt; +--error 1235 +select max(b collate utf8mb4_bin) from tt; desc format='brief' select min(c) from tt; select min(c) from tt; +--error 1235 desc format='brief' select min(c collate utf8mb4_bin) from tt; -# Fix me later. -# select min(c collate utf8mb4_bin) from tt; +--error 1235 +select min(c collate utf8mb4_bin) from tt; desc format='brief' select max(c) from tt; select max(c) from tt; +--error 1235 desc format='brief' select max(c collate utf8mb4_bin) from tt; -# Fix me later. -# select max(c collate utf8mb4_bin) from tt; +--error 1235 +select max(c collate utf8mb4_bin) from tt; desc format='brief' select min(d) from tt; select min(d) from tt; --error 1253 diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 324d39bc469ec..807f65715202b 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -143,6 +143,7 @@ go_test( "integration_test.go", "main_test.go", "modify_column_test.go", + "multi_schema_change_test.go", "options_test.go", "partition_test.go", "placement_policy_ddl_test.go", diff --git a/ddl/cancel_test.go b/ddl/cancel_test.go index cca3047083a99..9c53952534bb3 100644 --- a/ddl/cancel_test.go +++ b/ddl/cancel_test.go @@ -170,11 +170,11 @@ var allTestCase = []testCancelJob{ {"alter table t_partition truncate partition p3", true, model.StateNone, true, false, nil}, {"alter table t_partition truncate partition p3", false, model.StatePublic, false, true, nil}, // Add columns. - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateNone, true, false, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateDeleteOnly, true, true, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateWriteOnly, true, true, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", true, model.StateWriteReorganization, true, true, nil}, - {"alter table t add column c41 bigint, add column c42 bigint", false, model.StatePublic, false, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateNone, model.StateNone}, true, false, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateDeleteOnly, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateWriteOnly, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", true, subStates{model.StateWriteReorganization, model.StateNone}, true, true, nil}, + {"alter table t add column c41 bigint, add column c42 bigint", false, subStates{model.StatePublic, model.StatePublic}, false, true, nil}, // Drop columns. // TODO: fix schema state. {"alter table t drop column c41, drop column c42", true, model.StateNone, true, false, nil}, diff --git a/ddl/column.go b/ddl/column.go index 0ac17d69f5bb7..6b9a00ea5b3db 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -83,7 +83,7 @@ func adjustColumnInfoInDropColumn(tblInfo *model.TableInfo, offset int) { tblInfo.Columns = newCols } -func createColumnInfo(tblInfo *model.TableInfo, colInfo *model.ColumnInfo, pos *ast.ColumnPosition) (*model.ColumnInfo, *ast.ColumnPosition, int, error) { +func createColumnInfoWithPosCheck(tblInfo *model.TableInfo, colInfo *model.ColumnInfo, pos *ast.ColumnPosition) (*model.ColumnInfo, *ast.ColumnPosition, int, error) { // Check column name duplicate. cols := tblInfo.Columns offset := len(cols) @@ -115,19 +115,34 @@ func createColumnInfo(tblInfo *model.TableInfo, colInfo *model.ColumnInfo, pos * return colInfo, pos, offset, nil } -func checkAddColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.ColumnInfo, *model.ColumnInfo, *ast.ColumnPosition, int, error) { +func initAndAddColumnToTable(tblInfo *model.TableInfo, colInfo *model.ColumnInfo) *model.ColumnInfo { + cols := tblInfo.Columns + colInfo.ID = allocateColumnID(tblInfo) + colInfo.State = model.StateNone + // To support add column asynchronous, we should mark its offset as the last column. + // So that we can use origin column offset to get value from row. + colInfo.Offset = len(cols) + // Append the column info to the end of the tblInfo.Columns. + // It will reorder to the right offset in "Columns" when it state change to public. + tblInfo.Columns = append(cols, colInfo) + return colInfo +} + +func checkAddColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.ColumnInfo, *model.ColumnInfo, + *ast.ColumnPosition, bool /* ifNotExists */, error) { schemaID := job.SchemaID tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { - return nil, nil, nil, nil, 0, errors.Trace(err) + return nil, nil, nil, nil, false, errors.Trace(err) } col := &model.ColumnInfo{} pos := &ast.ColumnPosition{} offset := 0 - err = job.DecodeArgs(col, pos, &offset) + ifNotExists := false + err = job.DecodeArgs(col, pos, &offset, &ifNotExists) if err != nil { job.State = model.JobStateCancelled - return nil, nil, nil, nil, 0, errors.Trace(err) + return nil, nil, nil, nil, false, errors.Trace(err) } columnInfo := model.FindColumnInfo(tblInfo.Columns, col.Name.L) @@ -135,10 +150,17 @@ func checkAddColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Colu if columnInfo.State == model.StatePublic { // We already have a column with the same column name. job.State = model.JobStateCancelled - return nil, nil, nil, nil, 0, infoschema.ErrColumnExists.GenWithStackByArgs(col.Name) + return nil, nil, nil, nil, ifNotExists, infoschema.ErrColumnExists.GenWithStackByArgs(col.Name) } } - return tblInfo, columnInfo, col, pos, offset, nil + + err = checkAfterPositionExists(tblInfo, pos) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, nil, nil, false, infoschema.ErrColumnExists.GenWithStackByArgs(col.Name) + } + + return tblInfo, columnInfo, col, pos, false, nil } func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { @@ -157,21 +179,18 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) } }) - tblInfo, columnInfo, col, pos, offset, err := checkAddColumn(t, job) + tblInfo, columnInfo, colFromArgs, pos, ifNotExists, err := checkAddColumn(t, job) if err != nil { + if ifNotExists && infoschema.ErrColumnExists.Equal(err) { + job.Warning = toTError(err) + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil + } return ver, errors.Trace(err) } if columnInfo == nil { - columnInfo, _, offset, err = createColumnInfo(tblInfo, col, pos) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - logutil.BgLogger().Info("[ddl] run add column job", zap.String("job", job.String()), zap.Reflect("columnInfo", *columnInfo), zap.Int("offset", offset)) - // Set offset arg to job. - if offset != 0 { - job.Args = []interface{}{columnInfo, pos, offset} - } + columnInfo = initAndAddColumnToTable(tblInfo, colFromArgs) + logutil.BgLogger().Info("[ddl] run add column job", zap.String("job", job.String()), zap.Reflect("columnInfo", *columnInfo)) if err = checkAddColumnTooManyColumns(len(tblInfo.Columns)); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -206,9 +225,14 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) } // Update the job state when all affairs done. job.SchemaState = model.StateWriteReorganization + job.MarkNonRevertible() case model.StateWriteReorganization: // reorganization -> public // Adjust table column offset. + offset, err := locateOffsetToMove(columnInfo.Offset, pos, tblInfo) + if err != nil { + return ver, errors.Trace(err) + } tblInfo.MoveColumnInfo(columnInfo.Offset, offset) columnInfo.State = model.StatePublic ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) @@ -276,6 +300,18 @@ func setColumnsState(columnInfos []*model.ColumnInfo, state model.SchemaState) { } } +// checkAfterPositionExists makes sure the column specified in AFTER clause is exists. +// For example, ALTER TABLE t ADD COLUMN c3 INT AFTER c1. +func checkAfterPositionExists(tblInfo *model.TableInfo, pos *ast.ColumnPosition) error { + if pos != nil && pos.Tp == ast.ColumnPositionAfter { + c := model.FindColumnInfo(tblInfo.Columns, pos.RelativeColumn.Name.L) + if c == nil { + return infoschema.ErrColumnNotExists.GenWithStackByArgs(pos.RelativeColumn, tblInfo.Name) + } + } + return nil +} + func setIndicesState(indexInfos []*model.IndexInfo, state model.SchemaState) { for _, indexInfo := range indexInfos { indexInfo.State = state @@ -308,7 +344,7 @@ func onAddColumns(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error return ver, nil } for i := range columns { - columnInfo, pos, offset, err := createColumnInfo(tblInfo, columns[i], positions[i]) + columnInfo, pos, offset, err := createColumnInfoWithPosCheck(tblInfo, columns[i], positions[i]) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -856,7 +892,7 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return ver, errors.Trace(err) } - _, _, _, err = createColumnInfo(tblInfo, modifyInfo.changingCol, changingColPos) + _, _, _, err = createColumnInfoWithPosCheck(tblInfo, modifyInfo.changingCol, changingColPos) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index e1f206514366a..a6de4bc964d2f 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -417,7 +417,13 @@ func testCheckJobDone(t *testing.T, store kv.Storage, jobID int64, isAdd bool) { require.NoError(t, err) require.Equal(t, historyJob.State, model.JobStateSynced) if isAdd { - require.Equal(t, historyJob.SchemaState, model.StatePublic) + if historyJob.Type == model.ActionMultiSchemaChange { + for _, sub := range historyJob.MultiSchemaInfo.SubJobs { + require.Equal(t, sub.SchemaState, model.StatePublic) + } + } else { + require.Equal(t, historyJob.SchemaState, model.StatePublic) + } } else { require.Equal(t, historyJob.SchemaState, model.StateNone) } diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index e715890e67bca..2863aa2685f5a 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -837,11 +837,11 @@ func runTestInSchemaState( _, err = se.Execute(context.Background(), "use test_db_state") require.NoError(t, err) cbFunc := func(job *model.Job) { - if job.SchemaState == prevState || checkErr != nil { + if jobStateOrLastSubJobState(job) == prevState || checkErr != nil { return } - prevState = job.SchemaState - if job.SchemaState != state { + prevState = jobStateOrLastSubJobState(job) + if prevState != state { return } for _, sqlWithErr := range sqlWithErrs { @@ -877,6 +877,14 @@ func runTestInSchemaState( } } +func jobStateOrLastSubJobState(job *model.Job) model.SchemaState { + if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { + subs := job.MultiSchemaInfo.SubJobs + return subs[len(subs)-1].SchemaState + } + return job.SchemaState +} + func TestShowIndex(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 200*time.Millisecond) defer clean() diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 23c10805724be..d44f69c29c994 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -401,7 +401,7 @@ func TestIssue5092(t *testing.T) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // The following two statements are consistent with MariaDB. tk.MustGetErrCode("alter table t_issue_5092 add column if not exists d int, add column d int", errno.ErrDupFieldName) - tk.MustExec("alter table t_issue_5092 add column dd int, add column if not exists dd int") + tk.MustGetErrCode("alter table t_issue_5092 add column dd int, add column if not exists dd int", errno.ErrUnsupportedDDLOperation) tk.MustExec("alter table t_issue_5092 add column if not exists (d int, e int), add column ff text") tk.MustExec("alter table t_issue_5092 add column b2 int after b1, add column c2 int first") tk.MustQuery("show create table t_issue_5092").Check(testkit.Rows("t_issue_5092 CREATE TABLE `t_issue_5092` (\n" + @@ -417,7 +417,6 @@ func TestIssue5092(t *testing.T) { " `c1` int(11) DEFAULT NULL,\n" + " `f` int(11) DEFAULT NULL,\n" + " `g` int(11) DEFAULT NULL,\n" + - " `dd` int(11) DEFAULT NULL,\n" + " `ff` text DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) tk.MustExec("drop table t_issue_5092") diff --git a/ddl/ddl.go b/ddl/ddl.go index 21edcd8590d8e..81c0d01bb3d52 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -101,14 +101,14 @@ var ( // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. type DDL interface { - CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt, placementPolicyRef *model.PolicyRefInfo) error + CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) error AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) error - DropSchema(ctx sessionctx.Context, schema model.CIStr) error + DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) error CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error - DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error) + DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo) (err error) - DropView(ctx sessionctx.Context, tableIdent ast.Ident) (err error) + DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) CreateIndex(ctx sessionctx.Context, tableIdent ast.Ident, keyType ast.IndexKeyType, indexName model.CIStr, columnNames []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error DropIndex(ctx sessionctx.Context, tableIdent ast.Ident, indexName model.CIStr, ifExists bool) error @@ -122,7 +122,7 @@ type DDL interface { UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error - DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error) + DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) error DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) error @@ -476,6 +476,14 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { // If RunWorker is true, we need campaign owner and do DDL job. // Otherwise, we needn't do that. if RunWorker { + d.ownerManager.SetBeOwnerHook(func() { + var err error + d.ddlSeqNumMu.seqNum, err = d.GetNextDDLSeqNum() + if err != nil { + logutil.BgLogger().Error("error when getting the ddl history count", zap.Error(err)) + } + }) + err := d.ownerManager.CampaignOwner() if err != nil { return errors.Trace(err) @@ -497,11 +505,6 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { asyncNotify(worker.ddlJobCh) } - d.ddlSeqNumMu.seqNum, err = d.GetNextDDLSeqNum() - if err != nil { - return err - } - go d.schemaSyncer.StartCleanWork() if config.TableLockEnabled() { d.wg.Add(1) @@ -709,6 +712,12 @@ func setDDLJobQuery(ctx sessionctx.Context, job *model.Job) { // - context.Cancel: job has been sent to worker, but not found in history DDL job before cancel // - other: found in history DDL job and return that job error func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { + if mci := ctx.GetSessionVars().StmtCtx.MultiSchemaInfo; mci != nil { + // In multiple schema change, we don't run the job. + // Instead, we merge all the jobs into one pending job. + return appendToSubJobs(mci, job) + } + // Get a global job ID and put the DDL job in the queue. setDDLJobQuery(ctx, job) task := &limitJobTask{job, make(chan error)} @@ -728,6 +737,11 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { var historyJob *model.Job jobID := job.ID + + // Attach the context of the jobId to the calling session so that + // KILL can cancel this DDL job. + ctx.GetSessionVars().StmtCtx.DDLJobID = jobID + // For a job from start to end, the state of it will be none -> delete only -> write only -> reorganization -> public // For every state changes, we will wait as lease 2 * lease time, so here the ticker check is 10 * lease. // But we use etcd to speed up, normally it takes less than 0.5s now, so we use 0.5s or 1s or 3s as the max value. @@ -783,12 +797,7 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { } } } - - if historyJob.MultiSchemaInfo != nil && len(historyJob.MultiSchemaInfo.Warnings) != 0 { - for _, warning := range historyJob.MultiSchemaInfo.Warnings { - ctx.GetSessionVars().StmtCtx.AppendWarning(warning) - } - } + appendMultiChangeWarningsToOwnerCtx(ctx, historyJob) logutil.BgLogger().Info("[ddl] DDL job is finished", zap.Int64("jobID", jobID)) return nil diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 93bdaad56a0c4..d0b799731a779 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -82,21 +82,76 @@ const ( tiflashCheckPendingTablesRetry = 7 ) -func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt, placementPolicyRef *model.PolicyRefInfo) (err error) { - dbInfo := &model.DBInfo{Name: schema} - if charsetInfo != nil { - chs, coll, err := ResolveCharsetCollation(ast.CharsetOpt{Chs: charsetInfo.Chs, Col: charsetInfo.Col}) +func (d *ddl) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) (err error) { + var placementPolicyRef *model.PolicyRefInfo + sessionVars := ctx.GetSessionVars() + + // If no charset and/or collation is specified use collation_server and character_set_server + charsetOpt := &ast.CharsetOpt{} + if sessionVars.GlobalVarsAccessor != nil { + charsetOpt.Col, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CollationServer) if err != nil { - return errors.Trace(err) + return err + } + charsetOpt.Chs, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CharacterSetServer) + if err != nil { + return err + } + } + + explicitCharset := false + explicitCollation := false + if len(stmt.Options) != 0 { + for _, val := range stmt.Options { + switch val.Tp { + case ast.DatabaseOptionCharset: + charsetOpt.Chs = val.Value + explicitCharset = true + case ast.DatabaseOptionCollate: + charsetOpt.Col = val.Value + explicitCollation = true + case ast.DatabaseOptionPlacementPolicy: + placementPolicyRef = &model.PolicyRefInfo{ + Name: model.NewCIStr(val.Value), + } + } } - dbInfo.Charset = chs - dbInfo.Collate = coll - } else { - dbInfo.Charset, dbInfo.Collate = charset.GetDefaultCharsetAndCollate() } + if charsetOpt.Col != "" { + coll, err := collate.GetCollationByName(charsetOpt.Col) + if err != nil { + return err + } + + // The collation is not valid for the specified character set. + // Try to remove any of them, but not if they are explicitly defined. + if coll.CharsetName != charsetOpt.Chs { + if explicitCollation && !explicitCharset { + // Use the explicitly set collation, not the implicit charset. + charsetOpt.Chs = "" + } + if !explicitCollation && explicitCharset { + // Use the explicitly set charset, not the (session) collation. + charsetOpt.Col = "" + } + } + + } + dbInfo := &model.DBInfo{Name: stmt.Name} + chs, coll, err := ResolveCharsetCollation(ast.CharsetOpt{Chs: charsetOpt.Chs, Col: charsetOpt.Col}) + if err != nil { + return errors.Trace(err) + } + dbInfo.Charset = chs + dbInfo.Collate = coll dbInfo.PlacementPolicyRef = placementPolicyRef - return d.CreateSchemaWithInfo(ctx, dbInfo, OnExistError) + + onExist := OnExistError + if stmt.IfNotExists { + onExist = OnExistIgnore + } + return d.CreateSchemaWithInfo(ctx, dbInfo, onExist) } func (d *ddl) CreateSchemaWithInfo( @@ -147,6 +202,12 @@ func (d *ddl) CreateSchemaWithInfo( err = d.DoDDLJob(ctx, job) err = d.callHookOnChanged(job, err) + + if infoschema.ErrDatabaseExists.Equal(err) && onExist == OnExistIgnore { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) } @@ -158,7 +219,7 @@ func (d *ddl) ModifySchemaCharsetAndCollate(ctx sessionctx.Context, stmt *ast.Al } // Check if need to change charset/collation. - dbName := model.NewCIStr(stmt.Name) + dbName := stmt.Name is := d.GetInfoSchemaWithInterceptor(ctx) dbInfo, ok := is.SchemaByName(dbName) if !ok { @@ -181,7 +242,7 @@ func (d *ddl) ModifySchemaCharsetAndCollate(ctx sessionctx.Context, stmt *ast.Al } func (d *ddl) ModifySchemaDefaultPlacement(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, placementPolicyRef *model.PolicyRefInfo) (err error) { - dbName := model.NewCIStr(stmt.Name) + dbName := stmt.Name is := d.GetInfoSchemaWithInterceptor(ctx) dbInfo, ok := is.SchemaByName(dbName) if !ok { @@ -276,7 +337,7 @@ func (d *ddl) waitPendingTableThreshold(sctx sessionctx.Context, schemaID int64, } func (d *ddl) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, tiflashReplica *ast.TiFlashReplicaSpec) error { - dbName := model.NewCIStr(stmt.Name) + dbName := stmt.Name is := d.GetInfoSchemaWithInterceptor(sctx) dbInfo, ok := is.SchemaByName(dbName) if !ok { @@ -520,11 +581,14 @@ func (d *ddl) AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) return nil } -func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) { +func (d *ddl) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (err error) { is := d.GetInfoSchemaWithInterceptor(ctx) - old, ok := is.SchemaByName(schema) + old, ok := is.SchemaByName(stmt.Name) if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists) + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) } job := &model.Job{ SchemaID: old.ID, @@ -537,13 +601,19 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) err = d.DoDDLJob(ctx, job) err = d.callHookOnChanged(job, err) if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) { + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) + } return errors.Trace(err) } if !config.TableLockEnabled() { return nil } // Clear table locks hold by the session. - tbs := is.SchemaTables(schema) + tbs := is.SchemaTables(stmt.Name) lockTableIDs := make([]int64, 0) for _, tb := range tbs { if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { @@ -2903,8 +2973,21 @@ func needToOverwriteColCharset(options []*ast.TableOption) bool { return false } +// resolveAlterTableAddColumns splits "add columns" to multiple spec. For example, +// `ALTER TABLE ADD COLUMN (c1 INT, c2 INT)` is split into +// `ALTER TABLE ADD COLUMN c1 INT, ADD COLUMN c2 INT`. +func resolveAlterTableAddColumns(spec *ast.AlterTableSpec) []*ast.AlterTableSpec { + specs := make([]*ast.AlterTableSpec, len(spec.NewColumns)) + for i, col := range spec.NewColumns { + t := *spec + t.NewColumns = []*ast.ColumnDef{col} + specs[i] = &t + } + return specs +} + // resolveAlterTableSpec resolves alter table algorithm and removes ignore table spec in specs. -// returns valied specs, and the occurred error. +// returns valid specs, and the occurred error. func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) ([]*ast.AlterTableSpec, error) { validSpecs := make([]*ast.AlterTableSpec, 0, len(specs)) algorithm := ast.AlgorithmTypeDefault @@ -2916,7 +2999,11 @@ func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) if isIgnorableSpec(spec.Tp) { continue } - validSpecs = append(validSpecs, spec) + if spec.Tp == ast.AlterTableAddColumns && len(spec.NewColumns) > 1 { + validSpecs = append(validSpecs, resolveAlterTableAddColumns(spec)...) + } else { + validSpecs = append(validSpecs, spec) + } } // Verify whether the algorithm is supported. @@ -2997,9 +3084,10 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast } if len(validSpecs) > 1 { + useMultiSchemaChange := false switch validSpecs[0].Tp { case ast.AlterTableAddColumns: - err = d.AddColumns(sctx, ident, validSpecs) + useMultiSchemaChange = true case ast.AlterTableDropColumn: err = d.DropColumns(sctx, ident, validSpecs) case ast.AlterTableDropPrimaryKey, ast.AlterTableDropIndex: @@ -3010,7 +3098,9 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast if err != nil { return errors.Trace(err) } - return nil + if !useMultiSchemaChange { + return nil + } } if len(validSpecs) > 1 { @@ -3021,11 +3111,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast var handledCharsetOrCollate bool switch spec.Tp { case ast.AlterTableAddColumns: - if len(spec.NewColumns) != 1 { - err = d.AddColumns(sctx, ident, []*ast.AlterTableSpec{spec}) - } else { - err = d.AddColumn(sctx, ident, spec) - } + err = d.AddColumn(sctx, ident, spec) case ast.AlterTableAddPartitions: err = d.AddTablePartitions(sctx, ident, spec) case ast.AlterTableCoalescePartitions: @@ -3476,6 +3562,10 @@ func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTab if col == nil { return nil } + err = checkAfterPositionExists(t.Meta(), spec.Position) + if err != nil { + return errors.Trace(err) + } job := &model.Job{ SchemaID: schema.ID, @@ -3484,15 +3574,10 @@ func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTab TableName: t.Meta().Name.L, Type: model.ActionAddColumn, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{col, spec.Position, 0}, + Args: []interface{}{col, spec.Position, 0, spec.IfNotExists}, } err = d.DoDDLJob(ctx, job) - // column exists, but if_not_exists flags is true, so we ignore this error. - if infoschema.ErrColumnExists.Equal(err) && spec.IfNotExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil - } err = d.callHookOnChanged(job, err) return errors.Trace(err) } @@ -4621,7 +4706,6 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe } switch { case oldRandBits == newRandBits: - break case oldRandBits < newRandBits: addingAutoRandom := oldRandBits == 0 if addingAutoRandom { @@ -5241,71 +5325,172 @@ func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt return errors.Trace(err) } -// DropTable will proceed even if some table in the list does not exists. -func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { - schema, tb, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } +// If one drop those tables by mistake, it's difficult to recover. +// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. +var systemTables = map[string]struct{}{ + "tidb": {}, + "gc_delete_range": {}, + "gc_delete_range_done": {}, +} - if tb.Meta().IsView() { - return infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name) - } - if tb.Meta().IsSequence() { - return infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name) +func isSystemTable(schema, table string) bool { + if schema != "mysql" { + return false } - if tb.Meta().TableCacheStatusType != model.TableCacheStatusDisable { - return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Table") + if _, ok := systemTables[table]; ok { + return true } + return false +} - job := &model.Job{ - SchemaID: schema.ID, - TableID: tb.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: schema.State, - TableName: tb.Meta().Name.L, - Type: model.ActionDropTable, - BinlogInfo: &model.HistoryInfo{}, - } +type objectType int + +const ( + tableObject objectType = iota + viewObject + sequenceObject +) + +// dropTableObject provides common logic to DROP TABLE/VIEW/SEQUENCE. +func (d *ddl) dropTableObject( + ctx sessionctx.Context, + objects []*ast.TableName, + ifExists bool, + tableObjectType objectType, +) error { + var ( + notExistTables []string + sessVars = ctx.GetSessionVars() + is = d.GetInfoSchemaWithInterceptor(ctx) + dropExistErr *terror.Error + jobType model.ActionType + ) + + switch tableObjectType { + case tableObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropTable + case viewObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropView + case sequenceObject: + dropExistErr = infoschema.ErrSequenceDropExists + jobType = model.ActionDropSequence + } + + for _, tn := range objects { + fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} + schema, ok := is.SchemaByName(tn.Schema) + if !ok { + // TODO: we should return special error for table not exist, checking "not exist" is not enough, + // because some other errors may contain this error string too. + notExistTables = append(notExistTables, fullti.String()) + continue + } + tableInfo, err := is.TableByName(tn.Schema, tn.Name) + if err != nil && infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return err + } + + // prechecks before build DDL job + + // Protect important system table from been dropped by a mistake. + // I can hardly find a case that a user really need to do this. + if isSystemTable(tn.Schema.L, tn.Name.L) { + return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) + } + switch tableObjectType { + case tableObject: + if !tableInfo.Meta().IsBaseTable() { + notExistTables = append(notExistTables, fullti.String()) + continue + } + + tempTableType := tableInfo.Meta().TempTableType + if config.CheckTableBeforeDrop && tempTableType == model.TempTableNone { + logutil.BgLogger().Warn("admin check table before drop", + zap.String("database", fullti.Schema.O), + zap.String("table", fullti.Name.O), + ) + exec := ctx.(sqlexec.RestrictedSQLExecutor) + _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) + if err != nil { + return err + } + } + + if tableInfo.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Table") + } + case viewObject: + if !tableInfo.Meta().IsView() { + return dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "VIEW") + } + case sequenceObject: + if !tableInfo.Meta().IsSequence() { + err = dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "SEQUENCE") + if ifExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + continue + } + return err + } + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tableInfo.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: schema.State, + TableName: tableInfo.Meta().Name.L, + Type: jobType, + BinlogInfo: &model.HistoryInfo{}, + } + + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return errors.Trace(err) + } + + // unlock table after drop + if tableObjectType != tableObject { + continue + } + if !config.TableLockEnabled() { + continue + } + if ok, _ := ctx.CheckTableLocked(tableInfo.Meta().ID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{tableInfo.Meta().ID}) + } - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - if err != nil { - return errors.Trace(err) } - if !config.TableLockEnabled() { - return nil + if len(notExistTables) > 0 && !ifExists { + return dropExistErr.GenWithStackByArgs(strings.Join(notExistTables, ",")) } - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) + // We need add warning when use if exists. + if len(notExistTables) > 0 && ifExists { + for _, table := range notExistTables { + sessVars.StmtCtx.AppendNote(dropExistErr.GenWithStackByArgs(table)) + } } return nil } -// DropView will proceed even if some view in the list does not exists. -func (d *ddl) DropView(ctx sessionctx.Context, ti ast.Ident) (err error) { - schema, tb, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } - - if !tb.Meta().IsView() { - return dbterror.ErrWrongObject.GenWithStackByArgs(ti.Schema, ti.Name, "VIEW") - } - - job := &model.Job{ - SchemaID: schema.ID, - TableID: tb.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: tb.Meta().State, - TableName: tb.Meta().Name.L, - Type: model.ActionDropView, - BinlogInfo: &model.HistoryInfo{}, - } +// DropTable will proceed even if some table in the list does not exists. +func (d *ddl) DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return d.dropTableObject(ctx, stmt.Tables, stmt.IfExists, tableObject) +} - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - return errors.Trace(err) +// DropView will proceed even if some view in the list does not exists. +func (d *ddl) DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return d.dropTableObject(ctx, stmt.Tables, stmt.IfExists, viewObject) } func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { @@ -6155,7 +6340,6 @@ func validateCommentLength(vars *variable.SessionVars, name string, comment *str case dbterror.ErrTooLongTableComment: maxLen *= 2 case dbterror.ErrTooLongFieldComment, dbterror.ErrTooLongIndexComment, dbterror.ErrTooLongTablePartitionComment: - break default: // add more types of terror.Error if need } @@ -6219,7 +6403,6 @@ func checkColumnsTypeAndValuesMatch(ctx sessionctx.Context, meta *model.TableInf case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: switch vkind { case types.KindString, types.KindBytes: - break default: return dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() } @@ -6591,34 +6774,8 @@ func (d *ddl) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) return errors.Trace(err) } -func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool) (err error) { - schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } - - if !tbl.Meta().IsSequence() { - err = dbterror.ErrWrongObject.GenWithStackByArgs(ti.Schema, ti.Name, "SEQUENCE") - if ifExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil - } - return err - } - - job := &model.Job{ - SchemaID: schema.ID, - TableID: tbl.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: tbl.Meta().State, - TableName: tbl.Meta().Name.L, - Type: model.ActionDropSequence, - BinlogInfo: &model.HistoryInfo{}, - } - - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - return errors.Trace(err) +func (d *ddl) DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) { + return d.dropTableObject(ctx, stmt.Sequences, stmt.IfExists, sequenceObject) } func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName model.CIStr, visibility ast.IndexVisibility) error { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index d12e304e844a9..50362923488a8 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -296,7 +296,7 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { job.Version = currentVersion job.StartTS = txn.StartTS() job.ID = ids[i] - job.State = model.JobStateQueueing + setJobStateToQueueing(job) if err = buildJobDependence(t, job); err != nil { return errors.Trace(err) } @@ -304,13 +304,7 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { if job.MayNeedReorg() { jobListKey = meta.AddIndexJobListKey } - failpoint.Inject("MockModifyJobArg", func(val failpoint.Value) { - if val.(bool) { - if len(job.Args) > 0 { - job.Args[0] = 1 - } - } - }) + injectModifyJobArgFailPoint(job) if err = t.EnQueueDDLJob(job, jobListKey); err != nil { return errors.Trace(err) } @@ -336,6 +330,30 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { } } +func injectModifyJobArgFailPoint(job *model.Job) { + failpoint.Inject("MockModifyJobArg", func(val failpoint.Value) { + if val.(bool) { + // Corrupt the DDL job argument. + if job.Type == model.ActionMultiSchemaChange { + if len(job.MultiSchemaInfo.SubJobs) > 0 && len(job.MultiSchemaInfo.SubJobs[0].Args) > 0 { + job.MultiSchemaInfo.SubJobs[0].Args[0] = 1 + } + } else if len(job.Args) > 0 { + job.Args[0] = 1 + } + } + }) +} + +func setJobStateToQueueing(job *model.Job) { + if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { + for _, sub := range job.MultiSchemaInfo.SubJobs { + sub.State = model.JobStateQueueing + } + } + job.State = model.JobStateQueueing +} + // getHistoryDDLJob gets a DDL job with job's ID from history queue. func (d *ddl) getHistoryDDLJob(id int64) (*model.Job, error) { se, err := d.sessPool.get() @@ -936,6 +954,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onAlterCacheTable(d, t, job) case model.ActionAlterNoCacheTable: ver, err = onAlterNoCacheTable(d, t, job) + case model.ActionMultiSchemaChange: + ver, err = onMultiSchemaChange(w, d, t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/index.go b/ddl/index.go index 467f84190eb34..9296d2507275c 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -470,7 +470,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo if len(hiddenCols) > 0 { pos := &ast.ColumnPosition{Tp: ast.ColumnPositionNone} for _, hiddenCol := range hiddenCols { - _, _, _, err = createColumnInfo(tblInfo, hiddenCol, pos) + _, _, _, err = createColumnInfoWithPosCheck(tblInfo, hiddenCol, pos) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 39950e424e1db..3ae2ae16482e0 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -882,7 +882,7 @@ func testDropIndexesIfExists(t *testing.T, store kv.Storage) { "[ddl:1091]index i3 doesn't exist", ) tk.MustExec("alter table test_drop_indexes_if_exists drop index i1, drop index if exists i3;") - tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1091|index i3 doesn't exist")) + tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Note|1091|index i3 doesn't exist")) // Verify the impact of deletion order when dropping duplicate indexes. tk.MustGetErrMsg( @@ -894,7 +894,7 @@ func testDropIndexesIfExists(t *testing.T, store kv.Storage) { "[ddl:1091]index i2 doesn't exist", ) tk.MustExec("alter table test_drop_indexes_if_exists drop index i2, drop index if exists i2;") - tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1091|index i2 doesn't exist")) + tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Note|1091|index i2 doesn't exist")) } func testDropIndexesFromPartitionedTable(t *testing.T, store kv.Storage) { diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index 414d2f1484909..bd0518404530a 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -17,6 +17,7 @@ package ddl import ( "github.com/pingcap/errors" ddlutil "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" @@ -56,18 +57,136 @@ func (d *ddl) MultiSchemaChange(ctx sessionctx.Context, ti ast.Ident) error { return d.callHookOnChanged(job, err) } -func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { - err := checkOperateSameColAndIdx(info) - if err != nil { - return err +func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if job.MultiSchemaInfo.Revertible { + // Handle the rolling back job. + if job.IsRollingback() { + // Rollback/cancel the sub-jobs in reverse order. + for i := len(job.MultiSchemaInfo.SubJobs) - 1; i >= 0; i-- { + sub := job.MultiSchemaInfo.SubJobs[i] + if sub.IsFinished() { + continue + } + proxyJob := sub.ToProxyJob(job) + ver, err = w.runDDLJob(d, t, proxyJob) + sub.FromProxyJob(proxyJob) + return ver, err + } + // The last rollback/cancelling sub-job is done. + job.State = model.JobStateRollbackDone + return ver, nil + } + + // The sub-jobs are normally running. + // Run the first executable sub-job. + for _, sub := range job.MultiSchemaInfo.SubJobs { + if !sub.Revertible || sub.IsFinished() { + // Skip the sub-jobs which related schema states + // are in the last revertible point. + // If a sub job is finished here, it should be a noop job. + continue + } + proxyJob := sub.ToProxyJob(job) + ver, err = w.runDDLJob(d, t, proxyJob) + sub.FromProxyJob(proxyJob) + handleRevertibleException(job, sub, proxyJob.Error) + return ver, err + } + + // Save table info and sub-jobs for rolling back. + var tblInfo *model.TableInfo + tblInfo, err = t.GetTable(job.SchemaID, job.TableID) + if err != nil { + return ver, err + } + subJobs := make([]model.SubJob, len(job.MultiSchemaInfo.SubJobs)) + // Step the sub-jobs to the non-revertible states all at once. + for i, sub := range job.MultiSchemaInfo.SubJobs { + if sub.IsFinished() { + continue + } + subJobs[i] = *sub + proxyJob := sub.ToProxyJob(job) + ver, err = w.runDDLJob(d, t, proxyJob) + sub.FromProxyJob(proxyJob) + if err != nil || proxyJob.Error != nil { + for j := i - 1; j >= 0; j-- { + job.MultiSchemaInfo.SubJobs[j] = &subJobs[j] + } + handleRevertibleException(job, sub, proxyJob.Error) + // The TableInfo and sub-jobs should be restored + // because some schema changes update the transaction aggressively. + return updateVersionAndTableInfo(d, t, job, tblInfo, true) + } + } + // All the sub-jobs are non-revertible. + job.MarkNonRevertible() + return ver, err + } + // Run the rest non-revertible sub-jobs one by one. + for _, sub := range job.MultiSchemaInfo.SubJobs { + if sub.IsFinished() { + continue + } + proxyJob := sub.ToProxyJob(job) + ver, err = w.runDDLJob(d, t, proxyJob) + sub.FromProxyJob(proxyJob) + return ver, err } + job.State = model.JobStateDone + return ver, err +} - err = checkVisibleColumnCnt(t, len(info.AddColumns), len(info.DropColumns)) +func handleRevertibleException(job *model.Job, subJob *model.SubJob, err *terror.Error) { + if subJob.IsNormal() { + return + } + job.State = model.JobStateRollingback + job.Error = err + // Flush the cancelling state and cancelled state to sub-jobs. + for _, sub := range job.MultiSchemaInfo.SubJobs { + switch sub.State { + case model.JobStateRunning: + sub.State = model.JobStateCancelling + case model.JobStateNone, model.JobStateQueueing: + sub.State = model.JobStateCancelled + } + } +} + +func appendToSubJobs(m *model.MultiSchemaInfo, job *model.Job) error { + err := fillMultiSchemaInfo(m, job) if err != nil { return err } + m.SubJobs = append(m.SubJobs, &model.SubJob{ + Type: job.Type, + Args: job.Args, + RawArgs: job.RawArgs, + SchemaState: job.SchemaState, + SnapshotVer: job.SnapshotVer, + Revertible: true, + CtxVars: job.CtxVars, + }) + return nil +} - return checkAddColumnTooManyColumns(len(t.Cols()) + len(info.AddColumns) - len(info.DropColumns)) +func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error) { + switch job.Type { + case model.ActionAddColumn: + col := job.Args[0].(*table.Column) + pos := job.Args[1].(*ast.ColumnPosition) + info.AddColumns = append(info.AddColumns, col.Name) + for colName := range col.Dependences { + info.RelativeColumns = append(info.RelativeColumns, model.CIStr{L: colName, O: colName}) + } + if pos != nil && pos.Tp == ast.ColumnPositionAfter { + info.PositionColumns = append(info.PositionColumns, pos.RelativeColumn.Name) + } + default: + return dbterror.ErrRunMultiSchemaChanges + } + return nil } func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { @@ -106,12 +225,15 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { if err := checkColumns(info.DropColumns, true); err != nil { return err } - if err := checkColumns(info.RelativeColumns, false); err != nil { + if err := checkColumns(info.PositionColumns, false); err != nil { return err } if err := checkColumns(info.ModifyColumns, true); err != nil { return err } + if err := checkColumns(info.RelativeColumns, false); err != nil { + return err + } if err := checkIndexes(info.AddIndexes, true); err != nil { return err @@ -121,3 +243,56 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { } return checkIndexes(info.AlterIndexes, true) } + +func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { + err := checkOperateSameColAndIdx(info) + if err != nil { + return err + } + + err = checkVisibleColumnCnt(t, len(info.AddColumns), len(info.DropColumns)) + if err != nil { + return err + } + + return checkAddColumnTooManyColumns(len(t.Cols()) + len(info.AddColumns) - len(info.DropColumns)) +} + +func appendMultiChangeWarningsToOwnerCtx(ctx sessionctx.Context, job *model.Job) { + if job.MultiSchemaInfo == nil { + return + } + if job.Type == model.ActionMultiSchemaChange { + for _, sub := range job.MultiSchemaInfo.SubJobs { + if sub.Warning != nil { + ctx.GetSessionVars().StmtCtx.AppendNote(sub.Warning) + } + } + } + for _, w := range job.MultiSchemaInfo.Warnings { + ctx.GetSessionVars().StmtCtx.AppendNote(w) + } + +} + +// rollingBackMultiSchemaChange updates a multi-schema change job +// from cancelling state to rollingback state. +func rollingBackMultiSchemaChange(job *model.Job) error { + if !job.MultiSchemaInfo.Revertible { + // Cannot rolling back because the jobs are non-revertible. + // Resume the job state to running. + job.State = model.JobStateRunning + return nil + } + // Mark all the jobs to cancelling. + for _, sub := range job.MultiSchemaInfo.SubJobs { + switch sub.State { + case model.JobStateRunning: + sub.State = model.JobStateCancelling + case model.JobStateNone, model.JobStateQueueing: + sub.State = model.JobStateCancelled + } + } + job.State = model.JobStateRollingback + return dbterror.ErrCancelledDDLJob +} diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go new file mode 100644 index 0000000000000..9e3cae966c43f --- /dev/null +++ b/ddl/multi_schema_change_test.go @@ -0,0 +1,100 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + "testing" + + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/testkit" +) + +func TestMultiSchemaChangeAddColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@global.tidb_enable_change_multi_schema = 1") + + // Test add multiple columns in multiple specs. + tk.MustExec("create table t (a int);") + tk.MustExec("insert into t values (1);") + tk.MustExec("alter table t add column b int default 2, add column c int default 3;") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3")) + + // Test add multiple columns in one spec. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int);") + tk.MustExec("insert into t values (1);") + tk.MustExec("alter table t add column (b int default 2, c int default 3);") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int);") + tk.MustExec("insert into t values (1, 2, 3);") + tk.MustExec("alter table t add column (d int default 4, e int default 5);") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3 4 5")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int default 1);") + tk.MustExec("insert into t values ();") + tk.MustExec("alter table t add column if not exists (b int default 2, c int default 3);") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3")) + tk.MustExec("alter table t add column if not exists (c int default 3, d int default 4);") + tk.MustQuery("show warnings;").Check(testkit.Rows("Note 1060 Duplicate column name 'c'")) + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3 4")) + + // Test referencing previous column in multi-schema change is not supported. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int);") + tk.MustGetErrCode("alter table t add column b int after a, add column c int after b", errno.ErrBadField) + tk.MustGetErrCode("alter table t add column c int after b, add column b int", errno.ErrBadField) + + // Test add multiple columns with different position. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int);") + tk.MustExec("insert into t values (1, 2, 3);") + tk.MustExec(`alter table t + add column d int default 4 first, + add column e int default 5 after b, + add column f int default 6 after b;`) + tk.MustQuery("select * from t;").Check(testkit.Rows("4 1 2 6 5 3")) + + // Test [if not exists] for adding columns. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int default 1);") + tk.MustExec("insert into t values ();") + tk.MustExec("alter table t add column b int default 2, add column if not exists a int;") + tk.MustQuery("show warnings;").Check(testkit.Rows("Note 1060 Duplicate column name 'a'")) + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2")) + + // Test add generate column + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values (1, 2);") + tk.MustExec("alter table t add column c double default 3.0, add column d double as (a + b);") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 3 3")) + + // Test add columns with same name + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int default 1, c int default 4);") + tk.MustGetErrCode("alter table t add column b int default 2, add column b int default 3", errno.ErrUnsupportedDDLOperation) + + // Test add generate column dependents on a modifying column + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values (1, 2);") + tk.MustGetErrCode("alter table t modify column b double, add column c double as (a + b);", errno.ErrUnsupportedDDLOperation) +} diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 659cf146e62ba..b1ea2da7c7592 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -481,6 +481,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility, model.ActionExchangeTablePartition, model.ActionModifySchemaDefaultPlacement: ver, err = cancelOnlyNotHandledJob(job, model.StateNone) + case model.ActionMultiSchemaChange: + err = rollingBackMultiSchemaChange(job) default: job.State = model.JobStateCancelled err = dbterror.ErrCancelledDDLJob diff --git a/domain/domain_test.go b/domain/domain_test.go index b5783a2b97013..98776381d1ae8 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -122,13 +122,22 @@ func TestInfo(t *testing.T) { } require.True(t, syncerStarted) - // Make sure loading schema is normal. - cs := &ast.CharsetOpt{ - Chs: "utf8", - Col: "utf8_bin", + stmt := &ast.CreateDatabaseStmt{ + Name: model.NewCIStr("aaa"), + // Make sure loading schema is normal. + Options: []*ast.DatabaseOption{ + { + Tp: ast.DatabaseOptionCharset, + Value: "utf8", + }, + { + Tp: ast.DatabaseOptionCollate, + Value: "utf8_bin", + }, + }, } ctx := mock.NewContext() - require.NoError(t, dom.ddl.CreateSchema(ctx, model.NewCIStr("aaa"), cs, nil)) + require.NoError(t, dom.ddl.CreateSchema(ctx, stmt)) require.NoError(t, dom.Reload()) require.Equal(t, int64(1), dom.InfoSchema().SchemaMetaVersion()) diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 295b9e07c1f3d..94f32782965ed 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -132,7 +132,6 @@ go_library( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", - "//sessiontxn/legacy", "//sessiontxn/staleread", "//statistics", "//statistics/handle", @@ -152,6 +151,7 @@ go_library( "//util", "//util/admin", "//util/bitmap", + "//util/breakpoint", "//util/chunk", "//util/codec", "//util/collate", diff --git a/executor/adapter.go b/executor/adapter.go index 57f85e38452a7..3a4ace6ba582b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/breakpoint" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hint" @@ -355,7 +356,7 @@ func IsFastPlan(p plannercore.Plan) bool { } // Exec builds an Executor from a plan. If the Executor doesn't return result, -// like the INSERT, UPDATE statements, it executes in this function, if the Executor returns +// like the INSERT, UPDATE statements, it executes in this function. If the Executor returns // result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { defer func() { @@ -420,10 +421,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) - failpoint.Inject("hookBeforeFirstRunExecutor", func() { - sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookBeforeFirstRunExecutorKey) - }) - + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointBeforeExecutorFirstRun) if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -715,7 +713,10 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { keys = filterTemporaryTableKeys(sctx.GetSessionVars(), keys) seVars := sctx.GetSessionVars() keys = filterLockTableKeys(seVars.StmtCtx, keys) - lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout, len(keys)) + lockCtx, err := newLockCtx(sctx, seVars.LockWaitTimeout, len(keys)) + if err != nil { + return err + } var lockKeyStats *util.LockKeysDetails ctx = context.WithValue(ctx, util.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() @@ -737,43 +738,18 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } } -// UpdateForUpdateTS updates the ForUpdateTS, if newForUpdateTS is 0, it obtain a new TS from PD. -func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { - txn, err := seCtx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - return errors.Trace(kv.ErrInvalidTxn) - } - - // The Oracle serializable isolation is actually SI in pessimistic mode. - // Do not update ForUpdateTS when the user is using the Serializable isolation level. - // It can be used temporarily on the few occasions when an Oracle-like isolation level is needed. - // Support for this does not mean that TiDB supports serializable isolation of MySQL. - // tidb_skip_isolation_level_check should still be disabled by default. - if seCtx.GetSessionVars().IsIsolation(ast.Serializable) { - return nil - } - if newForUpdateTS == 0 { - // Because the ForUpdateTS is used for the snapshot for reading data in DML. - // We can avoid allocating a global TSO here to speed it up by using the local TSO. - version, err := seCtx.GetStore().CurrentVersion(seCtx.GetSessionVars().TxnCtx.TxnScope) - if err != nil { - return err - } - newForUpdateTS = version.Ver - } - seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS) - txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) - return nil -} - // handlePessimisticLockError updates TS and rebuild executor if the err is write conflict. func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error) (_ Executor, err error) { if lockErr == nil { return nil, nil } + failpoint.Inject("assertPessimisticLockErr", func() { + if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errWriteConflict") + } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errDuplicateKey") + } + }) defer func() { if _, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { @@ -781,7 +757,8 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } }() - action, err := sessiontxn.GetTxnManager(a.Ctx).OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + txnManager := sessiontxn.GetTxnManager(a.Ctx) + action, err := txnManager.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) if err != nil { return nil, err } @@ -796,14 +773,18 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error a.retryCount++ a.retryStartTime = time.Now() - err = sessiontxn.GetTxnManager(a.Ctx).OnStmtRetry(ctx) + err = txnManager.OnStmtRetry(ctx) if err != nil { return nil, err } - failpoint.Inject("hookAfterOnStmtRetryWithLockError", func() { - sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookAfterOnStmtRetryWithLockErrorKey) - }) + // Without this line of code, the result will still be correct. But it can ensure that the update time of for update read + // is determined which is beneficial for testing. + if _, err = txnManager.GetStmtForUpdateTS(); err != nil { + return nil, err + } + + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() if err != nil { diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 764b2e624d7ac..f36e066505ad1 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1589,3 +1589,43 @@ func TestRandomPanicAggConsume(t *testing.T) { require.EqualError(t, err, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") } } + +func TestIssue35295(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t100") + // This bug only happens on partition prune mode = 'static' + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec(`CREATE TABLE t100 ( +ID bigint(20) unsigned NOT NULL AUTO_INCREMENT, +col1 int(10) NOT NULL DEFAULT '0' COMMENT 'test', +money bigint(20) NOT NULL COMMENT 'test', +logtime datetime NOT NULL COMMENT '记录时间', +PRIMARY KEY (ID,logtime) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin AUTO_INCREMENT=1 COMMENT='test' +PARTITION BY RANGE COLUMNS(logtime) ( +PARTITION p20220608 VALUES LESS THAN ("20220609"), +PARTITION p20220609 VALUES LESS THAN ("20220610"), +PARTITION p20220610 VALUES LESS THAN ("20220611"), +PARTITION p20220611 VALUES LESS THAN ("20220612"), +PARTITION p20220612 VALUES LESS THAN ("20220613"), +PARTITION p20220613 VALUES LESS THAN ("20220614"), +PARTITION p20220614 VALUES LESS THAN ("20220615"), +PARTITION p20220615 VALUES LESS THAN ("20220616"), +PARTITION p20220616 VALUES LESS THAN ("20220617"), +PARTITION p20220617 VALUES LESS THAN ("20220618"), +PARTITION p20220618 VALUES LESS THAN ("20220619"), +PARTITION p20220619 VALUES LESS THAN ("20220620"), +PARTITION p20220620 VALUES LESS THAN ("20220621"), +PARTITION p20220621 VALUES LESS THAN ("20220622"), +PARTITION p20220622 VALUES LESS THAN ("20220623"), +PARTITION p20220623 VALUES LESS THAN ("20220624"), +PARTITION p20220624 VALUES LESS THAN ("20220625") + );`) + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-09 00:00:00');") + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-10 00:00:00');") + tk.MustQuery("SELECT /*+STREAM_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) + tk.MustQuery("SELECT /*+HASH_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) +} diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 0ce745d172e4a..b5eb68a8b12de 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -56,7 +56,6 @@ type BatchPointGetExec struct { singlePart bool partTblID int64 idxVals [][]types.Datum - startTS uint64 readReplicaScope string isStaleness bool snapshotTS uint64 @@ -97,13 +96,9 @@ func (e *BatchPointGetExec) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *BatchPointGetExec) Open(context.Context) error { - e.snapshotTS = e.startTS sessVars := e.ctx.GetSessionVars() txnCtx := sessVars.TxnCtx stmtCtx := sessVars.StmtCtx - if e.lock { - e.snapshotTS = txnCtx.GetForUpdateTS() - } txn, err := e.ctx.Txn(false) if err != nil { return err @@ -111,8 +106,8 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.txn = txn var snapshot kv.Snapshot if txn.Valid() && txnCtx.StartTS == txnCtx.GetForUpdateTS() && txnCtx.StartTS == e.snapshotTS { - // We can safely reuse the transaction snapshot if startTS is equal to forUpdateTS. - // The snapshot may contains cache that can reduce RPC call. + // We can safely reuse the transaction snapshot if snapshotTS is equal to forUpdateTS. + // The snapshot may contain cache that can reduce RPC call. snapshot = txn.GetSnapshot() } else { snapshot = e.ctx.GetSnapshotWithTS(e.snapshotTS) @@ -540,13 +535,16 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { } // LockKeys locks the keys for pessimistic transaction. -func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { - txnCtx := seCtx.GetSessionVars().TxnCtx - lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime, len(keys)) +func LockKeys(ctx context.Context, sctx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { + txnCtx := sctx.GetSessionVars().TxnCtx + lctx, err := newLockCtx(sctx, lockWaitTime, len(keys)) + if err != nil { + return err + } if txnCtx.IsPessimistic { lctx.InitReturnValues(len(keys)) } - err := doLockKeys(ctx, seCtx, lctx, keys...) + err = doLockKeys(ctx, sctx, lctx, keys...) if err != nil { return err } diff --git a/executor/brie.go b/executor/brie.go index 3cd1ce7ea9e1e..78fdcbe9f1285 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -249,7 +249,6 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema) return nil } default: - break } if tidbCfg.Store != "tikv" { diff --git a/executor/builder.go b/executor/builder.go index 8a44c09aaf033..ba5cebc836363 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/helper" @@ -88,14 +87,11 @@ var ( // executorBuilder builds an Executor from a Plan. // The InfoSchema must not change during execution. type executorBuilder struct { - ctx sessionctx.Context - is infoschema.InfoSchema - snapshotTS uint64 // The ts for snapshot-read. A select statement without for update will use this ts - forUpdateTS uint64 // The ts should be used by insert/update/delete/select-for-update statement - snapshotTSCached bool - err error // err is set when there is error happened during Executor building process. - hasLock bool - Ti *TelemetryInfo + ctx sessionctx.Context + is infoschema.InfoSchema + err error // err is set when there is error happened during Executor building process. + hasLock bool + Ti *TelemetryInfo // isStaleness means whether this statement use stale read. isStaleness bool readReplicaScope string @@ -121,26 +117,13 @@ type CTEStorages struct { } func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema, ti *TelemetryInfo, replicaReadScope string) *executorBuilder { - b := &executorBuilder{ + return &executorBuilder{ ctx: ctx, is: is, Ti: ti, isStaleness: staleread.IsStmtStaleness(ctx), readReplicaScope: replicaReadScope, } - - txnManager := sessiontxn.GetTxnManager(ctx) - if provider, ok := txnManager.GetContextProvider().(*legacy.SimpleTxnContextProvider); ok { - provider.GetReadTSFunc = b.getReadTS - provider.GetForUpdateTSFunc = func() (uint64, error) { - if b.forUpdateTS != 0 { - return b.forUpdateTS, nil - } - return b.getReadTS() - } - } - - return b } // MockPhysicalPlan is used to return a specified executor in when build. @@ -657,11 +640,9 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true - if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } - // Build 'select for update' using the 'for update' ts. - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() src := b.build(v.Children()[0]) if b.err != nil { @@ -865,14 +846,10 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - if v.SelectPlan != nil { - // Try to update the forUpdateTS for insert/replace into select statements. - // Set the selectPlan parameter to nil to make it always update the forUpdateTS. - if b.err = b.updateForUpdateTSIfNeeded(nil); b.err != nil { - return nil - } + if b.err = b.updateForUpdateTS(); b.err != nil { + return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selectExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -1437,7 +1414,9 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for _, aggDesc := range v.AggFuncs { if aggDesc.HasDistinct || len(aggDesc.OrderByItems) > 0 { @@ -1493,10 +1472,14 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu groupChecker: newVecGroupChecker(b.ctx, v.GroupByItems), aggFuncs: make([]aggfuncs.AggFunc, 0, len(v.AggFuncs)), } + if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + // Only do this for final agg, see issue #35295, #30923 + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for i, aggDesc := range v.AggFuncs { aggFunc := aggfuncs.Build(b.ctx, aggDesc, i) @@ -1578,44 +1561,6 @@ func (b *executorBuilder) getSnapshotTS() (uint64, error) { return txnManager.GetStmtReadTS() } -// getReadTS returns the ts used by select (without for-update clause). The return value is affected by the isolation level -// and some stale/historical read contexts. For example, it will return txn.StartTS in RR and return -// the current timestamp in RC isolation -func (b *executorBuilder) getReadTS() (uint64, error) { - failpoint.Inject("assertNotStaleReadForExecutorGetReadTS", func() { - // after refactoring stale read will use its own context provider - staleread.AssertStmtStaleness(b.ctx, false) - }) - - if b.snapshotTSCached { - return b.snapshotTS, nil - } - - if snapshotTS := b.ctx.GetSessionVars().SnapshotTS; snapshotTS != 0 { - b.snapshotTS = snapshotTS - b.snapshotTSCached = true - return snapshotTS, nil - } - - if b.snapshotTS != 0 { - b.snapshotTSCached = true - // Return the cached value. - return b.snapshotTS, nil - } - - txn, err := b.ctx.Txn(true) - if err != nil { - return 0, err - } - - b.snapshotTS = txn.StartTS() - if b.snapshotTS == 0 { - return 0, errors.Trace(ErrGetStartTS) - } - b.snapshotTSCached = true - return b.snapshotTS, nil -} - func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executor { switch v.DBName.L { case util.MetricSchemaName.L: @@ -2116,10 +2061,10 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2173,10 +2118,11 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { for _, info := range v.TblColPosInfos { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2192,31 +2138,9 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } -// updateForUpdateTSIfNeeded updates the ForUpdateTS for a pessimistic transaction if needed. -// PointGet executor will get conflict error if the ForUpdateTS is older than the latest commitTS, -// so we don't need to update now for better latency. -func (b *executorBuilder) updateForUpdateTSIfNeeded(selectPlan plannercore.PhysicalPlan) error { - txnCtx := b.ctx.GetSessionVars().TxnCtx - if !txnCtx.IsPessimistic { - return nil - } - if _, ok := selectPlan.(*plannercore.PointGetPlan); ok { - return nil - } - // Activate the invalid txn, use the txn startTS as newForUpdateTS - txn, err := b.ctx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - _, err := b.ctx.Txn(true) - if err != nil { - return err - } - return nil - } - // GetStmtForUpdateTS will auto update the for update ts if necessary - _, err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() +func (b *executorBuilder) updateForUpdateTS() error { + // GetStmtForUpdateTS will auto update the for update ts if it is necessary + _, err := sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() return err } @@ -4663,18 +4587,26 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan return nil } - startTS, err := b.getSnapshotTS() + if plan.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() if err != nil { b.err = err return nil } + decoder := NewRowDecoder(b.ctx, plan.Schema(), plan.TblInfo) e := &BatchPointGetExec{ baseExecutor: newBaseExecutor(b.ctx, plan.Schema(), plan.ID()), tblInfo: plan.TblInfo, idxInfo: plan.IndexInfo, rowDecoder: decoder, - startTS: startTS, + snapshotTS: snapshotTS, readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, keepOrder: plan.KeepOrder, @@ -4687,9 +4619,11 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan partTblID: plan.PartTblID, columns: plan.Columns, } + if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(plan.TblInfo, startTS) + e.cacheTable = b.getCacheTable(plan.TblInfo, snapshotTS) } + if plan.TblInfo.TempTableType != model.TempTableNone { // Temporary table should not do any lock operations e.lock = false @@ -4699,6 +4633,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan if e.lock { b.hasLock = true } + var capacity int if plan.IndexInfo != nil && !isCommonHandleRead(plan.TblInfo, plan.IndexInfo) { e.idxVals = plan.IndexValues diff --git a/executor/ddl.go b/executor/ddl.go index f77091b5a399b..68115f2014e44 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -34,11 +34,9 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/temptable" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -134,7 +132,10 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { return err } - defer func() { e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false }() + defer func() { + e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false + e.ctx.GetSessionVars().StmtCtx.DDLJobID = 0 + }() switch x := e.stmt.(type) { case *ast.AlterDatabaseStmt: @@ -248,70 +249,7 @@ func (e *DDLExec) executeRenameTable(s *ast.RenameTableStmt) error { } func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { - var opt *ast.CharsetOpt - var placementPolicyRef *model.PolicyRefInfo - var err error - sessionVars := e.ctx.GetSessionVars() - - // If no charset and/or collation is specified use collation_server and character_set_server - opt = &ast.CharsetOpt{} - if sessionVars.GlobalVarsAccessor != nil { - opt.Col, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CollationServer) - if err != nil { - return err - } - opt.Chs, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CharacterSetServer) - if err != nil { - return err - } - } - - explicitCharset := false - explicitCollation := false - if len(s.Options) != 0 { - for _, val := range s.Options { - switch val.Tp { - case ast.DatabaseOptionCharset: - opt.Chs = val.Value - explicitCharset = true - case ast.DatabaseOptionCollate: - opt.Col = val.Value - explicitCollation = true - case ast.DatabaseOptionPlacementPolicy: - placementPolicyRef = &model.PolicyRefInfo{ - Name: model.NewCIStr(val.Value), - } - } - } - } - - if opt.Col != "" { - coll, err := collate.GetCollationByName(opt.Col) - if err != nil { - return err - } - - // The collation is not valid for the specified character set. - // Try to remove any of them, but not if they are explicitly defined. - if coll.CharsetName != opt.Chs { - if explicitCollation && !explicitCharset { - // Use the explicitly set collation, not the implicit charset. - opt.Chs = "" - } - if !explicitCollation && explicitCharset { - // Use the explicitly set charset, not the (session) collation. - opt.Col = "" - } - } - - } - - err = domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, model.NewCIStr(s.Name), opt, placementPolicyRef) - if err != nil { - if infoschema.ErrDatabaseExists.Equal(err) && s.IfNotExists { - err = nil - } - } + err := domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, s) return err } @@ -375,7 +313,7 @@ func (e *DDLExec) executeCreateIndex(s *ast.CreateIndexStmt) error { } func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { - dbName := model.NewCIStr(s.Name) + dbName := s.Name // Protect important system table from been dropped by a mistake. // I can hardly find a case that a user really need to do this. @@ -383,14 +321,7 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { return errors.New("Drop 'mysql' database is forbidden") } - err := domain.GetDomain(e.ctx).DDL().DropSchema(e.ctx, dbName) - if infoschema.ErrDatabaseNotExists.Equal(err) { - if s.IfExists { - err = nil - } else { - err = infoschema.ErrDatabaseDropExists.GenWithStackByArgs(s.Name) - } - } + err := domain.GetDomain(e.ctx).DDL().DropSchema(e.ctx, s) sessionVars := e.ctx.GetSessionVars() if err == nil && strings.ToLower(sessionVars.CurrentDB) == dbName.L { sessionVars.CurrentDB = "" @@ -406,117 +337,16 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { return err } -// If one drop those tables by mistake, it's difficult to recover. -// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. -var systemTables = map[string]struct{}{ - "tidb": {}, - "gc_delete_range": {}, - "gc_delete_range_done": {}, -} - -func isSystemTable(schema, table string) bool { - if schema != "mysql" { - return false - } - if _, ok := systemTables[table]; ok { - return true - } - return false -} - -type objectType int - -const ( - tableObject objectType = iota - viewObject - sequenceObject -) - func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { - return e.dropTableObject(s.Tables, tableObject, s.IfExists) + return domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, s) } func (e *DDLExec) executeDropView(s *ast.DropTableStmt) error { - return e.dropTableObject(s.Tables, viewObject, s.IfExists) + return domain.GetDomain(e.ctx).DDL().DropView(e.ctx, s) } func (e *DDLExec) executeDropSequence(s *ast.DropSequenceStmt) error { - return e.dropTableObject(s.Sequences, sequenceObject, s.IfExists) -} - -// dropTableObject actually applies to `tableObject`, `viewObject` and `sequenceObject`. -func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifExists bool) error { - var notExistTables []string - sessVars := e.ctx.GetSessionVars() - for _, tn := range objects { - fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} - _, ok := e.is.SchemaByName(tn.Schema) - if !ok { - // TODO: we should return special error for table not exist, checking "not exist" is not enough, - // because some other errors may contain this error string too. - notExistTables = append(notExistTables, fullti.String()) - continue - } - _, err := e.is.TableByName(tn.Schema, tn.Name) - if err != nil && infoschema.ErrTableNotExists.Equal(err) { - notExistTables = append(notExistTables, fullti.String()) - continue - } else if err != nil { - return err - } - - // Protect important system table from been dropped by a mistake. - // I can hardly find a case that a user really need to do this. - if isSystemTable(tn.Schema.L, tn.Name.L) { - return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) - } - tableInfo, err := e.is.TableByName(tn.Schema, tn.Name) - if err != nil { - return err - } - tempTableType := tableInfo.Meta().TempTableType - if obt == tableObject && config.CheckTableBeforeDrop && tempTableType == model.TempTableNone { - logutil.BgLogger().Warn("admin check table before drop", - zap.String("database", fullti.Schema.O), - zap.String("table", fullti.Name.O), - ) - exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) - if err != nil { - return err - } - } - switch obt { - case tableObject: - err = domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, fullti) - case viewObject: - err = domain.GetDomain(e.ctx).DDL().DropView(e.ctx, fullti) - case sequenceObject: - err = domain.GetDomain(e.ctx).DDL().DropSequence(e.ctx, fullti, ifExists) - } - if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { - notExistTables = append(notExistTables, fullti.String()) - } else if err != nil { - return err - } - } - if len(notExistTables) > 0 && !ifExists { - if obt == sequenceObject { - return infoschema.ErrSequenceDropExists.GenWithStackByArgs(strings.Join(notExistTables, ",")) - } - return infoschema.ErrTableDropExists.GenWithStackByArgs(strings.Join(notExistTables, ",")) - } - // We need add warning when use if exists. - if len(notExistTables) > 0 && ifExists { - for _, table := range notExistTables { - if obt == sequenceObject { - sessVars.StmtCtx.AppendNote(infoschema.ErrSequenceDropExists.GenWithStackByArgs(table)) - } else { - sessVars.StmtCtx.AppendNote(infoschema.ErrTableDropExists.GenWithStackByArgs(table)) - } - } - } - return nil + return domain.GetDomain(e.ctx).DDL().DropSequence(e.ctx, s) } func (e *DDLExec) dropLocalTemporaryTables(localTempTables []*ast.TableName) error { diff --git a/executor/executor.go b/executor/executor.go index 79a2fee7b95eb..7b370615582cc 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -1025,6 +1026,15 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) + if physTblID == 0 { + // select * from t1 left join t2 on t1.c = t2.c for update + // The join right side might be added NULL in left join + // In that case, physTblID is 0, so skip adding the lock. + // + // Note, we can't distinguish whether it's the left join case, + // or a bug that TiKV return without correct physical ID column. + continue + } } e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } @@ -1042,12 +1052,20 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { for id := range e.tblID2Handle { e.updateDeltaForTableID(id) } - - return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime, len(e.keys)), e.keys...) + lockCtx, err := newLockCtx(e.ctx, lockWaitTime, len(e.keys)) + if err != nil { + return err + } + return doLockKeys(ctx, e.ctx, lockCtx, e.keys...) } -func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) *tikvstore.LockCtx { - lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) +func newLockCtx(sctx sessionctx.Context, lockWaitTime int64, numKeys int) (*tikvstore.LockCtx, error) { + seVars := sctx.GetSessionVars() + forUpdateTS, err := sessiontxn.GetTxnManager(sctx).GetStmtForUpdateTS() + if err != nil { + return nil, err + } + lockCtx := tikvstore.NewLockCtx(forUpdateTS, lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited lockCtx.LockKeysDuration = &seVars.StmtCtx.LockKeysDuration @@ -1082,7 +1100,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) * if lockCtx.ForUpdateTS > 0 && seVars.AssertionLevel != variable.AssertionLevelOff { lockCtx.InitCheckExistence(numKeys) } - return lockCtx + return lockCtx, nil } // doLockKeys is the main entry for pessimistic lock keys @@ -1914,7 +1932,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.IgnoreTruncate = true sc.IgnoreZeroInDate = true sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() - if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors { + if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors || stmt.Tp == ast.ShowSessionStates { sc.InShowWarning = true sc.SetWarnings(vars.StmtCtx.GetWarnings()) } @@ -1922,6 +1940,11 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.IgnoreTruncate = false sc.IgnoreZeroInDate = true sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() + case *ast.SetSessionStatesStmt: + sc.InSetSessionStatesStmt = true + sc.IgnoreTruncate = true + sc.IgnoreZeroInDate = true + sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() default: sc.IgnoreTruncate = true sc.IgnoreZeroInDate = true @@ -1940,7 +1963,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.PrevLastInsertID = vars.StmtCtx.PrevLastInsertID } sc.PrevAffectedRows = 0 - if vars.StmtCtx.InUpdateStmt || vars.StmtCtx.InDeleteStmt || vars.StmtCtx.InInsertStmt { + if vars.StmtCtx.InUpdateStmt || vars.StmtCtx.InDeleteStmt || vars.StmtCtx.InInsertStmt || vars.StmtCtx.InSetSessionStatesStmt { sc.PrevAffectedRows = int64(vars.StmtCtx.AffectedRows()) } else if vars.StmtCtx.InSelectStmt { sc.PrevAffectedRows = -1 diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index a140809fd46d2..2a46243c6e6cc 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -785,11 +785,11 @@ func TestVectorizedMergeJoin(t *testing.T) { )).Check(testkit.Rows( fmt.Sprintf(`MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2), fmt.Sprintf(`├─Sort(Build) 3320.01 root test.%s.a`, t2), - fmt.Sprintf(`│ └─TableReader 3320.01 root data:Selection`), + `│ └─TableReader 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), fmt.Sprintf(`└─Sort(Probe) 3330.00 root test.%s.a`, t1), - fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`), + ` └─TableReader 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -797,10 +797,10 @@ func TestVectorizedMergeJoin(t *testing.T) { t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2), - fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`), + `├─TableReader(Build) 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), - fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`), + `└─TableReader(Probe) 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -903,14 +903,14 @@ func TestVectorizedShuffleMergeJoin(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5", t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( - fmt.Sprintf(`Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`), + `Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`, fmt.Sprintf(`└─MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2), fmt.Sprintf(` ├─Sort(Build) 3320.01 root test.%s.a`, t2), - fmt.Sprintf(` │ └─TableReader 3320.01 root data:Selection`), + ` │ └─TableReader 3320.01 root data:Selection`, fmt.Sprintf(` │ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(` │ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), fmt.Sprintf(` └─Sort(Probe) 3330.00 root test.%s.a`, t1), - fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`), + ` └─TableReader 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -918,10 +918,10 @@ func TestVectorizedShuffleMergeJoin(t *testing.T) { t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2), - fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`), + `├─TableReader(Build) 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), - fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`), + `└─TableReader(Probe) 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) diff --git a/executor/point_get.go b/executor/point_get.go index f33ba20b5dd5a..1b4d6666663b5 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -49,11 +49,19 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { return nil } - startTS, err := b.getSnapshotTS() + if p.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() if err != nil { b.err = err return nil } + e := &PointGetExecutor{ baseExecutor: newBaseExecutor(b.ctx, p.Schema(), p.ID()), readReplicaScope: b.readReplicaScope, @@ -61,14 +69,17 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { } if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(p.TblInfo, startTS) + e.cacheTable = b.getCacheTable(p.TblInfo, snapshotTS) } + e.base().initCap = 1 e.base().maxChunkSize = 1 - e.Init(p, startTS) + e.Init(p, snapshotTS) + if e.lock { b.hasLock = true } + return e } @@ -83,7 +94,7 @@ type PointGetExecutor struct { idxKey kv.Key handleVal []byte idxVals []types.Datum - startTS uint64 + snapshotTS uint64 readReplicaScope string isStaleness bool txn kv.Transaction @@ -106,13 +117,13 @@ type PointGetExecutor struct { } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field -func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, startTs uint64) { +func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, snapshotTS uint64) { decoder := NewRowDecoder(e.ctx, p.Schema(), p.TblInfo) e.tblInfo = p.TblInfo e.handle = p.Handle e.idxInfo = p.IndexInfo e.idxVals = p.IndexValues - e.startTS = startTs + e.snapshotTS = snapshotTS e.done = false if e.tblInfo.TempTableType == model.TempTableNone { e.lock = p.Lock @@ -142,10 +153,7 @@ func (e *PointGetExecutor) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *PointGetExecutor) Open(context.Context) error { txnCtx := e.ctx.GetSessionVars().TxnCtx - snapshotTS := e.startTS - if e.lock { - snapshotTS = txnCtx.GetForUpdateTS() - } + snapshotTS := e.snapshotTS var err error e.txn, err = e.ctx.Txn(false) if err != nil { @@ -381,9 +389,12 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro } if e.lock { seVars := e.ctx.GetSessionVars() - lockCtx := newLockCtx(seVars, e.lockWaitTime, 1) + lockCtx, err := newLockCtx(e.ctx, e.lockWaitTime, 1) + if err != nil { + return err + } lockCtx.InitReturnValues(1) - err := doLockKeys(ctx, e.ctx, lockCtx, key) + err = doLockKeys(ctx, e.ctx, lockCtx, key) if err != nil { return err } diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 62255326cbce0..51e50b670ade2 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -34,10 +34,8 @@ import ( func enableStaleReadCommonFailPoint(t *testing.T) func() { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) return func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan")) } } diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index fc5ee95b10655..d3c69b3988762 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -281,7 +281,7 @@ func TestMppExecution(t *testing.T) { require.NoError(t, err) ts := txn.StartTS() taskID := tk.Session().GetSessionVars().AllocMPPTaskID(ts) - require.Equal(t, int64(5), taskID) + require.Equal(t, int64(6), taskID) tk.MustExec("commit") taskID = tk.Session().GetSessionVars().AllocMPPTaskID(ts + 1) require.Equal(t, int64(1), taskID) diff --git a/executor/trace_test.go b/executor/trace_test.go index f8e8e91ddebd7..9b448670cc39a 100644 --- a/executor/trace_test.go +++ b/executor/trace_test.go @@ -33,7 +33,7 @@ func TestTraceExec(t *testing.T) { require.GreaterOrEqual(t, len(rows), 1) // +---------------------------+-----------------+------------+ - // | operation | startTS | duration | + // | operation | snapshotTS | duration | // +---------------------------+-----------------+------------+ // | session.getTxnFuture | 22:08:38.247834 | 78.909µs | // | ├─session.Execute | 22:08:38.247829 | 1.478487ms | diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 4979af65273bd..b585ba5f20c13 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -15,11 +15,13 @@ package executor_test import ( + "encoding/hex" "fmt" "testing" "time" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/benchdaily" @@ -456,6 +458,28 @@ func TestIssue28073(t *testing.T) { break } require.False(t, exist) + + // Another case, left join on partition table should not generate locks on physical ID = 0 + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str));") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int)) partition by hash (c_int) partitions 4;") + tk.MustExec("insert into t1 (`c_int`, `c_str`) values (1, 'upbeat solomon'), (5, 'sharp rubin');") + tk.MustExec("insert into t2 (`c_int`, `c_str`) values (1, 'clever haibt'), (4, 'kind margulis');") + tk.MustExec("begin pessimistic;") + tk.MustQuery("select * from t1 left join t2 on t1.c_int = t2.c_int for update;").Check(testkit.Rows( + "1 upbeat solomon 1 clever haibt", + "5 sharp rubin ", + )) + key, err := hex.DecodeString("7480000000000000005F728000000000000000") + require.NoError(t, err) + h := helper.NewHelper(store.(helper.Storage)) + resp, err := h.GetMvccByEncodedKey(key) + require.NoError(t, err) + require.Nil(t, resp.Info.Lock) + require.Len(t, resp.Info.Writes, 0) + require.Len(t, resp.Info.Values, 0) + + tk.MustExec("rollback;") } func TestIssue32422(t *testing.T) { diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 115559022279c..debeaf8d6399e 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -123,8 +123,6 @@ func (a *AggFuncDesc) Split(ordinal []int) (partialAggDesc, finalAggDesc *AggFun partialAggDesc.Mode = Partial1Mode } else if a.Mode == FinalMode { partialAggDesc.Mode = Partial2Mode - } else { - panic("Error happened during AggFuncDesc.Split, the AggFunctionMode is not CompleteMode or FinalMode.") } finalAggDesc = &AggFuncDesc{ Mode: FinalMode, // We only support FinalMode now in final phase. diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 3fc5c9a58adf0..6ef85a4b9c057 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -2540,21 +2540,21 @@ func (b *builtinNullEQIntSig) evalInt(row chunk.Row) (val int64, isNull bool, er case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case isUnsigned0 && isUnsigned1 && types.CompareUint64(uint64(arg0), uint64(arg1)) == 0: res = 1 case !isUnsigned0 && !isUnsigned1 && types.CompareInt64(arg0, arg1) == 0: res = 1 case isUnsigned0 && !isUnsigned1: if arg1 < 0 { - break + return res, false, nil } if types.CompareInt64(arg0, arg1) == 0 { res = 1 } case !isUnsigned0 && isUnsigned1: if arg0 < 0 { - break + return res, false, nil } if types.CompareInt64(arg0, arg1) == 0 { res = 1 @@ -2587,7 +2587,7 @@ func (b *builtinNullEQRealSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case types.CompareFloat64(arg0, arg1) == 0: res = 1 } @@ -2618,7 +2618,7 @@ func (b *builtinNullEQDecimalSig) evalInt(row chunk.Row) (val int64, isNull bool case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2649,7 +2649,7 @@ func (b *builtinNullEQStringSig) evalInt(row chunk.Row) (val int64, isNull bool, case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case types.CompareString(arg0, arg1, b.collation) == 0: res = 1 } @@ -2680,7 +2680,7 @@ func (b *builtinNullEQDurationSig) evalInt(row chunk.Row) (val int64, isNull boo case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2711,7 +2711,7 @@ func (b *builtinNullEQTimeSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2742,7 +2742,7 @@ func (b *builtinNullEQJSONSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil default: cmpRes := json.CompareBinary(arg0, arg1) if cmpRes == 0 { diff --git a/expression/expression.go b/expression/expression.go index ad434dc5e9b5d..9965ce0a7cecc 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -807,7 +807,7 @@ func evaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio for i, arg := range x.GetArgs() { args[i] = evaluateExprWithNull(ctx, schema, arg) } - return NewFunctionInternal(ctx, x.FuncName.L, x.RetType, args...) + return NewFunctionInternal(ctx, x.FuncName.L, x.RetType.Clone(), args...) case *Column: if !schema.Contains(x) { return x diff --git a/expression/expression_test.go b/expression/expression_test.go index 499f5c0bfea21..00a2c1bee360c 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -79,6 +79,30 @@ func TestEvaluateExprWithNullAndParameters(t *testing.T) { require.True(t, isScalarFunc) // the expression with parameters is not evaluated } +func TestEvaluateExprWithNullNoChangeRetType(t *testing.T) { + ctx := createContext(t) + tblInfo := newTestTableBuilder("").add("col_str", mysql.TypeString, 0).build() + schema := tableInfoToSchemaForTest(tblInfo) + + castStrAsJSON := BuildCastFunction(ctx, schema.Columns[0], types.NewFieldType(mysql.TypeJSON)) + jsonConstant := &Constant{Value: types.NewDatum("123"), RetType: types.NewFieldType(mysql.TypeJSON)} + + // initially has ParseToJSONFlag + flagInCast := castStrAsJSON.(*ScalarFunction).RetType.GetFlag() + require.True(t, mysql.HasParseToJSONFlag(flagInCast)) + + // cast's ParseToJSONFlag removed by `DisableParseJSONFlag4Expr` + eq, err := newFunctionForTest(ctx, ast.EQ, jsonConstant, castStrAsJSON) + require.NoError(t, err) + flagInCast = eq.(*ScalarFunction).GetArgs()[1].(*ScalarFunction).RetType.GetFlag() + require.False(t, mysql.HasParseToJSONFlag(flagInCast)) + + // after EvaluateExprWithNull, this flag should be still false + EvaluateExprWithNull(ctx, schema, eq) + flagInCast = eq.(*ScalarFunction).GetArgs()[1].(*ScalarFunction).RetType.GetFlag() + require.False(t, mysql.HasParseToJSONFlag(flagInCast)) +} + func TestConstant(t *testing.T) { ctx := createContext(t) sc := &stmtctx.StatementContext{TimeZone: time.Local} diff --git a/go.mod b/go.mod index 351a6ea1817cf..9ad9135be4085 100644 --- a/go.mod +++ b/go.mod @@ -98,6 +98,7 @@ require ( require ( github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 github.com/charithe/durationcheck v0.0.9 + github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a github.com/gordonklaus/ineffassign v0.0.0-20210914165742-4cc7213b9bc8 github.com/kyoh86/exportloopref v0.1.8 honnef.co/go/tools v0.0.1-2020.1.4 diff --git a/go.sum b/go.sum index 5bc93febdf2ab..2dcc2c3e4cfeb 100644 --- a/go.sum +++ b/go.sum @@ -343,6 +343,8 @@ github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a h1:iR3fYXUjHCR97qWS8ch1y9zPNsgXThGwjKPrYfqMPks= +github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a/go.mod h1:9qCChq59u/eW8im404Q2WWTrnBUQKjpNYKMbU4M7EFU= github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21 h1:leSNB7iYzLYSSx3J/s5sVf4Drkc68W2wm4Ixh/mr0us= github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21/go.mod h1:tf5+bzsHdTM0bsB7+8mt0GUMvjCgwLpTapNZHU8AajI= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= diff --git a/infoschema/cluster.go b/infoschema/cluster.go index d1d4f3c5a7fac..47bb7db4c3434 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -79,16 +79,14 @@ func isClusterTableByName(dbName, tableName string) bool { dbName = strings.ToUpper(dbName) switch dbName { case util.InformationSchemaName.O, util.PerformanceSchemaName.O: - break - default: - return false - } - tableName = strings.ToUpper(tableName) - for _, name := range memTableToClusterTables { - name = strings.ToUpper(name) - if name == tableName { - return true + tableName = strings.ToUpper(tableName) + for _, name := range memTableToClusterTables { + name = strings.ToUpper(name) + if name == tableName { + return true + } } + default: } return false } diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index fcb3dfa5b55ca..333ac46163b73 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "6.4.5" + "version": "7.5.11" }, { "type": "panel", @@ -51,12 +51,13 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": 28, - "iteration": 1617019008967, + "id": null, + "iteration": 1655990780337, "links": [], "panels": [ { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -72,13 +73,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB query durations by histogram buckets with different percents", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 1 }, + "hiddenSeries": false, "id": 80, "legend": { "alignAsTable": false, @@ -97,7 +104,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -185,7 +196,12 @@ "description": "MySQL commands processing numbers per second. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -193,6 +209,7 @@ "x": 12, "y": 1 }, + "hiddenSeries": false, "id": 42, "legend": { "alignAsTable": false, @@ -215,7 +232,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -306,7 +327,12 @@ "description": "TiDB statement statistics", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -314,6 +340,7 @@ "x": 0, "y": 7 }, + "hiddenSeries": false, "id": 21, "legend": { "alignAsTable": true, @@ -334,7 +361,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -410,7 +441,12 @@ "description": "TiDB command total statistics including both successful and failed ones", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -418,6 +454,7 @@ "x": 12, "y": 7 }, + "hiddenSeries": false, "id": 2, "legend": { "alignAsTable": true, @@ -440,7 +477,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -513,13 +554,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": null, "description": "TiDB failed query statistics by query type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 6, "w": 24, "x": 0, "y": 13 }, + "hiddenSeries": false, "id": 137, "legend": { "alignAsTable": true, @@ -539,7 +586,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -604,13 +655,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB slow query statistics with slow query durations and coprocessor waiting/executing durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, - "y": 25 + "y": 19 }, + "hiddenSeries": false, "id": 112, "legend": { "avg": false, @@ -625,7 +682,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -704,13 +765,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB connection idle durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, - "y": 25 + "y": 19 }, + "hiddenSeries": false, "id": 218, "legend": { "avg": false, @@ -725,7 +792,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -831,13 +902,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 99.9 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 25 }, + "hiddenSeries": false, "id": 136, "legend": { "avg": false, @@ -852,7 +929,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -917,13 +998,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 99 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, "y": 25 }, + "hiddenSeries": false, "id": 134, "legend": { "avg": false, @@ -938,7 +1025,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1003,13 +1094,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 95 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 31 }, + "hiddenSeries": false, "id": 132, "legend": { "avg": false, @@ -1024,7 +1121,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1089,13 +1190,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 80 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, "y": 31 }, + "hiddenSeries": false, "id": 130, "legend": { "avg": false, @@ -1110,7 +1217,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1175,6 +1286,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -1751,6 +1863,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -3337,11 +3450,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 3 }, "id": 141, "panels": [ @@ -5583,11 +5697,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 4 }, "id": 142, "panels": [ @@ -5601,14 +5716,20 @@ "description": "The time cost of parsing SQL to AST", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 82 + "y": 5 }, + "hiddenSeries": false, "id": 156, "interval": "", "legend": { @@ -5630,7 +5751,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5702,14 +5827,20 @@ "description": "The time cost of building the query plan", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 82 + "y": 5 }, + "hiddenSeries": false, "id": 154, "interval": "", "legend": { @@ -5731,7 +5862,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5803,14 +5938,20 @@ "description": "The time cost of executing the SQL which does not include the time to get the results of the query .", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 90 + "y": 13 }, + "hiddenSeries": false, "id": 169, "interval": "", "legend": { @@ -5832,7 +5973,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5904,14 +6049,20 @@ "description": "TiDB executors using more cpu and memory resources", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 90 + "y": 13 }, + "hiddenSeries": false, "id": 76, "legend": { "alignAsTable": true, @@ -5932,7 +6083,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6002,14 +6157,20 @@ "description": "TiDB plan cache hit total", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 98 + "y": 21 }, + "hiddenSeries": false, "id": 91, "legend": { "alignAsTable": true, @@ -6030,7 +6191,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6097,18 +6262,24 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "TiDB plan cache miss total", + "description": "TiDB plan cache hit total", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 98 + "y": 21 }, - "id": 249, + "hiddenSeries": false, + "id": 250, "legend": { "alignAsTable": true, "avg": true, @@ -6128,7 +6299,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6138,7 +6313,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_plan_cache_miss_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tidb_server_plan_cache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -6150,7 +6325,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Plan Cache Miss OPS", + "title": "Queries Using Plan Cache OPS", "tooltip": { "msResolution": false, "shared": true, @@ -6187,60 +6362,56 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Executor", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 6 - }, - "id": 143, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "durations of distsql execution by type", + "decimals": null, + "description": "TiDB read table cache hit total", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 122 + "y": 29 }, - "id": 12, + "hiddenSeries": false, + "id": 249, "legend": { "alignAsTable": true, - "avg": false, - "current": false, - "max": false, + "avg": true, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": null, + "sortDesc": null, "total": false, - "values": false + "values": true }, "lines": true, - "linewidth": 2, - "links": [ - { - "type": "dashboard" - } - ], + "linewidth": 1, + "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6250,22 +6421,139 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "999-{{type}}", - "refId": "D" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "exemplar": true, + "expr": "sum(rate(tidb_server_read_from_tablecache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": "99-{{type}}", - "metric": "tidb_distsql_handle_query_duration_seconds_bucket{}", + "legendFormat": "qps", "refId": "A", - "step": 4 - }, + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read From Table Cache OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:469", + "format": "short", + "label": null, + "logBase": 2, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:470", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "repeat": null, + "title": "Executor", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 143, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "durations of distsql execution by type", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 122 + }, + "id": 12, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [ + { + "url": "/" + } + ], + "nullPointMode": "null as zero", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "999-{{type}}", + "refId": "D" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "99-{{type}}", + "metric": "tidb_distsql_handle_query_duration_seconds_bucket{}", + "refId": "A", + "step": 4 + }, { "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", @@ -7007,11 +7295,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 6 }, "id": 144, "panels": [ @@ -7621,11 +7910,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 7 }, "id": 145, "panels": [ @@ -8097,55 +8387,35 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "KV Request", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 9 - }, - "id": 147, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "pd command count by type", - "editable": true, - "error": false, + "description": "TiDB successful region cache operations count", "fill": 1, - "grid": {}, "gridPos": { "h": 7, "w": 8, - "x": 0, - "y": 10 + "x": 16, + "y": 15 }, - "id": 20, + "id": 164, "legend": { - "alignAsTable": true, "avg": false, "current": false, "max": false, "min": false, - "rightSide": true, "show": true, "total": false, "values": false }, "lines": true, - "linewidth": 2, + "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "percentage": false, "pointradius": 5, "points": false, @@ -8156,24 +8426,23 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!=\"tso\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"ok\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", "refId": "A", - "step": 10 + "step": 30 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD Client CMD OPS", + "title": "Region Cache OK OPS", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -8212,25 +8481,20 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "pd client command durations by type within 99.9 percent buckets", - "editable": true, - "error": false, + "description": "TiDB error region cache operations count", "fill": 1, - "grid": {}, "gridPos": { "h": 7, "w": 8, - "x": 8, - "y": 10 + "x": 0, + "y": 22 }, - "id": 35, + "id": 250, "legend": { - "alignAsTable": true, "avg": false, "current": false, "max": false, "min": false, - "rightSide": true, "show": true, "total": false, "values": false @@ -8238,7 +8502,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "percentage": false, "pointradius": 5, "points": false, @@ -8249,38 +8513,23 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"err\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "999-{{type}}", + "legendFormat": "{{type}}-err", "refId": "A", - "step": 10 - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", 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(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "90-{{type}}", - "refId": "C" + "step": 30 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD Client CMD Duration", + "title": "Region Cache Error OPS", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -8292,7 +8541,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -8319,18 +8568,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "pd client command fail count by type", - "editable": true, - "error": false, + "description": "TiDB loading region cache durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, - "grid": {}, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, - "x": 16, - "y": 10 + "x": 8, + "y": 22 }, - "id": 43, + "hiddenSeries": false, + "id": 251, "legend": { "alignAsTable": true, "avg": false, @@ -8343,9 +8595,12 @@ "values": false }, "lines": true, - "linewidth": 2, + "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, "pointradius": 5, "points": false, @@ -8356,24 +8611,33 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_failed_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_load_region_cache_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "interval": "", "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "legendFormat": "99-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_load_region_cache_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type) / sum(rate(tidb_tikvclient_load_region_cache_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD Client CMD Fail OPS", + "title": "Load Region Duration", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -8385,7 +8649,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -8405,15 +8669,324 @@ "align": false, "alignLevel": null } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of a client calling GetTSAsync until received the TS result.", - "editable": true, + } + ], + "repeat": null, + "title": "KV Request", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 147, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "pd command count by type", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 10 + }, + "id": 20, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!=\"tso\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD Client CMD OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "pd client command durations by type within 99.9 percent buckets", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 10 + }, + "id": 35, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "999-{{type}}", + "refId": "A", + "step": 10 + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", 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(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "90-{{type}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD Client CMD Duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "pd client command fail count by type", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 10 + }, + "id": 43, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(pd_client_cmd_handle_failed_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD Client CMD Fail OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of a client calling GetTSAsync until received the TS result.", + "editable": true, "error": false, "fill": 1, "grid": {}, @@ -8824,8 +9397,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "It indicates if a request of PD client is forwarded by the PD follower", "decimals": null, + "description": "It indicates if a request of PD client is forwarded by the PD follower", "fill": 1, "gridPos": { "h": 7, @@ -9247,11 +9820,131 @@ "exemplar": true, "expr": "sum(rate(tidb_server_pd_api_request_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result!~\"200.*\"}[1m])) by (type, result)", "format": "time_series", - "hide": false, - "interval": "", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}} - {{result}}", + "refId": "B", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD HTTP Request Fail OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "repeat": null, + "title": "PD Client", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 148, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB loading schema time durations by instance", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 10 + }, + "hiddenSeries": false, + "id": 27, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, instance))", + "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{type}} - {{result}}", - "refId": "B", + "legendFormat": "{{instance}}", + "metric": "", + "refId": "A", "step": 10 } ], @@ -9259,12 +9952,12 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD HTTP Request Fail OPS", + "title": "Load Schema Duration", "tooltip": { "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -9276,7 +9969,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9296,22 +9989,7 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "PD Client", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 10 - }, - "id": 148, - "panels": [ + }, { "aliasColors": {}, "bars": false, @@ -9321,15 +9999,21 @@ "description": "TiDB loading schema time durations by instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 47 + "x": 12, + "y": 10 }, - "id": 27, + "hiddenSeries": false, + "id": 251, "legend": { "alignAsTable": true, "avg": false, @@ -9345,7 +10029,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -9412,22 +10100,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB loading schema times including both failed and successful ones", + "decimals": 2, + "description": "TiDB schema lease error counts", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 47 + "x": 0, + "y": 17 }, - "id": 28, + "hiddenSeries": false, + "id": 29, "legend": { "alignAsTable": true, "avg": false, "current": false, + "hideEmpty": true, + "hideZero": true, "max": false, "min": false, "rightSide": true, @@ -9439,26 +10136,25 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*failed/", - "bars": true - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_schema_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", + "expr": "sum(increase(tidb_session_schema_lease_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "metric": "tidb_domain_load_schema_duration_count", + "legendFormat": "{{instance}}", + "metric": "tidb_server_", "refId": "A", "step": 10 } @@ -9467,7 +10163,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Load Schema OPS", + "title": "Schema Lease Error OPM", "tooltip": { "msResolution": false, "shared": true, @@ -9486,9 +10182,9 @@ { "format": "short", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -9511,25 +10207,28 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 2, - "description": "TiDB schema lease error counts", + "description": "TiDB loading schema times including both failed and successful ones", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 54 + "x": 12, + "y": 17 }, - "id": 29, + "hiddenSeries": false, + "id": 28, "legend": { "alignAsTable": true, "avg": false, "current": false, - "hideEmpty": true, - "hideZero": true, "max": false, "min": false, "rightSide": true, @@ -9541,21 +10240,30 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*failed/", + "bars": true + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_session_schema_lease_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_domain_load_schema_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "tidb_server_", + "legendFormat": "{{instance}}-{{type}}", + "metric": "tidb_domain_load_schema_duration_count", "refId": "A", "step": 10 } @@ -9564,7 +10272,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Lease Error OPM", + "title": "Load Schema OPS", "tooltip": { "msResolution": false, "shared": true, @@ -9583,9 +10291,9 @@ { "format": "short", "label": null, - "logBase": 1, + "logBase": 10, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -9608,17 +10316,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB load privilege counts", + "description": "TiDB loading table cache time durations by instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 54 + "x": 0, + "y": 24 }, + "hiddenSeries": false, "id": 157, "legend": { "alignAsTable": true, @@ -9635,7 +10349,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -9650,10 +10368,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_privilege_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_load_table_cache_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, instance))", "format": "time_series", + "interval": "", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}", "metric": "tidb_domain_load_schema_duration_count", "refId": "A", "step": 10 @@ -9663,7 +10383,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Load Privilege OPS", + "title": "Load Data From Cached Table Duration", "tooltip": { "msResolution": false, "shared": true, @@ -9680,6 +10400,7 @@ }, "yaxes": [ { + "$$hashKey": "object:541", "format": "short", "label": null, "logBase": 10, @@ -9688,6 +10409,7 @@ "show": true }, { + "$$hashKey": "object:542", "format": "short", "label": null, "logBase": 1, @@ -9708,11 +10430,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 10 }, "id": 149, "panels": [ @@ -10635,11 +11358,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 11 }, "id": 150, "panels": [ @@ -11989,11 +12713,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 12 }, "id": 161, "panels": [ @@ -12183,11 +12908,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 13 }, "id": 151, "panels": [ @@ -12374,93 +13100,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB region cache operations count", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 21 - }, - "id": 164, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"err\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Region Cache Error OPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "aliasColors": {}, "bars": false, @@ -12472,8 +13111,8 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 21 + "x": 0, + "y": 42 }, "id": 52, "legend": { @@ -12556,11 +13195,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 14 }, "id": 152, "panels": [ @@ -13374,11 +14014,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 15 }, "id": 178, "panels": [ @@ -13782,6 +14423,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -14443,7 +15085,7 @@ } ], "refresh": "30s", - "schemaVersion": 18, + "schemaVersion": 27, "style": "dark", "tags": [], "templating": { @@ -14452,15 +15094,22 @@ "allValue": null, "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "K8s-cluster", "multi": false, "name": "k8s_cluster", "options": [], - "query": "label_values(pd_cluster_status, k8s_cluster)", + "query": { + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refId": "test-cluster-k8s_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", "tags": [], @@ -14472,15 +15121,22 @@ "allValue": null, "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "tidb_cluster", "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "query": { + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "refId": "test-cluster-tidb_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", "tags": [], diff --git a/owner/manager.go b/owner/manager.go index f90dd4cebdd2d..1f4eae6c11786 100644 --- a/owner/manager.go +++ b/owner/manager.go @@ -54,6 +54,9 @@ type Manager interface { Cancel() // RequireOwner requires the ownerManager is owner. RequireOwner(ctx context.Context) error + + // SetBeOwnerHook sets a hook. The hook is called before becoming an owner. + SetBeOwnerHook(hook func()) } const ( @@ -68,16 +71,17 @@ type DDLOwnerChecker interface { // ownerManager represents the structure which is used for electing owner. type ownerManager struct { - id string // id is the ID of the manager. - key string - ctx context.Context - prompt string - logPrefix string - logCtx context.Context - etcdCli *clientv3.Client - cancel context.CancelFunc - elec unsafe.Pointer - wg sync.WaitGroup + id string // id is the ID of the manager. + key string + ctx context.Context + prompt string + logPrefix string + logCtx context.Context + etcdCli *clientv3.Client + cancel context.CancelFunc + elec unsafe.Pointer + wg sync.WaitGroup + beOwnerHook func() } // NewOwnerManager creates a new Manager. @@ -117,6 +121,10 @@ func (m *ownerManager) RequireOwner(ctx context.Context) error { return nil } +func (m *ownerManager) SetBeOwnerHook(hook func()) { + m.beOwnerHook = hook +} + // ManagerSessionTTL is the etcd session's TTL in seconds. It's exported for testing. var ManagerSessionTTL = 60 @@ -166,6 +174,9 @@ func (m *ownerManager) ResignOwner(ctx context.Context) error { } func (m *ownerManager) toBeOwner(elec *concurrency.Election) { + if m.beOwnerHook != nil { + m.beOwnerHook() + } atomic.StorePointer(&m.elec, unsafe.Pointer(elec)) } diff --git a/owner/mock.go b/owner/mock.go index c13ff88f3fdf6..559c46650d080 100644 --- a/owner/mock.go +++ b/owner/mock.go @@ -27,9 +27,10 @@ var _ Manager = &mockManager{} // It's used for local store and testing. // So this worker will always be the owner. type mockManager struct { - owner int32 - id string // id is the ID of manager. - cancel context.CancelFunc + owner int32 + id string // id is the ID of manager. + cancel context.CancelFunc + beOwnerHook func() } // NewMockManager creates a new mock Manager. @@ -52,6 +53,9 @@ func (m *mockManager) IsOwner() bool { } func (m *mockManager) toBeOwner() { + if m.beOwnerHook != nil { + m.beOwnerHook() + } atomic.StoreInt32(&m.owner, 1) } @@ -91,3 +95,7 @@ func (m *mockManager) ResignOwner(ctx context.Context) error { func (m *mockManager) RequireOwner(context.Context) error { return nil } + +func (m *mockManager) SetBeOwnerHook(hook func()) { + m.beOwnerHook = hook +} diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 54b2734b8fde5..8ac6f0152749e 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -132,7 +132,7 @@ type CreateDatabaseStmt struct { ddlNode IfNotExists bool - Name string + Name model.CIStr Options []*DatabaseOption } @@ -142,7 +142,7 @@ func (n *CreateDatabaseStmt) Restore(ctx *format.RestoreCtx) error { if n.IfNotExists { ctx.WriteKeyWord("IF NOT EXISTS ") } - ctx.WriteName(n.Name) + ctx.WriteName(n.Name.O) for i, option := range n.Options { ctx.WritePlain(" ") err := option.Restore(ctx) @@ -168,7 +168,7 @@ func (n *CreateDatabaseStmt) Accept(v Visitor) (Node, bool) { type AlterDatabaseStmt struct { ddlNode - Name string + Name model.CIStr AlterDefaultDatabase bool Options []*DatabaseOption } @@ -191,7 +191,7 @@ func (n *AlterDatabaseStmt) Restore(ctx *format.RestoreCtx) error { ctx.WriteKeyWord("ALTER DATABASE") if !n.AlterDefaultDatabase { ctx.WritePlain(" ") - ctx.WriteName(n.Name) + ctx.WriteName(n.Name.O) } for i, option := range n.Options { ctx.WritePlain(" ") @@ -230,7 +230,7 @@ type DropDatabaseStmt struct { ddlNode IfExists bool - Name string + Name model.CIStr } // Restore implements Node interface. @@ -239,7 +239,7 @@ func (n *DropDatabaseStmt) Restore(ctx *format.RestoreCtx) error { if n.IfExists { ctx.WriteKeyWord("IF EXISTS ") } - ctx.WriteName(n.Name) + ctx.WriteName(n.Name.O) return nil } diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 3ec7d5c753ba8..38c465568634a 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -619,7 +619,6 @@ const ( func (n CompletionType) Restore(ctx *format.RestoreCtx) error { switch n { case CompletionTypeDefault: - break case CompletionTypeChain: ctx.WriteKeyWord(" AND CHAIN") case CompletionTypeRelease: diff --git a/parser/lexer.go b/parser/lexer.go index 659fc14dd9235..bcddc5f5bfcea 100644 --- a/parser/lexer.go +++ b/parser/lexer.go @@ -478,7 +478,6 @@ func startWithSlash(s *Scanner) (tok int, pos Pos, lit string) { } case 'M': // '/*M' maybe MariaDB-specific comments // no special treatment for now. - break case '+': // '/*+' optimizer hints // See https://dev.mysql.com/doc/refman/5.7/en/optimizer-hints.html @@ -502,7 +501,6 @@ func startWithSlash(s *Scanner) (tok int, pos Pos, lit string) { currentCharIsStar = true default: - break } // standard C-like comment. read until we see '*/' then drop it. @@ -574,7 +572,7 @@ func startWithAt(s *Scanner) (tok int, pos Pos, lit string) { tok, lit = doubleAtIdentifier, s.r.data(&pos) } case invalid: - break + return default: tok = singleAtIdentifier } diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 1769d3b526d7f..993c2c485f50c 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -202,9 +202,7 @@ func (h *HistoryInfo) AddTableInfo(schemaVer int64, tblInfo *TableInfo) { func (h *HistoryInfo) SetTableInfos(schemaVer int64, tblInfos []*TableInfo) { h.SchemaVersion = schemaVer h.MultipleTableInfos = make([]*TableInfo, len(tblInfos)) - for i, info := range tblInfos { - h.MultipleTableInfos[i] = info - } + copy(h.MultipleTableInfos, tblInfos) } // Clean cleans history information. @@ -270,6 +268,7 @@ type MultiSchemaInfo struct { AlterIndexes []CIStr `json:"-"` RelativeColumns []CIStr `json:"-"` + PositionColumns []CIStr `json:"-"` } func NewMultiSchemaInfo() *MultiSchemaInfo { @@ -292,6 +291,66 @@ type SubJob struct { CtxVars []interface{} `json:"-"` } +// IsNormal returns true if the sub-job is normally running. +func (sub *SubJob) IsNormal() bool { + switch sub.State { + case JobStateCancelling, JobStateCancelled, + JobStateRollingback, JobStateRollbackDone: + return false + default: + return true + } +} + +// IsFinished returns true if the job is done. +func (sub *SubJob) IsFinished() bool { + return sub.State == JobStateDone || + sub.State == JobStateRollbackDone || + sub.State == JobStateCancelled +} + +// ToProxyJob converts a sub-job to a proxy job. +func (sub *SubJob) ToProxyJob(parentJob *Job) *Job { + return &Job{ + ID: parentJob.ID, + Type: sub.Type, + SchemaID: parentJob.SchemaID, + TableID: parentJob.TableID, + SchemaName: parentJob.SchemaName, + State: sub.State, + Warning: sub.Warning, + Error: nil, + ErrorCount: 0, + RowCount: sub.RowCount, + Mu: sync.Mutex{}, + CtxVars: sub.CtxVars, + Args: sub.Args, + RawArgs: sub.RawArgs, + SchemaState: sub.SchemaState, + SnapshotVer: sub.SnapshotVer, + RealStartTS: parentJob.RealStartTS, + StartTS: parentJob.StartTS, + DependencyID: parentJob.DependencyID, + Query: parentJob.Query, + BinlogInfo: parentJob.BinlogInfo, + Version: parentJob.Version, + ReorgMeta: parentJob.ReorgMeta, + MultiSchemaInfo: &MultiSchemaInfo{Revertible: sub.Revertible}, + Priority: parentJob.Priority, + SeqNum: parentJob.SeqNum, + } +} + +func (sub *SubJob) FromProxyJob(proxyJob *Job) { + sub.Revertible = proxyJob.MultiSchemaInfo.Revertible + sub.SchemaState = proxyJob.SchemaState + sub.SnapshotVer = proxyJob.SnapshotVer + sub.Args = proxyJob.Args + sub.State = proxyJob.State + sub.Warning = proxyJob.Warning + sub.RowCount = proxyJob.RowCount +} + // Job is for a DDL operation. type Job struct { ID int64 `json:"id"` @@ -301,6 +360,7 @@ type Job struct { SchemaName string `json:"schema_name"` TableName string `json:"table_name"` State JobState `json:"state"` + Warning *terror.Error `json:"warning"` Error *terror.Error `json:"err"` // ErrorCount will be increased, every time we meet an error when running job. ErrorCount int64 `json:"err_count"` @@ -370,6 +430,14 @@ func (job *Job) FinishDBJob(jobState JobState, schemaState SchemaState, ver int6 job.BinlogInfo.AddDBInfo(ver, dbInfo) } +// MarkNonRevertible mark the current job to be non-revertible. +// It means the job cannot be cancelled or rollbacked. +func (job *Job) MarkNonRevertible() { + if job.MultiSchemaInfo != nil { + job.MultiSchemaInfo.Revertible = false + } +} + // TSConvert2Time converts timestamp to time. func TSConvert2Time(ts uint64) time.Time { t := int64(ts >> 18) // 18 is for the logical time. @@ -412,6 +480,18 @@ func (job *Job) Encode(updateRawArgs bool) ([]byte, error) { if err != nil { return nil, errors.Trace(err) } + if job.MultiSchemaInfo != nil { + for _, sub := range job.MultiSchemaInfo.SubJobs { + // Only update the args of executing sub-jobs. + if sub.Args == nil { + continue + } + sub.RawArgs, err = json.Marshal(sub.Args) + if err != nil { + return nil, errors.Trace(err) + } + } + } } var b []byte @@ -586,6 +666,8 @@ func (job *Job) IsRollbackable() bool { ActionModifySchemaCharsetAndCollate, ActionRepairTable, ActionModifyTableAutoIdCache, ActionModifySchemaDefaultPlacement: return job.SchemaState == StateNone + case ActionMultiSchemaChange: + return job.MultiSchemaInfo.Revertible } return true } diff --git a/parser/model/model_test.go b/parser/model/model_test.go index 73a6ec5e782e5..114b3a38bf5bf 100644 --- a/parser/model/model_test.go +++ b/parser/model/model_test.go @@ -18,6 +18,7 @@ import ( "fmt" "testing" "time" + "unsafe" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" @@ -578,3 +579,12 @@ func TestLocation(t *testing.T) { location := time.FixedZone("UTC", loc1.Offset) require.Equal(t, nLoc, location) } + +func TestDDLJobSize(t *testing.T) { + msg := `Please make sure that the following methods work as expected: +- SubJob.FromProxyJob() +- SubJob.ToProxyJob() +` + job := Job{} + require.Equal(t, 288, int(unsafe.Sizeof(job)), msg) +} diff --git a/parser/parser.go b/parser/parser.go index 6263a9718444b..051b5550596c1 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -13251,7 +13251,7 @@ yynewstate: case 330: { parser.yyVAL.statement = &ast.AlterDatabaseStmt{ - Name: yyS[yypt-1].ident, + Name: model.NewCIStr(yyS[yypt-1].ident), AlterDefaultDatabase: false, Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } @@ -13259,7 +13259,7 @@ yynewstate: case 331: { parser.yyVAL.statement = &ast.AlterDatabaseStmt{ - Name: "", + Name: model.NewCIStr(""), AlterDefaultDatabase: true, Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } @@ -13268,7 +13268,7 @@ yynewstate: { parser.yyVAL.statement = &ast.CreateDatabaseStmt{ IfNotExists: yyS[yypt-2].item.(bool), - Name: yyS[yypt-1].ident, + Name: model.NewCIStr(yyS[yypt-1].ident), Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } } @@ -13915,7 +13915,7 @@ yynewstate: } case 449: { - parser.yyVAL.statement = &ast.DropDatabaseStmt{IfExists: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident} + parser.yyVAL.statement = &ast.DropDatabaseStmt{IfExists: yyS[yypt-1].item.(bool), Name: model.NewCIStr(yyS[yypt-0].ident)} } case 450: { diff --git a/parser/parser.y b/parser/parser.y index 465a2c69101aa..131fa55aaaebd 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -3655,7 +3655,7 @@ AlterDatabaseStmt: "ALTER" DatabaseSym DBName DatabaseOptionList { $$ = &ast.AlterDatabaseStmt{ - Name: $3, + Name: model.NewCIStr($3), AlterDefaultDatabase: false, Options: $4.([]*ast.DatabaseOption), } @@ -3663,7 +3663,7 @@ AlterDatabaseStmt: | "ALTER" DatabaseSym DatabaseOptionList { $$ = &ast.AlterDatabaseStmt{ - Name: "", + Name: model.NewCIStr(""), AlterDefaultDatabase: true, Options: $3.([]*ast.DatabaseOption), } @@ -3685,7 +3685,7 @@ CreateDatabaseStmt: { $$ = &ast.CreateDatabaseStmt{ IfNotExists: $3.(bool), - Name: $4, + Name: model.NewCIStr($4), Options: $5.([]*ast.DatabaseOption), } } @@ -4463,7 +4463,7 @@ DatabaseSym: DropDatabaseStmt: "DROP" DatabaseSym IfExists DBName { - $$ = &ast.DropDatabaseStmt{IfExists: $3.(bool), Name: $4} + $$ = &ast.DropDatabaseStmt{IfExists: $3.(bool), Name: model.NewCIStr($4)} } /****************************************************************** diff --git a/parser/types/etc.go b/parser/types/etc.go index 2fe3d113e8820..1fdfeaf05367f 100644 --- a/parser/types/etc.go +++ b/parser/types/etc.go @@ -128,12 +128,8 @@ func TypeToStr(tp byte, cs string) (r string) { // Args: // ts: type string func StrToType(ts string) (tp byte) { - if strings.Contains(ts, "blob") { - ts = strings.Replace(ts, "blob", "text", 1) - } else if strings.Contains(ts, "binary") { - ts = strings.Replace(ts, "binary", "char", 1) - } - + ts = strings.Replace(ts, "blob", "text", 1) + ts = strings.Replace(ts, "binary", "char", 1) if tp, ok := str2Type[ts]; ok { return tp } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index f80ea20ad07c9..d662020cd50e4 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1199,6 +1199,10 @@ func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok } // SetCollationExpr sets the collation explicitly, even when the evaluation type of the expression is non-string. if _, ok := arg.(*expression.Column); ok { + if arg.GetType().GetType() == mysql.TypeEnum || arg.GetType().GetType() == mysql.TypeSet { + er.err = ErrNotSupportedYet.GenWithStackByArgs("use collate clause for enum or set") + break + } // Wrap a cast here to avoid changing the original FieldType of the column expression. exprType := arg.GetType().Clone() exprType.SetCollate(v.Collate) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index c0890093b0080..9fe9a7192cdcb 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -6717,3 +6717,22 @@ func TestIssue35083(t *testing.T) { "└─Projection 10000.00 cop[tikv] cast(test.t1.a, datetime BINARY)->Column#4", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) } + +func TestIssue25813(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a json);") + tk.MustExec("insert into t values('{\"id\": \"ish\"}');") + tk.MustQuery("select t2.a from t t1 left join t t2 on t1.a=t2.a where t2.a->'$.id'='ish';").Check(testkit.Rows("{\"id\": \"ish\"}")) + + tk.MustQuery("explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where t2.a->'$.id'='ish';").Check(testkit.Rows( + "Selection 8000.00 root eq(json_extract(test.t.a, \"$.id\"), cast(\"ish\", json BINARY))", + "└─HashJoin 10000.00 root left outer join, equal:[eq(test.t.a, test.t.a)]", + " ├─TableReader(Build) 8000.00 root data:Selection", + " │ └─Selection 8000.00 cop[tikv] not(isnull(cast(test.t.a, var_string(4294967295))))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index e868160af9800..fde76b3a41eec 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -326,14 +326,20 @@ func refineCETrace(sctx sessionctx.Context) { return i.RowCount < j.RowCount }) traceRecords := stmtCtx.OptimizerCETrace - is := sctx.GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) for _, rec := range traceRecords { tbl, ok := is.TableByID(rec.TableID) - if !ok { - logutil.BgLogger().Warn("[OptimizerTrace] Failed to find table in infoschema", - zap.Int64("table id", rec.TableID)) + if ok { + rec.TableName = tbl.Meta().Name.O + continue + } + tbl, _, _ = is.FindTableByPartitionID(rec.TableID) + if tbl != nil { + rec.TableName = tbl.Meta().Name.O + continue } - rec.TableName = tbl.Meta().Name.O + logutil.BgLogger().Warn("[OptimizerTrace] Failed to find table in infoschema", + zap.Int64("table id", rec.TableID)) } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 9f5721049c166..c5c43df0a0a91 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1077,7 +1077,7 @@ type basePhysicalAgg struct { MppPartitionCols []*property.MPPPartitionColumn } -func (p *basePhysicalAgg) isFinalAgg() bool { +func (p *basePhysicalAgg) IsFinalAgg() bool { if len(p.AggFuncs) > 0 { if p.AggFuncs[0].Mode == aggregation.FinalMode || p.AggFuncs[0].Mode == aggregation.CompleteMode { return true diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index 16d3478e49679..73758b562536a 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -1208,9 +1208,13 @@ func (p *PhysicalExchangeReceiver) GetPlanCost(taskType property.TaskType, costF return 0, err } p.planCost = childCost - // accumulate net cost: rows * row-size * net-factor - rowSize := getTblStats(p.children[0]).GetAvgRowSize(p.ctx, p.children[0].Schema().Columns, false, false) - p.planCost += getCardinality(p.children[0], costFlag) * rowSize * p.ctx.GetSessionVars().GetNetworkFactor(nil) + // accumulate net cost + if p.ctx.GetSessionVars().CostModelVersion == modelVer1 { + p.planCost += getCardinality(p.children[0], costFlag) * p.ctx.GetSessionVars().GetNetworkFactor(nil) + } else { // to avoid regression, only consider row-size on model ver2 + rowSize := getTblStats(p.children[0]).GetAvgRowSize(p.ctx, p.children[0].Schema().Columns, false, false) + p.planCost += getCardinality(p.children[0], costFlag) * rowSize * p.ctx.GetSessionVars().GetNetworkFactor(nil) + } p.planCostInit = true return p.planCost, nil } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index ad8388f15a8f4..b150788d36a56 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -4159,16 +4159,16 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err switch v := node.(type) { case *ast.AlterDatabaseStmt: if v.AlterDefaultDatabase { - v.Name = b.ctx.GetSessionVars().CurrentDB + v.Name = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB) } - if v.Name == "" { + if v.Name.O == "" { return nil, ErrNoDB } if b.ctx.GetSessionVars().User != nil { authErr = ErrDBaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Name) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Name, "", "", authErr) + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Name.L, "", "", authErr) case *ast.AlterTableStmt: if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername, @@ -4246,7 +4246,7 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err authErr = ErrDBaccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Name) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.Name, + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.Name.L, "", "", authErr) case *ast.CreateIndexStmt: if b.ctx.GetSessionVars().User != nil { @@ -4340,7 +4340,7 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err authErr = ErrDBaccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Name) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Name, + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Name.L, "", "", authErr) case *ast.DropIndexStmt: if b.ctx.GetSessionVars().User != nil { diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 6bdaf43b461e3..df47167354c2a 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -716,20 +716,20 @@ func (p *preprocessor) checkSetOprSelectList(stmt *ast.SetOprSelectList) { } func (p *preprocessor) checkCreateDatabaseGrammar(stmt *ast.CreateDatabaseStmt) { - if isIncorrectName(stmt.Name) { + if isIncorrectName(stmt.Name.L) { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } func (p *preprocessor) checkAlterDatabaseGrammar(stmt *ast.AlterDatabaseStmt) { // for 'ALTER DATABASE' statement, database name can be empty to alter default database. - if isIncorrectName(stmt.Name) && !stmt.AlterDefaultDatabase { + if isIncorrectName(stmt.Name.L) && !stmt.AlterDefaultDatabase { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) { - if isIncorrectName(stmt.Name) { + if isIncorrectName(stmt.Name.L) { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 826320ffdee71..11508a6ef0688 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -438,6 +438,16 @@ func (a *aggregationPushDownSolver) tryAggPushDownForUnion(union *LogicalUnionAl if pushedAgg == nil { return nil } + + // Update the agg mode for the pushed down aggregation. + for _, aggFunc := range pushedAgg.AggFuncs { + if aggFunc.Mode == aggregation.CompleteMode { + aggFunc.Mode = aggregation.Partial1Mode + } else if aggFunc.Mode == aggregation.FinalMode { + aggFunc.Mode = aggregation.Partial2Mode + } + } + newChildren := make([]LogicalPlan, 0, len(union.Children())) for _, child := range union.Children() { newChild, err := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 5516a242b486d..70a55f7e4e339 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -49,14 +49,14 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { // passing down the aggregation mode to TiFlash. if physicalAgg, ok := p.Children()[0].(*PhysicalHashAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } } if physicalAgg, ok := p.Children()[0].(*PhysicalStreamAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } diff --git a/planner/core/task.go b/planner/core/task.go index 26839811520c9..2f6d853f6b382 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1337,7 +1337,15 @@ func BuildFinalModeAggregation( finalAggFunc.OrderByItems = byItems finalAggFunc.HasDistinct = aggFunc.HasDistinct - finalAggFunc.Mode = aggregation.CompleteMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.CompleteMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } else { if aggFunc.Name == ast.AggFuncGroupConcat && len(aggFunc.OrderByItems) > 0 { // group_concat can only run in one phase if it has order by items but without distinct property @@ -1417,7 +1425,15 @@ func BuildFinalModeAggregation( } } - finalAggFunc.Mode = aggregation.FinalMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.FinalMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } finalAggFunc.Args = args @@ -1483,7 +1499,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { } // no avgs // for final agg, always add project due to in-compatibility between TiDB and TiFlash - if len(p.schema.Columns) == len(newSchema.Columns) && !p.isFinalAgg() { + if len(p.schema.Columns) == len(newSchema.Columns) && !p.IsFinalAgg() { return nil } // add remaining columns to exprs @@ -2070,8 +2086,7 @@ func (t *mppTask) enforceExchangerImpl(prop *property.PhysicalProperty) *mppTask sender.SetChildren(t.p) receiver := PhysicalExchangeReceiver{}.Init(ctx, t.p.statsInfo()) receiver.SetChildren(sender) - rowSize := getTblStats(sender.children[0]).GetAvgRowSize(sender.ctx, sender.children[0].Schema().Columns, false, false) - cst := t.cst + t.count()*rowSize*ctx.GetSessionVars().GetNetworkFactor(nil) + cst := t.cst + t.count()*ctx.GetSessionVars().GetNetworkFactor(nil) sender.cost = cst receiver.cost = cst return &mppTask{ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 8ce669292e9c7..f3f200872b93e 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -2389,12 +2389,12 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_24 3.00 5.05 root data:ExchangeSender_23", - "└─ExchangeSender_23 3.00 98.18 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_22 3.00 98.18 mpp[tiflash] Column#4", - " └─HashAgg_8 3.00 78.38 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_21 3.00 69.38 mpp[tiflash] ", - " └─ExchangeSender_20 3.00 69.38 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + "TableReader_24 3.00 4.16 root data:ExchangeSender_23", + "└─ExchangeSender_23 3.00 76.80 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_22 3.00 76.80 mpp[tiflash] Column#4", + " └─HashAgg_8 3.00 57.00 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_21 3.00 48.00 mpp[tiflash] ", + " └─ExchangeSender_20 3.00 48.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", " └─TableFullScan_19 3.00 45.00 mpp[tiflash] table:t2 keep order:false" ] }, @@ -2439,12 +2439,12 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_14 1.00 19.82 root funcs:count(1)->Column#7", - "└─TableReader_46 3.00 10.82 root data:ExchangeSender_45", - " └─ExchangeSender_45 3.00 216.75 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_42 3.00 216.75 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_21(Build) 3.00 78.38 mpp[tiflash] ", - " │ └─ExchangeSender_20 3.00 78.38 mpp[tiflash] ExchangeType: Broadcast", + "StreamAgg_14 1.00 18.93 root funcs:count(1)->Column#7", + "└─TableReader_46 3.00 9.93 root data:ExchangeSender_45", + " └─ExchangeSender_45 3.00 195.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_42 3.00 195.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_21(Build) 3.00 57.00 mpp[tiflash] ", + " │ └─ExchangeSender_20 3.00 57.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_19 3.00 54.00 mpp[tiflash] not(isnull(test.t1.a))", " │ └─TableFullScan_18 3.00 45.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_23(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", @@ -2454,15 +2454,15 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 62.51 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 53.51 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "StreamAgg_15 1.00 60.60 root funcs:count(1)->Column#10", + "└─HashJoin_65 3.00 51.60 root inner join, equal:[eq(test.t1.b, test.t3.b)]", " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", " │ └─IndexFullScan_52 3.00 130.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 13.05 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 250.13 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 250.13 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 111.75 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 111.75 mpp[tiflash] ExchangeType: Broadcast", + " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 204.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 204.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 66.00 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 66.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_33 3.00 63.00 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " │ └─TableFullScan_32 3.00 54.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_37(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", @@ -3739,19 +3739,18 @@ "TableReader 2.00 root data:ExchangeSender", "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 2.00 mpp[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 2.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#22)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─HashAgg 2.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", " └─ExchangeReceiver 2.00 mpp[tiflash] ", " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─HashAgg 2.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#22", - " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", - " │ └─TableFullScan 2.00 mpp[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", - " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#15, collate: binary]", - " └─Projection 8.00 mpp[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#14, cast(test.t2.v2, decimal(20,2))->Column#15", - " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", + " │ └─TableFullScan 2.00 mpp[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary], [name: Column#13, collate: binary]", + " └─Projection 8.00 mpp[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#12, cast(test.t2.v2, decimal(20,2))->Column#13", + " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -3760,18 +3759,17 @@ "TableReader 1.00 root data:ExchangeSender", "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 mpp[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#16)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", " └─ExchangeReceiver 1.00 mpp[tiflash] ", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#16", - " └─HashJoin 1.00 mpp[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", - " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", - " │ └─TableFullScan 1.00 mpp[tiflash] table:t3 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", - " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" + " └─HashJoin 1.00 mpp[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", + " │ └─TableFullScan 1.00 mpp[tiflash] table:t3 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" ] } ] @@ -5302,24 +5300,27 @@ { "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", "Plan": [ - "HashAgg 8000.00 root group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", - " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", - " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", - " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", - " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: utf8mb4_bin]", - " │ └─Projection 9990.00 mpp[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_key))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", - " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin]", - " └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_64))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo" + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin], [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", + " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: utf8mb4_bin]", + " │ └─Projection 9990.00 mpp[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] @@ -5385,12 +5386,15 @@ "SQL": "desc format = 'brief' select count(*), id + 1 from t group by id + 1", "Plan": [ "Projection 8000.00 root Column#4, plus(test.t.id, 1)->Column#5", - "└─HashAgg 8000.00 root group by:Column#13, funcs:count(Column#14)->Column#4, funcs:firstrow(Column#15)->test.t.id", - " └─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:count(1)->Column#14, funcs:firstrow(Column#16)->Column#15", - " └─Projection 10000.00 mpp[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:sum(Column#11)->Column#4, funcs:firstrow(Column#12)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:count(1)->Column#11, funcs:firstrow(Column#16)->Column#12", + " └─Projection 10000.00 mpp[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5518,11 +5522,10 @@ " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 mpp[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", @@ -5586,13 +5589,12 @@ "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", - " │ └─Projection 9990.00 mpp[tiflash] div(1, test.t.value)->Column#18, test.t.id", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#20, funcs:count(Column#18)->Column#7, funcs:firstrow(Column#19)->test.t.id", + " │ └─Projection 9990.00 mpp[tiflash] div(1, test.t.value)->Column#18, test.t.id, test.t.id", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", @@ -5602,14 +5604,18 @@ { "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", "Plan": [ - "HashAgg 6400.00 root group by:Column#20, funcs:sum(Column#19)->Column#4", - "└─Projection 6400.00 root cast(test.t.id, decimal(10,0) BINARY)->Column#19, test.t.value", - " └─HashAgg 6400.00 root group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " └─TableReader 6400.00 root data:ExchangeSender", - " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, test.t.value, ", - " └─Selection 8000.00 mpp[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "TableReader 6400.00 root data:ExchangeSender", + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#4", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#20, funcs:sum(Column#19)->Column#4", + " └─Projection 6400.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#19, test.t.value", + " └─Projection 6400.00 mpp[tiflash] test.t.id, test.t.value", + " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " └─ExchangeReceiver 6400.00 mpp[tiflash] ", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", + " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, test.t.value, ", + " └─Selection 8000.00 mpp[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5752,18 +5758,20 @@ { "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", "Plan": [ - "HashAgg 7992.00 root group by:test.t.id, funcs:sum(Column#9)->Column#7", - "└─TableReader 7992.00 root data:ExchangeSender", - " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 7992.00 mpp[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#9", + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 7992.00 mpp[tiflash] Column#7", + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", " └─Projection 12487.50 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", - " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeReceiver 12487.50 mpp[tiflash] ", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5776,18 +5784,16 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#7, test.t.id", - " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", - " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5800,28 +5806,26 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", - " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#17)->Column#11, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 7992.00 mpp[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", - " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", - " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 mpp[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:count(1)->Column#13", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#34, funcs:sum(Column#32)->Column#11, funcs:firstrow(Column#33)->test.t.id", + " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id, test.t.id", + " └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.id, Column#13", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:count(1)->Column#13", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6165,12 +6169,11 @@ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 mpp[tiflash] ", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", - " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6687,11 +6690,10 @@ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 mpp[tiflash] ", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#9", - " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#5", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6703,12 +6705,11 @@ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 mpp[tiflash] ", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", - " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", - " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6808,11 +6809,10 @@ "└─TableReader 8000.00 root data:ExchangeSender", " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", - " └─ExchangeReceiver 8000.00 mpp[tiflash] ", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#8", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#5", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6843,11 +6843,10 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", - " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] diff --git a/planner/core/testdata/window_push_down_suite_out.json b/planner/core/testdata/window_push_down_suite_out.json index e537e05904ce3..2b7b7b893cda4 100644 --- a/planner/core/testdata/window_push_down_suite_out.json +++ b/planner/core/testdata/window_push_down_suite_out.json @@ -340,12 +340,11 @@ "TableReader_84 8000.00 root data:ExchangeSender_83", "└─ExchangeSender_83 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Window_82 8000.00 mpp[tiflash] row_number()->Column#7 over(rows between current row and current row)", - " └─Projection_21 8000.00 mpp[tiflash] Column#5", - " └─HashAgg_22 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:sum(Column#8)->Column#5", - " └─ExchangeReceiver_24 8000.00 mpp[tiflash] ", - " └─ExchangeSender_23 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─HashAgg_13 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#8", - " └─TableFullScan_20 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Projection_19 8000.00 mpp[tiflash] Column#5", + " └─HashAgg_12 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#5", + " └─ExchangeReceiver_18 10000.00 mpp[tiflash] ", + " └─ExchangeSender_17 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─TableFullScan_16 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null }, @@ -373,13 +372,12 @@ "TableReader_61 8000.00 root data:ExchangeSender_60", "└─ExchangeSender_60 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Window_59 8000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.employee.deptid rows between current row and current row)", - " └─Sort_24 8000.00 mpp[tiflash] test.employee.deptid", - " └─Projection_20 8000.00 mpp[tiflash] Column#5, test.employee.deptid", - " └─HashAgg_21 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:sum(Column#8)->Column#5, funcs:firstrow(test.employee.deptid)->test.employee.deptid", - " └─ExchangeReceiver_23 8000.00 mpp[tiflash] ", - " └─ExchangeSender_22 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─HashAgg_13 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#8", - " └─TableFullScan_19 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Sort_18 8000.00 mpp[tiflash] test.employee.deptid", + " └─Projection_17 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " └─HashAgg_12 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#5, funcs:firstrow(test.employee.deptid)->test.employee.deptid", + " └─ExchangeReceiver_16 10000.00 mpp[tiflash] ", + " └─ExchangeSender_15 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null }, @@ -434,16 +432,15 @@ "Plan": [ "TableReader_54 10000.00 root data:ExchangeSender_53", "└─ExchangeSender_53 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_49 10000.00 mpp[tiflash] Column#7", - " └─HashAgg_50 10000.00 mpp[tiflash] group by:Column#6, funcs:sum(Column#10)->Column#7", - " └─ExchangeReceiver_52 10000.00 mpp[tiflash] ", - " └─ExchangeSender_51 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", - " └─HashAgg_47 10000.00 mpp[tiflash] group by:Column#6, funcs:count(test.employee.empid)->Column#10", - " └─Window_36 10000.00 mpp[tiflash] row_number()->Column#6 over(partition by test.employee.deptid rows between current row and current row)", - " └─Sort_21 10000.00 mpp[tiflash] test.employee.deptid", - " └─ExchangeReceiver_20 10000.00 mpp[tiflash] ", - " └─ExchangeSender_19 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─TableFullScan_18 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Projection_48 10000.00 mpp[tiflash] Column#7", + " └─HashAgg_46 10000.00 mpp[tiflash] group by:Column#6, funcs:count(test.employee.empid)->Column#7", + " └─ExchangeReceiver_32 10000.00 mpp[tiflash] ", + " └─ExchangeSender_31 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", + " └─Window_30 10000.00 mpp[tiflash] row_number()->Column#6 over(partition by test.employee.deptid rows between current row and current row)", + " └─Sort_21 10000.00 mpp[tiflash] test.employee.deptid", + " └─ExchangeReceiver_20 10000.00 mpp[tiflash] ", + " └─ExchangeSender_19 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null } diff --git a/server/server.go b/server/server.go index 13ed052391f59..63406905b5a3b 100644 --- a/server/server.go +++ b/server/server.go @@ -50,6 +50,7 @@ import ( "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" @@ -718,6 +719,25 @@ func killConn(conn *clientConn) { conn.mu.RLock() cancelFunc := conn.mu.cancelFunc conn.mu.RUnlock() + + // If the connection being killed is a DDL Job, + // we need to CANCEL the matching jobID first. + if sessVars.StmtCtx.IsDDLJobInQueue { + jobID := sessVars.StmtCtx.DDLJobID + err := kv.RunInNewTxn(context.Background(), conn.ctx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { + // errs is the error per job, there is only one submitted + // err is the error of the overall task + errs, err := ddl.CancelJobs(txn, []int64{jobID}) + if len(errs) > 0 { + logutil.BgLogger().Warn("error canceling DDL job", zap.Error(errs[0])) + } + return err + }) + if err != nil { + logutil.BgLogger().Warn("could not cancel DDL job", zap.Error(err)) + } + } + if cancelFunc != nil { cancelFunc() } diff --git a/session/BUILD.bazel b/session/BUILD.bazel index a48f8238cc5ca..91e160985bfd8 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -49,7 +49,6 @@ go_library( "//sessionctx/variable", "//sessiontxn", "//sessiontxn/isolation", - "//sessiontxn/legacy", "//sessiontxn/staleread", "//statistics", "//statistics/handle", diff --git a/session/session.go b/session/session.go index c1f7e1303385f..5d6a50246bce1 100644 --- a/session/session.go +++ b/session/session.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/sessionstates" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/helper" @@ -2384,7 +2383,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } - var is infoschema.InfoSchema var snapshotTS uint64 replicaReadScope := oracle.GlobalTxnScope @@ -2396,7 +2394,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ txnManager := sessiontxn.GetTxnManager(s) if staleReadProcessor.IsStaleness() { snapshotTS = staleReadProcessor.GetStalenessReadTS() - is = staleReadProcessor.GetStalenessInfoSchema() + is := staleReadProcessor.GetStalenessInfoSchema() replicaReadScope = config.GetTxnScopeFromConfig() err = txnManager.EnterNewTxn(ctx, &sessiontxn.EnterNewTxnRequest{ Type: sessiontxn.EnterNewTxnWithReplaceProvider, @@ -2406,8 +2404,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if err != nil { return nil, err } - } else { - is = s.GetInfoSchema().(infoschema.InfoSchema) } staleness := snapshotTS > 0 @@ -2428,10 +2424,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, err } - if p, isOK := txnManager.GetContextProvider().(*legacy.SimpleTxnContextProvider); isOK { - p.InfoSchema = is - } - if ok { rs, ok, err := s.cachedPointPlanExec(ctx, txnManager.GetTxnInfoSchema(), execStmt, preparedStmt, replicaReadScope, args) if err != nil { @@ -3544,15 +3536,16 @@ func (s *session) EncodeSessionStates(ctx context.Context, sctx sessionctx.Conte // DecodeSessionStates implements SessionStatesHandler.DecodeSessionStates interface. func (s *session) DecodeSessionStates(ctx context.Context, sctx sessionctx.Context, sessionStates *sessionstates.SessionStates) (err error) { - if err = s.sessionVars.DecodeSessionStates(ctx, sessionStates); err != nil { - return err - } - // Decode session variables. for name, val := range sessionStates.SystemVars { if err = variable.SetSessionSystemVar(s.sessionVars, name, val); err != nil { return err } } + + // Decode stmt ctx after session vars because setting session vars may override stmt ctx, such as warnings. + if err = s.sessionVars.DecodeSessionStates(ctx, sessionStates); err != nil { + return err + } return err } diff --git a/sessionctx/sessionstates/BUILD.bazel b/sessionctx/sessionstates/BUILD.bazel index ba5cb9254f9f8..0b0d2c6d1c6e8 100644 --- a/sessionctx/sessionstates/BUILD.bazel +++ b/sessionctx/sessionstates/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//parser/types", + "//sessionctx/stmtctx", "//types", ], ) diff --git a/sessionctx/sessionstates/session_states.go b/sessionctx/sessionstates/session_states.go index baf876ff87b4f..10a2756dd04f4 100644 --- a/sessionctx/sessionstates/session_states.go +++ b/sessionctx/sessionstates/session_states.go @@ -18,6 +18,7 @@ import ( "time" ptypes "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" ) @@ -52,4 +53,7 @@ type SessionStates struct { FoundInBinding bool `json:"in-binding,omitempty"` SequenceLatestValues map[int64]int64 `json:"seq-values,omitempty"` MPPStoreLastFailTime map[string]time.Time `json:"store-fail-time,omitempty"` + LastAffectedRows int64 `json:"affected-rows,omitempty"` + LastInsertID uint64 `json:"last-insert-id,omitempty"` + Warnings []stmtctx.SQLWarn `json:"warnings,omitempty"` } diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 847f50f4e9a2b..29101af06f392 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -435,6 +435,125 @@ func TestSessionCtx(t *testing.T) { } } +func TestStatementCtx(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.t1(id int auto_increment primary key, str char(1))") + + tests := []struct { + setFunc func(tk *testkit.TestKit) any + checkFunc func(tk *testkit.TestKit, param any) + }{ + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("show warnings") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("0")) + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("select 1") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("insert into test.t1(str) value('a'), ('b'), ('c')") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("3")) + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastInsertID + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("0")) + }, + }, + { + // check LastInsertID + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("insert into test.t1(str) value('d')") + rows := tk.MustQuery("select @@last_insert_id").Rows() + require.NotEqual(t, "0", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_insert_id").Check(param.([][]any)) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("select 1") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show errors").Check(testkit.Rows()) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show errors").Check(testkit.Rows()) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("0 0")) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustGetErrCode("insert into test.t1(str) value('ef')", errno.ErrDataTooLong) + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, 1, len(rows)) + tk.MustQuery("show errors").Check(rows) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(param.([][]any)) + tk.MustQuery("show errors").Check(param.([][]any)) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("1 1")) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("set sql_mode=''") + tk.MustExec("insert into test.t1(str) value('ef'), ('ef')") + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, 2, len(rows)) + tk.MustQuery("show errors").Check(testkit.Rows()) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(param.([][]any)) + tk.MustQuery("show errors").Check(testkit.Rows()) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("2 0")) + }, + }, + } + + for _, tt := range tests { + tk1 := testkit.NewTestKit(t, store) + var param any + if tt.setFunc != nil { + param = tt.setFunc(tk1) + } + tk2 := testkit.NewTestKit(t, store) + showSessionStatesAndSet(t, tk1, tk2) + tt.checkFunc(tk2, param) + } +} + func showSessionStatesAndSet(t *testing.T, tk1, tk2 *testkit.TestKit) { rows := tk1.MustQuery("show session_states").Rows() require.Len(t, rows, 1) diff --git a/sessionctx/stmtctx/BUILD.bazel b/sessionctx/stmtctx/BUILD.bazel index 168b86f32cf62..c59196a17766c 100644 --- a/sessionctx/stmtctx/BUILD.bazel +++ b/sessionctx/stmtctx/BUILD.bazel @@ -10,12 +10,14 @@ go_library( "//parser/ast", "//parser/model", "//parser/mysql", + "//parser/terror", "//util/disk", "//util/execdetails", "//util/memory", "//util/resourcegrouptag", "//util/topsql/stmtstats", "//util/tracing", + "@com_github_pingcap_errors//:errors", "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//util", "@org_uber_go_atomic//:atomic", @@ -32,9 +34,11 @@ go_test( embed = [":stmtctx"], deps = [ "//kv", + "//sessionctx/variable", "//testkit", "//testkit/testsetup", "//util/execdetails", + "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//util", "@org_uber_go_goleak//:goleak", diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 70cd4bec5f898..6d5041d2cdfbf 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -15,6 +15,7 @@ package stmtctx import ( + "encoding/json" "math" "sort" "strconv" @@ -22,10 +23,12 @@ import ( "sync/atomic" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" @@ -60,6 +63,43 @@ type SQLWarn struct { Err error } +type jsonSQLWarn struct { + Level string `json:"level"` + SQLErr *terror.Error `json:"err,omitempty"` + Msg string `json:"msg,omitempty"` +} + +// MarshalJSON implements the Marshaler.MarshalJSON interface. +func (warn *SQLWarn) MarshalJSON() ([]byte, error) { + w := &jsonSQLWarn{ + Level: warn.Level, + } + e := errors.Cause(warn.Err) + switch x := e.(type) { + case *terror.Error: + // Omit outter errors because only the most inner error matters. + w.SQLErr = x + default: + w.Msg = e.Error() + } + return json.Marshal(w) +} + +// UnmarshalJSON implements the Unmarshaler.UnmarshalJSON interface. +func (warn *SQLWarn) UnmarshalJSON(data []byte) error { + var w jsonSQLWarn + if err := json.Unmarshal(data, &w); err != nil { + return err + } + warn.Level = w.Level + if w.SQLErr != nil { + warn.Err = w.SQLErr + } else { + warn.Err = errors.New(w.Msg) + } + return nil +} + // StatementContext contains variables for a statement. // It should be reset before executing a statement. type StatementContext struct { @@ -69,6 +109,7 @@ type StatementContext struct { // IsDDLJobInQueue is used to mark whether the DDL job is put into the queue. // If IsDDLJobInQueue is true, it means the DDL job is in the queue of storage, and it can be handled by the DDL worker. IsDDLJobInQueue bool + DDLJobID int64 InInsertStmt bool InUpdateStmt bool InDeleteStmt bool @@ -76,6 +117,7 @@ type StatementContext struct { InLoadDataStmt bool InExplainStmt bool InCreateOrAlterStmt bool + InSetSessionStatesStmt bool InPreparedPlanBuilding bool IgnoreTruncate bool IgnoreZeroInDate bool @@ -406,6 +448,13 @@ func (sc *StatementContext) AddAffectedRows(rows uint64) { sc.mu.affectedRows += rows } +// SetAffectedRows sets affected rows. +func (sc *StatementContext) SetAffectedRows(rows uint64) { + sc.mu.Lock() + sc.mu.affectedRows = rows + sc.mu.Unlock() +} + // AffectedRows gets affected rows. func (sc *StatementContext) AffectedRows() uint64 { sc.mu.Lock() @@ -558,6 +607,7 @@ func (sc *StatementContext) SetWarnings(warns []SQLWarn) { sc.mu.Lock() defer sc.mu.Unlock() sc.mu.warnings = warns + sc.mu.errorCount = 0 for _, w := range warns { if w.Level == WarnLevelError { sc.mu.errorCount++ diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index 7a4ec77a90660..b8f36dcb25055 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -16,12 +16,15 @@ package stmtctx_test import ( "context" + "encoding/json" "fmt" "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/execdetails" "github.com/stretchr/testify/require" @@ -143,3 +146,43 @@ func TestWeakConsistencyRead(t *testing.T) { execAndCheck("execute s", testkit.Rows("1 1 2"), kv.SI) tk.MustExec("rollback") } + +func TestMarshalSQLWarn(t *testing.T) { + warns := []stmtctx.SQLWarn{ + { + Level: stmtctx.WarnLevelError, + Err: errors.New("any error"), + }, + { + Level: stmtctx.WarnLevelError, + Err: errors.Trace(errors.New("any error")), + }, + { + Level: stmtctx.WarnLevelWarning, + Err: variable.ErrUnknownSystemVar.GenWithStackByArgs("unknown"), + }, + { + Level: stmtctx.WarnLevelWarning, + Err: errors.Trace(variable.ErrUnknownSystemVar.GenWithStackByArgs("unknown")), + }, + } + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + // First query can trigger loading global variables, which produces warnings. + tk.MustQuery("select 1") + tk.Session().GetSessionVars().StmtCtx.SetWarnings(warns) + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, len(warns), len(rows)) + + // The unmarshalled result doesn't need to be exactly the same with the original one. + // We only need that the results of `show warnings` are the same. + bytes, err := json.Marshal(warns) + require.NoError(t, err) + var newWarns []stmtctx.SQLWarn + err = json.Unmarshal(bytes, &newWarns) + require.NoError(t, err) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(newWarns) + tk.MustQuery("show warnings").Check(rows) +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index fe4f469e76134..12546cde3c0ad 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1867,6 +1867,11 @@ func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *se sessionStates.MPPStoreLastFailTime = s.MPPStoreLastFailTime sessionStates.FoundInPlanCache = s.PrevFoundInPlanCache sessionStates.FoundInBinding = s.PrevFoundInBinding + + // Encode StatementContext. We encode it here to avoid circle dependency. + sessionStates.LastAffectedRows = s.StmtCtx.PrevAffectedRows + sessionStates.LastInsertID = s.StmtCtx.PrevLastInsertID + sessionStates.Warnings = s.StmtCtx.GetWarnings() return } @@ -1902,6 +1907,11 @@ func (s *SessionVars) DecodeSessionStates(ctx context.Context, sessionStates *se } s.FoundInPlanCache = sessionStates.FoundInPlanCache s.FoundInBinding = sessionStates.FoundInBinding + + // Decode StatementContext. + s.StmtCtx.SetAffectedRows(uint64(sessionStates.LastAffectedRows)) + s.StmtCtx.PrevLastInsertID = sessionStates.LastInsertID + s.StmtCtx.SetWarnings(sessionStates.Warnings) return } diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index fb821c9f5cf78..68d68e42b159c 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -674,6 +674,92 @@ func TestSettersandGetters(t *testing.T) { } } +// TestSkipInitIsUsed ensures that no new variables are added with skipInit: true. +// This feature is deprecated, and if you need to run code to differentiate between init and "SET" (rare), +// you can instead check if s.StmtCtx.StmtType == "Set". +// The reason it is deprecated is that the behavior is typically wrong: +// it means session settings won't inherit from global and don't apply until you first set +// them in each session. This is a very weird behavior. +// See: https://github.com/pingcap/tidb/issues/35051 +func TestSkipInitIsUsed(t *testing.T) { + for _, sv := range GetSysVars() { + if sv.skipInit { + // Many of these variables might allow skipInit to be removed, + // they need to be checked first. The purpose of this test is to make + // sure we don't introduce any new variables with skipInit, which seems + // to be a problem. + switch sv.Name { + case Timestamp, + WarningCount, + ErrorCount, + LastInsertID, + Identity, + TiDBTxnScope, + TiDBSnapshot, + TiDBOptDistinctAggPushDown, + TiDBOptWriteRowID, + TiDBChecksumTableConcurrency, + TiDBBatchInsert, + TiDBBatchDelete, + TiDBBatchCommit, + TiDBCurrentTS, + TiDBLastTxnInfo, + TiDBLastQueryInfo, + TiDBEnableChunkRPC, + TxnIsolationOneShot, + TiDBOptimizerSelectivityLevel, + TiDBOptimizerEnableOuterJoinReorder, + TiDBLogFileMaxDays, + TiDBConfig, + TiDBDDLReorgPriority, + TiDBSlowQueryFile, + TiDBWaitSplitRegionFinish, + TiDBWaitSplitRegionTimeout, + TiDBLowResolutionTSO, + TiDBAllowRemoveAutoInc, + TiDBMetricSchemaStep, + TiDBMetricSchemaRangeDuration, + TiDBFoundInPlanCache, + TiDBFoundInBinding, + RandSeed1, + RandSeed2, + TiDBLastDDLInfo, + TiDBGeneralLog, + TiDBSlowLogThreshold, + TiDBRecordPlanInSlowLog, + TiDBEnableSlowLog, + TiDBCheckMb4ValueInUTF8, + TiDBPProfSQLCPU, + TiDBDDLSlowOprThreshold, + TiDBForcePriority, + TiDBMemoryUsageAlarmRatio, + TiDBEnableCollectExecutionInfo, + TiDBPersistAnalyzeOptions, + TiDBEnableColumnTracking, + TiDBStatsLoadPseudoTimeout, + SQLLogBin, + ForeignKeyChecks, + CollationDatabase, + CharacterSetClient, + CharacterSetResults, + CollationConnection, + CharsetDatabase, + GroupConcatMaxLen, + CharacterSetConnection, + CharacterSetServer, + TiDBBuildStatsConcurrency, + TiDBOptTiFlashConcurrencyFactor, + TiDBOptSeekFactor, + TiDBOptJoinReorderThreshold, + TiDBStatsLoadSyncWait, + CharacterSetFilesystem: + continue + } + require.Equal(t, false, sv.skipInit, fmt.Sprintf("skipInit should not be set on new system variables. variable %s is in violation", sv.Name)) + } + } +} + func TestSecureAuth(t *testing.T) { sv := GetSysVar(SecureAuth) vars := NewSessionVars() diff --git a/sessiontxn/BUILD.bazel b/sessiontxn/BUILD.bazel index 922f80480ac34..bdb84de22657f 100644 --- a/sessiontxn/BUILD.bazel +++ b/sessiontxn/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "//sessionctx", "//sessiontxn/staleread", "//testkit", + "//testkit/testfork", "//testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 76ff4ea60e52a..b41be21165908 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -35,13 +35,17 @@ var AssertTxnInfoSchemaKey stringutil.StringerStr = "assertTxnInfoSchemaKey" // Only for test var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSchemaAfterRetryKey" -// HookBeforeFirstRunExecutorKey is the hook key for the executor first run +// BreakPointBeforeExecutorFirstRun is the key for the stop point where session stops before executor's first run // Only for test -var HookBeforeFirstRunExecutorKey stringutil.StringerStr = "testHookKeyBeforeFirstRunExecutor" +var BreakPointBeforeExecutorFirstRun = "beforeExecutorFirstRun" -// HookAfterOnStmtRetryWithLockErrorKey is the hook key for after OnStmtRetry with lock error +// BreakPointOnStmtRetryAfterLockError s the key for the stop point where session stops after OnStmtRetry when lock error happens // Only for test -var HookAfterOnStmtRetryWithLockErrorKey stringutil.StringerStr = "testHookKeyAfterOnStmtRetryWithLockError" +var BreakPointOnStmtRetryAfterLockError = "lockErrorAndThenOnStmtRetryCalled" + +// AssertLockErr is used to record the lock errors we encountered +// Only for test +var AssertLockErr stringutil.StringerStr = "assertLockError" // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { @@ -94,6 +98,20 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } +// AddAssertEntranceForLockError is used only for test +func AddAssertEntranceForLockError(sctx sessionctx.Context, name string) { + records, ok := sctx.Value(AssertLockErr).(map[string]int) + if !ok { + records = make(map[string]int) + sctx.SetValue(AssertLockErr, records) + } + if v, ok := records[name]; ok { + records[name] = v + 1 + } else { + records[name] = 1 + } +} + // ExecTestHook is used only for test. It consumes hookKey in session wait do what it gets from it. func ExecTestHook(sctx sessionctx.Context, hookKey fmt.Stringer) { c := sctx.Value(hookKey) diff --git a/sessiontxn/isolation/BUILD.bazel b/sessiontxn/isolation/BUILD.bazel index 35fb911f4b636..1345c09f08b21 100644 --- a/sessiontxn/isolation/BUILD.bazel +++ b/sessiontxn/isolation/BUILD.bazel @@ -53,6 +53,7 @@ go_test( "//testkit", "//testkit/testsetup", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//error", diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index bb26b70fdb852..9918f8b226e5b 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -32,20 +32,15 @@ import ( ) type stmtState struct { - stmtTS uint64 - stmtTSFuture oracle.Future - stmtUseStartTS bool - onNextRetryOrStmt func() error + stmtTS uint64 + stmtTSFuture oracle.Future + stmtUseStartTS bool } func (s *stmtState) prepareStmt(useStartTS bool) error { - onNextStmt := s.onNextRetryOrStmt *s = stmtState{ stmtUseStartTS: useStartTS, } - if onNextStmt != nil { - return onNextStmt() - } return nil } @@ -53,7 +48,9 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState - availableRCCheckTS uint64 + latestOracleTS uint64 + // latestOracleTSValid shows whether we have already fetched a ts from pd and whether the ts we fetched is still valid. + latestOracleTSValid bool } // NewPessimisticRCTxnContextProvider returns a new PessimisticRCTxnContextProvider @@ -66,12 +63,14 @@ func NewPessimisticRCTxnContextProvider(sctx sessionctx.Context, causalConsisten txnCtx.IsPessimistic = true txnCtx.Isolation = ast.ReadCommitted }, - onTxnActive: func(txn kv.Transaction) { - txn.SetOption(kv.Pessimistic, true) - }, }, } + provider.onTxnActive = func(txn kv.Transaction) { + txn.SetOption(kv.Pessimistic, true) + provider.latestOracleTS = txn.StartTS() + provider.latestOracleTSValid = true + } provider.getStmtReadTSFunc = provider.getStmtTS provider.getStmtForUpdateTSFunc = provider.getStmtTS return provider @@ -104,9 +103,6 @@ func NeedSetRCCheckTSFlag(ctx sessionctx.Context, node ast.Node) bool { // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { - // Invalid rc check for next statement or retry when error occurs - p.availableRCCheckTS = 0 - switch point { case sessiontxn.StmtErrAfterQuery: return p.handleAfterQueryError(err) @@ -135,15 +131,30 @@ func (p *PessimisticRCTxnContextProvider) prepareStmtTS() { switch { case p.stmtUseStartTS: stmtTSFuture = sessiontxn.FuncFuture(p.getTxnStartTS) - case p.availableRCCheckTS != 0 && sessVars.StmtCtx.RCCheckTS: - stmtTSFuture = sessiontxn.ConstantFuture(p.availableRCCheckTS) + case p.latestOracleTSValid && sessVars.StmtCtx.RCCheckTS: + stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) default: - stmtTSFuture = sessiontxn.NewOracleFuture(p.ctx, p.sctx, sessVars.TxnCtx.TxnScope) + stmtTSFuture = p.getOracleFuture() } p.stmtTSFuture = stmtTSFuture } +func (p *PessimisticRCTxnContextProvider) getOracleFuture() sessiontxn.FuncFuture { + txnCtx := p.sctx.GetSessionVars().TxnCtx + future := sessiontxn.NewOracleFuture(p.ctx, p.sctx, txnCtx.TxnScope) + return func() (ts uint64, err error) { + if ts, err = future.Wait(); err != nil { + return + } + txnCtx.SetForUpdateTS(ts) + ts = txnCtx.GetForUpdateTS() + p.latestOracleTS = ts + p.latestOracleTSValid = true + return + } +} + func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { if p.stmtTS != 0 { return p.stmtTS, nil @@ -159,13 +170,8 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { return 0, err } - // forUpdateTS should exactly equal to the read ts - txnCtx := p.sctx.GetSessionVars().TxnCtx - txnCtx.SetForUpdateTS(ts) txn.SetOption(kv.SnapshotTS, ts) - p.stmtTS = ts - p.availableRCCheckTS = ts return } @@ -173,16 +179,18 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { // At this point the query will be retried from the beginning. func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() - if sessVars.StmtCtx.RCCheckTS && errors.ErrorEqual(queryErr, kv.ErrWriteConflict) { - logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", - zap.String("sql", sessVars.StmtCtx.OriginalSQL)) - return sessiontxn.RetryReady() + if !errors.ErrorEqual(queryErr, kv.ErrWriteConflict) || !sessVars.StmtCtx.RCCheckTS { + return sessiontxn.NoIdea() } - return sessiontxn.NoIdea() + p.latestOracleTSValid = false + logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", + zap.String("sql", sessVars.StmtCtx.OriginalSQL), zap.Error(queryErr)) + return sessiontxn.RetryReady() } func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { + p.latestOracleTSValid = false txnCtx := p.sctx.GetSessionVars().TxnCtx retryable := false if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok && deadlock.IsRetryable { @@ -200,16 +208,9 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr retryable = true } - // force refresh ts in next retry or statement when lock error occurs - p.onNextRetryOrStmt = func() error { - _, err := p.getStmtTS() - return err - } - if retryable { return sessiontxn.RetryReady() } - return sessiontxn.ErrorAction(lockErr) } @@ -225,3 +226,31 @@ func (p *PessimisticRCTxnContextProvider) AdviseWarmup() error { p.prepareStmtTS() return nil } + +// AdviseOptimizeWithPlan in RC covers much fewer cases compared with pessimistic repeatable read. +// We only optimize with insert operator with no selection in that we do not fetch latest ts immediately. +// We only update ts if write conflict is incurred. +func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { + if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { + return nil + } + + if p.stmtUseStartTS || !p.latestOracleTSValid { + return nil + } + + plan, ok := val.(plannercore.Plan) + if !ok { + return nil + } + + if execute, ok := plan.(*plannercore.Execute); ok { + plan = execute.Plan + } + + if v, ok := plan.(*plannercore.Insert); ok && v.SelectPlan == nil { + p.stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) + } + + return nil +} diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index 19c75bfccb144..b7e56aa38839d 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -55,13 +56,13 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.NoError(t, err) forUpdateStmt := stmts[0] - compareTS := getOracleTS(t, se) + compareTS := se.GetSessionVars().TxnCtx.StartTS // first ts should request from tso require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) + require.Equal(t, ts, compareTS) rcCheckTS := ts // second ts should reuse first ts @@ -103,14 +104,11 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, errors.New("err")) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) - compareTS = getOracleTS(t, se) - require.Greater(t, compareTS, rcCheckTS) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) - rcCheckTS = ts + require.Equal(t, rcCheckTS, ts) // `StmtErrAfterPessimisticLock` will still disable rc check require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) @@ -438,6 +436,88 @@ func TestTidbSnapshotVarInRC(t *testing.T) { } } +func TestConflictErrorsInRC(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("set tx_isolation='READ-COMMITTED'") + + // Test for insert + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + for _, name := range errorsInInsert { + require.Equal(t, records[name], 1) + } + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("rollback") + + // Test for delete + tk.MustExec("truncate t") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + + // Test for update + tk.MustExec("truncate t") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) + + tk.MustExec("rollback") + + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + func activeRCTxnAssert(t *testing.T, sctx sessionctx.Context, inTxn bool) *txnAssert[*isolation.PessimisticRCTxnContextProvider] { return &txnAssert[*isolation.PessimisticRCTxnContextProvider]{ sctx: sctx, diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 735373b38ed06..4827446d27a6c 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -35,10 +35,11 @@ type PessimisticRRTxnContextProvider struct { baseTxnContextProvider // Used for ForUpdateRead statement - forUpdateTS uint64 + forUpdateTS uint64 + latestForUpdateTS uint64 // It may decide whether to update forUpdateTs when calling provider's getForUpdateTs // See more details in the comments of optimizeWithPlan - followingOperatorIsPointGetForUpdate bool + optimizeForNotFetchingLatestTS bool } // NewPessimisticRRTxnContextProvider returns a new PessimisticRRTxnContextProvider @@ -73,7 +74,7 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return 0, err } - if p.followingOperatorIsPointGetForUpdate { + if p.optimizeForNotFetchingLatestTS { p.forUpdateTS = p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() return p.forUpdateTS, nil } @@ -114,7 +115,8 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { } sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) - txn.SetOption(kv.SnapshotTS, sctx.GetSessionVars().TxnCtx.GetForUpdateTS()) + p.latestForUpdateTS = version.Ver + txn.SetOption(kv.SnapshotTS, version.Ver) return nil } @@ -126,7 +128,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context, node } p.forUpdateTS = 0 - p.followingOperatorIsPointGetForUpdate = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -137,15 +139,14 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err return err } - txnCtxForUpdateTS := p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() // If TxnCtx.forUpdateTS is updated in OnStmtErrorForNextAction, we assign the value to the provider - if txnCtxForUpdateTS > p.forUpdateTS { - p.forUpdateTS = txnCtxForUpdateTS + if p.latestForUpdateTS > p.forUpdateTS { + p.forUpdateTS = p.latestForUpdateTS } else { p.forUpdateTS = 0 } - p.followingOperatorIsPointGetForUpdate = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -165,6 +166,8 @@ func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point session // We expect that the data that the point get acquires has not been changed. // Benefit: Save the cost of acquiring ts from PD. // Drawbacks: If the data has been changed since the ts we used, we need to retry. +// One exception is insert operation, when it has no select plan, we do not fetch the latest ts immediately. We only update ts +// if write conflict is incurred. func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { return nil @@ -179,24 +182,44 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - mayOptimizeForPointGet := false - if v, ok := plan.(*plannercore.PhysicalLock); ok { - if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true - } - } else if v, ok := plan.(*plannercore.Update); ok { - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + p.optimizeForNotFetchingLatestTS = notNeedGetLatestTSFromPD(plan, false) + + return nil +} + +// notNeedGetLatestTSFromPD searches for optimization condition recursively +// Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, +// we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. +// inLockOrWriteStmt = true means one of the ancestor node is update/delete/physicalLock. +func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) bool { + switch v := plan.(type) { + case *plannercore.PointGetPlan: + // We do not optimize the point get/ batch point get if plan.lock = false and inLockOrWriteStmt = true. + // Theoretically, the plan.lock should be true if the flag is true. But due to the bug describing in Issue35524, + // the plan.lock can be false in the case of inLockOrWriteStmt being true. In this case, optimization here can lead to different results + // which cannot be accepted as AdviseOptimizeWithPlan cannot change results. + return !inLockOrWriteStmt || v.Lock + case *plannercore.BatchPointGetPlan: + return !inLockOrWriteStmt || v.Lock + case plannercore.PhysicalPlan: + if len(v.Children()) == 0 { + return false } - } else if v, ok := plan.(*plannercore.Delete); ok { - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + _, isPhysicalLock := v.(*plannercore.PhysicalLock) + for _, p := range v.Children() { + if !notNeedGetLatestTSFromPD(p, isPhysicalLock || inLockOrWriteStmt) { + return false + } } + return true + case *plannercore.Update: + return notNeedGetLatestTSFromPD(v.SelectPlan, true) + case *plannercore.Delete: + return notNeedGetLatestTSFromPD(v.SelectPlan, true) + case *plannercore.Insert: + return v.SelectPlan == nil } - - p.followingOperatorIsPointGetForUpdate = mayOptimizeForPointGet - - return nil + return false } func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index 04f15d283cd40..c1487a1bb0ae7 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -344,61 +345,89 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { tk.MustExec("insert into t values (1,1), (2,2)") se := tk.Session() provider := initializeRepeatableReadProvider(t, tk, true) - forUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() + lastFetchedForUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() txnManager := sessiontxn.GetTxnManager(se) - stmt, err := parser.New().ParseOneStmt("delete from t where id = 1", "", "") - require.NoError(t, err) - require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) - compareTs := getOracleTS(t, se) - compiler := executor.Compiler{Ctx: se} - execStmt, err := compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err := provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, compareTs, ts) - require.Equal(t, ts, forUpdateTS) + type testStruct struct { + sql string + shouldOptimize bool + } - stmt, err = parser.New().ParseOneStmt("update t set v = v + 10 where id = 1", "", "") - require.NoError(t, err) - require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + cases := []testStruct{ + { + "delete from t where id = 1", + true, + }, + { + "update t set v = v + 10 where id = 1", + true, + }, + { + "select * from (select * from t where id = 1 for update) as t1 for update", + true, + }, + { + "select * from t where id = 1 for update", + true, + }, + { + "select * from t where id = 1 or id = 2 for update", + true, + }, + { + "select * from t for update", + false, + }, + } - stmt, err = parser.New().ParseOneStmt("select * from (select * from t where id = 1 for update) as t1 for update", "", "") - require.NoError(t, err) - require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + var stmt ast.StmtNode + var err error + var execStmt *executor.ExecStmt + var compiler executor.Compiler + var ts, compareTS uint64 + var action sessiontxn.StmtErrorAction - // Now, test for one that does not use the optimization - stmt, err = parser.New().ParseOneStmt("select * from t for update", "", "") - require.NoError(t, err) - require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) - compareTs = getOracleTS(t, se) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, ts, compareTs) + for _, c := range cases { + compareTS = getOracleTS(t, se) + + require.NoError(t, txnManager.OnStmtStart(context.TODO(), nil)) + stmt, err = parser.New().ParseOneStmt(c.sql, "", "") + require.NoError(t, err) + + err = provider.OnStmtStart(context.TODO(), nil) + require.NoError(t, err) + + compiler = executor.Compiler{Ctx: se} + execStmt, err = compiler.Compile(context.TODO(), stmt) + require.NoError(t, err) + + err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) + require.NoError(t, err) + + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + + if c.shouldOptimize { + require.Greater(t, compareTS, ts) + require.Equal(t, ts, lastFetchedForUpdateTS) + } else { + require.Greater(t, ts, compareTS) + } + + // retry + if c.shouldOptimize { + action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + require.NoError(t, err) + require.Equal(t, sessiontxn.StmtActionRetryReady, action) + err = provider.OnStmtRetry(context.TODO()) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Greater(t, ts, compareTS) + + lastFetchedForUpdateTS = ts + } + } // Test use startTS after optimize when autocommit=0 activeAssert := activePessimisticRRAssert(t, tk.Session(), true) @@ -416,7 +445,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Equal(t, tk.Session().GetSessionVars().TxnCtx.StartTS, ts) // Test still fetch for update ts after optimize when autocommit=0 - compareTs = getOracleTS(t, se) + compareTS = getOracleTS(t, se) activeAssert = activePessimisticRRAssert(t, tk.Session(), true) provider = initializeRepeatableReadProvider(t, tk, false) stmt, err = parser.New().ParseOneStmt("select * from t", "", "") @@ -428,7 +457,179 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.NoError(t, err) ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) - require.Greater(t, ts, compareTs) + require.Greater(t, ts, compareTS) +} + +var errorsInInsert = []string{ + "errWriteConflict", + "errDuplicateKey", +} + +func TestConflictErrorInInsertInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + for _, name := range errorsInInsert { + require.Equal(t, records[name], 1) + } + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInPointGetForUpdateInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustExec("commit") + + // batch point get + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id = 1 or id = 2 for update").Check(testkit.Rows("1 21", "2 2")) + records, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustExec("commit") + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +// Delete should get the latest ts and thus does not incur write conflict +func TestConflictErrorInDeleteInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if sub select in delete is point get, we will incur one write conflict + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInUpdateInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if the sub select plan is point get, we should incur one write conflict + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInOtherQueryContainingPointGet(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id=1 and v > 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, diff --git a/sessiontxn/legacy/BUILD.bazel b/sessiontxn/legacy/BUILD.bazel deleted file mode 100644 index d9e9b5249cbbf..0000000000000 --- a/sessiontxn/legacy/BUILD.bazel +++ /dev/null @@ -1,41 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "legacy", - srcs = ["provider.go"], - importpath = "github.com/pingcap/tidb/sessiontxn/legacy", - visibility = ["//visibility:public"], - deps = [ - "//domain", - "//infoschema", - "//kv", - "//parser/ast", - "//parser/terror", - "//sessionctx", - "//sessionctx/variable", - "//sessiontxn", - "//sessiontxn/staleread", - "//table/temptable", - "//util/logutil", - "@com_github_pingcap_errors//:errors", - "@com_github_tikv_client_go_v2//error", - "@org_uber_go_zap//:zap", - ], -) - -go_test( - name = "legacy_test", - srcs = ["provider_test.go"], - deps = [ - ":legacy", - "//domain", - "//kv", - "//sessionctx", - "//sessiontxn", - "//testkit", - "@com_github_pingcap_errors//:errors", - "@com_github_pingcap_kvproto//pkg/kvrpcpb", - "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//error", - ], -) diff --git a/sessiontxn/legacy/provider.go b/sessiontxn/legacy/provider.go deleted file mode 100644 index ff10767122b99..0000000000000 --- a/sessiontxn/legacy/provider.go +++ /dev/null @@ -1,243 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package legacy - -import ( - "context" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/staleread" - "github.com/pingcap/tidb/table/temptable" - "github.com/pingcap/tidb/util/logutil" - tikverr "github.com/tikv/client-go/v2/error" - "go.uber.org/zap" -) - -// SimpleTxnContextProvider implements TxnContextProvider -// It is only used in refactor stage -// TODO: remove it after refactor finished -type SimpleTxnContextProvider struct { - Ctx context.Context - Sctx sessionctx.Context - InfoSchema infoschema.InfoSchema - GetReadTSFunc func() (uint64, error) - GetForUpdateTSFunc func() (uint64, error) - UpdateForUpdateTS func(seCtx sessionctx.Context, newForUpdateTS uint64) error - - Pessimistic bool - CausalConsistencyOnly bool - - isTxnActive bool -} - -// GetTxnInfoSchema returns the information schema used by txn -func (p *SimpleTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema { - return p.InfoSchema -} - -// GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update) -func (p *SimpleTxnContextProvider) GetStmtReadTS() (uint64, error) { - if p.GetReadTSFunc == nil { - return 0, errors.New("ReadTSFunc not set") - } - return p.GetReadTSFunc() -} - -// GetStmtForUpdateTS returns the read timestamp used by update/insert/delete or select ... for update -func (p *SimpleTxnContextProvider) GetStmtForUpdateTS() (uint64, error) { - if p.GetForUpdateTSFunc == nil { - return 0, errors.New("GetForUpdateTSFunc not set") - } - return p.GetForUpdateTSFunc() -} - -// OnInitialize is the hook that should be called when enter a new txn with this provider -func (p *SimpleTxnContextProvider) OnInitialize(ctx context.Context, tp sessiontxn.EnterNewTxnType) error { - p.Ctx = ctx - sessVars := p.Sctx.GetSessionVars() - switch tp { - case sessiontxn.EnterNewTxnDefault, sessiontxn.EnterNewTxnWithBeginStmt: - shouldReuseTxn := tp == sessiontxn.EnterNewTxnWithBeginStmt && sessiontxn.CanReuseTxnWhenExplicitBegin(p.Sctx) - if !shouldReuseTxn { - if err := p.Sctx.NewTxn(ctx); err != nil { - return err - } - } - - if tp == sessiontxn.EnterNewTxnWithBeginStmt { - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - sessVars.SetInTxn(true) - } - - sessVars.TxnCtx.IsPessimistic = p.Pessimistic - if _, err := p.activateTxn(); err != nil { - return err - } - - if is, ok := sessVars.TxnCtx.InfoSchema.(infoschema.InfoSchema); ok { - p.InfoSchema = is - } - case sessiontxn.EnterNewTxnBeforeStmt: - p.InfoSchema = temptable.AttachLocalTemporaryTableInfoSchema(p.Sctx, domain.GetDomain(p.Sctx).InfoSchema()) - sessVars.TxnCtx = &variable.TransactionContext{ - TxnCtxNoNeedToRestore: variable.TxnCtxNoNeedToRestore{ - InfoSchema: p.InfoSchema, - CreateTime: time.Now(), - ShardStep: int(sessVars.ShardAllocateStep), - TxnScope: sessVars.CheckAndGetTxnScope(), - IsPessimistic: p.Pessimistic, - }, - } - default: - return errors.Errorf("Unsupported type: %v", tp) - } - - return nil -} - -// OnStmtStart is the hook that should be called when a new statement started -func (p *SimpleTxnContextProvider) OnStmtStart(ctx context.Context, _ ast.StmtNode) error { - p.Ctx = ctx - p.InfoSchema = p.Sctx.GetInfoSchema().(infoschema.InfoSchema) - return nil -} - -// OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *SimpleTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { - switch point { - case sessiontxn.StmtErrAfterPessimisticLock: - return p.handleAfterPessimisticLockError(err) - default: - return sessiontxn.NoIdea() - } -} - -func (p *SimpleTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { - sessVars := p.Sctx.GetSessionVars() - if sessVars.IsIsolation(ast.Serializable) { - return sessiontxn.ErrorAction(lockErr) - } - - txnCtx := sessVars.TxnCtx - if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok { - if !deadlock.IsRetryable { - return sessiontxn.ErrorAction(lockErr) - } - logutil.Logger(p.Ctx).Info("single statement deadlock, retry statement", - zap.Uint64("txn", txnCtx.StartTS), - zap.Uint64("lockTS", deadlock.LockTs), - zap.Stringer("lockKey", kv.Key(deadlock.LockKey)), - zap.Uint64("deadlockKeyHash", deadlock.DeadlockKeyHash)) - } else if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { - errStr := lockErr.Error() - forUpdateTS := txnCtx.GetForUpdateTS() - logutil.Logger(p.Ctx).Debug("pessimistic write conflict, retry statement", - zap.Uint64("txn", txnCtx.StartTS), - zap.Uint64("forUpdateTS", forUpdateTS), - zap.String("err", errStr)) - // Always update forUpdateTS by getting a new timestamp from PD. - // If we use the conflict commitTS as the new forUpdateTS and async commit - // is used, the commitTS of this transaction may exceed the max timestamp - // that PD allocates. Then, the change may be invisible to a new transaction, - // which means linearizability is broken. - } else { - // this branch if err not nil, always update forUpdateTS to avoid problem described below - // for nowait, when ErrLock happened, ErrLockAcquireFailAndNoWaitSet will be returned, and in the same txn - // the select for updateTs must be updated, otherwise there maybe rollback problem. - // begin; select for update key1(here ErrLocked or other errors(or max_execution_time like util), - // key1 lock not get and async rollback key1 is raised) - // select for update key1 again(this time lock succ(maybe lock released by others)) - // the async rollback operation rollbacked the lock just acquired - tsErr := p.UpdateForUpdateTS(p.Sctx, 0) - if tsErr != nil { - logutil.Logger(p.Ctx).Warn("UpdateForUpdateTS failed", zap.Error(tsErr)) - } - return sessiontxn.ErrorAction(lockErr) - } - - if err := p.UpdateForUpdateTS(p.Sctx, 0); err != nil { - return sessiontxn.ErrorAction(lockErr) - } - - return sessiontxn.RetryReady() -} - -// OnStmtRetry is the hook that should be called when a statement retry -func (p *SimpleTxnContextProvider) OnStmtRetry(_ context.Context) error { - return nil -} - -func (p *SimpleTxnContextProvider) prepareTSFuture() error { - if p.Sctx.GetSessionVars().SnapshotTS != 0 || staleread.IsStmtStaleness(p.Sctx) || p.Sctx.GetPreparedTSFuture() != nil { - return nil - } - - txn, err := p.Sctx.Txn(false) - if err != nil { - return err - } - - if txn.Valid() { - return nil - } - - txnScope := p.Sctx.GetSessionVars().CheckAndGetTxnScope() - future := sessiontxn.NewOracleFuture(p.Ctx, p.Sctx, txnScope) - return p.Sctx.PrepareTSFuture(p.Ctx, future, txnScope) -} - -// activateTxn actives the txn -func (p *SimpleTxnContextProvider) activateTxn() (kv.Transaction, error) { - if p.isTxnActive { - return p.Sctx.Txn(true) - } - - txn, err := p.Sctx.Txn(true) - if err != nil { - return nil, err - } - - if p.Pessimistic { - txn.SetOption(kv.Pessimistic, true) - } - - if p.CausalConsistencyOnly { - txn.SetOption(kv.GuaranteeLinearizability, false) - } - - p.isTxnActive = true - return txn, nil -} - -// AdviseWarmup provides warmup for inner state -func (p *SimpleTxnContextProvider) AdviseWarmup() error { - return p.prepareTSFuture() -} - -// AdviseOptimizeWithPlan providers optimization according to the plan -func (p *SimpleTxnContextProvider) AdviseOptimizeWithPlan(_ interface{}) error { - return nil -} diff --git a/sessiontxn/legacy/provider_test.go b/sessiontxn/legacy/provider_test.go deleted file mode 100644 index 552b8924f78bd..0000000000000 --- a/sessiontxn/legacy/provider_test.go +++ /dev/null @@ -1,126 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package legacy_test - -import ( - "context" - "testing" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" - tikverr "github.com/tikv/client-go/v2/error" -) - -func TestErrorHandle(t *testing.T) { - store, do, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - - provider := newSimpleProvider(tk, do) - require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - expectedForUpdateTS := getForUpdateTS(t, provider) - - var lockErr error - - // StmtErrAfterLock: ErrWriteConflict should retry and update forUpdateTS - lockErr = kv.ErrWriteConflict - action, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - require.Nil(t, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: DeadLock that is not retryable will just return an error - lockErr = newDeadLockError(false) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionError, action) - require.Equal(t, lockErr, err) - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: DeadLock that is retryable should retry and update forUpdateTS - lockErr = newDeadLockError(true) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - require.Nil(t, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: other errors should only update forUpdateTS but not retry - lockErr = errors.New("other error") - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionError, action) - require.Equal(t, lockErr, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterQuery: always not retry - lockErr = kv.ErrWriteConflict - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, lockErr) - require.Equal(t, sessiontxn.StmtActionNoIdea, action) - require.Nil(t, err) - - tk.Session().GetSessionVars().StmtCtx.RCCheckTS = true - require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, lockErr) - require.Equal(t, sessiontxn.StmtActionNoIdea, action) - require.Nil(t, err) -} - -func getForUpdateTS(t *testing.T, provider *legacy.SimpleTxnContextProvider) uint64 { - forUpdateTS, err := provider.GetStmtForUpdateTS() - require.NoError(t, err) - return forUpdateTS -} - -func newDeadLockError(isRetryable bool) error { - return &tikverr.ErrDeadlock{ - Deadlock: &kvrpcpb.Deadlock{}, - IsRetryable: isRetryable, - } -} - -func newSimpleProvider(tk *testkit.TestKit, do *domain.Domain) *legacy.SimpleTxnContextProvider { - tk.MustExec("begin pessimistic") - readTS := uint64(1) - forUpdateTS := uint64(1) - return &legacy.SimpleTxnContextProvider{ - Ctx: context.TODO(), - Sctx: tk.Session(), - InfoSchema: do.InfoSchema(), - GetReadTSFunc: func() (uint64, error) { - return readTS, nil - }, - GetForUpdateTSFunc: func() (uint64, error) { - return forUpdateTS, nil - }, - UpdateForUpdateTS: func(seCtx sessionctx.Context, newForUpdateTS uint64) error { - if newForUpdateTS == 0 { - forUpdateTS += 1 - } else { - forUpdateTS = newForUpdateTS - } - return nil - }, - Pessimistic: true, - } -} diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 8ac6f0aaa7322..01cb607e68a33 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testfork" "github.com/pingcap/tidb/testkit/testsetup" "github.com/stretchr/testify/require" "go.uber.org/goleak" @@ -51,9 +52,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) @@ -84,9 +82,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) @@ -449,7 +444,7 @@ func TestTxnContextForHistoricalRead(t *testing.T) { }) doWithCheckPath(t, se, normalPathRecords, func() { - tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) }) tk.MustExec("rollback") @@ -729,102 +724,59 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { "update t1 set v=v+1 where id in (1, 2, 3) and v>0", } - for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { - for _, query := range queries { - for _, autocommit := range []bool{true, false} { - t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { - testStillWriteConflictAfterRetry(t, store, isolation, query, autocommit) - }) - } - } - } -} - -func testStillWriteConflictAfterRetry(t *testing.T, store kv.Storage, isolation string, query string, autocommit bool) { - tk := testkit.NewTestKit(t, store) - defer tk.MustExec("rollback") - - tk.MustExec("use test") - tk.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) - tk.MustExec("set autocommit=1") - tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - - se := tk.Session() - chanBeforeRunStmt := make(chan func(), 1) - chanAfterOnStmtRetry := make(chan func(), 1) - c2 := make(chan string, 1) - c3 := make(chan string, 1) - wait := func(ch chan string, expect string) { - select { - case got := <-ch: - if got != expect { - panic(fmt.Sprintf("expect '%s', got '%s'", expect, got)) - } - case <-time.After(time.Second * 10): - panic("wait2 timeout") - } - } - - if autocommit { - tk.MustExec("begin") - } else { - tk.MustExec("set @@autocommit=0") - } - - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, chanBeforeRunStmt) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, chanAfterOnStmtRetry) - defer func() { - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) - }() - - chanBeforeRunStmt <- func() { - c2 <- "now before session1 runStmt" - wait(c3, "session2 updated v=v+1 done") - } + testfork.RunTest(t, func(t *testfork.T) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + tk2 := testkit.NewSteppedTestKit(t, store) + defer tk2.MustExec("rollback") - chanAfterOnStmtRetry <- func() { - c2 <- "now after OnStmtRetry before rebuild executor" - wait(c3, "session2 updated v=v+1 again done") - } - - go func() { - tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", testfork.PickEnum(t, ast.RepeatableRead, ast.ReadCommitted))) + autocommit := testfork.PickEnum(t, 0, 1) + tk2.MustExec(fmt.Sprintf("set autocommit=%d", autocommit)) + if autocommit == 1 { + tk2.MustExec("begin") + } - // first conflict - wait(c2, "now before session1 runStmt") - tk2.MustExec("update t1 set v=v+1 where id=1") - c3 <- "session2 updated v=v+1 done" - - // second conflict - wait(c2, "now after OnStmtRetry before rebuild executor") - tk2.MustExec("update t1 set v=v+1 where id=1") - c3 <- "session2 updated v=v+1 again done" - chanAfterOnStmtRetry <- func() {} - c3 <- "done" - }() - - isSelect := false - if strings.HasPrefix(query, "update ") { - tk.MustExec(query) - } else if strings.HasPrefix(query, "select ") { - isSelect = true - tk.MustQuery(query).Check(testkit.Rows("1 12")) - } else { - require.FailNowf(t, "invalid query: %s", query) - } + tk2.SetBreakPoints( + sessiontxn.BreakPointBeforeExecutorFirstRun, + sessiontxn.BreakPointOnStmtRetryAfterLockError, + ) + + var isSelect, isUpdate bool + query := testfork.Pick(t, queries) + switch { + case strings.HasPrefix(query, "select"): + isSelect = true + tk2.SteppedMustQuery(query) + case strings.HasPrefix(query, "update"): + isUpdate = true + tk2.SteppedMustExec(query) + default: + require.FailNowf(t, "invalid query: ", query) + } - wait(c3, "done") + // Pause the session before the executor first run and then update the record in another session + tk2.ExpectStopOnBreakPoint(sessiontxn.BreakPointBeforeExecutorFirstRun) + tk.MustExec("update t1 set v=v+1") - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) - if isSelect { + // Session continues, it should get a lock error and retry, we pause the session before the executor's next run + // and then update the record in another session again. + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) tk.MustExec("update t1 set v=v+1") - } - tk.MustExec("commit") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - tk.MustExec("rollback") + + // Because the record is updated by another session again, when this session continues, it will get a lock error again. + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectIdle() + switch { + case isSelect: + tk2.GetQueryResult().Check(testkit.Rows("1 12")) + case isUpdate: + tk2.MustExec("commit") + tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) + } + }) } diff --git a/statistics/trace_test.go b/statistics/trace_test.go index 03d3c71e7f8ca..144dbff27d530 100644 --- a/statistics/trace_test.go +++ b/statistics/trace_test.go @@ -100,3 +100,28 @@ func TestTraceCE(t *testing.T) { require.ElementsMatch(t, resultJSON, out[i].Trace) } } + +func TestTraceCEPartitionTable(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, d varchar(10), index idx(a, b)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN MAXVALUE);") + tk.MustExec(`insert into t values(1, 1, "aaa"), + (1, 1, "bbb"), + (1, 2, "ccc"), + (1, 2, "ddd"), + (2, 2, "aaa"), + (2, 3, "bbb")`) + tk.MustExec("analyze table t") + result := tk.MustQuery("trace plan target='estimation' select * from t where a >=1") + require.Len(t, result.Rows(), 1) + resultStr := result.Rows()[0][0].(string) + var resultJSON []*tracing.CETraceRecord + err := json.Unmarshal([]byte(resultStr), &resultJSON) + require.NoError(t, err) + for _, r := range resultJSON { + require.Equal(t, "t", r.TableName) + } +} diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 2832abe60ecb1..828d3546abfae 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -860,7 +860,9 @@ func (worker *copIteratorWorker) handleCopPagingResult(bo *Backoffer, rpcCtx *ti pagingRange := resp.pbResp.Range // only paging requests need to calculate the next ranges if pagingRange == nil { - return nil, errors.New("lastRange in paging should not be nil") + // If the storage engine doesn't support paging protocol, it should have return all the region data. + // So we finish here. + return nil, nil } // calculate next ranges and grow the paging size task.ranges = worker.calculateRemain(task.ranges, pagingRange, worker.req.Desc) @@ -932,7 +934,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R zap.String("storeAddr", task.storeAddr), zap.Error(err)) if strings.Contains(err.Error(), "write conflict") { - return nil, kv.ErrWriteConflict + return nil, kv.ErrWriteConflict.FastGen("%s", otherErr) } return nil, errors.Trace(err) } diff --git a/store/helper/helper.go b/store/helper/helper.go index e3912ac9aff26..948a84c84635d 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -840,42 +840,46 @@ func (h *Helper) requestPD(apiName, method, uri string, body io.Reader, res inte if len(pdHosts) == 0 { return errors.New("pd unavailable") } - logutil.BgLogger().Debug("RequestPD URL", zap.String("url", util.InternalHTTPSchema()+"://"+pdHosts[0]+uri)) - req := new(http.Request) for _, host := range pdHosts { - req, err = http.NewRequest(method, util.InternalHTTPSchema()+"://"+host+uri, body) - if err != nil { - // Try to request from another PD node when some nodes may down. - if strings.Contains(err.Error(), "connection refused") { - continue - } - return errors.Trace(err) + err = requestPDForOneHost(host, apiName, method, uri, body, res) + if err == nil { + break } + // Try to request from another PD node when some nodes may down. } + return err +} + +func requestPDForOneHost(host, apiName, method, uri string, body io.Reader, res interface{}) error { + urlVar := fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), host, uri) + logutil.BgLogger().Debug("RequestPD URL", zap.String("url", urlVar)) + req, err := http.NewRequest(method, urlVar, body) if err != nil { - return err + logutil.BgLogger().Warn("requestPDForOneHost new request failed", + zap.String("url", urlVar), zap.Error(err)) + return errors.Trace(err) } start := time.Now() resp, err := util.InternalHTTPClient().Do(req) if err != nil { metrics.PDAPIRequestCounter.WithLabelValues(apiName, "network error").Inc() + logutil.BgLogger().Warn("requestPDForOneHost do request failed", + zap.String("url", urlVar), zap.Error(err)) return errors.Trace(err) } metrics.PDAPIExecutionHistogram.WithLabelValues(apiName).Observe(time.Since(start).Seconds()) metrics.PDAPIRequestCounter.WithLabelValues(apiName, resp.Status).Inc() - defer func() { err = resp.Body.Close() if err != nil { - logutil.BgLogger().Error("close body failed", zap.Error(err)) + logutil.BgLogger().Warn("requestPDForOneHost close body failed", + zap.String("url", urlVar), zap.Error(err)) } }() - err = json.NewDecoder(resp.Body).Decode(res) if err != nil { return errors.Trace(err) } - return nil } diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index c4c43cf90cfdc..17034f2e778f5 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "mocksessionmanager.go", "mockstore.go", "result.go", + "stepped.go", "testkit.go", ], importpath = "github.com/pingcap/tidb/testkit", @@ -23,8 +24,10 @@ go_library( "//store/mockstore", "//types", "//util", + "//util/breakpoint", "//util/sqlexec", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", diff --git a/testkit/stepped.go b/testkit/stepped.go new file mode 100644 index 0000000000000..fc6e434ef85b9 --- /dev/null +++ b/testkit/stepped.go @@ -0,0 +1,256 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testkit + +import ( + "testing" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/breakpoint" + "github.com/stretchr/testify/require" +) + +var errCommandRunFailed = errors.New("command run failed") + +var defaultChanTimeout = time.Second * 10 + +type steppedTestkitMsgType int + +const ( + msgTpCmdContinue steppedTestkitMsgType = iota + msgTpCmdStopOnBreakPoint + msgTpCmdDone +) + +type steppedTestKitMsg struct { + tp steppedTestkitMsgType + val any +} + +type steppedTestKitMsgChan chan *steppedTestKitMsg + +func (ch steppedTestKitMsgChan) sendMsg(tp steppedTestkitMsgType, val any) error { + select { + case ch <- &steppedTestKitMsg{tp: tp, val: val}: + return nil + case <-time.After(defaultChanTimeout): + return errors.New("send msg timeout") + } +} + +func (ch steppedTestKitMsgChan) sendMsgWithType(tp steppedTestkitMsgType) error { + return ch.sendMsg(tp, nil) +} + +func (ch steppedTestKitMsgChan) recvMsg() (*steppedTestKitMsg, error) { + select { + case msg := <-ch: + return msg, nil + case <-time.After(defaultChanTimeout): + return nil, errors.New("send msg timeout") + } +} + +func (ch steppedTestKitMsgChan) recvMsgWithCheck(tp steppedTestkitMsgType) (*steppedTestKitMsg, error) { + msg, err := ch.recvMsg() + if err != nil { + return nil, err + } + + if msg.tp != tp { + return nil, errors.Errorf("unexpected msg type: %v, expect: %v", msg.tp, tp) + } + + return msg, nil +} + +type steppedTestKitCommandContext struct { + t testing.TB + tk *TestKit + notifyBreakPointAndWait func(string) +} + +type steppedTestKitCommand func(ctx *steppedTestKitCommandContext) any + +// SteppedTestKit is the testkit that can run stepped command +type SteppedTestKit struct { + t testing.TB + tk *TestKit + + // ch1 is used to send msg from foreground to background + ch1 steppedTestKitMsgChan + // ch2 is used to send msg from background to foreground + ch2 steppedTestKitMsgChan + // breakPoints is the break points we want to stop at + breakPoints []string + // cmdStopAt is the current break point it stopped at + cmdStopAt string + // the result of the current command + cmdResult any +} + +// NewSteppedTestKit creates a new SteppedTestKit +func NewSteppedTestKit(t testing.TB, store kv.Storage) *SteppedTestKit { + tk := &SteppedTestKit{ + t: t, + tk: NewTestKit(t, store), + ch1: make(steppedTestKitMsgChan), + ch2: make(steppedTestKitMsgChan), + } + return tk +} + +// ExpectIdle checks no command is running +func (tk *SteppedTestKit) ExpectIdle() { + require.Equal(tk.t, "", tk.cmdStopAt) +} + +// ExpectStopOnBreakPoint checks stopped on the specified break point +func (tk *SteppedTestKit) ExpectStopOnBreakPoint(breakPoint string) { + require.Equal(tk.t, breakPoint, tk.cmdStopAt) +} + +// ExpectStopOnAnyBreakPoint checks stopped on any break point +func (tk *SteppedTestKit) ExpectStopOnAnyBreakPoint() { + require.NotEqual(tk.t, "", tk.cmdStopAt) +} + +// SetBreakPoints sets the break points we want to stop at +func (tk *SteppedTestKit) SetBreakPoints(breakPoints ...string) { + tk.breakPoints = breakPoints +} + +func (tk *SteppedTestKit) handleCommandMsg() { + msg, err := tk.ch2.recvMsg() + require.NoError(tk.t, err) + switch msg.tp { + case msgTpCmdDone: + tk.cmdStopAt = "" + if msg.val == errCommandRunFailed { + require.FailNow(tk.t, "internal command failed") + } else { + tk.cmdResult = msg.val + } + case msgTpCmdStopOnBreakPoint: + require.IsType(tk.t, "", msg.val) + require.NotEqual(tk.t, "", msg.val) + tk.cmdStopAt = msg.val.(string) + default: + require.FailNow(tk.t, "invalid msg type", "tp %v", msg.tp) + } +} + +func (tk *SteppedTestKit) beforeCommand() { + tk.ExpectIdle() + tk.cmdResult = nil +} + +func (tk *SteppedTestKit) steppedCommand(cmd steppedTestKitCommand) *SteppedTestKit { + tk.beforeCommand() + go func() { + var success bool + var result any + var breakPointPaths []string + defer func() { + if !success { + result = errCommandRunFailed + } + + tk.tk.Session().SetValue(breakpoint.NotifyBreakPointFuncKey, nil) + for _, path := range breakPointPaths { + require.NoError(tk.t, failpoint.Disable(path)) + } + + require.NoError(tk.t, tk.ch2.sendMsg(msgTpCmdDone, result)) + }() + + ctx := &steppedTestKitCommandContext{ + t: tk.t, + tk: tk.tk, + notifyBreakPointAndWait: func(breakPoint string) { + require.NoError(tk.t, tk.ch2.sendMsg(msgTpCmdStopOnBreakPoint, breakPoint)) + _, err := tk.ch1.recvMsgWithCheck(msgTpCmdContinue) + require.NoError(tk.t, err) + }, + } + + tk.tk.Session().SetValue(breakpoint.NotifyBreakPointFuncKey, ctx.notifyBreakPointAndWait) + for _, breakPoint := range tk.breakPoints { + path := "github.com/pingcap/tidb/util/breakpoint/" + breakPoint + require.NoError(tk.t, failpoint.Enable(path, "return")) + breakPointPaths = append(breakPointPaths, path) + } + + result = cmd(ctx) + success = true + }() + + tk.handleCommandMsg() + return tk +} + +// Continue continues current command +func (tk *SteppedTestKit) Continue() *SteppedTestKit { + tk.ExpectStopOnAnyBreakPoint() + require.NoError(tk.t, tk.ch1.sendMsgWithType(msgTpCmdContinue)) + tk.handleCommandMsg() + return tk +} + +// SteppedMustExec creates a new stepped task for MustExec +func (tk *SteppedTestKit) SteppedMustExec(sql string, args ...interface{}) *SteppedTestKit { + return tk.steppedCommand(func(_ *steppedTestKitCommandContext) any { + tk.MustExec(sql, args...) + return nil + }) +} + +// SteppedMustQuery creates a new stepped task for MustQuery +func (tk *SteppedTestKit) SteppedMustQuery(sql string, args ...interface{}) *SteppedTestKit { + return tk.steppedCommand(func(_ *steppedTestKitCommandContext) any { + return tk.MustQuery(sql, args...) + }) +} + +// MustExec executes a sql statement and asserts nil error. +func (tk *SteppedTestKit) MustExec(sql string, args ...interface{}) { + tk.beforeCommand() + tk.tk.MustExec(sql, args...) +} + +// MustQuery query the statements and returns result rows. +// If expected result is set it asserts the query result equals expected result. +func (tk *SteppedTestKit) MustQuery(sql string, args ...interface{}) *Result { + tk.beforeCommand() + result := tk.tk.MustQuery(sql, args...) + tk.cmdResult = result + return result +} + +// GetResult returns the result of the latest command +func (tk *SteppedTestKit) GetResult() any { + tk.ExpectIdle() + return tk.cmdResult +} + +// GetQueryResult returns the query result of the latest command +func (tk *SteppedTestKit) GetQueryResult() *Result { + tk.ExpectIdle() + require.IsType(tk.t, &Result{}, tk.cmdResult) + return tk.cmdResult.(*Result) +} diff --git a/testkit/testfork/BUILD.bazel b/testkit/testfork/BUILD.bazel new file mode 100644 index 0000000000000..743bd70da5b0d --- /dev/null +++ b/testkit/testfork/BUILD.bazel @@ -0,0 +1,18 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "testfork", + srcs = ["fork.go"], + importpath = "github.com/pingcap/tidb/testkit/testfork", + visibility = ["//visibility:public"], + deps = [ + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) + +go_test( + name = "testfork_test", + srcs = ["fork_test.go"], + embed = [":testfork"], +) diff --git a/testkit/testfork/fork.go b/testkit/testfork/fork.go new file mode 100644 index 0000000000000..aa70039b2f042 --- /dev/null +++ b/testkit/testfork/fork.go @@ -0,0 +1,135 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testfork + +import ( + "fmt" + "os" + "strings" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type pickStack struct { + stack [][]any + pos int + valid bool +} + +func newPickStack() *pickStack { + return &pickStack{ + valid: true, + } +} + +func (s *pickStack) NextStack() { + for len(s.stack) > 0 { + lastIndex := len(s.stack) - 1 + s.stack[lastIndex] = s.stack[lastIndex][1:] + if len(s.stack[lastIndex]) > 0 { + break + } + s.stack = s.stack[:lastIndex] + } + + s.pos = 0 + s.valid = len(s.stack) > 0 +} + +func (s *pickStack) PickValue(values []any) (any, error) { + if len(values) == 0 { + return nil, errors.New("values should not be empty") + } + + stackLen := len(s.stack) + if s.pos > stackLen { + return nil, errors.Newf("illegal state %d > %d", s.pos, stackLen) + } + + defer func() { + s.pos++ + }() + + if s.pos == stackLen { + s.stack = append(s.stack, values) + } + return s.stack[s.pos][0], nil +} + +func (s *pickStack) Values() []any { + values := make([]any, 0) + for _, v := range s.stack { + values = append(values, v[0]) + } + return values +} + +func (s *pickStack) ValuesText() string { + values := s.Values() + strValues := make([]string, len(values)) + for i, value := range values { + switch v := value.(type) { + case string: + strValues[i] = fmt.Sprintf(`"%s"`, v) + default: + strValues[i] = fmt.Sprintf("%v", v) + } + } + return "[" + strings.Join(strValues, " ") + "]" +} + +func (s *pickStack) Valid() bool { + return s.valid +} + +// T is used by for test +type T struct { + *testing.T + stack *pickStack +} + +// RunTest runs the test function `f` multiple times util all the values in `Pick` are tested. +func RunTest(t *testing.T, f func(t *T)) { + idx := 0 + for stack := newPickStack(); stack.Valid(); stack.NextStack() { + success := t.Run("", func(t *testing.T) { + f(&T{T: t, stack: stack}) + }) + + if !success { + _, err := fmt.Fprintf(os.Stderr, "SubTest #%v failed, failed values: %s\n", idx, stack.ValuesText()) + require.NoError(t, err) + } + idx++ + } +} + +// Pick returns a value from the values list +func Pick[E any](t *T, values []E) E { + slice := make([]any, len(values)) + for i, item := range values { + slice[i] = item + } + value, err := t.stack.PickValue(slice) + require.NoError(t, err) + return value.(E) +} + +// PickEnum returns a value from the value enums +func PickEnum[E any](t *T, item E, other ...E) E { + return Pick(t, append([]E{item}, other...)) +} diff --git a/testkit/testfork/fork_test.go b/testkit/testfork/fork_test.go new file mode 100644 index 0000000000000..64ed11b7a82e5 --- /dev/null +++ b/testkit/testfork/fork_test.go @@ -0,0 +1,50 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testfork + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestForkSubTest(t *testing.T) { + var values [][]any + RunTest(t, func(t *T) { + x := Pick(t, []int{1, 2, 3}) + y := PickEnum(t, "a", "b") + var z any + if x == 2 { + z = PickEnum(t, 10, 11) + } else { + z = Pick(t, []string{"g", "h"}) + } + values = append(values, []any{x, y, z}) + }) + require.Equal(t, [][]any{ + {1, "a", "g"}, + {1, "a", "h"}, + {1, "b", "g"}, + {1, "b", "h"}, + {2, "a", 10}, + {2, "a", 11}, + {2, "b", 10}, + {2, "b", 11}, + {3, "a", "g"}, + {3, "a", "h"}, + {3, "b", "g"}, + {3, "b", "h"}, + }, values) +} diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index 95c8ae77a9646..b0e5afb4dc0fd 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//kv", "//meta/autoid", "//parser", + "//parser/ast", "//parser/auth", "//parser/format", "//parser/model", diff --git a/tests/realtikvtest/sessiontest/session_test.go b/tests/realtikvtest/sessiontest/session_test.go index 4bc983d94b40f..9262ebe196498 100644 --- a/tests/realtikvtest/sessiontest/session_test.go +++ b/tests/realtikvtest/sessiontest/session_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" @@ -1363,7 +1364,7 @@ func TestDoDDLJobQuit(t *testing.T) { defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/storeCloseInLoop")) }() // this DDL call will enter deadloop before this fix - err = dom.DDL().CreateSchema(se, model.NewCIStr("testschema"), nil, nil) + err = dom.DDL().CreateSchema(se, &ast.CreateDatabaseStmt{Name: model.NewCIStr("testschema")}) require.Equal(t, "context canceled", err.Error()) } diff --git a/tools/check/ut.go b/tools/check/ut.go index 84b00d34d9e60..66a4fac7203a8 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -819,7 +819,7 @@ func (n *numa) testCommand(pkg string, fn string) *exec.Cmd { } func skipDIR(pkg string) bool { - skipDir := []string{"br", "cmd", "dumpling"} + skipDir := []string{"br", "cmd", "dumpling", "tests"} for _, ignore := range skipDir { if strings.HasPrefix(pkg, ignore) { return true diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 88770ae78c89f..926ba52b8cca0 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -221,6 +221,10 @@ func (bj BinaryJSON) Extract(pathExprList []PathExpression) (ret BinaryJSON, fou // even if len(pathExprList) equals to 1. found = true ret = buf[0] + // Fix https://github.com/pingcap/tidb/issues/30352 + if pathExprList[0].ContainsAnyAsterisk() { + ret = buildBinaryArray(buf) + } } else { found = true ret = buildBinaryArray(buf) diff --git a/types/json/binary_test.go b/types/json/binary_test.go index e276d81def524..4142c9cdf150f 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -62,12 +62,12 @@ func TestBinaryJSONExtract(t *testing.T) { {bj1, []string{"$.*[0]"}, mustParseBinaryFromString(t, `["world", 1, true, "d"]`), true, nil}, {bj1, []string{`$.a[*]."aa"`}, mustParseBinaryFromString(t, `["bb", "cc"]`), true, nil}, {bj1, []string{`$."\"hello\""`}, mustParseBinaryFromString(t, `"world"`), true, nil}, - {bj1, []string{`$**[1]`}, mustParseBinaryFromString(t, `"2"`), true, nil}, + {bj1, []string{`$**[1]`}, mustParseBinaryFromString(t, `["2"]`), true, nil}, {bj3, []string{`$.properties.$type`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, {bj4, []string{`$.properties.$type$type`}, mustParseBinaryFromString(t, `{"$a$a" : "TiDB"}`), true, nil}, {bj4, []string{`$.properties.$type$type.$a$a`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, {bj5, []string{`$.properties.$type.$a.$b`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, - {bj5, []string{`$.properties.$type.$a.*[0]`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, + {bj5, []string{`$.properties.$type.$a.*[0]`}, mustParseBinaryFromString(t, `["TiDB"]`), true, nil}, // test extract with multi path expressions. {bj1, []string{"$.a", "$[5]"}, mustParseBinaryFromString(t, `[[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]]`), true, nil}, diff --git a/util/breakpoint/BUILD.bazel b/util/breakpoint/BUILD.bazel new file mode 100644 index 0000000000000..ecc1f6e7f6e03 --- /dev/null +++ b/util/breakpoint/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "breakpoint", + srcs = ["breakpoint.go"], + importpath = "github.com/pingcap/tidb/util/breakpoint", + visibility = ["//visibility:public"], + deps = [ + "//sessionctx", + "//util/stringutil", + "@com_github_pingcap_failpoint//:failpoint", + ], +) diff --git a/util/breakpoint/breakpoint.go b/util/breakpoint/breakpoint.go new file mode 100644 index 0000000000000..b94a90be02954 --- /dev/null +++ b/util/breakpoint/breakpoint.go @@ -0,0 +1,34 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package breakpoint + +import ( + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" +) + +// NotifyBreakPointFuncKey is the key where break point notify function located +const NotifyBreakPointFuncKey = stringutil.StringerStr("breakPointNotifyFunc") + +// Inject injects a break point to a session +func Inject(sctx sessionctx.Context, name string) { + failpoint.Inject(name, func(_ failpoint.Value) { + val := sctx.Value(NotifyBreakPointFuncKey) + if breakPointNotifyAndWaitContinue, ok := val.(func(string)); ok { + breakPointNotifyAndWaitContinue(name) + } + }) +} diff --git a/util/column-mapping/column.go b/util/column-mapping/column.go index e276b0004882c..93a1443dc6173 100644 --- a/util/column-mapping/column.go +++ b/util/column-mapping/column.go @@ -112,7 +112,7 @@ func (r *Rule) Valid() error { if r.Expression == PartitionID { switch len(r.Arguments) { case 3, 4: - break + return nil default: return errors.NotValidf("arguments %v for patition id", r.Arguments) }