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 c9625069192e7..65f9d5b4b657c 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -220,6 +220,14 @@ def go_deps(): sum = "h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=", version = "v2.1.2", ) + go_repository( + name = "com_github_charithe_durationcheck", + build_file_proto_mode = "disable", + importpath = "github.com/charithe/durationcheck", + sum = "h1:mPP4ucLrf/rKZiIG/a9IPXHGlh8p4CzgpyTy6EEutYk=", + version = "v0.0.9", + ) + go_repository( name = "com_github_cheggaaa_pb_v3", build_file_proto_mode = "disable_global", @@ -919,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", @@ -1011,6 +1027,14 @@ def go_deps(): sum = "h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8=", version = "v0.0.0-20181017120253-0766667cb4d1", ) + go_repository( + name = "com_github_gordonklaus_ineffassign", + build_file_proto_mode = "disable", + importpath = "github.com/gordonklaus/ineffassign", + sum = "h1:PVRE9d4AQKmbelZ7emNig1+NT27DUmKZn5qXxfio54U=", + version = "v0.0.0-20210914165742-4cc7213b9bc8", + ) + go_repository( name = "com_github_gorilla_handlers", build_file_proto_mode = "disable_global", @@ -1541,6 +1565,14 @@ def go_deps(): sum = "h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=", version = "v0.2.0", ) + go_repository( + name = "com_github_kyoh86_exportloopref", + build_file_proto_mode = "disable", + importpath = "github.com/kyoh86/exportloopref", + sum = "h1:5Ry/at+eFdkX9Vsdw3qU4YkvGtzuVfzT4X7S77LoN/M=", + version = "v0.1.8", + ) + go_repository( name = "com_github_labstack_echo_v4", build_file_proto_mode = "disable_global", diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index df27a4848bdd0..22a47de065a71 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -534,7 +534,9 @@ func (bc *Client) BackupRanges( progressCallBack func(ProgressUnit), ) error { init := time.Now() - defer log.Info("Backup Ranges", zap.Duration("take", time.Since(init))) + defer func() { + log.Info("Backup Ranges", zap.Duration("take", time.Since(init))) + }() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("Client.BackupRanges", opentracing.ChildOf(span.Context())) 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/BUILD.bazel b/br/pkg/lightning/BUILD.bazel index 933ad8e1162d4..1cafc9ec5fbef 100644 --- a/br/pkg/lightning/BUILD.bazel +++ b/br/pkg/lightning/BUILD.bazel @@ -51,6 +51,7 @@ go_test( "//br/pkg/lightning/checkpoints", "//br/pkg/lightning/config", "//br/pkg/lightning/glue", + "//br/pkg/lightning/log", "//br/pkg/lightning/mydump", "//br/pkg/lightning/web", "@com_github_docker_go_units//:go-units", diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 76bdba192bd99..8a45472dddfcc 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 { 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/errormanager/BUILD.bazel b/br/pkg/lightning/errormanager/BUILD.bazel index e8c61ab00f0b2..7aea8447865e8 100644 --- a/br/pkg/lightning/errormanager/BUILD.bazel +++ b/br/pkg/lightning/errormanager/BUILD.bazel @@ -26,6 +26,7 @@ go_test( embed = [":errormanager"], deps = [ "//br/pkg/lightning/config", + "//br/pkg/lightning/log", "//br/pkg/utils", "@com_github_data_dog_go_sqlmock//:go-sqlmock", "@com_github_stretchr_testify//require", 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/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..6246c27ef411b 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1370,42 +1370,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/pkg/task/restore_raw.go b/br/pkg/task/restore_raw.go index d8fcb46475809..452cccfad8c42 100644 --- a/br/pkg/task/restore_raw.go +++ b/br/pkg/task/restore_raw.go @@ -146,8 +146,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR !cfg.LogProgress) // RawKV restore does not need to rewrite keys. - rewrite := &restore.RewriteRules{} - err = restore.SplitRanges(ctx, client, ranges, rewrite, updateCh, true) + err = restore.SplitRanges(ctx, client, ranges, nil, updateCh, true) if err != nil { return errors.Trace(err) } diff --git a/br/tests/br_rawkv/run.sh b/br/tests/br_rawkv/run.sh index 97450d3e65fc7..b32cca0f8e41f 100755 --- a/br/tests/br_rawkv/run.sh +++ b/br/tests/br_rawkv/run.sh @@ -98,12 +98,23 @@ run_test() { --key "$TEST_DIR/certs/br.key" \ --mode put --put-data "311121:31, 31112100:32, 311122:33, 31112200:34, 3111220000:35, 311123:36" + + # put some keys starts with t. https://github.com/pingcap/tidb/issues/35279 + # t_128_r_12 ----> 745f3132385f725f3132 + # t_128_r_13 ----> 745f3132385f725f3133 + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode put --put-data "745f3132385f725f3132:31, 745f3132385f725f3133:32" + checksum_ori=$(checksum 31 3130303030303030) checksum_partial=$(checksum 311111 311122) + checksum_t_prefix=$(checksum 745f3132385f725f3131 745f3132385f725f3134) # backup rawkv echo "backup start..." - run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 745f3132385f725f3134 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" # delete data in range[start-key, end-key) clean 31 3130303030303030 @@ -153,6 +164,21 @@ run_test() { fail_and_exit fi + echo "t prefix restore start..." + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start "745f3132385f725f3131" --end "745f3132385f725f3134" --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode scan --start-key 745f3132385f725f3131 --end-key 745f3132385f725f3134 + + checksum_new=$(checksum 745f3132385f725f3131 745f3132385f725f3134) + + if [ "$checksum_new" != "$checksum_t_prefix" ];then + echo "checksum failed after restore" + fail_and_exit + fi + export GO_FAILPOINTS="" } 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 22bc4b679c26a..6957be918b42b 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -81,6 +81,10 @@ nogo( "@org_golang_x_tools//go/analysis/passes/unreachable:go_default_library", "@org_golang_x_tools//go/analysis/passes/unsafeptr:go_default_library", "@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/durationcheck/BUILD.bazel b/build/linter/durationcheck/BUILD.bazel new file mode 100644 index 0000000000000..556720ba0609c --- /dev/null +++ b/build/linter/durationcheck/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "durationcheck", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/durationcheck", + visibility = ["//visibility:public"], + deps = [ + "//build/linter/util", + "@com_github_charithe_durationcheck//:durationcheck", + ], +) diff --git a/build/linter/durationcheck/analyzer.go b/build/linter/durationcheck/analyzer.go new file mode 100644 index 0000000000000..1ebee429959fd --- /dev/null +++ b/build/linter/durationcheck/analyzer.go @@ -0,0 +1,27 @@ +// 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 durationcheck + +import ( + "github.com/charithe/durationcheck" + "github.com/pingcap/tidb/build/linter/util" +) + +// Analyzer is the analyzer struct of durationcheck. +var Analyzer = durationcheck.Analyzer + +func init() { + util.SkipAnalyzer(Analyzer) +} diff --git a/build/linter/exportloopref/BUILD.bazel b/build/linter/exportloopref/BUILD.bazel new file mode 100644 index 0000000000000..29070c40f343c --- /dev/null +++ b/build/linter/exportloopref/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "exportloopref", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/exportloopref", + visibility = ["//visibility:public"], + deps = ["@com_github_kyoh86_exportloopref//:exportloopref"], +) diff --git a/build/linter/exportloopref/analyzer.go b/build/linter/exportloopref/analyzer.go new file mode 100644 index 0000000000000..4a448a8092ed4 --- /dev/null +++ b/build/linter/exportloopref/analyzer.go @@ -0,0 +1,20 @@ +// 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 exportloopref + +import "github.com/kyoh86/exportloopref" + +// Analyzer is the analyzer struct of exportloopref. +var Analyzer = exportloopref.Analyzer 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/linter/ineffassign/BUILD.bazel b/build/linter/ineffassign/BUILD.bazel new file mode 100644 index 0000000000000..706ea59702019 --- /dev/null +++ b/build/linter/ineffassign/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "ineffassign", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/ineffassign", + visibility = ["//visibility:public"], + deps = ["@com_github_gordonklaus_ineffassign//pkg/ineffassign"], +) diff --git a/build/linter/ineffassign/analyzer.go b/build/linter/ineffassign/analyzer.go new file mode 100644 index 0000000000000..42179cb3ad567 --- /dev/null +++ b/build/linter/ineffassign/analyzer.go @@ -0,0 +1,20 @@ +// 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 ineffassign + +import "github.com/gordonklaus/ineffassign/pkg/ineffassign" + +// Analyzer is the analyzer struct of ineffassign. +var Analyzer = ineffassign.Analyzer diff --git a/build/linter/util/util.go b/build/linter/util/util.go index d438ea0057a6e..d476173a973a0 100644 --- a/build/linter/util/util.go +++ b/build/linter/util/util.go @@ -55,7 +55,7 @@ func parseDirective(s string) (cmd skipType, args []string) { } return skipNone, nil } - s = strings.TrimPrefix(s, "//nolint: ") + s = strings.TrimPrefix(s, "//nolint:") return skipLinter, []string{s} } diff --git a/build/nogo_config.json b/build/nogo_config.json index f05718081c8ac..cefdb5fe4aa11 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -80,18 +80,40 @@ ".*_generated\\.go$": "ignore generated code" } }, + "durationcheck": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "errorsas": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "exportloopref": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "findcall": { "exclude_files": { "/external/": "no need to vet third party code", ".*_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", @@ -104,6 +126,12 @@ ".*_generated\\.go$": "ignore generated code" } }, + "ineffassign": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "inspect": { "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/config/config.go b/config/config.go index 48e069bcfb50b..0dc6cabc75727 100644 --- a/config/config.go +++ b/config/config.go @@ -117,6 +117,7 @@ var ( map[string]string{ "check-mb4-value-in-utf8": "tidb_check_mb4_value_in_utf8", "enable-collect-execution-info": "tidb_enable_collect_execution_info", + "max-server-connections": "max_connections", }, }, { @@ -474,6 +475,7 @@ type Instance struct { EnableCollectExecutionInfo bool `toml:"tidb_enable_collect_execution_info" json:"tidb_enable_collect_execution_info"` PluginDir string `toml:"plugin_dir" json:"plugin_dir"` PluginLoad string `toml:"plugin_load" json:"plugin_load"` + MaxConnections uint32 `toml:"max_connections" json:"max_connections"` } func (l *Log) getDisableTimestamp() bool { @@ -850,6 +852,7 @@ var defaultConf = Config{ EnableCollectExecutionInfo: true, PluginDir: "/data/deploy/plugin", PluginLoad: "", + MaxConnections: 0, }, Status: Status{ ReportStatus: true, diff --git a/config/config.toml.example b/config/config.toml.example index 1e1fc7aae73f5..afc97e60e74f8 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -89,9 +89,6 @@ repair-mode = false # In repair mode, repairing table which is not in repair list will get wrong database or wrong table error. repair-table-list = [] -# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. -max-server-connections = 0 - # Whether new collations are enabled, as indicated by its name, this configuration entry take effect ONLY when a TiDB cluster bootstraps for the first time. new_collations_enabled_on_first_bootstrap = true @@ -468,3 +465,6 @@ tidb_slow_log_threshold = 300 # tidb_record_plan_in_slow_log is used to enable record query plan in slow log. # 0 is disable. 1 is enable. tidb_record_plan_in_slow_log = 1 + +# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. +max_connections = 0 diff --git a/config/config_test.go b/config/config_test.go index 391bd874d3942..2e044062bd4e8 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -282,9 +282,6 @@ repair-mode = false # In repair mode, repairing table which is not in repair list will get wrong database or wrong table error. repair-table-list = [] -# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. -max-server-connections = 0 - # Whether new collations are enabled, as indicated by its name, this configuration entry take effect ONLY when a TiDB cluster bootstraps for the first time. new_collations_enabled_on_first_bootstrap = true @@ -309,6 +306,11 @@ deprecate-integer-display-length = false # See https://dev.mysql.com/doc/refman/8.0/en/string-type-syntax.html for more details. enable-enum-length-limit = true +[instance] + +# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. +max_connections = 0 + [log] # Log level: debug, info, warn, error, fatal. level = "info" @@ -707,7 +709,7 @@ unrecognized-option-test = true match, err := regexp.Match("(?:.|\n)*invalid configuration option(?:.|\n)*", []byte(err.Error())) require.NoError(t, err) require.True(t, match) - require.Equal(t, uint32(0), conf.MaxServerConnections) + require.Equal(t, uint32(0), conf.Instance.MaxConnections) err = f.Truncate(0) require.NoError(t, err) @@ -722,7 +724,6 @@ delay-clean-table-lock = 5 split-region-max-num=10000 server-version = "test_version" repair-mode = true -max-server-connections = 200 max-index-length = 3080 index-limit = 70 table-column-count-limit = 4000 @@ -768,6 +769,8 @@ grpc-keepalive-timeout = 10 grpc-concurrent-streams = 2048 grpc-initial-window-size = 10240 grpc-max-send-msg-size = 40960 +[instance] +max_connections = 200 `) require.NoError(t, err) @@ -797,7 +800,7 @@ grpc-max-send-msg-size = 40960 require.Equal(t, uint64(10000), conf.SplitRegionMaxNum) require.True(t, conf.RepairMode) require.Equal(t, uint64(16), conf.TiKVClient.ResolveLockLiteThreshold) - require.Equal(t, uint32(200), conf.MaxServerConnections) + require.Equal(t, uint32(200), conf.Instance.MaxConnections) require.Equal(t, []string{"tiflash"}, conf.IsolationRead.Engines) require.Equal(t, 3080, conf.MaxIndexLength) require.Equal(t, 70, conf.IndexLimit) 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/column_type_change_test.go b/ddl/column_type_change_test.go index 71fa1da9ee927..c68f87c9a7dde 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -661,7 +661,7 @@ func TestColumnTypeChangeFromStringToOthers(t *testing.T) { // MySQL will get "ERROR 1366 (HY000): Incorrect DECIMAL value: '0' for column '' at row -1" error. tk.MustExec("insert into t(vc) values ('abc')") - tk.MustGetErrCode("alter table t modify vc decimal(5,3)", errno.ErrBadNumber) + tk.MustGetErrCode("alter table t modify vc decimal(5,3)", errno.ErrTruncatedWrongValue) } func TestColumnTypeChangeFromNumericToOthers(t *testing.T) { @@ -1298,7 +1298,7 @@ func TestColumnTypeChangeFromJsonToOthers(t *testing.T) { tk.MustExec("alter table t modify ui decimal(20, 10)") tk.MustExec("alter table t modify f64 decimal(20, 10)") // MySQL will get "ERROR 1366 (HY000): Incorrect DECIMAL value: '0' for column '' at row -1". - tk.MustGetErrCode("alter table t modify str decimal(20, 10)", errno.ErrBadNumber) + tk.MustGetErrCode("alter table t modify str decimal(20, 10)", errno.ErrTruncatedWrongValue) tk.MustExec("alter table t modify nul decimal(20, 10)") tk.MustQuery("select * from t").Check(testkit.Rows("{\"obj\": 100} [-1, 0, 1] null 1.0000000000 0.0000000000 -22.0000000000 22.0000000000 323232323.3232323500 \"json string\" ")) 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 fff7e16f8a33c..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") @@ -3814,6 +3813,26 @@ func TestIssue29282(t *testing.T) { } } +// See https://github.com/pingcap/tidb/issues/35644 +func TestCreateTempTableInTxn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("begin") + tk.MustExec("create temporary table t1(id int)") + tk.MustQuery("select * from t1") + tk.MustExec("commit") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("create table tt(id int)") + tk1.MustExec("begin") + tk1.MustExec("create temporary table t1(id int)") + tk1.MustExec("insert into tt select * from t1") + tk1.MustExec("drop table tt") +} + // See https://github.com/pingcap/tidb/issues/29327 func TestEnumDefaultValue(t *testing.T) { store, clean := testkit.CreateMockStore(t) diff --git a/ddl/ddl.go b/ddl/ddl.go index 21edcd8590d8e..0ea01e1222eb5 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -101,9 +101,9 @@ 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) @@ -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)} @@ -783,12 +792,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..01f57c3ce1d18 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) } 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/ddl/table.go b/ddl/table.go index decbc8d3d0acd..9ccd9549ebeca 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -455,7 +455,7 @@ func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in failpoint.Inject("mockRecoverTableCommitErr", func(val failpoint.Value) { if val.(bool) && atomic.CompareAndSwapUint32(&mockRecoverTableCommitErrOnce, 0, 1) { - err = failpoint.Enable(`tikvclient/mockCommitErrorOpt`, "return(true)") + _ = failpoint.Enable(`tikvclient/mockCommitErrorOpt`, "return(true)") } }) diff --git a/distsql/select_result.go b/distsql/select_result.go index 58d0abf49d04c..2a41e318579b4 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -54,8 +54,8 @@ var ( ) var ( - coprCacheHistogramHit = metrics.DistSQLCoprCacheHistogram.WithLabelValues("hit") - coprCacheHistogramMiss = metrics.DistSQLCoprCacheHistogram.WithLabelValues("miss") + coprCacheCounterHit = metrics.DistSQLCoprCacheCounter.WithLabelValues("hit") + coprCacheCounterMiss = metrics.DistSQLCoprCacheCounter.WithLabelValues("miss") ) var ( @@ -158,8 +158,8 @@ type selectResult struct { func (r *selectResult) fetchResp(ctx context.Context) error { defer func() { if r.stats != nil { - coprCacheHistogramHit.Observe(float64(r.stats.CoprCacheHitNum)) - coprCacheHistogramMiss.Observe(float64(len(r.stats.copRespTime) - int(r.stats.CoprCacheHitNum))) + coprCacheCounterHit.Add(float64(r.stats.CoprCacheHitNum)) + coprCacheCounterMiss.Add(float64(len(r.stats.copRespTime) - int(r.stats.CoprCacheHitNum))) // Ignore internal sql. if !r.ctx.GetSessionVars().InRestrictedSQL && len(r.stats.copRespTime) > 0 { ratio := float64(r.stats.CoprCacheHitNum) / float64(len(r.stats.copRespTime)) 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..e9b7c23298645 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -152,6 +152,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 adb6c34865aa6..faf894816c702 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" @@ -415,10 +416,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 @@ -795,10 +793,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if err != nil { return nil, err } - - failpoint.Inject("hookAfterOnStmtRetryWithLockError", func() { - sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookAfterOnStmtRetryWithLockErrorKey) - }) + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() if err != nil { diff --git a/executor/ddl.go b/executor/ddl.go index f77091b5a399b..1553be2299fd7 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -34,7 +34,6 @@ 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" @@ -248,70 +247,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 +311,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 +319,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 = "" diff --git a/executor/set_test.go b/executor/set_test.go index eb171e872d8c4..9a7213571fddc 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1021,16 +1021,17 @@ func TestValidateSetVar(t *testing.T) { result.Check(testkit.Rows("SYSTEM")) // The following cases test value out of range and illegal type when setting system variables. - // See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html for more details. + // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html for more details. tk.MustExec("set @@global.max_connections=100001") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '100001'")) result = tk.MustQuery("select @@global.max_connections;") result.Check(testkit.Rows("100000")) + // "max_connections == 0" means there is no limitation on the number of connections. tk.MustExec("set @@global.max_connections=-1") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) result = tk.MustQuery("select @@global.max_connections;") - result.Check(testkit.Rows("1")) + result.Check(testkit.Rows("0")) err = tk.ExecToErr("set @@global.max_connections='hello'") require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar)) @@ -1077,7 +1078,7 @@ func TestValidateSetVar(t *testing.T) { tk.MustExec("set @@global.max_connections=-1") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) result = tk.MustQuery("select @@global.max_connections;") - result.Check(testkit.Rows("1")) + result.Check(testkit.Rows("0")) err = tk.ExecToErr("set @@global.max_connections='hello'") require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar)) @@ -1333,15 +1334,15 @@ func TestSelectGlobalVar(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("151")) - tk.MustQuery("select @@max_connections;").Check(testkit.Rows("151")) + tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("0")) + tk.MustQuery("select @@max_connections;").Check(testkit.Rows("0")) tk.MustExec("set @@global.max_connections=100;") tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("100")) tk.MustQuery("select @@max_connections;").Check(testkit.Rows("100")) - tk.MustExec("set @@global.max_connections=151;") + tk.MustExec("set @@global.max_connections=0;") // test for unknown variable. err := tk.ExecToErr("select @@invalid") diff --git a/executor/splittest/BUILD.bazel b/executor/splittest/BUILD.bazel index 759cfbfa64912..3fbafa70c12ca 100644 --- a/executor/splittest/BUILD.bazel +++ b/executor/splittest/BUILD.bazel @@ -10,6 +10,7 @@ go_test( shard_count = 5, deps = [ "//ddl", + "//domain/infosync", "//errno", "//parser/mysql", "//parser/terror", 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/expression/builtin_cast.go b/expression/builtin_cast.go index c281e2de80302..cc7b9d3683b71 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -917,7 +917,7 @@ func (b *builtinCastRealAsDurationSig) evalDuration(row chunk.Row) (res types.Du if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1095,7 +1095,7 @@ func (b *builtinCastDecimalAsDurationSig) evalDuration(row chunk.Row) (res types if isNull || err != nil { return res, true, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) // ErrTruncatedWrongVal needs to be considered NULL. @@ -1318,16 +1318,12 @@ func (b *builtinCastStringAsDurationSig) evalDuration(row chunk.Row) (res types. if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.GetDecimal()) + res, isNull, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) - // ZeroDuration of error ErrTruncatedWrongVal needs to be considered NULL. - if res == types.ZeroDuration { - return res, true, err - } } - return res, false, err + return res, isNull, err } type builtinCastTimeAsTimeSig struct { @@ -1765,7 +1761,7 @@ func (b *builtinCastJSONAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index ee29a768dd702..141ff49c26f13 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -939,7 +939,7 @@ func (b *builtinCastStringAsDurationSig) vecEvalDuration(input *chunk.Chunk, res if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), b.tp.GetDecimal()) + dur, isNull, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -947,7 +947,7 @@ func (b *builtinCastStringAsDurationSig) vecEvalDuration(input *chunk.Chunk, res if err != nil { return err } - if dur == types.ZeroDuration { + if isNull { result.SetNull(i, true) continue } @@ -1213,7 +1213,7 @@ func (b *builtinCastRealAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), b.tp.GetDecimal()) + dur, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1793,7 +1793,7 @@ func (b *builtinCastDecimalAsDurationSig) vecEvalDuration(input *chunk.Chunk, re if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), b.tp.GetDecimal()) + dur, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1880,7 +1880,7 @@ func (b *builtinCastJSONAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if err != nil { return nil } - dur, err = types.ParseDuration(ctx, s, b.tp.GetDecimal()) + dur, _, err = types.ParseDuration(ctx, s, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { err = ctx.HandleTruncate(err) } diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 4a8a5bcca13b4..d5d17bd4a6ecb 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -2158,7 +2158,7 @@ func (b *builtinCurrentTime0ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) if err != nil { return types.Duration{}, true, err } @@ -2186,7 +2186,7 @@ func (b *builtinCurrentTime1ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFSPFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) if err != nil { return types.Duration{}, true, err } @@ -2246,7 +2246,7 @@ func (b *builtinTimeSig) evalDuration(row chunk.Row) (res types.Duration, isNull fsp = tmpFsp sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseDuration(sc, expr, fsp) + res, _, err = types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2273,7 +2273,7 @@ func (c *timeLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if !isDuration(str) { return nil, types.ErrWrongValue.GenWithStackByArgs(types.TimeStr, str) } - duration, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, str, types.GetFsp(str)) + duration, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, str, types.GetFsp(str)) if err != nil { return nil, err } @@ -2639,7 +2639,7 @@ func (b *builtinExtractDatetimeFromStringSig) evalInt(row chunk.Row) (int64, boo sc := b.ctx.GetSessionVars().StmtCtx switch strings.ToUpper(unit) { case "DAY_MICROSECOND", "DAY_SECOND", "DAY_MINUTE", "DAY_HOUR": - dur, err := types.ParseDuration(sc, dtStr, types.GetFsp(dtStr)) + dur, _, err := types.ParseDuration(sc, dtStr, types.GetFsp(dtStr)) if err != nil { return 0, true, err } @@ -4373,7 +4373,7 @@ func (b *builtinTimestamp2ArgsSig) evalTime(row chunk.Row) (types.Time, bool, er if !isDuration(arg1) { return types.ZeroTime, true, nil } - duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -4535,7 +4535,7 @@ func strDatetimeAddDuration(sc *stmtctx.StatementContext, d string, arg1 types.D // strDurationAddDuration adds duration to duration string, returns a string value. func strDurationAddDuration(sc *stmtctx.StatementContext, d string, arg1 types.Duration) (string, error) { - arg0, err := types.ParseDuration(sc, d, types.MaxFsp) + arg0, _, err := types.ParseDuration(sc, d, types.MaxFsp) if err != nil { return "", err } @@ -4572,7 +4572,7 @@ func strDatetimeSubDuration(sc *stmtctx.StatementContext, d string, arg1 types.D // strDurationSubDuration subtracts duration from duration string, returns a string value. func strDurationSubDuration(sc *stmtctx.StatementContext, d string, arg1 types.Duration) (string, error) { - arg0, err := types.ParseDuration(sc, d, types.MaxFsp) + arg0, _, err := types.ParseDuration(sc, d, types.MaxFsp) if err != nil { return "", err } @@ -4721,7 +4721,7 @@ func (b *builtinAddDatetimeAndStringSig) evalTime(row chunk.Row) (types.Time, bo return types.ZeroDatetime, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4802,7 +4802,7 @@ func (b *builtinAddDurationAndStringSig) evalDuration(row chunk.Row) (types.Dura return types.ZeroDuration, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4904,7 +4904,7 @@ func (b *builtinAddStringAndStringSig) evalString(row chunk.Row) (result string, return "", isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err = types.ParseDuration(sc, arg1Str, getFsp4TimeAddSub(arg1Str)) + arg1, _, err = types.ParseDuration(sc, arg1Str, getFsp4TimeAddSub(arg1Str)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4987,7 +4987,7 @@ func (b *builtinAddDateAndStringSig) evalString(row chunk.Row) (string, bool, er return "", true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5253,7 +5253,8 @@ func (b *builtinMakeTimeSig) makeTime(hour int64, minute int64, second float64, second = 59 } fsp := b.tp.GetDecimal() - return types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) + d, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) + return d, err } // evalDuration evals a builtinMakeTimeIntSig. @@ -5547,7 +5548,7 @@ func (b *builtinSecToTimeSig) evalDuration(row chunk.Row) (types.Duration, bool, secondDemical = float64(second) + demical var dur types.Duration - dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) + dur, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) if err != nil { return types.Duration{}, err != nil, err } @@ -5676,7 +5677,7 @@ func (b *builtinSubDatetimeAndStringSig) evalTime(row chunk.Row) (types.Time, bo return types.ZeroDatetime, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5775,7 +5776,7 @@ func (b *builtinSubStringAndStringSig) evalString(row chunk.Row) (result string, return "", isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err = types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err = types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5867,7 +5868,7 @@ func (b *builtinSubDurationAndStringSig) evalDuration(row chunk.Row) (types.Dura return types.ZeroDuration, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5945,7 +5946,7 @@ func (b *builtinSubDateAndStringSig) evalString(row chunk.Row) (string, bool, er return "", true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -6310,7 +6311,7 @@ func (b *builtinUTCTimeWithoutArgSig) evalDuration(row chunk.Row) (types.Duratio if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), 0) + v, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), 0) return v, false, err } @@ -6341,7 +6342,7 @@ func (b *builtinUTCTimeWithArgSig) evalDuration(row chunk.Row) (types.Duration, if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) + v, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) return v, false, err } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index c081b11f0f0f8..fb8387eb022fb 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -962,7 +962,7 @@ func TestAddTimeSig(t *testing.T) { {"-110:00:00", "1 02:00:00", "-84:00:00"}, } for _, c := range tbl { - dur, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) + dur, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) require.NoError(t, err) tmpInput := types.NewDurationDatum(dur) tmpInputDuration := types.NewStringDatum(c.InputDuration) @@ -1063,7 +1063,7 @@ func TestSubTimeSig(t *testing.T) { {"235959", "00:00:01", "23:59:58"}, } for _, c := range tbl { - dur, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) + dur, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) require.NoError(t, err) tmpInput := types.NewDurationDatum(dur) tmpInputDuration := types.NewStringDatum(c.InputDuration) diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index b292164813dcc..0d6b4321095f5 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -421,7 +421,7 @@ func (b *builtinUTCTimeWithArgSig) vecEvalDuration(input *chunk.Chunk, result *c if fsp < int64(types.MinFsp) { return errors.Errorf("Invalid negative %d specified, must in [0, 6]", fsp) } - res, err := types.ParseDuration(stmtCtx, utc, int(fsp)) + res, _, err := types.ParseDuration(stmtCtx, utc, int(fsp)) if err != nil { return err } @@ -1954,7 +1954,7 @@ func (b *builtinSecToTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk. second = seconds % 60 } secondDemical := float64(second) + demical - duration, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) + duration, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) if err != nil { return err } @@ -1975,7 +1975,7 @@ func (b *builtinUTCTimeWithoutArgSig) vecEvalDuration(input *chunk.Chunk, result if err != nil { return err } - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), types.DefaultFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), types.DefaultFsp) if err != nil { return err } @@ -2378,7 +2378,7 @@ func (b *builtinCurrentTime0ArgSig) vecEvalDuration(input *chunk.Chunk, result * } tz := b.ctx.GetSessionVars().Location() dur := nowTs.In(tz).Format(types.TimeFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) if err != nil { return err } @@ -2426,7 +2426,7 @@ func (b *builtinTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Colum } fsp = tmpFsp - res, err := types.ParseDuration(sc, expr, fsp) + res, _, err := types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2572,7 +2572,7 @@ func (b *builtinCurrentTime1ArgSig) vecEvalDuration(input *chunk.Chunk, result * result.ResizeGoDuration(n, false) durations := result.GoDurations() for i := 0; i < n; i++ { - res, err := types.ParseDuration(stmtCtx, dur, int(i64s[i])) + res, _, err := types.ParseDuration(stmtCtx, dur, int(i64s[i])) if err != nil { return err } @@ -2751,7 +2751,7 @@ func (b *builtinTimestamp2ArgsSig) vecEvalTime(input *chunk.Chunk, result *chunk continue } - duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if err = handleInvalidTimeError(b.ctx, err); err != nil { return err diff --git a/expression/builtin_time_vec_generated.go b/expression/builtin_time_vec_generated.go index 7c18af6e81ddf..4e897affc66d9 100644 --- a/expression/builtin_time_vec_generated.go +++ b/expression/builtin_time_vec_generated.go @@ -122,7 +122,7 @@ func (b *builtinAddDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -248,7 +248,7 @@ func (b *builtinAddDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, res continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -406,7 +406,7 @@ func (b *builtinAddStringAndStringSig) vecEvalString(input *chunk.Chunk, result // calculate sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -566,7 +566,7 @@ func (b *builtinAddDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -737,7 +737,7 @@ func (b *builtinSubDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -862,7 +862,7 @@ func (b *builtinSubDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, res continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -1020,7 +1020,7 @@ func (b *builtinSubStringAndStringSig) vecEvalString(input *chunk.Chunk, result // calculate sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -1180,7 +1180,7 @@ func (b *builtinSubDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go index 3bf16f0076594..73067419f8cfc 100644 --- a/expression/generator/time_vec.go +++ b/expression/generator/time_vec.go @@ -63,7 +63,7 @@ import ( continue }{{ end }} sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, {{if eq .Output.TypeName "String"}}getFsp4TimeAddSub{{else}}types.GetFsp{{end}}(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, {{if eq .Output.TypeName "String"}}getFsp4TimeAddSub{{else}}types.GetFsp{{end}}(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -191,7 +191,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) diff --git a/expression/integration_test.go b/expression/integration_test.go index 8c10fc96b5d30..b3dc43fe084d1 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -6233,11 +6233,11 @@ func TestGlobalCacheCorrectness(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 151")) + tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 0")) tk.MustExec("SET GLOBAL max_connections=1234") tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 1234")) // restore - tk.MustExec("SET GLOBAL max_connections=151") + tk.MustExec("SET GLOBAL max_connections=0") } func TestRedundantColumnResolve(t *testing.T) { diff --git a/go.mod b/go.mod index cd5327db9c2c2..9ad9135be4085 100644 --- a/go.mod +++ b/go.mod @@ -97,6 +97,10 @@ 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 c5f5f515043a7..2dcc2c3e4cfeb 100644 --- a/go.sum +++ b/go.sum @@ -131,6 +131,8 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/charithe/durationcheck v0.0.9 h1:mPP4ucLrf/rKZiIG/a9IPXHGlh8p4CzgpyTy6EEutYk= +github.com/charithe/durationcheck v0.0.9/go.mod h1:SSbRIBVfMjCi/kEB6K65XEA83D6prSM8ap1UCpNKtgg= github.com/cheggaaa/pb/v3 v3.0.8 h1:bC8oemdChbke2FHIIGy9mn4DPJ2caZYQnfbRqwmdCoA= github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= github.com/cheynewallace/tabby v1.1.1 h1:JvUR8waht4Y0S3JF17G6Vhyt+FRhnqVCkk8l4YrOU54= @@ -341,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= @@ -395,6 +399,8 @@ github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pf github.com/googleapis/gax-go/v2 v2.1.1 h1:dp3bWCh+PPO1zjRRiCSczJav13sBvG4UhNyVTa1KqdU= github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0eJc8R6ouapiM= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gordonklaus/ineffassign v0.0.0-20210914165742-4cc7213b9bc8 h1:PVRE9d4AQKmbelZ7emNig1+NT27DUmKZn5qXxfio54U= +github.com/gordonklaus/ineffassign v0.0.0-20210914165742-4cc7213b9bc8/go.mod h1:Qcp2HIAYhR7mNUVSIxZww3Guk4it82ghYcEXIAk+QT0= github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= @@ -529,6 +535,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kyoh86/exportloopref v0.1.8 h1:5Ry/at+eFdkX9Vsdw3qU4YkvGtzuVfzT4X7S77LoN/M= +github.com/kyoh86/exportloopref v0.1.8/go.mod h1:1tUcJeiioIs7VWe5gcOObrux3lb66+sBqGZrRkMwPgg= github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= @@ -1196,6 +1204,7 @@ golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200622203043-20e05c1c8ffa/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index c96fa6d59dc83..3200166d21af0 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -487,6 +487,11 @@ func (is *LocalTemporaryTables) RemoveTable(schema, table model.CIStr) (exist bo return true } +// Count gets the count of the temporary tables. +func (is *LocalTemporaryTables) Count() int { + return len(is.idx2table) +} + // SchemaByTable get a table's schema name func (is *LocalTemporaryTables) SchemaByTable(tableInfo *model.TableInfo) (*model.DBInfo, bool) { if tableInfo == nil { diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index b27217a316c03..7487adadb6168 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -2513,10 +2513,9 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The total time of distsql execution(second)", }, "tidb_distsql_copr_cache": { - Comment: "The quantile of TiDB distsql coprocessor cache", - PromQL: "histogram_quantile($QUANTILE, sum(rate(tidb_distsql_copr_cache_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance))", - Labels: []string{"instance", "type"}, - Quantile: 0.95, + Comment: "The total count of TiDB distsql coprocessor cache", + PromQL: "sum(rate(tidb_distsql_copr_cache{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance))", + Labels: []string{"instance", "type"}, }, "tidb_execute_total_count": { PromQL: "sum(increase(tidb_session_execute_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,sql_type)", diff --git a/kv/BUILD.bazel b/kv/BUILD.bazel index 600cec889528f..fed476f803dfc 100644 --- a/kv/BUILD.bazel +++ b/kv/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//util/dbterror", "//util/logutil", "//util/memory", + "//util/set", "//util/trxevents", "@com_github_coocood_freecache//:freecache", "@com_github_pingcap_errors//:errors", diff --git a/metrics/distsql.go b/metrics/distsql.go index 3a4527da510ae..0452f54ee9913 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -53,12 +53,11 @@ var ( Help: "number of partial results for each query.", }, ) - DistSQLCoprCacheHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ + DistSQLCoprCacheCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "distsql", Name: "copr_cache", Help: "coprocessor cache hit, evict and miss number", - Buckets: prometheus.ExponentialBuckets(1, 2, 16), }, []string{LblType}) ) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index a9bbf1ab4b83f..fcb3dfa5b55ca 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -6855,7 +6855,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_copr_cache_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tidb_distsql_copr_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", diff --git a/metrics/metrics.go b/metrics/metrics.go index 31217926b6c30..565790480c6c7 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -102,7 +102,7 @@ func RegisterMetrics() { prometheus.MustRegister(DDLWorkerHistogram) prometheus.MustRegister(DeploySyncerHistogram) prometheus.MustRegister(DistSQLPartialCountHistogram) - prometheus.MustRegister(DistSQLCoprCacheHistogram) + prometheus.MustRegister(DistSQLCoprCacheCounter) prometheus.MustRegister(DistSQLQueryHistogram) prometheus.MustRegister(DistSQLScanKeysHistogram) prometheus.MustRegister(DistSQLScanKeysPartialHistogram) 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/lexer_test.go b/parser/lexer_test.go index 91d45d559d863..3328bc315237c 100644 --- a/parser/lexer_test.go +++ b/parser/lexer_test.go @@ -375,16 +375,16 @@ func TestFeatureIDsComment(t *testing.T) { requires.Equal(t, identifier, tok) requires.Equal(t, "auto_random", lit) requires.Equal(t, Pos{1, 16, 16}, pos) - tok, pos, _ = l.scan() + tok, _, _ = l.scan() requires.Equal(t, int('('), tok) _, pos, lit = l.scan() requires.Equal(t, "5", lit) requires.Equal(t, Pos{1, 28, 28}, pos) - tok, pos, _ = l.scan() + tok, _, _ = l.scan() requires.Equal(t, int(')'), tok) l = NewScanner("/*T![unsupported_feature] unsupported(123) */") - tok, pos, _ = l.scan() + tok, _, _ = l.scan() requires.Equal(t, 0, tok) } diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 1769d3b526d7f..549a8119e6b33 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -270,6 +270,7 @@ type MultiSchemaInfo struct { AlterIndexes []CIStr `json:"-"` RelativeColumns []CIStr `json:"-"` + PositionColumns []CIStr `json:"-"` } func NewMultiSchemaInfo() *MultiSchemaInfo { @@ -292,6 +293,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 +362,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 +432,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 +482,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 +668,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/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/plan_cost.go b/planner/core/plan_cost.go index c0598e74f2801..5612933b9cf9b 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/paging" @@ -223,11 +224,8 @@ func (p *PhysicalIndexLookUpReader) GetPlanCost(taskType property.TaskType, cost // table-side seek cost p.planCost += estimateNetSeekCost(p.tablePlan) - if p.ctx.GetSessionVars().CostModelVersion == modelVer2 { - // accumulate the real double-read cost: numDoubleReadTasks * seekFactor - numDoubleReadTasks := p.estNumDoubleReadTasks(costFlag) - p.planCost += numDoubleReadTasks * p.ctx.GetSessionVars().GetSeekFactor(ts.Table) - } + // double read cost + p.planCost += p.estDoubleReadCost(ts.Table, costFlag) // consider concurrency p.planCost /= float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) @@ -238,14 +236,21 @@ func (p *PhysicalIndexLookUpReader) GetPlanCost(taskType property.TaskType, cost return p.planCost, nil } -func (p *PhysicalIndexLookUpReader) estNumDoubleReadTasks(costFlag uint64) float64 { - doubleReadRows := p.indexPlan.StatsCount() +func (p *PhysicalIndexLookUpReader) estDoubleReadCost(tbl *model.TableInfo, costFlag uint64) float64 { + if p.ctx.GetSessionVars().CostModelVersion == modelVer1 { + // only consider double-read cost on modelVer2 + return 0 + } + // estimate the double-read cost: (numDoubleReadTasks * seekFactor) / concurrency + doubleReadRows := getCardinality(p.indexPlan, costFlag) batchSize := float64(p.ctx.GetSessionVars().IndexLookupSize) + concurrency := math.Max(1.0, float64(p.ctx.GetSessionVars().IndexLookupConcurrency())) + seekFactor := p.ctx.GetSessionVars().GetSeekFactor(tbl) // distRatio indicates how many requests corresponding to a batch, current value is from experiments. // TODO: estimate it by using index correlation or make it configurable. distRatio := 40.0 - numDoubleReadTasks := (doubleReadRows / batchSize) * distRatio - return numDoubleReadTasks // use Float64 instead of Int like `Ceil(...)` to make the cost continuous + numDoubleReadTasks := (doubleReadRows / batchSize) * distRatio // use Float64 instead of Int like `Ceil(...)` to make the cost continuous. + return (numDoubleReadTasks * seekFactor) / concurrency } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -453,7 +458,7 @@ func (p *PhysicalIndexScan) GetPlanCost(taskType property.TaskType, costFlag uin } // GetCost computes the cost of index join operator and its children. -func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt float64, outerCost, innerCost float64) float64 { +func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 sessVars := p.ctx.GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join @@ -490,6 +495,9 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt float64, outerCost, inner numPairs = 0 } } + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { + numPairs = getOperatorActRows(p) + } probeCost := numPairs * sessVars.GetCPUFactor() // Cost of additional concurrent goroutines. cpuCost += probeCost + (innerConcurrency+1.0)*sessVars.GetConcurrencyFactor() @@ -498,7 +506,23 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt float64, outerCost, inner memoryCost := innerConcurrency * (batchSize * distinctFactor) * innerCnt * sessVars.GetMemoryFactor() // Cost of inner child plan, i.e, mainly I/O and network cost. innerPlanCost := outerCnt * innerCost - return outerCost + innerPlanCost + cpuCost + memoryCost + return outerCost + innerPlanCost + cpuCost + memoryCost + p.estDoubleReadCost(outerCnt) +} + +func (p *PhysicalIndexJoin) estDoubleReadCost(doubleReadRows float64) float64 { + if p.ctx.GetSessionVars().CostModelVersion == modelVer1 { + // only consider double-read cost on modelVer2 + return 0 + } + // estimate the double read cost for IndexJoin: (double-read-tasks * seek-factor) / concurrency + seekFactor := p.ctx.GetSessionVars().GetSeekFactor(nil) + batchSize := math.Max(1.0, float64(p.ctx.GetSessionVars().IndexJoinBatchSize)) + concurrency := math.Max(1.0, float64(p.ctx.GetSessionVars().IndexLookupJoinConcurrency())) + // distRatio indicates how many requests corresponding to a batch, current value is from experiments. + // TODO: estimate it by using index correlation or make it configurable. + distRatio := 40.0 + numDoubleReadTasks := (doubleReadRows / batchSize) * distRatio + return (numDoubleReadTasks * seekFactor) / concurrency } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -517,13 +541,17 @@ func (p *PhysicalIndexJoin) GetPlanCost(taskType property.TaskType, costFlag uin } outerCnt := getCardinality(outerChild, costFlag) innerCnt := getCardinality(innerChild, costFlag) - p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost) + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) && outerCnt > 0 { + innerCnt /= outerCnt // corresponding to one outer row when calculating IndexJoin costs + innerCost /= outerCnt + } + p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost, costFlag) p.planCostInit = true return p.planCost, nil } // GetCost computes the cost of index merge join operator and its children. -func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64) float64 { +func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 sessVars := p.ctx.GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join @@ -561,6 +589,9 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost numPairs = 0 } } + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { + numPairs = getOperatorActRows(p) + } // Inner workers do hash join in parallel, but they can only save ONE outer // batch results. So as the number of outer batch exceeds inner concurrency, // it would fall back to linear execution. In a word, the hash join only runs @@ -579,7 +610,7 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost memoryCost := concurrency * (batchSize * distinctFactor) * innerCnt * sessVars.GetMemoryFactor() // Cost of inner child plan, i.e, mainly I/O and network cost. innerPlanCost := outerCnt * innerCost - return outerCost + innerPlanCost + cpuCost + memoryCost + return outerCost + innerPlanCost + cpuCost + memoryCost + p.estDoubleReadCost(outerCnt) } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -598,13 +629,17 @@ func (p *PhysicalIndexHashJoin) GetPlanCost(taskType property.TaskType, costFlag } outerCnt := getCardinality(outerChild, costFlag) innerCnt := getCardinality(innerChild, costFlag) - p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost) + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) && outerCnt > 0 { + innerCnt /= outerCnt // corresponding to one outer row when calculating IndexJoin costs + innerCost /= outerCnt + } + p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost, costFlag) p.planCostInit = true return p.planCost, nil } // GetCost computes the cost of index merge join operator and its children. -func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64) float64 { +func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 sessVars := p.ctx.GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join @@ -644,6 +679,9 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos numPairs = 0 } } + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { + numPairs = getOperatorActRows(p) + } avgProbeCnt := numPairs / outerCnt var probeCost float64 // Inner workers do merge join in parallel, but they can only save ONE outer batch @@ -662,7 +700,7 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos memoryCost := innerConcurrency * (batchSize * avgProbeCnt) * sessVars.GetMemoryFactor() innerPlanCost := outerCnt * innerCost - return outerCost + innerPlanCost + cpuCost + memoryCost + return outerCost + innerPlanCost + cpuCost + memoryCost + p.estDoubleReadCost(outerCnt) } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -681,7 +719,11 @@ func (p *PhysicalIndexMergeJoin) GetPlanCost(taskType property.TaskType, costFla } outerCnt := getCardinality(outerChild, costFlag) innerCnt := getCardinality(innerChild, costFlag) - p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost) + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) && outerCnt > 0 { + innerCnt /= outerCnt // corresponding to one outer row when calculating IndexJoin costs + innerCost /= outerCnt + } + p.planCost = p.GetCost(outerCnt, innerCnt, outerCost, innerCost, costFlag) p.planCostInit = true return p.planCost, nil } @@ -1166,9 +1208,9 @@ 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 + // TODO: this formula is wrong since it doesn't consider tableRowSize, fix it later + p.planCost += getCardinality(p.children[0], costFlag) * 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 5324eaf4de769..3d55d3f6c14d2 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/task.go b/planner/core/task.go index 4ecc67fc67a34..e5615e1c46f9b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -266,7 +266,7 @@ func (p *PhysicalIndexMergeJoin) attach2Task(tasks ...task) task { } t := &rootTask{ p: p, - cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost()), + cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost(), 0), } p.cost = t.cost() return t @@ -282,7 +282,7 @@ func (p *PhysicalIndexHashJoin) attach2Task(tasks ...task) task { } t := &rootTask{ p: p, - cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost()), + cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost(), 0), } p.cost = t.cost() return t @@ -298,7 +298,7 @@ func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task { } t := &rootTask{ p: p, - cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost()), + cst: p.GetCost(outerTask.count(), innerTask.count(), outerTask.cost(), innerTask.cost(), 0), } p.cost = t.cost() return t @@ -2070,8 +2070,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 90b4a7e27101b..13ed052391f59 100644 --- a/server/server.go +++ b/server/server.go @@ -352,7 +352,7 @@ func setTxnScope() { // Export config-related metrics func (s *Server) reportConfig() { metrics.ConfigStatus.WithLabelValues("token-limit").Set(float64(s.cfg.TokenLimit)) - metrics.ConfigStatus.WithLabelValues("max-server-connections").Set(float64(s.cfg.MaxServerConnections)) + metrics.ConfigStatus.WithLabelValues("max_connections").Set(float64(s.cfg.Instance.MaxConnections)) } // Run runs the server. @@ -514,11 +514,18 @@ func (s *Server) onConn(conn *clientConn) { }) terror.Log(err) } - if errors.Cause(err) == io.EOF { + switch errors.Cause(err) { + case io.EOF: // `EOF` means the connection is closed normally, we do not treat it as a noticeable error and log it in 'DEBUG' level. logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). Debug("EOF", zap.String("remote addr", conn.bufReadConn.RemoteAddr().String())) - } else { + case errConCount: + if err := conn.writeError(ctx, err); err != nil { + logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). + Warn("error in writing errConCount", zap.Error(err), + zap.String("remote addr", conn.bufReadConn.RemoteAddr().String())) + } + default: metrics.HandShakeErrorCounter.Inc() logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). Warn("Server.onConn handshake", zap.Error(err), @@ -605,8 +612,8 @@ func (cc *clientConn) connectInfo() *variable.ConnectionInfo { } func (s *Server) checkConnectionCount() error { - // When the value of MaxServerConnections is 0, the number of connections is unlimited. - if int(s.cfg.MaxServerConnections) == 0 { + // When the value of Instance.MaxConnections is 0, the number of connections is unlimited. + if int(s.cfg.Instance.MaxConnections) == 0 { return nil } @@ -614,9 +621,9 @@ func (s *Server) checkConnectionCount() error { conns := len(s.clients) s.rwlock.RUnlock() - if conns >= int(s.cfg.MaxServerConnections) { + if conns >= int(s.cfg.Instance.MaxConnections) { logutil.BgLogger().Error("too many connections", - zap.Uint32("max connections", s.cfg.MaxServerConnections), zap.Error(errConCount)) + zap.Uint32("max connections", s.cfg.Instance.MaxConnections), zap.Error(errConCount)) return errConCount } return nil diff --git a/server/util_test.go b/server/util_test.go index 889c2512eacad..7a5040b43cd3c 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -70,7 +70,7 @@ func TestDumpBinaryTime(t *testing.T) { d = dumpBinaryDateTime(nil, parsedTime) require.Equal(t, []byte{0}, d) - myDuration, err := types.ParseDuration(sc, "0000-00-00 00:00:00.000000", 6) + myDuration, _, err := types.ParseDuration(sc, "0000-00-00 00:00:00.000000", 6) require.NoError(t, err) d = dumpBinaryTime(myDuration.Duration) require.Equal(t, []byte{0}, d) @@ -194,7 +194,7 @@ func TestDumpTextValue(t *testing.T) { require.NoError(t, err) require.Equal(t, "2017-01-06 00:00:00", mustDecodeStr(t, bs)) - duration, err := types.ParseDuration(sc, "11:30:45", 0) + duration, _, err := types.ParseDuration(sc, "11:30:45", 0) require.NoError(t, err) d.SetMysqlDuration(duration) columns[0].Type = mysql.TypeDuration diff --git a/session/BUILD.bazel b/session/BUILD.bazel index e6cfe0f898682..a48f8238cc5ca 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -75,6 +75,7 @@ go_library( "//util/memory", "//util/parser", "//util/rowcodec", + "//util/sem", "//util/sli", "//util/sqlexec", "//util/tableutil", diff --git a/session/session_test/BUILD.bazel b/session/session_test/BUILD.bazel index 47c330e409de7..57900ac9d97cc 100644 --- a/session/session_test/BUILD.bazel +++ b/session/session_test/BUILD.bazel @@ -12,7 +12,9 @@ go_test( "//config", "//domain", "//kv", + "//parser/auth", "//parser/terror", + "//planner/core", "//session", "//store/mockstore", "//testkit", diff --git a/session/txninfo/BUILD.bazel b/session/txninfo/BUILD.bazel index 582036956862c..1388aacaa6814 100644 --- a/session/txninfo/BUILD.bazel +++ b/session/txninfo/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "txninfo", - srcs = ["txn_info.go"], + srcs = [ + "summary.go", + "txn_info.go", + ], importpath = "github.com/pingcap/tidb/session/txninfo", visibility = ["//visibility:public"], deps = [ diff --git a/sessionctx/sessionstates/BUILD.bazel b/sessionctx/sessionstates/BUILD.bazel index cb421dfeac1ec..ba5cb9254f9f8 100644 --- a/sessionctx/sessionstates/BUILD.bazel +++ b/sessionctx/sessionstates/BUILD.bazel @@ -16,7 +16,10 @@ go_test( srcs = ["session_states_test.go"], deps = [ "//errno", + "//parser/mysql", + "//sessionctx/variable", "//testkit", + "//util/sem", "@com_github_stretchr_testify//require", ], ) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 4f2cdac1aa690..6eb70beabfc99 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -24,7 +24,6 @@ import ( // but changing them has no effect on behavior. var noopSysVars = []*SysVar{ - {Scope: ScopeGlobal, Name: MaxConnections, Value: "151", Type: TypeUnsigned, MinValue: 1, MaxValue: 100000}, // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index aa91b574dc517..2e7865ac57040 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -431,6 +431,12 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeInstance, Name: PluginDir, Value: "/data/deploy/plugin", ReadOnly: true, GetGlobal: func(s *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.PluginDir, nil }}, + {Scope: ScopeInstance, Name: MaxConnections, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), Type: TypeUnsigned, MinValue: 0, MaxValue: 100000, SetGlobal: func(s *SessionVars, val string) error { + config.GetGlobalConfig().Instance.MaxConnections = uint32(TidbOptInt64(val, 0)) + return nil + }, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), nil + }}, /* The system variables below have GLOBAL scope */ {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576}, diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index ab878d2bb3054..38108e5cba932 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -430,7 +430,7 @@ func parseTimeZone(s string) (*time.Location, error) { // The value can be given as a string indicating an offset from UTC, such as '+10:00' or '-6:00'. // The time zone's value should in [-12:59,+14:00]. if strings.HasPrefix(s, "+") || strings.HasPrefix(s, "-") { - d, err := types.ParseDuration(nil, s[1:], 0) + d, _, err := types.ParseDuration(nil, s[1:], 0) if err == nil { if s[0] == '-' { if d.Duration > 12*time.Hour+59*time.Minute { diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 76ff4ea60e52a..d33984649b371 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -35,13 +35,13 @@ 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" // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 8ac6f0aaa7322..d3e1f32124575 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -52,8 +52,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { 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")) @@ -85,8 +83,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { 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")) @@ -729,102 +725,65 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { "update t1 set v=v+1 where id in (1, 2, 3) and v>0", } + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") 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) + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + tk2 := testkit.NewSteppedTestKit(t, store) + defer tk2.MustExec("rollback") + + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) + if autocommit { + tk2.MustExec("set autocommit=1") + tk2.MustExec("begin") + } else { + tk2.MustExec("set autocommit=0") + } + + tk2.SetBreakPoints( + sessiontxn.BreakPointBeforeExecutorFirstRun, + sessiontxn.BreakPointOnStmtRetryAfterLockError, + ) + + var isSelect, isUpdate bool + 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) + } + + // 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") + + // 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") + + // 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")) + } }) } } } } - -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") - } - - 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") - - // 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) - } - - wait(c3, "done") - - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) - if isSelect { - tk.MustExec("update t1 set v=v+1") - } - tk.MustExec("commit") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - tk.MustExec("rollback") -} diff --git a/statistics/scalar_test.go b/statistics/scalar_test.go index 32eca78c5c80b..34a911579137f 100644 --- a/statistics/scalar_test.go +++ b/statistics/scalar_test.go @@ -35,7 +35,7 @@ func getDecimal(value float64) *types.MyDecimal { } func getDuration(value string) types.Duration { - dur, _ := types.ParseDuration(nil, value, 0) + dur, _, _ := types.ParseDuration(nil, value, 0) return dur } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index bef71c20aeb51..0b217583bb9b3 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -53,7 +53,7 @@ import ( "go.uber.org/zap" ) -var coprCacheHistogramEvict = tidbmetrics.DistSQLCoprCacheHistogram.WithLabelValues("evict") +var coprCacheCounterEvict = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("evict") // Maximum total sleep time(in ms) for kv/cop commands. const ( @@ -687,7 +687,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } } if worker.store.coprCache != nil && worker.store.coprCache.cache.Metrics != nil { - coprCacheHistogramEvict.Observe(float64(worker.store.coprCache.cache.Metrics.KeysEvicted())) + coprCacheCounterEvict.Add(float64(worker.store.coprCache.cache.Metrics.KeysEvicted())) } } @@ -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) diff --git a/table/temptable/ddl_test.go b/table/temptable/ddl_test.go index 1418a35ad83d1..e08343c1177a6 100644 --- a/table/temptable/ddl_test.go +++ b/table/temptable/ddl_test.go @@ -164,7 +164,7 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { // truncate when empty err := ddl.TruncateLocalTemporaryTable(model.NewCIStr("db1"), model.NewCIStr("t1")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) - require.Nil(t, sessVars.LocalTemporaryTables) + require.Equal(t, 0, sessVars.LocalTemporaryTables.(*infoschema.LocalTemporaryTables).Count()) require.Nil(t, sessVars.TemporaryTableData) // add one table diff --git a/table/temptable/infoschema.go b/table/temptable/infoschema.go index 7947027ba0314..764bee0dbbce0 100644 --- a/table/temptable/infoschema.go +++ b/table/temptable/infoschema.go @@ -26,10 +26,6 @@ func AttachLocalTemporaryTableInfoSchema(sctx sessionctx.Context, is infoschema. } localTemporaryTables := getLocalTemporaryTables(sctx) - if localTemporaryTables == nil { - return is - } - return &infoschema.TemporaryTableAttachedInfoSchema{ InfoSchema: is, LocalTemporaryTables: localTemporaryTables, @@ -46,12 +42,8 @@ func DetachLocalTemporaryTableInfoSchema(is infoschema.InfoSchema) infoschema.In } func getLocalTemporaryTables(sctx sessionctx.Context) *infoschema.LocalTemporaryTables { - localTemporaryTables := sctx.GetSessionVars().LocalTemporaryTables - if localTemporaryTables == nil { - return nil - } - - return localTemporaryTables.(*infoschema.LocalTemporaryTables) + // Do not return nil so that new created tables can always be visited through the returned object. + return ensureLocalTemporaryTables(sctx) } func ensureLocalTemporaryTables(sctx sessionctx.Context) *infoschema.LocalTemporaryTables { diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 008ee8c5d7e1f..3f813c8747a7a 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -262,7 +262,7 @@ func TestTimeCodec(t *testing.T) { "2016-06-23 11:30:45") require.NoError(t, err) row[2] = types.NewDatum(ts) - du, err := types.ParseDuration(nil, "12:59:59.999999", 6) + du, _, err := types.ParseDuration(nil, "12:59:59.999999", 6) require.NoError(t, err) row[3] = types.NewDatum(du) 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..d63a2a1efc7b8 --- /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.T + 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.T + 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.T, 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/tests/realtikvtest/sessiontest/session_test.go b/tests/realtikvtest/sessiontest/session_test.go index f726c1118b39f..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()) } @@ -2951,6 +2952,17 @@ func TestCast(t *testing.T) { tk.MustQuery("select cast(0.5 as unsigned)") tk.MustQuery("select cast(-0.5 as signed)") tk.MustQuery("select hex(cast(0x10 as binary(2)))").Check(testkit.Rows("1000")) + + // test for issue: https://github.com/pingcap/tidb/issues/34539 + tk.MustQuery("select cast('0000-00-00' as TIME);").Check(testkit.Rows("00:00:00")) + tk.MustQuery("select cast('1234x' as TIME);").Check(testkit.Rows("00:12:34")) + tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect time value: '1234x'")) + tk.MustQuery("select cast('a' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('1234xxxxxxx' as TIME);").Check(testkit.Rows("00:12:34")) + tk.MustQuery("select cast('1234xxxxxxxx' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('-1234xxxxxxx' as TIME);").Check(testkit.Rows("-00:12:34")) + tk.MustQuery("select cast('-1234xxxxxxxx' as TIME);").Check(testkit.Rows("")) } func TestTableInfoMeta(t *testing.T) { diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index 795e2133e79d6..ad6526d69138e 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//privilege/privileges", "//server", "//session", + "//session/txninfo", "//sessionctx/binloginfo", "//sessionctx/variable", "//statistics", diff --git a/tidb-server/main.go b/tidb-server/main.go index 763d4bd5bc6a8..bc8795875f360 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -559,6 +559,8 @@ func setGlobalVars() { cfg.Instance.CheckMb4ValueInUTF8.Store(cfg.CheckMb4ValueInUTF8.Load()) case "enable-collect-execution-info": cfg.Instance.EnableCollectExecutionInfo = cfg.EnableCollectExecutionInfo + case "max-server-connections": + cfg.Instance.MaxConnections = cfg.MaxServerConnections } case "log": switch oldName { diff --git a/types/convert.go b/types/convert.go index 9433f26dbefc8..a6e33d5b2c51b 100644 --- a/types/convert.go +++ b/types/convert.go @@ -327,7 +327,7 @@ func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d Duratio } } - d, err = ParseDuration(sc, str, fsp) + d, _, err = ParseDuration(sc, str, fsp) if ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } diff --git a/types/convert_test.go b/types/convert_test.go index aed9980635b72..88a60171ed3ad 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -251,7 +251,7 @@ func TestConvertType(t *testing.T) { // Test Datum.ToDecimal with bad number. d := NewDatum("hello") _, err = d.ToDecimal(sc) - require.Truef(t, terror.ErrorEqual(err, ErrBadNumber), "err %v", err) + require.Truef(t, terror.ErrorEqual(err, ErrTruncatedWrongVal), "err %v", err) sc.IgnoreTruncate = true v, err = d.ToDecimal(sc) @@ -352,7 +352,7 @@ func TestConvertToString(t *testing.T) { require.NoError(t, err) testToString(t, t1, "2011-11-10 11:11:11.999999") - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testToString(t, td, "11:11:11.999999") @@ -819,7 +819,9 @@ func TestConvert(t *testing.T) { signedDeny(t, mysql.TypeDate, "2012-08-x", "0000-00-00") signedDeny(t, mysql.TypeDatetime, "2012-08-x", "0000-00-00 00:00:00") signedDeny(t, mysql.TypeTimestamp, "2012-08-x", "0000-00-00 00:00:00") - signedDeny(t, mysql.TypeDuration, "2012-08-x", "00:00:00") + signedDeny(t, mysql.TypeDuration, "2012-08-x", "00:20:12") + signedDeny(t, mysql.TypeDuration, "0000-00-00", "00:00:00") + signedDeny(t, mysql.TypeDuration, "1234abc", "00:12:34") // string from string signedAccept(t, mysql.TypeString, "abc", "abc") diff --git a/types/datum.go b/types/datum.go index a450b51a8a25c..1ad86c770a01b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -749,7 +749,7 @@ func (d *Datum) compareString(sc *stmtctx.StatementContext, s string, comparer c dt, err := ParseDatetime(sc, s) return d.GetMysqlTime().Compare(dt), errors.Trace(err) case KindMysqlDuration: - dur, err := ParseDuration(sc, s, MaxFsp) + dur, _, err := ParseDuration(sc, s, MaxFsp) return d.GetMysqlDuration().Compare(dur), errors.Trace(err) case KindMysqlSet: return comparer.Compare(d.GetMysqlSet().String(), s), nil @@ -796,7 +796,7 @@ func (d *Datum) compareMysqlDuration(sc *stmtctx.StatementContext, dur Duration) case KindMysqlDuration: return d.GetMysqlDuration().Compare(dur), nil case KindString, KindBytes: - dDur, err := ParseDuration(sc, d.GetString(), MaxFsp) + dDur, _, err := ParseDuration(sc, d.GetString(), MaxFsp) return dDur.Compare(dur), errors.Trace(err) default: return d.compareFloat64(sc, dur.Seconds()) @@ -1375,13 +1375,13 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie if timeNum < -MaxDuration { return ret, ErrWrongValue.GenWithStackByArgs(TimeStr, timeStr) } - t, err := ParseDuration(sc, timeStr, fsp) + t, _, err := ParseDuration(sc, timeStr, fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) } case KindString, KindBytes: - t, err := ParseDuration(sc, d.GetString(), fsp) + t, _, err := ParseDuration(sc, d.GetString(), fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) @@ -1392,7 +1392,7 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie if err != nil { return ret, errors.Trace(err) } - t, err := ParseDuration(sc, s, fsp) + t, _, err := ParseDuration(sc, s, fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) diff --git a/types/datum_test.go b/types/datum_test.go index 75627791c2342..698a69fd6b731 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -98,7 +98,7 @@ func TestToBool(t *testing.T) { require.NoError(t, err) testDatumToBool(t, t1, 1) - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testDatumToBool(t, td, 1) @@ -141,7 +141,7 @@ func TestToInt64(t *testing.T) { require.NoError(t, err) testDatumToInt64(t, t1, int64(20111110111112)) - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testDatumToInt64(t, td, int64(111112)) diff --git a/types/mydecimal.go b/types/mydecimal.go index c2229462dc8ae..0e2ca156a8bc4 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -403,7 +403,7 @@ func (d *MyDecimal) FromString(str []byte) error { } if len(str) == 0 { *d = zeroMyDecimal - return ErrBadNumber + return ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", str) } switch str[0] { case '-': @@ -431,7 +431,7 @@ func (d *MyDecimal) FromString(str []byte) error { } if digitsInt+digitsFrac == 0 { *d = zeroMyDecimal - return ErrBadNumber + return ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", str) } wordsInt := digitsToWords(digitsInt) wordsFrac := digitsToWords(digitsFrac) diff --git a/types/time.go b/types/time.go index d63669d7fe409..14c647f945788 100644 --- a/types/time.go +++ b/types/time.go @@ -1530,7 +1530,7 @@ func (d Duration) Compare(o Duration) int { // but parses str to Duration then compares. func (d Duration) CompareString(sc *stmtctx.StatementContext, str string) (int, error) { // use MaxFsp to parse the string - o, err := ParseDuration(sc, str, MaxFsp) + o, _, err := ParseDuration(sc, str, MaxFsp) if err != nil { return 0, err } @@ -1680,18 +1680,19 @@ func matchFrac(str string, fsp int) (bool, int, string, error) { return overflow, frac, rest, nil } -func matchDuration(str string, fsp int) (Duration, error) { +func matchDuration(str string, fsp int) (Duration, bool, error) { fsp, err := CheckFsp(fsp) if err != nil { - return ZeroDuration, errors.Trace(err) + return ZeroDuration, true, errors.Trace(err) } if len(str) == 0 { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } negative, rest := isNegativeDuration(str) rest = parser.Space0(rest) + charsLen := len(rest) hhmmss := [3]int{} @@ -1703,13 +1704,13 @@ func matchDuration(str string, fsp int) (Duration, error) { } else if hms, remain, err := matchHHMMSSCompact(rest); err == nil { rest, hhmmss = remain, hms } else { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } rest = parser.Space0(rest) overflow, frac, rest, err := matchFrac(rest, fsp) - if err != nil || len(rest) > 0 { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + if err != nil || (len(rest) > 0 && charsLen >= 12) { + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } if overflow { @@ -1718,7 +1719,7 @@ func matchDuration(str string, fsp int) (Duration, error) { } if !checkHHMMSS(hhmmss) { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } if hhmmss[0] > TimeMaxHour { @@ -1728,7 +1729,7 @@ func matchDuration(str string, fsp int) (Duration, error) { } else { t = MaxTime } - return Duration{t, fsp}, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return Duration{t, fsp}, false, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } d := gotime.Duration(hhmmss[0]*3600+hhmmss[1]*60+hhmmss[2])*gotime.Second + gotime.Duration(frac)*gotime.Microsecond //nolint:durationcheck @@ -1736,7 +1737,10 @@ func matchDuration(str string, fsp int) (Duration, error) { d = -d } d, err = TruncateOverflowMySQLTime(d) - return Duration{d, fsp}, errors.Trace(err) + if err == nil && len(rest) > 0 { + return Duration{d, fsp}, false, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + } + return Duration{d, fsp}, false, errors.Trace(err) } // canFallbackToDateTime return true @@ -1776,29 +1780,30 @@ func canFallbackToDateTime(str string) bool { } // ParseDuration parses the time form a formatted string with a fractional seconds part, -// returns the duration type Time value. +// returns the duration type Time value and bool to indicate whether the result is null. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html -func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, error) { +func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, bool, error) { rest := strings.TrimSpace(str) - d, err := matchDuration(rest, fsp) + d, isNull, err := matchDuration(rest, fsp) if err == nil { - return d, nil + return d, isNull, nil } if !canFallbackToDateTime(rest) { - return d, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return d, isNull, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } datetime, err := ParseDatetime(sc, rest) if err != nil { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } d, err = datetime.ConvertToDuration() if err != nil { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } - return d.RoundFrac(fsp, sc.TimeZone) + d, err = d.RoundFrac(fsp, sc.TimeZone) + return d, false, err } // TruncateOverflowMySQLTime truncates d when it overflows, and returns ErrTruncatedWrongVal. diff --git a/types/time_test.go b/types/time_test.go index a1dfa57ea5402..13e9191ec4175 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -334,8 +334,9 @@ func TestTime(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, types.MinFsp) + duration, isNull, err := types.ParseDuration(sc, test.Input, types.MinFsp) require.NoError(t, err) + require.False(t, isNull) require.Equal(t, test.Expect, duration.String()) } @@ -349,11 +350,27 @@ func TestTime(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, types.MaxFsp) + duration, _, err := types.ParseDuration(sc, test.Input, types.MaxFsp) require.NoError(t, err) require.Equal(t, test.Expect, duration.String()) } + table = []struct { + Input string + Expect string + }{ + {"0x", "00:00:00.000000"}, + {"1x", "00:00:01.000000"}, + {"0000-00-00", "00:00:00.000000"}, + } + + for _, test := range table { + duration, isNull, err := types.ParseDuration(sc, test.Input, types.MaxFsp) + require.False(t, isNull) + require.True(t, types.ErrTruncatedWrongVal.Equal(err)) + require.Equal(t, test.Expect, duration.String()) + } + errTable := []string{ "2011-11-11", "232 10", @@ -361,11 +378,11 @@ func TestTime(t *testing.T) { } for _, test := range errTable { - _, err := types.ParseDuration(sc, test, types.DefaultFsp) + _, _, err := types.ParseDuration(sc, test, types.DefaultFsp) require.Error(t, err) } - duration, err := types.ParseDuration(sc, "4294967295 0:59:59", types.DefaultFsp) + duration, _, err := types.ParseDuration(sc, "4294967295 0:59:59", types.DefaultFsp) require.Error(t, err) require.Equal(t, "838:59:59", duration.String()) @@ -408,15 +425,15 @@ func TestDurationAdd(t *testing.T) { {"00:00:00.099", 3, "00:00:00.001", 3, "00:00:00.100"}, } for _, test := range table { - duration, err := types.ParseDuration(nil, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(nil, test.Input, test.Fsp) require.NoError(t, err) - ta, err := types.ParseDuration(nil, test.InputAdd, test.FspAdd) + ta, _, err := types.ParseDuration(nil, test.InputAdd, test.FspAdd) require.NoError(t, err) result, err := duration.Add(ta) require.NoError(t, err) require.Equal(t, test.Expect, result.String()) } - duration, err := types.ParseDuration(nil, "00:00:00", 0) + duration, _, err := types.ParseDuration(nil, "00:00:00", 0) require.NoError(t, err) ta := new(types.Duration) result, err := duration.Add(*ta) @@ -424,7 +441,7 @@ func TestDurationAdd(t *testing.T) { require.Equal(t, "00:00:00", result.String()) duration = types.Duration{Duration: math.MaxInt64, Fsp: 0} - tatmp, err := types.ParseDuration(nil, "00:01:00", 0) + tatmp, _, err := types.ParseDuration(nil, "00:01:00", 0) require.NoError(t, err) _, err = duration.Add(tatmp) require.Error(t, err) @@ -444,9 +461,9 @@ func TestDurationSub(t *testing.T) { {"00:00:00", 0, "00:00:00.1", 1, "-00:00:00.1"}, } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) - ta, err := types.ParseDuration(sc, test.InputAdd, test.FspAdd) + ta, _, err := types.ParseDuration(sc, test.InputAdd, test.FspAdd) require.NoError(t, err) result, err := duration.Sub(ta) require.NoError(t, err) @@ -475,7 +492,7 @@ func TestTimeFsp(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) require.Equal(t, test.Expect, duration.String()) } @@ -488,7 +505,7 @@ func TestTimeFsp(t *testing.T) { } for _, test := range errTable { - _, err := types.ParseDuration(sc, test.Input, test.Fsp) + _, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.Error(t, err) } } @@ -753,7 +770,7 @@ func TestToNumber(t *testing.T) { } for _, test := range tblDuration { - v, err := types.ParseDuration(sc, test.Input, test.Fsp) + v, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) // now we can only changetypes.Duration's Fsp to check ToNumber with different Fsp require.Equal(t, test.Expect, v.ToNumber().String()) @@ -897,7 +914,7 @@ func TestRoundFrac(t *testing.T) { } for _, tt := range tbl { - v, err := types.ParseDuration(sc, tt.Input, types.MaxFsp) + v, _, err := types.ParseDuration(sc, tt.Input, types.MaxFsp) require.NoError(t, err) nv, err := v.RoundFrac(tt.Fsp, sc.TimeZone) require.NoError(t, err) @@ -959,7 +976,7 @@ func TestConvert(t *testing.T) { // test different time zone. sc.TimeZone = time.UTC for _, tt := range tblDuration { - v, err := types.ParseDuration(sc, tt.Input, tt.Fsp) + v, _, err := types.ParseDuration(sc, tt.Input, tt.Fsp) require.NoError(t, err) year, month, day := time.Now().In(sc.TimeZone).Date() n := time.Date(year, month, day, 0, 0, 0, 0, sc.TimeZone) @@ -1010,7 +1027,7 @@ func TestCompare(t *testing.T) { } for _, tt := range tbl { - v1, err := types.ParseDuration(nil, tt.Arg1, types.MaxFsp) + v1, _, err := types.ParseDuration(nil, tt.Arg1, types.MaxFsp) require.NoError(t, err) ret, err := v1.CompareString(nil, tt.Arg2) @@ -1034,7 +1051,7 @@ func TestDurationClock(t *testing.T) { } for _, tt := range tbl { - d, err := types.ParseDuration(&stmtctx.StatementContext{TimeZone: time.UTC}, tt.Input, types.MaxFsp) + d, _, err := types.ParseDuration(&stmtctx.StatementContext{TimeZone: time.UTC}, tt.Input, types.MaxFsp) require.NoError(t, err) require.Equal(t, tt.Hour, d.Hour()) require.Equal(t, tt.Minute, d.Minute()) @@ -1151,7 +1168,7 @@ func TestTimeAdd(t *testing.T) { for _, tt := range tbl { v1, err := types.ParseTime(sc, tt.Arg1, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) - dur, err := types.ParseDuration(sc, tt.Arg2, types.MaxFsp) + dur, _, err := types.ParseDuration(sc, tt.Arg2, types.MaxFsp) require.NoError(t, err) result, err := types.ParseTime(sc, tt.Ret, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) @@ -1976,7 +1993,7 @@ func TestTimeSub(t *testing.T) { require.NoError(t, err) v2, err := types.ParseTime(sc, tt.Arg2, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) - dur, err := types.ParseDuration(sc, tt.Ret, types.MaxFsp) + dur, _, err := types.ParseDuration(sc, tt.Ret, types.MaxFsp) require.NoError(t, err) rec := v1.Sub(sc, &v2) require.Equal(t, dur, rec) @@ -2209,7 +2226,7 @@ func BenchmarkTimeAdd(b *testing.B) { TimeZone: time.UTC, } arg1, _ := types.ParseTime(sc, "2017-01-18", mysql.TypeDatetime, types.MaxFsp) - arg2, _ := types.ParseDuration(sc, "12:30:59", types.MaxFsp) + arg2, _, _ := types.ParseDuration(sc, "12:30:59", types.MaxFsp) for i := 0; i < b.N; i++ { _, err := arg1.Add(sc, arg2) if err != 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/chunk/mutrow_test.go b/util/chunk/mutrow_test.go index 1293240013a3d..532206cf42782 100644 --- a/util/chunk/mutrow_test.go +++ b/util/chunk/mutrow_test.go @@ -80,7 +80,7 @@ func TestMutRow(t *testing.T) { retTypes := []*types.FieldType{types.NewFieldType(mysql.TypeDuration)} chk := New(retTypes, 1, 1) - dur, err := types.ParseDuration(sc, "01:23:45", 0) + dur, _, err := types.ParseDuration(sc, "01:23:45", 0) require.NoError(t, err) chk.AppendDuration(0, dur) mutRow = MutRowFromTypes(retTypes) diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index 33af7c59a9840..a1a5461dcd5fa 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -527,7 +527,7 @@ func parseTime(t *testing.T, s string) types.Time { } func parseDuration(t *testing.T, s string) types.Duration { - m, err := types.ParseDuration(nil, s, types.DefaultFsp) + m, _, err := types.ParseDuration(nil, s, types.DefaultFsp) require.NoError(t, err) return m } diff --git a/util/dbutil/common.go b/util/dbutil/common.go index ec0ffd785349e..674314af3492f 100644 --- a/util/dbutil/common.go +++ b/util/dbutil/common.go @@ -319,7 +319,7 @@ func GetTimeZoneOffset(ctx context.Context, db QueryExecutor) (time.Duration, er } hour, minute, second := t.Clock() - // nolint:durationcheck + //nolint:durationcheck return time.Duration(hour*3600+minute*60+second) * time.Second * factor, nil } diff --git a/util/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index 51c965f095cd6..8772a50eb3d5d 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -890,7 +890,7 @@ var ( } } getDuration = func(value string) types.Duration { - dur, _ := types.ParseDuration(nil, value, 0) + dur, _, _ := types.ParseDuration(nil, value, 0) return dur } getOldDatumByte = func(d types.Datum) []byte { diff --git a/util/set/BUILD.bazel b/util/set/BUILD.bazel index 790799294974f..0f733ad14ee41 100644 --- a/util/set/BUILD.bazel +++ b/util/set/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "float64_set.go", "int_set.go", + "mem_aware_map.go", "set_with_memory_usage.go", "string_set.go", ], @@ -19,6 +20,7 @@ go_test( "float64_set_test.go", "int_set_test.go", "main_test.go", + "mem_aware_map_test.go", "set_with_memory_usage_test.go", "string_set_test.go", ],