diff --git a/.github/workflows/integration-test-br-compatibility.yml b/.github/workflows/integration-test-br-compatibility.yml deleted file mode 100644 index 5df7168467005..0000000000000 --- a/.github/workflows/integration-test-br-compatibility.yml +++ /dev/null @@ -1,62 +0,0 @@ -name: BR / Compatibility Test - -on: - push: - # merged git action - branches: - - master - - "release-[0-9].[0-9]*" - paths: - - "br/**" - - "!**.html" - - "!**.md" - - "!CNAME" - - "!LICENSE" - - "!br/docs/**" - - "!br/tests/**" - - "!br/docker/**" - # disable pull request only keep the merge action since it is very costly to run those tests - # pull_request: - -concurrency: - group: ${{ github.ref }}-${{ github.workflow }} - cancel-in-progress: true - -permissions: - contents: read # to fetch code (actions/checkout) - -jobs: - check: - runs-on: ubuntu-latest - timeout-minutes: 25 - steps: - - uses: actions/checkout@v2 - - - name: Set up Go - uses: actions/setup-go@v3 - with: - go-version-file: 'go.mod' - - - name: Generate compatibility test backup data - timeout-minutes: 15 - run: sh br/compatibility/prepare_backup.sh - - - name: Start server - run: | - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml rm -s -v - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml build - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml up --remove-orphans -d - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml exec -T control go mod tidy - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml exec -T control make build_br - TAG=nightly PORT_SUFFIX=1 docker-compose -f br/compatibility/backup_cluster.yaml exec -T control br/tests/run_compatible.sh run - - - name: Collect component log - if: ${{ failure() }} - run: | - tar czvf ${{ github.workspace }}/logs.tar.gz /tmp/br/docker/backup_logs/* - - - uses: actions/upload-artifact@v2 - if: ${{ failure() }} - with: - name: logs - path: ${{ github.workspace }}/logs.tar.gz diff --git a/DEPS.bzl b/DEPS.bzl index 51e247cbb8961..43f76556ecdd9 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2936,8 +2936,8 @@ def go_deps(): name = "com_github_pingcap_tipb", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/tipb", - sum = "h1:Yoo8j5xQGxjlsC3yt0ndsiAz0WZXED9rzsKmEN0U0DY=", - version = "v0.0.0-20221020071514-cd933387bcb5", + sum = "h1:DbmCfCbcavo0JG+gSp0ySvv1ub/c/j3hsnYzyYPzONo=", + version = "v0.0.0-20221123081521-2fb828910813", ) go_repository( name = "com_github_pkg_browser", @@ -3519,8 +3519,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:5FFJAKukKDTsLqrEeeDgC89aDAteGEFXBHwKRa3wnnQ=", - version = "v2.0.3-0.20221125022819-f05c6886bbad", + sum = "h1:nfdLYzkmYR2b0XurXSXcVFk06eO//P4VtkqoADX2tR4=", + version = "v2.0.3-0.20221128025602-81939ec8b2bb", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/Makefile b/Makefile index 66b3ba0686917..2f2dba4b010f0 100644 --- a/Makefile +++ b/Makefile @@ -415,7 +415,7 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare bazel_build: bazel_ci_prepare mkdir -p bin - bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \ + bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) --remote_download_minimal \ //... --//build:with_nogo_flag=true bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \ //cmd/importer:importer //tidb-server:tidb-server //tidb-server:tidb-server-check --//build:with_nogo_flag=true @@ -442,27 +442,27 @@ bazel_golangcilinter: -- run $$($(PACKAGE_DIRECTORIES)) --config ./.golangci.yaml bazel_brietest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/brietest/... bazel_pessimistictest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/pessimistictest/... bazel_sessiontest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/sessiontest/... bazel_statisticstest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/statisticstest/... bazel_txntest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/txntest/... bazel_addindextest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ -- //tests/realtikvtest/addindextest/... bazel_lint: bazel_prepare diff --git a/WORKSPACE b/WORKSPACE index 51d161ebf9918..6a3af98f950c6 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -18,7 +18,7 @@ http_archive( ], ) -load("@io_bazel_rules_go//go:deps.bzl", "go_register_toolchains", "go_rules_dependencies") +load("@io_bazel_rules_go//go:deps.bzl", "go_download_sdk", "go_register_toolchains", "go_rules_dependencies") load("@bazel_gazelle//:deps.bzl", "gazelle_dependencies") load("//:DEPS.bzl", "go_deps") @@ -27,9 +27,19 @@ go_deps() go_rules_dependencies() +go_download_sdk( + name = "go_sdk", + urls = [ + "http://ats.apps.svc/golang/{}", + "http://bazel-cache.pingcap.net:8080/golang/{}", + "https://mirrors.aliyun.com/golang/{}", + "https://dl.google.com/go/{}", + ], + version = "1.19.3", +) + go_register_toolchains( nogo = "@//build:tidb_nogo", - version = "1.19.3", ) gazelle_dependencies() diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel index df3d361d412ed..c973afd85c4e7 100644 --- a/autoid_service/BUILD.bazel +++ b/autoid_service/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//meta", "//metrics", "//owner", + "//parser/model", "//util/logutil", "//util/mathutil", "@com_github_pingcap_errors//:errors", diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index f2836fc80fe85..fd0feae76f6bf 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -76,7 +77,7 @@ func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage, ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) var err1 error newBase, err1 = idAcc.Get() if err1 != nil { @@ -137,7 +138,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context, ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) var err1 error newBase, err1 = idAcc.Get() if err1 != nil { @@ -188,7 +189,7 @@ func (alloc *autoIDValue) rebase4Unsigned(ctx context.Context, startTime := time.Now() ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 @@ -221,7 +222,7 @@ func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, d var newBase, newEnd int64 ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 @@ -451,7 +452,7 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (* func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbID, tblID, requiredBase int64, isUnsigned bool) error { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index e6baebe3ea960..a2e03931798a8 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -33,13 +33,12 @@ import ( // SessionHandle is used to handle all session sql bind operations. type SessionHandle struct { - ch *bindCache - parser *parser.Parser + ch *bindCache } // NewSessionBindHandle creates a new SessionBindHandle. -func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { - sessionHandle := &SessionHandle{parser: parser} +func NewSessionBindHandle() *SessionHandle { + sessionHandle := &SessionHandle{} sessionHandle.ch = newBindCache() return sessionHandle } diff --git a/br/cmd/br/restore.go b/br/cmd/br/restore.go index 5f91bee91c6a9..e826df0e59e77 100644 --- a/br/cmd/br/restore.go +++ b/br/cmd/br/restore.go @@ -199,6 +199,5 @@ func newStreamRestoreCommand() *cobra.Command { } task.DefineFilterFlags(command, filterOutSysAndMemTables, true) task.DefineStreamRestoreFlags(command) - command.Hidden = true return command } diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 7614ca78e52c7..865e7fa2f3078 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -290,10 +290,12 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { ranges = ranger.FullIntRange(false) } + retRanges := make([]kv.KeyRange, 0, 1+len(tbl.Indices)) kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil) if err != nil { return nil, errors.Trace(err) } + retRanges = kvRanges.AppendSelfTo(retRanges) for _, index := range tbl.Indices { if index.State != model.StatePublic { @@ -304,9 +306,9 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { if err != nil { return nil, errors.Trace(err) } - kvRanges = append(kvRanges, idxRanges...) + retRanges = idxRanges.AppendSelfTo(retRanges) } - return kvRanges, nil + return retRanges, nil } // BuildBackupRangeAndSchema gets KV range and schema of tables. diff --git a/br/pkg/checksum/executor_test.go b/br/pkg/checksum/executor_test.go index adcaed9c314f9..876103bc055a2 100644 --- a/br/pkg/checksum/executor_test.go +++ b/br/pkg/checksum/executor_test.go @@ -104,7 +104,7 @@ func TestChecksum(t *testing.T) { first = false ranges, err := backup.BuildTableRanges(tableInfo3) require.NoError(t, err) - require.Equalf(t, ranges[:1], req.KeyRanges, "%v", req.KeyRanges) + require.Equalf(t, ranges[:1], req.KeyRanges.FirstPartitionRange(), "%v", req.KeyRanges.FirstPartitionRange()) } return nil })) diff --git a/br/pkg/errors/errors.go b/br/pkg/errors/errors.go index 07e9fb6317cb9..2b7d76e28d795 100644 --- a/br/pkg/errors/errors.go +++ b/br/pkg/errors/errors.go @@ -83,8 +83,9 @@ var ( ErrStorageInvalidPermission = errors.Normalize("external storage permission", errors.RFCCodeText("BR:ExternalStorage:ErrStorageInvalidPermission")) // Snapshot restore - ErrRestoreTotalKVMismatch = errors.Normalize("restore total tikvs mismatch", errors.RFCCodeText("BR:EBS:ErrRestoreTotalKVMismatch")) - ErrRestoreInvalidPeer = errors.Normalize("restore met a invalid peer", errors.RFCCodeText("BR:EBS:ErrRestoreInvalidPeer")) + ErrRestoreTotalKVMismatch = errors.Normalize("restore total tikvs mismatch", errors.RFCCodeText("BR:EBS:ErrRestoreTotalKVMismatch")) + ErrRestoreInvalidPeer = errors.Normalize("restore met a invalid peer", errors.RFCCodeText("BR:EBS:ErrRestoreInvalidPeer")) + ErrRestoreRegionWithoutPeer = errors.Normalize("restore met a region without any peer", errors.RFCCodeText("BR:EBS:ErrRestoreRegionWithoutPeer")) // Errors reported from TiKV. ErrKVStorage = errors.Normalize("tikv storage occur I/O error", errors.RFCCodeText("BR:KV:ErrKVStorage")) diff --git a/br/pkg/lightning/backend/kv/allocator.go b/br/pkg/lightning/backend/kv/allocator.go index 02f46ea8c7e36..14703e1143a45 100644 --- a/br/pkg/lightning/backend/kv/allocator.go +++ b/br/pkg/lightning/backend/kv/allocator.go @@ -34,6 +34,7 @@ type panickingAllocator struct { func NewPanickingAllocators(base int64) autoid.Allocators { sharedBase := &base return autoid.NewAllocators( + false, &panickingAllocator{base: sharedBase, ty: autoid.RowIDAllocType}, &panickingAllocator{base: sharedBase, ty: autoid.AutoIncrementType}, &panickingAllocator{base: sharedBase, ty: autoid.AutoRandomType}, diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index 6e2b5e9a1c43c..c034e6bdb2b3c 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -94,7 +94,7 @@ go_test( ], embed = [":local"], flaky = True, - shard_count = 20, + shard_count = 40, deps = [ "//br/pkg/errors", "//br/pkg/lightning/backend", diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index b2858a8456f36..25bc7fabf514e 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -211,7 +211,7 @@ func physicalTableIDs(tableInfo *model.TableInfo) []int64 { } // tableHandleKeyRanges returns all key ranges associated with the tableInfo. -func tableHandleKeyRanges(tableInfo *model.TableInfo) ([]tidbkv.KeyRange, error) { +func tableHandleKeyRanges(tableInfo *model.TableInfo) (*tidbkv.KeyRanges, error) { ranges := ranger.FullIntRange(false) if tableInfo.IsCommonHandle { ranges = ranger.FullRange() @@ -221,18 +221,9 @@ func tableHandleKeyRanges(tableInfo *model.TableInfo) ([]tidbkv.KeyRange, error) } // tableIndexKeyRanges returns all key ranges associated with the tableInfo and indexInfo. -func tableIndexKeyRanges(tableInfo *model.TableInfo, indexInfo *model.IndexInfo) ([]tidbkv.KeyRange, error) { +func tableIndexKeyRanges(tableInfo *model.TableInfo, indexInfo *model.IndexInfo) (*tidbkv.KeyRanges, error) { tableIDs := physicalTableIDs(tableInfo) - //nolint: prealloc - var keyRanges []tidbkv.KeyRange - for _, tid := range tableIDs { - partitionKeysRanges, err := distsql.IndexRangesToKVRanges(nil, tid, indexInfo.ID, ranger.FullRange(), nil) - if err != nil { - return nil, errors.Trace(err) - } - keyRanges = append(keyRanges, partitionKeysRanges...) - } - return keyRanges, nil + return distsql.IndexRangesToKVRangesForTables(nil, tableIDs, indexInfo.ID, ranger.FullRange(), nil) } // DupKVStream is a streaming interface for collecting duplicate key-value pairs. @@ -561,14 +552,20 @@ func (m *DuplicateManager) buildDupTasks() ([]dupTask, error) { if err != nil { return nil, errors.Trace(err) } - tasks := make([]dupTask, 0, len(keyRanges)) - for _, kr := range keyRanges { - tableID := tablecodec.DecodeTableID(kr.StartKey) - tasks = append(tasks, dupTask{ - KeyRange: kr, - tableID: tableID, - }) + tasks := make([]dupTask, 0, keyRanges.TotalRangeNum()*(1+len(m.tbl.Meta().Indices))) + putToTaskFunc := func(ranges []tidbkv.KeyRange) { + if len(ranges) == 0 { + return + } + tid := tablecodec.DecodeTableID(ranges[0].StartKey) + for _, r := range ranges { + tasks = append(tasks, dupTask{ + KeyRange: r, + tableID: tid, + }) + } } + keyRanges.ForEachPartition(putToTaskFunc) for _, indexInfo := range m.tbl.Meta().Indices { if indexInfo.State != model.StatePublic { continue @@ -577,14 +574,7 @@ func (m *DuplicateManager) buildDupTasks() ([]dupTask, error) { if err != nil { return nil, errors.Trace(err) } - for _, kr := range keyRanges { - tableID := tablecodec.DecodeTableID(kr.StartKey) - tasks = append(tasks, dupTask{ - KeyRange: kr, - tableID: tableID, - indexInfo: indexInfo, - }) - } + keyRanges.ForEachPartition(putToTaskFunc) } return tasks, nil } @@ -598,15 +588,19 @@ func (m *DuplicateManager) buildIndexDupTasks() ([]dupTask, error) { if err != nil { return nil, errors.Trace(err) } - tasks := make([]dupTask, 0, len(keyRanges)) - for _, kr := range keyRanges { - tableID := tablecodec.DecodeTableID(kr.StartKey) - tasks = append(tasks, dupTask{ - KeyRange: kr, - tableID: tableID, - indexInfo: indexInfo, - }) - } + tasks := make([]dupTask, 0, keyRanges.TotalRangeNum()) + keyRanges.ForEachPartition(func(ranges []tidbkv.KeyRange) { + if len(ranges) == 0 { + return + } + tid := tablecodec.DecodeTableID(ranges[0].StartKey) + for _, r := range ranges { + tasks = append(tasks, dupTask{ + KeyRange: r, + tableID: tid, + }) + } + }) return tasks, nil } return nil, nil diff --git a/br/pkg/lightning/mydump/reader.go b/br/pkg/lightning/mydump/reader.go index 2988c3675dfa9..4837b35aceab2 100644 --- a/br/pkg/lightning/mydump/reader.go +++ b/br/pkg/lightning/mydump/reader.go @@ -70,6 +70,13 @@ func decodeCharacterSet(data []byte, characterSet string) ([]byte, error) { // ExportStatement exports the SQL statement in the schema file. func ExportStatement(ctx context.Context, store storage.ExternalStorage, sqlFile FileInfo, characterSet string) ([]byte, error) { + if sqlFile.FileMeta.Compression != CompressionNone { + compressType, err := ToStorageCompressType(sqlFile.FileMeta.Compression) + if err != nil { + return nil, errors.Trace(err) + } + store = storage.WithCompression(store, compressType) + } fd, err := store.Open(ctx, sqlFile.FileMeta.Path) if err != nil { return nil, errors.Trace(err) diff --git a/br/pkg/lightning/mydump/reader_test.go b/br/pkg/lightning/mydump/reader_test.go index e7506ea869782..1f67f2c31c43a 100644 --- a/br/pkg/lightning/mydump/reader_test.go +++ b/br/pkg/lightning/mydump/reader_test.go @@ -15,6 +15,7 @@ package mydump_test import ( + "compress/gzip" "context" "errors" "os" @@ -173,3 +174,28 @@ func TestExportStatementHandleNonEOFError(t *testing.T) { _, err := ExportStatement(ctx, mockStorage, f, "auto") require.Contains(t, err.Error(), "read error") } + +func TestExportStatementCompressed(t *testing.T) { + dir := t.TempDir() + file, err := os.Create(filepath.Join(dir, "tidb_lightning_test_reader")) + require.NoError(t, err) + defer os.Remove(file.Name()) + + store, err := storage.NewLocalStorage(dir) + require.NoError(t, err) + + gzipFile := gzip.NewWriter(file) + _, err = gzipFile.Write([]byte("CREATE DATABASE whatever;")) + require.NoError(t, err) + err = gzipFile.Close() + require.NoError(t, err) + stat, err := file.Stat() + require.NoError(t, err) + err = file.Close() + require.NoError(t, err) + + f := FileInfo{FileMeta: SourceFileMeta{Path: stat.Name(), FileSize: stat.Size(), Compression: CompressionGZ}} + data, err := ExportStatement(context.TODO(), store, f, "auto") + require.NoError(t, err) + require.Equal(t, []byte("CREATE DATABASE whatever;"), data) +} diff --git a/br/pkg/lightning/mydump/region.go b/br/pkg/lightning/mydump/region.go index 8562acc2867b3..ffd9173483896 100644 --- a/br/pkg/lightning/mydump/region.go +++ b/br/pkg/lightning/mydump/region.go @@ -31,9 +31,13 @@ import ( ) const ( - tableRegionSizeWarningThreshold int64 = 1024 * 1024 * 1024 + tableRegionSizeWarningThreshold int64 = 1024 * 1024 * 1024 + compressedTableRegionSizeWarningThreshold int64 = 410 * 1024 * 1024 // 0.4 * tableRegionSizeWarningThreshold // the increment ratio of large CSV file size threshold by `region-split-size` largeCSVLowerThresholdRation = 10 + // TableFileSizeINF for compressed size, for lightning 10TB is a relatively big value and will strongly affect efficiency + // It's used to make sure compressed files can be read until EOF. Because we can't get the exact decompressed size of the compressed files. + TableFileSizeINF = 10 * 1024 * tableRegionSizeWarningThreshold ) // TableRegion contains information for a table region during import. @@ -292,19 +296,34 @@ func MakeSourceFileRegion( return regions, subFileSizes, err } + fileSize := fi.FileMeta.FileSize + rowIDMax := fileSize / divisor + // for compressed files, suggest the compress ratio is 1% to calculate the rowIDMax. + // set fileSize to INF to make sure compressed files can be read until EOF. Because we can't get the exact size of the compressed files. + // TODO: update progress bar calculation for compressed files. + if fi.FileMeta.Compression != CompressionNone { + rowIDMax = fileSize * 100 / divisor // FIXME: this is not accurate. Need more tests and fix solution. + fileSize = TableFileSizeINF + } tableRegion := &TableRegion{ DB: meta.DB, Table: meta.Name, FileMeta: fi.FileMeta, Chunk: Chunk{ Offset: 0, - EndOffset: fi.FileMeta.FileSize, + EndOffset: fileSize, PrevRowIDMax: 0, - RowIDMax: fi.FileMeta.FileSize / divisor, + RowIDMax: rowIDMax, }, } - if tableRegion.Size() > tableRegionSizeWarningThreshold { + regionTooBig := false + if fi.FileMeta.Compression == CompressionNone { + regionTooBig = tableRegion.Size() > tableRegionSizeWarningThreshold + } else { + regionTooBig = fi.FileMeta.FileSize > compressedTableRegionSizeWarningThreshold + } + if regionTooBig { log.FromContext(ctx).Warn( "file is too big to be processed efficiently; we suggest splitting it at 256 MB each", zap.String("file", fi.FileMeta.Path), diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index 0830d378f47ff..0c990278e65cd 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -217,7 +217,7 @@ func TestMakeSourceFileRegion(t *testing.T) { assert.NoError(t, err) assert.Len(t, regions, 1) assert.Equal(t, int64(0), regions[0].Chunk.Offset) - assert.Equal(t, fileInfo.FileMeta.FileSize, regions[0].Chunk.EndOffset) + assert.Equal(t, TableFileSizeINF, regions[0].Chunk.EndOffset) assert.Len(t, regions[0].Chunk.Columns, 0) } diff --git a/br/pkg/lightning/mydump/router.go b/br/pkg/lightning/mydump/router.go index bdc2a922f12f7..bf0ccba834fe0 100644 --- a/br/pkg/lightning/mydump/router.go +++ b/br/pkg/lightning/mydump/router.go @@ -134,7 +134,7 @@ func parseCompressionType(t string) (Compression, error) { return CompressionGZ, nil case "lz4": return CompressionLZ4, nil - case "zstd": + case "zstd", "zst": return CompressionZStd, nil case "xz": return CompressionXZ, nil @@ -324,6 +324,9 @@ func (p regexRouterParser) Parse(r *config.FileRouteRule, logger log.Logger) (*R if err != nil { return err } + if result.Type == SourceTypeParquet && compression != CompressionNone { + return errors.Errorf("can't support whole compressed parquet file, should compress parquet files by choosing correct parquet compress writer, path: %s", r.Path) + } result.Compression = compression return nil }) diff --git a/br/pkg/lightning/mydump/router_test.go b/br/pkg/lightning/mydump/router_test.go index 4e3d8a4215a0d..ab97769e30ce8 100644 --- a/br/pkg/lightning/mydump/router_test.go +++ b/br/pkg/lightning/mydump/router_test.go @@ -292,3 +292,21 @@ func TestRouteWithPath(t *testing.T) { require.NoError(t, err) require.Nil(t, res) } + +func TestRouteWithCompressedParquet(t *testing.T) { + fileName := "myschema.my_table.000.parquet.gz" + rule := &config.FileRouteRule{ + Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)(?:\.([0-9]+))?\.(sql|csv|parquet)(?:\.(\w+))?$`, + Schema: "$1", + Table: "$2", + Type: "$4", + Key: "$3", + Compression: "$5", + Unescape: true, + } + r := *rule + router, err := NewFileRouter([]*config.FileRouteRule{&r}, log.L()) + require.NoError(t, err) + _, err = router.Route(fileName) + require.Error(t, err) +} diff --git a/br/pkg/lightning/restore/get_pre_info.go b/br/pkg/lightning/restore/get_pre_info.go index 287d59c6145a4..93927c6956809 100644 --- a/br/pkg/lightning/restore/get_pre_info.go +++ b/br/pkg/lightning/restore/get_pre_info.go @@ -444,15 +444,7 @@ func (p *PreRestoreInfoGetterImpl) ReadFirstNRowsByTableName(ctx context.Context // ReadFirstNRowsByFileMeta reads the first N rows of an data file. // It implements the PreRestoreInfoGetter interface. func (p *PreRestoreInfoGetterImpl) ReadFirstNRowsByFileMeta(ctx context.Context, dataFileMeta mydump.SourceFileMeta, n int) ([]string, [][]types.Datum, error) { - var ( - reader storage.ReadSeekCloser - err error - ) - if dataFileMeta.Type == mydump.SourceTypeParquet { - reader, err = mydump.OpenParquetReader(ctx, p.srcStorage, dataFileMeta.Path, dataFileMeta.FileSize) - } else { - reader, err = p.srcStorage.Open(ctx, dataFileMeta.Path) - } + reader, err := openReader(ctx, dataFileMeta, p.srcStorage) if err != nil { return nil, nil, errors.Trace(err) } @@ -590,13 +582,7 @@ func (p *PreRestoreInfoGetterImpl) sampleDataFromTable( return resultIndexRatio, isRowOrdered, nil } sampleFile := tableMeta.DataFiles[0].FileMeta - var reader storage.ReadSeekCloser - var err error - if sampleFile.Type == mydump.SourceTypeParquet { - reader, err = mydump.OpenParquetReader(ctx, p.srcStorage, sampleFile.Path, sampleFile.FileSize) - } else { - reader, err = p.srcStorage.Open(ctx, sampleFile.Path) - } + reader, err := openReader(ctx, sampleFile, p.srcStorage) if err != nil { return 0.0, false, errors.Trace(err) } diff --git a/br/pkg/lightning/restore/get_pre_info_test.go b/br/pkg/lightning/restore/get_pre_info_test.go index 8ea57d023c679..f66a76901116f 100644 --- a/br/pkg/lightning/restore/get_pre_info_test.go +++ b/br/pkg/lightning/restore/get_pre_info_test.go @@ -14,6 +14,8 @@ package restore import ( + "bytes" + "compress/gzip" "context" "database/sql" "fmt" @@ -24,6 +26,7 @@ import ( mysql_sql_driver "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/restore/mock" ropts "github.com/pingcap/tidb/br/pkg/lightning/restore/opts" "github.com/pingcap/tidb/errno" @@ -412,6 +415,118 @@ INSERT INTO db01.tbl01 (ival, sval) VALUES (444, 'ddd');` require.Equal(t, theDataInfo.ExpectFirstRowDatums, rowDatums) } +func compressGz(t *testing.T, data []byte) []byte { + t.Helper() + var buf bytes.Buffer + w := gzip.NewWriter(&buf) + _, err := w.Write(data) + require.NoError(t, err) + require.NoError(t, w.Close()) + return buf.Bytes() +} + +func TestGetPreInfoReadCompressedFirstRow(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + var ( + testCSVData01 = []byte(`ival,sval +111,"aaa" +222,"bbb" +`) + testSQLData01 = []byte(`INSERT INTO db01.tbl01 (ival, sval) VALUES (333, 'ccc'); +INSERT INTO db01.tbl01 (ival, sval) VALUES (444, 'ddd');`) + ) + + test1CSVCompressed := compressGz(t, testCSVData01) + test1SQLCompressed := compressGz(t, testSQLData01) + + testDataInfos := []struct { + FileName string + Data []byte + FirstN int + CSVConfig *config.CSVConfig + ExpectFirstRowDatums [][]types.Datum + ExpectColumns []string + }{ + { + FileName: "/db01/tbl01/data.001.csv.gz", + Data: test1CSVCompressed, + FirstN: 1, + ExpectFirstRowDatums: [][]types.Datum{ + { + types.NewStringDatum("111"), + types.NewStringDatum("aaa"), + }, + }, + ExpectColumns: []string{"ival", "sval"}, + }, + { + FileName: "/db01/tbl01/data.001.sql.gz", + Data: test1SQLCompressed, + FirstN: 1, + ExpectFirstRowDatums: [][]types.Datum{ + { + types.NewUintDatum(333), + types.NewStringDatum("ccc"), + }, + }, + ExpectColumns: []string{"ival", "sval"}, + }, + } + + tbl01SchemaBytes := []byte("CREATE TABLE db01.tbl01(id INTEGER PRIMARY KEY AUTO_INCREMENT, ival INTEGER, sval VARCHAR(64));") + tbl01SchemaBytesCompressed := compressGz(t, tbl01SchemaBytes) + + tblMockSourceData := &mock.MockTableSourceData{ + DBName: "db01", + TableName: "tbl01", + SchemaFile: &mock.MockSourceFile{ + FileName: "/db01/tbl01/tbl01.schema.sql.gz", + Data: tbl01SchemaBytesCompressed, + }, + DataFiles: []*mock.MockSourceFile{}, + } + for _, testInfo := range testDataInfos { + tblMockSourceData.DataFiles = append(tblMockSourceData.DataFiles, &mock.MockSourceFile{ + FileName: testInfo.FileName, + Data: testInfo.Data, + }) + } + mockDataMap := map[string]*mock.MockDBSourceData{ + "db01": { + Name: "db01", + Tables: map[string]*mock.MockTableSourceData{ + "tbl01": tblMockSourceData, + }, + }, + } + mockSrc, err := mock.NewMockImportSource(mockDataMap) + require.Nil(t, err) + mockTarget := mock.NewMockTargetInfo() + cfg := config.NewConfig() + cfg.TikvImporter.Backend = config.BackendLocal + ig, err := NewPreRestoreInfoGetter(cfg, mockSrc.GetAllDBFileMetas(), mockSrc.GetStorage(), mockTarget, nil, nil) + require.NoError(t, err) + + cfg.Mydumper.CSV.Header = true + tblMeta := mockSrc.GetDBMetaMap()["db01"].Tables[0] + for i, dataFile := range tblMeta.DataFiles { + theDataInfo := testDataInfos[i] + dataFile.FileMeta.Compression = mydump.CompressionGZ + cols, rowDatums, err := ig.ReadFirstNRowsByFileMeta(ctx, dataFile.FileMeta, theDataInfo.FirstN) + require.Nil(t, err) + t.Logf("%v, %v", cols, rowDatums) + require.Equal(t, theDataInfo.ExpectColumns, cols) + require.Equal(t, theDataInfo.ExpectFirstRowDatums, rowDatums) + } + + theDataInfo := testDataInfos[0] + cols, rowDatums, err := ig.ReadFirstNRowsByTableName(ctx, "db01", "tbl01", theDataInfo.FirstN) + require.NoError(t, err) + require.Equal(t, theDataInfo.ExpectColumns, cols) + require.Equal(t, theDataInfo.ExpectFirstRowDatums, rowDatums) +} + func TestGetPreInfoSampleSource(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -497,6 +612,100 @@ func TestGetPreInfoSampleSource(t *testing.T) { } } +func TestGetPreInfoSampleSourceCompressed(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + dataFileName := "/db01/tbl01/tbl01.data.001.csv.gz" + schemaFileData := []byte("CREATE TABLE db01.tbl01 (id INTEGER PRIMARY KEY AUTO_INCREMENT, ival INTEGER, sval VARCHAR(64));") + schemaFileDataCompressed := compressGz(t, schemaFileData) + mockDataMap := map[string]*mock.MockDBSourceData{ + "db01": { + Name: "db01", + Tables: map[string]*mock.MockTableSourceData{ + "tbl01": { + DBName: "db01", + TableName: "tbl01", + SchemaFile: &mock.MockSourceFile{ + FileName: "/db01/tbl01/tbl01.schema.sql.gz", + Data: schemaFileDataCompressed, + }, + DataFiles: []*mock.MockSourceFile{ + { + FileName: dataFileName, + Data: []byte(nil), + }, + }, + }, + }, + }, + } + mockSrc, err := mock.NewMockImportSource(mockDataMap) + require.Nil(t, err) + mockTarget := mock.NewMockTargetInfo() + cfg := config.NewConfig() + cfg.TikvImporter.Backend = config.BackendLocal + ig, err := NewPreRestoreInfoGetter(cfg, mockSrc.GetAllDBFileMetas(), mockSrc.GetStorage(), mockTarget, nil, nil, ropts.WithIgnoreDBNotExist(true)) + require.NoError(t, err) + + mdDBMeta := mockSrc.GetAllDBFileMetas()[0] + mdTblMeta := mdDBMeta.Tables[0] + dbInfos, err := ig.GetAllTableStructures(ctx) + require.NoError(t, err) + + data := [][]byte{ + []byte(`id,ival,sval +1,111,"aaa" +2,222,"bbb" +`), + []byte(`sval,ival,id +"aaa",111,1 +"bbb",222,2 +`), + []byte(`id,ival,sval +2,222,"bbb" +1,111,"aaa" +`), + []byte(`sval,ival,id +"aaa",111,2 +"bbb",222,1 +`), + } + compressedData := make([][]byte, 0, 4) + for _, d := range data { + compressedData = append(compressedData, compressGz(t, d)) + } + + subTests := []struct { + Data []byte + ExpectIsOrdered bool + }{ + { + Data: compressedData[0], + ExpectIsOrdered: true, + }, + { + Data: compressedData[1], + ExpectIsOrdered: true, + }, + { + Data: compressedData[2], + ExpectIsOrdered: false, + }, + { + Data: compressedData[3], + ExpectIsOrdered: false, + }, + } + for _, subTest := range subTests { + require.NoError(t, mockSrc.GetStorage().WriteFile(ctx, dataFileName, subTest.Data)) + sampledIndexRatio, isRowOrderedFromSample, err := ig.sampleDataFromTable(ctx, "db01", mdTblMeta, dbInfos["db01"].Tables["tbl01"].Core, nil, defaultImportantVariables) + require.NoError(t, err) + t.Logf("%v, %v", sampledIndexRatio, isRowOrderedFromSample) + require.Greater(t, sampledIndexRatio, 1.0) + require.Equal(t, subTest.ExpectIsOrdered, isRowOrderedFromSample) + } +} + func TestGetPreInfoEstimateSourceSize(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/br/pkg/lightning/restore/mock/mock.go b/br/pkg/lightning/restore/mock/mock.go index f43e6c022673e..5556e1caf3363 100644 --- a/br/pkg/lightning/restore/mock/mock.go +++ b/br/pkg/lightning/restore/mock/mock.go @@ -77,14 +77,19 @@ func NewMockImportSource(dbSrcDataMap map[string]*MockDBSourceData) (*MockImport tblMeta := mydump.NewMDTableMeta("binary") tblMeta.DB = dbName tblMeta.Name = tblName + compression := mydump.CompressionNone + if strings.HasSuffix(tblData.SchemaFile.FileName, ".gz") { + compression = mydump.CompressionGZ + } tblMeta.SchemaFile = mydump.FileInfo{ TableName: filter.Table{ Schema: dbName, Name: tblName, }, FileMeta: mydump.SourceFileMeta{ - Path: tblData.SchemaFile.FileName, - Type: mydump.SourceTypeTableSchema, + Path: tblData.SchemaFile.FileName, + Type: mydump.SourceTypeTableSchema, + Compression: compression, }, } tblMeta.DataFiles = []mydump.FileInfo{} @@ -108,12 +113,17 @@ func NewMockImportSource(dbSrcDataMap map[string]*MockDBSourceData) (*MockImport FileSize: int64(fileSize), }, } + fileName := tblDataFile.FileName + if strings.HasSuffix(fileName, ".gz") { + fileName = strings.TrimSuffix(tblDataFile.FileName, ".gz") + fileInfo.FileMeta.Compression = mydump.CompressionGZ + } switch { - case strings.HasSuffix(tblDataFile.FileName, ".csv"): + case strings.HasSuffix(fileName, ".csv"): fileInfo.FileMeta.Type = mydump.SourceTypeCSV - case strings.HasSuffix(tblDataFile.FileName, ".sql"): + case strings.HasSuffix(fileName, ".sql"): fileInfo.FileMeta.Type = mydump.SourceTypeSQL - case strings.HasSuffix(tblDataFile.FileName, ".parquet"): + case strings.HasSuffix(fileName, ".parquet"): fileInfo.FileMeta.Type = mydump.SourceTypeParquet default: return nil, errors.Errorf("unsupported file type: %s", tblDataFile.FileName) diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 99b56d05414ce..210435640473f 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -2199,23 +2199,7 @@ func newChunkRestore( ) (*chunkRestore, error) { blockBufSize := int64(cfg.Mydumper.ReadBlockSize) - var ( - reader storage.ReadSeekCloser - compressType storage.CompressType - err error - ) - switch { - case chunk.FileMeta.Type == mydump.SourceTypeParquet: - reader, err = mydump.OpenParquetReader(ctx, store, chunk.FileMeta.Path, chunk.FileMeta.FileSize) - case chunk.FileMeta.Compression != mydump.CompressionNone: - compressType, err = mydump.ToStorageCompressType(chunk.FileMeta.Compression) - if err != nil { - break - } - reader, err = storage.WithCompression(store, compressType).Open(ctx, chunk.FileMeta.Path) - default: - reader, err = store.Open(ctx, chunk.FileMeta.Path) - } + reader, err := openReader(ctx, chunk.FileMeta, store) if err != nil { return nil, errors.Trace(err) } @@ -2790,3 +2774,20 @@ func (cr *chunkRestore) restore( } return errors.Trace(firstErr(encodeErr, deliverErr)) } + +func openReader(ctx context.Context, fileMeta mydump.SourceFileMeta, store storage.ExternalStorage) ( + reader storage.ReadSeekCloser, err error) { + switch { + case fileMeta.Type == mydump.SourceTypeParquet: + reader, err = mydump.OpenParquetReader(ctx, store, fileMeta.Path, fileMeta.FileSize) + case fileMeta.Compression != mydump.CompressionNone: + compressType, err2 := mydump.ToStorageCompressType(fileMeta.Compression) + if err2 != nil { + return nil, err2 + } + reader, err = storage.WithCompression(store, compressType).Open(ctx, fileMeta.Path) + default: + reader, err = store.Open(ctx, fileMeta.Path) + } + return +} diff --git a/br/pkg/logutil/logging.go b/br/pkg/logutil/logging.go index 028cfc00e5f43..41b8e135c220f 100644 --- a/br/pkg/logutil/logging.go +++ b/br/pkg/logutil/logging.go @@ -306,3 +306,13 @@ func (rng StringifyRange) String() string { sb.WriteString(")") return sb.String() } + +// StringifyMany returns an array marshaler for a slice of stringers. +func StringifyMany[T fmt.Stringer](items []T) zapcore.ArrayMarshaler { + return zapcore.ArrayMarshalerFunc(func(ae zapcore.ArrayEncoder) error { + for _, item := range items { + ae.AppendString(item.String()) + } + return nil + }) +} diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 77c2fc2976570..772ec438976d7 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -120,7 +120,7 @@ go_test( embed = [":restore"], flaky = True, race = "on", - shard_count = 20, + shard_count = 50, deps = [ "//br/pkg/backup", "//br/pkg/conn", diff --git a/br/pkg/restore/client_test.go b/br/pkg/restore/client_test.go index 36de6c9b2eff7..af1aa023560b6 100644 --- a/br/pkg/restore/client_test.go +++ b/br/pkg/restore/client_test.go @@ -1348,13 +1348,6 @@ func TestApplyKVFilesWithBatchMethod3(t *testing.T) { Cf: stream.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, - }, { - Path: "log4", - NumberOfEntries: 5, - Length: 100, - Cf: stream.WriteCF, - Type: backuppb.FileType_Put, - RegionId: 2, }, { Path: "log5", NumberOfEntries: 5, @@ -1394,15 +1387,13 @@ func TestApplyKVFilesWithBatchMethod3(t *testing.T) { applyFunc, ) - require.Equal(t, runCount, 5) - require.Equal(t, totalKVCount, int64(30)) + require.Equal(t, totalKVCount, int64(25)) require.Equal(t, logs, [][]string{ {"log2"}, {"log5", "log6"}, {"log3"}, - {"log4"}, {"log1"}, }, ) diff --git a/br/pkg/restore/data.go b/br/pkg/restore/data.go index f7efce83362f5..7432c3d9af0ce 100644 --- a/br/pkg/restore/data.go +++ b/br/pkg/restore/data.go @@ -372,6 +372,7 @@ type RecoverRegion struct { // 2. build a leader list for all region during the tikv startup // 3. get max allocate id func (recovery *Recovery) MakeRecoveryPlan() error { + storeBalanceScore := make(map[uint64]int, len(recovery.allStores)) // Group region peer info by region id. find the max allocateId // region [id] [peer[0-n]] var regions = make(map[uint64][]*RecoverRegion, 0) @@ -410,16 +411,20 @@ func (recovery *Recovery) MakeRecoveryPlan() error { } } else { // Generate normal commands. - log.Debug("detected valid peer", zap.Uint64("region id", regionId)) - for i, peer := range peers { - log.Debug("make plan", zap.Uint64("store id", peer.StoreId), zap.Uint64("region id", peer.RegionId)) - plan := &recovpb.RecoverRegionRequest{RegionId: peer.RegionId, AsLeader: i == 0} - // sorted by log term -> last index -> commit index in a region - if plan.AsLeader { - log.Debug("as leader peer", zap.Uint64("store id", peer.StoreId), zap.Uint64("region id", peer.RegionId)) - recovery.RecoveryPlan[peer.StoreId] = append(recovery.RecoveryPlan[peer.StoreId], plan) - } + log.Debug("detected valid region", zap.Uint64("region id", regionId)) + // calc the leader candidates + leaderCandidates, err := LeaderCandidates(peers) + if err != nil { + log.Warn("region without peer", zap.Uint64("region id", regionId)) + return errors.Trace(err) } + + // select the leader base on tikv storeBalanceScore + leader := SelectRegionLeader(storeBalanceScore, leaderCandidates) + log.Debug("as leader peer", zap.Uint64("store id", leader.StoreId), zap.Uint64("region id", leader.RegionId)) + plan := &recovpb.RecoverRegionRequest{RegionId: leader.RegionId, AsLeader: true} + recovery.RecoveryPlan[leader.StoreId] = append(recovery.RecoveryPlan[leader.StoreId], plan) + storeBalanceScore[leader.StoreId] += 1 } } return nil diff --git a/br/pkg/restore/util.go b/br/pkg/restore/util.go index 259d3fa28d888..73a4411c445c1 100644 --- a/br/pkg/restore/util.go +++ b/br/pkg/restore/util.go @@ -750,3 +750,43 @@ func CheckConsistencyAndValidPeer(regionInfos []*RecoverRegionInfo) (map[uint64] } return validPeers, nil } + +// in cloud, since iops and bandwidth limitation, write operator in raft is slow, so raft state (logterm, lastlog, commitlog...) are the same among the peers +// LeaderCandidates select all peers can be select as a leader during the restore +func LeaderCandidates(peers []*RecoverRegion) ([]*RecoverRegion, error) { + if peers == nil { + return nil, errors.Annotatef(berrors.ErrRestoreRegionWithoutPeer, + "invalid region range") + } + candidates := make([]*RecoverRegion, 0, len(peers)) + // by default, the peers[0] to be assign as a leader, since peers already sorted by leader selection rule + leader := peers[0] + candidates = append(candidates, leader) + for _, peer := range peers[1:] { + // qualificated candidate is leader.logterm = candidate.logterm && leader.lastindex = candidate.lastindex && && leader.commitindex = candidate.commitindex + if peer.LastLogTerm == leader.LastLogTerm && peer.LastIndex == leader.LastIndex && peer.CommitIndex == leader.CommitIndex { + log.Debug("leader candidate", zap.Uint64("store id", peer.StoreId), zap.Uint64("region id", peer.RegionId), zap.Uint64("peer id", peer.PeerId)) + candidates = append(candidates, peer) + } + } + return candidates, nil +} + +// for region A, has candidate leader x, y, z +// peer x on store 1 with storeBalanceScore 3 +// peer y on store 3 with storeBalanceScore 2 +// peer z on store 4 with storeBalanceScore 1 +// result: peer z will be select as leader on store 4 +func SelectRegionLeader(storeBalanceScore map[uint64]int, peers []*RecoverRegion) *RecoverRegion { + // by default, the peers[0] to be assign as a leader + leader := peers[0] + minLeaderStore := storeBalanceScore[leader.StoreId] + for _, peer := range peers[1:] { + log.Debug("leader candidate", zap.Int("score", storeBalanceScore[peer.StoreId]), zap.Int("min-score", minLeaderStore), zap.Uint64("store id", peer.StoreId), zap.Uint64("region id", peer.RegionId), zap.Uint64("peer id", peer.PeerId)) + if storeBalanceScore[peer.StoreId] < minLeaderStore { + minLeaderStore = storeBalanceScore[peer.StoreId] + leader = peer + } + } + return leader +} diff --git a/br/pkg/restore/util_test.go b/br/pkg/restore/util_test.go index 44620e9cb4e5c..482818a1ad958 100644 --- a/br/pkg/restore/util_test.go +++ b/br/pkg/restore/util_test.go @@ -460,3 +460,52 @@ func TestCheckConsistencyAndValidPeer(t *testing.T) { require.Error(t, err) require.Regexp(t, ".*invalid restore range.*", err.Error()) } + +func TestLeaderCandidates(t *testing.T) { + //key space is continuous + validPeer1 := newPeerMeta(9, 11, 2, []byte(""), []byte("bb"), 2, 1, 0, 0, false) + validPeer2 := newPeerMeta(19, 22, 3, []byte("bb"), []byte("cc"), 2, 1, 0, 1, false) + validPeer3 := newPeerMeta(29, 30, 1, []byte("cc"), []byte(""), 2, 1, 0, 2, false) + + peers := []*restore.RecoverRegion{ + validPeer1, + validPeer2, + validPeer3, + } + + candidates, err := restore.LeaderCandidates(peers) + require.NoError(t, err) + require.Equal(t, 3, len(candidates)) +} + +func TestSelectRegionLeader(t *testing.T) { + validPeer1 := newPeerMeta(9, 11, 2, []byte(""), []byte("bb"), 2, 1, 0, 0, false) + validPeer2 := newPeerMeta(19, 22, 3, []byte("bb"), []byte("cc"), 2, 1, 0, 1, false) + validPeer3 := newPeerMeta(29, 30, 1, []byte("cc"), []byte(""), 2, 1, 0, 2, false) + + peers := []*restore.RecoverRegion{ + validPeer1, + validPeer2, + validPeer3, + } + // init store banlance score all is 0 + storeBalanceScore := make(map[uint64]int, len(peers)) + leader := restore.SelectRegionLeader(storeBalanceScore, peers) + require.Equal(t, validPeer1, leader) + + // change store banlance store + storeBalanceScore[2] = 3 + storeBalanceScore[3] = 2 + storeBalanceScore[1] = 1 + leader = restore.SelectRegionLeader(storeBalanceScore, peers) + require.Equal(t, validPeer3, leader) + + // one peer + peer := []*restore.RecoverRegion{ + validPeer3, + } + // init store banlance score all is 0 + storeScore := make(map[uint64]int, len(peer)) + leader = restore.SelectRegionLeader(storeScore, peer) + require.Equal(t, validPeer3, leader) +} diff --git a/br/pkg/streamhelper/BUILD.bazel b/br/pkg/streamhelper/BUILD.bazel index 93e13b1f8d543..83d80e52620ef 100644 --- a/br/pkg/streamhelper/BUILD.bazel +++ b/br/pkg/streamhelper/BUILD.bazel @@ -12,7 +12,6 @@ go_library( "models.go", "prefix_scanner.go", "regioniter.go", - "tsheap.go", ], importpath = "github.com/pingcap/tidb/br/pkg/streamhelper", visibility = ["//visibility:public"], @@ -21,6 +20,7 @@ go_library( "//br/pkg/logutil", "//br/pkg/redact", "//br/pkg/streamhelper/config", + "//br/pkg/streamhelper/spans", "//br/pkg/utils", "//config", "//kv", @@ -29,7 +29,6 @@ go_library( "//util/mathutil", "@com_github_gogo_protobuf//proto", "@com_github_golang_protobuf//proto", - "@com_github_google_btree//:btree", "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/brpb", @@ -44,7 +43,6 @@ go_library( "@org_golang_google_grpc//keepalive", "@org_golang_x_sync//errgroup", "@org_uber_go_zap//:zap", - "@org_uber_go_zap//zapcore", ], ) @@ -56,7 +54,6 @@ go_test( "basic_lib_for_test.go", "integration_test.go", "regioniter_test.go", - "tsheap_test.go", ], flaky = True, race = "on", @@ -68,6 +65,7 @@ go_test( "//br/pkg/redact", "//br/pkg/storage", "//br/pkg/streamhelper/config", + "//br/pkg/streamhelper/spans", "//br/pkg/utils", "//kv", "//tablecodec", diff --git a/br/pkg/streamhelper/advancer.go b/br/pkg/streamhelper/advancer.go index ac01c5167ffc7..60bb2928dc08a 100644 --- a/br/pkg/streamhelper/advancer.go +++ b/br/pkg/streamhelper/advancer.go @@ -3,11 +3,7 @@ package streamhelper import ( - "bytes" "context" - "math" - "reflect" - "sort" "strings" "sync" "time" @@ -17,6 +13,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/streamhelper/config" + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -60,81 +57,28 @@ type CheckpointAdvancer struct { // once tick begin, this should not be changed for now. cfg config.Config - // the cache of region checkpoints. - // so we can advance only ranges with huge gap. - cache CheckpointsCache - - // the internal state of advancer. - state advancerState // the cached last checkpoint. // if no progress, this cache can help us don't to send useless requests. lastCheckpoint uint64 -} - -// advancerState is the sealed type for the state of advancer. -// the advancer has two stage: full scan and update small tree. -type advancerState interface { - // Note: - // Go doesn't support sealed classes or ADTs currently. - // (it can only be used at generic constraints...) - // Leave it empty for now. - - // ~*fullScan | ~*updateSmallTree -} -// fullScan is the initial state of advancer. -// in this stage, we would "fill" the cache: -// insert ranges that union of them become the full range of task. -type fullScan struct { - fullScanTick int -} - -// updateSmallTree is the "incremental stage" of advancer. -// we have build a "filled" cache, and we can pop a subrange of it, -// try to advance the checkpoint of those ranges. -type updateSmallTree struct { - consistencyCheckTick int + checkpoints *spans.ValueSortedFull + checkpointsMu sync.Mutex } // NewCheckpointAdvancer creates a checkpoint advancer with the env. func NewCheckpointAdvancer(env Env) *CheckpointAdvancer { return &CheckpointAdvancer{ - env: env, - cfg: config.Default(), - cache: NewCheckpoints(), - state: &fullScan{}, + env: env, + cfg: config.Default(), } } -// disableCache removes the cache. -// note this won't lock the checkpoint advancer at `fullScan` state forever, -// you may need to change the config `AdvancingByCache`. -func (c *CheckpointAdvancer) disableCache() { - c.cache = NoOPCheckpointCache{} - c.state = &fullScan{} -} - -// enable the cache. -// also check `AdvancingByCache` in the config. -func (c *CheckpointAdvancer) enableCache() { - c.cache = NewCheckpoints() - c.state = &fullScan{} -} - // UpdateConfig updates the config for the advancer. // Note this should be called before starting the loop, because there isn't locks, // TODO: support updating config when advancer starts working. // (Maybe by applying changes at begin of ticking, and add locks.) func (c *CheckpointAdvancer) UpdateConfig(newConf config.Config) { - needRefreshCache := newConf.AdvancingByCache != c.cfg.AdvancingByCache c.cfg = newConf - if needRefreshCache { - if c.cfg.AdvancingByCache { - c.enableCache() - } else { - c.disableCache() - } - } } // UpdateConfigWith updates the config by modifying the current config. @@ -183,28 +127,24 @@ func (c *CheckpointAdvancer) recordTimeCost(message string, fields ...zap.Field) } // tryAdvance tries to advance the checkpoint ts of a set of ranges which shares the same checkpoint. -func (c *CheckpointAdvancer) tryAdvance(ctx context.Context, rst RangesSharesTS) (err error) { - defer c.recordTimeCost("try advance", zap.Uint64("checkpoint", rst.TS), zap.Int("len", len(rst.Ranges)))() - defer func() { - if err != nil { - log.Warn("failed to advance", logutil.ShortError(err), zap.Object("target", rst.Zap())) - c.cache.InsertRanges(rst) - } - }() +func (c *CheckpointAdvancer) tryAdvance(ctx context.Context, length int, getRange func(int) kv.KeyRange) (err error) { + defer c.recordTimeCost("try advance", zap.Int("len", length))() defer utils.PanicToErr(&err) - ranges := CollapseRanges(len(rst.Ranges), func(i int) kv.KeyRange { - return rst.Ranges[i] - }) - workers := utils.NewWorkerPool(4, "sub ranges") + ranges := spans.Collapse(length, getRange) + workers := utils.NewWorkerPool(uint(config.DefaultMaxConcurrencyAdvance)*4, "sub ranges") eg, cx := errgroup.WithContext(ctx) collector := NewClusterCollector(ctx, c.env) - collector.setOnSuccessHook(c.cache.InsertRange) + collector.setOnSuccessHook(func(u uint64, kr kv.KeyRange) { + c.checkpointsMu.Lock() + defer c.checkpointsMu.Unlock() + c.checkpoints.Merge(spans.Valued{Key: kr, Value: u}) + }) clampedRanges := utils.IntersectAll(ranges, utils.CloneSlice(c.taskRange)) for _, r := range clampedRanges { r := r workers.ApplyOnErrorGroup(eg, func() (e error) { - defer c.recordTimeCost("get regions in range", zap.Uint64("checkpoint", rst.TS))() + defer c.recordTimeCost("get regions in range")() defer utils.PanicToErr(&e) return c.GetCheckpointInRange(cx, r.StartKey, r.EndKey, collector) }) @@ -214,121 +154,44 @@ func (c *CheckpointAdvancer) tryAdvance(ctx context.Context, rst RangesSharesTS) return err } - result, err := collector.Finish(ctx) + _, err = collector.Finish(ctx) if err != nil { return err } - fr := result.FailureSubRanges - if len(fr) != 0 { - log.Debug("failure regions collected", zap.Int("size", len(fr))) - c.cache.InsertRanges(RangesSharesTS{ - TS: rst.TS, - Ranges: fr, - }) - } return nil } +func tsoBefore(n time.Duration) uint64 { + now := time.Now() + return oracle.ComposeTS(now.UnixMilli()-n.Milliseconds(), 0) +} + // CalculateGlobalCheckpointLight tries to advance the global checkpoint by the cache. func (c *CheckpointAdvancer) CalculateGlobalCheckpointLight(ctx context.Context) (uint64, error) { - log.Info("[log backup advancer hint] advancer with cache: current tree", zap.Stringer("ct", c.cache)) - rsts := c.cache.PopRangesWithGapGT(config.DefaultTryAdvanceThreshold) - if len(rsts) == 0 { + var targets []spans.Valued + c.checkpoints.TraverseValuesLessThan(tsoBefore(config.DefaultTryAdvanceThreshold), func(v spans.Valued) bool { + targets = append(targets, v) + return true + }) + if len(targets) == 0 { return 0, nil } - samples := rsts - if len(rsts) > 3 { - samples = rsts[:3] + samples := targets + if len(targets) > 3 { + samples = targets[:3] } for _, sample := range samples { - log.Info("[log backup advancer hint] sample range.", zap.Object("range", sample.Zap()), zap.Int("total-len", len(rsts))) + log.Info("[log backup advancer hint] sample range.", zap.Stringer("sample", sample), zap.Int("total-len", len(targets))) } - workers := utils.NewWorkerPool(uint(config.DefaultMaxConcurrencyAdvance), "regions") - eg, cx := errgroup.WithContext(ctx) - for _, rst := range rsts { - rst := rst - workers.ApplyOnErrorGroup(eg, func() (err error) { - return c.tryAdvance(cx, *rst) - }) - } - err := eg.Wait() + err := c.tryAdvance(ctx, len(targets), func(i int) kv.KeyRange { return targets[i].Key }) if err != nil { return 0, err } - ts := c.cache.CheckpointTS() + ts := c.checkpoints.MinValue() return ts, nil } -// CalculateGlobalCheckpoint calculates the global checkpoint, which won't use the cache. -func (c *CheckpointAdvancer) CalculateGlobalCheckpoint(ctx context.Context) (uint64, error) { - var ( - cp = uint64(math.MaxInt64) - thisRun []kv.KeyRange = c.taskRange - nextRun []kv.KeyRange - ) - defer c.recordTimeCost("record all") - for { - coll := NewClusterCollector(ctx, c.env) - coll.setOnSuccessHook(c.cache.InsertRange) - for _, u := range thisRun { - err := c.GetCheckpointInRange(ctx, u.StartKey, u.EndKey, coll) - if err != nil { - return 0, err - } - } - result, err := coll.Finish(ctx) - if err != nil { - return 0, err - } - log.Debug("full: a run finished", zap.Any("checkpoint", result)) - - nextRun = append(nextRun, result.FailureSubRanges...) - if cp > result.Checkpoint { - cp = result.Checkpoint - } - if len(nextRun) == 0 { - return cp, nil - } - thisRun = nextRun - nextRun = nil - log.Debug("backoffing with subranges", zap.Int("subranges", len(thisRun))) - time.Sleep(c.cfg.BackoffTime) - } -} - -// CollapseRanges collapse ranges overlapping or adjacent. -// Example: -// CollapseRanges({[1, 4], [2, 8], [3, 9]}) == {[1, 9]} -// CollapseRanges({[1, 3], [4, 7], [2, 3]}) == {[1, 3], [4, 7]} -func CollapseRanges(length int, getRange func(int) kv.KeyRange) []kv.KeyRange { - frs := make([]kv.KeyRange, 0, length) - for i := 0; i < length; i++ { - frs = append(frs, getRange(i)) - } - - sort.Slice(frs, func(i, j int) bool { - return bytes.Compare(frs[i].StartKey, frs[j].StartKey) < 0 - }) - - result := make([]kv.KeyRange, 0, len(frs)) - i := 0 - for i < len(frs) { - item := frs[i] - for { - i++ - if i >= len(frs) || (len(item.EndKey) != 0 && bytes.Compare(frs[i].StartKey, item.EndKey) > 0) { - break - } - if len(item.EndKey) != 0 && bytes.Compare(item.EndKey, frs[i].EndKey) < 0 || len(frs[i].EndKey) == 0 { - item.EndKey = frs[i].EndKey - } - } - result = append(result, item) - } - return result -} - func (c *CheckpointAdvancer) consumeAllTask(ctx context.Context, ch <-chan TaskEvent) error { for { select { @@ -414,18 +277,18 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error case EventAdd: utils.LogBackupTaskCountInc() c.task = e.Info - c.taskRange = CollapseRanges(len(e.Ranges), func(i int) kv.KeyRange { return e.Ranges[i] }) + c.taskRange = spans.Collapse(len(e.Ranges), func(i int) kv.KeyRange { return e.Ranges[i] }) + c.checkpoints = spans.Sorted(spans.NewFullWith(e.Ranges, 0)) log.Info("added event", zap.Stringer("task", e.Info), zap.Stringer("ranges", logutil.StringifyKeys(c.taskRange))) case EventDel: utils.LogBackupTaskCountDec() c.task = nil c.taskRange = nil - c.state = &fullScan{} + c.checkpoints = nil if err := c.env.ClearV3GlobalCheckpointForTask(ctx, e.Name); err != nil { log.Warn("failed to clear global checkpoint", logutil.ShortError(err)) } metrics.LastCheckpoint.DeleteLabelValues(e.Name) - c.cache.Clear() case EventErr: return e.Err } @@ -460,58 +323,17 @@ func (c *CheckpointAdvancer) advanceCheckpointBy(ctx context.Context, getCheckpo return nil } -func (c *CheckpointAdvancer) onConsistencyCheckTick(s *updateSmallTree) error { - if s.consistencyCheckTick > 0 { - s.consistencyCheckTick-- +func (c *CheckpointAdvancer) tick(ctx context.Context) error { + c.taskMu.Lock() + defer c.taskMu.Unlock() + if c.task == nil { + log.Debug("No tasks yet, skipping advancing.") return nil } - defer c.recordTimeCost("consistency check")() - err := c.cache.ConsistencyCheck(c.taskRange) + err := c.advanceCheckpointBy(ctx, c.CalculateGlobalCheckpointLight) if err != nil { - log.Error("consistency check failed! log backup may lose data! rolling back to full scan for saving.", logutil.ShortError(err)) - c.state = &fullScan{} return err } - log.Debug("consistency check passed.") - s.consistencyCheckTick = config.DefaultConsistencyCheckTick - return nil -} -func (c *CheckpointAdvancer) tick(ctx context.Context) error { - c.taskMu.Lock() - defer c.taskMu.Unlock() - - switch s := c.state.(type) { - case *fullScan: - if s.fullScanTick > 0 { - s.fullScanTick-- - break - } - if c.task == nil { - log.Debug("No tasks yet, skipping advancing.") - return nil - } - defer func() { - s.fullScanTick = c.cfg.FullScanTick - }() - err := c.advanceCheckpointBy(ctx, c.CalculateGlobalCheckpoint) - if err != nil { - return err - } - - if c.cfg.AdvancingByCache { - c.state = &updateSmallTree{} - } - case *updateSmallTree: - if err := c.onConsistencyCheckTick(s); err != nil { - return err - } - err := c.advanceCheckpointBy(ctx, c.CalculateGlobalCheckpointLight) - if err != nil { - return err - } - default: - log.Error("Unknown state type, skipping tick", zap.Stringer("type", reflect.TypeOf(c.state))) - } return nil } diff --git a/br/pkg/streamhelper/basic_lib_for_test.go b/br/pkg/streamhelper/basic_lib_for_test.go index b41d5baf19528..9b73745ef65d3 100644 --- a/br/pkg/streamhelper/basic_lib_for_test.go +++ b/br/pkg/streamhelper/basic_lib_for_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/kv" "go.uber.org/zap" @@ -82,16 +83,6 @@ type fakeCluster struct { onGetClient func(uint64) error } -func overlaps(a, b kv.KeyRange) bool { - if len(b.EndKey) == 0 { - return len(a.EndKey) == 0 || bytes.Compare(a.EndKey, b.StartKey) > 0 - } - if len(a.EndKey) == 0 { - return len(b.EndKey) == 0 || bytes.Compare(b.EndKey, a.StartKey) > 0 - } - return bytes.Compare(a.StartKey, b.EndKey) < 0 && bytes.Compare(b.StartKey, a.EndKey) < 0 -} - func (r *region) splitAt(newID uint64, k string) *region { newRegion := ®ion{ rng: kv.KeyRange{StartKey: []byte(k), EndKey: r.rng.EndKey}, @@ -178,7 +169,7 @@ func (f *fakeCluster) RegionScan(ctx context.Context, key []byte, endKey []byte, result := make([]streamhelper.RegionWithLeader, 0, limit) for _, region := range f.regions { - if overlaps(kv.KeyRange{StartKey: key, EndKey: endKey}, region.rng) && len(result) < limit { + if spans.Overlaps(kv.KeyRange{StartKey: key, EndKey: endKey}, region.rng) && len(result) < limit { regionInfo := streamhelper.RegionWithLeader{ Region: &metapb.Region{ Id: region.id, diff --git a/br/pkg/streamhelper/regioniter_test.go b/br/pkg/streamhelper/regioniter_test.go index 04ccc04da8a66..c8281d7a5f33b 100644 --- a/br/pkg/streamhelper/regioniter_test.go +++ b/br/pkg/streamhelper/regioniter_test.go @@ -13,6 +13,7 @@ import ( "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" "github.com/pingcap/tidb/kv" "github.com/stretchr/testify/require" ) @@ -55,7 +56,7 @@ func (c constantRegions) String() string { func (c constantRegions) RegionScan(ctx context.Context, key []byte, endKey []byte, limit int) ([]streamhelper.RegionWithLeader, error) { result := make([]streamhelper.RegionWithLeader, 0, limit) for _, region := range c { - if overlaps(kv.KeyRange{StartKey: key, EndKey: endKey}, kv.KeyRange{StartKey: region.Region.StartKey, EndKey: region.Region.EndKey}) && len(result) < limit { + if spans.Overlaps(kv.KeyRange{StartKey: key, EndKey: endKey}, kv.KeyRange{StartKey: region.Region.StartKey, EndKey: region.Region.EndKey}) && len(result) < limit { result = append(result, region) } else if bytes.Compare(region.Region.StartKey, key) > 0 { break diff --git a/br/pkg/streamhelper/spans/BUILD.bazel b/br/pkg/streamhelper/spans/BUILD.bazel new file mode 100644 index 0000000000000..899f6f6ade6b1 --- /dev/null +++ b/br/pkg/streamhelper/spans/BUILD.bazel @@ -0,0 +1,31 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "spans", + srcs = [ + "sorted.go", + "utils.go", + "value_sorted.go", + ], + importpath = "github.com/pingcap/tidb/br/pkg/streamhelper/spans", + visibility = ["//visibility:public"], + deps = [ + "//br/pkg/logutil", + "//br/pkg/utils", + "//kv", + "@com_github_google_btree//:btree", + ], +) + +go_test( + name = "spans_test", + srcs = [ + "sorted_test.go", + "utils_test.go", + "value_sorted_test.go", + ], + deps = [ + ":spans", + "@com_github_stretchr_testify//require", + ], +) diff --git a/br/pkg/streamhelper/spans/sorted.go b/br/pkg/streamhelper/spans/sorted.go new file mode 100644 index 0000000000000..7b9692f529e5b --- /dev/null +++ b/br/pkg/streamhelper/spans/sorted.go @@ -0,0 +1,159 @@ +package spans + +import ( + "bytes" + "fmt" + + "github.com/google/btree" + "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/kv" +) + +// Value is the value type of stored in the span tree. +type Value = uint64 + +// join finds the upper bound of two values. +func join(a, b Value) Value { + if a > b { + return a + } + return b +} + +// Span is the type of an adjacent sub key space. +type Span = kv.KeyRange + +// Valued is span binding to a value, which is the entry type of span tree. +type Valued struct { + Key Span + Value Value +} + +func (r Valued) String() string { + return fmt.Sprintf("(%s, %d)", logutil.StringifyRange(r.Key), r.Value) +} + +func (r Valued) Less(other btree.Item) bool { + return bytes.Compare(r.Key.StartKey, other.(Valued).Key.StartKey) < 0 +} + +// ValuedFull represents a set of valued ranges, which doesn't overlap and union of them all is the full key space. +type ValuedFull struct { + inner *btree.BTree +} + +// NewFullWith creates a set of a subset of spans. +func NewFullWith(initSpans []Span, init Value) *ValuedFull { + t := btree.New(16) + for _, r := range Collapse(len(initSpans), func(i int) Span { return initSpans[i] }) { + t.ReplaceOrInsert(Valued{Value: init, Key: r}) + } + return &ValuedFull{inner: t} +} + +func (f *ValuedFull) Merge(val Valued) { + overlaps := make([]Valued, 0, 16) + f.overlapped(val.Key, &overlaps) + f.mergeWithOverlap(val, overlaps, nil) +} + +func (f *ValuedFull) Traverse(m func(Valued) bool) { + f.inner.Ascend(func(item btree.Item) bool { + return m(item.(Valued)) + }) +} + +func (f *ValuedFull) mergeWithOverlap(val Valued, overlapped []Valued, newItems *[]Valued) { + // There isn't any range overlaps with the input range, perhaps the input range is empty. + // do nothing for this case. + if len(overlapped) == 0 { + return + } + + for _, r := range overlapped { + f.inner.Delete(r) + // Assert All overlapped ranges are deleted. + } + + var ( + initialized = false + collected Valued + rightTrail *Valued + flushCollected = func() { + if initialized { + f.inner.ReplaceOrInsert(collected) + if newItems != nil { + *newItems = append(*newItems, collected) + } + } + } + emitToCollected = func(rng Valued, standalone bool) { + merged := rng.Value + if !standalone { + merged = join(val.Value, rng.Value) + } + if !initialized { + collected = rng + collected.Value = merged + initialized = true + return + } + if merged == collected.Value && utils.CompareBytesExt(collected.Key.EndKey, true, rng.Key.StartKey, false) == 0 { + collected.Key.EndKey = rng.Key.EndKey + } else { + flushCollected() + collected = Valued{ + Key: rng.Key, + Value: merged, + } + } + } + ) + + leftmost := overlapped[0] + if bytes.Compare(leftmost.Key.StartKey, val.Key.StartKey) < 0 { + emitToCollected(Valued{ + Key: Span{StartKey: leftmost.Key.StartKey, EndKey: val.Key.StartKey}, + Value: leftmost.Value, + }, true) + overlapped[0].Key.StartKey = val.Key.StartKey + } + + rightmost := overlapped[len(overlapped)-1] + if utils.CompareBytesExt(rightmost.Key.EndKey, true, val.Key.EndKey, true) > 0 { + rightTrail = &Valued{ + Key: Span{StartKey: val.Key.EndKey, EndKey: rightmost.Key.EndKey}, + Value: rightmost.Value, + } + overlapped[len(overlapped)-1].Key.EndKey = val.Key.EndKey + } + + for _, rng := range overlapped { + emitToCollected(rng, false) + } + + if rightTrail != nil { + emitToCollected(*rightTrail, true) + } + + flushCollected() +} + +// overlapped inserts the overlapped ranges of the span into the `result` slice. +func (f *ValuedFull) overlapped(k Span, result *[]Valued) { + var first Span + f.inner.DescendLessOrEqual(Valued{Key: k}, func(item btree.Item) bool { + first = item.(Valued).Key + return false + }) + + f.inner.AscendGreaterOrEqual(Valued{Key: first}, func(item btree.Item) bool { + r := item.(Valued) + if !Overlaps(r.Key, k) { + return false + } + *result = append(*result, r) + return true + }) +} diff --git a/br/pkg/streamhelper/spans/sorted_test.go b/br/pkg/streamhelper/spans/sorted_test.go new file mode 100644 index 0000000000000..4cea720577e59 --- /dev/null +++ b/br/pkg/streamhelper/spans/sorted_test.go @@ -0,0 +1,172 @@ +package spans_test + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" + "github.com/stretchr/testify/require" +) + +func s(a, b string) spans.Span { + return spans.Span{ + StartKey: []byte(a), + EndKey: []byte(b), + } +} + +func kv(s spans.Span, v spans.Value) spans.Valued { + return spans.Valued{ + Key: s, + Value: v, + } +} + +func TestBasic(t *testing.T) { + type Case struct { + InputSequence []spans.Valued + Result []spans.Valued + } + + run := func(t *testing.T, c Case) { + full := spans.NewFullWith(spans.Full(), 0) + fmt.Println(t.Name()) + for _, i := range c.InputSequence { + full.Merge(i) + var result []spans.Valued + full.Traverse(func(v spans.Valued) bool { + result = append(result, v) + return true + }) + fmt.Printf("%s -> %s\n", i, result) + } + + var result []spans.Valued + full.Traverse(func(v spans.Valued) bool { + result = append(result, v) + return true + }) + + require.True(t, spans.ValuedSetEquals(result, c.Result), "%s\nvs\n%s", result, c.Result) + } + + cases := []Case{ + { + InputSequence: []spans.Valued{ + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + }, + Result: []spans.Valued{ + kv(s("", "0001"), 0), + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + kv(s("0003", ""), 0), + }, + }, + { + InputSequence: []spans.Valued{ + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + kv(s("0001", "0003"), 4), + }, + Result: []spans.Valued{ + kv(s("", "0001"), 0), + kv(s("0001", "0003"), 4), + kv(s("0003", ""), 0), + }, + }, + { + InputSequence: []spans.Valued{ + kv(s("0001", "0004"), 3), + kv(s("0004", "0008"), 5), + kv(s("0001", "0007"), 4), + kv(s("", "0002"), 2), + }, + Result: []spans.Valued{ + kv(s("", "0001"), 2), + kv(s("0001", "0004"), 4), + kv(s("0004", "0008"), 5), + kv(s("0008", ""), 0), + }, + }, + { + InputSequence: []spans.Valued{ + kv(s("0001", "0004"), 3), + kv(s("0004", "0008"), 5), + kv(s("0001", "0009"), 4), + }, + Result: []spans.Valued{ + kv(s("", "0001"), 0), + kv(s("0001", "0004"), 4), + kv(s("0004", "0008"), 5), + kv(s("0008", "0009"), 4), + kv(s("0009", ""), 0), + }, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("#%d", i+1), func(t *testing.T) { run(t, c) }) + } +} + +func TestSubRange(t *testing.T) { + type Case struct { + Range []spans.Span + InputSequence []spans.Valued + Result []spans.Valued + } + + run := func(t *testing.T, c Case) { + full := spans.NewFullWith(c.Range, 0) + fmt.Println(t.Name()) + for _, i := range c.InputSequence { + full.Merge(i) + var result []spans.Valued + full.Traverse(func(v spans.Valued) bool { + result = append(result, v) + return true + }) + fmt.Printf("%s -> %s\n", i, result) + } + + var result []spans.Valued + full.Traverse(func(v spans.Valued) bool { + result = append(result, v) + return true + }) + + require.True(t, spans.ValuedSetEquals(result, c.Result), "%s\nvs\n%s", result, c.Result) + } + + cases := []Case{ + { + Range: []spans.Span{s("0001", "0004"), s("0008", "")}, + InputSequence: []spans.Valued{ + kv(s("0001", "0007"), 42), + kv(s("0000", "0009"), 41), + kv(s("0002", "0005"), 43), + }, + Result: []spans.Valued{ + kv(s("0001", "0002"), 42), + kv(s("0002", "0004"), 43), + kv(s("0008", "0009"), 41), + kv(s("0009", ""), 0), + }, + }, + { + Range: []spans.Span{ + s("0001", "0004"), + s("0008", "")}, + InputSequence: []spans.Valued{kv(s("", ""), 42)}, + Result: []spans.Valued{ + kv(s("0001", "0004"), 42), + kv(s("0008", ""), 42), + }, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("#%d", i+1), func(t *testing.T) { run(t, c) }) + } +} diff --git a/br/pkg/streamhelper/spans/utils.go b/br/pkg/streamhelper/spans/utils.go new file mode 100644 index 0000000000000..b6369c58f5840 --- /dev/null +++ b/br/pkg/streamhelper/spans/utils.go @@ -0,0 +1,148 @@ +package spans + +import ( + "bytes" + "fmt" + "math" + "sort" + + "github.com/pingcap/tidb/br/pkg/utils" +) + +// Overlaps checks whether two spans have overlapped part. +func Overlaps(a, b Span) bool { + if len(b.EndKey) == 0 { + return len(a.EndKey) == 0 || bytes.Compare(a.EndKey, b.StartKey) > 0 + } + if len(a.EndKey) == 0 { + return len(b.EndKey) == 0 || bytes.Compare(b.EndKey, a.StartKey) > 0 + } + return bytes.Compare(a.StartKey, b.EndKey) < 0 && bytes.Compare(b.StartKey, a.EndKey) < 0 +} + +func Debug(full *ValueSortedFull) { + var result []Valued + full.Traverse(func(v Valued) bool { + result = append(result, v) + return true + }) + var idx []Valued + full.TraverseValuesLessThan(math.MaxUint64, func(v Valued) bool { + idx = append(idx, v) + return true + }) + fmt.Printf("%s\n\tidx = %s\n", result, idx) +} + +// Collapse collapse ranges overlapping or adjacent. +// Example: +// Collapse({[1, 4], [2, 8], [3, 9]}) == {[1, 9]} +// Collapse({[1, 3], [4, 7], [2, 3]}) == {[1, 3], [4, 7]} +func Collapse(length int, getRange func(int) Span) []Span { + frs := make([]Span, 0, length) + for i := 0; i < length; i++ { + frs = append(frs, getRange(i)) + } + + sort.Slice(frs, func(i, j int) bool { + start := bytes.Compare(frs[i].StartKey, frs[j].StartKey) + if start != 0 { + return start < 0 + } + return utils.CompareBytesExt(frs[i].EndKey, true, frs[j].EndKey, true) < 0 + }) + + result := make([]Span, 0, len(frs)) + i := 0 + for i < len(frs) { + item := frs[i] + for { + i++ + if i >= len(frs) || (len(item.EndKey) != 0 && bytes.Compare(frs[i].StartKey, item.EndKey) > 0) { + break + } + if len(item.EndKey) != 0 && bytes.Compare(item.EndKey, frs[i].EndKey) < 0 || len(frs[i].EndKey) == 0 { + item.EndKey = frs[i].EndKey + } + } + result = append(result, item) + } + return result +} + +// Full returns a full span crossing the key space. +func Full() []Span { + return []Span{{}} +} + +func (x Valued) Equals(y Valued) bool { + return x.Value == y.Value && bytes.Equal(x.Key.StartKey, y.Key.StartKey) && bytes.Equal(x.Key.EndKey, y.Key.EndKey) +} + +func ValuedSetEquals(xs, ys []Valued) bool { + if len(xs) == 0 || len(ys) == 0 { + return len(ys) == len(xs) + } + + sort.Slice(xs, func(i, j int) bool { + start := bytes.Compare(xs[i].Key.StartKey, xs[j].Key.StartKey) + if start != 0 { + return start < 0 + } + return utils.CompareBytesExt(xs[i].Key.EndKey, true, xs[j].Key.EndKey, true) < 0 + }) + sort.Slice(ys, func(i, j int) bool { + start := bytes.Compare(ys[i].Key.StartKey, ys[j].Key.StartKey) + if start != 0 { + return start < 0 + } + return utils.CompareBytesExt(ys[i].Key.EndKey, true, ys[j].Key.EndKey, true) < 0 + }) + + xi := 0 + yi := 0 + + for { + if xi >= len(xs) || yi >= len(ys) { + return (xi >= len(xs)) == (yi >= len(ys)) + } + x := xs[xi] + y := ys[yi] + + if !bytes.Equal(x.Key.StartKey, y.Key.StartKey) { + return false + } + + for { + if xi >= len(xs) || yi >= len(ys) { + return (xi >= len(xs)) == (yi >= len(ys)) + } + x := xs[xi] + y := ys[yi] + + if x.Value != y.Value { + return false + } + + c := utils.CompareBytesExt(x.Key.EndKey, true, y.Key.EndKey, true) + if c == 0 { + xi++ + yi++ + break + } + if c < 0 { + xi++ + // If not adjacent key, return false directly. + if xi < len(xs) && utils.CompareBytesExt(x.Key.EndKey, true, xs[xi].Key.StartKey, false) != 0 { + return false + } + } + if c > 0 { + yi++ + if yi < len(ys) && utils.CompareBytesExt(y.Key.EndKey, true, ys[yi].Key.StartKey, false) != 0 { + return false + } + } + } + } +} diff --git a/br/pkg/streamhelper/spans/utils_test.go b/br/pkg/streamhelper/spans/utils_test.go new file mode 100644 index 0000000000000..0e591d3143ec9 --- /dev/null +++ b/br/pkg/streamhelper/spans/utils_test.go @@ -0,0 +1,81 @@ +package spans_test + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" + "github.com/stretchr/testify/require" +) + +func TestValuedEquals(t *testing.T) { + s := func(start, end string, val spans.Value) spans.Valued { + return spans.Valued{ + Key: spans.Span{ + StartKey: []byte(start), + EndKey: []byte(end), + }, + Value: val, + } + } + type Case struct { + inputA []spans.Valued + inputB []spans.Valued + required bool + } + cases := []Case{ + { + inputA: []spans.Valued{s("0001", "0002", 3)}, + inputB: []spans.Valued{s("0001", "0003", 3)}, + required: false, + }, + { + inputA: []spans.Valued{s("0001", "0002", 3)}, + inputB: []spans.Valued{s("0001", "0002", 3)}, + required: true, + }, + { + inputA: []spans.Valued{s("0001", "0003", 3)}, + inputB: []spans.Valued{s("0001", "0002", 3), s("0002", "0003", 3)}, + required: true, + }, + { + inputA: []spans.Valued{s("0001", "0003", 4)}, + inputB: []spans.Valued{s("0001", "0002", 3), s("0002", "0003", 3)}, + required: false, + }, + { + inputA: []spans.Valued{s("0001", "0003", 3)}, + inputB: []spans.Valued{s("0001", "0002", 4), s("0002", "0003", 3)}, + required: false, + }, + { + inputA: []spans.Valued{s("0001", "0003", 3)}, + inputB: []spans.Valued{s("0001", "0002", 3), s("0002", "0004", 3)}, + required: false, + }, + { + inputA: []spans.Valued{s("", "0003", 3)}, + inputB: []spans.Valued{s("0001", "0002", 3), s("0002", "0003", 3)}, + required: false, + }, + { + inputA: []spans.Valued{s("0001", "", 1)}, + inputB: []spans.Valued{s("0001", "0003", 1), s("0004", "", 1)}, + required: false, + }, + { + inputA: []spans.Valued{s("0001", "0004", 1), s("0001", "0002", 1)}, + inputB: []spans.Valued{s("0001", "0002", 1), s("0001", "0004", 1)}, + required: true, + }, + } + run := func(t *testing.T, c Case) { + require.Equal(t, c.required, spans.ValuedSetEquals(c.inputA, c.inputB)) + require.Equal(t, c.required, spans.ValuedSetEquals(c.inputB, c.inputA)) + } + + for i, c := range cases { + t.Run(fmt.Sprintf("#%d", i+1), func(t *testing.T) { run(t, c) }) + } +} diff --git a/br/pkg/streamhelper/spans/value_sorted.go b/br/pkg/streamhelper/spans/value_sorted.go new file mode 100644 index 0000000000000..6e2cf942af578 --- /dev/null +++ b/br/pkg/streamhelper/spans/value_sorted.go @@ -0,0 +1,64 @@ +package spans + +import "github.com/google/btree" + +type sortedByValueThenStartKey Valued + +func (s sortedByValueThenStartKey) Less(o btree.Item) bool { + other := o.(sortedByValueThenStartKey) + if s.Value != other.Value { + return s.Value < other.Value + } + return Valued(s).Less(Valued(other)) +} + +type ValueSortedFull struct { + *ValuedFull + valueIdx *btree.BTree +} + +func Sorted(f *ValuedFull) *ValueSortedFull { + vf := &ValueSortedFull{ + ValuedFull: f, + valueIdx: btree.New(16), + } + f.Traverse(func(v Valued) bool { + vf.valueIdx.ReplaceOrInsert(sortedByValueThenStartKey(v)) + return true + }) + return vf +} + +func (v *ValueSortedFull) Merge(newItem Valued) { + v.MergeAll([]Valued{newItem}) +} + +func (v *ValueSortedFull) MergeAll(newItems []Valued) { + var overlapped []Valued + var inserted []Valued + + for _, item := range newItems { + overlapped = overlapped[:0] + inserted = inserted[:0] + + v.overlapped(item.Key, &overlapped) + v.mergeWithOverlap(item, overlapped, &inserted) + + for _, o := range overlapped { + v.valueIdx.Delete(sortedByValueThenStartKey(o)) + } + for _, i := range inserted { + v.valueIdx.ReplaceOrInsert(sortedByValueThenStartKey(i)) + } + } +} + +func (v *ValueSortedFull) TraverseValuesLessThan(n Value, action func(Valued) bool) { + v.valueIdx.AscendLessThan(sortedByValueThenStartKey{Value: n}, func(item btree.Item) bool { + return action(Valued(item.(sortedByValueThenStartKey))) + }) +} + +func (v *ValueSortedFull) MinValue() Value { + return v.valueIdx.Min().(sortedByValueThenStartKey).Value +} diff --git a/br/pkg/streamhelper/spans/value_sorted_test.go b/br/pkg/streamhelper/spans/value_sorted_test.go new file mode 100644 index 0000000000000..36ef744fb4ef4 --- /dev/null +++ b/br/pkg/streamhelper/spans/value_sorted_test.go @@ -0,0 +1,80 @@ +package spans_test + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/br/pkg/streamhelper/spans" + "github.com/stretchr/testify/require" +) + +func TestSortedBasic(t *testing.T) { + type Case struct { + InputSequence []spans.Valued + RetainLessThan spans.Value + Result []spans.Valued + } + + run := func(t *testing.T, c Case) { + full := spans.Sorted(spans.NewFullWith(spans.Full(), 0)) + fmt.Println(t.Name()) + for _, i := range c.InputSequence { + full.Merge(i) + spans.Debug(full) + } + + var result []spans.Valued + full.TraverseValuesLessThan(c.RetainLessThan, func(v spans.Valued) bool { + result = append(result, v) + return true + }) + + require.True(t, spans.ValuedSetEquals(result, c.Result), "%s\nvs\n%s", result, c.Result) + } + + cases := []Case{ + { + InputSequence: []spans.Valued{ + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + }, + Result: []spans.Valued{ + kv(s("", "0001"), 0), + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + kv(s("0003", ""), 0), + }, + RetainLessThan: 10, + }, + { + InputSequence: []spans.Valued{ + kv(s("0001", "0002"), 1), + kv(s("0002", "0003"), 2), + kv(s("0001", "0003"), 4), + }, + RetainLessThan: 1, + Result: []spans.Valued{ + kv(s("", "0001"), 0), + kv(s("0003", ""), 0), + }, + }, + { + InputSequence: []spans.Valued{ + kv(s("0001", "0004"), 3), + kv(s("0004", "0008"), 5), + kv(s("0001", "0007"), 4), + kv(s("", "0002"), 2), + }, + RetainLessThan: 5, + Result: []spans.Valued{ + kv(s("", "0001"), 2), + kv(s("0001", "0004"), 4), + kv(s("0008", ""), 0), + }, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("#%d", i+1), func(t *testing.T) { run(t, c) }) + } +} diff --git a/br/pkg/streamhelper/tsheap.go b/br/pkg/streamhelper/tsheap.go deleted file mode 100644 index 6c2fb510776e7..0000000000000 --- a/br/pkg/streamhelper/tsheap.go +++ /dev/null @@ -1,326 +0,0 @@ -// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. - -package streamhelper - -import ( - "encoding/hex" - "fmt" - "strings" - "sync" - "time" - - "github.com/google/btree" - "github.com/pingcap/errors" - berrors "github.com/pingcap/tidb/br/pkg/errors" - "github.com/pingcap/tidb/br/pkg/logutil" - "github.com/pingcap/tidb/br/pkg/redact" - "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/kv" - "github.com/tikv/client-go/v2/oracle" - "go.uber.org/zap/zapcore" -) - -// CheckpointsCache is the heap-like cache for checkpoints. -// -// "Checkpoint" is the "Resolved TS" of some range. -// A resolved ts is a "watermark" for the system, which: -// - implies there won't be any transactions (in some range) commit with `commit_ts` smaller than this TS. -// - is monotonic increasing. -// A "checkpoint" is a "safe" Resolved TS, which: -// - is a TS *less than* the real resolved ts of now. -// - is based on range (it only promises there won't be new committed txns in the range). -// - the checkpoint of union of ranges is the minimal checkpoint of all ranges. -// As an example: -/* - +----------------------------------+ - ^-----------^ (Checkpoint = 42) - ^---------------^ (Checkpoint = 76) - ^-----------------------^ (Checkpoint = min(42, 76) = 42) -*/ -// For calculating the global checkpoint, we can make a heap-like structure: -// Checkpoint Ranges -// 42 -> {[0, 8], [16, 100]} -// 1002 -> {[8, 16]} -// 1082 -> {[100, inf]} -// For now, the checkpoint of range [8, 16] and [100, inf] won't affect the global checkpoint -// directly, so we can try to advance only the ranges of {[0, 8], [16, 100]} (which's checkpoint is steal). -// Once them get advance, the global checkpoint would be advanced then, -// and we don't need to update all ranges (because some new ranges don't need to be advanced so quickly.) -type CheckpointsCache interface { - fmt.Stringer - // InsertRange inserts a range with specified TS to the cache. - InsertRange(ts uint64, rng kv.KeyRange) - // InsertRanges inserts a set of ranges that sharing checkpoint to the cache. - InsertRanges(rst RangesSharesTS) - // CheckpointTS returns the now global (union of all ranges) checkpoint of the cache. - CheckpointTS() uint64 - // PopRangesWithGapGT pops the ranges which's checkpoint is - PopRangesWithGapGT(d time.Duration) []*RangesSharesTS - // Check whether the ranges in the cache is integrate. - ConsistencyCheck(ranges []kv.KeyRange) error - // Clear the cache. - Clear() -} - -// NoOPCheckpointCache is used when cache disabled. -type NoOPCheckpointCache struct{} - -func (NoOPCheckpointCache) InsertRange(ts uint64, rng kv.KeyRange) {} - -func (NoOPCheckpointCache) InsertRanges(rst RangesSharesTS) {} - -func (NoOPCheckpointCache) Clear() {} - -func (NoOPCheckpointCache) String() string { - return "NoOPCheckpointCache" -} - -func (NoOPCheckpointCache) CheckpointTS() uint64 { - panic("invalid state: NoOPCheckpointCache should never be used in advancing!") -} - -func (NoOPCheckpointCache) PopRangesWithGapGT(d time.Duration) []*RangesSharesTS { - panic("invalid state: NoOPCheckpointCache should never be used in advancing!") -} - -func (NoOPCheckpointCache) ConsistencyCheck([]kv.KeyRange) error { - return errors.Annotatef(berrors.ErrUnsupportedOperation, "invalid state: NoOPCheckpointCache should never be used in advancing!") -} - -// RangesSharesTS is a set of ranges shares the same timestamp. -type RangesSharesTS struct { - TS uint64 - Ranges []kv.KeyRange -} - -func (rst *RangesSharesTS) Zap() zapcore.ObjectMarshaler { - return zapcore.ObjectMarshalerFunc(func(oe zapcore.ObjectEncoder) error { - rngs := rst.Ranges - if len(rst.Ranges) > 3 { - rngs = rst.Ranges[:3] - } - - oe.AddUint64("checkpoint", rst.TS) - return oe.AddArray("items", zapcore.ArrayMarshalerFunc(func(ae zapcore.ArrayEncoder) error { - return ae.AppendObject(zapcore.ObjectMarshalerFunc(func(oe1 zapcore.ObjectEncoder) error { - for _, rng := range rngs { - oe1.AddString("start-key", redact.String(hex.EncodeToString(rng.StartKey))) - oe1.AddString("end-key", redact.String(hex.EncodeToString(rng.EndKey))) - } - return nil - })) - })) - }) -} - -func (rst *RangesSharesTS) String() string { - // Make a more friendly string. - return fmt.Sprintf("@%sR%d", oracle.GetTimeFromTS(rst.TS).Format("0405"), len(rst.Ranges)) -} - -func (rst *RangesSharesTS) Less(other btree.Item) bool { - return rst.TS < other.(*RangesSharesTS).TS -} - -// Checkpoints is a heap that collects all checkpoints of -// regions, it supports query the latest checkpoint fast. -// This structure is thread safe. -type Checkpoints struct { - tree *btree.BTree - - mu sync.Mutex -} - -func NewCheckpoints() *Checkpoints { - return &Checkpoints{ - tree: btree.New(32), - } -} - -// String formats the slowest 5 ranges sharing TS to string. -func (h *Checkpoints) String() string { - h.mu.Lock() - defer h.mu.Unlock() - - b := new(strings.Builder) - count := 0 - total := h.tree.Len() - h.tree.Ascend(func(i btree.Item) bool { - rst := i.(*RangesSharesTS) - b.WriteString(rst.String()) - b.WriteString(";") - count++ - return count < 5 - }) - if total-count > 0 { - fmt.Fprintf(b, "O%d", total-count) - } - return b.String() -} - -// InsertRanges insert a RangesSharesTS directly to the tree. -func (h *Checkpoints) InsertRanges(r RangesSharesTS) { - h.mu.Lock() - defer h.mu.Unlock() - if items := h.tree.Get(&r); items != nil { - i := items.(*RangesSharesTS) - i.Ranges = append(i.Ranges, r.Ranges...) - } else { - h.tree.ReplaceOrInsert(&r) - } -} - -// InsertRange inserts the region and its TS into the region tree. -func (h *Checkpoints) InsertRange(ts uint64, rng kv.KeyRange) { - h.mu.Lock() - defer h.mu.Unlock() - r := h.tree.Get(&RangesSharesTS{TS: ts}) - if r == nil { - r = &RangesSharesTS{TS: ts} - h.tree.ReplaceOrInsert(r) - } - rr := r.(*RangesSharesTS) - rr.Ranges = append(rr.Ranges, rng) -} - -// Clear removes all records in the checkpoint cache. -func (h *Checkpoints) Clear() { - h.mu.Lock() - defer h.mu.Unlock() - h.tree.Clear(false) -} - -// PopRangesWithGapGT pops ranges with gap greater than the specified duration. -// NOTE: maybe make something like `DrainIterator` for better composing? -func (h *Checkpoints) PopRangesWithGapGT(d time.Duration) []*RangesSharesTS { - h.mu.Lock() - defer h.mu.Unlock() - result := []*RangesSharesTS{} - for { - item, ok := h.tree.Min().(*RangesSharesTS) - if !ok { - return result - } - if time.Since(oracle.GetTimeFromTS(item.TS)) >= d { - result = append(result, item) - h.tree.DeleteMin() - } else { - return result - } - } -} - -// CheckpointTS returns the cached checkpoint TS by the current state of the cache. -func (h *Checkpoints) CheckpointTS() uint64 { - h.mu.Lock() - defer h.mu.Unlock() - item, ok := h.tree.Min().(*RangesSharesTS) - if !ok { - return 0 - } - return item.TS -} - -// ConsistencyCheck checks whether the tree contains the full range of key space. -func (h *Checkpoints) ConsistencyCheck(rangesIn []kv.KeyRange) error { - h.mu.Lock() - rangesReal := make([]kv.KeyRange, 0, 1024) - h.tree.Ascend(func(i btree.Item) bool { - rangesReal = append(rangesReal, i.(*RangesSharesTS).Ranges...) - return true - }) - h.mu.Unlock() - - r := CollapseRanges(len(rangesReal), func(i int) kv.KeyRange { return rangesReal[i] }) - ri := CollapseRanges(len(rangesIn), func(i int) kv.KeyRange { return rangesIn[i] }) - - return errors.Annotatef(checkIntervalIsSubset(r, ri), "ranges: (current) %s (not in) %s", logutil.StringifyKeys(r), - logutil.StringifyKeys(ri)) -} - -// A simple algorithm to detect non-overlapped ranges. -// It maintains the "current" probe, and let the ranges to check "consume" it. -// For example: -// toCheck: |_____________________| |_____________| -// . ^checking -// subsetOf: |_________| |_______| |__________| -// . ^probing -// probing is the subrange of checking, consume it and move forward the probe. -// toCheck: |_____________________| |_____________| -// . ^checking -// subsetOf: |_________| |_______| |__________| -// . ^probing -// consume it, too. -// toCheck: |_____________________| |_____________| -// . ^checking -// subsetOf: |_________| |_______| |__________| -// . ^probing -// checking is at the left of probing and no overlaps, moving it forward. -// toCheck: |_____________________| |_____________| -// . ^checking -// subsetOf: |_________| |_______| |__________| -// . ^probing -// consume it. all subset ranges are consumed, check passed. -func checkIntervalIsSubset(toCheck []kv.KeyRange, subsetOf []kv.KeyRange) error { - i := 0 - si := 0 - - for { - // We have checked all ranges. - if si >= len(subsetOf) { - return nil - } - // There are some ranges doesn't reach the end. - if i >= len(toCheck) { - return errors.Annotatef(berrors.ErrPiTRMalformedMetadata, - "there remains a range doesn't be fully consumed: %s", - logutil.StringifyRange(subsetOf[si])) - } - - checking := toCheck[i] - probing := subsetOf[si] - // checking: |___________| - // probing: |_________| - // A rare case: the "first" range is out of bound or not fully covers the probing range. - if utils.CompareBytesExt(checking.StartKey, false, probing.StartKey, false) > 0 { - holeEnd := checking.StartKey - if utils.CompareBytesExt(holeEnd, false, probing.EndKey, true) > 0 { - holeEnd = probing.EndKey - } - return errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "probably a hole in key ranges: %s", logutil.StringifyRange{ - StartKey: probing.StartKey, - EndKey: holeEnd, - }) - } - - // checking: |_____| - // probing: |_______| - // Just move forward checking. - if utils.CompareBytesExt(checking.EndKey, true, probing.StartKey, false) < 0 { - i += 1 - continue - } - - // checking: |_________| - // probing: |__________________| - // Given all of the ranges are "collapsed", the next checking range must - // not be adjacent with the current checking range. - // And hence there must be a "hole" in the probing key space. - if utils.CompareBytesExt(checking.EndKey, true, probing.EndKey, true) < 0 { - next := probing.EndKey - if i+1 < len(toCheck) { - next = toCheck[i+1].EndKey - } - return errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "probably a hole in key ranges: %s", logutil.StringifyRange{ - StartKey: checking.EndKey, - EndKey: next, - }) - } - // checking: |________________| - // probing: |_____________| - // The current checking range fills the current probing range, - // or the current checking range is out of the current range. - // let's move the probing forward. - si += 1 - } -} diff --git a/br/pkg/streamhelper/tsheap_test.go b/br/pkg/streamhelper/tsheap_test.go deleted file mode 100644 index 173bc2e0a0334..0000000000000 --- a/br/pkg/streamhelper/tsheap_test.go +++ /dev/null @@ -1,248 +0,0 @@ -// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. -package streamhelper_test - -import ( - "fmt" - "math" - "math/rand" - "testing" - - "github.com/pingcap/tidb/br/pkg/streamhelper" - "github.com/pingcap/tidb/kv" - "github.com/stretchr/testify/require" -) - -func TestInsert(t *testing.T) { - cases := []func(func(ts uint64, a, b string)){ - func(insert func(ts uint64, a, b string)) { - insert(1, "", "01") - insert(1, "01", "02") - insert(2, "02", "022") - insert(4, "022", "") - }, - func(insert func(ts uint64, a, b string)) { - insert(1, "", "01") - insert(2, "", "01") - insert(2, "011", "02") - insert(1, "", "") - insert(65, "03", "04") - }, - } - - for _, c := range cases { - cps := streamhelper.NewCheckpoints() - expected := map[uint64]*streamhelper.RangesSharesTS{} - checkpoint := uint64(math.MaxUint64) - insert := func(ts uint64, a, b string) { - cps.InsertRange(ts, kv.KeyRange{ - StartKey: []byte(a), - EndKey: []byte(b), - }) - i, ok := expected[ts] - if !ok { - expected[ts] = &streamhelper.RangesSharesTS{TS: ts, Ranges: []kv.KeyRange{{StartKey: []byte(a), EndKey: []byte(b)}}} - } else { - i.Ranges = append(i.Ranges, kv.KeyRange{StartKey: []byte(a), EndKey: []byte(b)}) - } - if ts < checkpoint { - checkpoint = ts - } - } - c(insert) - require.Equal(t, checkpoint, cps.CheckpointTS()) - rngs := cps.PopRangesWithGapGT(0) - for _, rng := range rngs { - other := expected[rng.TS] - require.Equal(t, other, rng) - } - } -} - -func TestMergeRanges(t *testing.T) { - r := func(a, b string) kv.KeyRange { - return kv.KeyRange{StartKey: []byte(a), EndKey: []byte(b)} - } - type Case struct { - expected []kv.KeyRange - parameter []kv.KeyRange - } - cases := []Case{ - { - parameter: []kv.KeyRange{r("01", "01111"), r("0111", "0112")}, - expected: []kv.KeyRange{r("01", "0112")}, - }, - { - parameter: []kv.KeyRange{r("01", "03"), r("02", "04")}, - expected: []kv.KeyRange{r("01", "04")}, - }, - { - parameter: []kv.KeyRange{r("04", "08"), r("09", "10")}, - expected: []kv.KeyRange{r("04", "08"), r("09", "10")}, - }, - { - parameter: []kv.KeyRange{r("01", "03"), r("02", "04"), r("05", "07"), r("08", "09")}, - expected: []kv.KeyRange{r("01", "04"), r("05", "07"), r("08", "09")}, - }, - { - parameter: []kv.KeyRange{r("01", "02"), r("012", "")}, - expected: []kv.KeyRange{r("01", "")}, - }, - { - parameter: []kv.KeyRange{r("", "01"), r("02", "03"), r("021", "")}, - expected: []kv.KeyRange{r("", "01"), r("02", "")}, - }, - { - parameter: []kv.KeyRange{r("", "01"), r("001", "")}, - expected: []kv.KeyRange{r("", "")}, - }, - { - parameter: []kv.KeyRange{r("", "01"), r("", ""), r("", "02")}, - expected: []kv.KeyRange{r("", "")}, - }, - { - parameter: []kv.KeyRange{r("", "01"), r("01", ""), r("", "02"), r("", "03"), r("01", "02")}, - expected: []kv.KeyRange{r("", "")}, - }, - { - parameter: []kv.KeyRange{r("", ""), r("", "01"), r("01", ""), r("01", "02")}, - expected: []kv.KeyRange{r("", "")}, - }, - } - - for i, c := range cases { - result := streamhelper.CollapseRanges(len(c.parameter), func(i int) kv.KeyRange { - return c.parameter[i] - }) - require.Equal(t, c.expected, result, "case = %d", i) - } -} - -func TestInsertRanges(t *testing.T) { - r := func(a, b string) kv.KeyRange { - return kv.KeyRange{StartKey: []byte(a), EndKey: []byte(b)} - } - rs := func(ts uint64, ranges ...kv.KeyRange) streamhelper.RangesSharesTS { - return streamhelper.RangesSharesTS{TS: ts, Ranges: ranges} - } - - type Case struct { - Expected []streamhelper.RangesSharesTS - Parameters []streamhelper.RangesSharesTS - } - - cases := []Case{ - { - Parameters: []streamhelper.RangesSharesTS{ - rs(1, r("0", "1"), r("1", "2")), - rs(1, r("2", "3"), r("3", "4")), - }, - Expected: []streamhelper.RangesSharesTS{ - rs(1, r("0", "1"), r("1", "2"), r("2", "3"), r("3", "4")), - }, - }, - { - Parameters: []streamhelper.RangesSharesTS{ - rs(1, r("0", "1")), - rs(2, r("2", "3")), - rs(1, r("4", "5"), r("6", "7")), - }, - Expected: []streamhelper.RangesSharesTS{ - rs(1, r("0", "1"), r("4", "5"), r("6", "7")), - rs(2, r("2", "3")), - }, - }, - } - - for _, c := range cases { - theTree := streamhelper.NewCheckpoints() - for _, p := range c.Parameters { - theTree.InsertRanges(p) - } - ranges := theTree.PopRangesWithGapGT(0) - for i, rs := range ranges { - require.ElementsMatch(t, c.Expected[i].Ranges, rs.Ranges, "case = %#v", c) - } - } -} - -func TestConsistencyCheckOverRange(t *testing.T) { - r := func(a, b string) kv.KeyRange { - return kv.KeyRange{StartKey: []byte(a), EndKey: []byte(b)} - } - type Case struct { - checking []kv.KeyRange - probing []kv.KeyRange - isSubset bool - } - - cases := []Case{ - // basic: exactly match. - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "0005")}, - probing: []kv.KeyRange{r("0001", "0003"), r("0004", "0005")}, - isSubset: true, - }, - // not fully match, probing longer. - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "0005")}, - probing: []kv.KeyRange{r("0000", "0003"), r("0004", "00051")}, - isSubset: false, - }, - // with infinity end keys. - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "")}, - probing: []kv.KeyRange{r("0001", "0003"), r("0004", "")}, - isSubset: true, - }, - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "")}, - probing: []kv.KeyRange{r("0001", "0003"), r("0004", "0005")}, - isSubset: true, - }, - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "0005")}, - probing: []kv.KeyRange{r("0001", "0003"), r("0004", "")}, - isSubset: false, - }, - // overlapped probe. - { - checking: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "0007")}, - probing: []kv.KeyRange{r("0001", "0008")}, - isSubset: false, - }, - { - checking: []kv.KeyRange{r("0001", "0008")}, - probing: []kv.KeyRange{r("0001", "0002"), r("0002", "0003"), r("0004", "0007")}, - isSubset: true, - }, - { - checking: []kv.KeyRange{r("0100", "0120"), r("0130", "0141")}, - probing: []kv.KeyRange{r("0000", "0001")}, - isSubset: false, - }, - { - checking: []kv.KeyRange{r("0100", "0120")}, - probing: []kv.KeyRange{r("0090", "0110"), r("0115", "0120")}, - isSubset: false, - }, - } - - run := func(t *testing.T, c Case) { - tree := streamhelper.NewCheckpoints() - for _, r := range c.checking { - tree.InsertRange(rand.Uint64()%10, r) - } - err := tree.ConsistencyCheck(c.probing) - if c.isSubset { - require.NoError(t, err) - } else { - require.Error(t, err) - } - } - - for i, c := range cases { - t.Run(fmt.Sprintf("#%d", i), func(tc *testing.T) { - run(tc, c) - }) - } -} diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index c5a3871bd2ae5..83c22a29e61db 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -58,18 +58,19 @@ const ( // FlagStreamFullBackupStorage is used for log restore, represents the full backup storage. FlagStreamFullBackupStorage = "full-backup-storage" // FlagPiTRBatchCount and FlagPiTRBatchSize are used for restore log with batch method. - FlagPiTRBatchCount = "pitr-batch-count" - FlagPiTRBatchSize = "pitr-batch-size" - - defaultPiTRBatchCount = 16 - defaultPiTRBatchSize = 32 * 1024 * 1024 - defaultRestoreConcurrency = 128 - defaultRestoreStreamConcurrency = 128 - maxRestoreBatchSizeLimit = 10240 - defaultPDConcurrency = 1 - defaultBatchFlushInterval = 16 * time.Second - defaultFlagDdlBatchSize = 128 - resetSpeedLimitRetryTimes = 3 + FlagPiTRBatchCount = "pitr-batch-count" + FlagPiTRBatchSize = "pitr-batch-size" + FlagPiTRConcurrency = "pitr-concurrency" + + defaultPiTRBatchCount = 8 + defaultPiTRBatchSize = 16 * 1024 * 1024 + defaultRestoreConcurrency = 128 + defaultPiTRConcurrency = 16 + maxRestoreBatchSizeLimit = 10240 + defaultPDConcurrency = 1 + defaultBatchFlushInterval = 16 * time.Second + defaultFlagDdlBatchSize = 128 + resetSpeedLimitRetryTimes = 3 ) const ( @@ -175,6 +176,7 @@ type RestoreConfig struct { tiflashRecorder *tiflashrec.TiFlashRecorder `json:"-" toml:"-"` PitrBatchCount uint32 `json:"pitr-batch-count" toml:"pitr-batch-count"` PitrBatchSize uint32 `json:"pitr-batch-size" toml:"pitr-batch-size"` + PitrConcurrency uint32 `json:"-" toml:"-"` // for ebs-based restore FullBackupType FullBackupType `json:"full-backup-type" toml:"full-backup-type"` @@ -206,10 +208,9 @@ func DefineStreamRestoreFlags(command *cobra.Command) { "support TSO or datetime, e.g. '400036290571534337' or '2018-05-11 01:42:23+0800'") command.Flags().String(FlagStreamFullBackupStorage, "", "specify the backup full storage. "+ "fill it if want restore full backup before restore log.") - command.Flags().Uint32(FlagPiTRBatchCount, defaultPiTRBatchCount, "") - command.Flags().Uint32(FlagPiTRBatchSize, defaultPiTRBatchSize, "") - _ = command.Flags().MarkHidden(FlagPiTRBatchCount) - _ = command.Flags().MarkHidden(FlagPiTRBatchSize) + command.Flags().Uint32(FlagPiTRBatchCount, defaultPiTRBatchCount, "specify the batch count to restore log.") + command.Flags().Uint32(FlagPiTRBatchSize, defaultPiTRBatchSize, "specify the batch size to retore log.") + command.Flags().Uint32(FlagPiTRConcurrency, defaultPiTRConcurrency, "specify the concurrency to restore log.") } // ParseStreamRestoreFlags parses the `restore stream` flags from the flag set. @@ -244,6 +245,9 @@ func (cfg *RestoreConfig) ParseStreamRestoreFlags(flags *pflag.FlagSet) error { if cfg.PitrBatchSize, err = flags.GetUint32(FlagPiTRBatchSize); err != nil { return errors.Trace(err) } + if cfg.PitrConcurrency, err = flags.GetUint32(FlagPiTRConcurrency); err != nil { + return errors.Trace(err) + } return nil } @@ -370,10 +374,18 @@ func (cfg *RestoreConfig) Adjust() { } func (cfg *RestoreConfig) adjustRestoreConfigForStreamRestore() { - if cfg.Config.Concurrency == 0 { - log.Info("set restore kv files concurrency", zap.Int("concurrency", defaultRestoreStreamConcurrency)) - cfg.Config.Concurrency = defaultRestoreStreamConcurrency + if cfg.PitrConcurrency == 0 { + cfg.PitrConcurrency = defaultPiTRConcurrency } + if cfg.PitrBatchCount == 0 { + cfg.PitrBatchCount = defaultPiTRBatchCount + } + if cfg.PitrBatchSize == 0 { + cfg.PitrBatchSize = defaultPiTRBatchSize + } + + log.Info("set restore kv files concurrency", zap.Int("concurrency", int(cfg.PitrConcurrency))) + cfg.Config.Concurrency = cfg.PitrConcurrency } func configureRestoreClient(ctx context.Context, client *restore.Client, cfg *RestoreConfig) error { diff --git a/br/pkg/task/restore_test.go b/br/pkg/task/restore_test.go index 94bbcb3c3692c..b13ecf0eccc08 100644 --- a/br/pkg/task/restore_test.go +++ b/br/pkg/task/restore_test.go @@ -63,6 +63,16 @@ func TestConfigureRestoreClient(t *testing.T) { require.True(t, client.IsOnline()) } +func TestAdjustRestoreConfigForStreamRestore(t *testing.T) { + restoreCfg := RestoreConfig{} + + restoreCfg.adjustRestoreConfigForStreamRestore() + require.Equal(t, restoreCfg.PitrBatchCount, uint32(defaultPiTRBatchCount)) + require.Equal(t, restoreCfg.PitrBatchSize, uint32(defaultPiTRBatchSize)) + require.Equal(t, restoreCfg.PitrConcurrency, uint32(defaultPiTRConcurrency)) + require.Equal(t, restoreCfg.Concurrency, restoreCfg.PitrConcurrency) +} + func TestCheckRestoreDBAndTable(t *testing.T) { cases := []struct { cfgSchemas map[string]struct{} diff --git a/br/pkg/utils/BUILD.bazel b/br/pkg/utils/BUILD.bazel index 0ae948d18a779..c3bcc629183d5 100644 --- a/br/pkg/utils/BUILD.bazel +++ b/br/pkg/utils/BUILD.bazel @@ -79,6 +79,7 @@ go_test( ], embed = [":utils"], flaky = True, + shard_count = 20, deps = [ "//br/pkg/errors", "//br/pkg/metautil", diff --git a/br/pkg/utils/key.go b/br/pkg/utils/key.go index 062f4b5aac52d..62d194ca57a2e 100644 --- a/br/pkg/utils/key.go +++ b/br/pkg/utils/key.go @@ -163,7 +163,7 @@ func CloneSlice[T any](s []T) []T { // toClampIn: |_____| |____| |________________| // result: |_____| |_| |______________| // we are assuming the arguments are sorted by the start key and no overlaps. -// you can call CollapseRanges to get key ranges fits this requirements. +// you can call spans.Collapse to get key ranges fits this requirements. // Note: this algorithm is pretty like the `checkIntervalIsSubset`, can we get them together? func IntersectAll(s1 []kv.KeyRange, s2 []kv.KeyRange) []kv.KeyRange { currentClamping := 0 diff --git a/br/tests/lightning_compress/config.toml b/br/tests/lightning_compress/config.toml new file mode 100644 index 0000000000000..000018c5c41d4 --- /dev/null +++ b/br/tests/lightning_compress/config.toml @@ -0,0 +1,18 @@ +[mydumper.csv] +separator = ',' +delimiter = '"' +header = true +not-null = false +null = '\N' +backslash-escape = true +trim-last-separator = false + +[checkpoint] +enable = true +schema = "tidb_lightning_checkpoint_test" +driver = "mysql" +keep-after-success = true + +[tikv-importer] +send-kv-pairs=10 +region-split-size = 1024 diff --git a/br/tests/lightning_compress/data.gzip/compress-schema-create.sql.gz b/br/tests/lightning_compress/data.gzip/compress-schema-create.sql.gz new file mode 100644 index 0000000000000..6571d2a15b507 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress-schema-create.sql.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.empty_strings-schema.sql.gz b/br/tests/lightning_compress/data.gzip/compress.empty_strings-schema.sql.gz new file mode 100644 index 0000000000000..542898561bab1 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.empty_strings-schema.sql.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.empty_strings.000000000.csv.gz b/br/tests/lightning_compress/data.gzip/compress.empty_strings.000000000.csv.gz new file mode 100644 index 0000000000000..bfa13ed67b006 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.empty_strings.000000000.csv.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.escapes-schema.sql.gz b/br/tests/lightning_compress/data.gzip/compress.escapes-schema.sql.gz new file mode 100644 index 0000000000000..bed4b7859ac92 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.escapes-schema.sql.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.escapes.000000000.csv.gz b/br/tests/lightning_compress/data.gzip/compress.escapes.000000000.csv.gz new file mode 100644 index 0000000000000..37028e36d9de8 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.escapes.000000000.csv.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.multi_rows-schema.sql.gz b/br/tests/lightning_compress/data.gzip/compress.multi_rows-schema.sql.gz new file mode 100644 index 0000000000000..328fed9cb3df8 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.multi_rows-schema.sql.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.multi_rows.000000000.csv.gz b/br/tests/lightning_compress/data.gzip/compress.multi_rows.000000000.csv.gz new file mode 100644 index 0000000000000..c732af263d576 Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.multi_rows.000000000.csv.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.threads-schema.sql.gz b/br/tests/lightning_compress/data.gzip/compress.threads-schema.sql.gz new file mode 100644 index 0000000000000..1782675bfc7fe Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.threads-schema.sql.gz differ diff --git a/br/tests/lightning_compress/data.gzip/compress.threads.000000000.csv.gz b/br/tests/lightning_compress/data.gzip/compress.threads.000000000.csv.gz new file mode 100644 index 0000000000000..683eade1cdb9f Binary files /dev/null and b/br/tests/lightning_compress/data.gzip/compress.threads.000000000.csv.gz differ diff --git a/br/tests/lightning_compress/data.snappy/compress-schema-create.sql.snappy b/br/tests/lightning_compress/data.snappy/compress-schema-create.sql.snappy new file mode 100644 index 0000000000000..afa2211c77475 Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress-schema-create.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.empty_strings-schema.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.empty_strings-schema.sql.snappy new file mode 100644 index 0000000000000..cab30d082385a Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.empty_strings-schema.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.empty_strings.000000000.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.empty_strings.000000000.sql.snappy new file mode 100644 index 0000000000000..9c81e8f78f234 Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.empty_strings.000000000.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.escapes-schema.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.escapes-schema.sql.snappy new file mode 100644 index 0000000000000..9e27befa522a0 Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.escapes-schema.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.escapes.000000000.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.escapes.000000000.sql.snappy new file mode 100644 index 0000000000000..1380b47d9881e Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.escapes.000000000.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.multi_rows-schema.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.multi_rows-schema.sql.snappy new file mode 100644 index 0000000000000..5cc0365d1c65d Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.multi_rows-schema.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.multi_rows.000000000.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.multi_rows.000000000.sql.snappy new file mode 100644 index 0000000000000..7f5bf585e106c Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.multi_rows.000000000.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.threads-schema.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.threads-schema.sql.snappy new file mode 100644 index 0000000000000..b1c8b89565bfb Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.threads-schema.sql.snappy differ diff --git a/br/tests/lightning_compress/data.snappy/compress.threads.000000000.sql.snappy b/br/tests/lightning_compress/data.snappy/compress.threads.000000000.sql.snappy new file mode 100644 index 0000000000000..dc7c1ee8adc0b Binary files /dev/null and b/br/tests/lightning_compress/data.snappy/compress.threads.000000000.sql.snappy differ diff --git a/br/tests/lightning_compress/data.zstd/compress-schema-create.sql.zst b/br/tests/lightning_compress/data.zstd/compress-schema-create.sql.zst new file mode 100644 index 0000000000000..12bdbd710973e Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress-schema-create.sql.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.empty_strings-schema.sql.zst b/br/tests/lightning_compress/data.zstd/compress.empty_strings-schema.sql.zst new file mode 100644 index 0000000000000..f9b922954ff3d Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.empty_strings-schema.sql.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.empty_strings.000000000.csv.zst b/br/tests/lightning_compress/data.zstd/compress.empty_strings.000000000.csv.zst new file mode 100644 index 0000000000000..aa89918bb2cee Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.empty_strings.000000000.csv.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.escapes-schema.sql.zst b/br/tests/lightning_compress/data.zstd/compress.escapes-schema.sql.zst new file mode 100644 index 0000000000000..fa4b4e6b3497d Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.escapes-schema.sql.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.escapes.000000000.csv.zst b/br/tests/lightning_compress/data.zstd/compress.escapes.000000000.csv.zst new file mode 100644 index 0000000000000..40994e745bdf3 Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.escapes.000000000.csv.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.multi_rows-schema.sql.zst b/br/tests/lightning_compress/data.zstd/compress.multi_rows-schema.sql.zst new file mode 100644 index 0000000000000..d64a9a4a879d3 Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.multi_rows-schema.sql.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.multi_rows.000000000.csv.zst b/br/tests/lightning_compress/data.zstd/compress.multi_rows.000000000.csv.zst new file mode 100644 index 0000000000000..4db1bea4c69f9 Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.multi_rows.000000000.csv.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.threads-schema.sql.zst b/br/tests/lightning_compress/data.zstd/compress.threads-schema.sql.zst new file mode 100644 index 0000000000000..3a41c8de4816c Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.threads-schema.sql.zst differ diff --git a/br/tests/lightning_compress/data.zstd/compress.threads.000000000.csv.zst b/br/tests/lightning_compress/data.zstd/compress.threads.000000000.csv.zst new file mode 100644 index 0000000000000..13eef0ba83011 Binary files /dev/null and b/br/tests/lightning_compress/data.zstd/compress.threads.000000000.csv.zst differ diff --git a/br/tests/lightning_compress/run.sh b/br/tests/lightning_compress/run.sh new file mode 100755 index 0000000000000..bf48b09b2cccd --- /dev/null +++ b/br/tests/lightning_compress/run.sh @@ -0,0 +1,61 @@ +#!/bin/sh + +set -eu + +for BACKEND in tidb local; do + for compress in gzip snappy zstd; do + if [ "$BACKEND" = 'local' ]; then + check_cluster_version 4 0 0 'local backend' || continue + fi + + # Set minDeliverBytes to a small enough number to only write only 1 row each time + # Set the failpoint to kill the lightning instance as soon as one row is written + PKG="github.com/pingcap/tidb/br/pkg/lightning/restore" + export GO_FAILPOINTS="$PKG/SlowDownWriteRows=sleep(1000);$PKG/FailAfterWriteRows=panic;$PKG/SetMinDeliverBytes=return(1)" + + # Start importing the tables. + run_sql 'DROP DATABASE IF EXISTS compress' + run_sql 'DROP DATABASE IF EXISTS tidb_lightning_checkpoint_test' + set +e + run_lightning --backend $BACKEND -d "tests/$TEST_NAME/data.$compress" --enable-checkpoint=1 2> /dev/null + set -e + + # restart lightning from checkpoint, the second line should be written successfully + export GO_FAILPOINTS= + set +e + run_lightning --backend $BACKEND -d "tests/$TEST_NAME/data.$compress" --enable-checkpoint=1 2> /dev/null + set -e + + run_sql 'SELECT count(*), sum(PROCESSLIST_TIME), sum(THREAD_OS_ID), count(PROCESSLIST_STATE) FROM compress.threads' + check_contains 'count(*): 43' + check_contains 'sum(PROCESSLIST_TIME): 322253' + check_contains 'sum(THREAD_OS_ID): 303775702' + check_contains 'count(PROCESSLIST_STATE): 3' + + run_sql 'SELECT count(*) FROM compress.threads WHERE PROCESSLIST_TIME IS NOT NULL' + check_contains 'count(*): 12' + + run_sql 'SELECT count(*) FROM compress.multi_rows WHERE a="aaaaaaaaaa"' + check_contains 'count(*): 100000' + + run_sql 'SELECT hex(t), j, hex(b) FROM compress.escapes WHERE i = 1' + check_contains 'hex(t): 5C' + check_contains 'j: {"?": []}' + check_contains 'hex(b): FFFFFFFF' + + run_sql 'SELECT hex(t), j, hex(b) FROM compress.escapes WHERE i = 2' + check_contains 'hex(t): 22' + check_contains 'j: "\n\n\n"' + check_contains 'hex(b): 0D0A0D0A' + + run_sql 'SELECT hex(t), j, hex(b) FROM compress.escapes WHERE i = 3' + check_contains 'hex(t): 0A' + check_contains 'j: [",,,"]' + check_contains 'hex(b): 5C2C5C2C' + + run_sql 'SELECT id FROM compress.empty_strings WHERE a = """"' + check_contains 'id: 3' + run_sql 'SELECT id FROM compress.empty_strings WHERE b <> ""' + check_not_contains 'id:' + done +done diff --git a/cmd/explaintest/r/access_path_selection.result b/cmd/explaintest/r/access_path_selection.result index cafdc72269eed..75ede06c71dd3 100644 --- a/cmd/explaintest/r/access_path_selection.result +++ b/cmd/explaintest/r/access_path_selection.result @@ -41,5 +41,5 @@ explain format = 'brief' select count(1) from access_path_selection; id estRows task access object operator info StreamAgg 1.00 root funcs:count(Column#18)->Column#4 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#18 + └─StreamAgg 1.00 cop[tikv] funcs:count(test.access_path_selection._tidb_rowid)->Column#18 └─TableFullScan 10000.00 cop[tikv] table:access_path_selection keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/clustered_index.result b/cmd/explaintest/r/clustered_index.result index bd1824d07edb3..d3e131e6a5a65 100644 --- a/cmd/explaintest/r/clustered_index.result +++ b/cmd/explaintest/r/clustered_index.result @@ -30,25 +30,25 @@ explain select count(*) from with_cluster_index.tbl_0 where col_0 < 5429 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(with_cluster_index.tbl_0.col_0)->Column#8 └─IndexRangeScan_16 798.90 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,5429), keep order:false explain select count(*) from wout_cluster_index.tbl_0 where col_0 < 5429 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(wout_cluster_index.tbl_0.col_0)->Column#9 └─IndexRangeScan_16 798.90 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,5429), keep order:false explain select count(*) from with_cluster_index.tbl_0 where col_0 < 41 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(with_cluster_index.tbl_0.col_0)->Column#8 └─IndexRangeScan_16 41.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,41), keep order:false explain select count(*) from wout_cluster_index.tbl_0 where col_0 < 41 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(wout_cluster_index.tbl_0.col_0)->Column#9 └─IndexRangeScan_16 41.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,41), keep order:false explain select col_14 from with_cluster_index.tbl_2 where col_11 <> '2013-11-01' ; id estRows task access object operator info @@ -109,24 +109,24 @@ explain select count(*) from with_cluster_index.tbl_0 where col_0 <= 0 ; id estRows task access object operator info StreamAgg_16 1.00 root funcs:count(Column#8)->Column#6 └─IndexReader_17 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(with_cluster_index.tbl_0.col_0)->Column#8 └─IndexRangeScan_11 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false explain select count(*) from wout_cluster_index.tbl_0 where col_0 <= 0 ; id estRows task access object operator info StreamAgg_16 1.00 root funcs:count(Column#9)->Column#7 └─IndexReader_17 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(wout_cluster_index.tbl_0.col_0)->Column#9 └─IndexRangeScan_11 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false explain select count(*) from with_cluster_index.tbl_0 where col_0 >= 803163 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(with_cluster_index.tbl_0.col_0)->Column#8 └─IndexRangeScan_16 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false explain select count(*) from wout_cluster_index.tbl_0 where col_0 >= 803163 ; id estRows task access object operator info StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7 └─IndexReader_18 1.00 root index:StreamAgg_9 - └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9 + └─StreamAgg_9 1.00 cop[tikv] funcs:count(wout_cluster_index.tbl_0.col_0)->Column#9 └─IndexRangeScan_16 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false set @@tidb_enable_outer_join_reorder=false; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 6e367f1f1a0dc..bf0b3e4c2884b 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -218,14 +218,14 @@ id estRows task access object operator info Projection 1.00 root 1->Column#6 └─StreamAgg 1.00 root funcs:count(Column#14)->Column#9 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#14 + └─StreamAgg 1.00 cop[tikv] funcs:count(test.t1.c1)->Column#14 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select max(c2), count(c3) as m from t1) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#6 └─StreamAgg 1.00 root funcs:count(Column#13)->Column#8 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#13 + └─StreamAgg 1.00 cop[tikv] funcs:count(test.t1.c1)->Column#13 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select count(c2) from t1 group by c3) k; id estRows task access object operator info diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 1870b6bcd8d5d..7e02e6255834f 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -126,7 +126,7 @@ Sort_11 5098.44 root test.t1.c1 ├─StreamAgg_23(Build) 1.00 root funcs:min(Column#9)->Column#10, funcs:sum(0)->Column#11, funcs:count(1)->Column#12 │ └─StreamAgg_43 1.00 root funcs:count(Column#25)->Column#9 │ └─IndexReader_44 1.00 root index:StreamAgg_27 - │ └─StreamAgg_27 1.00 cop[tikv] funcs:count(1)->Column#25 + │ └─StreamAgg_27 1.00 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#25 │ └─IndexFullScan_41 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo └─IndexMerge_21(Probe) 2825.66 root ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo @@ -148,7 +148,7 @@ Sort_11 5098.44 root test.t1.c1 ├─StreamAgg_23(Build) 1.00 root funcs:min(Column#9)->Column#10, funcs:sum(0)->Column#11, funcs:count(1)->Column#12 │ └─StreamAgg_43 1.00 root funcs:count(Column#25)->Column#9 │ └─IndexReader_44 1.00 root index:StreamAgg_27 - │ └─StreamAgg_27 1.00 cop[tikv] funcs:count(1)->Column#25 + │ └─StreamAgg_27 1.00 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#25 │ └─IndexFullScan_41 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo └─IndexMerge_21(Probe) 2825.66 root ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo @@ -170,7 +170,7 @@ Sort_11 5542.21 root test.t1.c1 ├─StreamAgg_22(Build) 1.00 root funcs:max(Column#9)->Column#10, funcs:sum(0)->Column#11, funcs:count(1)->Column#12 │ └─StreamAgg_42 1.00 root funcs:count(Column#25)->Column#9 │ └─IndexReader_43 1.00 root index:StreamAgg_26 - │ └─StreamAgg_26 1.00 cop[tikv] funcs:count(1)->Column#25 + │ └─StreamAgg_26 1.00 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#25 │ └─IndexFullScan_40 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo └─IndexMerge_20(Probe) 5542.21 root ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo @@ -192,7 +192,7 @@ Sort_39 5542.21 root test.t1.c1 ├─StreamAgg_50(Build) 1.00 root funcs:max(Column#13)->Column#14, funcs:sum(0)->Column#15, funcs:count(1)->Column#16 │ └─StreamAgg_70 1.00 root funcs:count(Column#38)->Column#13 │ └─IndexReader_71 1.00 root index:StreamAgg_54 - │ └─StreamAgg_54 1.00 cop[tikv] funcs:count(1)->Column#38 + │ └─StreamAgg_54 1.00 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#38 │ └─IndexFullScan_68 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo └─IndexMerge_48(Probe) 5542.21 root ├─IndexRangeScan_45(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo diff --git a/ddl/column.go b/ddl/column.go index 96ab745377a92..9f5174737915a 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1543,7 +1543,7 @@ func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, return nil } idAcc := m.GetAutoIDAccessors(dbInfo.ID, tblInfo.ID) - err := checkNewAutoRandomBits(idAcc, oldCol, newCol, newAutoRandBits, tblInfo.AutoRandomRangeBits, tblInfo.Version) + err := checkNewAutoRandomBits(idAcc, oldCol, newCol, newAutoRandBits, tblInfo.AutoRandomRangeBits, tblInfo.SepAutoInc()) if err != nil { return err } @@ -1552,13 +1552,17 @@ func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, // checkNewAutoRandomBits checks whether the new auto_random bits number can cause overflow. func checkNewAutoRandomBits(idAccessors meta.AutoIDAccessors, oldCol *model.ColumnInfo, - newCol *model.ColumnInfo, newShardBits, newRangeBits uint64, tblVer uint16) error { + newCol *model.ColumnInfo, newShardBits, newRangeBits uint64, sepAutoInc bool) error { shardFmt := autoid.NewShardIDFormat(&newCol.FieldType, newShardBits, newRangeBits) idAcc := idAccessors.RandomID() convertedFromAutoInc := mysql.HasAutoIncrementFlag(oldCol.GetFlag()) if convertedFromAutoInc { - idAcc = idAccessors.IncrementID(tblVer) + if sepAutoInc { + idAcc = idAccessors.IncrementID(model.TableInfoVersion5) + } else { + idAcc = idAccessors.RowID() + } } // Generate a new auto ID first to prevent concurrent update in DML. _, err := idAcc.Inc(1) diff --git a/ddl/concurrentddltest/BUILD.bazel b/ddl/concurrentddltest/BUILD.bazel index b70619f27da1a..d5acc141896c5 100644 --- a/ddl/concurrentddltest/BUILD.bazel +++ b/ddl/concurrentddltest/BUILD.bazel @@ -9,6 +9,7 @@ go_test( ], flaky = True, race = "on", + shard_count = 2, deps = [ "//config", "//ddl", diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 6de1376e2b75b..cf11036a9935e 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1719,8 +1719,6 @@ func TestCreateUniqueExpressionIndex(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - // TODO: Will check why tidb_ddl_enable_fast_reorg could not default be on in another PR. - tk.MustExec("set global tidb_ddl_enable_fast_reorg = 0;") tk.MustExec("create table t(a int default 0, b int default 0)") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (4, 4)") @@ -1740,8 +1738,6 @@ func TestCreateUniqueExpressionIndex(t *testing.T) { if checkErr != nil { return } - err := originalCallback.OnChanged(nil) - require.NoError(t, err) switch job.SchemaState { case model.StateDeleteOnly: for _, sql := range stateDeleteOnlySQLs { diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index b25aa87f2fce8..6d2e5cf39c468 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2926,94 +2926,84 @@ func TestAutoIncrementForce(t *testing.T) { return gid } - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase _tidb_row_id. - tk.MustExec("create table t (a int)" + str) - tk.MustExec("alter table t force auto_increment = 2;") - tk.MustExec("insert into t values (1),(2);") - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t force auto_increment = 1;") - require.Equal(t, uint64(1), getNextGlobalID()) - // inserting new rows can overwrite the existing data. - tk.MustExec("insert into t values (3);") - require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) - tk.MustExec("drop table if exists t;") - } + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int)") + tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(1), getNextGlobalID()) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase auto_increment. - tk.MustExec("create table t (a int primary key auto_increment, b int)" + str) - tk.MustExec("insert into t values (1, 1);") - tk.MustExec("insert into t values (100000000, 1);") - tk.MustExec("delete from t where a = 100000000;") - require.Greater(t, getNextGlobalID(), uint64(100000000)) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t(b) values (2);") - tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) - tk.MustExec("drop table if exists t;") - } + // Rebase auto_increment. + tk.MustExec("create table t (a int primary key auto_increment, b int)") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (100000000, 1);") + tk.MustExec("delete from t where a = 100000000;") + require.Greater(t, getNextGlobalID(), uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase auto_random. - tk.MustExec("create table t (a bigint primary key auto_random(5))" + str) - tk.MustExec("insert into t values ();") - tk.MustExec("set @@allow_auto_random_explicit_insert = true") - tk.MustExec("insert into t values (100000000);") - tk.MustExec("delete from t where a = 100000000;") - require.Greater(t, getNextGlobalID(), uint64(100000000)) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t force auto_random_base = 2;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t values ();") - tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) - tk.MustExec("drop table if exists t;") - } + // Rebase auto_random. + tk.MustExec("create table t (a bigint primary key auto_random(5))") + tk.MustExec("insert into t values ();") + tk.MustExec("set @@allow_auto_random_explicit_insert = true") + tk.MustExec("insert into t values (100000000);") + tk.MustExec("delete from t where a = 100000000;") + require.Greater(t, getNextGlobalID(), uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_random_base = 2;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t values ();") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Change next global ID. - tk.MustExec("create table t (a bigint primary key auto_increment)" + str) - tk.MustExec("insert into t values (1);") - bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} - lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) - for _, b := range bases { - fmt.Println("execute alter table force increment to ==", b) - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) - } + // Change next global ID. + tk.MustExec("create table t (a bigint primary key auto_increment)") + tk.MustExec("insert into t values (1);") + bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} + lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) + for _, b := range bases { + fmt.Println("execute alter table force increment to ==", b) + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + require.Equal(t, b, getNextGlobalID()) + } + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) + // Force alter unsigned int auto_increment column. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint unsigned primary key auto_increment)") + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + require.Equal(t, b, getNextGlobalID()) tk.MustExec("insert into t values ();") - tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) - // Force alter unsigned int auto_increment column. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a bigint unsigned primary key auto_increment)" + str) - for _, b := range bases { - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) - tk.MustExec("insert into t values ();") - tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) - tk.MustExec("delete from t;") - } - tk.MustExec("drop table if exists t;") + tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) + tk.MustExec("delete from t;") } + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Force alter with @@auto_increment_increment and @@auto_increment_offset. - tk.MustExec("create table t(a int key auto_increment)" + str) - tk.MustExec("set @@auto_increment_offset=2;") - tk.MustExec("set @@auto_increment_increment = 11;") - tk.MustExec("insert into t values (500);") - tk.MustExec("alter table t force auto_increment=100;") - tk.MustExec("insert into t values (), ();") - tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) - tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) - tk.MustExec("drop table if exists t;") - } + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + tk.MustExec("create table t(a int key auto_increment)") + tk.MustExec("set @@auto_increment_offset=2;") + tk.MustExec("set @@auto_increment_increment = 11;") + tk.MustExec("insert into t values (500);") + tk.MustExec("alter table t force auto_increment=100;") + tk.MustExec("insert into t values (), ();") + tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) + tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) + tk.MustExec("drop table if exists t;") // Check for warning in case we can't set the auto_increment to the desired value tk.MustExec("create table t(a int primary key auto_increment)") @@ -3033,6 +3023,147 @@ func TestAutoIncrementForce(t *testing.T) { tk.MustExec("drop table t") } +func TestAutoIncrementForceAutoIDCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists auto_inc_force;") + tk.MustExec("create database auto_inc_force;") + tk.MustExec("use auto_inc_force;") + getNextGlobalID := func() uint64 { + gidStr := tk.MustQuery("show table t next_row_id").Rows()[0][3] + gid, err := strconv.ParseUint(gidStr.(string), 10, 64) + require.NoError(t, err) + return gid + } + + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int) AUTO_ID_CACHE 1") + tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 1", "2 2")) + // Cannot set next global ID to 0. + tk.MustExec("alter table t force auto_increment = 0;") + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(3), getNextGlobalID()) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + tk.MustExec("insert into t values (3);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 1", "2 2", "3 3", "3 4")) + tk.MustExec("drop table if exists t;") + + // When AUTO_ID_CACHE is 1, row id and auto increment id use separate allocator, so the behaviour differs. + // "Alter table t force auto_increment" has no effect on row id. + tk.MustExec("create table t (a int) AUTO_ID_CACHE 1") + tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 1", "2 2")) + // Cannot set next global ID to 0. + tk.MustExec("alter table t force auto_increment = 0;") + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(3), getNextGlobalID()) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + tk.MustExec("insert into t values (3);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 1", "2 2", "3 3", "3 4")) + tk.MustExec("drop table if exists t;") + + // Rebase auto_increment. + tk.MustExec("create table t (a int primary key auto_increment, b int) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (100000000, 1);") + tk.MustExec("delete from t where a = 100000000;") + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + "auto_inc_force t a 100000001 AUTO_INCREMENT", + )) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + "auto_inc_force t a 2 AUTO_INCREMENT", + )) + + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table if exists t;") + + // Rebase auto_random. + tk.MustExec("create table t (a bigint primary key auto_random(5)) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values ();") + tk.MustExec("set @@allow_auto_random_explicit_insert = true") + tk.MustExec("insert into t values (100000000);") + tk.MustExec("delete from t where a = 100000000;") + require.Greater(t, getNextGlobalID(), uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_random_base = 2;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t values ();") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + tk.MustExec("drop table if exists t;") + + // Change next global ID. + tk.MustExec("create table t (a bigint primary key auto_increment) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values (1);") + bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} + lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) + for _, b := range bases { + fmt.Println("execute alter table force increment to ==", b) + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + fmt.Sprintf("auto_inc_force t a %d AUTO_INCREMENT", b), + )) + } + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) + // Force alter unsigned int auto_increment column. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint unsigned primary key auto_increment) AUTO_ID_CACHE 1") + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + fmt.Sprintf("auto_inc_force t a %d AUTO_INCREMENT", b), + )) + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) + tk.MustExec("delete from t;") + } + tk.MustExec("drop table if exists t;") + + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + tk.MustExec("create table t(a int key auto_increment) AUTO_ID_CACHE 1") + tk.MustExec("set @@auto_increment_offset=2;") + tk.MustExec("set @@auto_increment_increment = 11;") + tk.MustExec("insert into t values (500);") + tk.MustExec("alter table t force auto_increment=100;") + tk.MustExec("insert into t values (), ();") + tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) + tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) + tk.MustExec("drop table if exists t;") + + // Check for warning in case we can't set the auto_increment to the desired value + tk.MustExec("create table t(a int primary key auto_increment) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values (200)") + tk.MustQuery("show create table t").Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL AUTO_INCREMENT,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=1 */")) + tk.MustExec("alter table t auto_increment=100;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Can't reset AUTO_INCREMENT to 100 without FORCE option, using 201 instead")) + tk.MustExec("insert into t values ()") + tk.MustQuery("select * from t").Check(testkit.Rows("200", "211")) + tk.MustQuery("show create table t").Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL AUTO_INCREMENT,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=1 */")) + tk.MustExec("drop table t") +} + func TestIssue20490(t *testing.T) { store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index e5ad2aa2bbfec..d714ed716f9f9 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -1409,7 +1409,7 @@ func TestAlterTableDropPartitionByList(t *testing.T) { );`) tk.MustExec(`insert into t values (1),(3),(5),(null)`) tk.MustExec(`alter table t drop partition p1`) - tk.MustQuery("select * from t").Sort().Check(testkit.Rows("1", "5", "")) + tk.MustQuery("select * from t order by id").Check(testkit.Rows("", "1", "5")) ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) diff --git a/ddl/db_test.go b/ddl/db_test.go index ce2ea307e97af..5990df4634a68 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -898,11 +898,11 @@ func TestAutoIncrementIDOnTemporaryTable(t *testing.T) { tk.MustExec("drop table if exists global_temp_auto_id") tk.MustExec("create global temporary table global_temp_auto_id(id int primary key auto_increment) on commit delete rows") tk.MustExec("begin") - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 1 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 1 _TIDB_ROWID")) tk.MustExec("insert into global_temp_auto_id value(null)") tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("1")) tk.MustQuery("select id from global_temp_auto_id").Check(testkit.Rows("1")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 2 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 2 _TIDB_ROWID")) tk.MustExec("commit") tk.MustExec("drop table global_temp_auto_id") @@ -914,12 +914,12 @@ func TestAutoIncrementIDOnTemporaryTable(t *testing.T) { " `id` int(11) NOT NULL AUTO_INCREMENT,\n" + " PRIMARY KEY (`id`) /*T![clustered_index] CLUSTERED */\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=100 ON COMMIT DELETE ROWS")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 100 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 100 _TIDB_ROWID")) tk.MustExec("begin") tk.MustExec("insert into global_temp_auto_id value(null)") tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("100")) tk.MustQuery("select id from global_temp_auto_id").Check(testkit.Rows("100")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 101 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 101 _TIDB_ROWID")) tk.MustExec("commit") } tk.MustExec("drop table global_temp_auto_id") diff --git a/ddl/ddl.go b/ddl/ddl.go index af8a0ca67a8d5..6e6488ca0d1c9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1217,8 +1217,10 @@ func (d *ddl) SwitchConcurrentDDL(toConcurrentDDL bool) error { } if err == nil { variable.EnableConcurrentDDL.Store(toConcurrentDDL) + logutil.BgLogger().Info("[ddl] SwitchConcurrentDDL", zap.Bool("toConcurrentDDL", toConcurrentDDL)) + } else { + logutil.BgLogger().Warn("[ddl] SwitchConcurrentDDL", zap.Bool("toConcurrentDDL", toConcurrentDDL), zap.Error(err)) } - logutil.BgLogger().Info("[ddl] SwitchConcurrentDDL", zap.Bool("toConcurrentDDL", toConcurrentDDL), zap.Error(err)) return err } @@ -1279,9 +1281,10 @@ func (d *ddl) SwitchMDL(enable bool) error { return err }) if err != nil { + logutil.BgLogger().Warn("[ddl] switch metadata lock feature", zap.Bool("enable", enable), zap.Error(err)) return err } - logutil.BgLogger().Info("[ddl] switch metadata lock feature", zap.Bool("enable", enable), zap.Error(err)) + logutil.BgLogger().Info("[ddl] switch metadata lock feature", zap.Bool("enable", enable)) return nil } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 505d413fff8c2..1816e0d65891d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2152,7 +2152,7 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T // checkTableInfoValid uses to check table info valid. This is used to validate table info. func checkTableInfoValid(tblInfo *model.TableInfo) error { - _, err := tables.TableFromMeta(nil, tblInfo) + _, err := tables.TableFromMeta(autoid.NewAllocators(false), tblInfo) if err != nil { return err } @@ -2493,7 +2493,13 @@ func (d *ddl) createTableWithInfoPost( // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. newEnd := tbInfo.AutoIncID - 1 - if err = d.handleAutoIncID(tbInfo, schemaID, newEnd, autoid.RowIDAllocType); err != nil { + var allocType autoid.AllocatorType + if tbInfo.SepAutoInc() { + allocType = autoid.AutoIncrementType + } else { + allocType = autoid.RowIDAllocType + } + if err = d.handleAutoIncID(tbInfo, schemaID, newEnd, allocType); err != nil { return errors.Trace(err) } } @@ -3361,7 +3367,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast } err = d.ShardRowID(sctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.RowIDAllocType, opt.BoolValue) + err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.AutoIncrementType, opt.BoolValue) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. @@ -3492,6 +3498,10 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 actionType = model.ActionRebaseAutoRandomBase case autoid.RowIDAllocType: actionType = model.ActionRebaseAutoID + case autoid.AutoIncrementType: + actionType = model.ActionRebaseAutoID + default: + panic(fmt.Sprintf("unimplemented rebase autoid type %s", tp)) } if !force { @@ -5203,6 +5213,11 @@ func (d *ddl) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident, new if err != nil { return errors.Trace(err) } + tbInfo := tb.Meta() + if (newCache == 1 && tbInfo.AutoIdCache != 1) || + (newCache != 1 && tbInfo.AutoIdCache == 1) { + return fmt.Errorf("Can't Alter AUTO_ID_CACHE between 1 and non-1, the underlying implementation is different") + } job := &model.Job{ SchemaID: schema.ID, diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 1a13feb81e079..850caca29e404 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -1232,7 +1232,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: - ver, err = onRebaseRowIDType(d, t, job) + ver, err = onRebaseAutoIncrementIDType(d, t, job) case model.ActionRebaseAutoRandomBase: ver, err = onRebaseAutoRandomType(d, t, job) case model.ActionRenameTable: diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 85352cb6d08d1..f9dcc99154dc5 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -37,8 +37,6 @@ func TestIndexChange(t *testing.T) { ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - // TODO: Will check why tidb_ddl_enable_fast_reorg could not default be on in another PR. - tk.MustExec("set global tidb_ddl_enable_fast_reorg = 0;") tk.MustExec("create table t (c1 int primary key, c2 int)") tk.MustExec("insert t values (1, 1), (2, 2), (3, 3);") @@ -221,6 +219,7 @@ func checkAddWriteOnlyForAddIndex(ctx sessionctx.Context, delOnlyTbl, writeOnlyT } func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table.Table) error { + var err1 error // WriteOnlyTable: insert t values (6, 6) err := sessiontxn.NewTxn(context.Background(), ctx) if err != nil { @@ -231,7 +230,11 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table return errors.Trace(err) } err = checkIndexExists(ctx, publicTbl, 6, 6, true) - if err != nil { + if ddl.IsEnableFastReorg() { + // Need check temp index also. + err1 = checkIndexExists(ctx, writeTbl, 6, 6, true) + } + if err != nil && err1 != nil { return errors.Trace(err) } // PublicTable: insert t values (7, 7) @@ -250,10 +253,18 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table return errors.Trace(err) } err = checkIndexExists(ctx, publicTbl, 5, 7, true) - if err != nil { + if ddl.IsEnableFastReorg() { + // Need check temp index also. + err1 = checkIndexExists(ctx, writeTbl, 5, 7, true) + } + if err != nil && err1 != nil { return errors.Trace(err) } - err = checkIndexExists(ctx, publicTbl, 7, 7, false) + if ddl.IsEnableFastReorg() { + err = checkIndexExists(ctx, writeTbl, 7, 7, false) + } else { + err = checkIndexExists(ctx, publicTbl, 7, 7, false) + } if err != nil { return errors.Trace(err) } @@ -283,7 +294,11 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table idxVal := row[1].GetInt64() handle := row[0].GetInt64() err = checkIndexExists(ctx, publicTbl, idxVal, handle, true) - if err != nil { + if ddl.IsEnableFastReorg() { + // Need check temp index also. + err1 = checkIndexExists(ctx, writeTbl, idxVal, handle, true) + } + if err != nil && err1 != nil { return errors.Trace(err) } } diff --git a/ddl/index_merge_tmp_test.go b/ddl/index_merge_tmp_test.go index 8dae95e590438..389339ac15ad4 100644 --- a/ddl/index_merge_tmp_test.go +++ b/ddl/index_merge_tmp_test.go @@ -259,3 +259,75 @@ func TestPessimisticAmendIncompatibleWithFastReorg(t *testing.T) { tk.MustGetErrMsg("set @@tidb_enable_amend_pessimistic_txn = 1;", "amend pessimistic transactions is not compatible with tidb_ddl_enable_fast_reorg") } + +// TestCreateUniqueIndexKeyExist this case will test below things: +// Create one unique index idx((a*b+1)); +// insert (0, 6) and delete it; +// insert (0, 9), it should be successful; +// Should check temp key exist and skip deleted mark +// The error returned below: +// Error: Received unexpected error: +// +// [kv:1062]Duplicate entry '1' for key 't.idx' +func TestCreateUniqueIndexKeyExist(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int default 0, b int default 0)") + tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (4, 4)") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + stateDeleteOnlySQLs := []string{"insert into t values (5, 5)", "begin pessimistic;", "insert into t select * from t", "rollback", "insert into t set b = 6", "update t set b = 7 where a = 1", "delete from t where b = 4"} + + // If waitReorg timeout, the worker may enter writeReorg more than 2 times. + reorgTime := 0 + d := dom.DDL() + originalCallback := d.GetHook() + defer d.SetHook(originalCallback) + callback := &ddl.TestDDLCallback{} + onJobUpdatedExportedFunc := func(job *model.Job) { + if t.Failed() { + return + } + var err error + switch job.SchemaState { + case model.StateDeleteOnly: + for _, sql := range stateDeleteOnlySQLs { + _, err = tk1.Exec(sql) + assert.NoError(t, err) + } + // (1, 7), (2, 2), (3, 3), (5, 5), (0, 6) + case model.StateWriteOnly: + _, err = tk1.Exec("insert into t values (8, 8)") + assert.NoError(t, err) + _, err = tk1.Exec("update t set b = 7 where a = 2") + assert.NoError(t, err) + _, err = tk1.Exec("delete from t where b = 3") + assert.NoError(t, err) + // (1, 7), (2, 7), (5, 5), (0, 6), (8, 8) + case model.StateWriteReorganization: + if reorgTime < 1 { + reorgTime++ + } else { + return + } + _, err = tk1.Exec("insert into t values (10, 10)") + assert.NoError(t, err) + _, err = tk1.Exec("delete from t where b = 6") + assert.NoError(t, err) + _, err = tk1.Exec("insert into t set b = 9") + assert.NoError(t, err) + _, err = tk1.Exec("update t set b = 7 where a = 5") + assert.NoError(t, err) + // (1, 7), (2, 7), (5, 7), (8, 8), (10, 10), (0, 9) + } + } + callback.OnJobUpdatedExported.Store(&onJobUpdatedExportedFunc) + d.SetHook(callback) + tk.MustExec("alter table t add unique index idx((a*b+1))") + tk.MustExec("admin check table t") + tk.MustQuery("select * from t order by a, b").Check(testkit.Rows("0 9", "1 7", "2 7", "5 7", "8 8", "10 10")) +} diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index 6f4a0012cdfff..64bdf77d55707 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -1105,3 +1105,16 @@ func TestMDLRenameTable(t *testing.T) { tk.MustGetErrCode("select * from test2.t1;", mysql.ErrNoSuchTable) tk.MustExec("commit") } + +func TestMDLPrepareFail(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int);") + _, _, _, err := tk.Session().PrepareStmt("select b from t") + require.Error(t, err) + + tk2.MustExec("alter table test.t add column c int") +} diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index f302737d60c3b..0f8aeca87802c 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -1177,7 +1177,7 @@ func TestMultiSchemaChangeUnsupportedType(t *testing.T) { tk.MustExec("use test;") tk.MustExec("create table t (a int, b int);") - tk.MustGetErrMsg("alter table t add column c int, auto_id_cache = 1;", + tk.MustGetErrMsg("alter table t add column c int, auto_id_cache = 10;", "[ddl:8200]Unsupported multi schema change for modify auto id cache") } diff --git a/ddl/partition.go b/ddl/partition.go index a8947d091bfc5..cf4bd7aed962f 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -2077,7 +2077,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo failpoint.Return(ver, err) } sess := newSession(se) - _, err = sess.execute(context.Background(), "insert into test.pt values (40000000)", "exchange_partition_test") + _, err = sess.execute(context.Background(), "insert ignore into test.pt values (40000000)", "exchange_partition_test") if err != nil { failpoint.Return(ver, err) } diff --git a/ddl/reorg.go b/ddl/reorg.go index d8b31916fba37..a03cf417177dc 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -234,7 +234,12 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo return dbterror.ErrCancelledDDLJob } rowCount, _, _ := rc.getRowCountAndKey() - logutil.BgLogger().Info("[ddl] run reorg job done", zap.Int64("handled rows", rowCount), zap.Error(err)) + if err != nil { + logutil.BgLogger().Warn("[ddl] run reorg job done", zap.Int64("handled rows", rowCount), zap.Error(err)) + } else { + logutil.BgLogger().Info("[ddl] run reorg job done", zap.Int64("handled rows", rowCount)) + } + job.SetRowCount(rowCount) // Update a job's warnings. diff --git a/ddl/table.go b/ddl/table.go index a6e2d93b5ac7e..9e6fab762d3c5 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -812,8 +812,8 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, nil } -func onRebaseRowIDType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - return onRebaseAutoID(d, d.store, t, job, autoid.RowIDAllocType) +func onRebaseAutoIncrementIDType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(d, d.store, t, job, autoid.AutoIncrementType) } func onRebaseAutoRandomType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { @@ -862,7 +862,7 @@ func onRebaseAutoID(d *ddlCtx, store kv.Storage, t *meta.Meta, job *model.Job, t newBase = newBaseTemp } - if tp == autoid.RowIDAllocType { + if tp == autoid.AutoIncrementType { tblInfo.AutoIncID = newBase } else { tblInfo.AutoRandID = newBase diff --git a/ddl/table_test.go b/ddl/table_test.go index 9641c24eb0f19..e0e1c45e3b0ef 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -160,7 +160,7 @@ func testGetTableWithError(store kv.Storage, schemaID, tableID int64) (table.Tab return nil, errors.New("table not found") } alloc := autoid.NewAllocator(store, schemaID, tblInfo.ID, false, autoid.RowIDAllocType) - tbl, err := table.TableFromMeta(autoid.NewAllocators(alloc), tblInfo) + tbl, err := table.TableFromMeta(autoid.NewAllocators(false, alloc), tblInfo) if err != nil { return nil, errors.Trace(err) } diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 52aa62ba112fa..bf5593596a18b 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -107,7 +107,8 @@ func TestSelectWithRuntimeStats(t *testing.T) { } func TestSelectResultRuntimeStats(t *testing.T) { - basic := &execdetails.BasicRuntimeStats{} + stmtStats := execdetails.NewRuntimeStatsColl(nil) + basic := stmtStats.GetBasicRuntimeStats(1) basic.Record(time.Second, 20) s1 := &selectResultRuntimeStats{ copRespTime: []time.Duration{time.Second, time.Millisecond}, @@ -120,8 +121,6 @@ func TestSelectResultRuntimeStats(t *testing.T) { } s2 := *s1 - stmtStats := execdetails.NewRuntimeStatsColl(nil) - stmtStats.RegisterStats(1, basic) stmtStats.RegisterStats(1, s1) stmtStats.RegisterStats(1, &s2) stats := stmtStats.GetRootStats(1) @@ -136,7 +135,7 @@ func TestSelectResultRuntimeStats(t *testing.T) { } stmtStats.RegisterStats(2, s1) stats = stmtStats.GetRootStats(2) - expect = "cop_task: {num: 2, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 1s, tot_wait: 1s, rpc_num: 1, rpc_time: 1s, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15}, backoff{RegionMiss: 1ms}" + expect = "time:0s, loops:0, cop_task: {num: 2, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 1s, tot_wait: 1s, rpc_num: 1, rpc_time: 1s, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15}, backoff{RegionMiss: 1ms}" require.Equal(t, expect, stats.String()) // Test for idempotence. require.Equal(t, expect, stats.String()) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 4a8b3ddfeab13..a293c4d10963e 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -20,7 +20,6 @@ import ( "sort" "sync/atomic" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/ddl/placement" @@ -71,6 +70,9 @@ func (builder *RequestBuilder) Build() (*kv.Request, error) { if err != nil { builder.err = err } + if builder.Request.KeyRanges == nil { + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(nil) + } return &builder.Request, builder.err } @@ -86,7 +88,7 @@ func (builder *RequestBuilder) SetMemTracker(tracker *memory.Tracker) *RequestBu // br refers it, so have to keep it. func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges = TableRangesToKVRanges(tid, tableRanges, fb) + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(TableRangesToKVRanges(tid, tableRanges, fb)) } return builder } @@ -112,7 +114,9 @@ func (builder *RequestBuilder) SetIndexRangesForTables(sc *stmtctx.StatementCont // SetHandleRanges sets "KeyRanges" for "kv.Request" by converting table handle range // "ranges" to "KeyRanges" firstly. func (builder *RequestBuilder) SetHandleRanges(sc *stmtctx.StatementContext, tid int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { - return builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges, fb) + builder = builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges, fb) + builder.err = builder.Request.KeyRanges.SetToNonPartitioned() + return builder } // SetHandleRangesForTables sets "KeyRanges" for "kv.Request" by converting table handle range @@ -127,14 +131,17 @@ func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementCon // SetTableHandles sets "KeyRanges" for "kv.Request" by converting table handles // "handles" to "KeyRanges" firstly. func (builder *RequestBuilder) SetTableHandles(tid int64, handles []kv.Handle) *RequestBuilder { - builder.Request.KeyRanges, builder.FixedRowCountHint = TableHandlesToKVRanges(tid, handles) + var keyRanges []kv.KeyRange + keyRanges, builder.FixedRowCountHint = TableHandlesToKVRanges(tid, handles) + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(keyRanges) return builder } // SetPartitionsAndHandles sets "KeyRanges" for "kv.Request" by converting ParitionHandles to KeyRanges. // handles in slice must be kv.PartitionHandle. func (builder *RequestBuilder) SetPartitionsAndHandles(handles []kv.Handle) *RequestBuilder { - builder.Request.KeyRanges = PartitionHandlesToKVRanges(handles) + keyRanges := PartitionHandlesToKVRanges(handles) + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(keyRanges) return builder } @@ -183,10 +190,22 @@ func (builder *RequestBuilder) SetChecksumRequest(checksum *tipb.ChecksumRequest // SetKeyRanges sets "KeyRanges" for "kv.Request". func (builder *RequestBuilder) SetKeyRanges(keyRanges []kv.KeyRange) *RequestBuilder { + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(keyRanges) + return builder +} + +// SetWrappedKeyRanges sets "KeyRanges" for "kv.Request". +func (builder *RequestBuilder) SetWrappedKeyRanges(keyRanges *kv.KeyRanges) *RequestBuilder { builder.Request.KeyRanges = keyRanges return builder } +// SetPartitionKeyRanges sets the "KeyRanges" for "kv.Request" on partitioned table cases. +func (builder *RequestBuilder) SetPartitionKeyRanges(keyRanges [][]kv.KeyRange) *RequestBuilder { + builder.Request.KeyRanges = kv.NewPartitionedKeyRanges(keyRanges) + return builder +} + // SetStartTS sets "StartTS" for "kv.Request". func (builder *RequestBuilder) SetStartTS(startTS uint64) *RequestBuilder { builder.Request.StartTs = startTS @@ -318,13 +337,12 @@ func (builder *RequestBuilder) verifyTxnScope() error { return nil } visitPhysicalTableID := make(map[int64]struct{}) - for _, keyRange := range builder.Request.KeyRanges { - tableID := tablecodec.DecodeTableID(keyRange.StartKey) - if tableID > 0 { - visitPhysicalTableID[tableID] = struct{}{} - } else { - return errors.New("requestBuilder can't decode tableID from keyRange") - } + tids, err := tablecodec.VerifyTableIDForRanges(builder.Request.KeyRanges) + if err != nil { + return err + } + for _, tid := range tids { + visitPhysicalTableID[tid] = struct{}{} } for phyTableID := range visitPhysicalTableID { @@ -376,7 +394,7 @@ func (builder *RequestBuilder) SetClosestReplicaReadAdjuster(chkFn kv.CoprReques } // TableHandleRangesToKVRanges convert table handle ranges to "KeyRanges" for multiple tables. -func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { +func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { if !isCommonHandle { return tablesRangesToKVRanges(tid, ranges, fb), nil } @@ -387,14 +405,18 @@ func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCo // Note this function should not be exported, but currently // br refers to it, so have to keep it. func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange { - return tablesRangesToKVRanges([]int64{tid}, ranges, fb) + if len(ranges) == 0 { + return []kv.KeyRange{} + } + return tablesRangesToKVRanges([]int64{tid}, ranges, fb).FirstPartitionRange() } // tablesRangesToKVRanges converts table ranges to "KeyRange". -func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange { +func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) *kv.KeyRanges { if fb == nil || fb.Hist == nil { return tableRangesToKVRangesWithoutSplit(tids, ranges) } + // The following codes are deprecated since the feedback is deprecated. krs := make([]kv.KeyRange, 0, len(ranges)) feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { @@ -420,20 +442,23 @@ func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics } } fb.StoreRanges(feedbackRanges) - return krs + return kv.NewNonParitionedKeyRanges(krs) } -func tableRangesToKVRangesWithoutSplit(tids []int64, ranges []*ranger.Range) []kv.KeyRange { - krs := make([]kv.KeyRange, 0, len(ranges)*len(tids)) +func tableRangesToKVRangesWithoutSplit(tids []int64, ranges []*ranger.Range) *kv.KeyRanges { + krs := make([][]kv.KeyRange, len(tids)) + for i := range krs { + krs[i] = make([]kv.KeyRange, 0, len(ranges)) + } for _, ran := range ranges { low, high := encodeHandleKey(ran) - for _, tid := range tids { + for i, tid := range tids { startKey := tablecodec.EncodeRowKey(tid, low) endKey := tablecodec.EncodeRowKey(tid, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) + krs[i] = append(krs[i], kv.KeyRange{StartKey: startKey, EndKey: endKey}) } } - return krs + return kv.NewPartitionedKeyRanges(krs) } func encodeHandleKey(ran *ranger.Range) ([]byte, []byte) { @@ -587,27 +612,33 @@ func PartitionHandlesToKVRanges(handles []kv.Handle) []kv.KeyRange { } // IndexRangesToKVRanges converts index ranges to "KeyRange". -func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { +func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, fb, nil, nil) } // IndexRangesToKVRangesWithInterruptSignal converts index ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { - return indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, fb, memTracker, interruptSignal) +func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + keyRanges, err := indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, fb, memTracker, interruptSignal) + if err != nil { + return nil, err + } + err = keyRanges.SetToNonPartitioned() + return keyRanges, err } // IndexRangesToKVRangesForTables converts indexes ranges to "KeyRange". -func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { +func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, fb, nil, nil) } // IndexRangesToKVRangesForTablesWithInterruptSignal converts indexes ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { +func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { if fb == nil || fb.Hist == nil { return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) } + // The following code is non maintained since the feedback deprecated. feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { low, high, err := EncodeIndexKey(sc, ran) @@ -642,11 +673,11 @@ func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementCont } } fb.StoreRanges(feedbackRanges) - return krs, nil + return kv.NewNonParitionedKeyRanges(krs), nil } // CommonHandleRangesToKVRanges converts common handle ranges to "KeyRange". -func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ranges []*ranger.Range) ([]kv.KeyRange, error) { +func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ranges []*ranger.Range) (*kv.KeyRanges, error) { rans := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { low, high, err := EncodeIndexKey(sc, ran) @@ -656,20 +687,23 @@ func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ra rans = append(rans, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true, Collators: collate.GetBinaryCollatorSlice(1)}) } - krs := make([]kv.KeyRange, 0, len(rans)) + krs := make([][]kv.KeyRange, len(tids)) + for i := range krs { + krs[i] = make([]kv.KeyRange, 0, len(ranges)) + } for _, ran := range rans { low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() if ran.LowExclude { low = kv.Key(low).PrefixNext() } ran.LowVal[0].SetBytes(low) - for _, tid := range tids { + for i, tid := range tids { startKey := tablecodec.EncodeRowKey(tid, low) endKey := tablecodec.EncodeRowKey(tid, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) + krs[i] = append(krs[i], kv.KeyRange{StartKey: startKey, EndKey: endKey}) } } - return krs, nil + return kv.NewPartitionedKeyRanges(krs), nil } // VerifyTxnScope verify whether the txnScope and visited physical table break the leader rule's dcLocation. @@ -691,8 +725,12 @@ func VerifyTxnScope(txnScope string, physicalTableID int64, is infoschema.InfoSc return true } -func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { - krs := make([]kv.KeyRange, 0, len(ranges)) +func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + krs := make([][]kv.KeyRange, len(tids)) + for i := range krs { + krs[i] = make([]kv.KeyRange, 0, len(ranges)) + } + const checkSignalStep = 8 var estimatedMemUsage int64 // encodeIndexKey and EncodeIndexSeekKey is time-consuming, thus we need to @@ -705,13 +743,13 @@ func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idx if i == 0 { estimatedMemUsage += int64(cap(low) + cap(high)) } - for _, tid := range tids { + for j, tid := range tids { startKey := tablecodec.EncodeIndexSeekKey(tid, idxID, low) endKey := tablecodec.EncodeIndexSeekKey(tid, idxID, high) if i == 0 { estimatedMemUsage += int64(cap(startKey)) + int64(cap(endKey)) } - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) + krs[j] = append(krs[j], kv.KeyRange{StartKey: startKey, EndKey: endKey}) } if i%checkSignalStep == 0 { if i == 0 && memTracker != nil { @@ -719,11 +757,11 @@ func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idx memTracker.Consume(estimatedMemUsage) } if interruptSignal != nil && interruptSignal.Load().(bool) { - return nil, nil + return kv.NewPartitionedKeyRanges(nil), nil } } } - return krs, nil + return kv.NewPartitionedKeyRanges(krs), nil } // EncodeIndexKey gets encoded keys containing low and high diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 2ffde4a512c0d..fa55229e36fa5 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -192,8 +192,8 @@ func TestIndexRangesToKVRanges(t *testing.T) { actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 12, 15, ranges, nil) require.NoError(t, err) - for i := range actual { - require.Equal(t, expect[i], actual[i]) + for i := range actual.FirstPartitionRange() { + require.Equal(t, expect[i], actual.FirstPartitionRange()[i]) } } @@ -242,7 +242,7 @@ func TestRequestBuilder1(t *testing.T) { Tp: 103, StartTs: 0x0, Data: []uint8{0x18, 0x0, 0x20, 0x0, 0x40, 0x0, 0x5a, 0x0}, - KeyRanges: []kv.KeyRange{ + KeyRanges: kv.NewNonParitionedKeyRanges([]kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x3}, @@ -263,7 +263,7 @@ func TestRequestBuilder1(t *testing.T) { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23}, }, - }, + }), Cacheable: true, KeepOrder: false, Desc: false, @@ -325,7 +325,7 @@ func TestRequestBuilder2(t *testing.T) { Tp: 103, StartTs: 0x0, Data: []uint8{0x18, 0x0, 0x20, 0x0, 0x40, 0x0, 0x5a, 0x0}, - KeyRanges: []kv.KeyRange{ + KeyRanges: kv.NewNonParitionedKeyRanges([]kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x3, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x3, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x3}, @@ -346,7 +346,7 @@ func TestRequestBuilder2(t *testing.T) { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x3, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x3, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23}, }, - }, + }), Cacheable: true, KeepOrder: false, Desc: false, @@ -378,7 +378,7 @@ func TestRequestBuilder3(t *testing.T) { Tp: 103, StartTs: 0x0, Data: []uint8{0x18, 0x0, 0x20, 0x0, 0x40, 0x0, 0x5a, 0x0}, - KeyRanges: []kv.KeyRange{ + KeyRanges: kv.NewNonParitionedKeyRanges([]kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, @@ -395,7 +395,7 @@ func TestRequestBuilder3(t *testing.T) { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x64}, EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x65}, }, - }, + }), Cacheable: true, KeepOrder: false, Desc: false, @@ -444,7 +444,7 @@ func TestRequestBuilder4(t *testing.T) { Tp: 103, StartTs: 0x0, Data: []uint8{0x18, 0x0, 0x20, 0x0, 0x40, 0x0, 0x5a, 0x0}, - KeyRanges: keyRanges, + KeyRanges: kv.NewNonParitionedKeyRanges(keyRanges), Cacheable: true, KeepOrder: false, Desc: false, @@ -491,7 +491,7 @@ func TestRequestBuilder5(t *testing.T) { Tp: 104, StartTs: 0x0, Data: []uint8{0x8, 0x0, 0x18, 0x0, 0x20, 0x0}, - KeyRanges: keyRanges, + KeyRanges: kv.NewNonParitionedKeyRanges(keyRanges), KeepOrder: true, Desc: false, Concurrency: 15, @@ -520,7 +520,7 @@ func TestRequestBuilder6(t *testing.T) { Tp: 105, StartTs: 0x0, Data: []uint8{0x10, 0x0, 0x18, 0x0}, - KeyRanges: keyRanges, + KeyRanges: kv.NewNonParitionedKeyRanges(keyRanges), KeepOrder: false, Desc: false, Concurrency: concurrency, @@ -557,6 +557,7 @@ func TestRequestBuilder7(t *testing.T) { Tp: 0, StartTs: 0x0, KeepOrder: false, + KeyRanges: kv.NewNonParitionedKeyRanges(nil), Desc: false, Concurrency: concurrency, IsolationLevel: 0, @@ -583,6 +584,7 @@ func TestRequestBuilder8(t *testing.T) { Tp: 0, StartTs: 0x0, Data: []uint8(nil), + KeyRanges: kv.NewNonParitionedKeyRanges(nil), Concurrency: variable.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, @@ -635,8 +637,8 @@ func TestIndexRangesToKVRangesWithFbs(t *testing.T) { EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x3, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5}, }, } - for i := 0; i < len(actual); i++ { - require.Equal(t, expect[i], actual[i]) + for i := 0; i < len(actual.FirstPartitionRange()); i++ { + require.Equal(t, expect[i], actual.FirstPartitionRange()[i]) } } diff --git a/distsql/select_result.go b/distsql/select_result.go index 0e807b360d0ad..6d1f6308e4120 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -359,13 +359,11 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr } if r.stats == nil { - id := r.rootPlanID r.stats = &selectResultRuntimeStats{ backoffSleep: make(map[string]time.Duration), rpcStat: tikv.NewRegionRequestRuntimeStats(), distSQLConcurrency: r.distSQLConcurrency, } - r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(id, r.stats) } r.stats.mergeCopRuntimeStats(copStats, respTime) @@ -456,6 +454,9 @@ func (r *selectResult) Close() error { if respSize > 0 { r.memConsume(-respSize) } + if r.stats != nil { + defer r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(r.rootPlanID, r.stats) + } return r.resp.Close() } diff --git a/distsql/select_result_test.go b/distsql/select_result_test.go index c12892083d641..4ec56a286e5ab 100644 --- a/distsql/select_result_test.go +++ b/distsql/select_result_test.go @@ -34,7 +34,7 @@ func TestUpdateCopRuntimeStats(t *testing.T) { require.Nil(t, ctx.GetSessionVars().StmtCtx.RuntimeStatsColl) sr.rootPlanID = 1234 - sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{CalleeAddress: "a"}}, 0) + sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{DetailsNeedP90: execdetails.DetailsNeedP90{CalleeAddress: "a"}}}, 0) ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl(nil) i := uint64(1) @@ -46,13 +46,13 @@ func TestUpdateCopRuntimeStats(t *testing.T) { require.NotEqual(t, len(sr.copPlanIDs), len(sr.selectResp.GetExecutionSummaries())) - sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{CalleeAddress: "callee"}}, 0) + sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{DetailsNeedP90: execdetails.DetailsNeedP90{CalleeAddress: "callee"}}}, 0) require.False(t, ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.ExistsCopStats(1234)) sr.copPlanIDs = []int{sr.rootPlanID} require.NotNil(t, ctx.GetSessionVars().StmtCtx.RuntimeStatsColl) require.Equal(t, len(sr.copPlanIDs), len(sr.selectResp.GetExecutionSummaries())) - sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{CalleeAddress: "callee"}}, 0) + sr.updateCopRuntimeStats(context.Background(), &copr.CopRuntimeStats{ExecDetails: execdetails.ExecDetails{DetailsNeedP90: execdetails.DetailsNeedP90{CalleeAddress: "callee"}}}, 0) require.Equal(t, "tikv_task:{time:1ns, loops:1}", ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetOrCreateCopStats(1234, "tikv").String()) } diff --git a/dumpling/tests/e2e/run.sh b/dumpling/tests/e2e/run.sh index f5da32acc33e0..73b580ca594d9 100644 --- a/dumpling/tests/e2e/run.sh +++ b/dumpling/tests/e2e/run.sh @@ -37,4 +37,24 @@ run_lightning $cur/conf/lightning.toml # check mysql and tidb data check_sync_diff $cur/conf/diff_config.toml +# test e2e with compress option again + +# drop database on tidb +export DUMPLING_TEST_PORT=4000 +run_sql "drop database if exists $DB_NAME;" + +export DUMPLING_TEST_PORT=3306 + +# dumping +export DUMPLING_TEST_DATABASE=$DB_NAME +rm -rf $DUMPLING_OUTPUT_DIR +run_dumpling --compress "snappy" + +cat "$cur/conf/lightning.toml" +# use lightning import data to tidb +run_lightning $cur/conf/lightning.toml + +# check mysql and tidb data +check_sync_diff $cur/conf/diff_config.toml + diff --git a/dumpling/tests/e2e_csv/run.sh b/dumpling/tests/e2e_csv/run.sh index d80e321d9294a..9c5afaca469d7 100644 --- a/dumpling/tests/e2e_csv/run.sh +++ b/dumpling/tests/e2e_csv/run.sh @@ -24,6 +24,7 @@ run_sql_file "$DUMPLING_TEST_DIR/data/e2e_csv.t.sql" run() { echo "*** running subtest case ***" + echo "compress is $compress" echo "escape_backslash is $escape_backslash" echo "csv_delimiter is $csv_delimiter" echo "csv_separator is $csv_separator" @@ -36,7 +37,11 @@ run() { # dumping export DUMPLING_TEST_PORT=3306 export DUMPLING_TEST_DATABASE=$DB_NAME - run_dumpling --filetype="csv" --escape-backslash=$escape_backslash --csv-delimiter="$csv_delimiter" --csv-separator="$csv_separator" + rm -rf "$DUMPLING_OUTPUT_DIR" + if [ $compress = "space" ]; then + compress="" + fi + run_dumpling --filetype="csv" --escape-backslash=$escape_backslash --csv-delimiter="$csv_delimiter" --csv-separator="$csv_separator" --compress="$compress" # construct lightning configuration mkdir -p $DUMPLING_TEST_DIR/conf @@ -67,18 +72,22 @@ run() { escape_backslash_arr="true false" csv_delimiter_arr="\" '" csv_separator_arr=', a aa |*|' +compress_arr='space gzip snappy zstd' -for escape_backslash in $escape_backslash_arr +for compress in $compress_arr do - for csv_separator in $csv_separator_arr + for escape_backslash in $escape_backslash_arr do - for csv_delimiter in $csv_delimiter_arr + for csv_separator in $csv_separator_arr do - run + for csv_delimiter in $csv_delimiter_arr + do + run + done + if [ "$escape_backslash" = "true" ]; then + csv_delimiter="" + run + fi done - if [ "$escape_backslash" = "true" ]; then - csv_delimiter="" - run - fi done done diff --git a/errors.toml b/errors.toml index e34de79661c62..df952cc7af45a 100644 --- a/errors.toml +++ b/errors.toml @@ -66,6 +66,11 @@ error = ''' restore met a invalid peer ''' +["BR:EBS:ErrRestoreRegionWithoutPeer"] +error = ''' +restore met a region without any peer +''' + ["BR:EBS:ErrRestoreTotalKVMismatch"] error = ''' restore total tikvs mismatch diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 7473570b03661..0d0f285ab3710 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -269,7 +269,6 @@ go_test( "delete_test.go", "distsql_test.go", "executor_failpoint_test.go", - "executor_issue_test.go", "executor_pkg_test.go", "executor_required_rows_test.go", "executor_test.go", @@ -358,7 +357,6 @@ go_test( "//parser", "//parser/ast", "//parser/auth", - "//parser/charset", "//parser/model", "//parser/mysql", "//parser/terror", diff --git a/executor/adapter.go b/executor/adapter.go index 3dd0e0ce0877e..aaa7aa8c4b8c9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -215,6 +215,7 @@ type PartitionTelemetryInfo struct { UseCreateIntervalPartition bool UseAddIntervalPartition bool UseDropIntervalPartition bool + UseCompactTablePartition bool } // AccountLockTelemetryInfo records account lock/unlock information during execution diff --git a/executor/admin.go b/executor/admin.go index 1a0f5579281cc..a0484ce957b30 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -265,10 +265,11 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio return nil, err } var builder distsql.RequestBuilder - builder.KeyRanges, err = buildRecoverIndexKeyRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalID, startHandle) + keyRanges, err := buildRecoverIndexKeyRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalID, startHandle) if err != nil { return nil, err } + builder.KeyRanges = kv.NewNonParitionedKeyRanges(keyRanges) kvReq, err := builder. SetDAGRequest(dagPB). SetStartTS(txn.StartTS()). @@ -737,7 +738,16 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio sc := e.ctx.GetSessionVars().StmtCtx var builder distsql.RequestBuilder ranges := ranger.FullRange() - kvReq, err := builder.SetIndexRanges(sc, e.physicalID, e.index.Meta().ID, ranges). + keyRanges, err := distsql.IndexRangesToKVRanges(sc, e.physicalID, e.index.Meta().ID, ranges, nil) + if err != nil { + return nil, err + } + err = keyRanges.SetToNonPartitioned() + if err != nil { + return nil, err + } + keyRanges.FirstPartitionRange()[0].StartKey = kv.Key(e.lastIdxKey).PrefixNext() + kvReq, err := builder.SetWrappedKeyRanges(keyRanges). SetDAGRequest(dagPB). SetStartTS(txn.StartTS()). SetKeepOrder(true). @@ -748,7 +758,6 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio return nil, err } - kvReq.KeyRanges[0].StartKey = kv.Key(e.lastIdxKey).PrefixNext() kvReq.Concurrency = 1 result, err := distsql.Select(ctx, e.ctx, kvReq, e.getIdxColTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) if err != nil { diff --git a/executor/aggregate.go b/executor/aggregate.go index 771d928c9bbad..30b86164ec371 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -244,6 +244,9 @@ func (d *HashAggIntermData) getPartialResultBatch(_ *stmtctx.StatementContext, p // Close implements the Executor Close interface. func (e *HashAggExec) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.isUnparallelExec { var firstErr error e.childResult = nil @@ -1131,7 +1134,6 @@ func (e *HashAggExec) initRuntimeStats() { stats.PartialStats = make([]*AggWorkerStat, 0, stats.PartialConcurrency) stats.FinalStats = make([]*AggWorkerStat, 0, stats.FinalConcurrency) e.stats = stats - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } } diff --git a/executor/autoidtest/autoid_test.go b/executor/autoidtest/autoid_test.go index 269f3966f8a01..f52f63b9c59a2 100644 --- a/executor/autoidtest/autoid_test.go +++ b/executor/autoidtest/autoid_test.go @@ -642,14 +642,23 @@ func TestAutoIDIncrementAndOffset(t *testing.T) { tk.MustExec(`insert into io(b) values (null),(null),(null)`) // AutoID allocation will take increment and offset into consideration. tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14")) - // HandleID allocation will ignore the increment and offset. - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + if str == "" { + // HandleID allocation will ignore the increment and offset. + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + } else { + // Separate row id and auto inc id, increment and offset works on auto inc id + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("1", "2", "3")) + } tk.MustExec(`delete from io`) tk.Session().GetSessionVars().AutoIncrementIncrement = 10 tk.MustExec(`insert into io(b) values (null),(null),(null)`) tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + if str == "" { + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + } else { + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("4", "5", "6")) + } // Test invalid value. tk.Session().GetSessionVars().AutoIncrementIncrement = -1 @@ -711,27 +720,19 @@ func TestAlterTableAutoIDCache(t *testing.T) { require.NoError(t, err2) tk.MustExec("alter table t_473 auto_id_cache = 100") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows( + fmt.Sprintf("test t_473 id %d _TIDB_ROWID", val), + "test t_473 id 1 AUTO_INCREMENT", + )) tk.MustExec("insert into t_473 values ()") tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) - - // Note that auto_id_cache=1 use a different implementation. - tk.MustExec("alter table t_473 auto_id_cache = 1") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) - tk.MustExec("insert into t_473 values ()") - tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val), fmt.Sprintf("%d", val+100))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+101))) - - // alter table from auto_id_cache=1 to default will discard the IDs cached by the autoid service. - // This is because they are two component and TiDB can't tell the autoid service to "save position and exit". - tk.MustExec("alter table t_473 auto_id_cache = 20000") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+4100))) - - tk.MustExec("insert into t_473 values ()") - tk.MustQuery("select * from t_473").Check(testkit.Rows("1", - fmt.Sprintf("%d", val), - fmt.Sprintf("%d", val+100), - fmt.Sprintf("%d", val+4100))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+24100))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows( + fmt.Sprintf("test t_473 id %d _TIDB_ROWID", val+100), + "test t_473 id 1 AUTO_INCREMENT", + )) + + // Note that auto_id_cache=1 use a different implementation, switch between them is not allowed. + // TODO: relax this restriction and update the test case. + _, err = tk.Exec("alter table t_473 auto_id_cache = 1") + require.Error(t, err) } diff --git a/executor/batch_checker.go b/executor/batch_checker.go index d3820ecb0d08c..79a6748b2d5c3 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -180,6 +180,10 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D if !distinct { continue } + // If index is used ingest ways, then we should check key from temp index. + if v.Meta().BackfillState != model.BackfillStateInapplicable { + _, key, _ = tables.GenTempIdxKeyByState(v.Meta(), key) + } colValStr, err1 := formatDataForDupError(colVals) if err1 != nil { return nil, err1 diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 6f9710dd14f0d..ee9808700aaec 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -159,6 +159,9 @@ func MockNewCacheTableSnapShot(snapshot kv.Snapshot, memBuffer kv.MemBuffer) *ca // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { + if e.runtimeStats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.runtimeStats != nil && e.snapshot != nil { e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 0968c629b4b0d..542ba5d5f963c 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -906,14 +906,10 @@ func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) joinSchema.Append(cols1...) } - joinKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) - for _, keyIdx := range testCase.keyIdx { - joinKeys = append(joinKeys, cols0[keyIdx]) - } - probeKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) - for _, keyIdx := range testCase.keyIdx { - probeKeys = append(probeKeys, cols1[keyIdx]) - } + joinKeysColIdx := make([]int, 0, len(testCase.keyIdx)) + joinKeysColIdx = append(joinKeysColIdx, testCase.keyIdx...) + probeKeysColIdx := make([]int, 0, len(testCase.keyIdx)) + probeKeysColIdx = append(probeKeysColIdx, testCase.keyIdx...) e := &HashJoinExec{ baseExecutor: newBaseExecutor(testCase.ctx, joinSchema, 5, innerExec, outerExec), hashJoinCtx: &hashJoinCtx{ @@ -921,26 +917,31 @@ func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) isOuterJoin: false, useOuterToBuild: testCase.useOuterToBuild, concurrency: uint(testCase.concurrency), + probeTypes: retTypes(outerExec), + buildTypes: retTypes(innerExec), }, probeSideTupleFetcher: &probeSideTupleFetcher{ probeSideExec: outerExec, }, - probeWorkers: make([]probeWorker, testCase.concurrency), - buildKeys: joinKeys, - probeKeys: probeKeys, - buildSideExec: innerExec, - buildSideEstCount: float64(testCase.rows), + probeWorkers: make([]*probeWorker, testCase.concurrency), + buildWorker: &buildWorker{ + buildKeyColIdx: joinKeysColIdx, + buildSideExec: innerExec, + }, } childrenUsedSchema := markChildrenUsedCols(e.Schema(), e.children[0].Schema(), e.children[1].Schema()) - defaultValues := make([]types.Datum, e.buildSideExec.Schema().Len()) + defaultValues := make([]types.Datum, e.buildWorker.buildSideExec.Schema().Len()) lhsTypes, rhsTypes := retTypes(innerExec), retTypes(outerExec) for i := uint(0); i < e.concurrency; i++ { - e.probeWorkers[i].workerID = i - e.probeWorkers[i].sessCtx = e.ctx - e.probeWorkers[i].hashJoinCtx = e.hashJoinCtx - e.probeWorkers[i].joiner = newJoiner(testCase.ctx, e.joinType, true, defaultValues, - nil, lhsTypes, rhsTypes, childrenUsedSchema, false) + e.probeWorkers[i] = &probeWorker{ + workerID: i, + sessCtx: e.ctx, + hashJoinCtx: e.hashJoinCtx, + joiner: newJoiner(testCase.ctx, e.joinType, true, defaultValues, + nil, lhsTypes, rhsTypes, childrenUsedSchema, false), + probeKeyColIdx: probeKeysColIdx, + } } memLimit := int64(-1) if testCase.disk { diff --git a/executor/builder.go b/executor/builder.go index eb8825dd0ce99..a046c6e386c9d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1400,17 +1400,6 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) Execu return e } -func (b *executorBuilder) buildSideEstCount(v *plannercore.PhysicalHashJoin) float64 { - buildSide := v.Children()[v.InnerChildIdx] - if v.UseOuterToBuild { - buildSide = v.Children()[1-v.InnerChildIdx] - } - if buildSide.Stats().HistColl == nil || buildSide.Stats().HistColl.Pseudo { - return 0.0 - } - return buildSide.StatsCount() -} - func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executor { leftExec := b.build(v.Children()[0]) if b.err != nil { @@ -1425,6 +1414,8 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo e := &HashJoinExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), leftExec, rightExec), probeSideTupleFetcher: &probeSideTupleFetcher{}, + probeWorkers: make([]*probeWorker, v.Concurrency), + buildWorker: &buildWorker{}, hashJoinCtx: &hashJoinCtx{ isOuterJoin: v.JoinType.IsOuterJoin(), useOuterToBuild: v.UseOuterToBuild, @@ -1449,15 +1440,17 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo leftIsBuildSide := true e.isNullEQ = v.IsNullEQ + var probeKeys, probeNAKeys, buildKeys, buildNAKeys []*expression.Column + var buildSideExec Executor if v.UseOuterToBuild { // update the buildSideEstCount due to changing the build side if v.InnerChildIdx == 1 { - e.buildSideExec, e.buildKeys, e.buildNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys - e.probeSideTupleFetcher.probeSideExec, e.probeKeys, e.probeNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys + buildSideExec, buildKeys, buildNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys + e.probeSideTupleFetcher.probeSideExec, probeKeys, probeNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys e.outerFilter = v.LeftConditions } else { - e.buildSideExec, e.buildKeys, e.buildNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys - e.probeSideTupleFetcher.probeSideExec, e.probeKeys, e.probeNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys + buildSideExec, buildKeys, buildNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys + e.probeSideTupleFetcher.probeSideExec, probeKeys, probeNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys e.outerFilter = v.RightConditions leftIsBuildSide = false } @@ -1466,30 +1459,48 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo } } else { if v.InnerChildIdx == 0 { - e.buildSideExec, e.buildKeys, e.buildNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys - e.probeSideTupleFetcher.probeSideExec, e.probeKeys, e.probeNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys + buildSideExec, buildKeys, buildNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys + e.probeSideTupleFetcher.probeSideExec, probeKeys, probeNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys e.outerFilter = v.RightConditions } else { - e.buildSideExec, e.buildKeys, e.buildNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys - e.probeSideTupleFetcher.probeSideExec, e.probeKeys, e.probeNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys + buildSideExec, buildKeys, buildNAKeys = rightExec, v.RightJoinKeys, v.RightNAJoinKeys + e.probeSideTupleFetcher.probeSideExec, probeKeys, probeNAKeys = leftExec, v.LeftJoinKeys, v.LeftNAJoinKeys e.outerFilter = v.LeftConditions leftIsBuildSide = false } if defaultValues == nil { - defaultValues = make([]types.Datum, e.buildSideExec.Schema().Len()) + defaultValues = make([]types.Datum, buildSideExec.Schema().Len()) } } + probeKeyColIdx := make([]int, len(probeKeys)) + probeNAKeColIdx := make([]int, len(probeNAKeys)) + buildKeyColIdx := make([]int, len(buildKeys)) + buildNAKeyColIdx := make([]int, len(buildNAKeys)) + for i := range buildKeys { + buildKeyColIdx[i] = buildKeys[i].Index + } + for i := range buildNAKeys { + buildNAKeyColIdx[i] = buildNAKeys[i].Index + } + for i := range probeKeys { + probeKeyColIdx[i] = probeKeys[i].Index + } + for i := range probeNAKeys { + probeNAKeColIdx[i] = probeNAKeys[i].Index + } isNAJoin := len(v.LeftNAJoinKeys) > 0 - e.buildSideEstCount = b.buildSideEstCount(v) childrenUsedSchema := markChildrenUsedCols(v.Schema(), v.Children()[0].Schema(), v.Children()[1].Schema()) - e.probeWorkers = make([]probeWorker, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.probeWorkers[i].hashJoinCtx = e.hashJoinCtx - e.probeWorkers[i].workerID = i - e.probeWorkers[i].sessCtx = e.ctx - e.probeWorkers[i].joiner = newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, - v.OtherConditions, lhsTypes, rhsTypes, childrenUsedSchema, isNAJoin) + e.probeWorkers[i] = &probeWorker{ + hashJoinCtx: e.hashJoinCtx, + workerID: i, + sessCtx: e.ctx, + joiner: newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes, childrenUsedSchema, isNAJoin), + probeKeyColIdx: probeKeyColIdx, + probeNAKeyColIdx: probeNAKeColIdx, + } } + e.buildWorker.buildKeyColIdx, e.buildWorker.buildNAKeyColIdx, e.buildWorker.buildSideExec = buildKeyColIdx, buildNAKeyColIdx, buildSideExec e.hashJoinCtx.isNullAware = isNAJoin executorCountHashJoinExec.Inc() @@ -4251,32 +4262,37 @@ type kvRangeBuilderFromRangeAndPartition struct { } func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*ranger.Range) ([]int64, [][]kv.KeyRange, error) { - ret := make([][]kv.KeyRange, 0, len(h.partitions)) + ret := make([][]kv.KeyRange, len(h.partitions)) pids := make([]int64, 0, len(h.partitions)) - for _, p := range h.partitions { + for i, p := range h.partitions { pid := p.GetPhysicalID() + pids = append(pids, pid) meta := p.Meta() + if len(ranges) == 0 { + continue + } kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) if err != nil { return nil, nil, err } - pids = append(pids, pid) - ret = append(ret, kvRange) + ret[i] = kvRange.AppendSelfTo(ret[i]) } return pids, ret, nil } -func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(ranges []*ranger.Range) ([]kv.KeyRange, error) { - //nolint: prealloc - var ret []kv.KeyRange - for _, p := range h.partitions { +func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(ranges []*ranger.Range) ([][]kv.KeyRange, error) { + ret := make([][]kv.KeyRange, len(h.partitions)) + if len(ranges) == 0 { + return ret, nil + } + for i, p := range h.partitions { pid := p.GetPhysicalID() meta := p.Meta() kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) if err != nil { return nil, err } - ret = append(ret, kvRange...) + ret[i] = kvRange.AppendSelfTo(ret[i]) } return ret, nil } @@ -4323,7 +4339,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T if err != nil { return nil, err } - e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) e.resultHandler = &tableResultHandler{} result, err := builder.SelectResult(ctx, builder.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) if err != nil { @@ -4346,6 +4362,8 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex } else { b.SetTableHandles(getPhysicalTableID(e.table), handles) } + } else { + b.SetKeyRanges(nil) } return builder.buildTableReaderBase(ctx, e, b) } @@ -4534,6 +4552,9 @@ func buildRangesForIndexJoin(ctx sessionctx.Context, lookUpContents []*indexJoin func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, memTracker *memory.Tracker, interruptSignal *atomic.Value) (_ []kv.KeyRange, err error) { kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) + if len(ranges) == 0 { + return []kv.KeyRange{}, nil + } lastPos := len(ranges[0].LowVal) - 1 sc := ctx.GetSessionVars().StmtCtx tmpDatumRanges := make([]*ranger.Range, 0, len(lookUpContents)) @@ -4546,7 +4567,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l } if cwc == nil { // Index id is -1 means it's a common handle. - var tmpKvRanges []kv.KeyRange + var tmpKvRanges *kv.KeyRanges var err error if indexID == -1 { tmpKvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{tableID}, ranges) @@ -4556,7 +4577,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l if err != nil { return nil, err } - kvRanges = append(kvRanges, tmpKvRanges...) + kvRanges = tmpKvRanges.AppendSelfTo(kvRanges) continue } nextColRanges, err := cwc.BuildRangesByRow(ctx, content.row) @@ -4593,9 +4614,11 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l } // Index id is -1 means it's a common handle. if indexID == -1 { - return distsql.CommonHandleRangesToKVRanges(ctx.GetSessionVars().StmtCtx, []int64{tableID}, tmpDatumRanges) + tmpKeyRanges, err := distsql.CommonHandleRangesToKVRanges(ctx.GetSessionVars().StmtCtx, []int64{tableID}, tmpDatumRanges) + return tmpKeyRanges.FirstPartitionRange(), err } - return distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil, memTracker, interruptSignal) + tmpKeyRanges, err := distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil, memTracker, interruptSignal) + return tmpKeyRanges.FirstPartitionRange(), err } func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) Executor { @@ -4883,7 +4906,6 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan SnapshotRuntimeStats: snapshotStats, } e.snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) - b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if plan.IndexInfo != nil { @@ -5279,6 +5301,10 @@ func (b *executorBuilder) buildCompactTable(v *plannercore.CompactTable) Executo } partitionIDs = append(partitionIDs, partitionID) } + if b.Ti.PartitionTelemetry == nil { + b.Ti.PartitionTelemetry = &PartitionTelemetryInfo{} + } + b.Ti.PartitionTelemetry.UseCompactTablePartition = true } return &CompactTableTiFlashExec{ diff --git a/executor/distsql.go b/executor/distsql.go index 0cef7e66d441e..aab5067a81b6a 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -243,11 +243,18 @@ func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return err } +// TODO: cleanup this method. func (e *IndexReaderExecutor) buildKeyRanges(sc *stmtctx.StatementContext, ranges []*ranger.Range, physicalID int64) ([]kv.KeyRange, error) { + var ( + rRanges *kv.KeyRanges + err error + ) if e.index.ID == -1 { - return distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) + rRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) + } else { + rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) } - return distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) + return rRanges.FirstPartitionRange(), err } // Open implements the Executor Open interface. @@ -458,9 +465,6 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { sc := e.ctx.GetSessionVars().StmtCtx if e.partitionTableMode { - if e.keepOrder { // this case should be prevented by the optimizer - return errors.New("invalid execution plan: cannot keep order when accessing a partition table by IndexLookUpReader") - } e.feedback.Invalidate() // feedback for partition tables is not ready e.partitionKVRanges = make([][]kv.KeyRange, 0, len(e.prunedPartitions)) for _, p := range e.prunedPartitions { @@ -472,7 +476,7 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if e.partitionRangeMap != nil && e.partitionRangeMap[physicalID] != nil { ranges = e.partitionRangeMap[physicalID] } - var kvRange []kv.KeyRange + var kvRange *kv.KeyRanges if e.index.ID == -1 { kvRange, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) } else { @@ -481,15 +485,17 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if err != nil { return err } - e.partitionKVRanges = append(e.partitionKVRanges, kvRange) + e.partitionKVRanges = append(e.partitionKVRanges, kvRange.FirstPartitionRange()) } } else { physicalID := getPhysicalTableID(e.table) + var kvRanges *kv.KeyRanges if e.index.ID == -1 { - e.kvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, e.ranges) + kvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, e.ranges) } else { - e.kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges, e.feedback) + kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges, e.feedback) } + e.kvRanges = kvRanges.FirstPartitionRange() } return err } @@ -718,6 +724,9 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup // Close implements Exec Close interface. func (e *IndexLookUpExecutor) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } e.kvRanges = e.kvRanges[:0] if e.dummy { return nil @@ -802,7 +811,6 @@ func (e *IndexLookUpExecutor) initRuntimeStats() { indexScanBasicStats: &execdetails.BasicRuntimeStats{}, Concurrency: e.ctx.GetSessionVars().IndexLookupConcurrency(), } - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } } @@ -870,7 +878,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes idxID := w.idxLookup.getIndexPlanRootID() if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if idxID != w.idxLookup.id && w.idxLookup.stats != nil { - w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(idxID, w.idxLookup.stats.indexScanBasicStats) + w.idxLookup.stats.indexScanBasicStats = w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetBasicRuntimeStats(idxID) } } for { diff --git a/executor/executor.go b/executor/executor.go index 6aac622b223a4..169a19ec252e8 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -272,8 +272,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id int, } if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if e.id > 0 { - e.runtimeStats = &execdetails.BasicRuntimeStats{} - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(id, e.runtimeStats) + e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetBasicRuntimeStats(id) } } if schema != nil { @@ -399,7 +398,7 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { tblMeta := tbl.Meta() allocators := tbl.Allocators(e.ctx) - for _, alloc := range allocators { + for _, alloc := range allocators.Allocs { nextGlobalID, err := alloc.NextGlobalAutoID() if err != nil { return err @@ -407,7 +406,16 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { var colName, idType string switch alloc.GetType() { - case autoid.RowIDAllocType, autoid.AutoIncrementType: + case autoid.RowIDAllocType: + idType = "_TIDB_ROWID" + if tblMeta.PKIsHandle { + if col := tblMeta.GetAutoIncrementColInfo(); col != nil { + colName = col.Name.O + } + } else { + colName = model.ExtraHandleName.O + } + case autoid.AutoIncrementType: idType = "AUTO_INCREMENT" if tblMeta.PKIsHandle { if col := tblMeta.GetAutoIncrementColInfo(); col != nil { diff --git a/executor/executor_test.go b/executor/executor_test.go index 5e8bb71daca6a..c73b3a3df7abd 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1922,7 +1922,7 @@ func TestCheckIndex(t *testing.T) { tbInfo := tbl.Meta() alloc := autoid.NewAllocator(store, dbInfo.ID, tbInfo.ID, false, autoid.RowIDAllocType) - tb, err := tables.TableFromMeta(autoid.NewAllocators(alloc), tbInfo) + tb, err := tables.TableFromMeta(autoid.NewAllocators(false, alloc), tbInfo) require.NoError(t, err) _, err = se.Execute(context.Background(), "admin check index t c") diff --git a/executor/fktest/BUILD.bazel b/executor/fktest/BUILD.bazel index dbdae1843edaf..86f7a48d28bb6 100644 --- a/executor/fktest/BUILD.bazel +++ b/executor/fktest/BUILD.bazel @@ -8,6 +8,7 @@ go_test( "main_test.go", ], flaky = True, + shard_count = 20, deps = [ "//config", "//executor", diff --git a/executor/fktest/foreign_key_test.go b/executor/fktest/foreign_key_test.go index 17a6f7226e545..dbe02ccd52772 100644 --- a/executor/fktest/foreign_key_test.go +++ b/executor/fktest/foreign_key_test.go @@ -2081,3 +2081,29 @@ func TestForeignKeyOnInsertOnDuplicateUpdate(t *testing.T) { tk.MustQuery("select * from t2").Check(testkit.Rows("1")) tk.MustQuery("select * from t3").Check(testkit.Rows("1")) } + +func TestForeignKeyIssue39419(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@global.tidb_enable_foreign_key=1") + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int key);") + tk.MustExec("create table t2 (id int key, a int, b int, " + + "foreign key fk_1 (a) references t1(id) ON DELETE SET NULL ON UPDATE SET NULL, " + + "foreign key fk_2 (b) references t1(id) ON DELETE CASCADE ON UPDATE CASCADE);") + tk.MustExec("insert into t1 values (1), (2), (3);") + tk.MustExec("insert into t2 values (1, 1, 1), (2, 2, 2), (3, 3, 3);") + tk.MustExec("update t1 set id=id+10 where id in (1, 3);") + tk.MustQuery("select * from t1 order by id").Check(testkit.Rows("2", "11", "13")) + tk.MustQuery("select * from t2 order by id").Check(testkit.Rows("1 11", "2 2 2", "3 13")) + tk.MustExec("delete from t1 where id = 2;") + tk.MustQuery("select * from t1 order by id").Check(testkit.Rows("11", "13")) + tk.MustQuery("select * from t2 order by id").Check(testkit.Rows("1 11", "3 13")) + + tk.MustExec("drop table t1,t2") + tk.MustExec("create table t1 (id int, b int, index(id), foreign key fk_2 (b) references t1(id) ON UPDATE CASCADE);") + tk.MustExec("insert into t1 values (1, 1), (2, 2), (3, 3);") + tk.MustExec("update t1 set id=id+10 where id > 1") + tk.MustQuery("select * from t1 order by id").Check(testkit.Rows("1 1", "12 12", "13 13")) +} diff --git a/executor/hash_table.go b/executor/hash_table.go index b7c875148bffa..2ba840d04fdc9 100644 --- a/executor/hash_table.go +++ b/executor/hash_table.go @@ -117,7 +117,7 @@ type hashRowContainer struct { chkBuf *chunk.Chunk } -func newHashRowContainer(sCtx sessionctx.Context, estCount int, hCtx *hashContext, allTypes []*types.FieldType) *hashRowContainer { +func newHashRowContainer(sCtx sessionctx.Context, hCtx *hashContext, allTypes []*types.FieldType) *hashRowContainer { maxChunkSize := sCtx.GetSessionVars().MaxChunkSize rc := chunk.NewRowContainer(allTypes, maxChunkSize) c := &hashRowContainer{ diff --git a/executor/hash_table_test.go b/executor/hash_table_test.go index 3b4a4acee5284..0a387e0e7e5b6 100644 --- a/executor/hash_table_test.go +++ b/executor/hash_table_test.go @@ -127,7 +127,7 @@ func testHashRowContainer(t *testing.T, hashFunc func() hash.Hash64, spill bool) for i := 0; i < numRows; i++ { hCtx.hashVals = append(hCtx.hashVals, hashFunc()) } - rowContainer := newHashRowContainer(sctx, 0, hCtx, colTypes) + rowContainer := newHashRowContainer(sctx, hCtx, colTypes) copiedRC = rowContainer.ShallowCopy() tracker := rowContainer.GetMemTracker() tracker.SetLabel(memory.LabelForBuildSideResult) diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index c54b60749601d..58bd84ff6e4d6 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -134,7 +134,6 @@ func (e *IndexNestedLoopHashJoin) Open(ctx context.Context) error { e.innerPtrBytes = make([][]byte, 0, 8) if e.runtimeStats != nil { e.stats = &indexLookUpJoinRuntimeStats{} - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } e.finished.Store(false) return nil @@ -288,6 +287,9 @@ func (e *IndexNestedLoopHashJoin) isDryUpTasks(ctx context.Context) bool { // Close implements the IndexNestedLoopHashJoin Executor interface. func (e *IndexNestedLoopHashJoin) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.cancelFunc != nil { e.cancelFunc() } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 05cc337d3d7ee..187e83cc0f763 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -171,7 +171,6 @@ func (e *IndexLookUpJoin) Open(ctx context.Context) error { e.finished.Store(false) if e.runtimeStats != nil { e.stats = &indexLookUpJoinRuntimeStats{} - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } e.cancelFunc = nil return nil @@ -765,6 +764,9 @@ func (iw *innerWorker) hasNullInJoinKey(row chunk.Row) bool { // Close implements the Executor interface. func (e *IndexLookUpJoin) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.cancelFunc != nil { e.cancelFunc() } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 8bd379944c825..e0fb176fff589 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -715,6 +715,9 @@ func (imw *innerMergeWorker) fetchNextInnerResult(ctx context.Context, task *loo // Close implements the Executor interface. func (e *IndexLookUpMergeJoin) Close() error { + if e.runtimeStats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.runtimeStats) + } if e.cancelFunc != nil { e.cancelFunc() e.cancelFunc = nil diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 0e7eb394710fd..cde353e2a64b2 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -194,7 +194,7 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang if err != nil { return nil, err } - keyRanges := append(firstKeyRanges, secondKeyRanges...) + keyRanges := append(firstKeyRanges.FirstPartitionRange(), secondKeyRanges.FirstPartitionRange()...) ranges = append(ranges, keyRanges) continue } @@ -202,7 +202,7 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang if err != nil { return nil, err } - ranges = append(ranges, keyRange) + ranges = append(ranges, keyRange.FirstPartitionRange()) } return ranges, nil } @@ -470,7 +470,6 @@ func (e *IndexMergeReaderExecutor) initRuntimeStats() { e.stats = &IndexMergeRuntimeStat{ Concurrency: e.ctx.GetSessionVars().IndexLookupConcurrency(), } - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } } @@ -704,6 +703,9 @@ func (e *IndexMergeReaderExecutor) handleHandlesFetcherPanic(ctx context.Context // Close implements Exec Close interface. func (e *IndexMergeReaderExecutor) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.finished == nil { return nil } @@ -821,8 +823,7 @@ func (w *partialIndexWorker) fetchHandles( var basicStats *execdetails.BasicRuntimeStats if w.stats != nil { if w.idxID != 0 { - basicStats = &execdetails.BasicRuntimeStats{} - w.sc.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(w.idxID, basicStats) + basicStats = w.sc.GetSessionVars().StmtCtx.RuntimeStatsColl.GetBasicRuntimeStats(w.idxID) } } for { diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 1faf10d490856..472220bb2dcc6 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -386,7 +386,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m if err != nil { return 0, err } - return tbl.Allocators(ctx).Get(autoid.RowIDAllocType).Base() + 1, nil + return tbl.Allocators(ctx).Get(autoid.AutoIncrementType).Base() + 1, nil } func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool { diff --git a/executor/insert.go b/executor/insert.go index 2450f0a117f70..36af152899bc3 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -15,6 +15,7 @@ package executor import ( + "bytes" "context" "encoding/hex" "fmt" @@ -31,6 +32,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -262,6 +264,14 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D } return err } + // Since the temp index stores deleted key with marked 'deleteu' for unique key at the end + // of value, So if return a key we check and skip deleted key. + if tablecodec.IsTempIndexKey(uk.newKey) { + rowVal := val[:len(val)-1] + if bytes.Equal(rowVal, tables.DeleteMarkerUnique) { + continue + } + } handle, err := tablecodec.DecodeHandleInUniqueIndexValue(val, uk.commonHandle) if err != nil { return err @@ -327,6 +337,9 @@ func (e *InsertExec) Next(ctx context.Context, req *chunk.Chunk) error { // Close implements the Executor Close interface. func (e *InsertExec) Close() error { + if e.runtimeStats != nil && e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } defer e.memTracker.ReplaceBytesUsed(0) e.ctx.GetSessionVars().CurrInsertValues = chunk.Row{} e.ctx.GetSessionVars().CurrInsertBatchExtraCols = e.ctx.GetSessionVars().CurrInsertBatchExtraCols[0:0:0] diff --git a/executor/insert_common.go b/executor/insert_common.go index 5bb7feb2441da..7fcc0821ce82d 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -778,7 +778,8 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [] } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true) + alloc := e.Table.Allocators(e.ctx).Get(autoid.AutoIncrementType) + err = alloc.Rebase(ctx, recordID, true) if err != nil { return nil, err } @@ -871,7 +872,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true) + err = e.Table.Allocators(e.ctx).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true) if err != nil { return types.Datum{}, err } @@ -1096,7 +1097,6 @@ func (e *InsertValues) collectRuntimeStatsEnabled() bool { for _, fkc := range e.fkChecks { fkc.stats = e.stats.FKCheckStats } - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } return true } diff --git a/executor/issuetest/BUILD.bazel b/executor/issuetest/BUILD.bazel new file mode 100644 index 0000000000000..77bfaf7f11290 --- /dev/null +++ b/executor/issuetest/BUILD.bazel @@ -0,0 +1,27 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "issuetest_test", + srcs = [ + "executor_issue_test.go", + "main_test.go", + ], + flaky = True, + shard_count = 50, + deps = [ + "//config", + "//kv", + "//meta/autoid", + "//parser/auth", + "//parser/charset", + "//parser/mysql", + "//sessionctx/variable", + "//statistics", + "//testkit", + "//util", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//tikv", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/executor/executor_issue_test.go b/executor/issuetest/executor_issue_test.go similarity index 96% rename from executor/executor_issue_test.go rename to executor/issuetest/executor_issue_test.go index 5d2912f78f449..643339dd157d0 100644 --- a/executor/executor_issue_test.go +++ b/executor/issuetest/executor_issue_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package executor_test +package issuetest_test import ( "context" @@ -684,6 +684,9 @@ func TestIssue22231(t *testing.T) { tk.MustQuery("select cast('2020-05-28 23:59:59 00:00:00' as datetime)").Check(testkit.Rows("2020-05-28 23:59:59")) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect datetime value: '2020-05-28 23:59:59 00:00:00'")) tk.MustExec("drop table if exists t_issue_22231") + + tk.MustQuery("SELECT CAST(\"1111111111-\" AS DATE);") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '1111111111-'")) } // TestIssue2612 is related with https://github.com/pingcap/tidb/issues/2612 @@ -1237,3 +1240,46 @@ func TestIssue33214(t *testing.T) { } } } + +func TestIssue982(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (c int auto_increment, key(c)) auto_id_cache 1;") + tk.MustExec("insert into t values();") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t;").Check(testkit.Rows("1", "2")) +} + +func TestIssue24627(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + for _, sql := range []string{ + "create table test(id float primary key clustered AUTO_INCREMENT, col1 int);", + "create table test(id float primary key nonclustered AUTO_INCREMENT, col1 int) AUTO_ID_CACHE 1;", + } { + tk.MustExec("drop table if exists test;") + tk.MustExec(sql) + tk.MustExec("replace into test(col1) values(1);") + tk.MustExec("replace into test(col1) values(2);") + tk.MustQuery("select * from test;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table test") + } + + for _, sql := range []string{ + "create table test2(id double primary key clustered AUTO_INCREMENT, col1 int);", + "create table test2(id double primary key nonclustered AUTO_INCREMENT, col1 int) AUTO_ID_CACHE 1;", + } { + tk.MustExec(sql) + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("insert into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("insert into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustQuery("select * from test2").Check(testkit.Rows("1 1", "2 1", "3 1", "4 1", "5 1", "6 1")) + tk.MustExec("drop table test2") + } +} diff --git a/executor/issuetest/main_test.go b/executor/issuetest/main_test.go new file mode 100644 index 0000000000000..daecbf2f06859 --- /dev/null +++ b/executor/issuetest/main_test.go @@ -0,0 +1,45 @@ +// 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 issuetest + +import ( + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/meta/autoid" + "github.com/tikv/client-go/v2/tikv" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + autoid.SetStep(5000) + config.UpdateGlobal(func(conf *config.Config) { + conf.Instance.SlowThreshold = 30000 // 30s + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + tikv.EnableFailpoints() + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/executor/join.go b/executor/join.go index 1c75252a5e876..214e2edb1d440 100644 --- a/executor/join.go +++ b/executor/join.go @@ -80,8 +80,10 @@ type probeSideTupleFetcher struct { type probeWorker struct { hashJoinCtx *hashJoinCtx sessCtx sessionctx.Context + workerID uint - workerID uint + probeKeyColIdx []int + probeNAKeyColIdx []int // We pre-alloc and reuse the Rows and RowPtrs for each probe goroutine, to avoid allocation frequently buildSideRows []chunk.Row buildSideRowPtrs []chunk.RowPtr @@ -99,20 +101,20 @@ type probeWorker struct { probeResultCh chan *chunk.Chunk } +type buildWorker struct { + buildSideExec Executor + buildKeyColIdx []int + buildNAKeyColIdx []int +} + // HashJoinExec implements the hash join algorithm. type HashJoinExec struct { baseExecutor *hashJoinCtx probeSideTupleFetcher *probeSideTupleFetcher - probeWorkers []probeWorker - - buildSideExec Executor - buildSideEstCount float64 - probeKeys []*expression.Column - probeNAKeys []*expression.Column - buildKeys []*expression.Column - buildNAKeys []*expression.Column + probeWorkers []*probeWorker + buildWorker *buildWorker worker util.WaitGroupWrapper waiter util.WaitGroupWrapper @@ -181,6 +183,9 @@ func (e *HashJoinExec) Close() error { if e.stats != nil && e.rowContainer != nil { e.stats.hashStat = *e.rowContainer.stat } + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } err := e.baseExecutor.Close() return err } @@ -204,17 +209,10 @@ func (e *HashJoinExec) Open(ctx context.Context) error { e.closeCh = make(chan struct{}) e.finished.Store(false) - if e.probeTypes == nil { - e.probeTypes = retTypes(e.probeSideTupleFetcher.probeSideExec) - } - if e.buildTypes == nil { - e.buildTypes = retTypes(e.buildSideExec) - } if e.runtimeStats != nil { e.stats = &hashJoinRuntimeStats{ concurrent: int(e.concurrency), } - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } return nil } @@ -311,8 +309,8 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu if e.finished.Load() { return } - chk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize, e.ctx.GetSessionVars().MaxChunkSize, e.AllocPool) - err = Next(ctx, e.buildSideExec, chk) + chk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.buildWorker.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize, e.ctx.GetSessionVars().MaxChunkSize, e.AllocPool) + err = Next(ctx, e.buildWorker.buildSideExec, chk) if err != nil { errCh <- errors.Trace(err) return @@ -373,19 +371,11 @@ func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { e.probeSideTupleFetcher.fetchProbeSideChunks(ctx, e.maxChunkSize) }, e.probeSideTupleFetcher.handleProbeSideFetcherPanic) - probeKeyColIdx := make([]int, len(e.probeKeys)) - probeNAKeColIdx := make([]int, len(e.probeNAKeys)) - for i := range e.probeKeys { - probeKeyColIdx[i] = e.probeKeys[i].Index - } - for i := range e.probeNAKeys { - probeNAKeColIdx[i] = e.probeNAKeys[i].Index - } for i := uint(0); i < e.concurrency; i++ { workerID := i e.worker.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinWorker").End() - e.probeWorkers[workerID].runJoinWorker(probeKeyColIdx, probeNAKeColIdx) + e.probeWorkers[workerID].runJoinWorker() }, e.probeWorkers[workerID].handleProbeWorkerPanic) } e.waiter.RunWithRecover(e.waitJoinWorkersAndCloseResultChan, nil) @@ -461,7 +451,7 @@ func (e *HashJoinExec) waitJoinWorkersAndCloseResultChan() { close(e.joinResultCh) } -func (w *probeWorker) runJoinWorker(probeKeyColIdx, probeNAKeyColIdx []int) { +func (w *probeWorker) runJoinWorker() { probeTime := int64(0) if w.hashJoinCtx.stats != nil { start := time.Now() @@ -488,8 +478,8 @@ func (w *probeWorker) runJoinWorker(probeKeyColIdx, probeNAKeyColIdx []int) { } hCtx := &hashContext{ allTypes: w.hashJoinCtx.probeTypes, - keyColIdx: probeKeyColIdx, - naKeyColIdx: probeNAKeyColIdx, + keyColIdx: w.probeKeyColIdx, + naKeyColIdx: w.probeNAKeyColIdx, } for ok := true; ok; { if w.hashJoinCtx.finished.Load() { @@ -1103,20 +1093,12 @@ func (w *probeWorker) join2ChunkForOuterHashJoin(probeSideChk *chunk.Chunk, hCtx func (e *HashJoinExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { if !e.prepared { e.buildFinished = make(chan error, 1) - buildKeyColIdx := make([]int, len(e.buildKeys)) - for i := range e.buildKeys { - buildKeyColIdx[i] = e.buildKeys[i].Index - } - buildNAKeyColIdx := make([]int, len(e.buildNAKeys)) - for i := range e.buildNAKeys { - buildNAKeyColIdx[i] = e.buildNAKeys[i].Index - } hCtx := &hashContext{ allTypes: e.buildTypes, - keyColIdx: buildKeyColIdx, - naKeyColIdx: buildNAKeyColIdx, + keyColIdx: e.buildWorker.buildKeyColIdx, + naKeyColIdx: e.buildWorker.buildNAKeyColIdx, } - e.rowContainer = newHashRowContainer(e.ctx, int(e.buildSideEstCount), hCtx, retTypes(e.buildSideExec)) + e.rowContainer = newHashRowContainer(e.ctx, hCtx, retTypes(e.buildWorker.buildSideExec)) // we shallow copies rowContainer for each probe worker to avoid lock contention for i := uint(0); i < e.concurrency; i++ { if i == 0 { @@ -1291,7 +1273,6 @@ func (e *NestedLoopApplyExec) Close() error { e.memTracker = nil if e.runtimeStats != nil { runtimeStats := newJoinRuntimeStats() - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, runtimeStats) if e.canUseCache { var hitRatio float64 if e.cacheAccessCounter > 0 { @@ -1302,6 +1283,7 @@ func (e *NestedLoopApplyExec) Close() error { runtimeStats.setCacheInfo(false, 0) } runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", 0)) + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, runtimeStats) } return e.outerExec.Close() } @@ -1559,6 +1541,17 @@ func (e *joinRuntimeStats) Tp() int { return execdetails.TpJoinRuntimeStats } +func (e *joinRuntimeStats) Clone() execdetails.RuntimeStats { + newJRS := &joinRuntimeStats{ + RuntimeStatsWithConcurrencyInfo: e.RuntimeStatsWithConcurrencyInfo, + applyCache: e.applyCache, + cache: e.cache, + hasHashStat: e.hasHashStat, + hashStat: e.hashStat, + } + return newJRS +} + type hashJoinRuntimeStats struct { fetchAndBuildHashTable time.Duration hashStat hashStatistic diff --git a/executor/load_data.go b/executor/load_data.go index e11137c3916ae..a5db464ce705e 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -80,6 +80,9 @@ func (e *LoadDataExec) Next(ctx context.Context, req *chunk.Chunk) error { // Close implements the Executor Close interface. func (e *LoadDataExec) Close() error { + if e.runtimeStats != nil && e.loadDataInfo != nil && e.loadDataInfo.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.loadDataInfo.stats) + } return nil } diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index 2c4499b14818c..a0d418cc0e441 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -176,7 +176,6 @@ func (e *ParallelNestedLoopApplyExec) Close() error { if e.runtimeStats != nil { runtimeStats := newJoinRuntimeStats() - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, runtimeStats) if e.useCache { var hitRatio float64 if e.cacheAccessCounter > 0 { @@ -187,6 +186,7 @@ func (e *ParallelNestedLoopApplyExec) Close() error { runtimeStats.setCacheInfo(false, 0) } runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", e.concurrency)) + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, runtimeStats) } return err } diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 50bb68a7b5235..b2ba37634a8a4 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -84,7 +84,7 @@ partition p2 values less than (10))`) // Table reader: one partition tk.MustQuery("select * from pt where c > 8").Check(testkit.Rows("9 9")) // Table reader: more than one partition - tk.MustQuery("select * from pt where c < 2 or c >= 9").Check(testkit.Rows("0 0", "9 9")) + tk.MustQuery("select * from pt where c < 2 or c >= 9").Sort().Check(testkit.Rows("0 0", "9 9")) // Index reader tk.MustQuery("select c from pt").Sort().Check(testkit.Rows("0", "2", "4", "6", "7", "9", "")) @@ -96,7 +96,7 @@ partition p2 values less than (10))`) tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt").Sort().Check(testkit.Rows("0 0", "2 2", "4 4", "6 6", "7 7", "9 9", " ")) tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8").Check(testkit.Rows("9 9")) - tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9").Check(testkit.Rows("0 0", "9 9")) + tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9").Sort().Check(testkit.Rows("0 0", "9 9")) // Index Merge tk.MustExec("set @@tidb_enable_index_merge = 1") @@ -377,14 +377,67 @@ func TestOrderByandLimit(t *testing.T) { // regular table tk.MustExec("create table tregular(a int, b int, index idx_a(a))") + // range partition table with int pk + tk.MustExec(`create table trange_intpk(a int primary key, b int) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table with int pk + tk.MustExec("create table thash_intpk(a int primary key, b int) partition by hash(a) partitions 4;") + + // regular table with int pk + tk.MustExec("create table tregular_intpk(a int primary key, b int)") + + // range partition table with clustered index + tk.MustExec(`create table trange_clustered(a int, b int, primary key(a, b) clustered) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table with clustered index + tk.MustExec("create table thash_clustered(a int, b int, primary key(a, b) clustered) partition by hash(a) partitions 4;") + + // regular table with clustered index + tk.MustExec("create table tregular_clustered(a int, b int, primary key(a, b) clustered)") + // generate some random data to be inserted vals := make([]string, 0, 2000) for i := 0; i < 2000; i++ { vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) } + + dedupValsA := make([]string, 0, 2000) + dedupMapA := make(map[int]struct{}, 2000) + for i := 0; i < 2000; i++ { + valA := rand.Intn(1100) + if _, ok := dedupMapA[valA]; ok { + continue + } + dedupValsA = append(dedupValsA, fmt.Sprintf("(%v, %v)", valA, rand.Intn(2000))) + dedupMapA[valA] = struct{}{} + } + + dedupValsAB := make([]string, 0, 2000) + dedupMapAB := make(map[string]struct{}, 2000) + for i := 0; i < 2000; i++ { + val := fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000)) + if _, ok := dedupMapAB[val]; ok { + continue + } + dedupValsAB = append(dedupValsAB, val) + dedupMapAB[val] = struct{}{} + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) tk.MustExec("insert into thash values " + strings.Join(vals, ",")) tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + tk.MustExec("insert into trange_intpk values " + strings.Join(dedupValsA, ",")) + tk.MustExec("insert into thash_intpk values " + strings.Join(dedupValsA, ",")) + tk.MustExec("insert into tregular_intpk values " + strings.Join(dedupValsA, ",")) + tk.MustExec("insert into trange_clustered values " + strings.Join(dedupValsAB, ",")) + tk.MustExec("insert into thash_clustered values " + strings.Join(dedupValsAB, ",")) + tk.MustExec("insert into tregular_clustered values " + strings.Join(dedupValsAB, ",")) // test indexLookUp for i := 0; i < 100; i++ { @@ -398,6 +451,29 @@ func TestOrderByandLimit(t *testing.T) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } + // test indexLookUp with order property pushed down. + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + // Since we only use order by a not order by a, b, the result is not stable when we read both a and b. + // We cut the max element so that the result can be stable. + maxEle := tk.MustQuery(fmt.Sprintf("select ifnull(max(a), 1100) from (select * from tregular use index(idx_a) where a > %v order by a limit %v) t", x, y)).Rows()[0][0] + queryRangePartitionWithLimitHint := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from trange use index(idx_a) where a > %v and a < greatest(%v+1, %v) order by a limit %v", x, x+1, maxEle, y) + queryHashPartitionWithLimitHint := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from thash use index(idx_a) where a > %v and a < greatest(%v+1, %v) order by a limit %v", x, x+1, maxEle, y) + queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v and a < greatest(%v+1, %v) order by a limit %v;", x, x+1, maxEle, y) + require.True(t, tk.HasPlan(queryRangePartitionWithLimitHint, "Limit")) + require.True(t, tk.HasPlan(queryRangePartitionWithLimitHint, "IndexLookUp")) + require.True(t, tk.HasPlan(queryHashPartitionWithLimitHint, "Limit")) + require.True(t, tk.HasPlan(queryHashPartitionWithLimitHint, "IndexLookUp")) + require.True(t, tk.HasPlan(queryRangePartitionWithLimitHint, "TopN")) // but not fully pushed + require.True(t, tk.HasPlan(queryHashPartitionWithLimitHint, "TopN")) + regularResult := tk.MustQuery(queryRegular).Sort().Rows() + tk.MustQuery(queryRangePartitionWithLimitHint).Sort().Check(regularResult) + tk.MustQuery(queryHashPartitionWithLimitHint).Sort().Check(regularResult) + } + // test tableReader for i := 0; i < 100; i++ { // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used @@ -410,6 +486,51 @@ func TestOrderByandLimit(t *testing.T) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } + // test tableReader with order property pushed down. + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryRangePartition := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryHashPartition := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from thash ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + require.True(t, tk.HasPlan(queryRangePartition, "TableReader")) // check if tableReader is used + require.True(t, tk.HasPlan(queryHashPartition, "TableReader")) + require.False(t, tk.HasPlan(queryRangePartition, "Limit")) // check if order property is not pushed + require.False(t, tk.HasPlan(queryHashPartition, "Limit")) + regularResult := tk.MustQuery(queryRegular).Sort().Rows() + tk.MustQuery(queryRangePartition).Sort().Check(regularResult) + tk.MustQuery(queryHashPartition).Sort().Check(regularResult) + + // test int pk + // To be simplified, we only read column a. + queryRangePartition = fmt.Sprintf("select /*+ LIMIT_TO_COP() */ a from trange_intpk use index(primary) where a > %v order by a limit %v", x, y) + queryHashPartition = fmt.Sprintf("select /*+ LIMIT_TO_COP() */ a from thash_intpk use index(primary) where a > %v order by a limit %v", x, y) + queryRegular = fmt.Sprintf("select a from tregular_intpk where a > %v order by a limit %v", x, y) + require.True(t, tk.HasPlan(queryRangePartition, "TableReader")) + require.True(t, tk.HasPlan(queryHashPartition, "TableReader")) + require.True(t, tk.HasPlan(queryRangePartition, "Limit")) // check if order property is not pushed + require.True(t, tk.HasPlan(queryHashPartition, "Limit")) + regularResult = tk.MustQuery(queryRegular).Rows() + tk.MustQuery(queryRangePartition).Check(regularResult) + tk.MustQuery(queryHashPartition).Check(regularResult) + + // test clustered index + queryRangePartition = fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from trange_clustered use index(primary) where a > %v order by a, b limit %v;", x, y) + queryHashPartition = fmt.Sprintf("select /*+ LIMIT_TO_COP() */ * from thash_clustered use index(primary) where a > %v order by a, b limit %v;", x, y) + queryRegular = fmt.Sprintf("select * from tregular_clustered where a > %v order by a, b limit %v;", x, y) + require.True(t, tk.HasPlan(queryRangePartition, "TableReader")) // check if tableReader is used + require.True(t, tk.HasPlan(queryHashPartition, "TableReader")) + require.True(t, tk.HasPlan(queryRangePartition, "Limit")) // check if order property is pushed + require.True(t, tk.HasPlan(queryHashPartition, "Limit")) + require.True(t, tk.HasPlan(queryRangePartition, "TopN")) // but not fully pushed + require.True(t, tk.HasPlan(queryHashPartition, "TopN")) + regularResult = tk.MustQuery(queryRegular).Rows() + tk.MustQuery(queryRangePartition).Check(regularResult) + tk.MustQuery(queryHashPartition).Check(regularResult) + } + // test indexReader for i := 0; i < 100; i++ { // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used @@ -422,6 +543,24 @@ func TestOrderByandLimit(t *testing.T) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } + // test indexReader with order property pushed down. + for i := 0; i < 100; i++ { + // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select a from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryRangePartition := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryHashPartition := fmt.Sprintf("select /*+ LIMIT_TO_COP() */ a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) + require.True(t, tk.HasPlan(queryRangePartition, "IndexReader")) // check if indexReader is used + require.True(t, tk.HasPlan(queryHashPartition, "IndexReader")) + require.True(t, tk.HasPlan(queryRangePartition, "Limit")) // check if order property is pushed + require.True(t, tk.HasPlan(queryHashPartition, "Limit")) + regularResult := tk.MustQuery(queryRegular).Sort().Rows() + tk.MustQuery(queryRangePartition).Sort().Check(regularResult) + tk.MustQuery(queryHashPartition).Sort().Check(regularResult) + } + // test indexMerge for i := 0; i < 100; i++ { // explain select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // check if IndexMerge is used @@ -2834,7 +2973,7 @@ partition p1 values less than (7), partition p2 values less than (10))`) tk.MustExec("alter table p add unique idx(id)") tk.MustExec("insert into p values (1,3), (3,4), (5,6), (7,9)") - tk.MustQuery("select id from p use index (idx)").Check(testkit.Rows("1", "3", "5", "7")) + tk.MustQuery("select id from p use index (idx) order by id").Check(testkit.Rows("1", "3", "5", "7")) } func TestGlobalIndexDoubleRead(t *testing.T) { diff --git a/executor/point_get.go b/executor/point_get.go index e83c5f32e0927..3e3cddb08d9ba 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -80,7 +80,6 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { SnapshotRuntimeStats: snapshotStats, } e.snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) - b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if p.IndexInfo != nil { @@ -194,6 +193,9 @@ func (e *PointGetExecutor) Open(context.Context) error { // Close implements the Executor interface. func (e *PointGetExecutor) Close() error { + if e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.runtimeStats != nil && e.snapshot != nil { e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } diff --git a/executor/replace.go b/executor/replace.go index 221cbf87b2504..f028d5e6db32c 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -43,6 +43,9 @@ type ReplaceExec struct { // Close implements the Executor Close interface. func (e *ReplaceExec) Close() error { e.setMessage() + if e.runtimeStats != nil && e.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + } if e.SelectExec != nil { return e.SelectExec.Close() } diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7f16fd68da5e7..bf78e40e7ac3d 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -773,43 +773,45 @@ func HelperTestAdminShowNextID(t *testing.T, store kv.Storage, str string) { tk.MustExec("create table t(id int, c int)") // Start handle is 1. r := tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 1 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 1 _TIDB_ROWID")) // Row ID is step + 1. tk.MustExec("insert into t values(1, 1)") r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 11 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 11 _TIDB_ROWID")) // Row ID is original + step. for i := 0; i < int(step); i++ { tk.MustExec("insert into t values(10000, 1)") } r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 21 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 21 _TIDB_ROWID")) tk.MustExec("drop table t") // test for a table with the primary key tk.MustExec("create table tt(id int primary key auto_increment, c int)") // Start handle is 1. r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 1 AUTO_INCREMENT")) + r.Check(testkit.Rows("test tt id 1 _TIDB_ROWID", "test tt id 1 AUTO_INCREMENT")) // After rebasing auto ID, row ID is 20 + step + 1. tk.MustExec("insert into tt values(20, 1)") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 31 AUTO_INCREMENT")) + r.Check(testkit.Rows("test tt id 31 _TIDB_ROWID", "test tt id 1 AUTO_INCREMENT")) // test for renaming the table tk.MustExec("drop database if exists test1") tk.MustExec("create database test1") tk.MustExec("rename table test.tt to test1.tt") tk.MustExec("use test1") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 31 AUTO_INCREMENT")) + r.Check(testkit.Rows("test1 tt id 31 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) tk.MustExec("insert test1.tt values ()") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 41 AUTO_INCREMENT")) + r.Check(testkit.Rows("test1 tt id 41 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) tk.MustExec("drop table tt") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int auto_increment primary key nonclustered, b int);") - tk.MustQuery("show table t next_row_id;").Check(testkit.Rows("test1 t _tidb_rowid 1 AUTO_INCREMENT")) + tk.MustQuery("show table t next_row_id;").Check(testkit.Rows( + "test1 t _tidb_rowid 1 _TIDB_ROWID", + "test1 t _tidb_rowid 1 AUTO_INCREMENT")) tk.MustExec("set @@allow_auto_random_explicit_insert = true") @@ -830,19 +832,19 @@ func HelperTestAdminShowNextID(t *testing.T, store kv.Storage, str string) { // Test for a sequence. tk.MustExec("create sequence seq1 start 15 cache 57") r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 15 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 15 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("15")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("16")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select setval(seq1, 96)") r.Check(testkit.Rows("96")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 97 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 97 SEQUENCE")) } func TestNoHistoryWhenDisableRetry(t *testing.T) { diff --git a/executor/table_reader.go b/executor/table_reader.go index ce5a10b125754..984212dcf7328 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -61,7 +61,7 @@ func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Con } type kvRangeBuilder interface { - buildKeyRange(ranges []*ranger.Range) ([]kv.KeyRange, error) + buildKeyRange(ranges []*ranger.Range) ([][]kv.KeyRange, error) buildKeyRangeSeparately(ranges []*ranger.Range) ([]int64, [][]kv.KeyRange, error) } @@ -205,13 +205,13 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { if err != nil { return err } - e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) if len(secondPartRanges) != 0 { kvReq, err = e.buildKVReq(ctx, secondPartRanges) if err != nil { return err } - e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) } return nil } @@ -314,10 +314,10 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra if err != nil { return nil, err } - slices.SortFunc(kvReq.KeyRanges, func(i, j kv.KeyRange) bool { + kvReq.KeyRanges.SortByFunc(func(i, j kv.KeyRange) bool { return bytes.Compare(i.StartKey, j.StartKey) < 0 }) - e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) result, err := e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) if err != nil { @@ -409,7 +409,7 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R if err != nil { return nil, err } - reqBuilder = builder.SetKeyRanges(kvRange) + reqBuilder = builder.SetPartitionKeyRanges(kvRange) } else { reqBuilder = builder.SetHandleRanges(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) } diff --git a/executor/update.go b/executor/update.go index cf0a6ae2e33f4..0068392653cdd 100644 --- a/executor/update.go +++ b/executor/update.go @@ -434,6 +434,7 @@ func (e *UpdateExec) Close() error { if err == nil && txn.Valid() && txn.GetSnapshot() != nil { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, nil) } + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } return e.children[0].Close() } @@ -463,7 +464,6 @@ func (e *UpdateExec) collectRuntimeStatsEnabled() bool { SnapshotRuntimeStats: &txnsnapshot.SnapshotRuntimeStats{}, AllocatorRuntimeStats: autoid.NewAllocatorRuntimeStats(), } - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } return true } diff --git a/executor/write.go b/executor/write.go index 4c36884cc570d..363bb097fd02c 100644 --- a/executor/write.go +++ b/executor/write.go @@ -109,7 +109,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old if err != nil { return false, err } - if err = t.Allocators(sctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true); err != nil { + if err = t.Allocators(sctx).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true); err != nil { return false, err } } diff --git a/expression/integration_test.go b/expression/integration_test.go index bb5bddfa7d9a4..55c8f389a5df3 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -3762,8 +3762,6 @@ func TestExprPushdownBlacklist(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) - tk.MustQuery(`select * from mysql.expr_pushdown_blacklist`).Check(testkit.Rows( - "date_add tiflash DST(daylight saving time) does not take effect in TiFlash date_add")) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/go.mod b/go.mod index 773e081417fe9..f225545bfafd1 100644 --- a/go.mod +++ b/go.mod @@ -72,7 +72,7 @@ require ( github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5 + github.com/pingcap/tipb v0.0.0-20221123081521-2fb828910813 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.13.0 github.com/prometheus/client_model v0.2.0 @@ -86,7 +86,7 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.3-0.20221125022819-f05c6886bbad + github.com/tikv/client-go/v2 v2.0.3-0.20221128025602-81939ec8b2bb github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index 039b9c08f356c..70c92d02d2881 100644 --- a/go.sum +++ b/go.sum @@ -789,8 +789,8 @@ github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c h1:crhkw6DD+07Bg1wYh github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= -github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5 h1:Yoo8j5xQGxjlsC3yt0ndsiAz0WZXED9rzsKmEN0U0DY= -github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20221123081521-2fb828910813 h1:DbmCfCbcavo0JG+gSp0ySvv1ub/c/j3hsnYzyYPzONo= +github.com/pingcap/tipb v0.0.0-20221123081521-2fb828910813/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -930,8 +930,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.3-0.20221125022819-f05c6886bbad h1:5FFJAKukKDTsLqrEeeDgC89aDAteGEFXBHwKRa3wnnQ= -github.com/tikv/client-go/v2 v2.0.3-0.20221125022819-f05c6886bbad/go.mod h1:mQQhAIZ2uJwWXOG2UEz9s9oLGRcNKGGGtDOk4b13Bos= +github.com/tikv/client-go/v2 v2.0.3-0.20221128025602-81939ec8b2bb h1:nfdLYzkmYR2b0XurXSXcVFk06eO//P4VtkqoADX2tR4= +github.com/tikv/client-go/v2 v2.0.3-0.20221128025602-81939ec8b2bb/go.mod h1:kqFVxpx40hAgqqLHXLEPJDM/j6ZVfH5CNdJEtkJvO58= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= diff --git a/infoschema/builder.go b/infoschema/builder.go index 8ff50a8b50435..67153fcc5f3bd 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -693,18 +693,23 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i ConvertCharsetCollateToLowerCaseIfNeed(tblInfo) ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) - if len(allocs) == 0 { + if len(allocs.Allocs) == 0 { allocs = autoid.NewAllocatorsFromTblInfo(b.store, dbInfo.ID, tblInfo) } else { tblVer := autoid.AllocOptionTableInfoVersion(tblInfo.Version) switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: idCacheOpt := autoid.CustomAutoIncCacheOption(tblInfo.AutoIdCache) - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, tblVer, idCacheOpt) - allocs = append(allocs, newAlloc) + // If the allocator type might be AutoIncrementType, create both AutoIncrementType + // and RowIDAllocType allocator for it. Because auto id and row id could share the same allocator. + // Allocate auto id may route to allocate row id, if row id allocator is nil, the program panic! + for _, tp := range [2]autoid.AllocatorType{autoid.AutoIncrementType, autoid.RowIDAllocType} { + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), tp, tblVer, idCacheOpt) + allocs = allocs.Append(newAlloc) + } case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) + allocs = allocs.Append(newAlloc) case model.ActionModifyColumn: // Change column attribute from auto_increment to auto_random. if tblInfo.ContainsAutoRandomBits() && allocs.Get(autoid.AutoRandomType) == nil { @@ -713,7 +718,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType }) newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) + allocs = allocs.Append(newAlloc) } } } diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 040dbd239eef6..11c9bd8b0e7af 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -270,7 +270,7 @@ func (is *infoSchema) TableByID(id int64) (val table.Table, ok bool) { func (is *infoSchema) AllocByID(id int64) (autoid.Allocators, bool) { tbl, ok := is.TableByID(id) if !ok { - return nil, false + return autoid.Allocators{}, false } return tbl.Allocators(nil), true } diff --git a/infoschema/tables.go b/infoschema/tables.go index 8df6cab9cb84c..d76d8b8be60b0 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -2123,7 +2123,7 @@ func (it *infoschemaTable) UpdateRecord(gctx context.Context, ctx sessionctx.Con // Allocators implements table.Table Allocators interface. func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators { - return nil + return autoid.Allocators{} } // Meta implements table.Table Meta interface. @@ -2206,7 +2206,7 @@ func (vt *VirtualTable) UpdateRecord(ctx context.Context, sctx sessionctx.Contex // Allocators implements table.Table Allocators interface. func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators { - return nil + return autoid.Allocators{} } // Meta implements table.Table Meta interface. diff --git a/kv/BUILD.bazel b/kv/BUILD.bazel index 32dd9f1474179..992d99d382e42 100644 --- a/kv/BUILD.bazel +++ b/kv/BUILD.bazel @@ -48,6 +48,7 @@ go_library( "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//util", "@com_github_tikv_pd_client//:client", + "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/kv/kv.go b/kv/kv.go index 72e8111f0343d..8263746093a5c 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -15,6 +15,7 @@ package kv import ( + "bytes" "context" "crypto/tls" "time" @@ -33,6 +34,7 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" pd "github.com/tikv/pd/client" + "golang.org/x/exp/slices" ) // UnCommitIndexKVFlag uses to indicate the index key/value is no need to commit. @@ -335,13 +337,148 @@ func (t StoreType) Name() string { return "unspecified" } +// KeyRanges wrap the ranges for partitioned table cases. +// We might send ranges from different in the one request. +type KeyRanges struct { + ranges [][]KeyRange + + isPartitioned bool +} + +// NewPartitionedKeyRanges constructs a new RequestRange for partitioned table. +func NewPartitionedKeyRanges(ranges [][]KeyRange) *KeyRanges { + return &KeyRanges{ + ranges: ranges, + isPartitioned: true, + } +} + +// NewNonParitionedKeyRanges constructs a new RequestRange for a non partitioned table. +func NewNonParitionedKeyRanges(ranges []KeyRange) *KeyRanges { + return &KeyRanges{ + ranges: [][]KeyRange{ranges}, + isPartitioned: false, + } +} + +// FirstPartitionRange returns the the result of first range. +// We may use some func to generate ranges for both partitioned table and non partitioned table. +// This method provides a way to fallback to non-partitioned ranges. +func (rr *KeyRanges) FirstPartitionRange() []KeyRange { + if len(rr.ranges) == 0 { + return []KeyRange{} + } + return rr.ranges[0] +} + +// SetToNonPartitioned set the status to non-partitioned. +func (rr *KeyRanges) SetToNonPartitioned() error { + if len(rr.ranges) > 1 { + return errors.Errorf("you want to change the partitioned ranges to non-partitioned ranges") + } + rr.isPartitioned = false + return nil +} + +// AppendSelfTo appends itself to another slice. +func (rr *KeyRanges) AppendSelfTo(ranges []KeyRange) []KeyRange { + for _, r := range rr.ranges { + ranges = append(ranges, r...) + } + return ranges +} + +// SortByFunc sorts each partition's ranges. +// Since the ranges are sorted in most cases, we check it first. +func (rr *KeyRanges) SortByFunc(sortFunc func(i, j KeyRange) bool) { + if !slices.IsSortedFunc(rr.ranges, func(i, j []KeyRange) bool { + // A simple short-circuit since the empty range actually won't make anything wrong. + if len(i) == 0 || len(j) == 0 { + return true + } + return sortFunc(i[0], j[0]) + }) { + slices.SortFunc(rr.ranges, func(i, j []KeyRange) bool { + if len(i) == 0 { + return true + } + if len(j) == 0 { + return false + } + return sortFunc(i[0], j[0]) + }) + } + for i := range rr.ranges { + if !slices.IsSortedFunc(rr.ranges[i], sortFunc) { + slices.SortFunc(rr.ranges[i], sortFunc) + } + } +} + +// ForEachPartitionWithErr runs the func for each partition with an error check. +func (rr *KeyRanges) ForEachPartitionWithErr(theFunc func([]KeyRange) error) (err error) { + for i := range rr.ranges { + err = theFunc(rr.ranges[i]) + if err != nil { + return err + } + } + return nil +} + +// ForEachPartition runs the func for each partition without error check. +func (rr *KeyRanges) ForEachPartition(theFunc func([]KeyRange)) { + for i := range rr.ranges { + theFunc(rr.ranges[i]) + } +} + +// PartitionNum returns how many partition is involved in the ranges. +func (rr *KeyRanges) PartitionNum() int { + return len(rr.ranges) +} + +// IsFullySorted checks whether the ranges are sorted inside partition and each partition is also sorated. +func (rr *KeyRanges) IsFullySorted() bool { + sortedByPartition := slices.IsSortedFunc(rr.ranges, func(i, j []KeyRange) bool { + // A simple short-circuit since the empty range actually won't make anything wrong. + if len(i) == 0 || len(j) == 0 { + return true + } + return bytes.Compare(i[0].StartKey, j[0].StartKey) < 0 + }) + if !sortedByPartition { + return false + } + for _, ranges := range rr.ranges { + if !slices.IsSortedFunc(ranges, func(i, j KeyRange) bool { + return bytes.Compare(i.StartKey, j.StartKey) < 0 + }) { + return false + } + } + return true +} + +// TotalRangeNum returns how many ranges there are. +func (rr *KeyRanges) TotalRangeNum() int { + ret := 0 + for _, r := range rr.ranges { + ret += len(r) + } + return ret +} + // Request represents a kv request. type Request struct { // Tp is the request type. - Tp int64 - StartTs uint64 - Data []byte - KeyRanges []KeyRange + Tp int64 + StartTs uint64 + Data []byte + + // KeyRanges makes sure that the request is sent first by partition then by region. + // When the table is small, it's possible that multiple partitions are in the same region. + KeyRanges *KeyRanges // For PartitionTableScan used by tiflash. PartitionIDAndRanges []PartitionIDAndRanges diff --git a/kv/option.go b/kv/option.go index ee5354141cd7b..a0e658f45aade 100644 --- a/kv/option.go +++ b/kv/option.go @@ -167,4 +167,6 @@ const ( InternalTxnBR = InternalTxnTools // InternalTxnTrace handles the trace statement. InternalTxnTrace = "Trace" + // InternalTxnTTL is the type of TTL usage + InternalTxnTTL = "TTL" ) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index def3245bb2da3..2a181a088d802 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -205,16 +205,36 @@ type Allocator interface { } // Allocators represents a set of `Allocator`s. -type Allocators []Allocator +type Allocators struct { + SepAutoInc bool + Allocs []Allocator +} // NewAllocators packs multiple `Allocator`s into Allocators. -func NewAllocators(allocators ...Allocator) Allocators { - return allocators +func NewAllocators(sepAutoInc bool, allocators ...Allocator) Allocators { + return Allocators{ + SepAutoInc: sepAutoInc, + Allocs: allocators, + } +} + +// Append add an allocator to the allocators. +func (all Allocators) Append(a Allocator) Allocators { + return Allocators{ + SepAutoInc: all.SepAutoInc, + Allocs: append(all.Allocs, a), + } } // Get returns the Allocator according to the AllocatorType. func (all Allocators) Get(allocType AllocatorType) Allocator { - for _, a := range all { + if !all.SepAutoInc { + if allocType == AutoIncrementType { + allocType = RowIDAllocType + } + } + + for _, a := range all.Allocs { if a.GetType() == allocType { return a } @@ -224,13 +244,16 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { // Filter filters all the allocators that match pred. func (all Allocators) Filter(pred func(Allocator) bool) Allocators { - var ret Allocators - for _, a := range all { + var ret []Allocator + for _, a := range all.Allocs { if pred(a) { ret = append(ret, a) } } - return ret + return Allocators{ + SepAutoInc: all.SepAutoInc, + Allocs: ret, + } } type allocator struct { @@ -593,7 +616,7 @@ func NewAllocator(store kv.Storage, dbID, tbID int64, isUnsigned bool, } // Use the MySQL compatible AUTO_INCREMENT mode. - if allocType == RowIDAllocType && alloc.customStep && alloc.step == 1 { + if allocType == AutoIncrementType && alloc.customStep && alloc.step == 1 && alloc.tbVersion >= model.TableInfoVersion5 { alloc1 := newSinglePointAlloc(store, dbID, tbID, isUnsigned) if alloc1 != nil { return alloc1 @@ -630,6 +653,10 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, idCacheOpt, tblVer) allocs = append(allocs, alloc) } + if hasAutoIncID { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, idCacheOpt, tblVer) + allocs = append(allocs, alloc) + } hasAutoRandID := tblInfo.ContainsAutoRandomBits() if hasAutoRandID { alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt, tblVer) @@ -638,7 +665,7 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T if tblInfo.IsSequence() { allocs = append(allocs, NewSequenceAllocator(store, dbID, tblInfo.ID, tblInfo.Sequence)) } - return NewAllocators(allocs...) + return NewAllocators(tblInfo.SepAutoInc(), allocs...) } // Alloc implements autoid.Allocator Alloc interface. @@ -839,7 +866,7 @@ func (alloc *allocator) alloc4Signed(ctx context.Context, n uint64, increment, o var newBase, newEnd int64 startTime := time.Now() nextStep := alloc.step - if !alloc.customStep { + if !alloc.customStep && alloc.end > 0 { // Although it may skip a segment here, we still think it is consumed. consumeDur := startTime.Sub(alloc.lastAllocTime) nextStep = NextStep(alloc.step, consumeDur) diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index 6133dfdfc3cb2..e524397e0041f 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -152,7 +152,11 @@ func (sp *singlePointAlloc) resetConn() { // Close grpc.ClientConn to release resource. if grpcConn != nil { err := grpcConn.Close() - logutil.BgLogger().Info("[autoid client] AllocAutoID grpc error, reconnect", zap.Error(err)) + if err != nil { + logutil.BgLogger().Warn("[autoid client] AllocAutoID grpc error, reconnect", zap.Error(err)) + } else { + logutil.BgLogger().Info("[autoid client] AllocAutoID grpc error, reconnect") + } } } @@ -232,5 +236,5 @@ func (sp *singlePointAlloc) NextGlobalAutoID() (int64, error) { } func (*singlePointAlloc) GetType() AllocatorType { - return RowIDAllocType + return AutoIncrementType } diff --git a/meta/meta_autoid.go b/meta/meta_autoid.go index 18d384b2b25a7..5763aa268051a 100644 --- a/meta/meta_autoid.go +++ b/meta/meta_autoid.go @@ -102,7 +102,7 @@ type autoIDAccessors struct { access autoIDAccessor } -const sepAutoIncVer = model.TableInfoVersion4 + 1 +const sepAutoIncVer = model.TableInfoVersion5 // Get implements the interface AutoIDAccessors. func (a *autoIDAccessors) Get() (autoIDs AutoIDGroup, err error) { diff --git a/metrics/telemetry.go b/metrics/telemetry.go index 486a2d43a0c59..7460ca5ceb04c 100644 --- a/metrics/telemetry.go +++ b/metrics/telemetry.go @@ -155,6 +155,13 @@ var ( Name: "flashback_cluster_usage", Help: "Counter of usage of flashback cluster", }) + TelemetryCompactPartitionCnt = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "telemetry", + Name: "compact_partition_usage", + Help: "Counter of compact table partition", + }) ) // readCounter reads the value of a prometheus.Counter. @@ -254,6 +261,7 @@ type TablePartitionUsageCounter struct { TablePartitionCreateIntervalPartitionsCnt int64 `json:"table_partition_create_interval_partitions_cnt"` TablePartitionAddIntervalPartitionsCnt int64 `json:"table_partition_add_interval_partitions_cnt"` TablePartitionDropIntervalPartitionsCnt int64 `json:"table_partition_drop_interval_partitions_cnt"` + TablePartitionComactCnt int64 `json:"table_TablePartitionComactCnt"` } // ExchangePartitionUsageCounter records the usages of exchange partition. @@ -291,6 +299,7 @@ func (c TablePartitionUsageCounter) Cal(rhs TablePartitionUsageCounter) TablePar TablePartitionCreateIntervalPartitionsCnt: c.TablePartitionCreateIntervalPartitionsCnt - rhs.TablePartitionCreateIntervalPartitionsCnt, TablePartitionAddIntervalPartitionsCnt: c.TablePartitionAddIntervalPartitionsCnt - rhs.TablePartitionAddIntervalPartitionsCnt, TablePartitionDropIntervalPartitionsCnt: c.TablePartitionDropIntervalPartitionsCnt - rhs.TablePartitionDropIntervalPartitionsCnt, + TablePartitionComactCnt: c.TablePartitionComactCnt - rhs.TablePartitionComactCnt, } } @@ -326,6 +335,7 @@ func GetTablePartitionCounter() TablePartitionUsageCounter { TablePartitionCreateIntervalPartitionsCnt: readCounter(TelemetryTablePartitionCreateIntervalPartitionsCnt), TablePartitionAddIntervalPartitionsCnt: readCounter(TelemetryTablePartitionAddIntervalPartitionsCnt), TablePartitionDropIntervalPartitionsCnt: readCounter(TelemetryTablePartitionDropIntervalPartitionsCnt), + TablePartitionComactCnt: readCounter(TelemetryCompactPartitionCnt), } } diff --git a/parser/model/model.go b/parser/model/model.go index e38e5b368f4a1..ba7c46bcd6333 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -446,14 +446,16 @@ const ( // However, the convert is missed in some scenarios before v2.1.9, so for all those tables prior to TableInfoVersion3, their // charsets / collations will be converted to lower-case while loading from the storage. TableInfoVersion3 = uint16(3) - // TableInfoVersion4 indicates that the auto_increment allocator in TiDB has been separated from - // _tidb_rowid allocator. This version is introduced to preserve the compatibility of old tables: - // the tables with version < TableInfoVersion4 still use a single allocator for auto_increment and _tidb_rowid. - // Also see https://github.com/pingcap/tidb/issues/982. + // TableInfoVersion4 is not used. TableInfoVersion4 = uint16(4) + // TableInfoVersion5 indicates that the auto_increment allocator in TiDB has been separated from + // _tidb_rowid allocator when AUTO_ID_CACHE is 1. This version is introduced to preserve the compatibility of old tables: + // the tables with version <= TableInfoVersion4 still use a single allocator for auto_increment and _tidb_rowid. + // Also see https://github.com/pingcap/tidb/issues/982. + TableInfoVersion5 = uint16(5) // CurrLatestTableInfoVersion means the latest table info in the current TiDB. - CurrLatestTableInfoVersion = TableInfoVersion4 + CurrLatestTableInfoVersion = TableInfoVersion5 ) // ExtraHandleName is the name of ExtraHandle Column. @@ -552,6 +554,11 @@ type TableInfo struct { TTLInfo *TTLInfo `json:"ttl_info"` } +// SepAutoInc decides whether _rowid and auto_increment id use separate allocator. +func (t *TableInfo) SepAutoInc() bool { + return t.Version >= TableInfoVersion5 && t.AutoIdCache == 1 +} + // TableCacheStatusType is the type of the table cache status type TableCacheStatusType int diff --git a/parser/mysql/type.go b/parser/mysql/type.go index c54d0f8984b63..f79be8ab30d96 100644 --- a/parser/mysql/type.go +++ b/parser/mysql/type.go @@ -43,7 +43,7 @@ const ( TypeLongBlob byte = 0xfb TypeBlob byte = 0xfc TypeVarString byte = 0xfd - TypeString byte = 0xfe + TypeString byte = 0xfe /* TypeString is char type */ TypeGeometry byte = 0xff ) diff --git a/parser/types/field_type.go b/parser/types/field_type.go index efc1c20d0c18f..f555c2dc70acf 100644 --- a/parser/types/field_type.go +++ b/parser/types/field_type.go @@ -77,6 +77,16 @@ func (ft *FieldType) IsDecimalValid() bool { return true } +// IsVarLengthType Determine whether the column type is a variable-length type +func (ft *FieldType) IsVarLengthType() bool { + switch ft.tp { + case mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeJSON, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + return true + default: + return false + } +} + // GetType returns the type of the FieldType. func (ft *FieldType) GetType() byte { return ft.tp diff --git a/planner/cascades/testdata/integration_suite_in.json b/planner/cascades/testdata/integration_suite_in.json index 569cb12860ac3..5533e6c672fcb 100644 --- a/planner/cascades/testdata/integration_suite_in.json +++ b/planner/cascades/testdata/integration_suite_in.json @@ -142,7 +142,7 @@ { "name": "TestCascadePlannerHashedPartTable", "cases": [ - "select * from pt1" + "select * from pt1 order by a" ] }, { diff --git a/planner/cascades/testdata/integration_suite_out.json b/planner/cascades/testdata/integration_suite_out.json index e8d98a41ec557..262a825256e41 100644 --- a/planner/cascades/testdata/integration_suite_out.json +++ b/planner/cascades/testdata/integration_suite_out.json @@ -1198,17 +1198,18 @@ "Name": "TestCascadePlannerHashedPartTable", "Cases": [ { - "SQL": "select * from pt1", + "SQL": "select * from pt1 order by a", "Plan": [ - "TableReader_5 10000.00 root partition:all data:TableFullScan_6", - "└─TableFullScan_6 10000.00 cop[tikv] table:pt1 keep order:false, stats:pseudo" + "Sort_11 10000.00 root test.pt1.a", + "└─TableReader_9 10000.00 root partition:all data:TableFullScan_10", + " └─TableFullScan_10 10000.00 cop[tikv] table:pt1 keep order:false, stats:pseudo" ], "Result": [ - "4 40", "1 10", - "5 50", "2 20", - "3 30" + "3 30", + "4 40", + "5 50" ] } ] diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 9db73e6d5fcb8..214148fbf198a 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -48,6 +48,7 @@ go_library( "rule_aggregation_skew_rewrite.go", "rule_build_key_info.go", "rule_column_pruning.go", + "rule_count_star_rewriter.go", "rule_decorrelate.go", "rule_eliminate_projection.go", "rule_generate_column_substitute.go", diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index a26f77d523278..a3fbd62cf12e3 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -747,7 +747,8 @@ func (e *Explain) RenderResult() error { } if pp.SCtx().GetSessionVars().CostModelVersion == modelVer2 { // output cost formula and factor costs through warning under model ver2 and true_card_cost mode for cost calibration. - trace, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption()) + cost, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption()) + trace := cost.trace pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("cost formula: %v", trace.formula)) data, err := json.Marshal(trace.factorCosts) if err != nil { @@ -942,7 +943,9 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (string, string, string, st if e.ctx != nil && e.ctx.GetSessionVars().CostModelVersion == modelVer2 { costVer2, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption()) estCost = strconv.FormatFloat(costVer2.cost, 'f', 2, 64) - costFormula = costVer2.formula + if costVer2.trace != nil { + costFormula = costVer2.trace.formula + } } else { planCost, _ := getPlanCost(pp, property.RootTaskType, NewDefaultPlanCostOption()) estCost = strconv.FormatFloat(planCost, 'f', 2, 64) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5d90ff851be76..52eab05629d59 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -2357,6 +2357,7 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper physicalTableID: ds.physicalTableID, tblColHists: ds.TblColHists, pkIsHandleCol: ds.getPKIsHandleCol(), + constColsByCond: path.ConstCols, prop: prop, }.Init(ds.ctx, ds.blockOffset) statsTbl := ds.statisticTable diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 5dfa93186826f..c6aec17f21e6d 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -406,7 +406,7 @@ func (e *mppTaskGenerator) constructMPPBuildTaskReqForPartitionedTable(ts *Physi return nil, nil, errors.Trace(err) } partitionIDAndRanges[i].ID = pid - partitionIDAndRanges[i].KeyRanges = kvRanges + partitionIDAndRanges[i].KeyRanges = kvRanges.FirstPartitionRange() allPartitionsIDs[i] = pid } return &kv.MPPBuildTasksRequest{PartitionIDAndRanges: partitionIDAndRanges}, allPartitionsIDs, nil @@ -417,5 +417,5 @@ func (e *mppTaskGenerator) constructMPPBuildTaskForNonPartitionTable(ts *Physica if err != nil { return nil, errors.Trace(err) } - return &kv.MPPBuildTasksRequest{KeyRanges: kvRanges}, nil + return &kv.MPPBuildTasksRequest{KeyRanges: kvRanges.FirstPartitionRange()}, nil } diff --git a/planner/core/integration_partition_test.go b/planner/core/integration_partition_test.go index 7823f18474ad1..f1b915b66d038 100644 --- a/planner/core/integration_partition_test.go +++ b/planner/core/integration_partition_test.go @@ -1458,12 +1458,12 @@ func TestRangeColumnsExpr(t *testing.T) { "TableReader 1.14 root partition:p5,p12 data:Selection", "└─Selection 1.14 cop[tikv] in(rce.t.a, 4, 14), in(rce.t.b, NULL, 10)", " └─TableFullScan 21.00 cop[tikv] table:t keep order:false")) - tk.MustQuery(`select * from tref where a in (4,14) and b in (null,10)`).Check(testkit.Rows( - "4 10 3", - "14 10 4")) - tk.MustQuery(`select * from t where a in (4,14) and b in (null,10)`).Check(testkit.Rows( - "4 10 3", - "14 10 4")) + tk.MustQuery(`select * from tref where a in (4,14) and b in (null,10)`).Sort().Check(testkit.Rows( + "14 10 4", + "4 10 3")) + tk.MustQuery(`select * from t where a in (4,14) and b in (null,10)`).Sort().Check(testkit.Rows( + "14 10 4", + "4 10 3")) tk.MustQuery(`explain format = 'brief' select * from t where a in (4,14) and (b in (11,10) OR b is null)`).Check(testkit.Rows( "TableReader 3.43 root partition:p1,p5,p6,p11,p12 data:Selection", "└─Selection 3.43 cop[tikv] in(rce.t.a, 4, 14), or(in(rce.t.b, 11, 10), isnull(rce.t.b))", diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index ff2ec6b6fd631..815eb5b08da60 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1365,16 +1365,18 @@ func TestViewHintScope(t *testing.T) { tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop view if exists v, v1, v2, v3") - tk.MustExec("drop table if exists t, t1, t2") + tk.MustExec("drop view if exists v, v1, v2, v3, v4") + tk.MustExec("drop table if exists t, t1, t2, t3, t4") tk.MustExec("create table t(a int, b int);") tk.MustExec("create table t1(a int, b int);") tk.MustExec("create table t2(a int, b int);") tk.MustExec("create table t3(a int, b int)") + tk.MustExec("create table t4(a int, b int, index idx_a(a), index idx_b(b))") tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 join t3 where t1.b=t2.b and t2.a = t3.a group by t2.a) tt on t.a = tt.a;") tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") tk.MustExec("create definer='root'@'localhost' view v3 as select /*+ merge_join(t) */ t.a, t.b from t join (select /*+ stream_agg() */ count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v4 as select * from t4 where a > 2 and b > 3;") var input []string var output []struct { @@ -1403,7 +1405,7 @@ func TestViewHintWithBinding(t *testing.T) { tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop view if exists v, v1") - tk.MustExec("drop table if exists t, t1, t2") + tk.MustExec("drop table if exists t, t1, t2, t3") tk.MustExec("create table t(a int, b int);") tk.MustExec("create table t1(a int, b int);") tk.MustExec("create table t2(a int, b int);") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 8ec9b8f48fc06..ebabe1ba95795 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -204,6 +204,7 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu b.optFlag |= flagPredicatePushDown b.optFlag |= flagEliminateAgg b.optFlag |= flagEliminateProjection + b.optFlag |= flagCountStarRewriter if b.ctx.GetSessionVars().EnableSkewDistinctAgg { b.optFlag |= flagSkewDistinctAgg @@ -4437,8 +4438,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as // Because of the nested views, so we should check the left table list in hint when build the data source from the view inside the current view. currentQBNameMap4View[qbName] = viewQBNameHintTable[1:] currentViewHints[qbName] = b.hintProcessor.QbHints4View[qbName] - delete(b.hintProcessor.QbNameMap4View, qbName) - delete(b.hintProcessor.QbHints4View, qbName) + b.hintProcessor.QbNameUsed4View[qbName] = struct{}{} } } return b.BuildDataSourceFromView(ctx, dbName, tableInfo, currentQBNameMap4View, currentViewHints) @@ -5041,6 +5041,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model. hintProcessor.QbNameMap4View = qbNameMap4View hintProcessor.QbHints4View = viewHints + hintProcessor.QbNameUsed4View = make(map[string]struct{}) hintProcessor.QbHints = currentQbHints hintProcessor.QbNameMap = currentQbNameMap @@ -5049,6 +5050,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model. b.hintProcessor = hintProcessor b.ctx.GetSessionVars().PlannerSelectBlockAsName = make([]ast.HintTable, hintProcessor.MaxSelectStmtOffset()+1) defer func() { + b.hintProcessor.HandleUnusedViewHints() b.hintProcessor = originHintProcessor b.ctx.GetSessionVars().PlannerSelectBlockAsName = originPlannerSelectBlockAsName }() diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index fe3219fcdaf21..d96f70462cc33 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -72,6 +72,7 @@ const ( flagSyncWaitStatsLoadPoint flagJoinReOrder flagPrunColumnsAgain + flagCountStarRewriter ) var optRuleList = []logicalOptRule{ @@ -94,6 +95,7 @@ var optRuleList = []logicalOptRule{ &syncWaitStatsLoadPoint{}, &joinReOrderSolver{}, &columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver + &countStarRewriter{}, } type logicalOptimizeOp struct { diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 5f07340f626e1..f037d7fe887e7 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -91,12 +91,12 @@ func TestRangeColumnPartitionPruningForIn(t *testing.T) { "└─PartitionUnion 2.00 root ", " ├─HashAgg 1.00 root funcs:count(Column#7)->Column#5", " │ └─IndexReader 1.00 root index:HashAgg", - " │ └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + " │ └─HashAgg 1.00 cop[tikv] funcs:count(test_range_col_in.t1.dt)->Column#7", " │ └─Selection 20.00 cop[tikv] in(test_range_col_in.t1.dt, 2020-11-27 00:00:00.000000, 2020-11-28 00:00:00.000000)", " │ └─IndexFullScan 10000.00 cop[tikv] table:t1, partition:p20201127, index:PRIMARY(id, dt) keep order:false, stats:pseudo", " └─HashAgg 1.00 root funcs:count(Column#10)->Column#5", " └─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#10", + " └─HashAgg 1.00 cop[tikv] funcs:count(test_range_col_in.t1.dt)->Column#10", " └─Selection 20.00 cop[tikv] in(test_range_col_in.t1.dt, 2020-11-27 00:00:00.000000, 2020-11-28 00:00:00.000000)", " └─IndexFullScan 10000.00 cop[tikv] table:t1, partition:p20201128, index:PRIMARY(id, dt) keep order:false, stats:pseudo")) @@ -312,15 +312,15 @@ func TestListPartitionPruner(t *testing.T) { for i, tt := range input { testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - result := tk.MustQuery(tt) + result := tk.MustQuery(tt).Sort() result.Check(testkit.Rows(output[i].Result...)) // If the query doesn't specified the partition, compare the result with normal table if !strings.Contains(tt, "partition(") { - result.Check(tk2.MustQuery(tt).Rows()) + result.Check(tk.MustQuery(tt).Sort().Rows()) valid = true } require.True(t, valid) @@ -393,7 +393,7 @@ func TestListColumnsPartitionPruner(t *testing.T) { indexPlanTree := testdata.ConvertRowsToStrings(indexPlan.Rows()) testdata.OnRecord(func() { output[i].SQL = tt.SQL - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Sort().Rows()) // Test for table without index. output[i].Plan = planTree // Test for table with index. @@ -408,14 +408,14 @@ func TestListColumnsPartitionPruner(t *testing.T) { checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, indexPlanTree) // compare the result. - result := tk.MustQuery(tt.SQL) + result := tk.MustQuery(tt.SQL).Sort() idxResult := tk1.MustQuery(tt.SQL) - result.Check(idxResult.Rows()) + result.Check(idxResult.Sort().Rows()) result.Check(testkit.Rows(output[i].Result...)) // If the query doesn't specified the partition, compare the result with normal table if !strings.Contains(tt.SQL, "partition(") { - result.Check(tk2.MustQuery(tt.SQL).Rows()) + result.Check(tk2.MustQuery(tt.SQL).Sort().Rows()) valid = true } } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 3e41ae2d182d2..36fecf966964c 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -2412,7 +2412,7 @@ func TestPhysicalPlanMemoryTrace(t *testing.T) { ls.ByItems = append(ls.ByItems, &util.ByItems{}) require.Greater(t, ls.MemoryUsage(), size) - //PhysicalProperty + // PhysicalProperty pp := property.PhysicalProperty{} size = pp.MemoryUsage() pp.MPPPartitionCols = append(pp.MPPPartitionCols, &property.MPPPartitionColumn{}) @@ -2460,3 +2460,74 @@ func TestNoDecorrelateHint(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) } } + +func TestCountStarForTikv(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") + tk.MustExec("create table t_pick_row_id (a char(20) not null)") + + // tikv + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestCountStarForTiFlash(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") + tk.MustExec("create table t_pick_row_id (a char(20) not null)") + + // tiflash + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + tableName := tblInfo.Name.L + if tableName == "t" || tableName == "t_pick_row_id" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 5146e63e42c25..ced23204f639d 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -677,7 +677,12 @@ type PhysicalIndexScan struct { // tblColHists contains all columns before pruning, which are used to calculate row-size tblColHists *statistics.HistColl pkIsHandleCol *expression.Column - prop *property.PhysicalProperty + + // constColsByCond records the constant part of the index columns caused by the access conds. + // e.g. the index is (a, b, c) and there's filter a = 1 and b = 2, then the column a and b are const part. + constColsByCond []bool + + prop *property.PhysicalProperty } // Clone implements PhysicalPlan interface. diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 733c4c6845098..1eaa67a6fc06b 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -17,7 +17,7 @@ package core import ( "fmt" "math" - "strings" + "strconv" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" @@ -30,6 +30,11 @@ import ( "github.com/pingcap/tipb/go-tipb" ) +// GetPlanCost returns the cost of this plan. +func GetPlanCost(p PhysicalPlan, taskType property.TaskType, option *PlanCostOption) (float64, error) { + return getPlanCost(p, taskType, option) +} + func getPlanCost(p PhysicalPlan, taskType property.TaskType, option *PlanCostOption) (float64, error) { if p.SCtx().GetSessionVars().CostModelVersion == modelVer2 { planCost, err := p.getPlanCostVer2(taskType, option) @@ -57,7 +62,7 @@ func (p *basePhysicalPlan) getPlanCostVer2(taskType property.TaskType, option *P p.planCostVer2 = sumCostVer2(childCosts...) } p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -79,7 +84,7 @@ func (p *PhysicalSelection) getPlanCostVer2(taskType property.TaskType, option * p.planCostVer2 = sumCostVer2(filterCost, childCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -103,7 +108,7 @@ func (p *PhysicalProjection) getPlanCostVer2(taskType property.TaskType, option p.planCostVer2 = sumCostVer2(childCost, divCostVer2(projCost, concurrency)) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -120,7 +125,7 @@ func (p *PhysicalIndexScan) getPlanCostVer2(taskType property.TaskType, option * p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -149,7 +154,7 @@ func (p *PhysicalTableScan) getPlanCostVer2(taskType property.TaskType, option * } p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -174,7 +179,7 @@ func (p *PhysicalIndexReader) getPlanCostVer2(taskType property.TaskType, option p.planCostVer2 = divCostVer2(sumCostVer2(childCost, netCost), concurrency) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -209,7 +214,7 @@ func (p *PhysicalTableReader) getPlanCostVer2(taskType property.TaskType, option !hasCostFlag(option.CostFlag, CostFlagRecalculate) { // show the real cost in explain-statements p.planCostVer2 = divCostVer2(p.planCostVer2, 1000000000) } - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -255,7 +260,7 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, doubleReadRows := math.Max(indexRows, 1) doubleReadCPUCost := newCostVer2(option, cpuFactor, indexRows*cpuFactor.Value, - "double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) + func() string { return fmt.Sprintf("double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) }) batchSize := float64(p.ctx.GetSessionVars().IndexLookupSize) taskPerBatch := 32.0 // TODO: remove this magic number doubleReadTasks := doubleReadRows / batchSize * taskPerBatch @@ -271,7 +276,7 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, } p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -316,7 +321,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o p.planCostVer2 = sumCostVer2(tableSideCost, sumIndexSideCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -351,15 +356,15 @@ func (p *PhysicalSort) getPlanCostVer2(taskType property.TaskType, option *PlanC if !spill { sortMemCost = newCostVer2(option, memFactor, rows*rowSize*memFactor.Value, - "sortMem(%v*%v*%v)", rows, rowSize, memFactor) + func() string { return fmt.Sprintf("sortMem(%v*%v*%v)", rows, rowSize, memFactor) }) sortDiskCost = zeroCostVer2 } else { sortMemCost = newCostVer2(option, memFactor, float64(memQuota)*memFactor.Value, - "sortMem(%v*%v)", memQuota, memFactor) + func() string { return fmt.Sprintf("sortMem(%v*%v)", memQuota, memFactor) }) sortDiskCost = newCostVer2(option, diskFactor, rows*rowSize*diskFactor.Value, - "sortDisk(%v*%v*%v)", rows, rowSize, diskFactor) + func() string { return fmt.Sprintf("sortDisk(%v*%v*%v)", rows, rowSize, diskFactor) }) } childCost, err := p.children[0].getPlanCostVer2(taskType, option) @@ -369,7 +374,7 @@ func (p *PhysicalSort) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostVer2 = sumCostVer2(childCost, sortCPUCost, sortMemCost, sortDiskCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -390,7 +395,7 @@ func (p *PhysicalTopN) getPlanCostVer2(taskType property.TaskType, option *PlanC topNCPUCost := orderCostVer2(option, rows, N, p.ByItems, cpuFactor) topNMemCost := newCostVer2(option, memFactor, N*rowSize*memFactor.Value, - "topMem(%v*%v*%v)", N, rowSize, memFactor) + func() string { return fmt.Sprintf("topMem(%v*%v*%v)", N, rowSize, memFactor) }) childCost, err := p.children[0].getPlanCostVer2(taskType, option) if err != nil { @@ -399,7 +404,7 @@ func (p *PhysicalTopN) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostVer2 = sumCostVer2(childCost, topNCPUCost, topNMemCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -422,7 +427,7 @@ func (p *PhysicalStreamAgg) getPlanCostVer2(taskType property.TaskType, option * p.planCostVer2 = sumCostVer2(childCost, aggCost, groupCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -441,8 +446,8 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl aggCost := aggCostVer2(option, inputRows, p.AggFuncs, cpuFactor) groupCost := groupCostVer2(option, inputRows, p.GroupByItems, cpuFactor) - hashBuildCost := hashBuildCostVer2(option, outputRows, outputRowSize, p.GroupByItems, cpuFactor, memFactor) - hashProbeCost := hashProbeCostVer2(option, inputRows, p.GroupByItems, cpuFactor) + hashBuildCost := hashBuildCostVer2(option, outputRows, outputRowSize, float64(len(p.GroupByItems)), cpuFactor, memFactor) + hashProbeCost := hashProbeCostVer2(option, inputRows, float64(len(p.GroupByItems)), cpuFactor) childCost, err := p.children[0].getPlanCostVer2(taskType, option) if err != nil { @@ -451,7 +456,7 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl p.planCostVer2 = sumCostVer2(childCost, divCostVer2(sumCostVer2(aggCost, groupCost, hashBuildCost, hashProbeCost), concurrency)) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -481,7 +486,7 @@ func (p *PhysicalMergeJoin) getPlanCostVer2(taskType property.TaskType, option * p.planCostVer2 = sumCostVer2(leftChildCost, rightChildCost, filterCost, groupCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -509,10 +514,10 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P memFactor := getTaskMemFactorVer2(p, taskType) buildFilterCost := filterCostVer2(option, buildRows, buildFilters, cpuFactor) - buildHashCost := hashBuildCostVer2(option, buildRows, buildRowSize, cols2Exprs(buildKeys), cpuFactor, memFactor) + buildHashCost := hashBuildCostVer2(option, buildRows, buildRowSize, float64(len(buildKeys)), cpuFactor, memFactor) probeFilterCost := filterCostVer2(option, probeRows, probeFilters, cpuFactor) - probeHashCost := hashProbeCostVer2(option, probeRows, cols2Exprs(probeKeys), cpuFactor) + probeHashCost := hashProbeCostVer2(option, probeRows, float64(len(probeKeys)), cpuFactor) buildChildCost, err := build.getPlanCostVer2(taskType, option) if err != nil { @@ -531,7 +536,7 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P divCostVer2(sumCostVer2(probeFilterCost, probeHashCost), tidbConcurrency)) } p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, option *PlanCostOption, indexJoinType int) (costVer2, error) { @@ -557,7 +562,7 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt } buildTaskCost := newCostVer2(option, cpuFactor, buildRows*10*cpuFactor.Value, - "cpu(%v*10*%v)", buildRows, cpuFactor) + func() string { return fmt.Sprintf("cpu(%v*10*%v)", buildRows, cpuFactor) }) probeFilterCost := filterCostVer2(option, probeRowsTot, probeFilters, cpuFactor) probeChildCost, err := probe.getPlanCostVer2(taskType, option) @@ -568,11 +573,11 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt var hashTableCost costVer2 switch indexJoinType { case 1: // IndexHashJoin - hashTableCost = hashBuildCostVer2(option, buildRows, buildRowSize, cols2Exprs(p.RightJoinKeys), cpuFactor, memFactor) + hashTableCost = hashBuildCostVer2(option, buildRows, buildRowSize, float64(len(p.RightJoinKeys)), cpuFactor, memFactor) case 2: // IndexMergeJoin hashTableCost = newZeroCostVer2(traceCost(option)) default: // IndexJoin - hashTableCost = hashBuildCostVer2(option, probeRowsTot, probeRowSize, cols2Exprs(p.LeftJoinKeys), cpuFactor, memFactor) + hashTableCost = hashBuildCostVer2(option, probeRowsTot, probeRowSize, float64(len(p.LeftJoinKeys)), cpuFactor, memFactor) } // IndexJoin executes a batch of rows at a time, so the actual cost of this part should be @@ -584,7 +589,7 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt p.planCostVer2 = sumCostVer2(buildChildCost, buildFilterCost, buildTaskCost, divCostVer2(sumCostVer2(probeCost, probeFilterCost, hashTableCost), probeConcurrency)) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -631,7 +636,7 @@ func (p *PhysicalApply) getPlanCostVer2(taskType property.TaskType, option *Plan p.planCostVer2 = sumCostVer2(buildChildCost, buildFilterCost, probeCost, probeFilterCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -652,7 +657,7 @@ func (p *PhysicalUnionAll) getPlanCostVer2(taskType property.TaskType, option *P } p.planCostVer2 = divCostVer2(sumCostVer2(childCosts...), concurrency) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -682,7 +687,7 @@ func (p *PhysicalExchangeReceiver) getPlanCostVer2(taskType property.TaskType, o p.planCostVer2 = sumCostVer2(childCost, netCost) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -701,7 +706,7 @@ func (p *PointGetPlan) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostVer2 = netCostVer2(option, 1, rowSize, netFactor) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } // getPlanCostVer2 returns the plan-cost of this sub-plan, which is: @@ -721,41 +726,44 @@ func (p *BatchPointGetPlan) getPlanCostVer2(taskType property.TaskType, option * p.planCostVer2 = netCostVer2(option, rows, rowSize, netFactor) p.planCostInit = true - return p.planCostVer2.label(p), nil + return p.planCostVer2, nil } func scanCostVer2(option *PlanCostOption, rows, rowSize float64, scanFactor costVer2Factor) costVer2 { + if rowSize < 1 { + rowSize = 1 + } return newCostVer2(option, scanFactor, // rows * log(row-size) * scanFactor, log2 from experiments - rows*math.Log2(math.Max(1, rowSize))*scanFactor.Value, - "scan(%v*logrowsize(%v)*%v)", rows, rowSize, scanFactor) + rows*math.Log2(rowSize)*scanFactor.Value, + func() string { return fmt.Sprintf("scan(%v*logrowsize(%v)*%v)", rows, rowSize, scanFactor) }) } func netCostVer2(option *PlanCostOption, rows, rowSize float64, netFactor costVer2Factor) costVer2 { return newCostVer2(option, netFactor, rows*rowSize*netFactor.Value, - "net(%v*rowsize(%v)*%v)", rows, rowSize, netFactor) + func() string { return fmt.Sprintf("net(%v*rowsize(%v)*%v)", rows, rowSize, netFactor) }) } func filterCostVer2(option *PlanCostOption, rows float64, filters []expression.Expression, cpuFactor costVer2Factor) costVer2 { numFuncs := numFunctions(filters) return newCostVer2(option, cpuFactor, rows*numFuncs*cpuFactor.Value, - "cpu(%v*filters(%v)*%v)", rows, numFuncs, cpuFactor) + func() string { return fmt.Sprintf("cpu(%v*filters(%v)*%v)", rows, numFuncs, cpuFactor) }) } func aggCostVer2(option *PlanCostOption, rows float64, aggFuncs []*aggregation.AggFuncDesc, cpuFactor costVer2Factor) costVer2 { return newCostVer2(option, cpuFactor, // TODO: consider types of agg-funcs rows*float64(len(aggFuncs))*cpuFactor.Value, - "agg(%v*aggs(%v)*%v)", rows, len(aggFuncs), cpuFactor) + func() string { return fmt.Sprintf("agg(%v*aggs(%v)*%v)", rows, len(aggFuncs), cpuFactor) }) } func groupCostVer2(option *PlanCostOption, rows float64, groupItems []expression.Expression, cpuFactor costVer2Factor) costVer2 { numFuncs := numFunctions(groupItems) return newCostVer2(option, cpuFactor, rows*numFuncs*cpuFactor.Value, - "group(%v*cols(%v)*%v)", rows, numFuncs, cpuFactor) + func() string { return fmt.Sprintf("group(%v*cols(%v)*%v)", rows, numFuncs, cpuFactor) }) } func numFunctions(exprs []expression.Expression) float64 { @@ -779,35 +787,35 @@ func orderCostVer2(option *PlanCostOption, rows, N float64, byItems []*util.ByIt } exprCost := newCostVer2(option, cpuFactor, rows*float64(numFuncs)*cpuFactor.Value, - "exprCPU(%v*%v*%v)", rows, numFuncs, cpuFactor) + func() string { return fmt.Sprintf("exprCPU(%v*%v*%v)", rows, numFuncs, cpuFactor) }) orderCost := newCostVer2(option, cpuFactor, rows*math.Log2(N)*cpuFactor.Value, - "orderCPU(%v*log(%v)*%v)", rows, N, cpuFactor) + func() string { return fmt.Sprintf("orderCPU(%v*log(%v)*%v)", rows, N, cpuFactor) }) return sumCostVer2(exprCost, orderCost) } -func hashBuildCostVer2(option *PlanCostOption, buildRows, buildRowSize float64, keys []expression.Expression, cpuFactor, memFactor costVer2Factor) costVer2 { +func hashBuildCostVer2(option *PlanCostOption, buildRows, buildRowSize, nKeys float64, cpuFactor, memFactor costVer2Factor) costVer2 { // TODO: 1) consider types of keys, 2) dedicated factor for build-probe hash table hashKeyCost := newCostVer2(option, cpuFactor, - buildRows*float64(len(keys))*cpuFactor.Value, - "hashkey(%v*%v*%v)", buildRows, len(keys), cpuFactor) + buildRows*nKeys*cpuFactor.Value, + func() string { return fmt.Sprintf("hashkey(%v*%v*%v)", buildRows, nKeys, cpuFactor) }) hashMemCost := newCostVer2(option, memFactor, buildRows*buildRowSize*memFactor.Value, - "hashmem(%v*%v*%v)", buildRows, buildRowSize, memFactor) + func() string { return fmt.Sprintf("hashmem(%v*%v*%v)", buildRows, buildRowSize, memFactor) }) hashBuildCost := newCostVer2(option, cpuFactor, buildRows*cpuFactor.Value, - "hashbuild(%v*%v)", buildRows, cpuFactor) + func() string { return fmt.Sprintf("hashbuild(%v*%v)", buildRows, cpuFactor) }) return sumCostVer2(hashKeyCost, hashMemCost, hashBuildCost) } -func hashProbeCostVer2(option *PlanCostOption, probeRows float64, keys []expression.Expression, cpuFactor costVer2Factor) costVer2 { +func hashProbeCostVer2(option *PlanCostOption, probeRows, nKeys float64, cpuFactor costVer2Factor) costVer2 { // TODO: 1) consider types of keys, 2) dedicated factor for build-probe hash table hashKeyCost := newCostVer2(option, cpuFactor, - probeRows*float64(len(keys))*cpuFactor.Value, - "hashkey(%v*%v*%v)", probeRows, len(keys), cpuFactor) + probeRows*nKeys*cpuFactor.Value, + func() string { return fmt.Sprintf("hashkey(%v*%v*%v)", probeRows, nKeys, cpuFactor) }) hashProbeCost := newCostVer2(option, cpuFactor, probeRows*cpuFactor.Value, - "hashprobe(%v*%v)", probeRows, cpuFactor) + func() string { return fmt.Sprintf("hashprobe(%v*%v)", probeRows, cpuFactor) }) return sumCostVer2(hashKeyCost, hashProbeCost) } @@ -815,7 +823,7 @@ func hashProbeCostVer2(option *PlanCostOption, probeRows float64, keys []express func doubleReadCostVer2(option *PlanCostOption, numTasks float64, requestFactor costVer2Factor) costVer2 { return newCostVer2(option, requestFactor, numTasks*requestFactor.Value, - "doubleRead(tasks(%v)*%v)", numTasks, requestFactor) + func() string { return fmt.Sprintf("doubleRead(tasks(%v)*%v)", numTasks, requestFactor) }) } type costVer2Factor struct { @@ -975,19 +983,14 @@ func cols2Exprs(cols []*expression.Column) []expression.Expression { return exprs } -type costVer2 struct { - cost float64 - trace bool // Whether to trace the cost calculation. +type costTrace struct { factorCosts map[string]float64 // map[factorName]cost, used to calibrate the cost model formula string // It used to trace the cost calculation. } -func (c costVer2) label(p PhysicalPlan) costVer2 { - if !c.trace { - return c - } - c.formula = p.ExplainID().String() - return c +type costVer2 struct { + cost float64 + trace *costTrace } func traceCost(option *PlanCostOption) bool { @@ -999,65 +1002,63 @@ func traceCost(option *PlanCostOption) bool { func newZeroCostVer2(trace bool) (ret costVer2) { if trace { - ret.trace = true - ret.factorCosts = make(map[string]float64) - ret.formula = "0" + ret.trace = &costTrace{make(map[string]float64), ""} } return } -func newCostVer2(option *PlanCostOption, factor costVer2Factor, cost float64, - formulaFormat string, formulaArgs ...any) costVer2 { - ret := newZeroCostVer2(traceCost(option)) +func newCostVer2(option *PlanCostOption, factor costVer2Factor, cost float64, lazyFormula func() string) (ret costVer2) { ret.cost = cost - if ret.trace { - ret.factorCosts[factor.Name] = cost - ret.formula = fmt.Sprintf(formulaFormat, formulaArgs...) + if traceCost(option) { + ret.trace = &costTrace{make(map[string]float64), ""} + ret.trace.factorCosts[factor.Name] = cost + ret.trace.formula = lazyFormula() } return ret } -func sumCostVer2(costs ...costVer2) costVer2 { +func sumCostVer2(costs ...costVer2) (ret costVer2) { if len(costs) == 0 { - return newZeroCostVer2(false) + return } - ret := newZeroCostVer2(costs[0].trace) - var subFormulas []string - for _, c := range costs { + for i, c := range costs { ret.cost += c.cost - if ret.trace { - for factor, factorCost := range c.factorCosts { - ret.factorCosts[factor] += factorCost + if c.trace != nil { + if i == 0 { // init + ret.trace = &costTrace{make(map[string]float64), ""} + } + for factor, factorCost := range c.trace.factorCosts { + ret.trace.factorCosts[factor] += factorCost } - subFormulas = append(subFormulas, fmt.Sprintf("(%v)", c.formula)) + if ret.trace.formula != "" { + ret.trace.formula += " + " + } + ret.trace.formula += "(" + c.trace.formula + ")" } } - if ret.trace { - ret.formula = strings.Join(subFormulas, " + ") - } return ret } -func divCostVer2(cost costVer2, denominator float64) costVer2 { - ret := newZeroCostVer2(cost.trace) +func divCostVer2(cost costVer2, denominator float64) (ret costVer2) { ret.cost = cost.cost / denominator - if ret.trace { - for f, c := range cost.factorCosts { - ret.factorCosts[f] = c / denominator + if cost.trace != nil { + ret.trace = &costTrace{make(map[string]float64), ""} + for f, c := range cost.trace.factorCosts { + ret.trace.factorCosts[f] = c / denominator } - ret.formula = fmt.Sprintf("(%v)/%v", cost.formula, denominator) + ret.trace.formula = "(" + cost.trace.formula + ")/" + strconv.FormatFloat(denominator, 'f', 2, 64) } return ret } -func mulCostVer2(cost costVer2, scale float64) costVer2 { - ret := newZeroCostVer2(cost.trace) +func mulCostVer2(cost costVer2, scale float64) (ret costVer2) { ret.cost = cost.cost * scale - if ret.trace { - for f, c := range cost.factorCosts { - ret.factorCosts[f] = c * scale + if cost.trace != nil { + ret.trace = &costTrace{make(map[string]float64), ""} + for f, c := range cost.trace.factorCosts { + ret.trace.factorCosts[f] = c * scale } - ret.formula = fmt.Sprintf("(%v)*%v", cost.formula, scale) + ret.trace.formula = "(" + cost.trace.formula + ")*" + strconv.FormatFloat(scale, 'f', 2, 64) } return ret } diff --git a/planner/core/plan_cost_ver2_test.go b/planner/core/plan_cost_ver2_test.go index 27b5b913980e4..38ee51738d81a 100644 --- a/planner/core/plan_cost_ver2_test.go +++ b/planner/core/plan_cost_ver2_test.go @@ -15,6 +15,7 @@ package core_test import ( + "context" "encoding/json" "fmt" "math" @@ -22,6 +23,11 @@ import ( "strings" "testing" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/planner" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -144,8 +150,8 @@ func TestCostModelShowFormula(t *testing.T) { actual = append(actual, []interface{}{row[0], row[3]}) // id,costFormula } require.Equal(t, actual, [][]interface{}{ - {"TableReader_7", "((Selection_6) + (net(2*rowsize(16)*tidb_kv_net_factor(3.96))))/15"}, - {"└─Selection_6", "(cpu(3*filters(1)*tikv_cpu_factor(49.9))) + (TableFullScan_5)"}, + {"TableReader_7", "(((cpu(3*filters(1)*tikv_cpu_factor(49.9))) + (scan(3*logrowsize(32)*tikv_scan_factor(40.7)))) + (net(2*rowsize(16)*tidb_kv_net_factor(3.96))))/15.00"}, + {"└─Selection_6", "(cpu(3*filters(1)*tikv_cpu_factor(49.9))) + (scan(3*logrowsize(32)*tikv_scan_factor(40.7)))"}, {" └─TableFullScan_5", "scan(3*logrowsize(32)*tikv_scan_factor(40.7))"}, }) } @@ -242,3 +248,35 @@ func TestCostModelTraceVer2(t *testing.T) { require.True(t, ok) } } + +func BenchmarkGetPlanCost(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int);") + + p := parser.New() + sql := "select sum(t1.b), t1.a from t t1, t t2 where t1.a>0 and t2.a>10 and t1.b=t2.b group by t1.a order by t1.a limit 5" + stmt, err := p.ParseOneStmt(sql, "", "") + if err != nil { + b.Fatal(err) + } + sctx := tk.Session() + sctx.GetSessionVars().CostModelVersion = 2 + is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() + plan, _, err := planner.Optimize(context.TODO(), sctx, stmt, is) + if err != nil { + b.Fatal(err) + } + phyPlan := plan.(core.PhysicalPlan) + _, err = core.GetPlanCost(phyPlan, property.RootTaskType, core.NewDefaultPlanCostOption().WithCostFlag(core.CostFlagRecalculate)) + if err != nil { + b.Fatal(err) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, _ = core.GetPlanCost(phyPlan, property.RootTaskType, core.NewDefaultPlanCostOption().WithCostFlag(core.CostFlagRecalculate)) + } +} diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 5eac4a88e88bd..0769a473a6f34 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -336,21 +336,23 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *log appendColumnPruneTraceStep(ds, prunedColumns, opt) // For SQL like `select 1 from t`, tikv's response will be empty if no column is in schema. // So we'll force to push one if schema doesn't have any column. + // There are two situations + // case 1: tiflash. Select a non-empty and narrowest column. + // The main reason is that tiflash is a column storage structure, + // and choosing the narrowest column can reduce the amount of data read as much as possible, + // making the reading efficiency the best. + // case 2: tikv. Select row_id or pk column. + // The main reason is that tikv is a kv store. + // Select the key column for the best read efficiency. if ds.schema.Len() == 0 { var handleCol *expression.Column var handleColInfo *model.ColumnInfo - if ds.table.Type().IsClusterTable() && len(originColumns) > 0 { - // use the first line. - handleCol = originSchemaColumns[0] - handleColInfo = originColumns[0] + // case 1: tiflash + if ds.tableInfo.TiFlashReplica != nil { + handleCol, handleColInfo = preferNotNullColumnFromTable(ds) } else { - if ds.handleCols != nil { - handleCol = ds.handleCols.GetCol(0) - handleColInfo = handleCol.ToInfo() - } else { - handleCol = ds.newExtraHandleSchemaCol() - handleColInfo = model.NewExtraHandleColInfo() - } + // case 2: tikv + handleCol, handleColInfo = preferKeyColumnFromTable(ds, originSchemaColumns, originColumns) } ds.Columns = append(ds.Columns, handleColInfo) ds.schema.Append(handleCol) @@ -658,3 +660,55 @@ func appendItemPruneTraceStep(p LogicalPlan, itemType string, prunedObjects []fm } opt.appendStepToCurrent(p.ID(), p.TP(), reason, action) } + +// pick a not null and narrowest column from table +func preferNotNullColumnFromTable(dataSource *DataSource) (*expression.Column, *model.ColumnInfo) { + var resultColumnInfo *model.ColumnInfo + var resultColumn *expression.Column + + if dataSource.handleCols != nil { + resultColumn = dataSource.handleCols.GetCol(0) + resultColumnInfo = resultColumn.ToInfo() + } else { + resultColumn = dataSource.newExtraHandleSchemaCol() + resultColumnInfo = model.NewExtraHandleColInfo() + } + + for _, columnInfo := range dataSource.tableInfo.Columns { + if columnInfo.FieldType.IsVarLengthType() { + continue + } + if mysql.HasNotNullFlag(columnInfo.GetFlag()) { + if columnInfo.GetFlen() < resultColumnInfo.GetFlen() { + resultColumnInfo = columnInfo + resultColumn = &expression.Column{ + UniqueID: dataSource.ctx.GetSessionVars().AllocPlanColumnID(), + ID: resultColumnInfo.ID, + RetType: resultColumnInfo.FieldType.Clone(), + OrigName: fmt.Sprintf("%s.%s.%s", dataSource.DBName.L, dataSource.tableInfo.Name.L, resultColumnInfo.Name), + } + } + } + } + return resultColumn, resultColumnInfo +} + +func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expression.Column, + originSchemaColumns []*model.ColumnInfo) (*expression.Column, *model.ColumnInfo) { + var resultColumnInfo *model.ColumnInfo + var resultColumn *expression.Column + if dataSource.table.Type().IsClusterTable() && len(originColumns) > 0 { + // use the first column. + resultColumnInfo = originSchemaColumns[0] + resultColumn = originColumns[0] + } else { + if dataSource.handleCols != nil { + resultColumn = dataSource.handleCols.GetCol(0) + resultColumnInfo = resultColumn.ToInfo() + } else { + resultColumn = dataSource.newExtraHandleSchemaCol() + resultColumnInfo = model.NewExtraHandleColInfo() + } + } + return resultColumn, resultColumnInfo +} diff --git a/planner/core/rule_count_star_rewriter.go b/planner/core/rule_count_star_rewriter.go new file mode 100644 index 0000000000000..97835b311d8ff --- /dev/null +++ b/planner/core/rule_count_star_rewriter.go @@ -0,0 +1,121 @@ +// 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 core + +import ( + "context" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/mysql" +) + +/** +The countStarRewriter is used to rewrite + count(*) -> count(not null column) +Attention: +Since count(*) is directly translated into count(1) during grammar parsing, +the rewritten pattern actually matches count(constant) + +Pattern: +LogcialAggregation: count(constant) + | + DataSource + +Optimize: +Table + + +Case1 there are columns from datasource +Query: select count(*) from table where k3=1 +CountStarRewriterRule: pick the narrowest not null column from datasource +Rewritten Query: select count(k3) from table where k3=1 + +Case2 there is no columns from datasource +Query: select count(*) from table +ColumnPruningRule: pick k1 as the narrowest not null column from origin table @Function.preferNotNullColumnFromTable + datasource.columns: k1 +CountStarRewriterRule: rewrite count(*) -> count(k1) +Rewritten Query: select count(k1) from table + +*/ + +type countStarRewriter struct { +} + +func (c *countStarRewriter) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { + return c.countStarRewriter(p, opt) +} + +func (c *countStarRewriter) countStarRewriter(p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { + c.matchPatternAndRewrite(p) + newChildren := make([]LogicalPlan, 0, len(p.Children())) + for _, child := range p.Children() { + newChild, err := c.countStarRewriter(child, opt) + if err != nil { + return nil, err + } + newChildren = append(newChildren, newChild) + } + p.SetChildren(newChildren...) + return p, nil +} + +func (c *countStarRewriter) matchPatternAndRewrite(p LogicalPlan) { + // match pattern agg(count(constant)) -> datasource + agg, ok := p.(*LogicalAggregation) + if !ok || len(agg.GroupByItems) > 0 { + return + } + dataSource, ok := agg.Children()[0].(*DataSource) + if !ok { + return + } + for _, aggFunc := range agg.AggFuncs { + if aggFunc.Name != "count" || len(aggFunc.Args) != 1 || aggFunc.HasDistinct { + continue + } + constExpr, ok := aggFunc.Args[0].(*expression.Constant) + if !ok || constExpr.Value.IsNull() || len(dataSource.Columns) == 0 { + continue + } + // rewrite + rewriteCountConstantToCountColumn(dataSource, aggFunc) + } +} + +// Pick the narrowest and not null column from Data Source +// If there is no not null column in Data Source, the count(constant) will not be rewritten. +func rewriteCountConstantToCountColumn(dataSource *DataSource, aggFunc *aggregation.AggFuncDesc) { + var newAggColumn *expression.Column + for _, columnFromDataSource := range dataSource.schema.Columns { + if columnFromDataSource.GetType().IsVarLengthType() { + continue + } + if mysql.HasNotNullFlag(columnFromDataSource.GetType().GetFlag()) { + if newAggColumn == nil || columnFromDataSource.GetType().GetFlen() < newAggColumn.GetType().GetFlen() { + newAggColumn = columnFromDataSource + } + } + } + if newAggColumn != nil { + // update count(1) -> count(newAggColumn) + aggFunc.Args[0] = newAggColumn + } +} + +func (*countStarRewriter) name() string { + return "count_star_rewriter" +} diff --git a/planner/core/task.go b/planner/core/task.go index c446b4ea27696..ac3d10a323a7b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -24,11 +24,13 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" @@ -959,6 +961,10 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { } needPushDown := len(cols) > 0 if copTask, ok := t.(*copTask); ok && needPushDown && p.canPushDown(copTask.getStoreType()) && len(copTask.rootTaskConds) == 0 { + newTask, changed := p.pushTopNDownToDynamicPartition(copTask) + if changed { + return newTask + } // If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and // push it to table plan. var pushedDownTopN *PhysicalTopN @@ -978,6 +984,138 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { return attachPlan2Task(p, rootTask) } +// pushTopNDownToDynamicPartition is a temp solution for partition table. It actually does the same thing as DataSource's isMatchProp. +// We need to support a more enhanced read strategy in the execution phase. So that we can achieve Limit(TiDB)->Reader(TiDB)->Limit(TiKV/TiFlash)->Scan(TiKV/TiFlash). +// Before that is done, we use this logic to provide a way to keep the order property when reading from TiKV, so that we can use the orderliness of index to speed up the query. +// Here we can change the execution plan to TopN(TiDB)->Reader(TiDB)->Limit(TiKV)->Scan(TiKV).(TiFlash is not supported). +func (p *PhysicalTopN) pushTopNDownToDynamicPartition(copTsk *copTask) (task, bool) { + copTsk = copTsk.copy().(*copTask) + if len(copTsk.rootTaskConds) > 0 { + return nil, false + } + colsProp, ok := GetPropByOrderByItems(p.ByItems) + if !ok { + return nil, false + } + allSameOrder, isDesc := colsProp.AllSameOrder() + if !allSameOrder { + return nil, false + } + checkIndexMatchProp := func(idxCols []*expression.Column, idxColLens []int, constColsByCond []bool, colsProp *property.PhysicalProperty) bool { + // If the number of the by-items is bigger than the index columns. We cannot push down since it must not keep order. + if len(idxCols) < len(colsProp.SortItems) { + return false + } + idxPos := 0 + for _, byItem := range colsProp.SortItems { + found := false + for ; idxPos < len(idxCols); idxPos++ { + if idxColLens[idxPos] == types.UnspecifiedLength && idxCols[idxPos].Equal(p.SCtx(), byItem.Col) { + found = true + idxPos++ + break + } + if len(constColsByCond) == 0 || idxPos > len(constColsByCond) || !constColsByCond[idxPos] { + found = false + break + } + } + if !found { + return false + } + } + return true + } + var ( + idxScan *PhysicalIndexScan + tblScan *PhysicalTableScan + tblInfo *model.TableInfo + err error + ) + if copTsk.indexPlan != nil { + copTsk.indexPlan, err = copTsk.indexPlan.Clone() + if err != nil { + return nil, false + } + finalIdxScanPlan := copTsk.indexPlan + for len(finalIdxScanPlan.Children()) > 0 && finalIdxScanPlan.Children()[0] != nil { + finalIdxScanPlan = finalIdxScanPlan.Children()[0] + } + idxScan = finalIdxScanPlan.(*PhysicalIndexScan) + tblInfo = idxScan.Table + } + if copTsk.tablePlan != nil { + copTsk.tablePlan, err = copTsk.tablePlan.Clone() + if err != nil { + return nil, false + } + finalTblScanPlan := copTsk.tablePlan + for len(finalTblScanPlan.Children()) > 0 { + finalTblScanPlan = finalTblScanPlan.Children()[0] + } + tblScan = finalTblScanPlan.(*PhysicalTableScan) + tblInfo = tblScan.Table + } + + pi := tblInfo.GetPartitionInfo() + if pi == nil { + return nil, false + } + if pi.Type == model.PartitionTypeList { + return nil, false + } + + if !copTsk.indexPlanFinished { + // If indexPlan side isn't finished, there's no selection on the table side. + + propMatched := checkIndexMatchProp(idxScan.IdxCols, idxScan.IdxColLens, idxScan.constColsByCond, colsProp) + if !propMatched { + return nil, false + } + + idxScan.Desc = isDesc + childProfile := copTsk.plan().statsInfo() + newCount := p.Offset + p.Count + stats := deriveLimitStats(childProfile, float64(newCount)) + pushedLimit := PhysicalLimit{ + Count: newCount, + }.Init(p.SCtx(), stats, p.SelectBlockOffset()) + pushedLimit.SetSchema(copTsk.indexPlan.Schema()) + copTsk = attachPlan2Task(pushedLimit, copTsk).(*copTask) + } else if copTsk.indexPlan == nil { + if tblScan.HandleCols == nil { + return nil, false + } + + if tblScan.HandleCols.IsInt() { + pk := tblScan.HandleCols.GetCol(0) + if len(colsProp.SortItems) != 1 || !colsProp.SortItems[0].Col.Equal(p.SCtx(), pk) { + return nil, false + } + } else { + idxCols, idxColLens := expression.IndexInfo2PrefixCols(tblScan.Columns, tblScan.Schema().Columns, tables.FindPrimaryIndex(tblScan.Table)) + matched := checkIndexMatchProp(idxCols, idxColLens, nil, colsProp) + if !matched { + return nil, false + } + } + tblScan.Desc = isDesc + childProfile := copTsk.plan().statsInfo() + newCount := p.Offset + p.Count + stats := deriveLimitStats(childProfile, float64(newCount)) + pushedLimit := PhysicalLimit{ + Count: newCount, + }.Init(p.SCtx(), stats, p.SelectBlockOffset()) + pushedLimit.SetSchema(copTsk.tablePlan.Schema()) + copTsk = attachPlan2Task(pushedLimit, copTsk).(*copTask) + } else { + return nil, false + } + + rootTask := copTsk.convertToRootTask(p.ctx) + return attachPlan2Task(p, rootTask), true +} + func (p *PhysicalProjection) attach2Task(tasks ...task) task { t := tasks[0].copy() if cop, ok := t.(*copTask); ok { diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index e1e3242fb55b5..e2034d2a536b4 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -1083,8 +1083,8 @@ " └─HashAgg_28 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#21, funcs:sum(Column#19)->Column#22, funcs:sum(Column#20)->Column#23, funcs:count(distinct test.t.c)->Column#24", " └─ExchangeReceiver_30 1.00 mpp[tiflash] ", " └─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", - " └─HashAgg_26 1.00 mpp[tiflash] group by:Column#27, funcs:sum(Column#25)->Column#18, funcs:count(Column#26)->Column#19, funcs:count(1)->Column#20", - " └─Projection_35 10000.00 mpp[tiflash] cast(test.t.b, decimal(20,0) BINARY)->Column#25, test.t.a, test.t.c", + " └─HashAgg_26 1.00 mpp[tiflash] group by:Column#28, funcs:sum(Column#25)->Column#18, funcs:count(Column#26)->Column#19, funcs:count(Column#27)->Column#20", + " └─Projection_35 10000.00 mpp[tiflash] cast(test.t.b, decimal(20,0) BINARY)->Column#25, test.t.a, test.t.b, test.t.c", " └─TableFullScan_13 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1102,8 +1102,8 @@ " └─HashAgg_32 1.00 mpp[tiflash] funcs:sum(Column#22)->Column#25, funcs:sum(Column#23)->Column#26, funcs:count(distinct test.t.c)->Column#27, funcs:sum(Column#24)->Column#28", " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", - " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#31, funcs:sum(Column#29)->Column#22, funcs:count(1)->Column#23, funcs:count(Column#30)->Column#24", - " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#29, test.t.a, test.t.c", + " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#32, funcs:sum(Column#29)->Column#22, funcs:count(Column#30)->Column#23, funcs:count(Column#31)->Column#24", + " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#29, test.t.b, test.t.a, test.t.c", " └─TableFullScan_17 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1122,8 +1122,8 @@ " └─HashAgg_32 1.00 mpp[tiflash] funcs:sum(Column#21)->Column#24, funcs:sum(Column#22)->Column#25, funcs:sum(Column#23)->Column#26, funcs:count(distinct test.t.c)->Column#27", " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", - " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#30, funcs:sum(Column#28)->Column#21, funcs:count(1)->Column#22, funcs:count(Column#29)->Column#23", - " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#28, test.t.a, test.t.c", + " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#31, funcs:sum(Column#28)->Column#21, funcs:count(Column#29)->Column#22, funcs:count(Column#30)->Column#23", + " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#28, test.t.b, test.t.a, test.t.c", " └─TableFullScan_17 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1142,8 +1142,8 @@ " └─HashAgg_32 1.00 mpp[tiflash] funcs:sum(Column#25)->Column#29, funcs:max(Column#26)->Column#30, funcs:count(distinct test.t.c)->Column#31, funcs:sum(Column#27)->Column#32, funcs:sum(Column#28)->Column#33", " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", - " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#37, funcs:sum(Column#34)->Column#25, funcs:max(Column#35)->Column#26, funcs:count(1)->Column#27, funcs:count(Column#36)->Column#28", - " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#34, test.t.b, test.t.a, test.t.c", + " └─HashAgg_30 1.00 mpp[tiflash] group by:Column#38, funcs:sum(Column#34)->Column#25, funcs:max(Column#35)->Column#26, funcs:count(Column#36)->Column#27, funcs:count(Column#37)->Column#28", + " └─Projection_42 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#34, test.t.b, test.t.b, test.t.a, test.t.c", " └─TableFullScan_17 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index bcce484092644..df990e52c65eb 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -667,8 +667,8 @@ // Hint for view v1 "explain format = 'brief' select /*+ qb_name(qb_v1_2, v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v1;", "explain format = 'brief' select /*+ qb_name(qb_v1_2, v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2), stream_agg(@qb_v1_2), qb_name(qb_v1_1, v1@sel_1 .@sel_1), merge_join(t@qb_v1_1) */ * from v1;", - "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", - "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2), stream_agg(@qb_v1_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_1 .@sel_1), merge_join(t@qb_v1_1) */ * from v2;", + "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_2 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", + "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_2 .@sel_2), merge_join(t1@qb_v1_2), stream_agg(@qb_v1_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_2 .@sel_1), merge_join(t@qb_v1_1) */ * from v2;", // Hint for view v2 "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2) */ * from v2;", @@ -684,7 +684,6 @@ "explain format = 'brief' select /*+ qb_name(qb_v, v2@sel_1 .v1@sel_2 .v@sel_2 .@sel_2), qb_name(qb_v, v2@sel_1 .v1@sel_2 .v@sel_2 .@sel_1), merge_join(t1@qb_v) */ * from v2;", // Set the unappeared view name - // TODO: add the warning for the unused the view hints "explain format = 'brief' select /*+ qb_name(qb_v1_2, v@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v1;", "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", @@ -694,18 +693,17 @@ // Tht view hint isn't set in the first query block. "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2) */ * from v) t;", - "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v) t;", + "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v.@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v) t;", "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2) */ * from v1) t;", - "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v1) t;", + "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v1.v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v1) t;", - // TODO: add the warning when the view hints don't set in the first query block - "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v) t;", - "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v) t;", - "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v1) t;", - "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v1) t;", + "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v) t;", + "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_2 .@sel_2), qb_name(qb_v_1, v@sel_2 .@sel1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v) t;", + "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_2 . v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v1) t;", + "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_2 . v@sel_2 .@sel_2), qb_name(qb_v_1, v1@sel_2 . v@sel_2 .@sel_1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v1) t;", // Define more tables in one view hint - "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_1 .@sel_1), merge_join(t1@qb_v1_2, t@qb_v1_1) */ * from v2;", + "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2. v1@sel_2 .@sel_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_2 .@sel_1), merge_join(t1@qb_v1_2, t@qb_v1_1), merge_join(t1@qb_v1_2) */ * from v2;", "explain format = 'brief' select /*+ qb_name(qb_v_2, v2@sel_1 . v1@sel_2 . v@sel_2 .@sel_2), qb_name(qb_v_1, v2@sel_1 . v1@sel_2 . v@sel_2 .@sel_1), merge_join(t1@qb_v_2, t3@qb_v_2) */ * from v2;", "explain format = 'brief' select /*+ qb_name(qb_v_2, v2@sel_1 . v1@sel_2 . v@sel_2 .@sel_2), qb_name(qb_v_1, v2@sel_1 . v1@sel_2 . v@sel_2 .@sel_1), merge_join(@qb_v_2 t1, t3) */ * from v2;", @@ -721,7 +719,12 @@ // The view contains the hint when creation "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2) */ * from v3;", - "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2), hash_agg(@qb_v3_2), qb_name(qb_v3_1, v3@sel_1 .@sel_1), hash_join(t@qb_v3_1) */ * from v3;" + "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2), hash_agg(@qb_v3_2), qb_name(qb_v3_1, v3@sel_1 .@sel_1), hash_join(t@qb_v3_1) */ * from v3;", + + // The view is in the CTE + "explain with d1 as (\n select a from (\n select a from (\n select /*+ qb_name(qb, v4) use_index(t4@qb, idx_a) */ a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select /*+ qb_name(qb2, v4) use_index(t4@qb2, idx_b) */ a from v4 where b < 10)\n\nselect * from (select * from d1) as t0 join (select * from d2) as t1;", + "explain with d1 as (\n select a from (\n select a from (\n select a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select a from v4 where b < 10)\n\nselect /*+ qb_name(qb, v4@sel_4) use_index(t4@qb, idx_a) qb_name(qb2, v4@sel_5) use_index(t4@qb, idx_b) */ * from (select * from d1) as t0 join (select * from d2) as t1;", + "explain with d1 as (\n select a from (\n select a from (\n select /*+ qb_name(qb, v5) use_index(t4@qb, idx_a) */ a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select /*+ qb_name(qb2, v4) use_index(t4@qb2, idx_b) */ a from v4 where b < 10)\n\nselect * from (select * from d1) as t0 join (select * from d2) as t1;" ] }, { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 3ab44fabad3ee..f466f20e7e1b7 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1486,7 +1486,7 @@ { "SQL": "explain format = 'brief' SELECT /*+ use_index_merge(t1)*/ COUNT(*) FROM t1 WHERE (key4=42 AND key6 IS NOT NULL) OR (key1=4 AND key3=6)", "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#10", + "StreamAgg 1.00 root funcs:count(test.t1.key1)->Column#10", "└─IndexMerge 0.02 root ", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i4(key4) range:[42,42], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i1(key1) range:[4,4], keep order:false, stats:pseudo", @@ -2415,7 +2415,7 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#5)->Column#4", "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#5", " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" ] }, @@ -2428,7 +2428,7 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#5)->Column#4", "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#5", " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" ] } @@ -2487,7 +2487,7 @@ "Plan": [ "HashAgg_12 1.00 77.75 root funcs:count(Column#6)->Column#4", "└─IndexReader_13 1.00 47.49 root index:HashAgg_5", - " └─HashAgg_5 1.00 680.68 cop[tikv] funcs:count(1)->Column#6", + " └─HashAgg_5 1.00 680.68 cop[tikv] funcs:count(test.t3._tidb_rowid)->Column#6", " └─IndexFullScan_11 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" ] }, @@ -2496,7 +2496,7 @@ "Plan": [ "HashAgg_14 1.00 82.51 root funcs:count(Column#5)->Column#4", "└─TableReader_15 1.00 52.25 root data:HashAgg_6", - " └─HashAgg_6 1.00 752.10 cop[tikv] funcs:count(1)->Column#5", + " └─HashAgg_6 1.00 752.10 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#5", " └─TableFullScan_12 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, @@ -2625,7 +2625,7 @@ "│ └─HashAgg_52 1.00 30974.28 root funcs:count(Column#31)->Column#18", "│ └─TableReader_54 1.00 30944.02 root data:ExchangeSender_53", "│ └─ExchangeSender_53 1.00 464142.64 mpp[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg_43 1.00 464142.64 mpp[tiflash] funcs:count(1)->Column#31", + "│ └─HashAgg_43 1.00 464142.64 mpp[tiflash] funcs:count(test.t1._tidb_rowid)->Column#31", "│ └─TableFullScan_51 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", "└─Projection_20(Probe) 3.00 129648.62 root 1->Column#28", " └─Apply_22 3.00 129648.32 root CARTESIAN left outer join", @@ -4603,28 +4603,30 @@ "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_2 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#24)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#24", "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#23)]", "│ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#23", - "│ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#22)]", - "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", - "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", - "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ │ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#22)]", + "│ │ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", + "│ │ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", @@ -4639,35 +4641,40 @@ "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), merge_join(t1@qb_v1_2), stream_agg(@qb_v1_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_1 .@sel_1), merge_join(t@qb_v1_1) */ * from v2;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_2 .@sel_2), merge_join(t1@qb_v1_2), stream_agg(@qb_v1_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_2 .@sel_1), merge_join(t@qb_v1_1) */ * from v2;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#24)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#24", - "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#23)]", - "│ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#23", - "│ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#22)]", - "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", - "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", - "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.b, test.t.b)]", + "│ ├─MergeJoin(Build) 9980.01 root inner join, left key:test.t.a, right key:Column#23", + "│ │ ├─Sort(Build) 7984.01 root Column#23", + "│ │ │ └─StreamAgg 7984.01 root group by:test.t.a, funcs:count(1)->Column#23", + "│ │ │ └─Sort 12475.01 root test.t.a", + "│ │ │ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ │ │ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ │ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#22)]", + "│ │ │ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", + "│ │ │ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ │ └─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9980.01 root test.t.a", + "│ │ └─TableReader 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4892,7 +4899,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": null + "Warn": [ + "The qb_name hint qb_v1_2 is unused, please check whether the table list in the qb_name hint qb_v1_2 is correct" + ] }, { "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v@sel_1 .@sel_2), merge_join(t1@qb_v1_2) */ * from v2;", @@ -4932,7 +4941,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": null + "Warn": [ + "The qb_name hint qb_v1_2 is unused, please check whether the table list in the qb_name hint qb_v1_2 is correct" + ] }, { "SQL": "explain format = 'brief' select /*+ qb_name(qb_v2_2, vv@sel_1 .@sel_2), merge_join(t1@qb_v2_2) */ * from v2 vv;", @@ -5014,7 +5025,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": null + "Warn": [ + "The qb_name hint qb_v2_2 is unused, please check whether the table list in the qb_name hint qb_v2_2 is correct" + ] }, { "SQL": "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2) */ * from v) t;", @@ -5037,36 +5050,34 @@ " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "The qb_name hint for view only supports to be defined in the first query block", - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name" + "The qb_name hint qb_v_2 is unused, please check whether the table list in the qb_name hint qb_v_2 is correct" ] }, { - "SQL": "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v) t;", + "SQL": "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v.@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#13)]", - "├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", - "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "├─StreamAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", + "│ └─Sort 15593.77 root test.t2.a", + "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ └─TableReader 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "The qb_name hint for view only supports to be defined in the first query block", - "The qb_name hint for view only supports to be defined in the first query block", - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name", - "Hint stream_agg(@`qb_v_2`) is ignored due to unknown query block name", - "Hint merge_join(`t`@`qb_v_1`) is ignored due to unknown query block name" + "The qb_name hint qb_v_1 is unused, please check whether the table list in the qb_name hint qb_v_1 is correct" ] }, { @@ -5099,28 +5110,30 @@ " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "The qb_name hint for view only supports to be defined in the first query block", - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name" + "The qb_name hint qb_v_2 is unused, please check whether the table list in the qb_name hint qb_v_2 is correct" ] }, { - "SQL": "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v1) t;", + "SQL": "explain format = 'brief' select * from (select /*+ qb_name(qb_v_2, v1.v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v1) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#20)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#20", "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#19)]", - "│ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#19", - "│ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ ├─StreamAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#19", + "│ │ └─Sort 15593.77 root test.t2.a", + "│ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ │ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ │ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ │ └─TableReader 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", @@ -5133,81 +5146,80 @@ " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "The qb_name hint for view only supports to be defined in the first query block", - "The qb_name hint for view only supports to be defined in the first query block", - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name", - "Hint stream_agg(@`qb_v_2`) is ignored due to unknown query block name", - "Hint merge_join(`t`@`qb_v_1`) is ignored due to unknown query block name" + "The qb_name hint qb_v_1 is unused, please check whether the table list in the qb_name hint qb_v_1 is correct" ] }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v) t;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#13)]", "├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", - "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ └─TableReader 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name" - ] + "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v) t;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_2 .@sel_2), qb_name(qb_v_1, v@sel_2 .@sel1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#13)]", - "├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", - "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "├─StreamAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", + "│ └─Sort 15593.77 root test.t2.a", + "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ └─TableReader 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name", - "Hint stream_agg(@`qb_v_2`) is ignored due to unknown query block name", - "Hint merge_join(`t`@`qb_v_1`) is ignored due to unknown query block name" + "The qb_name hint qb_v_1 is unused, please check whether the table list in the qb_name hint qb_v_1 is correct" ] }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v1) t;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_2 . v@sel_2 .@sel_2) */ * from (select /*+ merge_join(t1@qb_v_2) */ * from v1) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#20)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#20", "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#19)]", "│ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#19", - "│ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", "│ │ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ │ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ │ └─TableReader 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", @@ -5219,72 +5231,73 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name" - ] + "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v1) t;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_2 . v@sel_2 .@sel_2), qb_name(qb_v_1, v1@sel_2 . v@sel_2 .@sel_1) */ * from (select /*+ merge_join(t1@qb_v_2), stream_agg(@qb_v_2), merge_join(t@qb_v_1) */ * from v1) t;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#20)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#20", - "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#19)]", - "│ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#19", - "│ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.b, test.t.b)]", + "│ ├─MergeJoin(Build) 9980.01 root inner join, left key:test.t.a, right key:Column#19", + "│ │ ├─Sort(Build) 7984.01 root Column#19", + "│ │ │ └─StreamAgg 7984.01 root group by:test.t2.a, funcs:count(1)->Column#19", + "│ │ │ └─Sort 15593.77 root test.t2.a", + "│ │ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ │ │ └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ │ │ ├─Sort(Build) 9980.01 root test.t2.b", + "│ │ │ │ └─TableReader 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ │ └─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9980.01 root test.t.a", + "│ │ └─TableReader 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "Hint merge_join(`t1`@`qb_v_2`) is ignored due to unknown query block name", - "Hint stream_agg(@`qb_v_2`) is ignored due to unknown query block name", - "Hint merge_join(`t`@`qb_v_1`) is ignored due to unknown query block name" - ] + "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2@sel_1 . v1@sel_1 .@sel_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_1 .@sel_1), merge_join(t1@qb_v1_2, t@qb_v1_1) */ * from v2;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1_2, v2. v1@sel_2 .@sel_2), qb_name(qb_v1_1, v2@sel_1 . v1@sel_2 .@sel_1), merge_join(t1@qb_v1_2, t@qb_v1_1), merge_join(t1@qb_v1_2) */ * from v2;", "Plan": [ "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#27)]", "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#27", "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#26)]", "│ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#26", - "│ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#25)]", - "│ │ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#25", - "│ │ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ │ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#25)]", + "│ │ │ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#25", + "│ │ │ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ │ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─Sort(Probe) 9990.00 root test.t1.b", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", "│ ├─TableReader(Build) 9990.00 root data:Selection", "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", @@ -5640,6 +5653,51 @@ "[planner:1815]Join hints are conflict, you can only specify one type of join", "[planner:1815]Optimizer aggregation hints are conflicted" ] + }, + { + "SQL": "explain with d1 as (\n select a from (\n select a from (\n select /*+ qb_name(qb, v4) use_index(t4@qb, idx_a) */ a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select /*+ qb_name(qb2, v4) use_index(t4@qb2, idx_b) */ a from v4 where b < 10)\n\nselect * from (select * from d1) as t0 join (select * from d2) as t1;", + "Plan": [ + "HashJoin_41 6944.44 root CARTESIAN inner join", + "├─IndexLookUp_50(Build) 83.33 root ", + "│ ├─IndexRangeScan_47(Build) 250.00 cop[tikv] table:t4, index:idx_b(b) range:(3,10), keep order:false, stats:pseudo", + "│ └─Selection_49(Probe) 83.33 cop[tikv] gt(test.t4.a, 2)", + "│ └─TableRowIDScan_48 250.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─IndexLookUp_46(Probe) 83.33 root ", + " ├─IndexRangeScan_43(Build) 250.00 cop[tikv] table:t4, index:idx_a(a) range:(2,8), keep order:false, stats:pseudo", + " └─Selection_45(Probe) 83.33 cop[tikv] gt(test.t4.b, 3)", + " └─TableRowIDScan_44 250.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain with d1 as (\n select a from (\n select a from (\n select a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select a from v4 where b < 10)\n\nselect /*+ qb_name(qb, v4@sel_4) use_index(t4@qb, idx_a) qb_name(qb2, v4@sel_5) use_index(t4@qb, idx_b) */ * from (select * from d1) as t0 join (select * from d2) as t1;", + "Plan": [ + "HashJoin_41 6944.44 root CARTESIAN inner join", + "├─TableReader_53(Build) 83.33 root data:Selection_52", + "│ └─Selection_52 83.33 cop[tikv] gt(test.t4.a, 2), gt(test.t4.b, 3), lt(test.t4.b, 10)", + "│ └─TableFullScan_51 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─IndexLookUp_46(Probe) 83.33 root ", + " ├─IndexRangeScan_43(Build) 250.00 cop[tikv] table:t4, index:idx_a(a) range:(2,8), keep order:false, stats:pseudo", + " └─Selection_45(Probe) 83.33 cop[tikv] gt(test.t4.b, 3)", + " └─TableRowIDScan_44 250.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain with d1 as (\n select a from (\n select a from (\n select /*+ qb_name(qb, v5) use_index(t4@qb, idx_a) */ a from v4 where a < 10\n ) as t0 where a < 9\n ) as t1 where a < 8\n), d2 as (select /*+ qb_name(qb2, v4) use_index(t4@qb2, idx_b) */ a from v4 where b < 10)\n\nselect * from (select * from d1) as t0 join (select * from d2) as t1;", + "Plan": [ + "HashJoin_41 6944.44 root CARTESIAN inner join", + "├─IndexLookUp_57(Build) 83.33 root ", + "│ ├─IndexRangeScan_54(Build) 250.00 cop[tikv] table:t4, index:idx_b(b) range:(3,10), keep order:false, stats:pseudo", + "│ └─Selection_56(Probe) 83.33 cop[tikv] gt(test.t4.a, 2)", + "│ └─TableRowIDScan_55 250.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─TableReader_45(Probe) 83.33 root data:Selection_44", + " └─Selection_44 83.33 cop[tikv] gt(test.t4.a, 2), gt(test.t4.b, 3), lt(test.t4.a, 10), lt(test.t4.a, 8), lt(test.t4.a, 9)", + " └─TableFullScan_43 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warn": [ + "The qb_name hint qb is unused, please check whether the table list in the qb_name hint qb is correct" + ] } ] }, @@ -6499,7 +6557,7 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#15)->Column#14", "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#15", + " └─HashAgg 1.00 cop[tikv] funcs:count(test.t._tidb_rowid)->Column#15", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, @@ -6526,7 +6584,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(Column#15)->Column#14", "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#15", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t._tidb_rowid)->Column#15", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, @@ -6692,7 +6750,7 @@ "HashAgg 1.00 root funcs:count(Column#16)->Column#14", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#16", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#16", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -6724,7 +6782,7 @@ "HashAgg 1.00 root funcs:count(Column#17)->Column#14", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#17", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#17", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -6867,7 +6925,7 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#7)->Column#6", "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(test.t._tidb_rowid)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -6896,7 +6954,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(Column#7)->Column#6", "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t._tidb_rowid)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -7028,7 +7086,7 @@ "HashAgg 1.00 root funcs:count(Column#8)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#8", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -7060,7 +7118,7 @@ "HashAgg 1.00 root funcs:count(Column#9)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#9", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -7527,7 +7585,7 @@ "HashAgg 1.00 root funcs:count(Column#7)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#7", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -7548,7 +7606,7 @@ "HashAgg 1.00 root funcs:count(Column#6)->Column#4", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#6", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -7606,7 +7664,7 @@ "├─HashAgg(Build) 1.00 root funcs:count(Column#11)->Column#7", "│ └─TableReader 1.00 root data:ExchangeSender", "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#11", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#11", "│ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", @@ -7953,7 +8011,7 @@ "├─HashAgg(Build) 1.00 root funcs:count(Column#10)->Column#7", "│ └─TableReader 1.00 root data:ExchangeSender", "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#10", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t._tidb_rowid)->Column#10", "│ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", @@ -8345,8 +8403,8 @@ "TableReader 1.00 root data:ExchangeSender", "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", - " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", - " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.col_1", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(Column#12)->Column#6, funcs:min(Column#13)->Column#7", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.id, test.ts.col_1", " └─ExchangeReceiver 10000.00 mpp[tiflash] ", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" @@ -8365,7 +8423,7 @@ " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#14, test.ts.col_0, Column#10, Column#11", " └─ExchangeReceiver 1.00 mpp[tiflash] ", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#10, funcs:max(test.ts.col_0)->Column#11", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(test.ts.id)->Column#10, funcs:max(test.ts.col_0)->Column#11", " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 3206c3dc5853d..0da53482a34cc 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -847,9 +847,9 @@ { "SQL": "select * from t7 where a is null or a > 0 order by a;", "Result": [ - "", "1", - "2" + "2", + "" ], "Plan": [ "Sort 3343.33 root test_partition.t7.a", @@ -866,8 +866,8 @@ { "SQL": "select * from t1 order by id,a", "Result": [ - " 10 ", "1 1 1", + "10 10 10", "2 2 2", "3 3 3", "4 4 4", @@ -876,7 +876,7 @@ "7 7 7", "8 8 8", "9 9 9", - "10 10 10" + " 10 " ], "Plan": [ "Sort 10000.00 root test_partition.t1.id, test_partition.t1.a", @@ -1341,8 +1341,8 @@ { "SQL": "select * from t1 where a = 1 or true order by id,a", "Result": [ - " 10 ", "1 1 1", + "10 10 10", "2 2 2", "3 3 3", "4 4 4", @@ -1351,7 +1351,7 @@ "7 7 7", "8 8 8", "9 9 9", - "10 10 10" + " 10 " ], "Plan": [ "Sort 10000.00 root test_partition.t1.id, test_partition.t1.a", @@ -1973,13 +1973,13 @@ "SQL": "select * from t1 where a < 3 or b > 4", "Result": [ "1 1 1", + "10 10 10", "2 2 2", "5 5 5", "6 6 6", "7 7 7", "8 8 8", - "9 9 9", - "10 10 10" + "9 9 9" ], "Plan": [ "TableReader 5548.89 root partition:p0,p1 data:Selection", @@ -2059,11 +2059,11 @@ "SQL": "select * from t1 where (a<=1 and b<=1) or (a >=6 and b>=6)", "Result": [ "1 1 1", + "10 10 10", "6 6 6", "7 7 7", "8 8 8", - "9 9 9", - "10 10 10" + "9 9 9" ], "Plan": [ "TableReader 2092.85 root partition:p0,p1 data:Selection", @@ -2080,6 +2080,7 @@ "SQL": "select * from t1 where a <= 100 and b <= 100", "Result": [ "1 1 1", + "10 10 10", "2 2 2", "3 3 3", "4 4 4", @@ -2087,8 +2088,7 @@ "6 6 6", "7 7 7", "8 8 8", - "9 9 9", - "10 10 10" + "9 9 9" ], "Plan": [ "TableReader 1104.45 root partition:p0,p1 data:Selection", @@ -2126,10 +2126,10 @@ { "SQL": "select * from t1 left join t2 on true where (t1.a <=1 or t1.a <= 3 and (t1.b >=3 and t1.b <= 5)) and (t2.a >= 6 and t2.a <= 8) and t2.b>=7 and t2.id>=7 order by t1.id,t1.a", "Result": [ - "1 1 1 8 8 8", "1 1 1 7 7 7", - "3 3 3 8 8 8", - "3 3 3 7 7 7" + "1 1 1 8 8 8", + "3 3 3 7 7 7", + "3 3 3 8 8 8" ], "Plan": [ "Sort 93855.70 root test_partition.t1.id, test_partition.t1.a", @@ -2326,8 +2326,8 @@ { "SQL": "select * from t1 where a = 3 or true order by id,a", "Result": [ - " 10 ", "1 1 1", + "10 10 10", "2 2 2", "3 3 3", "4 4 4", @@ -2336,7 +2336,7 @@ "7 7 7", "8 8 8", "9 9 9", - "10 10 10" + " 10 " ], "Plan": [ "Sort 10000.00 root test_partition.t1.id, test_partition.t1.a", @@ -2463,6 +2463,7 @@ "SQL": "select * from t1 where (a >= 1 and a <= 6) or (a>=3 and b >=3)", "Result": [ "1 1 1", + "10 10 10", "2 2 2", "3 3 3", "4 4 4", @@ -2470,8 +2471,7 @@ "6 6 6", "7 7 7", "8 8 8", - "9 9 9", - "10 10 10" + "9 9 9" ], "Plan": [ "TableReader 1333.33 root partition:p0,p1 data:Selection", diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index d718be8d6dfb4..47feb2ca0e437 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -1094,5 +1094,37 @@ "SELECT ta.NAME FROM ta WHERE EXISTS (select /*+ no_decorrelate() */ 1 from tb where ta.code = tb.code and tb.NAME LIKE 'chad9%') AND (select /*+ no_decorrelate() */ max(id) from tc where ta.name=tc.name and tc.name like 'chad99%') > 100 and (select /*+ no_decorrelate() */ max(id) from td where ta.id=td.id and td.name like 'chad999%') > 100" ] + }, + { + "name": "TestCountStarForTikv", + "cases": [ + "select count(*) from t", + "select count(1), count(3.1415), count(0), count(null) from t -- be rewritten count(row_id) but count(null)", + "select count(*) from t where a=1", + "select count(*) from t_pick_row_id", + "select t.b, t.c from (select count(*) as c from t) a, t where a.c=t.a -- test recursive, be rewritten count(row_id)", + "select * from t out where out.a > (select count(*) from t inn where inn.a = out.b) -- shouldn't be rewritten for correlated sub query", + "select count(*) from t t1, t t2 where t1.a=t2.e -- shouldn't be rewritten when join under agg", + "select count(distinct 1) from t -- shouldn't be rewritten", + "select count(1), count(a), count(b) from t -- count(1) to count(a)", + "select a, count(*) from t group by a -- shouldn't be rewritten", + "select sum(a) from t -- sum shouldn't be rewritten" + ] + }, + { + "name": "TestCountStarForTiFlash", + "cases": [ + "select count(*) from t", + "select count(1), count(3.1415), count(0), count(null) from t -- every count but count(null) can be rewritten", + "select count(*) from t where a=1", + "select count(*) from t_pick_row_id", + "select t.b, t.c from (select count(*) as c from t) a, t where a.c=t.a -- test recursive", + "select * from t out where out.a > (select count(*) from t inn where inn.a = out.b) -- shouldn't be rewritten for correlated sub query", + "select count(*) from t t1, t t2 where t1.a=t2.e -- shouldn't be rewritten when join under agg", + "select count(distinct 1) from t -- shouldn't be rewritten", + "select count(1), count(a), count(b) from t -- count(1) to count(a)", + "select a, count(*) from t group by a -- shouldn't be rewritten", + "select sum(a) from t -- sum shouldn't be rewritten" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 040b149e4f180..8d246a2af2f1a 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -6438,5 +6438,276 @@ "Warning": null } ] + }, + { + "Name": "TestCountStarForTikv", + "Cases": [ + { + "SQL": "select count(*) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#12)->Column#10", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t._tidb_rowid)->Column#12", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(1), count(3.1415), count(0), count(null) from t -- be rewritten count(row_id) but count(null)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#18)->Column#10, funcs:count(Column#19)->Column#11, funcs:count(Column#20)->Column#12, funcs:count(Column#21)->Column#13", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t._tidb_rowid)->Column#18, funcs:count(test.t._tidb_rowid)->Column#19, funcs:count(test.t._tidb_rowid)->Column#20, funcs:count(NULL)->Column#21", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t where a=1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#12)->Column#10", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#12", + " └─Selection 10.00 cop[tikv] eq(test.t.a, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t_pick_row_id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#5)->Column#3", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t_pick_row_id._tidb_rowid)->Column#5", + " └─TableFullScan 10000.00 cop[tikv] table:t_pick_row_id keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select t.b, t.c from (select count(*) as c from t) a, t where a.c=t.a -- test recursive, be rewritten count(row_id)", + "Plan": [ + "HashJoin 1.25 root inner join, equal:[eq(test.t.a, Column#10)]", + "├─StreamAgg(Build) 1.00 root funcs:count(Column#21)->Column#10", + "│ └─TableReader 1.00 root data:StreamAgg", + "│ └─StreamAgg 1.00 cop[tikv] funcs:count(test.t._tidb_rowid)->Column#21", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from t out where out.a > (select count(*) from t inn where inn.a = out.b) -- shouldn't be rewritten for correlated sub query", + "Plan": [ + "Projection 10000.00 root test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.f, test.t.g, test.t.h", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(test.t.a, Column#19)", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:out keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:count(Column#21)->Column#19", + " └─TableReader 10000.00 root data:StreamAgg", + " └─StreamAgg 10000.00 cop[tikv] funcs:count(test.t.a)->Column#21", + " └─Selection 80000000.00 cop[tikv] eq(cast(test.t.a, double BINARY), cast(test.t.b, double BINARY))", + " └─TableFullScan 100000000.00 cop[tikv] table:inn keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t t1, t t2 where t1.a=t2.e -- shouldn't be rewritten when join under agg", + "Plan": [ + "HashAgg 1.00 root funcs:count(1)->Column#19", + "└─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t.e)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(distinct 1) from t -- shouldn't be rewritten", + "Plan": [ + "StreamAgg 1.00 root funcs:count(distinct 1)->Column#10", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(1), count(a), count(b) from t -- count(1) to count(a)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#16)->Column#10, funcs:count(Column#17)->Column#11, funcs:count(Column#18)->Column#12", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#16, funcs:count(test.t.a)->Column#17, funcs:count(test.t.b)->Column#18", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select a, count(*) from t group by a -- shouldn't be rewritten", + "Plan": [ + "Projection 8000.00 root test.t.a, Column#10", + "└─HashAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#10, funcs:firstrow(test.t.a)->test.t.a", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select sum(a) from t -- sum shouldn't be rewritten", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(Column#12)->Column#10", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:sum(test.t.a)->Column#12", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "Name": "TestCountStarForTiFlash", + "Cases": [ + { + "SQL": "select count(*) from t", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#19)->Column#10", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t.d)->Column#19", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(1), count(3.1415), count(0), count(null) from t -- every count but count(null) can be rewritten", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#25)->Column#10, funcs:count(Column#26)->Column#11, funcs:count(Column#27)->Column#12, funcs:count(Column#28)->Column#13", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t.d)->Column#25, funcs:count(test.t.d)->Column#26, funcs:count(test.t.d)->Column#27, funcs:count(NULL)->Column#28", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t where a=1", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#10", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t.a)->Column#12", + " └─Selection 10.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t_pick_row_id", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#5)->Column#3", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t_pick_row_id._tidb_rowid)->Column#5", + " └─TableFullScan 10000.00 mpp[tiflash] table:t_pick_row_id keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select t.b, t.c from (select count(*) as c from t) a, t where a.c=t.a -- test recursive", + "Plan": [ + "HashJoin 1.25 root inner join, equal:[eq(test.t.a, Column#10)]", + "├─HashAgg(Build) 1.00 root funcs:count(Column#29)->Column#10", + "│ └─TableReader 1.00 root data:ExchangeSender", + "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t.d)->Column#29", + "│ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from t out where out.a > (select count(*) from t inn where inn.a = out.b) -- shouldn't be rewritten for correlated sub query", + "Plan": [ + "Projection 10000.00 root test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.f, test.t.g, test.t.h", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(test.t.a, Column#19)", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:out keep order:false, stats:pseudo", + " └─HashAgg(Probe) 10000.00 root funcs:count(Column#21)->Column#19", + " └─TableReader 10000.00 root data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 10000.00 mpp[tiflash] funcs:count(test.t.a)->Column#21", + " └─Selection 80000000.00 mpp[tiflash] eq(cast(test.t.a, double BINARY), cast(test.t.b, double BINARY))", + " └─TableFullScan 100000000.00 mpp[tiflash] table:inn keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(*) from t t1, t t2 where t1.a=t2.e -- shouldn't be rewritten when join under agg", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#20)->Column#19", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#20", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.e)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(distinct 1) from t -- shouldn't be rewritten", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#10", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct Column#19)->Column#10", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select count(1), count(a), count(b) from t -- count(1) to count(a)", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#16)->Column#10, funcs:count(Column#17)->Column#11, funcs:count(Column#18)->Column#12", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(test.t.a)->Column#16, funcs:count(test.t.a)->Column#17, funcs:count(test.t.b)->Column#18", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select a, count(*) from t group by a -- shouldn't be rewritten", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#10", + " └─Projection 8000.00 mpp[tiflash] Column#10, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:count(1)->Column#10, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select sum(a) from t -- sum shouldn't be rewritten", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#12)->Column#10", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#15)->Column#12", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#15", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warning": null + } + ] } ] diff --git a/planner/optimize.go b/planner/optimize.go index 747dd5541596a..def76aba9c9c3 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -356,6 +356,7 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in // build logical plan hintProcessor := &hint.BlockHintProcessor{Ctx: sctx} node.Accept(hintProcessor) + defer hintProcessor.HandleUnusedViewHints() builder := planBuilderPool.Get().(*core.PlanBuilder) defer planBuilderPool.Put(builder.ResetForReuse()) builder.Init(sctx, is, hintProcessor) diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 2a79502bf985e..d0d68a9142035 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -136,6 +136,7 @@ go_test( "//expression", "//kv", "//meta", + "//meta/autoid", "//parser/ast", "//parser/auth", "//parser/model", diff --git a/session/bootstrap.go b/session/bootstrap.go index 4b4cfe51aaab1..450959ee68742 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -2267,8 +2267,6 @@ func doDMLWorks(s Session) { writeNewCollationParameter(s, config.GetGlobalConfig().NewCollationsEnabledOnFirstBootstrap) - writeDefaultExprPushDownBlacklist(s) - writeStmtSummaryVars(s) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 53076c82a0d85..b9b0cc767500b 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -114,6 +114,12 @@ func TestBootstrap(t *testing.T) { se, err = CreateSession4Test(store) require.NoError(t, err) doDMLWorks(se) + r = mustExec(t, se, "select * from mysql.expr_pushdown_blacklist where name = 'date_add'") + req = r.NewChunk(nil) + err = r.Next(ctx, req) + require.NoError(t, err) + require.Equal(t, 0, req.NumRows()) + se.Close() } func globalVarsCount() int64 { diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index e82faba8e6e56..565b4861a9400 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" @@ -276,7 +277,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { initTblColIdxID(oldTblMeta) // Indices[0] does not exist at the start. oldTblMeta.Indices = oldTblMeta.Indices[1:] - oldTbInfo, err := table.TableFromMeta(nil, oldTblMeta) + oldTbInfo, err := table.TableFromMeta(autoid.NewAllocators(false), oldTblMeta) require.NoError(t, err) oldTblMeta.Indices[0].State = startState oldTblMeta.Indices[2].State = endState @@ -296,7 +297,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { // The last index "c_d_e_str_prefix is dropped. newTblMeta.Indices = newTblMeta.Indices[:len(newTblMeta.Indices)-1] newTblMeta.Indices[0].Unique = false - newTblInfo, err := table.TableFromMeta(nil, newTblMeta) + newTblInfo, err := table.TableFromMeta(autoid.NewAllocators(false), newTblMeta) require.NoError(t, err) newTblMeta.Indices[0].State = endState // Indices[1] is newly created. diff --git a/session/session.go b/session/session.go index e95e4e7283679..437c663999b83 100644 --- a/session/session.go +++ b/session/session.go @@ -145,6 +145,7 @@ var ( telemetryTablePartitionAddIntervalUsage = metrics.TelemetryTablePartitionAddIntervalPartitionsCnt telemetryTablePartitionDropIntervalUsage = metrics.TelemetryTablePartitionDropIntervalPartitionsCnt telemetryExchangePartitionUsage = metrics.TelemetryExchangePartitionCnt + telemetryTableCompactPartitionUsage = metrics.TelemetryCompactPartitionCnt telemetryLockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("lockUser") telemetryUnlockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("unlockUser") @@ -1588,6 +1589,7 @@ func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecu Info: sql, CurTxnStartTS: curTxnStartTS, StmtCtx: s.sessionVars.StmtCtx, + RefCountOfStmtCtx: &s.sessionVars.RefCountOfStmtCtx, MemTracker: s.sessionVars.MemTracker, DiskTracker: s.sessionVars.DiskTracker, StatsInfo: plannercore.GetStatsInfo, @@ -2449,7 +2451,6 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields } ctx := context.Background() - inTxn := s.GetSessionVars().InTxn() // NewPrepareExec may need startTS to build the executor, for example prepare statement has subquery in int. // So we have to call PrepareTxnCtx here. if err = s.PrepareTxnCtx(ctx); err != nil { @@ -2466,13 +2467,12 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields } prepareExec := executor.NewPrepareExec(s, sql) err = prepareExec.Next(ctx, nil) + // Rollback even if err is nil. + s.rollbackOnError(ctx) + if err != nil { return } - if !inTxn { - // We could start a transaction to build the prepare executor before, we should rollback it here. - s.RollbackTxn(ctx) - } return prepareExec.ID, prepareExec.ParamCount, prepareExec.Fields, nil } @@ -2898,11 +2898,17 @@ func InitMDLVariable(store kv.Storage) error { if err != nil { return err } + if isNull { + // Workaround for version: nightly-2022-11-07 to nightly-2022-11-17. + enable = true + logutil.BgLogger().Warn("metadata lock is null") + err = t.SetMetadataLock(true) + if err != nil { + return err + } + } return nil }) - if isNull { - return errors.New("metadata lock is null") - } variable.EnableMDL.Store(enable) return err } @@ -3136,7 +3142,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { s.sessionVars.BinlogClient = binloginfo.GetPumpsClient() s.txn.init() - sessionBindHandle := bindinfo.NewSessionBindHandle(parser.New()) + sessionBindHandle := bindinfo.NewSessionBindHandle() s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) s.SetSessionStatesHandler(sessionstates.StateBinding, sessionBindHandle) return s, nil @@ -3632,6 +3638,9 @@ func (s *session) updateTelemetryMetric(es *executor.ExecStmt) { if ti.PartitionTelemetry.UseDropIntervalPartition { telemetryTablePartitionDropIntervalUsage.Inc() } + if ti.PartitionTelemetry.UseCompactTablePartition { + telemetryTableCompactPartitionUsage.Inc() + } } if ti.AccountLockTelemetry != nil { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 328b61f87ee69..c8fdc2ce32f2e 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -101,6 +101,42 @@ func (warn *SQLWarn) UnmarshalJSON(data []byte) error { return nil } +// ReferenceCount indicates the reference count of StmtCtx. +type ReferenceCount int32 + +const ( + // ReferenceCountIsFrozen indicates the current StmtCtx is resetting, it'll refuse all the access from other sessions. + ReferenceCountIsFrozen int32 = -1 + // ReferenceCountNoReference indicates the current StmtCtx is not accessed by other sessions. + ReferenceCountNoReference int32 = 0 +) + +// TryIncrease tries to increase the reference count. +// There is a small chance that TryIncrease returns true while TryFreeze and +// UnFreeze are invoked successfully during the execution of TryIncrease. +func (rf *ReferenceCount) TryIncrease() bool { + refCnt := atomic.LoadInt32((*int32)(rf)) + for ; refCnt != ReferenceCountIsFrozen && !atomic.CompareAndSwapInt32((*int32)(rf), refCnt, refCnt+1); refCnt = atomic.LoadInt32((*int32)(rf)) { + } + return refCnt != ReferenceCountIsFrozen +} + +// Decrease decreases the reference count. +func (rf *ReferenceCount) Decrease() { + for refCnt := atomic.LoadInt32((*int32)(rf)); !atomic.CompareAndSwapInt32((*int32)(rf), refCnt, refCnt-1); refCnt = atomic.LoadInt32((*int32)(rf)) { + } +} + +// TryFreeze tries to freeze the StmtCtx to frozen before resetting the old StmtCtx. +func (rf *ReferenceCount) TryFreeze() bool { + return atomic.LoadInt32((*int32)(rf)) == ReferenceCountNoReference && atomic.CompareAndSwapInt32((*int32)(rf), ReferenceCountNoReference, ReferenceCountIsFrozen) +} + +// UnFreeze unfreeze the frozen StmtCtx thus the other session can access this StmtCtx. +func (rf *ReferenceCount) UnFreeze() { + atomic.StoreInt32((*int32)(rf), ReferenceCountNoReference) +} + // StatementContext contains variables for a statement. // It should be reset before executing a statement. type StatementContext struct { @@ -177,7 +213,7 @@ type StatementContext struct { warnings []SQLWarn errorCount uint16 execDetails execdetails.ExecDetails - allExecDetails []*execdetails.ExecDetails + allExecDetails []*execdetails.DetailsNeedP90 } // PrevAffectedRows is the affected-rows value(DDL is 0, DML is the number of affected rows). PrevAffectedRows int64 @@ -827,7 +863,7 @@ func (sc *StatementContext) resetMuForRetry() { sc.mu.errorCount = 0 sc.mu.warnings = nil sc.mu.execDetails = execdetails.ExecDetails{} - sc.mu.allExecDetails = make([]*execdetails.ExecDetails, 0, 4) + sc.mu.allExecDetails = make([]*execdetails.DetailsNeedP90, 0, 4) } // ResetForRetry resets the changed states during execution. @@ -851,7 +887,13 @@ func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails, c sc.mu.execDetails.RequestCount++ sc.MergeScanDetail(details.ScanDetail) sc.MergeTimeDetail(details.TimeDetail) - sc.mu.allExecDetails = append(sc.mu.allExecDetails, details) + sc.mu.allExecDetails = append(sc.mu.allExecDetails, + &execdetails.DetailsNeedP90{ + BackoffSleep: details.BackoffSleep, + BackoffTimes: details.BackoffTimes, + CalleeAddress: details.CalleeAddress, + TimeDetail: details.TimeDetail, + }) } if commitDetails != nil { if sc.mu.execDetails.CommitDetail == nil { @@ -970,14 +1012,14 @@ func (sc *StatementContext) CopTasksDetails() *CopTasksDetails { d.AvgProcessTime = sc.mu.execDetails.TimeDetail.ProcessTime / time.Duration(n) d.AvgWaitTime = sc.mu.execDetails.TimeDetail.WaitTime / time.Duration(n) - slices.SortFunc(sc.mu.allExecDetails, func(i, j *execdetails.ExecDetails) bool { + slices.SortFunc(sc.mu.allExecDetails, func(i, j *execdetails.DetailsNeedP90) bool { return i.TimeDetail.ProcessTime < j.TimeDetail.ProcessTime }) d.P90ProcessTime = sc.mu.allExecDetails[n*9/10].TimeDetail.ProcessTime d.MaxProcessTime = sc.mu.allExecDetails[n-1].TimeDetail.ProcessTime d.MaxProcessAddress = sc.mu.allExecDetails[n-1].CalleeAddress - slices.SortFunc(sc.mu.allExecDetails, func(i, j *execdetails.ExecDetails) bool { + slices.SortFunc(sc.mu.allExecDetails, func(i, j *execdetails.DetailsNeedP90) bool { return i.TimeDetail.WaitTime < j.TimeDetail.WaitTime }) d.P90WaitTime = sc.mu.allExecDetails[n*9/10].TimeDetail.WaitTime diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index cc21b3e4812c7..67520c36e7b80 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -36,12 +36,14 @@ func TestCopTasksDetails(t *testing.T) { backoffs := []string{"tikvRPC", "pdRPC", "regionMiss"} for i := 0; i < 100; i++ { d := &execdetails.ExecDetails{ - CalleeAddress: fmt.Sprintf("%v", i+1), - BackoffSleep: make(map[string]time.Duration), - BackoffTimes: make(map[string]int), - TimeDetail: util.TimeDetail{ - ProcessTime: time.Second * time.Duration(i+1), - WaitTime: time.Millisecond * time.Duration(i+1), + DetailsNeedP90: execdetails.DetailsNeedP90{ + CalleeAddress: fmt.Sprintf("%v", i+1), + BackoffSleep: make(map[string]time.Duration), + BackoffTimes: make(map[string]int), + TimeDetail: util.TimeDetail{ + ProcessTime: time.Second * time.Duration(i+1), + WaitTime: time.Millisecond * time.Duration(i+1), + }, }, } for _, backoff := range backoffs { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index cb8caf59819f5..5115c0a834ad6 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -742,6 +742,11 @@ type SessionVars struct { // StmtCtx holds variables for current executing statement. StmtCtx *stmtctx.StatementContext + // RefCountOfStmtCtx indicates the reference count of StmtCtx. When the + // StmtCtx is accessed by other sessions, e.g. oom-alarm-handler/expensive-query-handler, add one first. + // Note: this variable should be accessed and updated by atomic operations. + RefCountOfStmtCtx stmtctx.ReferenceCount + // AllowAggPushDown can be set to false to forbid aggregation push down. AllowAggPushDown bool @@ -1389,7 +1394,12 @@ func (s *SessionVars) InitStatementContext() *stmtctx.StatementContext { if sc == s.StmtCtx { sc = &s.cachedStmtCtx[1] } - *sc = stmtctx.StatementContext{} + if s.RefCountOfStmtCtx.TryFreeze() { + *sc = stmtctx.StatementContext{} + s.RefCountOfStmtCtx.UnFreeze() + } else { + sc = &stmtctx.StatementContext{} + } return sc } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 92049902618c2..732ce4ad606cf 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -164,9 +164,11 @@ func TestSlowLogFormat(t *testing.T) { ProcessedKeys: 20001, TotalKeys: 10000, }, - TimeDetail: util.TimeDetail{ - ProcessTime: time.Second * time.Duration(2), - WaitTime: time.Minute, + DetailsNeedP90: execdetails.DetailsNeedP90{ + TimeDetail: util.TimeDetail{ + ProcessTime: time.Second * time.Duration(2), + WaitTime: time.Minute, + }, }, } statsInfos := make(map[string]uint64) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index bfd3bbcc94fdd..5f6e435028e3b 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -695,7 +695,8 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V } tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs) } else { - ranges := NewKeyRanges(req.KeyRanges) + // TODO: merge the if branch. + ranges := NewKeyRanges(req.KeyRanges.FirstPartitionRange()) tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0) } diff --git a/store/copr/copr_test/coprocessor_test.go b/store/copr/copr_test/coprocessor_test.go index f92db7ba7c334..208f2e2bd2190 100644 --- a/store/copr/copr_test/coprocessor_test.go +++ b/store/copr/copr_test/coprocessor_test.go @@ -43,7 +43,7 @@ func TestBuildCopIteratorWithRowCountHint(t *testing.T) { req := &kv.Request{ Tp: kv.ReqTypeDAG, - KeyRanges: copr.BuildKeyRanges("a", "c", "d", "e", "h", "x", "y", "z"), + KeyRanges: kv.NewNonParitionedKeyRanges(copr.BuildKeyRanges("a", "c", "d", "e", "h", "x", "y", "z")), FixedRowCountHint: []int{1, 1, 3, copr.CopSmallTaskRow}, Concurrency: 15, } @@ -57,7 +57,7 @@ func TestBuildCopIteratorWithRowCountHint(t *testing.T) { req = &kv.Request{ Tp: kv.ReqTypeDAG, - KeyRanges: copr.BuildKeyRanges("a", "c", "d", "e", "h", "x", "y", "z"), + KeyRanges: kv.NewNonParitionedKeyRanges(copr.BuildKeyRanges("a", "c", "d", "e", "h", "x", "y", "z")), FixedRowCountHint: []int{1, 1, 3, 3}, Concurrency: 15, } @@ -72,7 +72,7 @@ func TestBuildCopIteratorWithRowCountHint(t *testing.T) { // cross-region long range req = &kv.Request{ Tp: kv.ReqTypeDAG, - KeyRanges: copr.BuildKeyRanges("a", "z"), + KeyRanges: kv.NewNonParitionedKeyRanges(copr.BuildKeyRanges("a", "z")), FixedRowCountHint: []int{10}, Concurrency: 15, } @@ -86,7 +86,7 @@ func TestBuildCopIteratorWithRowCountHint(t *testing.T) { req = &kv.Request{ Tp: kv.ReqTypeDAG, - KeyRanges: copr.BuildKeyRanges("a", "z"), + KeyRanges: kv.NewNonParitionedKeyRanges(copr.BuildKeyRanges("a", "z")), FixedRowCountHint: []int{copr.CopSmallTaskRow + 1}, Concurrency: 15, } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 982e981f24c79..fdff94a09717d 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -15,7 +15,6 @@ package copr import ( - "bytes" "context" "fmt" "math" @@ -53,7 +52,6 @@ import ( "github.com/tikv/client-go/v2/txnkv/txnsnapshot" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" - "golang.org/x/exp/slices" ) var coprCacheCounterEvict = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("evict") @@ -121,10 +119,7 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars } failpoint.Inject("checkKeyRangeSortedForPaging", func(_ failpoint.Value) { if req.Paging.Enable { - isSorted := slices.IsSortedFunc(req.KeyRanges, func(i, j kv.KeyRange) bool { - return bytes.Compare(i.StartKey, j.StartKey) < 0 - }) - if !isSorted { + if !req.KeyRanges.IsFullySorted() { logutil.BgLogger().Fatal("distsql request key range not sorted!") } } @@ -138,8 +133,27 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars }) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) - ranges := NewKeyRanges(req.KeyRanges) - tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req, eventCb) + var ( + tasks []*copTask + err error + ) + buildTaskFunc := func(ranges []kv.KeyRange) error { + keyRanges := NewKeyRanges(ranges) + tasksFromRanges, err := buildCopTasks(bo, c.store.GetRegionCache(), keyRanges, req, eventCb) + if err != nil { + return err + } + if len(tasks) == 0 { + tasks = tasksFromRanges + return nil + } + tasks = append(tasks, tasksFromRanges...) + return nil + } + // Here we build the task by partition, not directly by region. + // This is because it's possible that TiDB merge multiple small partition into one region which break some assumption. + // Keep it split by partition would be more safe. + err = req.KeyRanges.ForEachPartitionWithErr(buildTaskFunc) reqType := "null" if req.ClosestReplicaReadAdjuster != nil { reqType = "miss" diff --git a/store/helper/helper.go b/store/helper/helper.go index f24443433e724..8d584b216d9ad 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -653,11 +653,11 @@ func newTableWithKeyRange(db *model.DBInfo, table *model.TableInfo) TableInfoWit // NewIndexWithKeyRange constructs TableInfoWithKeyRange for given index, it is exported only for test. func NewIndexWithKeyRange(db *model.DBInfo, table *model.TableInfo, index *model.IndexInfo) TableInfoWithKeyRange { - return newIndexWithKeyRange(db, table, index) + return newIndexWithKeyRange(db, table, index, table.ID) } -func newIndexWithKeyRange(db *model.DBInfo, table *model.TableInfo, index *model.IndexInfo) TableInfoWithKeyRange { - sk, ek := tablecodec.GetTableIndexKeyRange(table.ID, index.ID) +func newIndexWithKeyRange(db *model.DBInfo, table *model.TableInfo, index *model.IndexInfo, physicalID int64) TableInfoWithKeyRange { + sk, ek := tablecodec.GetTableIndexKeyRange(physicalID, index.ID) startKey := bytesKeyToHex(codec.EncodeBytes(nil, sk)) endKey := bytesKeyToHex(codec.EncodeBytes(nil, ek)) return TableInfoWithKeyRange{ @@ -727,7 +727,13 @@ func (*Helper) GetTablesInfoWithKeyRange(schemas []*model.DBInfo) []TableInfoWit tables = append(tables, newTableWithKeyRange(db, table)) } for _, index := range table.Indices { - tables = append(tables, newIndexWithKeyRange(db, table, index)) + if table.Partition == nil || index.Global { + tables = append(tables, newIndexWithKeyRange(db, table, index, table.ID)) + continue + } + for _, partition := range table.Partition.Definitions { + tables = append(tables, newIndexWithKeyRange(db, table, index, partition.ID)) + } } } } diff --git a/table/table.go b/table/table.go index 6aef5ed4497e6..813131df90896 100644 --- a/table/table.go +++ b/table/table.go @@ -206,7 +206,8 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte } increment := sctx.GetSessionVars().AutoIncrementIncrement offset := sctx.GetSessionVars().AutoIncrementOffset - _, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(1), int64(increment), int64(offset)) + alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType) + _, max, err := alloc.Alloc(ctx, uint64(1), int64(increment), int64(offset)) if err != nil { return 0, err } @@ -218,7 +219,8 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) (firstID int64, increment int64, err error) { increment = int64(sctx.GetSessionVars().AutoIncrementIncrement) offset := int64(sctx.GetSessionVars().AutoIncrementOffset) - min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(N), increment, offset) + alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType) + min, max, err := alloc.Alloc(ctx, uint64(N), increment, offset) if err != nil { return min, max, err } diff --git a/table/tables/index.go b/table/tables/index.go index 9fc1042a110fd..b3a481efba29f 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -15,7 +15,9 @@ package tables import ( + "bytes" "context" + "errors" "sync" "github.com/opentracing/opentracing-go" @@ -127,7 +129,7 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue keyIsTempIdxKey bool ) if !opt.FromBackFill { - key, tempKey, keyVer = genTempIdxKeyByState(c.idxInfo, key) + key, tempKey, keyVer = GenTempIdxKeyByState(c.idxInfo, key) if keyVer == TempIndexKeyTypeBackfill { key, tempKey = tempKey, nil keyIsTempIdxKey = true @@ -226,11 +228,26 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } if err != nil || len(value) == 0 { lazyCheck := sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil + var needPresumeKey tempIndexKeyState if keyIsTempIdxKey { idxVal = append(idxVal, keyVer) + needPresumeKey, _, err = KeyExistInTempIndex(ctx, txn, key, distinct, h, c.tblInfo.IsCommonHandle) + if err != nil { + return nil, err + } + } else { + if len(tempKey) > 0 { + needPresumeKey, _, err = KeyExistInTempIndex(ctx, txn, tempKey, distinct, h, c.tblInfo.IsCommonHandle) + if err != nil { + return nil, err + } + } } if lazyCheck { - flags := []kv.FlagsOp{kv.SetPresumeKeyNotExists} + var flags []kv.FlagsOp + if needPresumeKey != KeyInTempIndexIsDeleted { + flags = []kv.FlagsOp{kv.SetPresumeKeyNotExists} + } if !vars.ConstraintCheckInPlacePessimistic && vars.TxnCtx.IsPessimistic && vars.InTxn() && !vars.InRestrictedSQL && vars.ConnectionID > 0 { flags = append(flags, kv.SetNeedConstraintCheckInPrewrite) @@ -244,7 +261,7 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } if len(tempKey) > 0 { idxVal = append(idxVal, keyVer) - if lazyCheck { + if lazyCheck && needPresumeKey != KeyInTempIndexIsDeleted { err = txn.GetMemBuffer().SetWithFlags(tempKey, idxVal, kv.SetPresumeKeyNotExists) } else { err = txn.GetMemBuffer().Set(tempKey, idxVal) @@ -285,7 +302,7 @@ func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexed return err } - key, tempKey, tempKeyVer := genTempIdxKeyByState(c.idxInfo, key) + key, tempKey, tempKeyVer := GenTempIdxKeyByState(c.idxInfo, key) if distinct { if len(key) > 0 { @@ -336,9 +353,9 @@ const ( TempIndexKeyTypeMerge byte = 'm' ) -// genTempIdxKeyByState is used to get the key version and the temporary key. +// GenTempIdxKeyByState is used to get the key version and the temporary key. // The tempKeyVer means the temp index key/value version. -func genTempIdxKeyByState(indexInfo *model.IndexInfo, indexKey kv.Key) (key, tempKey kv.Key, tempKeyVer byte) { +func GenTempIdxKeyByState(indexInfo *model.IndexInfo, indexKey kv.Key) (key, tempKey kv.Key, tempKeyVer byte) { if indexInfo.State != model.StatePublic { switch indexInfo.BackfillState { case model.BackfillStateInapplicable: @@ -364,6 +381,28 @@ func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedV return false, nil, err } + var ( + tempKey []byte + keyVer byte + ) + // If index current is in creating status and using ingest mode, we need first + // check key exist status in temp index. + key, tempKey, keyVer = GenTempIdxKeyByState(c.idxInfo, key) + if keyVer != TempIndexKeyTypeNone { + KeyExistInfo, h1, err1 := KeyExistInTempIndex(context.TODO(), txn, tempKey, distinct, h, c.tblInfo.IsCommonHandle) + if err1 != nil { + return false, nil, err + } + switch KeyExistInfo { + case KeyInTempIndexNotExist, KeyInTempIndexIsDeleted: + return false, nil, nil + case KeyInTempIndexConflict: + return true, h1, kv.ErrKeyExists + case KeyInTempIndexIsItself: + return true, h, nil + } + } + value, err := txn.Get(context.TODO(), key) if kv.IsErrNotFound(err) { return false, nil, nil @@ -463,3 +502,64 @@ func TryAppendCommonHandleRowcodecColInfos(colInfo []rowcodec.ColInfo, tblInfo * } return colInfo } + +type tempIndexKeyState byte + +const ( + // KeyInTempIndexUnknown whether the key exists or not in temp index is unknown. + KeyInTempIndexUnknown tempIndexKeyState = iota + // KeyInTempIndexNotExist the key is not exist in temp index. + KeyInTempIndexNotExist + // KeyInTempIndexIsDeleted the key is marked deleted in temp index. + KeyInTempIndexIsDeleted + // KeyInTempIndexIsItself the key is correlated to itself in temp index. + KeyInTempIndexIsItself + // KeyInTempIndexConflict the key is conflict in temp index. + KeyInTempIndexConflict +) + +// KeyExistInTempIndex is used to check the unique key exist status in temp index. +func KeyExistInTempIndex(ctx context.Context, txn kv.Transaction, key kv.Key, distinct bool, h kv.Handle, IsCommonHandle bool) (tempIndexKeyState, kv.Handle, error) { + // Only check temp index key. + if !tablecodec.IsTempIndexKey(key) { + return KeyInTempIndexUnknown, nil, nil + } + value, err := txn.Get(ctx, key) + if kv.IsErrNotFound(err) { + return KeyInTempIndexNotExist, nil, nil + } + if err != nil { + return KeyInTempIndexUnknown, nil, err + } + + // Since KeyExistInTempIndex only accept temp index key, so the value length should great than 1 for key version. + if len(value) < 1 { + return KeyInTempIndexUnknown, nil, errors.New("temp index value length should great than 1") + } + length := len(value) + // Firstly, we will remove the last byte of key version. + // It should be TempIndexKeyTypeBackfill or TempIndexKeyTypeMerge. + value = value[:length-1] + if distinct { + if bytes.Equal(value, DeleteMarkerUnique) { + return KeyInTempIndexIsDeleted, nil, nil + } + } else { + if bytes.Equal(value, DeleteMarker) { + return KeyInTempIndexIsDeleted, nil, nil + } + } + + // Check if handle equal. + var handle kv.Handle + if distinct { + handle, err = tablecodec.DecodeHandleInUniqueIndexValue(value, IsCommonHandle) + if err != nil { + return KeyInTempIndexUnknown, nil, err + } + if !handle.Equal(h) { + return KeyInTempIndexConflict, handle, kv.ErrKeyExists + } + } + return KeyInTempIndexIsItself, handle, nil +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 631b26ef4296e..9a5eaae0d6096 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -85,7 +85,7 @@ func MockTableFromMeta(tblInfo *model.TableInfo) table.Table { } var t TableCommon - initTableCommon(&t, tblInfo, tblInfo.ID, columns, nil) + initTableCommon(&t, tblInfo, tblInfo.ID, columns, autoid.NewAllocators(false)) if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { ret, err := newCachedTable(&t) if err != nil { @@ -526,7 +526,12 @@ func (t *TableCommon) rebuildIndices(ctx sessionctx.Context, txn kv.Transaction, break } // If txn is auto commit and index is untouched, no need to write index value. - if untouched && !ctx.GetSessionVars().InTxn() { + // If InHandleForeignKeyTrigger or ForeignKeyTriggerCtx.HasFKCascades is true indicate we may have + // foreign key cascade need to handle later, then we still need to write index value, + // otherwise, the later foreign cascade executor may see data-index inconsistency in txn-mem-buffer. + sessVars := ctx.GetSessionVars() + if untouched && !sessVars.InTxn() && + !sessVars.StmtCtx.InHandleForeignKeyTrigger && !sessVars.StmtCtx.ForeignKeyTriggerCtx.HasFKCascades { continue } newVs, err := idx.FetchValues(newData, nil) @@ -1542,7 +1547,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { // Use an independent allocator for global temporary tables. if t.meta.TempTableType == model.TempTableGlobal { if alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator(); alloc != nil { - return autoid.Allocators{alloc} + return autoid.NewAllocators(false, alloc) } // If the session is not in a txn, for example, in "show create table", use the original allocator. // Otherwise the would be a nil pointer dereference. @@ -1552,8 +1557,9 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { // Replace the row id allocator with the one in session variables. sessAlloc := ctx.GetSessionVars().IDAllocator - retAllocs := make([]autoid.Allocator, 0, len(t.allocs)) - copy(retAllocs, t.allocs) + allocs := t.allocs.Allocs + retAllocs := make([]autoid.Allocator, 0, len(allocs)) + copy(retAllocs, allocs) overwritten := false for i, a := range retAllocs { @@ -1566,7 +1572,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { if !overwritten { retAllocs = append(retAllocs, sessAlloc) } - return retAllocs + return autoid.NewAllocators(t.allocs.SepAutoInc, retAllocs...) } // Type implements table.Table Type interface. @@ -1920,7 +1926,7 @@ func maxIndexLen(idxA, idxB *model.IndexColumn) *model.IndexColumn { } func getSequenceAllocator(allocs autoid.Allocators) (autoid.Allocator, error) { - for _, alloc := range allocs { + for _, alloc := range allocs.Allocs { if alloc.GetType() == autoid.SequenceType { return alloc, nil } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index d659f918a8168..661770b868383 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -378,18 +378,18 @@ func TestTableFromMeta(t *testing.T) { // For test coverage tbInfo.Columns[0].GeneratedExprString = "a" - _, err = tables.TableFromMeta(nil, tbInfo) + _, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.NoError(t, err) tbInfo.Columns[0].GeneratedExprString = "test" - _, err = tables.TableFromMeta(nil, tbInfo) + _, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Error(t, err) tbInfo.Columns[0].State = model.StateNone - tb, err = tables.TableFromMeta(nil, tbInfo) + tb, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Nil(t, tb) require.Error(t, err) tbInfo.State = model.StateNone - tb, err = tables.TableFromMeta(nil, tbInfo) + tb, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Nil(t, tb) require.Error(t, err) diff --git a/table/temptable/BUILD.bazel b/table/temptable/BUILD.bazel index 30c41bd1c55b3..8487a26533b51 100644 --- a/table/temptable/BUILD.bazel +++ b/table/temptable/BUILD.bazel @@ -41,6 +41,7 @@ go_test( deps = [ "//infoschema", "//kv", + "//meta/autoid", "//parser/model", "//parser/mysql", "//sessionctx", diff --git a/table/temptable/ddl.go b/table/temptable/ddl.go index ccad2b7b0214c..d464cb3c48618 100644 --- a/table/temptable/ddl.go +++ b/table/temptable/ddl.go @@ -182,7 +182,7 @@ func newTemporaryTableFromTableInfo(sctx sessionctx.Context, tbInfo *model.Table if alloc != nil { allocs = append(allocs, alloc) } - return tables.TableFromMeta(allocs, tbInfo) + return tables.TableFromMeta(autoid.NewAllocators(false, allocs...), tbInfo) } // GetTemporaryTableDDL gets the temptable.TemporaryTableDDL from session context diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index 7dbc5e86d37d9..8c5c4f557e1ae 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table" @@ -86,7 +87,7 @@ func (is *mockedInfoSchema) TableByID(tblID int64) (table.Table, bool) { State: model.StatePublic, } - tbl, err := table.TableFromMeta(nil, tblInfo) + tbl, err := table.TableFromMeta(autoid.NewAllocators(false), tblInfo) require.NoError(is.t, err) return tbl, true diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index b9400b0271d41..c2d98f5a2b17e 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -1143,6 +1143,20 @@ func TempIndexKey2IndexKey(originIdxID int64, tempIdxKey []byte) { binary.BigEndian.PutUint64(tempIdxKey[prefixLen:], eid) } +// IsTempIndexKey check whether the input key is for a temp index. +func IsTempIndexKey(indexKey []byte) bool { + var ( + indexIDKey []byte + indexID int64 + tempIndexID int64 + ) + // Get encoded indexID from key, Add uint64 8 byte length. + indexIDKey = indexKey[prefixLen : prefixLen+8] + indexID = codec.DecodeCmpUintToInt(binary.BigEndian.Uint64(indexIDKey)) + tempIndexID = int64(TempIndexPrefix) | indexID + return tempIndexID == indexID +} + // GenIndexValuePortal is the portal for generating index value. // Value layout: // @@ -1613,3 +1627,30 @@ func IndexKVIsUnique(value []byte) bool { segs := SplitIndexValue(value) return segs.IntHandle != nil || segs.CommonHandle != nil } + +// VerifyTableIDForRanges verifies that all given ranges are valid to decode the table id. +func VerifyTableIDForRanges(keyRanges *kv.KeyRanges) ([]int64, error) { + tids := make([]int64, 0, keyRanges.PartitionNum()) + collectFunc := func(ranges []kv.KeyRange) error { + if len(ranges) == 0 { + return nil + } + tid := DecodeTableID(ranges[0].StartKey) + if tid <= 0 { + return errors.New("Incorrect keyRange is constrcuted") + } + tids = append(tids, tid) + for i := 1; i < len(ranges); i++ { + tmpTID := DecodeTableID(ranges[i].StartKey) + if tmpTID <= 0 { + return errors.New("Incorrect keyRange is constrcuted") + } + if tid != tmpTID { + return errors.Errorf("Using multi partition's ranges as single table's") + } + } + return nil + } + err := keyRanges.ForEachPartitionWithErr(collectFunc) + return tids, err +} diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index 770f4a7d3d08e..ebe93d6dbaa52 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -143,7 +143,7 @@ func TestAutoIDNoCache(t *testing.T) { usage, err = telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) require.True(t, usage.AutoIDNoCache) - tk.MustExec("alter table tele_autoid auto_id_cache=0") + tk.MustExec("drop table tele_autoid") usage, err = telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) require.False(t, usage.AutoIDNoCache) @@ -286,6 +286,12 @@ func TestTablePartition(t *testing.T) { usage, err = telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) require.Equal(t, int64(1), usage.ExchangePartition.ExchangePartitionCnt) + + require.Equal(t, int64(0), usage.TablePartition.TablePartitionComactCnt) + tk.MustExec(`alter table pt2 compact partition p0 tiflash replica;`) + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(1), usage.TablePartition.TablePartitionComactCnt) } func TestPlacementPolicies(t *testing.T) { diff --git a/testkit/result.go b/testkit/result.go index 0f7ad0ce53cbc..210d32d4c57b9 100644 --- a/testkit/result.go +++ b/testkit/result.go @@ -49,6 +49,11 @@ func (res *Result) Check(expected [][]interface{}) { res.require.Equal(needBuff.String(), resBuff.String(), res.comment) } +// AddComment adds the extra comment for the Result's output. +func (res *Result) AddComment(c string) { + res.comment += "\n" + c +} + // CheckWithFunc asserts the result match the expected results in the way `f` specifies. func (res *Result) CheckWithFunc(expected [][]interface{}, f func([]string, []interface{}) bool) { res.require.Equal(len(res.rows), len(expected), res.comment+"\nResult length mismatch") diff --git a/ttl/BUILD.bazel b/ttl/BUILD.bazel index e6a76c69d8df5..e5ec05168b29b 100644 --- a/ttl/BUILD.bazel +++ b/ttl/BUILD.bazel @@ -3,17 +3,25 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "ttl", srcs = [ + "session.go", "sql.go", "table.go", ], importpath = "github.com/pingcap/tidb/ttl", visibility = ["//visibility:public"], deps = [ + "//infoschema", + "//kv", "//parser/ast", "//parser/format", "//parser/model", "//parser/mysql", + "//parser/terror", + "//sessionctx", + "//sessiontxn", + "//table/tables", "//types", + "//util/chunk", "//util/sqlexec", "@com_github_pingcap_errors//:errors", "@com_github_pkg_errors//:errors", @@ -24,11 +32,13 @@ go_test( name = "ttl_test", srcs = [ "main_test.go", + "session_test.go", "sql_test.go", + "table_test.go", ], + embed = [":ttl"], flaky = True, deps = [ - ":ttl", "//kv", "//parser", "//parser/ast", @@ -38,6 +48,7 @@ go_test( "//testkit/testsetup", "//types", "//util/sqlexec", + "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], diff --git a/ttl/session.go b/ttl/session.go new file mode 100644 index 0000000000000..b3321e0d53c06 --- /dev/null +++ b/ttl/session.go @@ -0,0 +1,123 @@ +// 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 ttl + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" +) + +// Session is used to execute queries for TTL case +type Session interface { + sessionctx.Context + // SessionInfoSchema returns information schema of current session + SessionInfoSchema() infoschema.InfoSchema + // ExecuteSQL executes the sql + ExecuteSQL(ctx context.Context, sql string, args ...interface{}) ([]chunk.Row, error) + // RunInTxn executes the specified function in a txn + RunInTxn(ctx context.Context, fn func() error) (err error) + // Close closes the session + Close() +} + +type session struct { + sessionctx.Context + sqlExec sqlexec.SQLExecutor + closeFn func() +} + +// NewSession creates a new Session +func NewSession(sctx sessionctx.Context, sqlExec sqlexec.SQLExecutor, closeFn func()) Session { + return &session{ + Context: sctx, + sqlExec: sqlExec, + closeFn: closeFn, + } +} + +// SessionInfoSchema returns information schema of current session +func (s *session) SessionInfoSchema() infoschema.InfoSchema { + if s.Context == nil { + return nil + } + return sessiontxn.GetTxnManager(s.Context).GetTxnInfoSchema() +} + +// ExecuteSQL executes the sql +func (s *session) ExecuteSQL(ctx context.Context, sql string, args ...interface{}) ([]chunk.Row, error) { + if s.sqlExec == nil { + return nil, errors.New("session is closed") + } + + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnTTL) + rs, err := s.sqlExec.ExecuteInternal(ctx, sql, args...) + if err != nil { + return nil, err + } + + if rs == nil { + return nil, nil + } + + defer func() { + terror.Log(rs.Close()) + }() + + return sqlexec.DrainRecordSet(ctx, rs, 8) +} + +// RunInTxn executes the specified function in a txn +func (s *session) RunInTxn(ctx context.Context, fn func() error) (err error) { + if _, err = s.ExecuteSQL(ctx, "BEGIN"); err != nil { + return err + } + + success := false + defer func() { + if !success { + _, err = s.ExecuteSQL(ctx, "ROLLBACK") + terror.Log(err) + } + }() + + if err = fn(); err != nil { + return err + } + + if _, err = s.ExecuteSQL(ctx, "COMMIT"); err != nil { + return err + } + + success = true + return err +} + +// Close closes the session +func (s *session) Close() { + if s.closeFn != nil { + s.closeFn() + s.Context = nil + s.sqlExec = nil + s.closeFn = nil + } +} diff --git a/ttl/session_test.go b/ttl/session_test.go new file mode 100644 index 0000000000000..90d47ed313e73 --- /dev/null +++ b/ttl/session_test.go @@ -0,0 +1,52 @@ +// 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 ttl + +import ( + "context" + "testing" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestSessionRunInTxn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(id int primary key, v int)") + se := NewSession(tk.Session(), tk.Session(), nil) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + require.NoError(t, se.RunInTxn(context.TODO(), func() error { + tk.MustExec("insert into t values (1, 10)") + return nil + })) + tk2.MustQuery("select * from t order by id asc").Check(testkit.Rows("1 10")) + + require.NoError(t, se.RunInTxn(context.TODO(), func() error { + tk.MustExec("insert into t values (2, 20)") + return errors.New("err") + })) + tk2.MustQuery("select * from t order by id asc").Check(testkit.Rows("1 10")) + + require.NoError(t, se.RunInTxn(context.TODO(), func() error { + tk.MustExec("insert into t values (3, 30)") + return nil + })) + tk2.MustQuery("select * from t order by id asc").Check(testkit.Rows("1 10", "3 30")) +} diff --git a/ttl/sql.go b/ttl/sql.go index 9cdf762846d4d..3d100fd62eee7 100644 --- a/ttl/sql.go +++ b/ttl/sql.go @@ -31,6 +31,8 @@ import ( "github.com/pkg/errors" ) +const dateTimeFormat = "2006-01-02 15:04:05.999999" + func writeHex(in io.Writer, d types.Datum) error { _, err := fmt.Fprintf(in, "x'%s'", hex.EncodeToString(d.GetBytes())) return err @@ -179,7 +181,7 @@ func (b *SQLBuilder) WriteExpireCondition(expire time.Time) error { b.writeColNames([]*model.ColumnInfo{b.tbl.TimeColumn}, false) b.restoreCtx.WritePlain(" < ") b.restoreCtx.WritePlain("'") - b.restoreCtx.WritePlain(expire.Format("2006-01-02 15:04:05.999999")) + b.restoreCtx.WritePlain(expire.Format(dateTimeFormat)) b.restoreCtx.WritePlain("'") b.hasWriteExpireCond = true return nil diff --git a/ttl/table.go b/ttl/table.go index b9c59a34e5c17..4885da0e137b4 100644 --- a/ttl/table.go +++ b/ttl/table.go @@ -15,23 +15,115 @@ package ttl import ( + "context" + "fmt" + "time" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" ) +func getTableKeyColumns(tbl *model.TableInfo) ([]*model.ColumnInfo, []*types.FieldType, error) { + if tbl.PKIsHandle { + for i, col := range tbl.Columns { + if mysql.HasPriKeyFlag(col.GetFlag()) { + return []*model.ColumnInfo{tbl.Columns[i]}, []*types.FieldType{&tbl.Columns[i].FieldType}, nil + } + } + return nil, nil, errors.Errorf("Cannot find primary key for table: %s", tbl.Name) + } + + if tbl.IsCommonHandle { + idxInfo := tables.FindPrimaryIndex(tbl) + columns := make([]*model.ColumnInfo, len(idxInfo.Columns)) + fieldTypes := make([]*types.FieldType, len(idxInfo.Columns)) + for i, idxCol := range idxInfo.Columns { + columns[i] = tbl.Columns[idxCol.Offset] + fieldTypes[i] = &tbl.Columns[idxCol.Offset].FieldType + } + return columns, fieldTypes, nil + } + + extraHandleColInfo := model.NewExtraHandleColInfo() + return []*model.ColumnInfo{extraHandleColInfo}, []*types.FieldType{&extraHandleColInfo.FieldType}, nil +} + // PhysicalTable is used to provide some information for a physical table in TTL job type PhysicalTable struct { + // Schema is the database name of the table Schema model.CIStr *model.TableInfo + // Partition is the partition name + Partition model.CIStr // PartitionDef is the partition definition PartitionDef *model.PartitionDefinition // KeyColumns is the cluster index key columns for the table KeyColumns []*model.ColumnInfo + // KeyColumnTypes is the types of the key columns + KeyColumnTypes []*types.FieldType // TimeColum is the time column used for TTL TimeColumn *model.ColumnInfo } +// NewPhysicalTable create a new PhysicalTable +func NewPhysicalTable(schema model.CIStr, tbl *model.TableInfo, partition model.CIStr) (*PhysicalTable, error) { + if tbl.State != model.StatePublic { + return nil, errors.Errorf("table '%s.%s' is not a public table", schema, tbl.Name) + } + + ttlInfo := tbl.TTLInfo + if ttlInfo == nil { + return nil, errors.Errorf("table '%s.%s' is not a ttl table", schema, tbl.Name) + } + + timeColumn := tbl.FindPublicColumnByName(ttlInfo.ColumnName.L) + if timeColumn == nil { + return nil, errors.Errorf("time column '%s' is not public in ttl table '%s.%s'", ttlInfo.ColumnName, schema, tbl.Name) + } + + keyColumns, keyColumTypes, err := getTableKeyColumns(tbl) + if err != nil { + return nil, err + } + + var partitionDef *model.PartitionDefinition + if tbl.Partition == nil { + if partition.L != "" { + return nil, errors.Errorf("table '%s.%s' is not a partitioned table", schema, tbl.Name) + } + } else { + if partition.L == "" { + return nil, errors.Errorf("partition name is required, table '%s.%s' is a partitioned table", schema, tbl.Name) + } + + for i := range tbl.Partition.Definitions { + def := &tbl.Partition.Definitions[i] + if def.Name.L == partition.L { + partitionDef = def + } + } + + if partitionDef == nil { + return nil, errors.Errorf("partition '%s' is not found in ttl table '%s.%s'", partition.O, schema, tbl.Name) + } + } + + return &PhysicalTable{ + Schema: schema, + TableInfo: tbl, + Partition: partition, + PartitionDef: partitionDef, + KeyColumns: keyColumns, + KeyColumnTypes: keyColumTypes, + TimeColumn: timeColumn, + }, nil +} + // ValidateKey validates a key func (t *PhysicalTable) ValidateKey(key []types.Datum) error { if len(t.KeyColumns) != len(key) { @@ -39,3 +131,25 @@ func (t *PhysicalTable) ValidateKey(key []types.Datum) error { } return nil } + +// EvalExpireTime returns the expired time +func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se Session, now time.Time) (expire time.Time, err error) { + tz := se.GetSessionVars().TimeZone + + expireExpr := t.TTLInfo.IntervalExprStr + unit := ast.TimeUnitType(t.TTLInfo.IntervalTimeUnit) + + var rows []chunk.Row + rows, err = se.ExecuteSQL( + ctx, + // FROM_UNIXTIME does not support negative value, so we use `FROM_UNIXTIME(0) + INTERVAL ` to present current time + fmt.Sprintf("SELECT FROM_UNIXTIME(0) + INTERVAL %d SECOND - INTERVAL %s %s", now.Unix(), expireExpr, unit.String()), + ) + + if err != nil { + return + } + + tm := rows[0].GetTime(0) + return tm.CoreTime().GoTime(tz) +} diff --git a/ttl/table_test.go b/ttl/table_test.go new file mode 100644 index 0000000000000..f77556c98dc09 --- /dev/null +++ b/ttl/table_test.go @@ -0,0 +1,213 @@ +// 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 ttl_test + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/ttl" + "github.com/stretchr/testify/require" +) + +func TestNewTTLTable(t *testing.T) { + cases := []struct { + db string + tbl string + def string + timeCol string + keyCols []string + }{ + { + db: "test", + tbl: "t1", + def: "(a int)", + }, + { + db: "test", + tbl: "ttl1", + def: "(a int, t datetime) ttl = `t` + interval 2 hour", + timeCol: "t", + keyCols: []string{"_tidb_rowid"}, + }, + { + db: "test", + tbl: "ttl2", + def: "(id int primary key, t datetime) ttl = `t` + interval 3 hour", + timeCol: "t", + keyCols: []string{"id"}, + }, + { + db: "test", + tbl: "ttl3", + def: "(a int, b varchar(32), c binary(32), t datetime, primary key (a, b, c)) ttl = `t` + interval 1 month", + timeCol: "t", + keyCols: []string{"a", "b", "c"}, + }, + { + db: "test", + tbl: "ttl4", + def: "(id int primary key, t datetime) " + + "ttl = `t` + interval 1 day " + + "PARTITION BY RANGE (id) (" + + " PARTITION p0 VALUES LESS THAN (10)," + + " PARTITION p1 VALUES LESS THAN (100)," + + " PARTITION p2 VALUES LESS THAN (1000)," + + " PARTITION p3 VALUES LESS THAN MAXVALUE)", + timeCol: "t", + keyCols: []string{"id"}, + }, + { + db: "test", + tbl: "ttl5", + def: "(id int primary key nonclustered, t datetime) ttl = `t` + interval 3 hour", + timeCol: "t", + keyCols: []string{"_tidb_rowid"}, + }, + } + + store, do := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + for _, c := range cases { + tk.MustExec("use " + c.db) + tk.MustExec("create table " + c.tbl + c.def) + } + + for _, c := range cases { + is := do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr(c.db), model.NewCIStr(c.tbl)) + require.NoError(t, err) + tblInfo := tbl.Meta() + var physicalTbls []*ttl.PhysicalTable + if tblInfo.Partition == nil { + ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr("")) + if c.timeCol == "" { + require.Error(t, err) + continue + } + require.NoError(t, err) + physicalTbls = append(physicalTbls, ttlTbl) + } else { + for _, partition := range tblInfo.Partition.Definitions { + ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr(partition.Name.O)) + if c.timeCol == "" { + require.Error(t, err) + continue + } + require.NoError(t, err) + physicalTbls = append(physicalTbls, ttlTbl) + } + if c.timeCol == "" { + continue + } + } + + for i, ttlTbl := range physicalTbls { + require.Equal(t, c.db, ttlTbl.Schema.O) + require.Same(t, tblInfo, ttlTbl.TableInfo) + timeColumn := tblInfo.FindPublicColumnByName(c.timeCol) + require.NotNil(t, timeColumn) + require.Same(t, timeColumn, ttlTbl.TimeColumn) + + if tblInfo.Partition == nil { + require.Equal(t, "", ttlTbl.Partition.L) + require.Nil(t, ttlTbl.PartitionDef) + } else { + def := tblInfo.Partition.Definitions[i] + require.Equal(t, def.Name.L, ttlTbl.Partition.L) + require.Equal(t, def, *(ttlTbl.PartitionDef)) + } + + require.Equal(t, len(c.keyCols), len(ttlTbl.KeyColumns)) + require.Equal(t, len(c.keyCols), len(ttlTbl.KeyColumnTypes)) + + for j, keyCol := range c.keyCols { + msg := fmt.Sprintf("%s, col: %s", c.tbl, keyCol) + var col *model.ColumnInfo + if keyCol == model.ExtraHandleName.L { + col = model.NewExtraHandleColInfo() + } else { + col = tblInfo.FindPublicColumnByName(keyCol) + } + colJ := ttlTbl.KeyColumns[j] + colFieldJ := ttlTbl.KeyColumnTypes[j] + + require.NotNil(t, col, msg) + require.Equal(t, col.ID, colJ.ID, msg) + require.Equal(t, col.Name.L, colJ.Name.L, msg) + require.Equal(t, col.FieldType, colJ.FieldType, msg) + require.Equal(t, col.FieldType, *colFieldJ, msg) + } + } + } +} + +func TestEvalTTLExpireTime(t *testing.T) { + store, do := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.t(a int, t datetime) ttl = `t` + interval 1 day") + tk.MustExec("create table test.t2(a int, t datetime) ttl = `t` + interval 3 month") + + tb, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr("test"), tblInfo, model.NewCIStr("")) + require.NoError(t, err) + + tb2, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tblInfo2 := tb2.Meta() + ttlTbl2, err := ttl.NewPhysicalTable(model.NewCIStr("test"), tblInfo2, model.NewCIStr("")) + require.NoError(t, err) + + se := ttl.NewSession(tk.Session(), tk.Session(), nil) + + now := time.UnixMilli(0) + tz1, err := time.LoadLocation("Asia/Shanghai") + require.NoError(t, err) + tz2, err := time.LoadLocation("Europe/Berlin") + require.NoError(t, err) + + se.GetSessionVars().TimeZone = tz1 + tm, err := ttlTbl.EvalExpireTime(context.TODO(), se, now) + require.NoError(t, err) + require.Equal(t, now.Add(-time.Hour*24).Unix(), tm.Unix()) + require.Equal(t, "1969-12-31 08:00:00", tm.Format("2006-01-02 15:04:05")) + require.Equal(t, tz1.String(), tm.Location().String()) + + se.GetSessionVars().TimeZone = tz2 + tm, err = ttlTbl.EvalExpireTime(context.TODO(), se, now) + require.NoError(t, err) + require.Equal(t, now.Add(-time.Hour*24).Unix(), tm.Unix()) + require.Equal(t, "1969-12-31 01:00:00", tm.Format("2006-01-02 15:04:05")) + require.Equal(t, tz2.String(), tm.Location().String()) + + se.GetSessionVars().TimeZone = tz1 + tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) + require.NoError(t, err) + require.Equal(t, "1969-10-01 08:00:00", tm.Format("2006-01-02 15:04:05")) + require.Equal(t, tz1.String(), tm.Location().String()) + + se.GetSessionVars().TimeZone = tz2 + tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) + require.NoError(t, err) + require.Equal(t, "1969-10-01 01:00:00", tm.Format("2006-01-02 15:04:05")) + require.Equal(t, tz2.String(), tm.Location().String()) +} diff --git a/types/time.go b/types/time.go index 974b69602dd63..ebdfc462d8cdb 100644 --- a/types/time.go +++ b/types/time.go @@ -18,6 +18,7 @@ import ( "bytes" "encoding/json" "fmt" + "io" "math" "regexp" "strconv" @@ -1155,6 +1156,9 @@ func parseDatetime(sc *stmtctx.StatementContext, str string, fsp int, isFloat bo hhmmss = true } if err != nil { + if err == io.EOF { + return ZeroDatetime, errors.Trace(ErrWrongValue.GenWithStackByArgs(DateTimeStr, str)) + } return ZeroDatetime, errors.Trace(err) } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index c1c65499ec288..e3652b2c49aec 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -32,19 +32,24 @@ import ( // ExecDetails contains execution detail information. type ExecDetails struct { - BackoffSleep map[string]time.Duration - BackoffTimes map[string]int + DetailsNeedP90 CommitDetail *util.CommitDetails LockKeysDetail *util.LockKeysDetails ScanDetail *util.ScanDetail - CalleeAddress string - TimeDetail util.TimeDetail CopTime time.Duration BackoffTime time.Duration LockKeysDuration time.Duration RequestCount int } +// DetailsNeedP90 contains execution detail information which need calculate P90. +type DetailsNeedP90 struct { + BackoffSleep map[string]time.Duration + BackoffTimes map[string]int + CalleeAddress string + TimeDetail util.TimeDetail +} + type stmtExecDetailKeyType struct{} // StmtExecDetailKey used to carry StmtExecDetail info in context.Context. @@ -320,13 +325,15 @@ func (d ExecDetails) ToZapFields() (fields []zap.Field) { type basicCopRuntimeStats struct { storeType string BasicRuntimeStats - threads int32 + threads int32 + totalTasks int32 + procTimes []time.Duration } // String implements the RuntimeStats interface. func (e *basicCopRuntimeStats) String() string { if e.storeType == "tiflash" { - return fmt.Sprintf("time:%v, loops:%d, threads:%d", FormatDuration(time.Duration(e.consume)), e.loop, e.threads) + return fmt.Sprintf("time:%v, loops:%d, threads:%d, ", FormatDuration(time.Duration(e.consume)), e.loop, e.threads) + e.BasicRuntimeStats.tiflashScanContext.String() } return fmt.Sprintf("time:%v, loops:%d", FormatDuration(time.Duration(e.consume)), e.loop) } @@ -334,9 +341,11 @@ func (e *basicCopRuntimeStats) String() string { // Clone implements the RuntimeStats interface. func (e *basicCopRuntimeStats) Clone() RuntimeStats { return &basicCopRuntimeStats{ - BasicRuntimeStats: BasicRuntimeStats{loop: e.loop, consume: e.consume, rows: e.rows}, + BasicRuntimeStats: BasicRuntimeStats{loop: e.loop, consume: e.consume, rows: e.rows, tiflashScanContext: e.tiflashScanContext.Clone()}, threads: e.threads, storeType: e.storeType, + totalTasks: e.totalTasks, + procTimes: e.procTimes, } } @@ -350,6 +359,13 @@ func (e *basicCopRuntimeStats) Merge(rs RuntimeStats) { e.consume += tmp.consume e.rows += tmp.rows e.threads += tmp.threads + e.totalTasks += tmp.totalTasks + if len(tmp.procTimes) > 0 { + e.procTimes = append(e.procTimes, tmp.procTimes...) + } else { + e.procTimes = append(e.procTimes, time.Duration(tmp.consume)) + } + e.tiflashScanContext.Merge(tmp.tiflashScanContext) } // Tp implements the RuntimeStats interface. @@ -364,7 +380,7 @@ type CopRuntimeStats struct { // have many region leaders, several coprocessor tasks can be sent to the // same tikv-server instance. We have to use a list to maintain all tasks // executed on each instance. - stats map[string][]*basicCopRuntimeStats + stats map[string]*basicCopRuntimeStats scanDetail *util.ScanDetail // do not use kv.StoreType because it will meet cycle import error storeType string @@ -375,35 +391,48 @@ type CopRuntimeStats struct { func (crs *CopRuntimeStats) RecordOneCopTask(address string, summary *tipb.ExecutorExecutionSummary) { crs.Lock() defer crs.Unlock() - crs.stats[address] = append(crs.stats[address], - &basicCopRuntimeStats{BasicRuntimeStats: BasicRuntimeStats{loop: int32(*summary.NumIterations), + + if crs.stats[address] == nil { + crs.stats[address] = &basicCopRuntimeStats{ + storeType: crs.storeType, + } + } + crs.stats[address].Merge(&basicCopRuntimeStats{ + storeType: crs.storeType, + BasicRuntimeStats: BasicRuntimeStats{loop: int32(*summary.NumIterations), consume: int64(*summary.TimeProcessedNs), - rows: int64(*summary.NumProducedRows)}, - threads: int32(summary.GetConcurrency()), - storeType: crs.storeType}) + rows: int64(*summary.NumProducedRows), + tiflashScanContext: TiFlashScanContext{ + totalDmfileScannedPacks: summary.GetTiflashScanContext().GetTotalDmfileScannedPacks(), + totalDmfileSkippedPacks: summary.GetTiflashScanContext().GetTotalDmfileSkippedPacks(), + totalDmfileScannedRows: summary.GetTiflashScanContext().GetTotalDmfileScannedRows(), + totalDmfileSkippedRows: summary.GetTiflashScanContext().GetTotalDmfileSkippedRows(), + totalDmfileRoughSetIndexLoadTimeMs: summary.GetTiflashScanContext().GetTotalDmfileRoughSetIndexLoadTimeMs(), + totalDmfileReadTimeMs: summary.GetTiflashScanContext().GetTotalDmfileReadTimeMs(), + totalCreateSnapshotTimeMs: summary.GetTiflashScanContext().GetTotalCreateSnapshotTimeMs()}}, threads: int32(summary.GetConcurrency()), + totalTasks: 1, + }) } // GetActRows return total rows of CopRuntimeStats. func (crs *CopRuntimeStats) GetActRows() (totalRows int64) { for _, instanceStats := range crs.stats { - for _, stat := range instanceStats { - totalRows += stat.rows - } + totalRows += instanceStats.rows } return totalRows } // MergeBasicStats traverses basicCopRuntimeStats in the CopRuntimeStats and collects some useful information. -func (crs *CopRuntimeStats) MergeBasicStats() (procTimes []time.Duration, totalTime time.Duration, totalTasks, totalLoops, totalThreads int32) { +func (crs *CopRuntimeStats) MergeBasicStats() (procTimes []time.Duration, totalTime time.Duration, totalTasks, totalLoops, totalThreads int32, totalTiFlashScanContext TiFlashScanContext) { procTimes = make([]time.Duration, 0, 32) + totalTiFlashScanContext = TiFlashScanContext{} for _, instanceStats := range crs.stats { - for _, stat := range instanceStats { - procTimes = append(procTimes, time.Duration(stat.consume)*time.Nanosecond) - totalTime += time.Duration(stat.consume) - totalLoops += stat.loop - totalThreads += stat.threads - totalTasks++ - } + procTimes = append(procTimes, instanceStats.procTimes...) + totalTime += time.Duration(instanceStats.consume) + totalLoops += instanceStats.loop + totalThreads += instanceStats.threads + totalTiFlashScanContext.Merge(instanceStats.tiflashScanContext) + totalTasks += instanceStats.totalTasks } return } @@ -413,7 +442,7 @@ func (crs *CopRuntimeStats) String() string { return "" } - procTimes, totalTime, totalTasks, totalLoops, totalThreads := crs.MergeBasicStats() + procTimes, totalTime, totalTasks, totalLoops, totalThreads, totalTiFlashScanContext := crs.MergeBasicStats() avgTime := time.Duration(totalTime.Nanoseconds() / int64(totalTasks)) isTiFlashCop := crs.storeType == "tiflash" @@ -422,6 +451,9 @@ func (crs *CopRuntimeStats) String() string { buf.WriteString(fmt.Sprintf("%v_task:{time:%v, loops:%d", crs.storeType, FormatDuration(procTimes[0]), totalLoops)) if isTiFlashCop { buf.WriteString(fmt.Sprintf(", threads:%d}", totalThreads)) + if !totalTiFlashScanContext.Empty() { + buf.WriteString(", " + totalTiFlashScanContext.String()) + } } else { buf.WriteString("}") } @@ -433,6 +465,9 @@ func (crs *CopRuntimeStats) String() string { FormatDuration(procTimes[n*4/5]), FormatDuration(procTimes[n*19/20]), totalLoops, totalTasks)) if isTiFlashCop { buf.WriteString(fmt.Sprintf(", threads:%d}", totalThreads)) + if !totalTiFlashScanContext.Empty() { + buf.WriteString(", " + totalTiFlashScanContext.String()) + } } else { buf.WriteString("}") } @@ -490,6 +525,50 @@ type RuntimeStats interface { Tp() int } +// TiFlashScanContext is used to express the table scan information in tiflash +type TiFlashScanContext struct { + totalDmfileScannedPacks uint64 + totalDmfileScannedRows uint64 + totalDmfileSkippedPacks uint64 + totalDmfileSkippedRows uint64 + totalDmfileRoughSetIndexLoadTimeMs uint64 + totalDmfileReadTimeMs uint64 + totalCreateSnapshotTimeMs uint64 +} + +// Clone implements the deep copy of * TiFlashshScanContext +func (context *TiFlashScanContext) Clone() TiFlashScanContext { + return TiFlashScanContext{ + totalDmfileScannedPacks: context.totalDmfileScannedPacks, + totalDmfileScannedRows: context.totalDmfileScannedRows, + totalDmfileSkippedPacks: context.totalDmfileSkippedPacks, + totalDmfileSkippedRows: context.totalDmfileSkippedRows, + totalDmfileRoughSetIndexLoadTimeMs: context.totalDmfileRoughSetIndexLoadTimeMs, + totalDmfileReadTimeMs: context.totalDmfileReadTimeMs, + totalCreateSnapshotTimeMs: context.totalCreateSnapshotTimeMs, + } +} +func (context *TiFlashScanContext) String() string { + return fmt.Sprintf("tiflash_scan:{dmfile:{total_scanned_packs:%d, total_skipped_packs:%d, total_scanned_rows:%d, total_skipped_rows:%d, total_rough_set_index_load_time: %dms, total_read_time: %dms}, total_create_snapshot_time: %dms}", context.totalDmfileScannedPacks, context.totalDmfileSkippedPacks, context.totalDmfileScannedRows, context.totalDmfileSkippedRows, context.totalDmfileRoughSetIndexLoadTimeMs, context.totalDmfileReadTimeMs, context.totalCreateSnapshotTimeMs) +} + +// Merge make sum to merge the information in TiFlashScanContext +func (context *TiFlashScanContext) Merge(other TiFlashScanContext) { + context.totalDmfileScannedPacks += other.totalDmfileScannedPacks + context.totalDmfileScannedRows += other.totalDmfileScannedRows + context.totalDmfileSkippedPacks += other.totalDmfileSkippedPacks + context.totalDmfileSkippedRows += other.totalDmfileSkippedRows + context.totalDmfileRoughSetIndexLoadTimeMs += other.totalDmfileRoughSetIndexLoadTimeMs + context.totalDmfileReadTimeMs += other.totalDmfileReadTimeMs + context.totalCreateSnapshotTimeMs += other.totalCreateSnapshotTimeMs +} + +// Empty check whether TiFlashScanContext is Empty, if scan no pack and skip no pack, we regard it as empty +func (context *TiFlashScanContext) Empty() bool { + res := (context.totalDmfileScannedPacks == 0 && context.totalDmfileSkippedPacks == 0) + return res +} + // BasicRuntimeStats is the basic runtime stats. type BasicRuntimeStats struct { // executor's Next() called times. @@ -498,6 +577,8 @@ type BasicRuntimeStats struct { consume int64 // executor return row count. rows int64 + // executor extra infos + tiflashScanContext TiFlashScanContext } // GetActRows return total rows of BasicRuntimeStats. @@ -508,9 +589,10 @@ func (e *BasicRuntimeStats) GetActRows() int64 { // Clone implements the RuntimeStats interface. func (e *BasicRuntimeStats) Clone() RuntimeStats { return &BasicRuntimeStats{ - loop: e.loop, - consume: e.consume, - rows: e.rows, + loop: e.loop, + consume: e.consume, + rows: e.rows, + tiflashScanContext: e.tiflashScanContext.Clone(), } } @@ -523,6 +605,7 @@ func (e *BasicRuntimeStats) Merge(rs RuntimeStats) { e.loop += tmp.loop e.consume += tmp.consume e.rows += tmp.rows + e.tiflashScanContext.Merge(tmp.tiflashScanContext) } // Tp implements the RuntimeStats interface. @@ -532,57 +615,23 @@ func (*BasicRuntimeStats) Tp() int { // RootRuntimeStats is the executor runtime stats that combine with multiple runtime stats. type RootRuntimeStats struct { - basics []*BasicRuntimeStats - groupRss [][]RuntimeStats + basic *BasicRuntimeStats + groupRss []RuntimeStats } -// GetActRows return total rows of RootRuntimeStats. -func (e *RootRuntimeStats) GetActRows() int64 { - num := int64(0) - for _, basic := range e.basics { - num += basic.GetActRows() - } - return num -} - -// MergeBasicStats merges BasicRuntimeStats in the RootRuntimeStats into single one. -func (e *RootRuntimeStats) MergeBasicStats() *BasicRuntimeStats { - if len(e.basics) == 0 { - return nil - } - basic := e.basics[0].Clone().(*BasicRuntimeStats) - for i := 1; i < len(e.basics); i++ { - basic.Merge(e.basics[i]) - } - return basic +// NewRootRuntimeStats returns a new RootRuntimeStats +func NewRootRuntimeStats() *RootRuntimeStats { + return &RootRuntimeStats{basic: &BasicRuntimeStats{}} } -// MergeGroupStats merges every slice in e.groupRss into single RuntimeStats. -func (e *RootRuntimeStats) MergeGroupStats() (res []RuntimeStats) { - if len(e.groupRss) == 0 { - return nil - } - for _, rss := range e.groupRss { - if len(rss) == 0 { - continue - } else if len(rss) == 1 { - res = append(res, rss[0]) - continue - } - rs := rss[0].Clone() - for i := 1; i < len(rss); i++ { - rs.Merge(rss[i]) - } - res = append(res, rs) - } - return +// GetActRows return total rows of RootRuntimeStats. +func (e *RootRuntimeStats) GetActRows() int64 { + return e.basic.rows } // MergeStats merges stats in the RootRuntimeStats and return the stats suitable for display directly. func (e *RootRuntimeStats) MergeStats() (basic *BasicRuntimeStats, groups []RuntimeStats) { - basic = e.MergeBasicStats() - groups = e.MergeGroupStats() - return + return e.basic, e.groupRss } // String implements the RuntimeStats interface. @@ -664,38 +713,46 @@ func (e *RuntimeStatsColl) RegisterStats(planID int, info RuntimeStats) { e.mu.Lock() stats, ok := e.rootStats[planID] if !ok { - stats = &RootRuntimeStats{} + stats = NewRootRuntimeStats() e.rootStats[planID] = stats } - if basic, ok := info.(*BasicRuntimeStats); ok { - stats.basics = append(stats.basics, basic) - } else { - tp := info.Tp() - found := false - for i, rss := range stats.groupRss { - if len(rss) == 0 { - continue - } - if rss[0].Tp() == tp { - stats.groupRss[i] = append(stats.groupRss[i], info) - found = true - break - } - } - if !found { - stats.groupRss = append(stats.groupRss, []RuntimeStats{info}) + tp := info.Tp() + found := false + for _, rss := range stats.groupRss { + if rss.Tp() == tp { + rss.Merge(info) + found = true + break } } + if !found { + stats.groupRss = append(stats.groupRss, info.Clone()) + } e.mu.Unlock() } +// GetBasicRuntimeStats gets basicRuntimeStats for a executor. +func (e *RuntimeStatsColl) GetBasicRuntimeStats(planID int) *BasicRuntimeStats { + e.mu.Lock() + defer e.mu.Unlock() + stats, ok := e.rootStats[planID] + if !ok { + stats = NewRootRuntimeStats() + e.rootStats[planID] = stats + } + if stats.basic == nil { + stats.basic = &BasicRuntimeStats{} + } + return stats.basic +} + // GetRootStats gets execStat for a executor. func (e *RuntimeStatsColl) GetRootStats(planID int) *RootRuntimeStats { e.mu.Lock() defer e.mu.Unlock() runtimeStats, exists := e.rootStats[planID] if !exists { - runtimeStats = &RootRuntimeStats{} + runtimeStats = NewRootRuntimeStats() e.rootStats[planID] = runtimeStats } return runtimeStats @@ -719,7 +776,7 @@ func (e *RuntimeStatsColl) GetOrCreateCopStats(planID int, storeType string) *Co copStats, ok := e.copStats[planID] if !ok { copStats = &CopRuntimeStats{ - stats: make(map[string][]*basicCopRuntimeStats), + stats: make(map[string]*basicCopRuntimeStats), scanDetail: &util.ScanDetail{}, storeType: storeType, } diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 1f14f9f933509..a7c17ca43db87 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -109,10 +109,10 @@ func TestString(t *testing.T) { RocksdbBlockReadByte: 100, RocksdbBlockReadDuration: time.Millisecond, }, - TimeDetail: util.TimeDetail{ + DetailsNeedP90: DetailsNeedP90{TimeDetail: util.TimeDetail{ ProcessTime: 2*time.Second + 5*time.Millisecond, WaitTime: time.Second, - }, + }}, } expected := "Cop_time: 1.003 Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Prewrite_time: 1 Commit_time: " + "1 Get_commit_ts_time: 1 Get_latest_ts_time: 1 Commit_backoff_time: 1 " + @@ -135,9 +135,18 @@ func mockExecutorExecutionSummary(TimeProcessedNs, NumProducedRows, NumIteration NumIterations: &NumIterations, XXX_unrecognized: nil} } -func mockExecutorExecutionSummaryForTiFlash(TimeProcessedNs, NumProducedRows, NumIterations, Concurrency uint64, ExecutorID string) *tipb.ExecutorExecutionSummary { +func mockExecutorExecutionSummaryForTiFlash(TimeProcessedNs, NumProducedRows, NumIterations, Concurrency, totalDmfileScannedPacks, totalDmfileScannedRows, totalDmfileSkippedPacks, totalDmfileSkippedRows, totalDmfileRoughSetIndexLoadTimeMs, totalDmfileReadTimeMs, totalCreateSnapshotTimeMs uint64, ExecutorID string) *tipb.ExecutorExecutionSummary { + tiflashScanContext := tipb.TiFlashScanContext{ + TotalDmfileScannedPacks: &totalDmfileScannedPacks, + TotalDmfileSkippedPacks: &totalDmfileSkippedPacks, + TotalDmfileScannedRows: &totalDmfileScannedRows, + TotalDmfileSkippedRows: &totalDmfileSkippedRows, + TotalDmfileRoughSetIndexLoadTimeMs: &totalDmfileRoughSetIndexLoadTimeMs, + TotalDmfileReadTimeMs: &totalDmfileReadTimeMs, + TotalCreateSnapshotTimeMs: &totalCreateSnapshotTimeMs, + } return &tipb.ExecutorExecutionSummary{TimeProcessedNs: &TimeProcessedNs, NumProducedRows: &NumProducedRows, - NumIterations: &NumIterations, Concurrency: &Concurrency, ExecutorId: &ExecutorID, XXX_unrecognized: nil} + NumIterations: &NumIterations, Concurrency: &Concurrency, ExecutorId: &ExecutorID, DetailInfo: &tipb.ExecutorExecutionSummary_TiflashScanContext{TiflashScanContext: &tiflashScanContext}, XXX_unrecognized: nil} } func TestCopRuntimeStats(t *testing.T) { @@ -170,9 +179,11 @@ func TestCopRuntimeStats(t *testing.T) { copStats := cop.stats["8.8.8.8"] require.NotNil(t, copStats) - copStats[0].SetRowNum(10) - copStats[0].Record(time.Second, 10) - require.Equal(t, "time:1s, loops:2", copStats[0].String()) + newCopStats := &basicCopRuntimeStats{} + newCopStats.SetRowNum(10) + newCopStats.Record(time.Second, 10) + copStats.Merge(newCopStats) + require.Equal(t, "time:1s, loops:2", copStats.String()) require.Equal(t, "tikv_task:{proc max:4ns, min:3ns, avg: 3ns, p80:4ns, p95:4ns, iters:7, tasks:2}", stats.GetOrCreateCopStats(aggID, "tikv").String()) rootStats := stats.GetRootStats(tableReaderID) @@ -184,7 +195,7 @@ func TestCopRuntimeStats(t *testing.T) { cop.scanDetail.RocksdbKeySkippedCount = 0 cop.scanDetail.RocksdbBlockReadCount = 0 // Print all fields even though the value of some fields is 0. - str := "tikv_task:{proc max:1s, min:2ns, avg: 500ms, p80:1s, p95:1s, iters:4, tasks:2}, " + + str := "tikv_task:{proc max:1s, min:1ns, avg: 500ms, p80:1s, p95:1s, iters:4, tasks:2}, " + "scan_detail: {total_keys: 15, rocksdb: {delete_skipped_count: 5, block: {cache_hit_count: 10, read_byte: 100 Bytes}}}" require.Equal(t, str, cop.String()) @@ -197,10 +208,10 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) { tableScanID := 1 aggID := 2 tableReaderID := 3 - stats.RecordOneCopTask(aggID, "tiflash", "8.8.8.8", mockExecutorExecutionSummaryForTiFlash(1, 1, 1, 1, "tablescan_"+strconv.Itoa(tableScanID))) - stats.RecordOneCopTask(aggID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(2, 2, 2, 1, "tablescan_"+strconv.Itoa(tableScanID))) - stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.8", mockExecutorExecutionSummaryForTiFlash(3, 3, 3, 1, "aggregation_"+strconv.Itoa(aggID))) - stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(4, 4, 4, 1, "aggregation_"+strconv.Itoa(aggID))) + stats.RecordOneCopTask(aggID, "tiflash", "8.8.8.8", mockExecutorExecutionSummaryForTiFlash(1, 1, 1, 1, 1, 8192, 0, 0, 15, 200, 40, "tablescan_"+strconv.Itoa(tableScanID))) + stats.RecordOneCopTask(aggID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(2, 2, 2, 1, 0, 0, 0, 0, 0, 2, 0, "tablescan_"+strconv.Itoa(tableScanID))) + stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.8", mockExecutorExecutionSummaryForTiFlash(3, 3, 3, 1, 2, 12000, 1, 6000, 60, 1000, 20, "aggregation_"+strconv.Itoa(aggID))) + stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(4, 4, 4, 1, 1, 8192, 10, 80000, 40, 2000, 30, "aggregation_"+strconv.Itoa(aggID))) scanDetail := &util.ScanDetail{ TotalKeys: 10, ProcessedKeys: 10, @@ -214,15 +225,15 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) { require.True(t, stats.ExistsCopStats(tableScanID)) cop := stats.GetOrCreateCopStats(tableScanID, "tiflash") - require.Equal(t, "tiflash_task:{proc max:2ns, min:1ns, avg: 1ns, p80:2ns, p95:2ns, iters:3, tasks:2, threads:2}", cop.String()) + require.Equal(t, "tiflash_task:{proc max:2ns, min:1ns, avg: 1ns, p80:2ns, p95:2ns, iters:3, tasks:2, threads:2}, tiflash_scan:{dmfile:{total_scanned_packs:1, total_skipped_packs:0, total_scanned_rows:8192, total_skipped_rows:0, total_rough_set_index_load_time: 15ms, total_read_time: 202ms}, total_create_snapshot_time: 40ms}", cop.String()) copStats := cop.stats["8.8.8.8"] require.NotNil(t, copStats) - copStats[0].SetRowNum(10) - copStats[0].Record(time.Second, 10) - require.Equal(t, "time:1s, loops:2, threads:1", copStats[0].String()) - expected := "tiflash_task:{proc max:4ns, min:3ns, avg: 3ns, p80:4ns, p95:4ns, iters:7, tasks:2, threads:2}" + copStats.SetRowNum(10) + copStats.Record(time.Second, 10) + require.Equal(t, "time:1s, loops:2, threads:1, tiflash_scan:{dmfile:{total_scanned_packs:1, total_skipped_packs:0, total_scanned_rows:8192, total_skipped_rows:0, total_rough_set_index_load_time: 15ms, total_read_time: 200ms}, total_create_snapshot_time: 40ms}", copStats.String()) + expected := "tiflash_task:{proc max:4ns, min:3ns, avg: 3ns, p80:4ns, p95:4ns, iters:7, tasks:2, threads:2}, tiflash_scan:{dmfile:{total_scanned_packs:3, total_skipped_packs:11, total_scanned_rows:20192, total_skipped_rows:86000, total_rough_set_index_load_time: 100ms, total_read_time: 3000ms}, total_create_snapshot_time: 50ms}" require.Equal(t, expected, stats.GetOrCreateCopStats(aggID, "tiflash").String()) rootStats := stats.GetRootStats(tableReaderID) @@ -376,17 +387,14 @@ func TestRuntimeStatsWithCommit(t *testing.T) { } func TestRootRuntimeStats(t *testing.T) { - basic1 := &BasicRuntimeStats{} - basic2 := &BasicRuntimeStats{} - basic1.Record(time.Second, 20) - basic2.Record(time.Second*2, 30) pid := 1 stmtStats := NewRuntimeStatsColl(nil) - stmtStats.RegisterStats(pid, basic1) - stmtStats.RegisterStats(pid, basic2) + basic1 := stmtStats.GetBasicRuntimeStats(pid) + basic2 := stmtStats.GetBasicRuntimeStats(pid) + basic1.Record(time.Second, 20) + basic2.Record(time.Second*2, 30) concurrency := &RuntimeStatsWithConcurrencyInfo{} concurrency.SetConcurrencyInfo(NewConcurrencyInfo("worker", 15)) - stmtStats.RegisterStats(pid, concurrency) commitDetail := &util.CommitDetails{ GetCommitTsTime: time.Second, PrewriteTime: time.Second, @@ -396,6 +404,7 @@ func TestRootRuntimeStats(t *testing.T) { PrewriteRegionNum: 5, TxnRetry: 2, } + stmtStats.RegisterStats(pid, concurrency) stmtStats.RegisterStats(pid, &RuntimeStatsWithCommit{ Commit: commitDetail, }) diff --git a/util/expensivequery/expensivequery.go b/util/expensivequery/expensivequery.go index e761d98ab05a2..88eb24bdd01dd 100644 --- a/util/expensivequery/expensivequery.go +++ b/util/expensivequery/expensivequery.go @@ -62,9 +62,9 @@ func (eqh *Handle) Run() { } costTime := time.Since(info.Time) - if !info.ExceedExpensiveTimeThresh && costTime >= time.Second*time.Duration(threshold) && log.GetLevel() <= zapcore.WarnLevel { + if time.Since(info.ExpensiveLogTime) > 60*time.Second && costTime >= time.Second*time.Duration(threshold) && log.GetLevel() <= zapcore.WarnLevel { logExpensiveQuery(costTime, info, "expensive_query") - info.ExceedExpensiveTimeThresh = true + info.ExpensiveLogTime = time.Now() } if info.MaxExecutionTime > 0 && costTime > time.Duration(info.MaxExecutionTime)*time.Millisecond { logutil.BgLogger().Warn("execution timeout, kill it", zap.Duration("costTime", costTime), @@ -111,5 +111,9 @@ func (eqh *Handle) LogOnQueryExceedMemQuota(connID uint64) { // logExpensiveQuery logs the queries which exceed the time threshold or memory threshold. func logExpensiveQuery(costTime time.Duration, info *util.ProcessInfo, msg string) { - logutil.BgLogger().Warn(msg, util.GenLogFields(costTime, info, true)...) + fields := util.GenLogFields(costTime, info, true) + if fields == nil { + return + } + logutil.BgLogger().Warn(msg, fields...) } diff --git a/util/gctuner/memory_limit_tuner.go b/util/gctuner/memory_limit_tuner.go index 9dba1bd0bd546..1679e012579d0 100644 --- a/util/gctuner/memory_limit_tuner.go +++ b/util/gctuner/memory_limit_tuner.go @@ -38,6 +38,9 @@ type memoryLimitTuner struct { nextGCTriggeredByMemoryLimit atomicutil.Bool } +// fallbackPercentage indicates the fallback memory limit percentage when turning. +const fallbackPercentage float64 = 1.1 + // tuning check the memory nextGC and judge whether this GC is trigger by memory limit. // Go runtime ensure that it will be called serially. func (t *memoryLimitTuner) tuning() { @@ -61,7 +64,7 @@ func (t *memoryLimitTuner) tuning() { go func() { memory.MemoryLimitGCLast.Store(time.Now()) memory.MemoryLimitGCTotal.Add(1) - debug.SetMemoryLimit(math.MaxInt64) + debug.SetMemoryLimit(t.calcMemoryLimit(fallbackPercentage)) resetInterval := 1 * time.Minute // Wait 1 minute and set back, to avoid frequent GC failpoint.Inject("testMemoryLimitTuner", func(val failpoint.Value) { if val, ok := val.(bool); val && ok { @@ -69,7 +72,7 @@ func (t *memoryLimitTuner) tuning() { } }) time.Sleep(resetInterval) - debug.SetMemoryLimit(t.calcMemoryLimit()) + debug.SetMemoryLimit(t.calcMemoryLimit(t.GetPercentage())) for !t.waitingReset.CompareAndSwap(true, false) { continue } @@ -106,23 +109,27 @@ func (t *memoryLimitTuner) GetPercentage() float64 { // UpdateMemoryLimit updates the memory limit. // This function should be called when `tidb_server_memory_limit` or `tidb_server_memory_limit_gc_trigger` is modified. func (t *memoryLimitTuner) UpdateMemoryLimit() { - var memoryLimit = t.calcMemoryLimit() + var memoryLimit = t.calcMemoryLimit(t.GetPercentage()) if memoryLimit == math.MaxInt64 { t.isTuning.Store(false) + memoryLimit = initGOMemoryLimitValue } else { t.isTuning.Store(true) } debug.SetMemoryLimit(memoryLimit) } -func (t *memoryLimitTuner) calcMemoryLimit() int64 { - memoryLimit := int64(float64(memory.ServerMemoryLimit.Load()) * t.percentage.Load()) // `tidb_server_memory_limit` * `tidb_server_memory_limit_gc_trigger` +func (*memoryLimitTuner) calcMemoryLimit(percentage float64) int64 { + memoryLimit := int64(float64(memory.ServerMemoryLimit.Load()) * percentage) // `tidb_server_memory_limit` * `tidb_server_memory_limit_gc_trigger` if memoryLimit == 0 { memoryLimit = math.MaxInt64 } return memoryLimit } +var initGOMemoryLimitValue int64 + func init() { + initGOMemoryLimitValue = debug.SetMemoryLimit(-1) GlobalMemoryLimitTuner.Start() } diff --git a/util/gctuner/memory_limit_tuner_test.go b/util/gctuner/memory_limit_tuner_test.go index 47d1d8409d8b5..c6f63215c01dd 100644 --- a/util/gctuner/memory_limit_tuner_test.go +++ b/util/gctuner/memory_limit_tuner_test.go @@ -15,7 +15,6 @@ package gctuner import ( - "math" "runtime" "runtime/debug" "testing" @@ -76,10 +75,9 @@ func TestGlobalMemoryTuner(t *testing.T) { checkNextGCEqualMemoryLimit := func() { runtime.ReadMemStats(r) nextGC := r.NextGC - memoryLimit := GlobalMemoryLimitTuner.calcMemoryLimit() - // In golang source, nextGC = memoryLimit - three parts memory. So check 90%~100% here. + memoryLimit := GlobalMemoryLimitTuner.calcMemoryLimit(GlobalMemoryLimitTuner.GetPercentage()) + // In golang source, nextGC = memoryLimit - three parts memory. require.True(t, nextGC < uint64(memoryLimit)) - require.True(t, nextGC > uint64(memoryLimit)/10*9) } memory600mb := allocator.alloc(600 << 20) @@ -91,7 +89,7 @@ func TestGlobalMemoryTuner(t *testing.T) { require.True(t, gcNum < getNowGCNum()) // Test waiting for reset time.Sleep(500 * time.Millisecond) - require.Equal(t, int64(math.MaxInt64), debug.SetMemoryLimit(-1)) + require.Equal(t, GlobalMemoryLimitTuner.calcMemoryLimit(fallbackPercentage), debug.SetMemoryLimit(-1)) gcNum = getNowGCNum() memory100mb := allocator.alloc(100 << 20) time.Sleep(100 * time.Millisecond) @@ -102,7 +100,7 @@ func TestGlobalMemoryTuner(t *testing.T) { runtime.GC() // Trigger GC in 80% again time.Sleep(500 * time.Millisecond) - require.Equal(t, GlobalMemoryLimitTuner.calcMemoryLimit(), debug.SetMemoryLimit(-1)) + require.Equal(t, GlobalMemoryLimitTuner.calcMemoryLimit(GlobalMemoryLimitTuner.GetPercentage()), debug.SetMemoryLimit(-1)) time.Sleep(100 * time.Millisecond) gcNum = getNowGCNum() checkNextGCEqualMemoryLimit() diff --git a/util/hint/hint_processor.go b/util/hint/hint_processor.go index b087981abd409..be1a1a4e7fc4c 100644 --- a/util/hint/hint_processor.go +++ b/util/hint/hint_processor.go @@ -325,8 +325,9 @@ type BlockHintProcessor struct { QbHints map[int][]*ast.TableOptimizerHint // Group all hints at same query block. // Used for the view's hint - QbNameMap4View map[string][]ast.HintTable // Map from view's query block name to view's table list. - QbHints4View map[string][]*ast.TableOptimizerHint // Group all hints at same query block for view hints. + QbNameMap4View map[string][]ast.HintTable // Map from view's query block name to view's table list. + QbHints4View map[string][]*ast.TableOptimizerHint // Group all hints at same query block for view hints. + QbNameUsed4View map[string]struct{} // Store all the qb_name hints which are used for view Ctx sessionctx.Context selectStmtOffset int @@ -346,15 +347,14 @@ func (p *BlockHintProcessor) Enter(in ast.Node) (ast.Node, bool) { p.checkQueryBlockHints(node.TableHints, 0) case *ast.SelectStmt: p.selectStmtOffset++ - // Only support view hints which appear in the outer select part - if p.selectStmtOffset == 1 { - // Handle the view hints and update the left hint. - node.TableHints = p.handleViewHints(node.TableHints) - } node.QueryBlockOffset = p.selectStmtOffset + // Handle the view hints and update the left hint. + node.TableHints = p.handleViewHints(node.TableHints, node.QueryBlockOffset) p.checkQueryBlockHints(node.TableHints, node.QueryBlockOffset) case *ast.ExplainStmt: return in, true + case *ast.CreateBindingStmt: + return in, true } return in, false } @@ -373,12 +373,6 @@ func (p *BlockHintProcessor) checkQueryBlockHints(hints []*ast.TableOptimizerHin if hint.HintName.L != hintQBName { continue } - if offset > 1 && len(hint.Tables) > 0 { - if p.Ctx != nil { - p.Ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("The qb_name hint for view only supports to be defined in the first query block")) - } - continue - } if qbName != "" { if p.Ctx != nil { p.Ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("There are more than two query names in same query block, using the first one %s", qbName)) @@ -402,7 +396,7 @@ func (p *BlockHintProcessor) checkQueryBlockHints(hints []*ast.TableOptimizerHin } } -func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint) (leftHints []*ast.TableOptimizerHint) { +func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint, offset int) (leftHints []*ast.TableOptimizerHint) { if len(hints) == 0 { return } @@ -416,6 +410,7 @@ func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint) (l usedHints[i] = true if p.QbNameMap4View == nil { p.QbNameMap4View = make(map[string][]ast.HintTable) + p.QbNameUsed4View = make(map[string]struct{}) } qbName := hint.QBName.L if qbName == "" { @@ -426,6 +421,14 @@ func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint) (l p.Ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("Duplicate query block name %s for view's query block hint, only the first one is effective", qbName)) } } else { + if offset != 1 { + // If there are some qb_name hints for view are not defined in the first query block, + // we should add the query block number where it is located to the first table in the view's qb_name hint table list. + qbNum := hint.Tables[0].QBName.L + if qbNum == "" { + hint.Tables[0].QBName = model.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, offset)) + } + } p.QbNameMap4View[qbName] = hint.Tables } } @@ -475,6 +478,18 @@ func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint) (l return } +// HandleUnusedViewHints handle the unused view hints. +func (p *BlockHintProcessor) HandleUnusedViewHints() { + if p.QbNameMap4View != nil { + for qbName := range p.QbNameMap4View { + _, ok := p.QbNameUsed4View[qbName] + if !ok && p.Ctx != nil { + p.Ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("The qb_name hint %s is unused, please check whether the table list in the qb_name hint %s is correct", qbName, qbName)) + } + } + } +} + const ( defaultUpdateBlockName = "upd_1" defaultDeleteBlockName = "del_1" diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 1604f65015809..9c2adf31ace14 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -24,7 +24,9 @@ import ( "time" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/util/logutil" atomicutil "go.uber.org/atomic" + "go.uber.org/zap" "golang.org/x/exp/slices" ) @@ -91,8 +93,9 @@ type Tracker struct { maxConsumed atomicutil.Int64 // max number of bytes consumed during execution. SessionID uint64 // SessionID indicates the sessionID the tracker is bound. NeedKill atomic.Bool // NeedKill indicates whether this session need kill because OOM - IsRootTrackerOfSess bool // IsRootTrackerOfSess indicates whether this tracker is bound for session - isGlobal bool // isGlobal indicates whether this tracker is global tracker + NeedKillReceived sync.Once + IsRootTrackerOfSess bool // IsRootTrackerOfSess indicates whether this tracker is bound for session + isGlobal bool // isGlobal indicates whether this tracker is global tracker } type actionMu struct { @@ -316,6 +319,7 @@ func (t *Tracker) Detach() { parent.actionMuForSoftLimit.actionOnExceed = nil parent.actionMuForSoftLimit.Unlock() parent.NeedKill.Store(false) + parent.NeedKillReceived = sync.Once{} } parent.remove(t) t.mu.Lock() @@ -449,6 +453,11 @@ func (t *Tracker) Consume(bs int64) { if bs > 0 && sessionRootTracker != nil { // Kill the Top1 session if sessionRootTracker.NeedKill.Load() { + sessionRootTracker.NeedKillReceived.Do( + func() { + logutil.BgLogger().Warn("global memory controller, NeedKill signal is received successfully", + zap.Uint64("connID", sessionRootTracker.SessionID)) + }) tryActionLastOne(&sessionRootTracker.actionMuForHardLimit, sessionRootTracker) } // Update the Top1 session diff --git a/util/memoryusagealarm/memoryusagealarm.go b/util/memoryusagealarm/memoryusagealarm.go index 882d8462ef60b..c8a6fd0eaecda 100644 --- a/util/memoryusagealarm/memoryusagealarm.go +++ b/util/memoryusagealarm/memoryusagealarm.go @@ -266,15 +266,16 @@ func (record *memoryUsageAlarm) printTop10SqlInfo(pinfo []*util.ProcessInfo, f * func (record *memoryUsageAlarm) getTop10SqlInfo(cmp func(i, j *util.ProcessInfo) bool, pinfo []*util.ProcessInfo) strings.Builder { slices.SortFunc(pinfo, cmp) list := pinfo - if len(list) > 10 { - list = list[:10] - } var buf strings.Builder oomAction := variable.OOMAction.Load() serverMemoryLimit := memory.ServerMemoryLimit.Load() - for i, info := range list { + for i, totalCnt := 0, 10; i < len(list) && totalCnt > 0; i++ { + info := list[i] buf.WriteString(fmt.Sprintf("SQL %v: \n", i)) fields := util.GenLogFields(record.lastCheckTime.Sub(info.Time), info, false) + if fields == nil { + continue + } fields = append(fields, zap.String("tidb_mem_oom_action", oomAction)) fields = append(fields, zap.Uint64("tidb_server_memory_limit", serverMemoryLimit)) fields = append(fields, zap.Int64("tidb_mem_quota_query", info.OOMAlarmVariablesInfo.SessionMemQuotaQuery)) @@ -294,6 +295,7 @@ func (record *memoryUsageAlarm) getTop10SqlInfo(cmp func(i, j *util.ProcessInfo) } buf.WriteString("\n") } + totalCnt-- } buf.WriteString("\n") return buf diff --git a/util/memoryusagealarm/memoryusagealarm_test.go b/util/memoryusagealarm/memoryusagealarm_test.go index f0b0af6bd99ec..6e5147805676f 100644 --- a/util/memoryusagealarm/memoryusagealarm_test.go +++ b/util/memoryusagealarm/memoryusagealarm_test.go @@ -104,12 +104,14 @@ func genMockProcessInfoList(memConsumeList []int64, startTimeList []time.Time, s for i := 0; i < size; i++ { tracker := memory.NewTracker(0, 0) tracker.Consume(memConsumeList[i]) + var stmtCtxRefCount stmtctx.ReferenceCount = 0 processInfo := util.ProcessInfo{Time: startTimeList[i], StmtCtx: &stmtctx.StatementContext{}, MemTracker: tracker, StatsInfo: func(interface{}) map[string]uint64 { return map[string]uint64{} }, + RefCountOfStmtCtx: &stmtCtxRefCount, } processInfoList = append(processInfoList, &processInfo) } diff --git a/util/processinfo.go b/util/processinfo.go index 6e379e49a8fe7..77f35ef94a5ee 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -41,8 +41,10 @@ type OOMAlarmVariablesInfo struct { // ProcessInfo is a struct used for show processlist statement. type ProcessInfo struct { Time time.Time + ExpensiveLogTime time.Time Plan interface{} StmtCtx *stmtctx.StatementContext + RefCountOfStmtCtx *stmtctx.ReferenceCount MemTracker *memory.Tracker DiskTracker *disk.Tracker StatsInfo func(interface{}) map[string]uint64 @@ -59,11 +61,10 @@ type ProcessInfo struct { CurTxnStartTS uint64 // MaxExecutionTime is the timeout for select statement, in milliseconds. // If the query takes too long, kill it. - MaxExecutionTime uint64 - State uint16 - Command byte - ExceedExpensiveTimeThresh bool - RedactSQL bool + MaxExecutionTime uint64 + State uint16 + Command byte + RedactSQL bool } // ToRowForShow returns []interface{} for the row data of "SHOW [FULL] PROCESSLIST". diff --git a/util/servermemorylimit/BUILD.bazel b/util/servermemorylimit/BUILD.bazel index 9e144b820922e..0d2c4d4f3cb59 100644 --- a/util/servermemorylimit/BUILD.bazel +++ b/util/servermemorylimit/BUILD.bazel @@ -9,8 +9,10 @@ go_library( "//parser/mysql", "//types", "//util", + "//util/logutil", "//util/memory", "@org_uber_go_atomic//:atomic", + "@org_uber_go_zap//:zap", ], ) diff --git a/util/servermemorylimit/servermemorylimit.go b/util/servermemorylimit/servermemorylimit.go index a13e2e8b0e081..511a86703db17 100644 --- a/util/servermemorylimit/servermemorylimit.go +++ b/util/servermemorylimit/servermemorylimit.go @@ -24,8 +24,10 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" atomicutil "go.uber.org/atomic" + "go.uber.org/zap" ) // Process global Observation indicators for memory limit. @@ -81,12 +83,24 @@ type sessionToBeKilled struct { sqlStartTime time.Time sessionID uint64 sessionTracker *memory.Tracker + + killStartTime time.Time + lastLogTime time.Time } func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) { if s.isKilling { if info, ok := sm.GetProcessInfo(s.sessionID); ok { if info.Time == s.sqlStartTime { + if time.Since(s.lastLogTime) > 5*time.Second { + logutil.BgLogger().Warn(fmt.Sprintf("global memory controller failed to kill the top-consumer in %ds", + time.Since(s.killStartTime)/time.Second), + zap.Uint64("connID", info.ID), + zap.String("sql digest", info.Digest), + zap.String("sql text", fmt.Sprintf("%.100v", info.Info)), + zap.Int64("sql memory usage", info.MemTracker.BytesConsumed())) + s.lastLogTime = time.Now() + } return } } @@ -95,6 +109,7 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) { memory.MemUsageTop1Tracker.CompareAndSwap(s.sessionTracker, nil) //nolint: all_revive,revive runtime.GC() + logutil.BgLogger().Warn("global memory controller killed the top1 memory consumer successfully") } if bt == 0 { @@ -108,6 +123,14 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) { t := memory.MemUsageTop1Tracker.Load() if t != nil { if info, ok := sm.GetProcessInfo(t.SessionID); ok { + logutil.BgLogger().Warn("global memory controller tries to kill the top1 memory consumer", + zap.Uint64("connID", info.ID), + zap.String("sql digest", info.Digest), + zap.String("sql text", fmt.Sprintf("%.100v", info.Info)), + zap.Uint64("tidb_server_memory_limit", bt), + zap.Uint64("heap inuse", instanceStats.HeapInuse), + zap.Int64("sql memory usage", info.MemTracker.BytesConsumed()), + ) s.sessionID = t.SessionID s.sqlStartTime = info.Time s.isKilling = true @@ -119,6 +142,8 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) { SessionKillLast.Store(killTime) IsKilling.Store(true) GlobalMemoryOpsHistoryManager.recordOne(info, killTime, bt, instanceStats.HeapInuse) + s.lastLogTime = time.Now() + s.killStartTime = time.Now() } } } diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 8ee767f2342e0..ac7e1b06e059b 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -175,9 +175,8 @@ func TestAddStatement(t *testing.T) { MaxWaitTime: 2500, }, ExecDetail: &execdetails.ExecDetails{ - CalleeAddress: "202", - BackoffTime: 180, - RequestCount: 20, + BackoffTime: 180, + RequestCount: 20, CommitDetail: &util.CommitDetails{ GetCommitTsTime: 500, PrewriteTime: 50000, @@ -214,9 +213,11 @@ func TestAddStatement(t *testing.T) { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: util.TimeDetail{ - ProcessTime: 1500, - WaitTime: 150, + DetailsNeedP90: execdetails.DetailsNeedP90{ + TimeDetail: util.TimeDetail{ + ProcessTime: 1500, + WaitTime: 150, + }, CalleeAddress: "202", }, }, StmtCtx: &stmtctx.StatementContext{ @@ -313,9 +314,8 @@ func TestAddStatement(t *testing.T) { MaxWaitTime: 250, }, ExecDetail: &execdetails.ExecDetails{ - CalleeAddress: "302", - BackoffTime: 18, - RequestCount: 2, + BackoffTime: 18, + RequestCount: 2, CommitDetail: &util.CommitDetails{ GetCommitTsTime: 50, PrewriteTime: 5000, @@ -352,9 +352,12 @@ func TestAddStatement(t *testing.T) { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: util.TimeDetail{ - ProcessTime: 150, - WaitTime: 15, + DetailsNeedP90: execdetails.DetailsNeedP90{ + TimeDetail: util.TimeDetail{ + ProcessTime: 150, + WaitTime: 15, + }, + CalleeAddress: "302", }, }, StmtCtx: &stmtctx.StatementContext{ @@ -605,9 +608,8 @@ func generateAnyExecInfo() *StmtExecInfo { MaxWaitTime: 1500, }, ExecDetail: &execdetails.ExecDetails{ - CalleeAddress: "129", - BackoffTime: 80, - RequestCount: 10, + BackoffTime: 80, + RequestCount: 10, CommitDetail: &util.CommitDetails{ GetCommitTsTime: 100, PrewriteTime: 10000, @@ -644,9 +646,12 @@ func generateAnyExecInfo() *StmtExecInfo { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: util.TimeDetail{ - ProcessTime: 500, - WaitTime: 50, + DetailsNeedP90: execdetails.DetailsNeedP90{ + TimeDetail: util.TimeDetail{ + ProcessTime: 500, + WaitTime: 50, + }, + CalleeAddress: "129", }, }, StmtCtx: &stmtctx.StatementContext{ diff --git a/util/util.go b/util/util.go index 8af2876240486..249db91f98d06 100644 --- a/util/util.go +++ b/util/util.go @@ -106,6 +106,11 @@ func Str2Int64Map(str string) map[int64]struct{} { // GenLogFields generate log fields. func GenLogFields(costTime time.Duration, info *ProcessInfo, needTruncateSQL bool) []zap.Field { + if info.RefCountOfStmtCtx != nil && !info.RefCountOfStmtCtx.TryIncrease() { + return nil + } + defer info.RefCountOfStmtCtx.Decrease() + logFields := make([]zap.Field, 0, 20) logFields = append(logFields, zap.String("cost_time", strconv.FormatFloat(costTime.Seconds(), 'f', -1, 64)+"s")) execDetail := info.StmtCtx.GetExecDetails() diff --git a/util/util_test.go b/util/util_test.go index 7eb06e1071073..ca68a55cd8ba6 100644 --- a/util/util_test.go +++ b/util/util_test.go @@ -28,6 +28,7 @@ func TestLogFormat(t *testing.T) { mem.Consume(1<<30 + 1<<29 + 1<<28 + 1<<27) mockTooLongQuery := make([]byte, 1024*9) + var refCount stmtctx.ReferenceCount = 0 info := &ProcessInfo{ ID: 233, User: "PingCAP", @@ -38,9 +39,10 @@ func TestLogFormat(t *testing.T) { StatsInfo: func(interface{}) map[string]uint64 { return nil }, - StmtCtx: &stmtctx.StatementContext{}, - MemTracker: mem, - RedactSQL: false, + StmtCtx: &stmtctx.StatementContext{}, + RefCountOfStmtCtx: &refCount, + MemTracker: mem, + RedactSQL: false, } costTime := time.Second * 233 logSQLTruncateLen := 1024 * 8