From 2c0c13053acad7ebbf8405f9be004dffaa5e462c Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 29 Nov 2022 22:12:00 +0800 Subject: [PATCH 01/13] session: workaround for upgrade from the old master version (#39451) close pingcap/tidb#39450 --- session/session.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/session/session.go b/session/session.go index bdaf8abf3f6ef..437c663999b83 100644 --- a/session/session.go +++ b/session/session.go @@ -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 } From 1cebd1dc56b6fabb392874de184648827c5d49fe Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Tue, 29 Nov 2022 22:58:00 +0800 Subject: [PATCH 02/13] *: improve `flashback to timestamp`, let it compatible with TiCDC (#39442) --- DEPS.bzl | 8 ++++---- br/pkg/restore/data.go | 6 +++--- ddl/cluster.go | 29 ++++++++++++++++++----------- ddl/cluster_test.go | 6 +++--- ddl/ddl_api.go | 13 +++++++++++-- go.mod | 4 ++-- go.sum | 8 ++++---- 7 files changed, 45 insertions(+), 29 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 43f76556ecdd9..351129c23b51f 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2915,8 +2915,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:hnUlIU5nCH6PAO9DC5DhODX1cwqoTcXTNIODyvNI9q4=", - version = "v0.0.0-20221123043343-cdc67325f05f", + sum = "h1:LzIZsQpXQlj8yF7+yvyOg680OaPq7bmPuDuszgXfHsw=", + version = "v0.0.0-20221129023506-621ec37aac7a", ) go_repository( name = "com_github_pingcap_log", @@ -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:nfdLYzkmYR2b0XurXSXcVFk06eO//P4VtkqoADX2tR4=", - version = "v2.0.3-0.20221128025602-81939ec8b2bb", + sum = "h1:vlgZedcfExiTzB3BB4nt5CpaghDfm9La/0Ofn7weIUA=", + version = "v2.0.3-0.20221129032117-857772dd0907", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/br/pkg/restore/data.go b/br/pkg/restore/data.go index 7432c3d9af0ce..b4ed1c1144dd8 100644 --- a/br/pkg/restore/data.go +++ b/br/pkg/restore/data.go @@ -57,7 +57,7 @@ func RecoverData(ctx context.Context, resolveTS uint64, allStores []*metapb.Stor return totalRegions, errors.Trace(err) } - if err := recovery.PrepareFlashbackToVersion(ctx); err != nil { + if err := recovery.PrepareFlashbackToVersion(ctx, resolveTS, restoreTS-1); err != nil { return totalRegions, errors.Trace(err) } @@ -304,12 +304,12 @@ func (recovery *Recovery) WaitApply(ctx context.Context) (err error) { } // prepare the region for flashback the data, the purpose is to stop region service, put region in flashback state -func (recovery *Recovery) PrepareFlashbackToVersion(ctx context.Context) (err error) { +func (recovery *Recovery) PrepareFlashbackToVersion(ctx context.Context, resolveTS uint64, startTS uint64) (err error) { var totalRegions atomic.Uint64 totalRegions.Store(0) handler := func(ctx context.Context, r tikvstore.KeyRange) (rangetask.TaskStat, error) { - stats, err := ddl.SendPrepareFlashbackToVersionRPC(ctx, recovery.mgr.GetStorage().(tikv.Storage), r) + stats, err := ddl.SendPrepareFlashbackToVersionRPC(ctx, recovery.mgr.GetStorage().(tikv.Storage), resolveTS, startTS, r) totalRegions.Add(uint64(stats.CompletedRegions)) return stats, err } diff --git a/ddl/cluster.go b/ddl/cluster.go index ebb833156cec2..96a7cd8544abb 100644 --- a/ddl/cluster.go +++ b/ddl/cluster.go @@ -50,11 +50,7 @@ import ( ) var pdScheduleKey = []string{ - "hot-region-schedule-limit", - "leader-schedule-limit", "merge-schedule-limit", - "region-schedule-limit", - "replica-schedule-limit", } const ( @@ -68,6 +64,7 @@ const ( autoAnalyzeOffset readOnlyOffset totalLockedRegionsOffset + startTSOffset commitTSOffset ) @@ -280,6 +277,7 @@ func GetFlashbackKeyRanges(sess sessionctx.Context) ([]kv.KeyRange, error) { func SendPrepareFlashbackToVersionRPC( ctx context.Context, s tikv.Storage, + flashbackTS, startTS uint64, r tikvstore.KeyRange, ) (rangetask.TaskStat, error) { startKey, rangeEndKey := r.StartKey, r.EndKey @@ -314,6 +312,8 @@ func SendPrepareFlashbackToVersionRPC( req := tikvrpc.NewRequest(tikvrpc.CmdPrepareFlashbackToVersion, &kvrpcpb.PrepareFlashbackToVersionRequest{ StartKey: startKey, EndKey: endKey, + StartTs: startTS, + Version: flashbackTS, }) resp, err := s.SendReq(bo, req, loc.Region, flashbackTimeout) @@ -481,11 +481,11 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve return ver, errors.Errorf("Not support flashback cluster in non-TiKV env") } - var flashbackTS, lockedRegions, commitTS uint64 + var flashbackTS, lockedRegions, startTS, commitTS uint64 var pdScheduleValue map[string]interface{} var autoAnalyzeValue, readOnlyValue string var gcEnabledValue bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &startTS, &commitTS); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -533,6 +533,13 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // We should get startTS here to avoid lost startTS when TiDB crashed during send prepare flashback RPC. + startTS, err = d.store.GetOracle().GetTimestamp(d.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + job.Args[startTSOffset] = startTS job.SchemaState = model.StateWriteOnly return ver, nil // Stage 3, get key ranges and get locks. @@ -552,7 +559,7 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve for _, r := range keyRanges { if err = flashbackToVersion(d.ctx, d, func(ctx context.Context, r tikvstore.KeyRange) (rangetask.TaskStat, error) { - stats, err := SendPrepareFlashbackToVersionRPC(ctx, d.store.(tikv.Storage), r) + stats, err := SendPrepareFlashbackToVersionRPC(ctx, d.store.(tikv.Storage), flashbackTS, startTS, r) totalRegions.Add(uint64(stats.CompletedRegions)) return stats, err }, r.StartKey, r.EndKey); err != nil { @@ -587,8 +594,8 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve for _, r := range keyRanges { if err = flashbackToVersion(d.ctx, d, func(ctx context.Context, r tikvstore.KeyRange) (rangetask.TaskStat, error) { - // Use commitTS - 1 as startTS, make sure it less than commitTS. - stats, err := SendFlashbackToVersionRPC(ctx, d.store.(tikv.Storage), flashbackTS, commitTS-1, commitTS, r) + // Use same startTS as prepare phase to simulate 1PC txn. + stats, err := SendFlashbackToVersionRPC(ctx, d.store.(tikv.Storage), flashbackTS, startTS, commitTS, r) completedRegions.Add(uint64(stats.CompletedRegions)) logutil.BgLogger().Info("[ddl] flashback cluster stats", zap.Uint64("complete regions", completedRegions.Load()), @@ -615,12 +622,12 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { return nil } - var flashbackTS, lockedRegions, commitTS uint64 + var flashbackTS, lockedRegions, startTS, commitTS uint64 var pdScheduleValue map[string]interface{} var autoAnalyzeValue, readOnlyValue string var gcEnabled bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &startTS, &commitTS); err != nil { return errors.Trace(err) } sess, err := w.sessPool.get() diff --git a/ddl/cluster_test.go b/ddl/cluster_test.go index 2cbf5ee45336a..4c1ec291f87f2 100644 --- a/ddl/cluster_test.go +++ b/ddl/cluster_test.go @@ -98,7 +98,7 @@ func TestFlashbackCloseAndResetPDSchedule(t *testing.T) { fmt.Sprintf("return(%v)", injectSafeTS))) oldValue := map[string]interface{}{ - "hot-region-schedule-limit": 1, + "merge-schedule-limit": 1, } require.NoError(t, infosync.SetPDScheduleConfig(context.Background(), oldValue)) @@ -112,7 +112,7 @@ func TestFlashbackCloseAndResetPDSchedule(t *testing.T) { if job.SchemaState == model.StateWriteReorganization { closeValue, err := infosync.GetPDScheduleConfig(context.Background()) assert.NoError(t, err) - assert.Equal(t, closeValue["hot-region-schedule-limit"], 0) + assert.Equal(t, closeValue["merge-schedule-limit"], 0) // cancel flashback job job.State = model.JobStateCancelled job.Error = dbterror.ErrCancelledDDLJob @@ -128,7 +128,7 @@ func TestFlashbackCloseAndResetPDSchedule(t *testing.T) { finishValue, err := infosync.GetPDScheduleConfig(context.Background()) require.NoError(t, err) - require.EqualValues(t, finishValue["hot-region-schedule-limit"], 1) + require.EqualValues(t, finishValue["merge-schedule-limit"], 1) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/mockFlashbackTest")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS")) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1816e0d65891d..6e2373d12f52f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2724,6 +2724,14 @@ func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo func (d *ddl) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error { logutil.BgLogger().Info("[ddl] get flashback cluster job", zap.String("flashbackTS", oracle.GetTimeFromTS(flashbackTS).String())) + nowTS, err := ctx.GetStore().GetOracle().GetTimestamp(d.ctx, &oracle.Option{}) + if err != nil { + return errors.Trace(err) + } + gap := time.Until(oracle.GetTimeFromTS(nowTS)).Abs() + if gap > 1*time.Second { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Gap between local time and PD TSO is %s, please check PD/system time", gap)) + } job := &model.Job{ Type: model.ActionFlashbackCluster, BinlogInfo: &model.HistoryInfo{}, @@ -2735,9 +2743,10 @@ func (d *ddl) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error variable.On, /* tidb_enable_auto_analyze */ variable.Off, /* tidb_super_read_only */ 0, /* totalRegions */ - 0 /* newCommitTS */}, + 0, /* startTS */ + 0 /* commitTS */}, } - err := d.DoDDLJob(ctx, job) + err = d.DoDDLJob(ctx, job) err = d.callHookOnChanged(job, err) return errors.Trace(err) } diff --git a/go.mod b/go.mod index f225545bfafd1..502ca90e47ea9 100644 --- a/go.mod +++ b/go.mod @@ -68,7 +68,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20221123043343-cdc67325f05f + github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a 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 @@ -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.20221128025602-81939ec8b2bb + github.com/tikv/client-go/v2 v2.0.3-0.20221129032117-857772dd0907 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 70c92d02d2881..dc6888f53d3b2 100644 --- a/go.sum +++ b/go.sum @@ -779,8 +779,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20221123043343-cdc67325f05f h1:hnUlIU5nCH6PAO9DC5DhODX1cwqoTcXTNIODyvNI9q4= -github.com/pingcap/kvproto v0.0.0-20221123043343-cdc67325f05f/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a h1:LzIZsQpXQlj8yF7+yvyOg680OaPq7bmPuDuszgXfHsw= +github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -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.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/client-go/v2 v2.0.3-0.20221129032117-857772dd0907 h1:vlgZedcfExiTzB3BB4nt5CpaghDfm9La/0Ofn7weIUA= +github.com/tikv/client-go/v2 v2.0.3-0.20221129032117-857772dd0907/go.mod h1:MDT4J9LzgS7Bj1DnEq6Gk/puy6mp8TgUC92zGEVVLLg= 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= From e135608dcf208ba822a0c35cd928d80f2e739c2f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 29 Nov 2022 23:24:00 +0800 Subject: [PATCH 03/13] executor: support Intersection of IndexMerge (#38636) close pingcap/tidb#39334 --- executor/index_merge_reader.go | 335 +++++++++++++++++++++------ executor/mem_reader.go | 53 ++++- sessionctx/variable/session.go | 42 +++- sessionctx/variable/sysvar.go | 7 + sessionctx/variable/tidb_vars.go | 18 +- sessionctx/variable/varsutil_test.go | 1 + 6 files changed, 366 insertions(+), 90 deletions(-) diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index cde353e2a64b2..d7475f71374b8 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -94,8 +94,8 @@ type IndexMergeReaderExecutor struct { workerStarted bool keyRanges [][]kv.KeyRange - resultCh chan *lookupTableTask - resultCurr *lookupTableTask + resultCh chan *indexMergeTableTask + resultCurr *indexMergeTableTask feedbacks []*statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. @@ -118,6 +118,16 @@ type IndexMergeReaderExecutor struct { isCorColInPartialFilters []bool isCorColInTableFilter bool isCorColInPartialAccess []bool + + // Whether it's intersection or union. + isIntersection bool +} + +type indexMergeTableTask struct { + lookupTableTask + + // parTblIdx are only used in indexMergeProcessWorker.fetchLoopIntersection. + parTblIdx int } // Table implements the dataSourceExecutor interface. @@ -150,7 +160,7 @@ func (e *IndexMergeReaderExecutor) Open(ctx context.Context) (err error) { } } e.finished = make(chan struct{}) - e.resultCh = make(chan *lookupTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize)) + e.resultCh = make(chan *indexMergeTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize)) e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) return nil @@ -209,8 +219,8 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang func (e *IndexMergeReaderExecutor) startWorkers(ctx context.Context) error { exitCh := make(chan struct{}) - workCh := make(chan *lookupTableTask, 1) - fetchCh := make(chan *lookupTableTask, len(e.keyRanges)) + workCh := make(chan *indexMergeTableTask, 1) + fetchCh := make(chan *indexMergeTableTask, len(e.keyRanges)) e.startIndexMergeProcessWorker(ctx, workCh, fetchCh) @@ -237,12 +247,12 @@ func (e *IndexMergeReaderExecutor) startWorkers(ctx context.Context) error { return nil } -func (e *IndexMergeReaderExecutor) waitPartialWorkersAndCloseFetchChan(fetchCh chan *lookupTableTask) { +func (e *IndexMergeReaderExecutor) waitPartialWorkersAndCloseFetchChan(fetchCh chan *indexMergeTableTask) { e.idxWorkerWg.Wait() close(fetchCh) } -func (e *IndexMergeReaderExecutor) startIndexMergeProcessWorker(ctx context.Context, workCh chan<- *lookupTableTask, fetch <-chan *lookupTableTask) { +func (e *IndexMergeReaderExecutor) startIndexMergeProcessWorker(ctx context.Context, workCh chan<- *indexMergeTableTask, fetch <-chan *indexMergeTableTask) { idxMergeProcessWorker := &indexMergeProcessWorker{ indexMerge: e, stats: e.stats, @@ -252,15 +262,19 @@ func (e *IndexMergeReaderExecutor) startIndexMergeProcessWorker(ctx context.Cont defer trace.StartRegion(ctx, "IndexMergeProcessWorker").End() util.WithRecovery( func() { - idxMergeProcessWorker.fetchLoop(ctx, fetch, workCh, e.resultCh, e.finished) + if e.isIntersection { + idxMergeProcessWorker.fetchLoopIntersection(ctx, fetch, workCh, e.resultCh, e.finished) + } else { + idxMergeProcessWorker.fetchLoopUnion(ctx, fetch, workCh, e.resultCh, e.finished) + } }, - idxMergeProcessWorker.handleLoopFetcherPanic(ctx, e.resultCh), + idxMergeProcessWorker.handleLoopFetcherPanic(ctx, e.resultCh, "IndexMergeProcessWorker", nil), ) e.processWokerWg.Done() }() } -func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, workID int) error { +func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *indexMergeTableTask, workID int) error { if e.runtimeStats != nil { collExec := true e.dagPBs[workID].CollectExecutionSummaries = &collExec @@ -297,7 +311,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, // We got correlated column, so need to refresh Selection operator. var err error if e.dagPBs[workID].Executors, err = constructDistExec(e.ctx, e.partialPlans[workID]); err != nil { - worker.syncErr(e.resultCh, err) + syncErr(e.resultCh, err) return } } @@ -331,12 +345,12 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, }) kvReq, err := builder.SetKeyRanges(keyRange).Build() if err != nil { - worker.syncErr(e.resultCh, err) + syncErr(e.resultCh, err) return } result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, e.handleCols.GetFieldsTypes(), e.feedbacks[workID], getPhysicalPlanIDs(e.partialPlans[workID]), e.getPartitalPlanID(workID)) if err != nil { - worker.syncErr(e.resultCh, err) + syncErr(e.resultCh, err) return } worker.batchSize = e.maxChunkSize @@ -349,7 +363,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, // fetch all data from this partition ctx1, cancel := context.WithCancel(ctx) - _, fetchErr := worker.fetchHandles(ctx1, result, exitCh, fetchCh, e.resultCh, e.finished, e.handleCols) + _, fetchErr := worker.fetchHandles(ctx1, result, exitCh, fetchCh, e.resultCh, e.finished, e.handleCols, parTblIdx) if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again e.feedbacks[workID].Invalidate() } @@ -370,7 +384,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, return nil } -func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, workID int) error { +func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *indexMergeTableTask, workID int) error { ts := e.partialPlans[workID][0].(*plannercore.PhysicalTableScan) tbls := make([]table.Table, 0, 1) @@ -412,13 +426,13 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, if e.isCorColInPartialFilters[workID] { if e.dagPBs[workID].Executors, err = constructDistExec(e.ctx, e.partialPlans[workID]); err != nil { - worker.syncErr(e.resultCh, err) + syncErr(e.resultCh, err) return } partialTableReader.dagPB = e.dagPBs[workID] } - for _, tbl := range tbls { + for parTblIdx, tbl := range tbls { // check if this executor is closed select { case <-e.finished: @@ -430,7 +444,7 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, partialTableReader.table = tbl if err = partialTableReader.Open(ctx); err != nil { logutil.Logger(ctx).Error("open Select result failed:", zap.Error(err)) - worker.syncErr(e.resultCh, err) + syncErr(e.resultCh, err) break } worker.batchSize = e.maxChunkSize @@ -443,7 +457,7 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, // fetch all handles from this table ctx1, cancel := context.WithCancel(ctx) - _, fetchErr := worker.fetchHandles(ctx1, exitCh, fetchCh, e.resultCh, e.finished, e.handleCols) + _, fetchErr := worker.fetchHandles(ctx1, exitCh, fetchCh, e.resultCh, e.finished, e.handleCols, parTblIdx) if fetchErr != nil { // this error is synced in fetchHandles, so don't sync it again e.feedbacks[workID].Invalidate() } @@ -497,16 +511,8 @@ type partialTableWorker struct { partition table.PhysicalTable // it indicates if this worker is accessing a particular partition table } -func (w *partialTableWorker) syncErr(resultCh chan<- *lookupTableTask, err error) { - doneCh := make(chan error, 1) - doneCh <- err - resultCh <- &lookupTableTask{ - doneCh: doneCh, - } -} - -func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, resultCh chan<- *lookupTableTask, - finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { +func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *indexMergeTableTask, resultCh chan<- *indexMergeTableTask, + finished <-chan struct{}, handleCols plannercore.HandleCols, parTblIdx int) (count int64, err error) { chk := w.sc.GetSessionVars().GetNewChunkWithCapacity(retTypes(w.tableReader), w.maxChunkSize, w.maxChunkSize, w.tableReader.base().AllocPool) var basic *execdetails.BasicRuntimeStats if be := w.tableReader.base(); be != nil && be.runtimeStats != nil { @@ -516,14 +522,14 @@ func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan str start := time.Now() handles, retChunk, err := w.extractTaskHandles(ctx, chk, handleCols) if err != nil { - w.syncErr(resultCh, err) + syncErr(resultCh, err) return count, err } if len(handles) == 0 { return count, nil } count += int64(len(handles)) - task := w.buildTableTask(handles, retChunk) + task := w.buildTableTask(handles, retChunk, parTblIdx) if w.stats != nil { atomic.AddInt64(&w.stats.FetchIdxTime, int64(time.Since(start))) } @@ -569,19 +575,22 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } -func (w *partialTableWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk) *lookupTableTask { - task := &lookupTableTask{ - handles: handles, - idxRows: retChk, +func (w *partialTableWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk, parTblIdx int) *indexMergeTableTask { + task := &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + handles: handles, + idxRows: retChk, - partitionTable: w.partition, + partitionTable: w.partition, + }, + parTblIdx: parTblIdx, } task.doneCh = make(chan error, 1) return task } -func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Context, workCh <-chan *lookupTableTask) { +func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Context, workCh <-chan *indexMergeTableTask) { lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency() e.tblWorkerWg.Add(lookupConcurrencyLimit) for i := 0; i < lookupConcurrencyLimit; i++ { @@ -596,7 +605,7 @@ func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Co ctx1, cancel := context.WithCancel(ctx) go func() { defer trace.StartRegion(ctx, "IndexMergeTableScanWorker").End() - var task *lookupTableTask + var task *indexMergeTableTask util.WithRecovery( func() { task = worker.pickAndExecTask(ctx1) }, worker.handlePickAndExecTaskPanic(ctx1, task), @@ -665,7 +674,7 @@ func (e *IndexMergeReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) e } } -func (e *IndexMergeReaderExecutor) getResultTask() (*lookupTableTask, error) { +func (e *IndexMergeReaderExecutor) getResultTask() (*indexMergeTableTask, error) { if e.resultCurr != nil && e.resultCurr.cursor < len(e.resultCurr.rows) { return e.resultCurr, nil } @@ -685,7 +694,7 @@ func (e *IndexMergeReaderExecutor) getResultTask() (*lookupTableTask, error) { return e.resultCurr, nil } -func (e *IndexMergeReaderExecutor) handleHandlesFetcherPanic(ctx context.Context, resultCh chan<- *lookupTableTask, worker string) func(r interface{}) { +func (e *IndexMergeReaderExecutor) handleHandlesFetcherPanic(ctx context.Context, resultCh chan<- *indexMergeTableTask, worker string) func(r interface{}) { return func(r interface{}) { if r == nil { return @@ -695,8 +704,10 @@ func (e *IndexMergeReaderExecutor) handleHandlesFetcherPanic(ctx context.Context logutil.Logger(ctx).Error(err4Panic.Error()) doneCh := make(chan error, 1) doneCh <- err4Panic - resultCh <- &lookupTableTask{ - doneCh: doneCh, + resultCh <- &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + doneCh: doneCh, + }, } } } @@ -724,8 +735,8 @@ type indexMergeProcessWorker struct { stats *IndexMergeRuntimeStat } -func (w *indexMergeProcessWorker) fetchLoop(ctx context.Context, fetchCh <-chan *lookupTableTask, - workCh chan<- *lookupTableTask, resultCh chan<- *lookupTableTask, finished <-chan struct{}) { +func (w *indexMergeProcessWorker) fetchLoopUnion(ctx context.Context, fetchCh <-chan *indexMergeTableTask, + workCh chan<- *indexMergeTableTask, resultCh chan<- *indexMergeTableTask, finished <-chan struct{}) { defer func() { close(workCh) close(resultCh) @@ -757,11 +768,13 @@ func (w *indexMergeProcessWorker) fetchLoop(ctx context.Context, fetchCh <-chan if len(fhs) == 0 { continue } - task := &lookupTableTask{ - handles: fhs, - doneCh: make(chan error, 1), + task := &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + handles: fhs, + doneCh: make(chan error, 1), - partitionTable: task.partitionTable, + partitionTable: task.partitionTable, + }, } if w.stats != nil { w.stats.IndexMergeProcess += time.Since(start) @@ -777,18 +790,196 @@ func (w *indexMergeProcessWorker) fetchLoop(ctx context.Context, fetchCh <-chan } } -func (w *indexMergeProcessWorker) handleLoopFetcherPanic(ctx context.Context, resultCh chan<- *lookupTableTask) func(r interface{}) { +type intersectionProcessWorker struct { + // key: parTblIdx, val: HandleMap + // Value of MemAwareHandleMap is *int to avoid extra Get(). + handleMapsPerWorker map[int]*kv.MemAwareHandleMap[*int] + workerID int + workerCh chan *indexMergeTableTask + indexMerge *IndexMergeReaderExecutor + memTracker *memory.Tracker + batchSize int + + // When rowDelta == memConsumeBatchSize, Consume(memUsage) + rowDelta int64 + mapUsageDelta int64 +} + +func (w *intersectionProcessWorker) consumeMemDelta() { + w.memTracker.Consume(w.mapUsageDelta + w.rowDelta*int64(unsafe.Sizeof(int(0)))) + w.mapUsageDelta = 0 + w.rowDelta = 0 +} + +func (w *intersectionProcessWorker) doIntersectionPerPartition(ctx context.Context, workCh chan<- *indexMergeTableTask, resultCh chan<- *indexMergeTableTask, finished <-chan struct{}) { + defer w.memTracker.Detach() + + for task := range w.workerCh { + var ok bool + var hMap *kv.MemAwareHandleMap[*int] + if hMap, ok = w.handleMapsPerWorker[task.parTblIdx]; !ok { + hMap = kv.NewMemAwareHandleMap[*int]() + w.handleMapsPerWorker[task.parTblIdx] = hMap + } + var mapDelta int64 + var rowDelta int64 + for _, h := range task.handles { + // Use *int to avoid Get() again. + if cntPtr, ok := hMap.Get(h); ok { + (*cntPtr)++ + } else { + cnt := 1 + mapDelta += hMap.Set(h, &cnt) + int64(h.ExtraMemSize()) + rowDelta += 1 + } + } + + logutil.BgLogger().Debug("intersectionProcessWorker handle tasks", zap.Int("workerID", w.workerID), + zap.Int("task.handles", len(task.handles)), zap.Int64("rowDelta", rowDelta)) + + w.mapUsageDelta += mapDelta + w.rowDelta += rowDelta + if w.rowDelta >= int64(w.batchSize) { + w.consumeMemDelta() + } + failpoint.Inject("testIndexMergeIntersectionWorkerPanic", nil) + } + if w.rowDelta > 0 { + w.consumeMemDelta() + } + + // We assume the result of intersection is small, so no need to track memory. + intersectedMap := make(map[int][]kv.Handle, len(w.handleMapsPerWorker)) + for parTblIdx, hMap := range w.handleMapsPerWorker { + hMap.Range(func(h kv.Handle, val interface{}) bool { + if *(val.(*int)) == len(w.indexMerge.partialPlans) { + // Means all partial paths have this handle. + intersectedMap[parTblIdx] = append(intersectedMap[parTblIdx], h) + } + return true + }) + } + + tasks := make([]*indexMergeTableTask, 0, len(w.handleMapsPerWorker)) + for parTblIdx, intersected := range intersectedMap { + // Split intersected[parTblIdx] to avoid task is too large. + for len(intersected) > 0 { + length := w.batchSize + if length > len(intersected) { + length = len(intersected) + } + task := &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + handles: intersected[:length], + doneCh: make(chan error, 1), + }, + } + intersected = intersected[length:] + if w.indexMerge.partitionTableMode { + task.partitionTable = w.indexMerge.prunedPartitions[parTblIdx] + } + tasks = append(tasks, task) + logutil.BgLogger().Debug("intersectionProcessWorker build tasks", + zap.Int("parTblIdx", parTblIdx), zap.Int("task.handles", len(task.handles))) + } + } + for _, task := range tasks { + select { + case <-ctx.Done(): + return + case <-finished: + return + case workCh <- task: + resultCh <- task + } + } +} + +// For each partition(dynamic mode), a map is used to do intersection. Key of the map is handle, and value is the number of times it occurs. +// If the value of handle equals the number of partial paths, it should be sent to final_table_scan_worker. +// To avoid too many goroutines, each intersectionProcessWorker can handle multiple partitions. +func (w *indexMergeProcessWorker) fetchLoopIntersection(ctx context.Context, fetchCh <-chan *indexMergeTableTask, + workCh chan<- *indexMergeTableTask, resultCh chan<- *indexMergeTableTask, finished <-chan struct{}) { + defer func() { + close(workCh) + close(resultCh) + }() + + if w.stats != nil { + start := time.Now() + defer func() { + w.stats.IndexMergeProcess += time.Since(start) + }() + } + + // One goroutine may handle one or multiple partitions. + // Max number of partition number is 8192, we use ExecutorConcurrency to avoid too many goroutines. + maxWorkerCnt := w.indexMerge.ctx.GetSessionVars().IndexMergeIntersectionConcurrency() + maxChannelSize := atomic.LoadInt32(&LookupTableTaskChannelSize) + batchSize := w.indexMerge.ctx.GetSessionVars().IndexLookupSize + + partCnt := 1 + if w.indexMerge.partitionTableMode { + partCnt = len(w.indexMerge.prunedPartitions) + } + workerCnt := mathutil.Min(partCnt, maxWorkerCnt) + failpoint.Inject("testIndexMergeIntersectionConcurrency", func(val failpoint.Value) { + con := val.(int) + if con != workerCnt { + panic(fmt.Sprintf("unexpected workerCnt, expect %d, got %d", con, workerCnt)) + } + }) + + workers := make([]*intersectionProcessWorker, 0, workerCnt) + wg := util.WaitGroupWrapper{} + errCh := make(chan bool, workerCnt) + for i := 0; i < workerCnt; i++ { + tracker := memory.NewTracker(w.indexMerge.id, -1) + tracker.AttachTo(w.indexMerge.memTracker) + worker := &intersectionProcessWorker{ + workerID: i, + handleMapsPerWorker: make(map[int]*kv.MemAwareHandleMap[*int]), + workerCh: make(chan *indexMergeTableTask, maxChannelSize), + indexMerge: w.indexMerge, + memTracker: tracker, + batchSize: batchSize, + } + wg.RunWithRecover(func() { + defer trace.StartRegion(ctx, "IndexMergeIntersectionProcessWorker").End() + worker.doIntersectionPerPartition(ctx, workCh, resultCh, finished) + }, w.handleLoopFetcherPanic(ctx, resultCh, "IndexMergeIntersectionProcessWorker", errCh)) + workers = append(workers, worker) + } + for task := range fetchCh { + select { + case workers[task.parTblIdx%workerCnt].workerCh <- task: + case <-errCh: + break + } + } + for _, processWorker := range workers { + close(processWorker.workerCh) + } + wg.Wait() +} + +func (w *indexMergeProcessWorker) handleLoopFetcherPanic(ctx context.Context, resultCh chan<- *indexMergeTableTask, worker string, extraCh chan bool) func(r interface{}) { return func(r interface{}) { if r == nil { return } + if extraCh != nil { + extraCh <- true + } - err4Panic := errors.Errorf("panic in IndexMergeReaderExecutor indexMergeTableWorker: %v", r) + err4Panic := errors.Errorf("panic in IndexMergeReaderExecutor %s: %v", worker, r) logutil.Logger(ctx).Error(err4Panic.Error()) doneCh := make(chan error, 1) doneCh <- err4Panic - resultCh <- &lookupTableTask{ - doneCh: doneCh, + resultCh <- &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + doneCh: doneCh, + }, } } } @@ -803,11 +994,13 @@ type partialIndexWorker struct { partition table.PhysicalTable // it indicates if this worker is accessing a particular partition table } -func (w *partialIndexWorker) syncErr(resultCh chan<- *lookupTableTask, err error) { +func syncErr(resultCh chan<- *indexMergeTableTask, err error) { doneCh := make(chan error, 1) doneCh <- err - resultCh <- &lookupTableTask{ - doneCh: doneCh, + resultCh <- &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + doneCh: doneCh, + }, } } @@ -815,10 +1008,11 @@ func (w *partialIndexWorker) fetchHandles( ctx context.Context, result distsql.SelectResult, exitCh <-chan struct{}, - fetchCh chan<- *lookupTableTask, - resultCh chan<- *lookupTableTask, + fetchCh chan<- *indexMergeTableTask, + resultCh chan<- *indexMergeTableTask, finished <-chan struct{}, - handleCols plannercore.HandleCols) (count int64, err error) { + handleCols plannercore.HandleCols, + parTblIdx int) (count int64, err error) { chk := chunk.NewChunkWithCapacity(handleCols.GetFieldsTypes(), w.maxChunkSize) var basicStats *execdetails.BasicRuntimeStats if w.stats != nil { @@ -830,7 +1024,7 @@ func (w *partialIndexWorker) fetchHandles( start := time.Now() handles, retChunk, err := w.extractTaskHandles(ctx, chk, result, handleCols) if err != nil { - w.syncErr(resultCh, err) + syncErr(resultCh, err) return count, err } if len(handles) == 0 { @@ -840,7 +1034,7 @@ func (w *partialIndexWorker) fetchHandles( return count, nil } count += int64(len(handles)) - task := w.buildTableTask(handles, retChunk) + task := w.buildTableTask(handles, retChunk, parTblIdx) if w.stats != nil { atomic.AddInt64(&w.stats.FetchIdxTime, int64(time.Since(start))) } @@ -886,12 +1080,15 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } -func (w *partialIndexWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk) *lookupTableTask { - task := &lookupTableTask{ - handles: handles, - idxRows: retChk, +func (w *partialIndexWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk, parTblIdx int) *indexMergeTableTask { + task := &indexMergeTableTask{ + lookupTableTask: lookupTableTask{ + handles: handles, + idxRows: retChk, - partitionTable: w.partition, + partitionTable: w.partition, + }, + parTblIdx: parTblIdx, } task.doneCh = make(chan error, 1) @@ -900,7 +1097,7 @@ func (w *partialIndexWorker) buildTableTask(handles []kv.Handle, retChk *chunk.C type indexMergeTableScanWorker struct { stats *IndexMergeRuntimeStat - workCh <-chan *lookupTableTask + workCh <-chan *indexMergeTableTask finished <-chan struct{} indexMergeExec *IndexMergeReaderExecutor tblPlans []plannercore.PhysicalPlan @@ -909,7 +1106,7 @@ type indexMergeTableScanWorker struct { memTracker *memory.Tracker } -func (w *indexMergeTableScanWorker) pickAndExecTask(ctx context.Context) (task *lookupTableTask) { +func (w *indexMergeTableScanWorker) pickAndExecTask(ctx context.Context) (task *indexMergeTableTask) { var ok bool for { waitStart := time.Now() @@ -932,7 +1129,7 @@ func (w *indexMergeTableScanWorker) pickAndExecTask(ctx context.Context) (task * } } -func (w *indexMergeTableScanWorker) handlePickAndExecTaskPanic(ctx context.Context, task *lookupTableTask) func(r interface{}) { +func (w *indexMergeTableScanWorker) handlePickAndExecTaskPanic(ctx context.Context, task *indexMergeTableTask) func(r interface{}) { return func(r interface{}) { if r == nil { return @@ -944,7 +1141,7 @@ func (w *indexMergeTableScanWorker) handlePickAndExecTaskPanic(ctx context.Conte } } -func (w *indexMergeTableScanWorker) executeTask(ctx context.Context, task *lookupTableTask) error { +func (w *indexMergeTableScanWorker) executeTask(ctx context.Context, task *indexMergeTableTask) error { tbl := w.indexMergeExec.table if w.indexMergeExec.partitionTableMode { tbl = task.partitionTable diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 3790a5d862539..647d785caeafc 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -595,6 +595,7 @@ type memIndexMergeReader struct { retFieldTypes []*types.FieldType indexMergeReader *IndexMergeReaderExecutor memReaders []memReader + isIntersection bool // partition mode partitionMode bool // if it is accessing a partition table @@ -651,6 +652,7 @@ func buildMemIndexMergeReader(ctx context.Context, us *UnionScanExec, indexMerge retFieldTypes: retTypes(us), indexMergeReader: indexMergeReader, memReaders: memReaders, + isIntersection: indexMergeReader.isIntersection, partitionMode: indexMergeReader.partitionTableMode, partitionTables: indexMergeReader.prunedPartitions, @@ -676,11 +678,20 @@ func (m *memIndexMergeReader) getMemRows(ctx context.Context) ([][]types.Datum, } else { kvRanges = append(kvRanges, m.indexMergeReader.keyRanges) } + if len(kvRanges) != len(tbls) { + return nil, errors.Errorf("length of tbls(size: %d) should be equals to length of kvRanges(size: %d)", len(tbls), len(kvRanges)) + } tblKVRanges := make([]kv.KeyRange, 0, 16) numHandles := 0 + var handles []kv.Handle + var err error for i, tbl := range tbls { - handles, err := m.unionHandles(kvRanges[i]) + if m.isIntersection { + handles, err = m.intersectionHandles(kvRanges[i]) + } else { + handles, err = m.unionHandles(kvRanges[i]) + } if err != nil { return nil, err } @@ -716,7 +727,7 @@ func (m *memIndexMergeReader) getMemRows(ctx context.Context) ([][]types.Datum, return memTblReader.getMemRows(ctx) } -// Union all handles of different Indexes. +// Union all handles of all partial paths. func (m *memIndexMergeReader) unionHandles(kvRanges [][]kv.KeyRange) (finalHandles []kv.Handle, err error) { if len(m.memReaders) != len(kvRanges) { return nil, errors.Errorf("len(kvRanges) should be equal to len(memReaders)") @@ -747,6 +758,44 @@ func (m *memIndexMergeReader) unionHandles(kvRanges [][]kv.KeyRange) (finalHandl return finalHandles, nil } +// Intersect handles of each partial paths. +func (m *memIndexMergeReader) intersectionHandles(kvRanges [][]kv.KeyRange) (finalHandles []kv.Handle, err error) { + if len(m.memReaders) != len(kvRanges) { + return nil, errors.Errorf("len(kvRanges) should be equal to len(memReaders)") + } + + hMap := kv.NewHandleMap() + var handles []kv.Handle + for i, reader := range m.memReaders { + switch r := reader.(type) { + case *memTableReader: + r.kvRanges = kvRanges[i] + case *memIndexReader: + r.kvRanges = kvRanges[i] + default: + return nil, errors.New("memReader have to be memTableReader or memIndexReader") + } + if handles, err = reader.getMemRowsHandle(); err != nil { + return nil, err + } + for _, h := range handles { + if cntPtr, ok := hMap.Get(h); !ok { + cnt := 1 + hMap.Set(h, &cnt) + } else { + *(cntPtr.(*int)) += 1 + } + } + } + hMap.Range(func(h kv.Handle, val interface{}) bool { + if *(val.(*int)) == len(m.memReaders) { + finalHandles = append(finalHandles, h) + } + return true + }) + return finalHandles, nil +} + func (m *memIndexMergeReader) getMemRowsHandle() ([]kv.Handle, error) { return nil, errors.New("getMemRowsHandle has not been implemented for memIndexMergeReader") } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 5115c0a834ad6..69c0d4d48c607 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1676,18 +1676,19 @@ func NewSessionVars(hctx HookContext) *SessionVars { } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ - indexLookupConcurrency: DefIndexLookupConcurrency, - indexSerialScanConcurrency: DefIndexSerialScanConcurrency, - indexLookupJoinConcurrency: DefIndexLookupJoinConcurrency, - hashJoinConcurrency: DefTiDBHashJoinConcurrency, - projectionConcurrency: DefTiDBProjectionConcurrency, - distSQLScanConcurrency: DefDistSQLScanConcurrency, - hashAggPartialConcurrency: DefTiDBHashAggPartialConcurrency, - hashAggFinalConcurrency: DefTiDBHashAggFinalConcurrency, - windowConcurrency: DefTiDBWindowConcurrency, - mergeJoinConcurrency: DefTiDBMergeJoinConcurrency, - streamAggConcurrency: DefTiDBStreamAggConcurrency, - ExecutorConcurrency: DefExecutorConcurrency, + indexLookupConcurrency: DefIndexLookupConcurrency, + indexSerialScanConcurrency: DefIndexSerialScanConcurrency, + indexLookupJoinConcurrency: DefIndexLookupJoinConcurrency, + hashJoinConcurrency: DefTiDBHashJoinConcurrency, + projectionConcurrency: DefTiDBProjectionConcurrency, + distSQLScanConcurrency: DefDistSQLScanConcurrency, + hashAggPartialConcurrency: DefTiDBHashAggPartialConcurrency, + hashAggFinalConcurrency: DefTiDBHashAggFinalConcurrency, + windowConcurrency: DefTiDBWindowConcurrency, + mergeJoinConcurrency: DefTiDBMergeJoinConcurrency, + streamAggConcurrency: DefTiDBStreamAggConcurrency, + indexMergeIntersectionConcurrency: DefTiDBIndexMergeIntersectionConcurrency, + ExecutorConcurrency: DefExecutorConcurrency, } vars.MemQuota = MemQuota{ MemQuotaQuery: DefTiDBMemQuotaQuery, @@ -2402,6 +2403,10 @@ type Concurrency struct { // streamAggConcurrency is deprecated, use ExecutorConcurrency instead. streamAggConcurrency int + // indexMergeIntersectionConcurrency is the number of indexMergeProcessWorker + // Only meaningful for dynamic pruned partition table. + indexMergeIntersectionConcurrency int + // indexSerialScanConcurrency is the number of concurrent index serial scan worker. indexSerialScanConcurrency int @@ -2462,6 +2467,11 @@ func (c *Concurrency) SetStreamAggConcurrency(n int) { c.streamAggConcurrency = n } +// SetIndexMergeIntersectionConcurrency set the number of concurrent intersection process worker. +func (c *Concurrency) SetIndexMergeIntersectionConcurrency(n int) { + c.indexMergeIntersectionConcurrency = n +} + // SetIndexSerialScanConcurrency set the number of concurrent index serial scan worker. func (c *Concurrency) SetIndexSerialScanConcurrency(n int) { c.indexSerialScanConcurrency = n @@ -2544,6 +2554,14 @@ func (c *Concurrency) StreamAggConcurrency() int { return c.ExecutorConcurrency } +// IndexMergeIntersectionConcurrency return the number of concurrent process worker. +func (c *Concurrency) IndexMergeIntersectionConcurrency() int { + if c.indexMergeIntersectionConcurrency != ConcurrencyUnset { + return c.indexMergeIntersectionConcurrency + } + return c.ExecutorConcurrency +} + // IndexSerialScanConcurrency return the number of concurrent index serial scan worker. // This option is not sync with ExecutorConcurrency since it's used by Analyze table. func (c *Concurrency) IndexSerialScanConcurrency() int { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0ba657405dd5e..1a641811ac20f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1592,6 +1592,13 @@ var defaultSysVars = []*SysVar{ appendDeprecationWarning(vars, TiDBStreamAggConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexMergeIntersectionConcurrency, Value: strconv.Itoa(DefTiDBIndexMergeIntersectionConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexMergeIntersectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexMergeIntersectionConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToOnOff(DefTiDBEnableParallelApply), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableParallelApply = TiDBOptOn(val) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 55ec073d85e51..6cad5f23b623d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -425,6 +425,9 @@ const ( // tidb_stream_agg_concurrency is deprecated, use tidb_executor_concurrency instead. TiDBStreamAggConcurrency = "tidb_streamagg_concurrency" + // TiDBIndexMergeIntersectionConcurrency is used for parallel worker of index merge intersection. + TiDBIndexMergeIntersectionConcurrency = "tidb_index_merge_intersection_concurrency" + // TiDBEnableParallelApply is used for parallel apply. TiDBEnableParallelApply = "tidb_enable_parallel_apply" @@ -1091,13 +1094,14 @@ const ( DefTiDBServerMemoryLimitGCTrigger = 0.7 DefTiDBEnableGOGCTuner = true // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. - DefTiDBGOGCTunerThreshold float64 = 0.6 - DefTiDBOptPrefixIndexSingleScan = true - DefTiDBExternalTS = 0 - DefTiDBEnableExternalTSRead = false - DefTiDBEnableReusechunk = true - DefTiDBUseAlloc = false - DefTiDBEnablePlanReplayerCapture = false + DefTiDBGOGCTunerThreshold float64 = 0.6 + DefTiDBOptPrefixIndexSingleScan = true + DefTiDBExternalTS = 0 + DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false + DefTiDBEnablePlanReplayerCapture = false + DefTiDBIndexMergeIntersectionConcurrency = ConcurrencyUnset ) // Process global variables. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 69c9caf294e5e..cdd48a789b9b5 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -74,6 +74,7 @@ func TestNewSessionVars(t *testing.T) { require.Equal(t, DefExecutorConcurrency, vars.HashAggPartialConcurrency()) require.Equal(t, DefExecutorConcurrency, vars.HashAggFinalConcurrency()) require.Equal(t, DefExecutorConcurrency, vars.WindowConcurrency()) + require.Equal(t, DefExecutorConcurrency, vars.IndexMergeIntersectionConcurrency()) require.Equal(t, DefTiDBMergeJoinConcurrency, vars.MergeJoinConcurrency()) require.Equal(t, DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) require.Equal(t, DefDistSQLScanConcurrency, vars.DistSQLScanConcurrency()) From c88fd23f90741657ec20d3ba50493fe329910753 Mon Sep 17 00:00:00 2001 From: fzzf678 <108643977+fzzf678@users.noreply.github.com> Date: Wed, 30 Nov 2022 00:04:00 +0800 Subject: [PATCH 04/13] planner: add sqlDigest and planDigest field when show bindings (#39156) ref pingcap/tidb#39199 --- bindinfo/bind_record.go | 6 +++- bindinfo/bind_test.go | 63 +++++++++++++++++++++++++++++++++ bindinfo/capture_test.go | 10 +++++- bindinfo/handle.go | 23 ++++++++++-- bindinfo/handle_test.go | 19 +++++----- bindinfo/session_handle_test.go | 12 +++---- executor/bind.go | 3 ++ executor/builder.go | 1 + executor/show.go | 2 ++ executor/showtest/show_test.go | 27 ++++++++++++++ infoschema/tables_test.go | 2 +- planner/core/common_plans.go | 2 ++ planner/core/planbuilder.go | 8 +++-- planner/optimize.go | 2 ++ session/bootstrap.go | 17 +++++++-- session/bootstrap_test.go | 9 +++++ 16 files changed, 181 insertions(+), 25 deletions(-) diff --git a/bindinfo/bind_record.go b/bindinfo/bind_record.go index 63517d91ac189..6395bbaa278ba 100644 --- a/bindinfo/bind_record.go +++ b/bindinfo/bind_record.go @@ -54,6 +54,8 @@ const ( Evolve = "evolve" // Builtin indicates the binding is a builtin record for internal locking purpose. It is also the status for the builtin binding. Builtin = "builtin" + // History indicate the binding is created from statement summary by plan digest + History = "history" ) // Binding stores the basic bind hint info. @@ -71,7 +73,9 @@ type Binding struct { // Hint is the parsed hints, it is used to bind hints to stmt node. Hint *hint.HintsSet `json:"-"` // ID is the string form of Hint. It would be non-empty only when the status is `Using` or `PendingVerify`. - ID string `json:"-"` + ID string `json:"-"` + SQLDigest string + PlanDigest string } func (b *Binding) isSame(rb *Binding) bool { diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 07e29923a9790..33f358de0d97d 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -29,6 +29,8 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" + utilparser "github.com/pingcap/tidb/util/parser" + "github.com/pingcap/tidb/util/stmtsummary" "github.com/stretchr/testify/require" ) @@ -1241,3 +1243,64 @@ func TestGCBindRecord(t *testing.T) { tk.MustQuery("show global bindings").Check(testkit.Rows()) tk.MustQuery("select status from mysql.bind_info where original_sql = 'select * from `test` . `t` where `a` = ?'").Check(testkit.Rows()) } + +func TestBindSQLDigest(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(pk int primary key, a int, b int, key(a), key(b))") + + cases := []struct { + origin string + hint string + }{ + // agg hints + {"select count(1) from t", "select /*+ hash_agg() */ count(1) from t"}, + {"select count(1) from t", "select /*+ stream_agg() */ count(1) from t"}, + // join hints + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ tidb_smj(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ tidb_hj(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ inl_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ tidb_inlj(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + {"select * from t t1, t t2 where t1.a=t2.a", "select /*+ inl_hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a"}, + // index hints + {"select * from t", "select * from t use index(primary)"}, + {"select * from t", "select /*+ use_index(primary) */ * from t"}, + {"select * from t", "select * from t use index(a)"}, + {"select * from t", "select /*+ use_index(a) */ * from t use index(a)"}, + {"select * from t", "select * from t use index(b)"}, + {"select * from t", "select /*+ use_index(b) */ * from t use index(b)"}, + {"select a, b from t where a=1 or b=1", "select /*+ use_index_merge(t, a, b) */ a, b from t where a=1 or b=1"}, + {"select * from t where a=1", "select /*+ ignore_index(t, a) */ * from t where a=1"}, + // push-down hints + {"select * from t limit 10", "select /*+ limit_to_cop() */ * from t limit 10"}, + {"select a, count(*) from t group by a", "select /*+ agg_to_cop() */ a, count(*) from t group by a"}, + // index-merge hints + {"select a, b from t where a>1 or b>1", "select /*+ no_index_merge() */ a, b from t where a>1 or b>1"}, + {"select a, b from t where a>1 or b>1", "select /*+ use_index_merge(t, a, b) */ a, b from t where a>1 or b>1"}, + // runtime hints + {"select * from t", "select /*+ memory_quota(1024 MB) */ * from t"}, + {"select * from t", "select /*+ max_execution_time(1000) */ * from t"}, + // storage hints + {"select * from t", "select /*+ read_from_storage(tikv[t]) */ * from t"}, + // others + {"select t1.a, t1.b from t t1 where t1.a in (select t2.a from t t2)", "select /*+ use_toja(true) */ t1.a, t1.b from t t1 where t1.a in (select t2.a from t t2)"}, + } + for _, c := range cases { + stmtsummary.StmtSummaryByDigestMap.Clear() + utilCleanBindingEnv(tk, dom) + sql := "create global binding for " + c.origin + " using " + c.hint + tk.MustExec(sql) + res := tk.MustQuery(`show global bindings`).Rows() + require.Equal(t, len(res[0]), 11) + + parser4binding := parser.New() + originNode, err := parser4binding.ParseOneStmt(c.origin, "utf8mb4", "utf8mb4_general_ci") + require.NoError(t, err) + _, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(originNode, "test", c.origin)) + require.Equal(t, res[0][9], sqlDigestWithDB.String()) + } +} diff --git a/bindinfo/capture_test.go b/bindinfo/capture_test.go index bff6b01045c0b..d1f375a6b63d7 100644 --- a/bindinfo/capture_test.go +++ b/bindinfo/capture_test.go @@ -22,9 +22,11 @@ import ( "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" + utilparser "github.com/pingcap/tidb/util/parser" "github.com/pingcap/tidb/util/stmtsummary" "github.com/stretchr/testify/require" "go.opencensus.io/stats/view" @@ -397,7 +399,7 @@ func TestConcurrentCapture(t *testing.T) { // Simulate an existing binding generated by concurrent CREATE BINDING, which has not been synchronized to current tidb-server yet. // Actually, it is more common to be generated by concurrent baseline capture, I use Manual just for simpler test verification. tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t`', 'select * from `test` . `t`', '', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustQuery("select original_sql, source from mysql.bind_info where source != 'builtin'").Check(testkit.Rows( "select * from `test` . `t` manual", )) @@ -1011,5 +1013,11 @@ func TestCaptureHints(t *testing.T) { res := tk.MustQuery(`show global bindings`).Rows() require.Equal(t, len(res), 1) // this query is captured, and require.True(t, strings.Contains(res[0][1].(string), capCase.hint)) // the binding contains the expected hint + // test sql digest + parser4binding := parser.New() + originNode, err := parser4binding.ParseOneStmt(capCase.query, "utf8mb4", "utf8mb4_general_ci") + require.NoError(t, err) + _, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(originNode, "test", capCase.query)) + require.Equal(t, res[0][9], sqlDigestWithDB.String()) } } diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 1b3858f508a6d..b502f31dbe68d 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -146,7 +146,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) // No need to acquire the session context lock for ExecRestrictedSQL, it // uses another background session. - rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source, sql_digest, plan_digest FROM mysql.bind_info WHERE update_time > %? ORDER BY update_time, create_time`, updateTime) if err != nil { @@ -261,7 +261,7 @@ func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecor record.Bindings[i].UpdateTime = now // Insert the BindRecord to the storage. - _, err = exec.ExecuteInternal(ctx, `INSERT INTO mysql.bind_info VALUES (%?,%?, %?, %?, %?, %?, %?, %?, %?)`, + _, err = exec.ExecuteInternal(ctx, `INSERT INTO mysql.bind_info VALUES (%?,%?, %?, %?, %?, %?, %?, %?, %?, %?, %?)`, record.OriginalSQL, record.Bindings[i].BindSQL, record.Db, @@ -271,6 +271,8 @@ func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecor record.Bindings[i].Charset, record.Bindings[i].Collation, record.Bindings[i].Source, + record.Bindings[i].SQLDigest, + record.Bindings[i].PlanDigest, ) if err != nil { return err @@ -349,8 +351,18 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) } record.Bindings[i].UpdateTime = now + if record.Bindings[i].SQLDigest == "" { + parser4binding := parser.New() + var originNode ast.StmtNode + originNode, err = parser4binding.ParseOneStmt(record.OriginalSQL, record.Bindings[i].Charset, record.Bindings[i].Collation) + if err != nil { + return err + } + _, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(originNode, record.Db, record.OriginalSQL)) + record.Bindings[i].SQLDigest = sqlDigestWithDB.String() + } // Insert the BindRecord to the storage. - _, err = exec.ExecuteInternal(ctx, `INSERT INTO mysql.bind_info VALUES (%?, %?, %?, %?, %?, %?, %?, %?, %?)`, + _, err = exec.ExecuteInternal(ctx, `INSERT INTO mysql.bind_info VALUES (%?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?)`, record.OriginalSQL, record.Bindings[i].BindSQL, record.Db, @@ -360,6 +372,8 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) record.Bindings[i].Charset, record.Bindings[i].Collation, record.Bindings[i].Source, + record.Bindings[i].SQLDigest, + record.Bindings[i].PlanDigest, ) if err != nil { return err @@ -680,6 +694,8 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { Charset: row.GetString(6), Collation: row.GetString(7), Source: row.GetString(8), + SQLDigest: row.GetString(9), + PlanDigest: row.GetString(10), } bindRecord := &BindRecord{ OriginalSQL: row.GetString(0), @@ -900,6 +916,7 @@ func (h *BindHandle) CaptureBaselines() { Charset: charset, Collation: collation, Source: Capture, + SQLDigest: digest.String(), } // We don't need to pass the `sctx` because the BindSQL has been validated already. err = h.CreateBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) diff --git a/bindinfo/handle_test.go b/bindinfo/handle_test.go index 01ff0628feb5c..ffea398781f8b 100644 --- a/bindinfo/handle_test.go +++ b/bindinfo/handle_test.go @@ -107,7 +107,7 @@ func TestBindingLastUpdateTimeWithInvalidBind(t *testing.T) { require.Equal(t, updateTime0, "0000-00-00 00:00:00") tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t`', 'select * from `test` . `t` use index(`idx`)', 'test', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -137,8 +137,9 @@ func TestBindParse(t *testing.T) { charset := "utf8mb4" collation := "utf8mb4_bin" source := bindinfo.Manual - sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation,source) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s', '%s')`, - originSQL, bindSQL, defaultDb, status, charset, collation, source) + mockDigest := "0f644e22c38ecc71d4592c52df127df7f86b6ca7f7c0ee899113b794578f9396" + sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation,source, sql_digest, plan_digest) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s', '%s', '%s', '%s')`, + originSQL, bindSQL, defaultDb, status, charset, collation, source, mockDigest, mockDigest) tk.MustExec(sql) bindHandle := bindinfo.NewBindHandle(tk.Session()) err := bindHandle.Update(true) @@ -221,7 +222,7 @@ func TestEvolveInvalidBindings(t *testing.T) { tk.MustExec("create global binding for select * from t where a > 10 using select /*+ USE_INDEX(t) */ * from t where a > 10") // Manufacture a rejected binding by hacking mysql.bind_info. tk.MustExec("insert into mysql.bind_info values('select * from test . t where a > ?', 'SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10', 'test', 'rejected', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustQuery("select bind_sql, status from mysql.bind_info where source != 'builtin'").Sort().Check(testkit.Rows( "SELECT /*+ USE_INDEX(`t` )*/ * FROM `test`.`t` WHERE `a` > 10 enabled", "SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10 rejected", @@ -242,6 +243,8 @@ func TestEvolveInvalidBindings(t *testing.T) { require.Equal(t, "SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10", rows[1][1]) status = rows[1][3].(string) require.True(t, status == bindinfo.Enabled || status == bindinfo.Rejected) + _, sqlDigestWithDB := parser.NormalizeDigest("select * from test.t where a > 10") // test sqlDigest if exists after add columns to mysql.bind_info + require.Equal(t, rows[0][9], sqlDigestWithDB.String()) } func TestSetBindingStatus(t *testing.T) { @@ -319,9 +322,9 @@ func TestSetBindingStatusWithoutBindingInCache(t *testing.T) { // Simulate creating bindings on other machines tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t` where `a` > ?', 'SELECT /*+ USE_INDEX(`t` `idx_a`)*/ * FROM `test`.`t` WHERE `a` > 10', 'test', 'deleted', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t` where `a` > ?', 'SELECT /*+ USE_INDEX(`t` `idx_a`)*/ * FROM `test`.`t` WHERE `a` > 10', 'test', 'enabled', '2000-01-02 09:00:00', '2000-01-02 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") dom.BindHandle().Clear() tk.MustExec("set binding disabled for select * from t where a > 10") tk.MustExec("admin reload bindings") @@ -334,9 +337,9 @@ func TestSetBindingStatusWithoutBindingInCache(t *testing.T) { // Simulate creating bindings on other machines tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t` where `a` > ?', 'SELECT * FROM `test`.`t` WHERE `a` > 10', 'test', 'deleted', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t` where `a` > ?', 'SELECT * FROM `test`.`t` WHERE `a` > 10', 'test', 'disabled', '2000-01-02 09:00:00', '2000-01-02 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") dom.BindHandle().Clear() tk.MustExec("set binding enabled for select * from t where a > 10") tk.MustExec("admin reload bindings") diff --git a/bindinfo/session_handle_test.go b/bindinfo/session_handle_test.go index a60f8ff41cd12..0b8f929d1d215 100644 --- a/bindinfo/session_handle_test.go +++ b/bindinfo/session_handle_test.go @@ -219,7 +219,7 @@ func TestBaselineDBLowerCase(t *testing.T) { // Simulate existing bindings with upper case default_db. tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t`', 'select * from `spm` . `t`', 'SPM', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustQuery("select original_sql, default_db from mysql.bind_info where original_sql = 'select * from `spm` . `t`'").Check(testkit.Rows( "select * from `spm` . `t` SPM", )) @@ -237,7 +237,7 @@ func TestBaselineDBLowerCase(t *testing.T) { utilCleanBindingEnv(tk, dom) // Simulate existing bindings with upper case default_db. tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t`', 'select * from `spm` . `t`', 'SPM', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustQuery("select original_sql, default_db from mysql.bind_info where original_sql = 'select * from `spm` . `t`'").Check(testkit.Rows( "select * from `spm` . `t` SPM", )) @@ -274,13 +274,13 @@ func TestShowGlobalBindings(t *testing.T) { require.Len(t, rows, 0) // Simulate existing bindings in the mysql.bind_info. tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t`', 'select * from `spm` . `t` USE INDEX (`a`)', 'SPM', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t0`', 'select * from `spm` . `t0` USE INDEX (`a`)', 'SPM', 'enabled', '2000-01-02 09:00:00', '2000-01-02 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t`', 'select /*+ use_index(`t` `a`)*/ * from `spm` . `t`', 'SPM', 'enabled', '2000-01-03 09:00:00', '2000-01-03 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("insert into mysql.bind_info values('select * from `spm` . `t0`', 'select /*+ use_index(`t0` `a`)*/ * from `spm` . `t0`', 'SPM', 'enabled', '2000-01-04 09:00:00', '2000-01-04 09:00:00', '', '','" + - bindinfo.Manual + "')") + bindinfo.Manual + "', '', '')") tk.MustExec("admin reload bindings") rows = tk.MustQuery("show global bindings").Rows() require.Len(t, rows, 4) diff --git a/executor/bind.go b/executor/bind.go index cf337968d4130..59a49bed48cd4 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -38,6 +38,8 @@ type SQLBindExec struct { isGlobal bool bindAst ast.StmtNode newStatus string + sqlDigest string + planDigest string } // Next implements the Executor Next interface. @@ -114,6 +116,7 @@ func (e *SQLBindExec) createSQLBind() error { Collation: e.collation, Status: bindinfo.Enabled, Source: bindinfo.Manual, + SQLDigest: e.sqlDigest, } record := &bindinfo.BindRecord{ OriginalSQL: e.normdOrigSQL, diff --git a/executor/builder.go b/executor/builder.go index a046c6e386c9d..1877bc8aae7d1 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4811,6 +4811,7 @@ func (b *executorBuilder) buildSQLBindExec(v *plannercore.SQLBindPlan) Executor isGlobal: v.IsGlobal, bindAst: v.BindStmt, newStatus: v.NewStatus, + sqlDigest: v.SQLDigest, } return e } diff --git a/executor/show.go b/executor/show.go index 10799b1ffce0d..b386879e55e27 100644 --- a/executor/show.go +++ b/executor/show.go @@ -369,6 +369,8 @@ func (e *ShowExec) fetchShowBind() error { hint.Charset, hint.Collation, hint.Source, + hint.SQLDigest, + hint.PlanDigest, }) } } diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index ae63782a0b3b8..4f7157b190488 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -2010,3 +2010,30 @@ func TestShowTTLOption(t *testing.T) { tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL \"15:20\" HOUR_MINUTE") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL _utf8mb4'15:20' HOUR_MINUTE TTL_ENABLE = 'ON'")) } + +func TestShowBindingDigestField(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(id int, key(id))") + tk.MustExec("create table t2(id int, key(id))") + tk.MustExec("create binding for select * from t1, t2 where t1.id = t2.id using select /*+ merge_join(t1, t2)*/ * from t1, t2 where t1.id = t2.id") + result := tk.MustQuery("show bindings;") + rows := result.Rows()[0] + require.Equal(t, len(rows), 11) + require.Equal(t, rows[9], "ac1ceb4eb5c01f7c03e29b7d0d6ab567e563f4c93164184cde218f20d07fd77c") + tk.MustExec("drop binding for select * from t1, t2 where t1.id = t2.id") + result = tk.MustQuery("show bindings;") + require.Equal(t, len(result.Rows()), 0) + + tk.MustExec("create global binding for select * from t1, t2 where t1.id = t2.id using select /*+ merge_join(t1, t2)*/ * from t1, t2 where t1.id = t2.id") + result = tk.MustQuery("show global bindings;") + rows = result.Rows()[0] + require.Equal(t, len(rows), 11) + require.Equal(t, rows[9], "ac1ceb4eb5c01f7c03e29b7d0d6ab567e563f4c93164184cde218f20d07fd77c") + tk.MustExec("drop global binding for select * from t1, t2 where t1.id = t2.id") + result = tk.MustQuery("show global bindings;") + require.Equal(t, len(result.Rows()), 0) +} diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index dbcb0f8da62b7..bd6168d035873 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1058,7 +1058,7 @@ func TestStmtSummaryInternalQuery(t *testing.T) { "where digest_text like \"select `original_sql` , `bind_sql` , `default_db` , status%\"" tk.MustQuery(sql).Check(testkit.Rows( "select `original_sql` , `bind_sql` , `default_db` , status , `create_time` , `update_time` , charset , " + - "collation , source from `mysql` . `bind_info` where `update_time` > ? order by `update_time` , `create_time`")) + "collation , source , `sql_digest` , `plan_digest` from `mysql` . `bind_info` where `update_time` > ? order by `update_time` , `create_time`")) // Test for issue #21642. tk.MustQuery(`select tidb_version()`) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index a3fbd62cf12e3..b2abca66a5532 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -265,6 +265,8 @@ type SQLBindPlan struct { Charset string Collation string NewStatus string + SQLDigest string + PlanDigest string } // Simple represents a simple statement plan which doesn't need any optimization. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 0ec0e73d37fc4..d96ed21839435 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1042,15 +1042,17 @@ func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (Plan, error return nil, err } + normdOrigSQL, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())) p := &SQLBindPlan{ SQLBindOp: OpSQLBindCreate, - NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())), + NormdOrigSQL: normdOrigSQL, BindSQL: utilparser.RestoreWithDefaultDB(v.HintedNode, b.ctx.GetSessionVars().CurrentDB, v.HintedNode.Text()), IsGlobal: v.GlobalScope, BindStmt: v.HintedNode, Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB), Charset: charSet, Collation: collation, + SQLDigest: sqlDigestWithDB.String(), } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) return p, nil @@ -4966,8 +4968,8 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp names = []string{"Privilege", "Context", "Comment"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar} case ast.ShowBindings: - names = []string{"Original_sql", "Bind_sql", "Default_db", "Status", "Create_time", "Update_time", "Charset", "Collation", "Source"} - ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar} + names = []string{"Original_sql", "Bind_sql", "Default_db", "Status", "Create_time", "Update_time", "Charset", "Collation", "Source", "Sql_digest", "Plan_digest"} + ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar} case ast.ShowBindingCacheStatus: names = []string{"bindings_in_cache", "bindings_in_table", "memory_usage", "memory_quota"} ftypes = []byte{mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar} diff --git a/planner/optimize.go b/planner/optimize.go index def76aba9c9c3..28bea204d449e 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -548,12 +548,14 @@ func handleEvolveTasks(ctx context.Context, sctx sessionctx.Context, br *bindinf return } charset, collation := sctx.GetSessionVars().GetCharsetInfo() + _, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmtNode, br.Db, br.OriginalSQL)) binding := bindinfo.Binding{ BindSQL: bindSQL, Status: bindinfo.PendingVerify, Charset: charset, Collation: collation, Source: bindinfo.Evolve, + SQLDigest: sqlDigestWithDB.String(), } globalHandle := domain.GetDomain(sctx).BindHandle() globalHandle.AddEvolvePlanTask(br.OriginalSQL, br.Db, binding) diff --git a/session/bootstrap.go b/session/bootstrap.go index 450959ee68742..5dbffc42aa5f8 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -265,6 +265,8 @@ const ( charset TEXT NOT NULL, collation TEXT NOT NULL, source VARCHAR(10) NOT NULL DEFAULT 'unknown', + sql_digest varchar(64), + plan_digest varchar(64), INDEX sql_index(original_sql(700),default_db(68)) COMMENT "accelerate the speed when add global binding query", INDEX time_index(update_time) COMMENT "accelerate the speed when querying with last update time" ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;` @@ -681,11 +683,13 @@ const ( version102 = 102 // version103 adds the tables mysql.stats_table_locked version103 = 103 + // version104 add `sql_digest` and `plan_digest` to `bind_info` + version104 = 104 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version103 +var currentBootstrapVersion int64 = version104 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -793,6 +797,7 @@ var ( upgradeToVer101, upgradeToVer102, upgradeToVer103, + upgradeToVer104, } ) @@ -1566,7 +1571,7 @@ func initBindInfoTable(s Session) { } func insertBuiltinBindInfoRow(s Session) { - mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.bind_info VALUES (%?, %?, "mysql", %?, "0000-00-00 00:00:00", "0000-00-00 00:00:00", "", "", %?)`, + mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.bind_info VALUES (%?, %?, "mysql", %?, "0000-00-00 00:00:00", "0000-00-00 00:00:00", "", "", %?, "", "")`, bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.Builtin, bindinfo.Builtin, ) } @@ -2079,6 +2084,14 @@ func upgradeToVer103(s Session, ver int64) { doReentrantDDL(s, CreateStatsTableLocked) } +func upgradeToVer104(s Session, ver int64) { + if ver >= version104 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.bind_info ADD COLUMN IF NOT EXISTS `sql_digest` varchar(64)") + doReentrantDDL(s, "ALTER TABLE mysql.bind_info ADD COLUMN IF NOT EXISTS `plan_digest` varchar(64)") +} + func writeOOMAction(s Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index b9b0cc767500b..b17ced93e081c 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -521,6 +521,9 @@ func TestUpdateBindInfo(t *testing.T) { defer func() { require.NoError(t, store.Close()) }() defer dom.Close() se := createSessionAndSetID(t, store) + + mustExec(t, se, "alter table mysql.bind_info drop column if exists plan_digest") + mustExec(t, se, "alter table mysql.bind_info drop column if exists sql_digest") for _, bindCase := range bindCases { sql := fmt.Sprintf("insert into mysql.bind_info values('%s', '%s', '%s', 'enabled', '2021-01-04 14:50:58.257', '2021-01-04 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')", bindCase.originText, @@ -559,6 +562,9 @@ func TestUpdateDuplicateBindInfo(t *testing.T) { defer func() { require.NoError(t, store.Close()) }() defer dom.Close() se := createSessionAndSetID(t, store) + mustExec(t, se, "alter table mysql.bind_info drop column if exists plan_digest") + mustExec(t, se, "alter table mysql.bind_info drop column if exists sql_digest") + mustExec(t, se, `insert into mysql.bind_info values('select * from t', 'select /*+ use_index(t, idx_a)*/ * from t', 'test', 'enabled', '2021-01-04 14:50:58.257', '2021-01-04 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) // The latest one. mustExec(t, se, `insert into mysql.bind_info values('select * from test . t', 'select /*+ use_index(t, idx_b)*/ * from test.t', 'test', 'enabled', '2021-01-04 14:50:58.257', '2021-01-09 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) @@ -1016,6 +1022,9 @@ func TestUpgradeToVer85(t *testing.T) { defer func() { require.NoError(t, store.Close()) }() defer dom.Close() se := createSessionAndSetID(t, store) + mustExec(t, se, "alter table mysql.bind_info drop column if exists plan_digest") + mustExec(t, se, "alter table mysql.bind_info drop column if exists sql_digest") + mustExec(t, se, `insert into mysql.bind_info values('select * from t', 'select /*+ use_index(t, idx_a)*/ * from t', 'test', 'using', '2021-01-04 14:50:58.257', '2021-01-04 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) mustExec(t, se, `insert into mysql.bind_info values('select * from t1', 'select /*+ use_index(t1, idx_a)*/ * from t1', 'test', 'enabled', '2021-01-05 14:50:58.257', '2021-01-05 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) mustExec(t, se, `insert into mysql.bind_info values('select * from t2', 'select /*+ use_index(t2, idx_a)*/ * from t2', 'test', 'disabled', '2021-01-06 14:50:58.257', '2021-01-06 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) From 4476173923d8ec063f6adf4004dca9d7a6f72571 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 30 Nov 2022 00:29:59 +0800 Subject: [PATCH 05/13] planner: update cost model2 (#39438) ref pingcap/tidb#35240 --- executor/explain_test.go | 2 +- executor/explainfor_test.go | 24 +- executor/join_test.go | 2 +- executor/partition_table_test.go | 10 +- executor/testdata/executor_suite_out.json | 4 +- planner/core/integration_test.go | 16 +- planner/core/physical_plan_trace_test.go | 2 +- planner/core/plan_cost_ver2.go | 18 +- planner/core/point_get_plan_test.go | 2 +- planner/core/testdata/analyze_suite_out.json | 6 +- .../core/testdata/binary_plan_suite_out.json | 8 +- .../core/testdata/enforce_mpp_suite_out.json | 149 ++- .../expression_rewriter_suite_out.json | 36 +- .../core/testdata/integration_suite_out.json | 947 +++++++++--------- .../core/testdata/join_reorder_suite_out.json | 86 +- .../core/testdata/partition_pruner_out.json | 28 +- planner/core/testdata/plan_suite_out.json | 40 +- .../testdata/clustered_index_suite_out.json | 12 +- util/ranger/testdata/ranger_suite_out.json | 2 +- 19 files changed, 668 insertions(+), 726 deletions(-) diff --git a/executor/explain_test.go b/executor/explain_test.go index c67c61a67a909..8c1ec9bad03a6 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -364,7 +364,7 @@ func TestCheckActRowsWithUnistore(t *testing.T) { }, { sql: "select count(*) from t_unistore_act_rows group by b", - expected: []string{"2", "2", "2", "4"}, + expected: []string{"2", "4", "4"}, }, { sql: "with cte(a) as (select a from t_unistore_act_rows) select (select 1 from cte limit 1) from cte;", diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 4cf45d998dac1..85652a5f04ddb 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -462,12 +462,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 1)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, + ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 1)`, + ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[1,1], keep order:true, stats:pseudo`, + ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, + ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustExec("set @a=2") tk.MustQuery("execute stmt using @a").Check(testkit.Rows( @@ -478,12 +478,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 2)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, + ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 2)`, + ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[2,2], keep order:true, stats:pseudo`, + ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, + ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustQuery("execute stmt using @a").Check(testkit.Rows( "2 4 2 2", )) diff --git a/executor/join_test.go b/executor/join_test.go index 5daa60d9a595f..a5d5f6efc9fb5 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -1479,7 +1479,7 @@ func TestIndexNestedLoopHashJoin(t *testing.T) { " └─TableRowIDScan 27.00 cop[tikv] table:l2 keep order:false")) tk.MustQuery("select * from t l1 where exists ( select * from t l2 where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey <> l1.l_suppkey )order by `l_orderkey`,`l_linenumber`;").Check(testkit.Rows("0 0 0 0", "0 1 0 1", "0 2 0 0", "1 0 1 0", "1 1 1 1", "1 2 1 0", "2 0 0 0", "2 1 0 1", "2 2 0 0")) tk.MustQuery("desc format = 'brief' select count(*) from t l1 where exists ( select * from t l2 where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey <> l1.l_suppkey );").Check(testkit.Rows( - "HashAgg 1.00 root funcs:count(1)->Column#11", + "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─IndexHashJoin 7.20 root semi join, inner:IndexLookUp, outer key:test.t.l_orderkey, inner key:test.t.l_orderkey, equal cond:eq(test.t.l_orderkey, test.t.l_orderkey), other cond:ne(test.t.l_suppkey, test.t.l_suppkey)", " ├─TableReader(Build) 9.00 root data:Selection", " │ └─Selection 9.00 cop[tikv] not(isnull(test.t.l_suppkey))", diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b2ba37634a8a4..f9ca811e87626 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3670,11 +3670,11 @@ func TestPartitionTableExplain(t *testing.T) { " └─IndexFullScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) keep order:false")) tk.MustQuery(`explain format = 'brief' select * from t,t2 where t2.a = 1 and t2.b = t.b and t.a = 1`).Check(testkit.Rows( "HashJoin 1.00 root inner join, equal:[eq(testpartitiontableexplain.t.b, testpartitiontableexplain.t2.b)]", - "├─TableReader(Build) 1.00 root data:Selection", - "│ └─Selection 1.00 cop[tikv] eq(testpartitiontableexplain.t2.a, 1), not(isnull(testpartitiontableexplain.t2.b))", - "│ └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false", - "└─Selection(Probe) 1.00 root not(isnull(testpartitiontableexplain.t.b))", - " └─Point_Get 1.00 root table:t, partition:p1 handle:1")) + `├─Selection(Build) 1.00 root not(isnull(testpartitiontableexplain.t.b))`, + `│ └─Point_Get 1.00 root table:t, partition:p1 handle:1`, + `└─TableReader(Probe) 1.00 root data:Selection`, + ` └─Selection 1.00 cop[tikv] eq(testpartitiontableexplain.t2.a, 1), not(isnull(testpartitiontableexplain.t2.b))`, + ` └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false`)) tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec(`analyze table t`) diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index c06ee935e8b37..e0fbfecb07095 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -674,7 +674,7 @@ { "SQL": "select count(*) from t as t1 inner join t as t2 on t1.c1 = t2.c1 where t1.c1 != NULL", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#7", + "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─MergeJoin 0.00 root inner join, left key:test.t.c1, right key:test.t.c1", " ├─TableDual(Build) 0.00 root rows:0", " └─TableDual(Probe) 0.00 root rows:0" @@ -704,7 +704,7 @@ { "SQL": "select count(*) from t as t1 left join t as t2 on t1.c1 = t2.c1 where t1.c1 != NULL", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#7", + "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─MergeJoin 0.00 root left outer join, left key:test.t.c1, right key:test.t.c1", " ├─TableDual(Build) 0.00 root rows:0", " └─TableDual(Probe) 0.00 root rows:0" diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index e8d6d243a607e..b50cc21397fa1 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1231,9 +1231,9 @@ func TestAggPushDownEngine(t *testing.T) { tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( - "HashAgg 1.00 root funcs:approx_count_distinct(Column#4)->Column#3", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#4", + "StreamAgg 1.00 root funcs:approx_count_distinct(Column#5)->Column#3", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#5", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") @@ -6697,7 +6697,7 @@ func TestAggPushToCopForCachedTable(t *testing.T) { tk.MustExec("alter table t32157 cache") tk.MustQuery("explain format = 'brief' select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows( - "HashAgg 1.00 root funcs:count(1)->Column#8]\n" + + "StreamAgg 1.00 root funcs:count(1)->Column#8]\n" + "[└─UnionScan 10.00 root eq(test.t32157.process_code, \"GDEP0071\")]\n" + "[ └─TableReader 10.00 root data:Selection]\n" + "[ └─Selection 10.00 cop[tikv] eq(test.t32157.process_code, \"GDEP0071\")]\n" + @@ -7965,10 +7965,10 @@ func TestNullConditionForPrefixIndex(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( - "HashAgg_12 1.00 root funcs:count(Column#6)->Column#5", - "└─IndexReader_13 1.00 root index:HashAgg_6", - " └─HashAgg_6 1.00 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_11 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) + "StreamAgg_18 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader_19 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_17 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) } func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) { diff --git a/planner/core/physical_plan_trace_test.go b/planner/core/physical_plan_trace_test.go index 1e2ad927e0c54..5390fc20af0bd 100644 --- a/planner/core/physical_plan_trace_test.go +++ b/planner/core/physical_plan_trace_test.go @@ -61,7 +61,7 @@ func TestPhysicalOptimizeWithTraceEnabled(t *testing.T) { "Limit_20", "IndexReader_21", "Limit_14", - "HashAgg_9", + "StreamAgg_10", "Projection_8", }, }, diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 1eaa67a6fc06b..b53d53c845c45 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -256,8 +256,7 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, } tableSideCost := divCostVer2(sumCostVer2(tableNetCost, tableChildCost), distConcurrency) - // double-read: assume at least 1 row to double-read to avoid 0 double-read cost. - doubleReadRows := math.Max(indexRows, 1) + doubleReadRows := indexRows doubleReadCPUCost := newCostVer2(option, cpuFactor, indexRows*cpuFactor.Value, func() string { return fmt.Sprintf("double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) }) @@ -448,13 +447,16 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl groupCost := groupCostVer2(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) + startCost := newCostVer2(option, cpuFactor, + 10*3*cpuFactor.Value, // 10rows * 3func * cpuFactor + func() string { return fmt.Sprintf("cpu(10*3*%v)", cpuFactor) }) childCost, err := p.children[0].getPlanCostVer2(taskType, option) if err != nil { return zeroCostVer2, err } - p.planCostVer2 = sumCostVer2(childCost, divCostVer2(sumCostVer2(aggCost, groupCost, hashBuildCost, hashProbeCost), concurrency)) + p.planCostVer2 = sumCostVer2(startCost, childCost, divCostVer2(sumCostVer2(aggCost, groupCost, hashBuildCost, hashProbeCost), concurrency)) p.planCostInit = true return p.planCostVer2, nil } @@ -532,7 +534,10 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P p.planCostVer2 = sumCostVer2(buildChildCost, probeChildCost, divCostVer2(sumCostVer2(buildHashCost, buildFilterCost, probeHashCost, probeFilterCost), mppConcurrency)) } else { // TiDB HashJoin - p.planCostVer2 = sumCostVer2(buildChildCost, probeChildCost, buildHashCost, buildFilterCost, + startCost := newCostVer2(option, cpuFactor, + 10*3*cpuFactor.Value, // 10rows * 3func * cpuFactor + func() string { return fmt.Sprintf("cpu(10*3*%v)", cpuFactor) }) + p.planCostVer2 = sumCostVer2(startCost, buildChildCost, probeChildCost, buildHashCost, buildFilterCost, divCostVer2(sumCostVer2(probeFilterCost, probeHashCost), tidbConcurrency)) } p.planCostInit = true @@ -563,6 +568,9 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt buildTaskCost := newCostVer2(option, cpuFactor, buildRows*10*cpuFactor.Value, func() string { return fmt.Sprintf("cpu(%v*10*%v)", buildRows, cpuFactor) }) + startCost := newCostVer2(option, cpuFactor, + 10*3*cpuFactor.Value, + func() string { return fmt.Sprintf("cpu(10*3*%v)", cpuFactor) }) probeFilterCost := filterCostVer2(option, probeRowsTot, probeFilters, cpuFactor) probeChildCost, err := probe.getPlanCostVer2(taskType, option) @@ -587,7 +595,7 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt batchRatio := 6.0 probeCost := divCostVer2(mulCostVer2(probeChildCost, buildRows), batchRatio) - p.planCostVer2 = sumCostVer2(buildChildCost, buildFilterCost, buildTaskCost, divCostVer2(sumCostVer2(probeCost, probeFilterCost, hashTableCost), probeConcurrency)) + p.planCostVer2 = sumCostVer2(startCost, buildChildCost, buildFilterCost, buildTaskCost, divCostVer2(sumCostVer2(probeCost, probeFilterCost, hashTableCost), probeConcurrency)) p.planCostInit = true return p.planCostVer2, nil } diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index 7801631325f37..812151afc26ec 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -189,7 +189,7 @@ func TestGetExtraColumn(t *testing.T) { tk.MustQuery(`explain format='brief' select t.*, _tidb_rowid from t where a = 1`).Check(testkit.Rows(`Point_Get 1.00 root table:t, index:idx(a) `)) tk.MustExec(`commit`) tk.MustQuery(`explain format='brief' select count(_tidb_rowid) from t where a=1`).Check(testkit.Rows( - `HashAgg 1.00 root funcs:count(test.t._tidb_rowid)->Column#4`, + `StreamAgg 1.00 root funcs:count(test.t._tidb_rowid)->Column#4`, `└─Point_Get 1.00 root table:t, index:idx(a) `)) tk.MustQuery(`explain format='brief' select *, date_format(b, "") from t where a =1 for update`).Check(testkit.Rows( `Projection 1.00 root test.t.a, test.t.b, date_format(cast(test.t.b, datetime BINARY), )->Column#4`, diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index 4d902fec467a5..ea1e33b3ca0b8 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -383,7 +383,7 @@ "└─Apply 10.00 root CARTESIAN left outer semi join, other cond:eq(test.t.c, Column#21)", " ├─IndexReader(Build) 10.00 root index:IndexFullScan", " │ └─IndexFullScan 10.00 cop[tikv] table:t, index:idx(c, b, a) keep order:false", - " └─HashAgg(Probe) 10.00 root funcs:count(1)->Column#21", + " └─StreamAgg(Probe) 10.00 root funcs:count(1)->Column#21", " └─HashJoin 10.00 root inner join, equal:[eq(test.t.a, test.t.a)]", " ├─IndexReader(Build) 10.00 root index:Selection", " │ └─Selection 10.00 cop[tikv] eq(test.t.a, test.t.a), not(isnull(test.t.a))", @@ -411,7 +411,7 @@ { "SQL": "explain format = 'brief' select max(e) from t where a='T3382' and b='ECO' and c='TOPIC' and d='23660fa1ace9455cb7f3ee831e14a342'", "Plan": [ - "HashAgg 1.00 root funcs:max(test.t.e)->Column#7", + "StreamAgg 1.00 root funcs:max(test.t.e)->Column#7", "└─TopN 0.00 root test.t.e:desc, offset:0, count:1", " └─IndexLookUp 0.00 root ", " ├─IndexRangeScan(Build) 0.01 cop[tikv] table:t, index:idx1(d, a) range:[\"23660fa1ace9455cb7f3ee831e14a342\" \"T3382\",\"23660fa1ace9455cb7f3ee831e14a342\" \"T3382\"], keep order:false", @@ -435,7 +435,7 @@ "Cases": [ "IndexReader(Index(t.e)[[NULL,+inf]]->StreamAgg)->StreamAgg", "IndexReader(Index(t.e)[[-inf,10]]->StreamAgg)->StreamAgg", - "IndexReader(Index(t.e)[[-inf,50]]->HashAgg)->HashAgg", + "IndexReader(Index(t.e)[[-inf,50]]->StreamAgg)->StreamAgg", "IndexReader(Index(t.b_c)[[NULL,+inf]]->Sel([gt(test.t.c, 1)])->StreamAgg)->StreamAgg", "IndexLookUp(Index(t.e)[[1,1]], Table(t))->HashAgg", "TableReader(Table(t)->Sel([gt(test.t.e, 1)])->HashAgg)->HashAgg", diff --git a/planner/core/testdata/binary_plan_suite_out.json b/planner/core/testdata/binary_plan_suite_out.json index 203f11acf364f..a361c5ca0b23f 100644 --- a/planner/core/testdata/binary_plan_suite_out.json +++ b/planner/core/testdata/binary_plan_suite_out.json @@ -123,7 +123,7 @@ "operator_info": "data:TableFullScan_16" } ], - "cost": 975351.9825195674, + "cost": 976848.9825195674, "est_rows": 100000000, "act_rows": 8, "task_type": 1, @@ -131,7 +131,7 @@ "operator_info": "CARTESIAN inner join" } ], - "cost": 998975351.9825196, + "cost": 998976848.9825196, "est_rows": 100000000, "act_rows": 8, "task_type": 1, @@ -139,7 +139,7 @@ "operator_info": "cast(test.t.a, decimal(10,0) BINARY)->Column#8" } ], - "cost": 1996975363.5625196, + "cost": 1996978357.5625196, "est_rows": 1, "act_rows": 1, "task_type": 1, @@ -283,7 +283,7 @@ "operator_info": "data:Selection_33" } ], - "cost": 1798740.0717058014, + "cost": 1800237.0717058014, "est_rows": 12487.5, "task_type": 1, "store_type": 1, diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index e2034d2a536b4..12cde0ede133e 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -31,30 +31,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "HashAgg_14 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_15 1.00 124.64 root index:HashAgg_6", - " └─HashAgg_6 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_13 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_24 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_25 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_23 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_12 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_13 1.00 124.64 root index:HashAgg_6", - " └─HashAgg_6 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_11 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_17 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_18 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_16 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_8 1.00 63718.81 root funcs:count(1)->Column#4", - "└─TableReader_17 10.00 63508.91 root data:Selection_16", - " └─Selection_16 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_15 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_10 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_24 10.00 63508.91 root data:Selection_23", + " └─Selection_23 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -71,32 +71,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "HashAgg_16 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_17 1.00 124.64 root index:HashAgg_7", - " └─HashAgg_7 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_15 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_14 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_15 1.00 124.64 root index:HashAgg_7", - " └─HashAgg_7 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_13 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_22 1.00 63498.78 root funcs:count(Column#6)->Column#4", - "└─TableReader_24 1.00 63468.52 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 952010.16 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 952010.16 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -108,32 +106,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "HashAgg_16 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_17 1.00 124.64 root index:HashAgg_7", - " └─HashAgg_7 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_15 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_14 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_15 1.00 124.64 root index:HashAgg_7", - " └─HashAgg_7 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_13 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_22 1.00 63498.78 root funcs:count(Column#6)->Column#4", - "└─TableReader_24 1.00 63468.52 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 952010.16 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 952010.16 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -145,22 +141,20 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "HashAgg_25 1.00 30.26 root funcs:count(Column#6)->Column#4", - "└─TableReader_27 1.00 0.00 root data:ExchangeSender_26", - " └─ExchangeSender_26 1.00 952010.16 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 952010.16 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_24 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_23 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_14 1.00 154.90 root funcs:count(Column#5)->Column#4", - "└─IndexReader_15 1.00 124.64 root index:HashAgg_7", - " └─HashAgg_7 1.00 1837.90 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_13 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." @@ -169,12 +163,11 @@ { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "HashAgg_22 1.00 30.26 root funcs:count(Column#6)->Column#4", - "└─TableReader_24 1.00 0.00 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 952010.16 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 952010.16 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_27 1.00 49.90 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 0.00 root data:StreamAgg_11", + " └─StreamAgg_11 1.00 1427024.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─Selection_26 10.00 1427000.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_25 10000.00 928000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null } @@ -191,9 +184,9 @@ { "SQL": "explain format = 'brief' select count(*) from t where a=1 -- 1. no replica", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#6", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ @@ -208,10 +201,10 @@ { "SQL": "explain select count(*) from t where a=1 -- 2. replica not ready", "Plan": [ - "HashAgg_12 1.00 root funcs:count(Column#7)->Column#6", - "└─IndexReader_13 1.00 root index:HashAgg_6", - " └─HashAgg_6 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_11 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "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", + " └─IndexRangeScan_16 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because tiflash replicas of table `t` not ready." @@ -230,10 +223,10 @@ { "SQL": "explain select count(*) from t where a=1 -- 3. isolation_engine not match", "Plan": [ - "HashAgg_12 1.00 root funcs:count(Column#7)->Column#6", - "└─IndexReader_13 1.00 root index:HashAgg_6", - " └─HashAgg_6 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_11 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "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", + " └─IndexRangeScan_16 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because 'tidb_isolation_read_engines'(value: 'tikv') not match, need 'tiflash'." @@ -247,9 +240,9 @@ { "SQL": "explain format = 'brief' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#6", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ @@ -287,7 +280,7 @@ { "SQL": "EXPLAIN format = 'brief' SELECT count(b) from t where a=1; -- 7. agg func has virtual column", "Plan": [ - "HashAgg 1.00 root funcs:count(test.t.b)->Column#6", + "StreamAgg 1.00 root funcs:count(test.t.b)->Column#6", "└─TableReader 10.00 root data:Selection", " └─Selection 10.00 cop[tiflash] eq(test.t.a, 1)", " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" @@ -334,7 +327,7 @@ { "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where c=1; -- 11. type not supported", "Plan": [ - "HashAgg 1.00 root funcs:count(test.t.a)->Column#6", + "StreamAgg 1.00 root funcs:count(test.t.a)->Column#6", "└─TableReader 10.00 root data:Selection", " └─Selection 10.00 cop[tikv] eq(test.t.c, 1)", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -350,7 +343,7 @@ { "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where d=1; -- 11.1. type not supported", "Plan": [ - "HashAgg 1.00 root funcs:count(test.t.a)->Column#6", + "StreamAgg 1.00 root funcs:count(test.t.a)->Column#6", "└─TableReader 10.00 root data:Selection", " └─Selection 10.00 cop[tikv] eq(test.t.d, 1)", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -381,11 +374,11 @@ { "SQL": "EXPLAIN SELECT count(*) from t where a=1; -- 1. static partition prune", "Plan": [ - "HashAgg_19 1.00 root funcs:count(Column#5)->Column#4", - "└─TableReader_20 1.00 root data:HashAgg_9", - " └─HashAgg_9 1.00 batchCop[tiflash] funcs:count(1)->Column#5", - " └─Selection_18 10.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_17 10000.00 batchCop[tiflash] table:t, partition:p0 keep order:false, stats:pseudo" + "StreamAgg_32 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader_33 1.00 root data:StreamAgg_13", + " └─StreamAgg_13 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─Selection_31 10.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_30 10000.00 batchCop[tiflash] table:t, partition:p0 keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because table `t`is a partition table which is not supported when `@@tidb_partition_prune_mode=static`." diff --git a/planner/core/testdata/expression_rewriter_suite_out.json b/planner/core/testdata/expression_rewriter_suite_out.json index b7f5a31f7a46a..72b86884a1412 100644 --- a/planner/core/testdata/expression_rewriter_suite_out.json +++ b/planner/core/testdata/expression_rewriter_suite_out.json @@ -7,10 +7,9 @@ "Plan": [ "HashJoin 2.25 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t1.b, test.t2.b)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", "└─TableReader(Probe) 2.25 root data:Selection", " └─Selection 2.25 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" @@ -37,15 +36,13 @@ "Plan": [ "HashJoin 1.69 root inner join, equal:[eq(test.t2.a, test.t1.a) eq(test.t2.b, Column#7)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", - "└─HashAgg(Probe) 2.25 root group by:test.t1.a, funcs:count(Column#14)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", - " └─TableReader 2.25 root data:HashAgg", - " └─HashAgg 2.25 cop[tikv] group by:test.t1.a, funcs:count(1)->Column#14", - " └─Selection 3.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "└─HashAgg(Probe) 2.25 root group by:test.t1.a, funcs:count(1)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", + " └─TableReader 3.00 root data:Selection", + " └─Selection 3.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1" @@ -57,10 +54,9 @@ "HashJoin 2.40 root CARTESIAN anti semi join, other cond:eq(Column#7, test.t2.b), eq(test.t1.a, test.t2.a)", "├─TableReader(Build) 4.00 root data:TableFullScan", "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", - "└─HashAgg(Probe) 3.00 root group by:test.t1.a, funcs:count(Column#12)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", - " └─TableReader 3.00 root data:HashAgg", - " └─HashAgg 3.00 cop[tikv] group by:test.t1.a, funcs:count(1)->Column#12", - " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" + "└─HashAgg(Probe) 3.00 root group by:test.t1.a, funcs:count(1)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", + " └─TableReader 4.00 root data:TableFullScan", + " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "4" @@ -73,9 +69,9 @@ "Cases": [ { "Plan": [ - "HashAgg 1.00 root funcs:bit_or(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:bit_or(cast(mul(cast(test.t.a, double BINARY), cast(test.t.b, double BINARY)), bigint(20) BINARY))->Column#5", + "StreamAgg 1.00 root funcs:bit_or(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:bit_or(cast(mul(cast(test.t.a, double BINARY), cast(test.t.b, double BINARY)), bigint(20) BINARY))->Column#6", " └─TableFullScan 1.00 cop[tikv] table:t keep order:false" ] } diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index f466f20e7e1b7..607f6d2947787 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -175,12 +175,12 @@ "Plan": [ "HashAgg 2.00 root group by:Column#5, funcs:firstrow(Column#5)->Column#5", "└─Union 2.00 root ", - " ├─HashAgg 1.00 root funcs:max(test.t.a)->Column#2", + " ├─StreamAgg 1.00 root funcs:max(test.t.a)->Column#2", " │ └─Limit 1.00 root offset:0, count:1", " │ └─TableReader 1.00 root data:Limit", " │ └─Limit 1.00 cop[tikv] offset:0, count:1", " │ └─TableFullScan 1.00 cop[tikv] table:t keep order:true, desc, stats:pseudo", - " └─HashAgg 1.00 root funcs:min(test.t.a)->Column#4", + " └─StreamAgg 1.00 root funcs:min(test.t.a)->Column#4", " └─Limit 1.00 root offset:0, count:1", " └─TableReader 1.00 root data:Limit", " └─Limit 1.00 cop[tikv] offset:0, count:1", @@ -191,12 +191,12 @@ "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t", "Plan": [ "HashJoin 1.00 root CARTESIAN inner join", - "├─HashAgg(Build) 1.00 root funcs:max(test.cluster_index_t.a)->Column#5", + "├─StreamAgg(Build) 1.00 root funcs:max(test.cluster_index_t.a)->Column#5", "│ └─Limit 1.00 root offset:0, count:1", "│ └─TableReader 1.00 root data:Limit", "│ └─Limit 1.00 cop[tikv] offset:0, count:1", "│ └─TableFullScan 1.00 cop[tikv] table:cluster_index_t keep order:true, desc, stats:pseudo", - "└─HashAgg(Probe) 1.00 root funcs:min(test.cluster_index_t.a)->Column#4", + "└─StreamAgg(Probe) 1.00 root funcs:min(test.cluster_index_t.a)->Column#4", " └─Limit 1.00 root offset:0, count:1", " └─TableReader 1.00 root data:Limit", " └─Limit 1.00 cop[tikv] offset:0, count:1", @@ -207,12 +207,12 @@ "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1", "Plan": [ "HashJoin 1.00 root CARTESIAN inner join", - "├─HashAgg(Build) 1.00 root funcs:max(test.cluster_index_t.b)->Column#5", + "├─StreamAgg(Build) 1.00 root funcs:max(test.cluster_index_t.b)->Column#5", "│ └─Limit 1.00 root offset:0, count:1", "│ └─TableReader 1.00 root data:Limit", "│ └─Limit 1.00 cop[tikv] offset:0, count:1", "│ └─TableRangeScan 1.00 cop[tikv] table:cluster_index_t range:[1,1], keep order:true, desc, stats:pseudo", - "└─HashAgg(Probe) 1.00 root funcs:min(test.cluster_index_t.b)->Column#4", + "└─StreamAgg(Probe) 1.00 root funcs:min(test.cluster_index_t.b)->Column#4", " └─Limit 1.00 root offset:0, count:1", " └─TableReader 1.00 root data:Limit", " └─Limit 1.00 cop[tikv] offset:0, count:1", @@ -222,9 +222,9 @@ { "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1", "Plan": [ - "HashAgg 1.00 root funcs:min(Column#6)->Column#4, funcs:max(Column#7)->Column#5", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:min(test.cluster_index_t.a)->Column#6, funcs:max(test.cluster_index_t.a)->Column#7", + "StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:min(test.cluster_index_t.a)->Column#8, funcs:max(test.cluster_index_t.a)->Column#9", " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" ] @@ -232,9 +232,9 @@ { "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1", "Plan": [ - "HashAgg 1.00 root funcs:min(Column#6)->Column#4, funcs:max(Column#7)->Column#5", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:min(test.cluster_index_t.b)->Column#6, funcs:max(test.cluster_index_t.b)->Column#7", + "StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:min(test.cluster_index_t.b)->Column#8, funcs:max(test.cluster_index_t.b)->Column#9", " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" ] @@ -1723,7 +1723,7 @@ { "SQL": "select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#10", + "StreamAgg 1.00 root funcs:count(1)->Column#10", "└─HashJoin 1.00 root inner join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", " ├─Selection(Build) 0.80 root not(isnull(test.t.id)), not(isnull(test.t.value))", " │ └─TopN 1.00 root test.t.value, offset:0, count:1", @@ -1754,21 +1754,21 @@ { "SQL": "select count(1) from s1", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#1", + "StreamAgg 1.00 root funcs:count(1)->Column#1", "└─TableDual 1.00 root rows:1" ] }, { "SQL": "select count(*) from s1", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#1", + "StreamAgg 1.00 root funcs:count(1)->Column#1", "└─TableDual 1.00 root rows:1" ] }, { "SQL": "select sum(1) from s1", "Plan": [ - "HashAgg 1.00 root funcs:sum(1)->Column#1", + "StreamAgg 1.00 root funcs:sum(1)->Column#1", "└─TableDual 1.00 root rows:1" ] }, @@ -1777,9 +1777,9 @@ "Plan": [ "HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3", "└─Union 2.00 root ", - " ├─HashAgg 1.00 root funcs:count(1)->Column#1", + " ├─StreamAgg 1.00 root funcs:count(1)->Column#1", " │ └─TableDual 1.00 root rows:1", - " └─HashAgg 1.00 root funcs:count(1)->Column#2", + " └─StreamAgg 1.00 root funcs:count(1)->Column#2", " └─TableDual 1.00 root rows:1" ] } @@ -2344,7 +2344,7 @@ " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─HashAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", " └─IndexMerge 30263.46 root ", " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", @@ -2413,9 +2413,9 @@ { "SQL": "explain format = 'brief' select count(*) from t31240;", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" ] }, @@ -2426,9 +2426,9 @@ { "SQL": "explain format = 'brief' select count(*) from t31240;", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" ] } @@ -2485,19 +2485,19 @@ { "SQL": "explain format = 'verbose' select count(*) from t3", "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(test.t3._tidb_rowid)->Column#6", - " └─IndexFullScan_11 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" + "StreamAgg_20 1.00 102.69 root funcs:count(Column#9)->Column#4", + "└─IndexReader_21 1.00 52.79 root index:StreamAgg_8", + " └─StreamAgg_8 1.00 760.20 cop[tikv] funcs:count(test.t3._tidb_rowid)->Column#9", + " └─IndexFullScan_19 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2", "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(test.t2._tidb_rowid)->Column#5", - " └─TableFullScan_12 3.00 681.92 cop[tikv] table:t2 keep order:false" + "StreamAgg_26 1.00 107.45 root funcs:count(Column#7)->Column#4", + "└─TableReader_27 1.00 57.55 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 831.62 cop[tikv] funcs:count(test.t2._tidb_rowid)->Column#7", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { @@ -2537,7 +2537,7 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "HashAgg_8 3.00 209.09 root group by:test.t2.a, funcs:count(1)->Column#4", + "HashAgg_8 3.00 1706.09 root group by:test.t2.a, funcs:count(1)->Column#4", "└─TableReader_17 3.00 58.13 root data:TableFullScan_16", " └─TableFullScan_16 3.00 681.92 cop[tikv] table:t2 keep order:false" ] @@ -2545,37 +2545,34 @@ { "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", "Plan": [ - "HashAgg_12 1.00 45.24 root funcs:count(Column#5)->Column#4", - "└─IndexReader_13 1.00 14.98 root index:HashAgg_6", - " └─HashAgg_6 1.00 193.06 cop[tikv] funcs:count(1)->Column#5", - " └─IndexRangeScan_11 0.00 162.80 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" + "StreamAgg_10 1.00 64.98 root funcs:count(1)->Column#4", + "└─IndexReader_15 0.00 15.08 root index:IndexRangeScan_14", + " └─IndexRangeScan_14 0.00 162.80 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", "Plan": [ - "HashAgg_13 1.00 1981.96 root funcs:count(Column#6)->Column#4", - "└─IndexLookUp_14 1.00 1951.70 root ", - " ├─IndexRangeScan_11(Build) 0.00 203.50 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", - " └─HashAgg_7(Probe) 1.00 257.57 cop[tikv] funcs:count(test.t3.a)->Column#6", - " └─TableRowIDScan_12 0.00 227.31 cop[tikv] table:t3 keep order:false" + "StreamAgg_10 1.00 2001.63 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 1951.73 root ", + " ├─IndexRangeScan_15(Build) 0.00 203.50 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", + " └─TableRowIDScan_16(Probe) 0.00 227.31 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", "Plan": [ - "HashAgg_17 1.00 89.83 root funcs:count(Column#5)->Column#4", - "└─TableReader_18 1.00 59.57 root data:HashAgg_7", - " └─HashAgg_7 1.00 861.88 cop[tikv] funcs:count(1)->Column#5", - " └─Selection_14 0.00 831.62 cop[tikv] eq(test.t2.a, 0)", - " └─TableFullScan_13 3.00 681.92 cop[tikv] table:t2 keep order:false" + "StreamAgg_12 1.00 109.57 root funcs:count(1)->Column#4", + "└─TableReader_21 0.00 59.67 root data:Selection_20", + " └─Selection_20 0.00 831.62 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_19 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", "Plan": [ - "HashAgg_9 1.00 552.41 root funcs:count(1)->Column#7", - "└─HashJoin_22 3.00 482.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + "StreamAgg_10 1.00 2128.93 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 1979.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", " ├─IndexReader_28(Build) 3.00 45.23 root index:IndexFullScan_27", " │ └─IndexFullScan_27 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", " └─TableReader_26(Probe) 3.00 68.11 root data:Selection_25", @@ -2586,24 +2583,23 @@ { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "HashAgg_34 1.00 61928.14 root funcs:count(Column#8)->Column#7", - "└─TableReader_36 1.00 61897.88 root data:ExchangeSender_35", - " └─ExchangeSender_35 1.00 928450.64 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_13 1.00 928450.64 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin_33 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", - " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", - " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" + "StreamAgg_15 1.00 62053.22 root funcs:count(1)->Column#7", + "└─TableReader_41 3.00 61903.52 root data:ExchangeSender_40", + " └─ExchangeSender_40 3.00 928447.20 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_37 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "HashAgg_14 1.00 70137.12 root funcs:count(1)->Column#10", - "└─HashJoin_27 3.00 70066.94 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "StreamAgg_15 1.00 71713.64 root funcs:count(1)->Column#10", + "└─HashJoin_59 3.00 71563.94 root inner join, equal:[eq(test.t1.b, test.t3.b)]", " ├─IndexReader_47(Build) 3.00 45.23 root index:IndexFullScan_46", " │ └─IndexFullScan_46 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", " └─TableReader_39(Probe) 3.00 69652.83 root data:ExchangeSender_38", @@ -2620,13 +2616,12 @@ { "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 160743.93 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 31024.18 root eq(2, Column#18)", - "│ └─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(test.t1._tidb_rowid)->Column#31", - "│ └─TableFullScan_51 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + "HashJoin_19 3.00 162261.76 root CARTESIAN left outer semi join", + "├─Selection_38(Build) 0.80 31045.01 root eq(2, Column#18)", + "│ └─StreamAgg_56 1.00 30995.11 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_57 1.00 30945.21 root data:StreamAgg_44", + "│ └─StreamAgg_44 1.00 464146.40 batchCop[tiflash] funcs:count(test.t1._tidb_rowid)->Column#32", + "│ └─TableFullScan_55 3.00 464139.20 batchCop[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", " ├─IndexReader_26(Build) 3.00 53.37 root index:IndexFullScan_25", @@ -2642,8 +2637,8 @@ { "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "HashAgg_11 1.00 62478.44 root funcs:count(1)->Column#7", - "└─MergeJoin_16 3.00 62408.26 root inner join, left key:test.t1.a, right key:test.t2.a", + "StreamAgg_14 1.00 62557.96 root funcs:count(1)->Column#7", + "└─MergeJoin_26 3.00 62408.26 root inner join, left key:test.t1.a, right key:test.t2.a", " ├─Sort_24(Build) 3.00 31202.63 root test.t2.a", " │ └─TableReader_23 3.00 30955.77 root data:Selection_22", " │ └─Selection_22 3.00 464146.40 cop[tiflash] not(isnull(test.t2.a))", @@ -2903,7 +2898,7 @@ { "SQL": "explain format = 'brief' select max(a) from t", "Plan": [ - "HashAgg 1.00 root funcs:max(test.t.a)->Column#3", + "StreamAgg 1.00 root funcs:max(test.t.a)->Column#3", "└─TopN 1.00 root test.t.a:desc, offset:0, count:1", " └─TableReader 1.00 root data:TopN", " └─TopN 1.00 batchCop[tiflash] test.t.a:desc, offset:0, count:1", @@ -2913,7 +2908,7 @@ { "SQL": "explain format = 'brief' select min(a) from t", "Plan": [ - "HashAgg 1.00 root funcs:min(test.t.a)->Column#3", + "StreamAgg 1.00 root funcs:min(test.t.a)->Column#3", "└─Limit 1.00 root offset:0, count:1", " └─TableReader 1.00 root data:Limit", " └─Limit 1.00 cop[tiflash] offset:0, count:1", @@ -2928,7 +2923,7 @@ { "SQL": "explain format = 'brief' select max(a) from t", "Plan": [ - "HashAgg 1.00 root funcs:max(test.t.a)->Column#3", + "StreamAgg 1.00 root funcs:max(test.t.a)->Column#3", "└─TopN 1.00 root test.t.a:desc, offset:0, count:1", " └─TableReader 1.00 root data:TopN", " └─TopN 1.00 batchCop[tiflash] test.t.a:desc, offset:0, count:1", @@ -2938,7 +2933,7 @@ { "SQL": "explain format = 'brief' select min(a) from t", "Plan": [ - "HashAgg 1.00 root funcs:min(test.t.a)->Column#3", + "StreamAgg 1.00 root funcs:min(test.t.a)->Column#3", "└─TopN 1.00 root test.t.a, offset:0, count:1", " └─TableReader 1.00 root data:TopN", " └─TopN 1.00 batchCop[tiflash] test.t.a, offset:0, count:1", @@ -2953,322 +2948,301 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + "StreamAgg 1.00 root funcs:count(1)->Column#17", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#17)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#17", - " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Projection 2.00 mpp[tiflash] test.d1_t.d1_k", - " │ └─HashAgg 2.00 mpp[tiflash] group by:test.d1_t.d1_k, funcs:firstrow(test.d1_t.d1_k)->test.d1_t.d1_k", - " │ └─ExchangeReceiver 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─HashAgg 2.00 mpp[tiflash] group by:test.d1_t.d1_k, ", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 2.00 mpp[tiflash] test.d1_t.d1_k", + " │ └─HashAgg 2.00 mpp[tiflash] group by:test.d1_t.d1_k, funcs:firstrow(test.d1_t.d1_k)->test.d1_t.d1_k", + " │ └─ExchangeReceiver 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 6.40 mpp[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] } ] @@ -3517,31 +3491,29 @@ { "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", - " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:a keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", + " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:a keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", - " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:a keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", + " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:a keep order:false" ] } ] @@ -3552,35 +3524,33 @@ { "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", - " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", - " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", + " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", - " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", - " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 mpp[tiflash] ", + " │ └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " │ └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false" ] } ] @@ -3591,35 +3561,33 @@ { "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 mpp[tiflash] ", - " └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 mpp[tiflash] ", + " └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#7", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", - " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 mpp[tiflash] ", - " └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 mpp[tiflash] ", + " └─ExchangeSender 3.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " └─Selection 3.00 mpp[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 mpp[tiflash] table:b keep order:false" ] } ] @@ -3871,69 +3839,65 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] } ] @@ -3992,18 +3956,17 @@ "TableReader 1.00 root data:ExchangeSender", "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 mpp[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#16)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", " └─ExchangeReceiver 1.00 mpp[tiflash] ", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─HashAgg 1.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#16", - " └─HashJoin 1.00 mpp[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", - " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", - " │ └─TableFullScan 1.00 mpp[tiflash] table:t3 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", - " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" + " └─HashJoin 1.00 mpp[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", + " │ └─TableFullScan 1.00 mpp[tiflash] table:t3 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" ] } ] @@ -4236,9 +4199,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a) from t", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#5", + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], @@ -4247,9 +4210,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a+1) from t", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#5", + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.a, 1), decimal(20,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], @@ -4258,9 +4221,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(isnull(a)) from t", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#5)->Column#4", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#5", + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", " └─Projection 10000.00 batchCop[tiflash] cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], @@ -6154,9 +6117,9 @@ " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 10000.00 root ", - " └─HashAgg 10000.00 root funcs:sum(Column#8)->Column#7", - " └─TableReader 10000.00 root data:HashAgg", - " └─HashAgg 10000.00 cop[tikv] funcs:sum(test.t2.a)->Column#8", + " └─StreamAgg 10000.00 root funcs:sum(Column#9)->Column#7", + " └─TableReader 10000.00 root data:StreamAgg", + " └─StreamAgg 10000.00 cop[tikv] funcs:sum(test.t2.a)->Column#9", " └─Selection 100000.00 cop[tikv] eq(test.t2.b, test.t1.b)", " └─TableFullScan 100000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], @@ -6209,12 +6172,11 @@ { "SQL": "explain format = 'brief' select /*+ qb_name(qb_v9, v9), AGG_TO_COP(@qb_v9) */ * from v9;", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#6)->Column#4", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#6", - " └─Projection 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#9", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:sum(Column#7)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#9)->Column#7", + " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#9", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -6768,33 +6730,30 @@ { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#14", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(Column#19)->Column#18", - " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#19", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#16)->Column#14", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#19)->Column#16", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#19", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "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(test.t._tidb_rowid)->Column#17", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#15)->Column#14", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t._tidb_rowid)->Column#15", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#18)->Column#14", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#19)->Column#18", - " └─Projection 10000.00 mpp[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#19", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:sum(Column#16)->Column#14", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#19)->Column#16", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#19", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7104,33 +7063,30 @@ { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#10)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(Column#11)->Column#10", - " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "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(test.t._tidb_rowid)->Column#9", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#7)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─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" ] }, { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#10)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#11)->Column#10", - " └─Projection 10000.00 mpp[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:sum(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7353,7 +7309,7 @@ { "SQL": "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt", "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#11", + "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─Union 0.00 root ", " ├─Projection 0.00 root test.t.a, cast(test.t.b, double BINARY)->Column#10", " │ └─TableDual 0.00 root rows:0", @@ -7603,11 +7559,10 @@ { "SQL": "desc format = 'brief' select count(*) from t", "Plan": [ - "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(test.t._tidb_rowid)->Column#6", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#7)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─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" ] }, { @@ -8134,7 +8089,7 @@ { "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", "Plan": [ - "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "StreamAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", "└─TopN 20.00 root test.t.value, offset:0, count:20", " └─TableReader 20.00 root data:ExchangeSender", " └─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", @@ -9585,9 +9540,9 @@ { "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is not null", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", " └─IndexRangeScan 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo" ], "Result": [ @@ -9621,9 +9576,9 @@ { "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is null", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", " └─Selection 3.33 cop[tikv] isnull(test.t1.c2)", " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" ], @@ -9634,12 +9589,11 @@ { "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#9)->Column#5", - "└─IndexLookUp 1.00 root ", + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─HashAgg(Probe) 1.00 cop[tikv] funcs:count(1)->Column#9", - " └─Selection 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " └─Selection(Probe) 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Result": [ "3" @@ -9648,12 +9602,11 @@ { "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#9)->Column#5", - "└─IndexLookUp 1.00 root ", + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─HashAgg(Probe) 1.00 cop[tikv] funcs:count(1)->Column#9", - " └─Selection 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " └─Selection(Probe) 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Result": [ "1" @@ -9728,9 +9681,9 @@ { "SQL": "select count(1) from t2 use index(idx) where b is null", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", " └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo" ], "Result": [ diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index e9bdcc1010cb6..e2a6c562726d4 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -6296,7 +6296,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -6915,7 +6915,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -6935,7 +6935,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -6957,7 +6957,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -6979,7 +6979,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -7002,7 +7002,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -7024,7 +7024,7 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─Apply 1.00 root CARTESIAN left outer join", - " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", @@ -7049,11 +7049,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(Column#20)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#20", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─IndexLookUp(Probe) 3.00 root ", "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", @@ -7079,11 +7078,10 @@ "└─IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", " ├─HashJoin(Build) 3.75 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", " │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(Column#18)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - " │ │ └─TableReader 3.00 root data:HashAgg", - " │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#18", - " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ │ └─TableReader 3.00 root data:Selection", + " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.b)]", " │ ├─TableReader(Build) 9980.01 root data:Selection", " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", @@ -7105,11 +7103,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(Column#20)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#20", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─IndexLookUp(Probe) 3.00 root ", "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", @@ -7134,11 +7131,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(Column#20)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#20", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─IndexLookUp(Probe) 3.00 root ", "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", @@ -7164,11 +7160,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(Column#17)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#17", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", @@ -7192,11 +7187,10 @@ "└─IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", " ├─HashJoin(Build) 3.75 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", " │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(Column#18)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - " │ │ └─TableReader 3.00 root data:HashAgg", - " │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#18", - " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ │ └─TableReader 3.00 root data:Selection", + " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.b)]", " │ ├─TableReader(Build) 9980.01 root data:Selection", " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", @@ -7218,11 +7212,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(Column#17)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#17", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", @@ -7245,11 +7238,10 @@ "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", - "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(Column#17)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", - "│ │ │ └─TableReader 3.00 root data:HashAgg", - "│ │ │ └─HashAgg 3.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#17", - "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", - "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 0da53482a34cc..e4647840aa220 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -802,7 +802,7 @@ "1" ], "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 4.00 root inner join, equal:[eq(test_partition.t6.b, test_partition.t5.b)]", " ├─IndexReader(Build) 4.00 root partition:p0,p1 index:Selection", " │ └─Selection 4.00 cop[tikv] not(isnull(test_partition.t5.b))", @@ -818,7 +818,7 @@ "1" ], "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─IndexJoin 4.00 root inner join, inner:IndexReader, outer key:test_partition.t6.b, inner key:test_partition.t5.b, equal cond:eq(test_partition.t6.b, test_partition.t5.b)", " ├─IndexReader(Build) 4.00 root partition:p0 index:Selection", " │ └─Selection 4.00 cop[tikv] not(isnull(test_partition.t6.b))", @@ -834,7 +834,7 @@ "1" ], "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─IndexHashJoin 4.00 root inner join, inner:IndexReader, outer key:test_partition.t6.b, inner key:test_partition.t5.b, equal cond:eq(test_partition.t6.b, test_partition.t5.b)", " ├─IndexReader(Build) 4.00 root partition:p0 index:Selection", " │ └─Selection 4.00 cop[tikv] not(isnull(test_partition.t6.b))", @@ -934,16 +934,16 @@ "2" ], "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", - "└─TableReader 1.00 root partition:p0 data:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─TableReader 1.00 root partition:p0 data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", " └─Selection 19.99 cop[tikv] or(eq(test_partition.t1.a, 1), eq(test_partition.t1.b, 2))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader 1.00 root partition:p0 index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + "StreamAgg 1.00 root funcs:count(Column#10)->Column#5", + "└─IndexReader 1.00 root partition:p0 index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#10", " └─Selection 19.99 cop[tikv] or(eq(test_partition_1.t1.a, 1), eq(test_partition_1.t1.b, 2))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1877,7 +1877,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─IndexHashJoin 0.41 root inner join, inner:IndexReader, outer key:test_partition_1.t2.b, inner key:test_partition_1.t1.b, equal cond:eq(test_partition_1.t2.b, test_partition_1.t1.b)", " ├─IndexReader(Build) 0.80 root partition:p0 index:Selection", " │ └─Selection 0.80 cop[tikv] not(isnull(test_partition_1.t2.b))", @@ -2789,7 +2789,7 @@ "2" ], "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition.t1.b, test_partition.t2.b)]", " ├─TableReader(Build) 6.25 root partition:p0,p1 data:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition.t2.a, 1), ge(test_partition.t2.b, 1), le(test_partition.t2.a, 6), le(test_partition.t2.b, 6), not(isnull(test_partition.t2.b))", @@ -2799,7 +2799,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t1.b, test_partition_1.t2.b)]", " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t2.b, 1), le(test_partition_1.t2.b, 6), not(isnull(test_partition_1.t2.b))", @@ -2815,7 +2815,7 @@ "2" ], "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition.t2.b, test_partition.t1.b)]", " ├─TableReader(Build) 6.25 root partition:p0,p1 data:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition.t1.a, 1), ge(test_partition.t1.b, 1), le(test_partition.t1.a, 6), le(test_partition.t1.b, 6), not(isnull(test_partition.t1.b))", @@ -2825,7 +2825,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashAgg 1.00 root funcs:count(1)->Column#9", + "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t2.b, test_partition_1.t1.b)]", " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t1.b, 1), le(test_partition_1.t1.b, 6), not(isnull(test_partition_1.t1.b))", diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 8d246a2af2f1a..6b8612f70992d 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -963,7 +963,7 @@ }, { "SQL": "select sum(e), avg(e + c) from t where c = 1 group by e", - "Best": "IndexReader(Index(t.c_d_e)[[1,1]]->HashAgg)->HashAgg" + "Best": "IndexReader(Index(t.c_d_e)[[1,1]])->Projection->HashAgg" }, { "SQL": "select sum(e), avg(b + c) from t where c = 1 and e = 1 group by d", @@ -1257,15 +1257,15 @@ "Cases": [ { "SQL": "select max(a) from t;", - "Best": "TableReader(Table(t)->Limit)->Limit->HashAgg" + "Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg" }, { "SQL": "select min(a) from t;", - "Best": "TableReader(Table(t)->Limit)->Limit->HashAgg" + "Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg" }, { "SQL": "select min(c_str) from t;", - "Best": "IndexReader(Index(t.c_d_e_str)[[-inf,+inf]]->Limit)->Limit->HashAgg" + "Best": "IndexReader(Index(t.c_d_e_str)[[-inf,+inf]]->Limit)->Limit->StreamAgg" }, { "SQL": "select max(a), b from t;", @@ -1273,23 +1273,23 @@ }, { "SQL": "select max(a+1) from t;", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([plus(test.t.a, 1) true],0,1))->Projection->TopN([Column#40 true],0,1)->Projection->Projection->HashAgg" + "Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([plus(test.t.a, 1) true],0,1))->Projection->TopN([Column#40 true],0,1)->Projection->Projection->StreamAgg" }, { "SQL": "select max(a), min(a) from t;", - "Best": "LeftHashJoin{TableReader(Table(t)->Limit)->Limit->HashAgg->TableReader(Table(t)->Limit)->Limit->HashAgg}" + "Best": "RightHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->TableReader(Table(t)->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(a), min(a) from t where a > 10", - "Best": "LeftHashJoin{TableReader(Table(t)->Limit)->Limit->HashAgg->TableReader(Table(t)->Limit)->Limit->HashAgg}" + "Best": "RightHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->TableReader(Table(t)->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(d), min(d) from t where c = 1 and d > 10", - "Best": "LeftHashJoin{IndexReader(Index(t.c_d_e)[(1 10,1 +inf]]->Limit)->Limit->HashAgg->IndexReader(Index(t.c_d_e)[(1 10,1 +inf]]->Limit)->Limit->HashAgg}" + "Best": "LeftHashJoin{IndexReader(Index(t.c_d_e)[(1 10,1 +inf]]->Limit)->Limit->StreamAgg->IndexReader(Index(t.c_d_e)[(1 10,1 +inf]]->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(a), max(c), min(f) from t", - "Best": "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->HashAgg->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit->HashAgg}->IndexReader(Index(t.f)[[NULL,+inf]]->Limit)->Limit->HashAgg}" + "Best": "LeftHashJoin{RightHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit->StreamAgg}->IndexReader(Index(t.f)[[NULL,+inf]]->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(a), max(b) from t", @@ -1309,7 +1309,7 @@ }, { "SQL": "select max(a) from (select t1.a from t t1 join t t2 on t1.a=t2.a) t", - "Best": "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)->Limit->HashAgg" + "Best": "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)->Limit->StreamAgg" } ] }, @@ -5935,7 +5935,7 @@ "└─Apply 10000.00 root CARTESIAN inner join", " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─HashAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#8, funcs:count(distinct Column#15)->Column#9, funcs:sum(Column#16)->Column#10, funcs:count(1)->Column#11", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#8, funcs:count(distinct Column#15)->Column#9, funcs:sum(Column#16)->Column#10, funcs:count(1)->Column#11", " └─Projection 10000.00 root test.t2.b, test.t2.b, cast(isnull(test.t2.b), decimal(20,0) BINARY)->Column#16", " └─TableReader 10000.00 root data:TableRangeScan", " └─TableRangeScan 10000.00 cop[tikv] table:t2 range: decided by [eq(test.t2.a, test.t1.b)], keep order:false, stats:pseudo" @@ -5953,7 +5953,7 @@ "└─Apply 10000.00 root CARTESIAN inner join", " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─HashAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#8, funcs:count(distinct Column#15)->Column#9, funcs:sum(Column#16)->Column#10, funcs:count(1)->Column#11", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#8, funcs:count(distinct Column#15)->Column#9, funcs:sum(Column#16)->Column#10, funcs:count(1)->Column#11", " └─Projection 10000.00 root test.t2.b, test.t2.b, cast(isnull(test.t2.b), decimal(20,0) BINARY)->Column#16", " └─TableReader 10000.00 root data:TableRangeScan", " └─TableRangeScan 10000.00 cop[tikv] table:t2 range: decided by [eq(test.t2.a, test.t1.b)], keep order:false, stats:pseudo" @@ -5971,7 +5971,7 @@ "└─Apply 10000.00 root CARTESIAN inner join", " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─HashAgg(Probe) 10000.00 root funcs:max(Column#19)->Column#8, funcs:sum(Column#20)->Column#9, funcs:count(1)->Column#10", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#19)->Column#8, funcs:sum(Column#20)->Column#9, funcs:count(1)->Column#10", " └─Projection 10000.00 root test.t2.b, cast(isnull(test.t2.b), decimal(20,0) BINARY)->Column#20", " └─TableReader 10000.00 root data:TableRangeScan", " └─TableRangeScan 10000.00 cop[tikv] table:t2 range: decided by [eq(test.t2.a, test.t1.b)], keep order:false, stats:pseudo" @@ -5989,7 +5989,7 @@ "└─Apply 10000.00 root CARTESIAN inner join", " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─HashAgg(Probe) 10000.00 root funcs:max(Column#19)->Column#8, funcs:sum(Column#20)->Column#9, funcs:count(1)->Column#10", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#19)->Column#8, funcs:sum(Column#20)->Column#9, funcs:count(1)->Column#10", " └─Projection 10000.00 root test.t2.b, cast(isnull(test.t2.b), decimal(20,0) BINARY)->Column#20", " └─TableReader 10000.00 root data:TableRangeScan", " └─TableRangeScan 10000.00 cop[tikv] table:t2 range: decided by [eq(test.t2.a, test.t1.b)], keep order:false, stats:pseudo" @@ -6159,9 +6159,9 @@ " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 10000.00 root ", - " └─HashAgg 10000.00 root funcs:count(Column#11)->Column#10", - " └─TableReader 10000.00 root data:HashAgg", - " └─HashAgg 10000.00 cop[tikv] funcs:count(test.t3.a)->Column#11", + " └─StreamAgg 10000.00 root funcs:count(Column#12)->Column#10", + " └─TableReader 10000.00 root data:StreamAgg", + " └─StreamAgg 10000.00 cop[tikv] funcs:count(test.t3.a)->Column#12", " └─Selection 100000.00 cop[tikv] eq(test.t3.b, test.t1.b)", " └─TableFullScan 100000000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" ], @@ -6234,7 +6234,7 @@ " │ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", " │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 250.00 root ", - " └─HashAgg 250.00 root funcs:sum(Column#33)->Column#18", + " └─StreamAgg 250.00 root funcs:sum(Column#33)->Column#18", " └─Projection 1998.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#33", " └─IndexHashJoin 1998.00 root semi join, inner:IndexLookUp, outer key:test.tb.name, inner key:test.tc.name, equal cond:eq(test.tb.name, test.tc.name)", " ├─IndexLookUp(Build) 2497.50 root ", @@ -6280,7 +6280,7 @@ " │ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", " │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 250.00 root ", - " └─HashAgg 250.00 root funcs:sum(Column#22)->Column#18", + " └─StreamAgg 250.00 root funcs:sum(Column#22)->Column#18", " └─Projection 2500.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#22", " └─Apply 2500.00 root CARTESIAN semi join", " ├─IndexLookUp(Build) 2500.00 root ", @@ -6323,7 +6323,7 @@ " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─Selection(Probe) 8.00 root gt(Column#9, 900)", " └─MaxOneRow 10.00 root ", - " └─HashAgg 10.00 root funcs:max(test.tb.code)->Column#9", + " └─StreamAgg 10.00 root funcs:max(test.tb.code)->Column#9", " └─TopN 10.00 root test.tb.code:desc, offset:0, count:1", " └─IndexLookUp 10.00 root ", " ├─IndexRangeScan(Build) 100.00 cop[tikv] table:tb, index:idx_tb_id(id) range: decided by [eq(test.ta.id, test.tb.id)], keep order:false, stats:pseudo", diff --git a/session/testdata/clustered_index_suite_out.json b/session/testdata/clustered_index_suite_out.json index dd0f1f630b9fc..e7bc6f0cc52e7 100644 --- a/session/testdata/clustered_index_suite_out.json +++ b/session/testdata/clustered_index_suite_out.json @@ -15,9 +15,9 @@ { "SQL": "select count(1) from t1 use index(idx1) where c2 = 'cd'", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", " └─Selection 10.00 cop[tikv] eq(test.t1.c2, \"cd\")", " └─IndexRangeScan 10.00 cop[tikv] table:t1, index:idx1(c2) range:[\"c\",\"c\"], keep order:false, stats:pseudo" ], @@ -38,9 +38,9 @@ { "SQL": "select count(1) from t2 use index(idx1) where c2 = 'cd'", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#4", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(Column#9)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#9", " └─Selection 10.00 cop[tikv] eq(test.t2.c2, \"cd\")", " └─IndexFullScan 10000.00 cop[tikv] table:t2, index:idx1(c1, c2) keep order:false, stats:pseudo" ], diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index 3934ff2f7bad3..4a4f0c85e2682 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -9,7 +9,7 @@ "└─Apply 2.00 root CARTESIAN left outer semi join, other cond:eq(test.t.e, Column#26)", " ├─TableReader(Build) 2.00 root data:TableFullScan", " │ └─TableFullScan 2.00 cop[tikv] table:t keep order:false", - " └─HashAgg(Probe) 2.00 root funcs:count(1)->Column#26", + " └─StreamAgg(Probe) 2.00 root funcs:count(1)->Column#26", " └─HashJoin 4.00 root inner join, equal:[eq(test.t.a, test.t.a)]", " ├─IndexReader(Build) 4.00 root index:IndexFullScan", " │ └─IndexFullScan 4.00 cop[tikv] table:t1, index:idx(b, c, d) keep order:false", From cca78e37e249c235c9664fc6b5f6c2ac4b4a3b79 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Wed, 30 Nov 2022 01:02:00 +0800 Subject: [PATCH 06/13] planner: forbid point get with select lock choose tiflash as datasource path (#39388) close pingcap/tidb#39344 --- planner/core/find_best_task.go | 68 ++++++++++++++++++++++++++++++++ planner/core/integration_test.go | 34 ++++++++++++++++ 2 files changed, 102 insertions(+) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 8596746822b23..52eab05629d59 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -912,6 +912,14 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter p: dual, }, cntPlan, nil } + + // if the path is the point get range path with for update lock, we should forbid tiflash as it's store path. + if path.StoreType == kv.TiFlash && ds.isForUpdateRead && ds.ctx.GetSessionVars().TxnCtx.IsExplicit { + if ds.isPointGetConditions() { + continue + } + } + canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV && ds.isPointGetConvertableSchema() if canConvertPointGet && expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) { @@ -1903,6 +1911,66 @@ func (s *LogicalIndexScan) GetPhysicalIndexScan(_ *expression.Schema, stats *pro return is } +// isPointGetConditions indicates whether the conditions are point-get-able. +// eg: create table t(a int, b int,c int unique, primary (a,b)) +// select * from t where a = 1 and b = 1 and c =1; +// the datasource can access by primary key(a,b) or unique key (c) which are both point-get-able +func (ds *DataSource) isPointGetConditions() bool { + t, _ := ds.is.TableByID(ds.physicalTableID) + columns := map[string]struct{}{} + for _, cond := range ds.allConds { + s, ok := cond.(*expression.ScalarFunction) + if !ok { + return false + } + if s.FuncName.L != ast.EQ || (s.FuncName.L == ast.In && len(s.GetArgs()) != 2) { + return false + } + arg0 := s.GetArgs()[0] + arg1 := s.GetArgs()[1] + _, ok1 := arg0.(*expression.Constant) + col, ok2 := arg1.(*expression.Column) + if ok1 && ok2 { + columns[t.Meta().FindColumnNameByID(col.ID)] = struct{}{} + continue + } + col, ok1 = arg0.(*expression.Column) + _, ok2 = arg1.(*expression.Constant) + if ok1 && ok2 { + columns[t.Meta().FindColumnNameByID(col.ID)] = struct{}{} + continue + } + } + return ds.findPKOrUniqueIndexMatchColumns(columns) +} + +func (ds *DataSource) findPKOrUniqueIndexMatchColumns(columns map[string]struct{}) bool { + for _, idx := range ds.tableInfo.Indices { + if !idx.Unique && !idx.Primary { + continue + } + if idx.HasPrefixIndex() { + continue + } + if len(idx.Columns) > len(columns) { + continue + } + flag := true + for _, idxCol := range idx.Columns { + _, ok := columns[idxCol.Name.String()] + if !ok { + flag = false + break + } + } + if !flag { + continue + } + return true + } + return false +} + // convertToTableScan converts the DataSource to table scan. func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *physicalOptimizeOp) (task task, err error) { // It will be handled in convertToIndexScan. diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index b50cc21397fa1..e03a2052d8ada 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7556,6 +7556,40 @@ func TestEnableTiFlashReadForWriteStmt(t *testing.T) { checkMpp(rs) } +func TestPointGetWithSelectLock(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, primary key(a, b));") + tk.MustExec("create table t1(c int unique, d int);") + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + tk.MustExec("set @@tidb_enable_tiflash_read_for_write_stmt = on;") + tk.MustExec("set @@tidb_isolation_read_engines='tidb,tiflash';") + tk.MustExec("begin;") + // assert point get condition with txn commit and tiflash store + tk.MustGetErrMsg("explain select a, b from t where a = 1 and b = 2 for update;", "[planner:1815]Internal : Can't find a proper physical plan for this query") + tk.MustGetErrMsg("explain select c, d from t1 where c = 1 for update;", "[planner:1815]Internal : Can't find a proper physical plan for this query") + tk.MustGetErrMsg("explain select c, d from t1 where c = 1 and d = 1 for update;", "[planner:1815]Internal : Can't find a proper physical plan for this query") + tk.MustQuery("explain select a, b from t where a = 1 for update;") + tk.MustQuery("explain select c, d from t1 where c > 1 for update;") + tk.MustExec("set tidb_isolation_read_engines='tidb,tikv,tiflash';") + tk.MustQuery("explain select a, b from t where a = 1 and b = 2 for update;") + tk.MustQuery("explain select c, d from t1 where c = 1 for update;") + tk.MustExec("commit") + tk.MustExec("set tidb_isolation_read_engines='tidb,tiflash';") + // assert point get condition with auto commit and tiflash store + tk.MustQuery("explain select a, b from t where a = 1 and b = 2 for update;") + tk.MustQuery("explain select c, d from t1 where c = 1 for update;") +} + func TestTableRangeFallback(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) From 6241d6054f06078be8601a3a1b8cc724fe1153db Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 30 Nov 2022 10:30:00 +0800 Subject: [PATCH 07/13] planner, executor: support intersection type IndexMerge with hint in planner (#38650) close pingcap/tidb#39335 --- cmd/explaintest/r/explain_indexmerge.result | 26 +- cmd/explaintest/r/index_merge.result | 88 ++-- executor/builder.go | 1 + executor/index_merge_reader_test.go | 22 +- planner/core/BUILD.bazel | 1 + planner/core/explain.go | 7 +- planner/core/indexmerge_intersection_test.go | 178 ++++++++ planner/core/integration_test.go | 20 +- planner/core/main_test.go | 4 + planner/core/stats.go | 141 +++++- .../core/testdata/index_merge_suite_in.json | 32 +- .../core/testdata/index_merge_suite_out.json | 422 ++++++++++++++++++ .../core/testdata/integration_suite_out.json | 54 +-- testkit/testkit.go | 12 + 14 files changed, 899 insertions(+), 109 deletions(-) create mode 100644 planner/core/indexmerge_intersection_test.go diff --git a/cmd/explaintest/r/explain_indexmerge.result b/cmd/explaintest/r/explain_indexmerge.result index 07530b9596bb5..46ba855e6c2a8 100644 --- a/cmd/explaintest/r/explain_indexmerge.result +++ b/cmd/explaintest/r/explain_indexmerge.result @@ -7,33 +7,33 @@ create index td on t (d); load stats 's/explain_indexmerge_stats_t.json'; explain format = 'brief' select * from t where a < 50 or b < 50; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false └─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where (a < 50 or b < 50) and f > 100; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false └─Selection(Probe) 98.00 cop[tikv] gt(test.t.f, 100) └─TableRowIDScan 98.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where b < 50 or c < 50; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tc(c) range:[-inf,50), keep order:false └─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false set session tidb_enable_index_merge = on; explain format = 'brief' select * from t where a < 50 or b < 50; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false └─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where (a < 50 or b < 50) and f > 100; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false └─Selection(Probe) 98.00 cop[tikv] gt(test.t.f, 100) @@ -45,7 +45,7 @@ TableReader 4999999.00 root data:Selection └─TableFullScan 5000000.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where b < 50 or c < 50; id estRows task access object operator info -IndexMerge 98.00 root +IndexMerge 98.00 root type: union ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tc(c) range:[-inf,50), keep order:false └─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false @@ -56,14 +56,14 @@ TableReader 4999999.00 root data:Selection └─TableFullScan 5000000.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where a < 50 or b < 50 or c < 50; id estRows task access object operator info -IndexMerge 147.00 root +IndexMerge 147.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tc(c) range:[-inf,50), keep order:false └─TableRowIDScan(Probe) 147.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where (b < 10000 or c < 10000) and (a < 10 or d < 10) and f < 10; id estRows task access object operator info -IndexMerge 0.00 root +IndexMerge 0.00 root type: union ├─TableRangeScan(Build) 9.00 cop[tikv] table:t range:[-inf,10), keep order:false ├─IndexRangeScan(Build) 9.00 cop[tikv] table:t, index:td(d) range:[-inf,10), keep order:false └─Selection(Probe) 0.00 cop[tikv] lt(test.t.f, 10), or(lt(test.t.b, 10000), lt(test.t.c, 10000)) @@ -104,20 +104,20 @@ label = "cop" set session tidb_enable_index_merge = off; explain format = 'brief' select /*+ use_index_merge(t, primary, tb, tc) */ * from t where a <= 500000 or b <= 1000000 or c <= 3000000; id estRows task access object operator info -IndexMerge 3560000.00 root +IndexMerge 3560000.00 root type: union ├─TableRangeScan(Build) 500000.00 cop[tikv] table:t range:[-inf,500000], keep order:false ├─IndexRangeScan(Build) 1000000.00 cop[tikv] table:t, index:tb(b) range:[-inf,1000000], keep order:false ├─IndexRangeScan(Build) 3000000.00 cop[tikv] table:t, index:tc(c) range:[-inf,3000000], keep order:false └─TableRowIDScan(Probe) 3560000.00 cop[tikv] table:t keep order:false explain format = 'brief' select /*+ use_index_merge(t, tb, tc) */ * from t where b < 50 or c < 5000000; id estRows task access object operator info -IndexMerge 4999999.00 root +IndexMerge 4999999.00 root type: union ├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 4999999.00 cop[tikv] table:t, index:tc(c) range:[-inf,5000000), keep order:false └─TableRowIDScan(Probe) 4999999.00 cop[tikv] table:t keep order:false explain format = 'brief' select /*+ use_index_merge(t, tb, tc) */ * from t where (b < 10000 or c < 10000) and (a < 10 or d < 10) and f < 10; id estRows task access object operator info -IndexMerge 0.00 root +IndexMerge 0.00 root type: union ├─IndexRangeScan(Build) 9999.00 cop[tikv] table:t, index:tb(b) range:[-inf,10000), keep order:false ├─IndexRangeScan(Build) 9999.00 cop[tikv] table:t, index:tc(c) range:[-inf,10000), keep order:false └─Selection(Probe) 0.00 cop[tikv] lt(test.t.f, 10), or(lt(test.t.a, 10), lt(test.t.d, 10)) @@ -134,7 +134,7 @@ TableReader 4999999.00 root data:Selection └─TableFullScan 5000000.00 cop[tikv] table:t keep order:false explain format = 'brief' select /*+ use_index_merge(t, primary, tb) */ * from t where a < 50 or b < 5000000; id estRows task access object operator info -IndexMerge 4999999.00 root +IndexMerge 4999999.00 root type: union ├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false ├─IndexRangeScan(Build) 4999999.00 cop[tikv] table:t, index:tb(b) range:[-inf,5000000), keep order:false └─TableRowIDScan(Probe) 4999999.00 cop[tikv] table:t keep order:false @@ -151,7 +151,7 @@ KEY `aid_c2` (`aid`,`c2`) ); desc select /*+ USE_INDEX_MERGE(t, aid_c1, aid_c2) */ * from t where (aid = 1 and c1='aaa') or (aid = 2 and c2='bbb'); id estRows task access object operator info -IndexMerge_8 8.08 root +IndexMerge_8 8.08 root type: union ├─IndexRangeScan_5(Build) 0.10 cop[tikv] table:t, index:aid_c1(aid, c1) range:[1 "aaa",1 "aaa"], keep order:false, stats:pseudo ├─IndexRangeScan_6(Build) 0.10 cop[tikv] table:t, index:aid_c2(aid, c2) range:[2 "bbb",2 "bbb"], keep order:false, stats:pseudo └─TableRowIDScan_7(Probe) 8.08 cop[tikv] table:t keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 7e02e6255834f..7d798cb972006 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -15,7 +15,7 @@ Sort_8 4433.77 root test.t1.c1 └─HashJoin_12 5542.21 root CARTESIAN left outer semi join, other cond:eq(test.t1.c3, test.t1.c3) ├─TableReader_18(Build) 10000.00 root data:TableFullScan_17 │ └─TableFullScan_17 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo - └─IndexMerge_16(Probe) 5542.21 root + └─IndexMerge_16(Probe) 5542.21 root type: union ├─IndexRangeScan_13(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_14(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_15(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -35,7 +35,7 @@ Sort_8 4433.77 root test.t1.c1 └─HashJoin_12 5542.21 root CARTESIAN anti left outer semi join, other cond:eq(test.t1.c3, test.t1.c3) ├─TableReader_18(Build) 10000.00 root data:TableFullScan_17 │ └─TableFullScan_17 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo - └─IndexMerge_16(Probe) 5542.21 root + └─IndexMerge_16(Probe) 5542.21 root type: union ├─IndexRangeScan_13(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_14(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_15(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -50,7 +50,7 @@ c1 c2 c3 explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 = (select max(c3) from t1) order by 1; id estRows task access object operator info Sort_33 3325.55 root test.t1.c1 -└─IndexMerge_40 1843.09 root +└─IndexMerge_40 1843.09 root type: union ├─IndexRangeScan_36(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_37(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_39(Probe) 1843.09 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), eq(test.t1.c3, 5))) @@ -71,7 +71,7 @@ Sort_9 4433.77 root test.t1.c1 └─HashJoin_22 5542.21 root left outer semi join, equal:[eq(test.t1.c1, test.t2.c1)] ├─IndexReader_30(Build) 10000.00 root index:IndexFullScan_29 │ └─IndexFullScan_29 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo - └─IndexMerge_26(Probe) 5542.21 root + └─IndexMerge_26(Probe) 5542.21 root type: union ├─IndexRangeScan_23(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_24(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_25(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -91,7 +91,7 @@ Sort_9 4433.77 root test.t1.c1 └─HashJoin_22 5542.21 root anti left outer semi join, equal:[eq(test.t1.c1, test.t2.c1)] ├─IndexReader_30(Build) 10000.00 root index:IndexFullScan_29 │ └─IndexFullScan_29 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo - └─IndexMerge_26(Probe) 5542.21 root + └─IndexMerge_26(Probe) 5542.21 root type: union ├─IndexRangeScan_23(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_24(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_25(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -106,7 +106,7 @@ c1 c2 c3 explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 = (select count(1) from t2) order by 1; id estRows task access object operator info Sort_38 3325.55 root test.t1.c1 -└─IndexMerge_45 1843.09 root +└─IndexMerge_45 1843.09 root type: union ├─IndexRangeScan_41(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_42(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_44(Probe) 1843.09 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), eq(test.t1.c3, 5))) @@ -128,7 +128,7 @@ Sort_11 5098.44 root test.t1.c1 │ └─IndexReader_44 1.00 root index:StreamAgg_27 │ └─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 + └─IndexMerge_21(Probe) 2825.66 root type: union ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_18(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_20(Probe) 2825.66 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), if(isnull(test.t1.c3), NULL, 1))) @@ -150,7 +150,7 @@ Sort_11 5098.44 root test.t1.c1 │ └─IndexReader_44 1.00 root index:StreamAgg_27 │ └─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 + └─IndexMerge_21(Probe) 2825.66 root type: union ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_18(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_20(Probe) 2825.66 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), if(isnull(test.t1.c3), NULL, 1))) @@ -172,7 +172,7 @@ Sort_11 5542.21 root test.t1.c1 │ └─IndexReader_43 1.00 root index:StreamAgg_26 │ └─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 + └─IndexMerge_20(Probe) 5542.21 root type: union ├─IndexRangeScan_17(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_18(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_19(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -194,7 +194,7 @@ Sort_39 5542.21 root test.t1.c1 │ └─IndexReader_71 1.00 root index:StreamAgg_54 │ └─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 + └─IndexMerge_48(Probe) 5542.21 root type: union ├─IndexRangeScan_45(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_46(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_47(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -220,7 +220,7 @@ Sort_14 4433.77 root test.t1.c1 │ └─TableReader_51(Probe) 9990.00 root data:Selection_50 │ └─Selection_50 9990.00 cop[tikv] not(isnull(test.t2.c2)) │ └─TableFullScan_49 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo - └─IndexMerge_22(Probe) 5542.21 root + └─IndexMerge_22(Probe) 5542.21 root type: union ├─IndexRangeScan_19(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_20(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_21(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -239,7 +239,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 └─Selection_12 2250.55 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -254,7 +254,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and id estRows task access object operator info Sort_5 5098.44 root test.t1.c1 └─Selection_12 2825.66 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), eq(test.t1.c3, plus(test.t1.c1, test.t1.c2)))) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -269,7 +269,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and id estRows task access object operator info Sort_5 5098.44 root test.t1.c1 └─Selection_12 2825.66 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), istrue_with_null(cast(substring(cast(test.t1.c3, var_string(20)), test.t1.c2), double BINARY)))) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -284,7 +284,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and id estRows task access object operator info Sort_5 4800.37 root test.t1.c1 └─Selection_12 2660.47 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), test.t1.c3)) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -304,7 +304,7 @@ explain select * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 └─Selection_12 2250.55 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -338,7 +338,7 @@ insert into t1 values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5); explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 -└─IndexMerge_12 2250.55 root +└─IndexMerge_12 2250.55 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -368,7 +368,7 @@ alter table t1 add index c1(c1); explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 -└─IndexMerge_12 2250.55 root +└─IndexMerge_12 2250.55 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -391,7 +391,7 @@ Delete_11 N/A root N/A └─SelectLock_17 4056.68 root for update 0 └─HashJoin_33 4056.68 root inner join, equal:[eq(test.t1.c1, test.t1.c1)] ├─HashAgg_36(Build) 3245.34 root group by:test.t1.c1, funcs:firstrow(test.t1.c1)->test.t1.c1 - │ └─IndexMerge_41 2248.30 root + │ └─IndexMerge_41 2248.30 root type: union │ ├─IndexRangeScan_37(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo │ ├─IndexRangeScan_38(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo │ └─Selection_40(Probe) 2248.30 cop[tikv] not(isnull(test.t1.c1)), or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -409,7 +409,7 @@ Update_10 N/A root N/A └─SelectLock_14 4056.68 root for update 0 └─HashJoin_30 4056.68 root inner join, equal:[eq(test.t1.c1, test.t1.c1)] ├─HashAgg_33(Build) 3245.34 root group by:test.t1.c1, funcs:firstrow(test.t1.c1)->test.t1.c1 - │ └─IndexMerge_38 2248.30 root + │ └─IndexMerge_38 2248.30 root type: union │ ├─IndexRangeScan_34(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo │ ├─IndexRangeScan_35(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo │ └─Selection_37(Probe) 2248.30 cop[tikv] not(isnull(test.t1.c1)), or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -426,7 +426,7 @@ id estRows task access object operator info Sort_6 4060.74 root test.t1.c1 └─Projection_8 4060.74 root test.t1.c1, test.t1.c2, test.t1.c3 └─SelectLock_9 4060.74 root for update 0 - └─IndexMerge_14 2250.55 root + └─IndexMerge_14 2250.55 root type: union ├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_11(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_13(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -471,7 +471,7 @@ insert into t1 values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5); explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 order by 1 limit 1 offset 2; id estRows task access object operator info TopN_10 1.00 root test.t1.c1, offset:2, count:1 -└─IndexMerge_19 1841.86 root +└─IndexMerge_19 1841.86 root type: union ├─IndexRangeScan_15(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_16(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_18(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10) @@ -485,7 +485,7 @@ id estRows task access object operator info Sort_6 1473.49 root Column#5 └─HashAgg_11 1473.49 root group by:Column#10, funcs:sum(Column#9)->Column#5 └─Projection_18 1841.86 root cast(test.t1.c1, decimal(10,0) BINARY)->Column#9, test.t1.c1 - └─IndexMerge_16 1841.86 root + └─IndexMerge_16 1841.86 root type: union ├─IndexRangeScan_12(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_13(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_15(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10) @@ -507,7 +507,7 @@ Sort_12 1841.86 root test.t1.c1 └─Projection_14 1841.86 root test.t1.c1, test.t1.c2, test.t1.c3 └─Apply_16 1841.86 root inner join, equal:[eq(Column#10, Column#9)] ├─Projection_17(Build) 1841.86 root test.t1.c1, test.t1.c2, test.t1.c3, cast(test.t1.c1, decimal(10,0) BINARY)->Column#10 - │ └─IndexMerge_22 1841.86 root + │ └─IndexMerge_22 1841.86 root type: union │ ├─IndexRangeScan_18(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo │ ├─IndexRangeScan_19(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,-1), keep order:false, stats:pseudo │ └─Selection_21(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10) @@ -530,7 +530,7 @@ Sort_16 1841.86 root test.t1.c1 └─Projection_18 1841.86 root test.t1.c1, test.t1.c2, test.t1.c3 └─Apply_20 1841.86 root inner join, equal:[eq(Column#11, Column#9)] ├─Projection_21(Build) 1841.86 root test.t1.c1, test.t1.c2, test.t1.c3, cast(test.t1.c1, decimal(10,0) BINARY)->Column#11 - │ └─IndexMerge_26 1841.86 root + │ └─IndexMerge_26 1841.86 root type: union │ ├─IndexRangeScan_22(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo │ ├─IndexRangeScan_23(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,-1), keep order:false, stats:pseudo │ └─Selection_25(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10) @@ -538,7 +538,7 @@ Sort_16 1841.86 root test.t1.c1 └─TopN_29(Probe) 1841.86 root test.t2.c1, offset:2, count:1 └─HashAgg_36 4900166.23 root group by:Column#21, funcs:avg(Column#19)->Column#9, funcs:firstrow(Column#20)->test.t2.c1 └─Projection_48 6125207.79 root cast(test.t2.c1, decimal(10,0) BINARY)->Column#19, test.t2.c1, test.t2.c1 - └─IndexMerge_41 6125207.79 root + └─IndexMerge_41 6125207.79 root type: union ├─Selection_38(Build) 6121.12 cop[tikv] eq(test.t1.c1, test.t2.c1) │ └─IndexRangeScan_37 6121120.92 cop[tikv] table:t2, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_39(Build) 6121120.92 cop[tikv] table:t2, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo @@ -553,7 +553,7 @@ insert into t1 values(1, 1, 1, 1, 1), (2, 2, 2, 2, 2), (3, 3, 3, 3, 3), (4, 4, 4 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and (c3 < 10 or c4 < 10) order by 1; id estRows task access object operator info Sort_5 3071.61 root test.t1.c1 -└─IndexMerge_12 3071.61 root +└─IndexMerge_12 3071.61 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 3071.61 cop[tikv] or(lt(test.t1.c3, 10), lt(test.t1.c4, 10)) @@ -568,7 +568,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 and c2 < 10) or (c3 < 10 and c4 < 10) order by 1; id estRows task access object operator info Sort_5 2086.93 root test.t1.c1 -└─IndexMerge_12 1156.62 root +└─IndexMerge_12 1156.62 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c3(c3) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 1156.62 cop[tikv] or(and(lt(test.t1.c1, 10), lt(test.t1.c2, 10)), and(lt(test.t1.c3, 10), lt(test.t1.c4, 10))) @@ -583,7 +583,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 and c2 < 10) or (c3 < 10 and c4 < 10) and c5 < 10 order by 1; id estRows task access object operator info Sort_5 1430.96 root test.t1.c1 -└─IndexMerge_12 793.07 root +└─IndexMerge_12 793.07 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c3(c3) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 793.07 cop[tikv] or(and(lt(test.t1.c1, 10), lt(test.t1.c2, 10)), and(lt(test.t1.c3, 10), and(lt(test.t1.c4, 10), lt(test.t1.c5, 10)))) @@ -598,7 +598,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where ((c1 < 10 and c4 < 10) or c2 < 10) and (c3 < 10 or c5 < 10) order by 1; id estRows task access object operator info Sort_5 2250.55 root test.t1.c1 -└─IndexMerge_12 1247.30 root +└─IndexMerge_12 1247.30 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 1247.30 cop[tikv] or(and(lt(test.t1.c1, 10), lt(test.t1.c4, 10)), lt(test.t1.c2, 10)), or(lt(test.t1.c3, 10), lt(test.t1.c5, 10)) @@ -628,7 +628,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (((c1 < 10 or c3 < 10) and c1 < 10) or c2 < 10) and (c3 < 10 or c5 < 10) order by 1; id estRows task access object operator info Sort_5 2523.42 root test.t1.c1 -└─IndexMerge_12 1398.53 root +└─IndexMerge_12 1398.53 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 1398.53 cop[tikv] or(and(or(lt(test.t1.c1, 10), lt(test.t1.c3, 10)), lt(test.t1.c1, 10)), lt(test.t1.c2, 10)), or(lt(test.t1.c3, 10), lt(test.t1.c5, 10)) @@ -645,7 +645,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and coalesce(c1, c2, c4) = 1 order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] eq(coalesce(test.t1.c1, test.t1.c2, test.t1.c4), 1) @@ -657,7 +657,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) a id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 └─Selection_12 4433.77 root eq(greatest(test.t1.c1, test.t1.c2, test.t1.c4), 1) - └─IndexMerge_11 5542.21 root + └─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -668,7 +668,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and abs(c1) = 1 order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] eq(abs(test.t1.c1), 1) @@ -679,7 +679,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and pi() order by 1; id estRows task access object operator info Sort_5 5542.21 root test.t1.c1 -└─IndexMerge_11 5542.21 root +└─IndexMerge_11 5542.21 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -693,7 +693,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and ceil(c1) order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] ceil(test.t1.c1) @@ -709,7 +709,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) a id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 └─Selection_8 4433.77 root eq(truncate(test.t1.c1, 1), 1) - └─IndexMerge_12 5542.21 root + └─IndexMerge_12 5542.21 root type: union ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -725,7 +725,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and substring(c3, 1, 1) = '1' order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] eq(substring(cast(test.t1.c3, var_string(20)), 1, 1), "1") @@ -737,7 +737,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and ifnull(c1, c2) order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] ifnull(test.t1.c1, test.t1.c2) @@ -752,7 +752,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and if(c1, c2, c3) order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root +└─IndexMerge_12 4433.77 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 4433.77 cop[tikv] if(test.t1.c1, test.t1.c2, test.t1.c3) @@ -767,7 +767,7 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and (c1 between 1 and 2) order by 1; id estRows task access object operator info Sort_5 138.56 root test.t1.c1 -└─IndexMerge_12 138.56 root +└─IndexMerge_12 138.56 root type: union ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_11(Probe) 138.56 cop[tikv] ge(test.t1.c1, 1), le(test.t1.c1, 2) @@ -782,7 +782,7 @@ explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) a id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 └─Selection_8 4433.77 root eq(length(substring(cast(sqrt(cast(test.t1.c3, double BINARY)), var_string(5)), getvar("a"), 1)), char_length(cast(if(test.t1.c1, test.t1.c2, test.t1.c3), var_string(20)))) - └─IndexMerge_12 5542.21 root + └─IndexMerge_12 5542.21 root type: union ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -800,7 +800,7 @@ insert into t1 values(1, 1, 1, 1, 1), (2, 2, 2, 2, 2), (3, 3, 3, 3, 3), (4, 4, 4 explain with cte1 as (select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10) select * from cte1 order by 1; id estRows task access object operator info Sort_10 4060.74 root test.t1.c1 -└─IndexMerge_17 2250.55 root +└─IndexMerge_17 2250.55 root type: union ├─IndexRangeScan_13(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_14(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo └─Selection_16(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) @@ -818,7 +818,7 @@ Sort_23 7309.33 root test.t1.c1 └─CTEFullScan_26 7309.33 root CTE:cte1 data:CTE_0 CTE_0 7309.33 root Recursive CTE ├─Projection_14(Seed Part) 4060.74 root test.t1.c1 -│ └─IndexMerge_19 2250.55 root +│ └─IndexMerge_19 2250.55 root type: union │ ├─IndexRangeScan_15(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo │ ├─IndexRangeScan_16(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo │ └─Selection_18(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) diff --git a/executor/builder.go b/executor/builder.go index 1877bc8aae7d1..8391964095877 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3969,6 +3969,7 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd isCorColInPartialFilters: isCorColInPartialFilters, isCorColInTableFilter: isCorColInTableFilter, isCorColInPartialAccess: isCorColInPartialAccess, + isIntersection: v.IsIntersectionType, } collectTable := false e.tableRequest.CollectRangeCounts = &collectTable diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 58dfa71814f28..4d719adbdf781 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -86,7 +86,7 @@ func TestIndexMergeReaderIssue25045(t *testing.T) { tk.MustExec("create table t1(a int primary key, b int, c int, key(b), key(c));") tk.MustExec("INSERT INTO t1 VALUES (10, 10, 10), (11, 11, 11)") tk.MustQuery("explain format='brief' select /*+ use_index_merge(t1) */ * from t1 where c=10 or (b=10 and a=10);").Check(testkit.Rows( - "IndexMerge 0.01 root ", + "IndexMerge 0.01 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c(c) range:[10,10], keep order:false, stats:pseudo", "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", "└─Selection(Probe) 0.01 cop[tikv] or(eq(test.t1.c, 10), and(eq(test.t1.b, 10), eq(test.t1.a, 10)))", @@ -230,14 +230,14 @@ func TestIndexMergeInTransaction(t *testing.T) { tk.MustExec("begin;") // Expect two IndexScan(c1, c2). tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows( - "IndexMerge_9 1841.86 root ", + "IndexMerge_9 1841.86 root type: union", "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", "└─Selection_8(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10)", " └─TableRowIDScan_7 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo")) // Expect one IndexScan(c2) and one TableScan(pk). tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows( - "IndexMerge_9 1106.67 root ", + "IndexMerge_9 1106.67 root type: union", "├─TableRangeScan_5(Build) 3333.33 cop[tikv] table:t1 range:[-inf,10), keep order:false, stats:pseudo", "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", "└─Selection_8(Probe) 1106.67 cop[tikv] lt(test.t1.c3, 10)", @@ -281,14 +281,14 @@ func TestIndexMergeInTransaction(t *testing.T) { tk.MustExec("begin;") tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows( "SelectLock_6 1841.86 root for update 0", - "└─IndexMerge_11 1841.86 root ", + "└─IndexMerge_11 1841.86 root type: union", " ├─IndexRangeScan_7(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", " └─Selection_10(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10)", " └─TableRowIDScan_9 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo")) tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows( "SelectLock_6 1106.67 root for update 0", - "└─IndexMerge_11 1106.67 root ", + "└─IndexMerge_11 1106.67 root type: union", " ├─TableRangeScan_7(Build) 3333.33 cop[tikv] table:t1 range:[-inf,10), keep order:false, stats:pseudo", " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", " └─Selection_10(Probe) 1106.67 cop[tikv] lt(test.t1.c3, 10)", @@ -403,7 +403,7 @@ func TestIndexMergeReaderInTransIssue30685(t *testing.T) { tk.MustExec("insert into t1 values(1, 1, 1, 1);") tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < -1 or c3 < 10) and c4 < 10;").Check(testkit.Rows( "UnionScan_6 1841.86 root lt(test.t1.c4, 10), or(lt(test.t1.c1, -1), lt(test.t1.c3, 10))", - "└─IndexMerge_11 1841.86 root ", + "└─IndexMerge_11 1841.86 root type: union", " ├─TableRangeScan_7(Build) 3323.33 cop[tikv] table:t1 range:[-inf,-1), keep order:false, stats:pseudo", " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c3(c3) range:[-inf,10), keep order:false, stats:pseudo", " └─Selection_10(Probe) 1841.86 cop[tikv] lt(test.t1.c4, 10)", @@ -422,7 +422,7 @@ func TestIndexMergeReaderInTransIssue30685(t *testing.T) { tk.MustExec("insert into t1 values('b', 1, 1, 1);") tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 'a' or c3 < 10) and c4 < 10;").Check(testkit.Rows( "UnionScan_6 1841.86 root lt(test.t1.c4, 10), or(lt(test.t1.c1, \"a\"), lt(test.t1.c3, 10))", - "└─IndexMerge_11 1841.86 root ", + "└─IndexMerge_11 1841.86 root type: union", " ├─TableRangeScan_7(Build) 3323.33 cop[tikv] table:t1 range:[-inf,\"a\"), keep order:false, stats:pseudo", " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c3(c3) range:[-inf,10), keep order:false, stats:pseudo", " └─Selection_10(Probe) 1841.86 cop[tikv] lt(test.t1.c4, 10)", @@ -524,19 +524,19 @@ func TestPessimisticLockOnPartitionForIndexMerge(t *testing.T) { " ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:c_datetime(c_datetime) keep order:false", " └─PartitionUnion(Probe) 5545.21 root ", - " ├─IndexMerge 5542.21 root ", + " ├─IndexMerge 5542.21 root type: union", " │ ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, partition:p0, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", " │ ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, partition:p0, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", " │ └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", - " ├─IndexMerge 1.00 root ", + " ├─IndexMerge 1.00 root type: union", " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:c1(c1) range:[-inf,10), keep order:false", " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:c2(c2) range:[-inf,10), keep order:false", " │ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - " ├─IndexMerge 1.00 root ", + " ├─IndexMerge 1.00 root type: union", " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p2, index:c1(c1) range:[-inf,10), keep order:false", " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p2, index:c2(c2) range:[-inf,10), keep order:false", " │ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p2 keep order:false", - " └─IndexMerge 1.00 root ", + " └─IndexMerge 1.00 root type: union", " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p3, index:c1(c1) range:[-inf,10), keep order:false", " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p3, index:c2(c2) range:[-inf,10), keep order:false", " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p3 keep order:false", diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 214148fbf198a..903d9895cc654 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -168,6 +168,7 @@ go_test( "find_best_task_test.go", "flat_plan_test.go", "fragment_test.go", + "indexmerge_intersection_test.go", "indexmerge_test.go", "integration_partition_test.go", "integration_test.go", diff --git a/planner/core/explain.go b/planner/core/explain.go index 3f4c37e60d131..16140495de3e7 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -285,8 +285,11 @@ func (p *PhysicalIndexLookUpReader) ExplainInfo() string { } // ExplainInfo implements Plan interface. -func (*PhysicalIndexMergeReader) ExplainInfo() string { - return "" +func (p *PhysicalIndexMergeReader) ExplainInfo() string { + if p.IsIntersectionType { + return "type: intersection" + } + return "type: union" } // ExplainInfo implements Plan interface. diff --git a/planner/core/indexmerge_intersection_test.go b/planner/core/indexmerge_intersection_test.go new file mode 100644 index 0000000000000..8b352f3b5cead --- /dev/null +++ b/planner/core/indexmerge_intersection_test.go @@ -0,0 +1,178 @@ +// 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_test + +import ( + "regexp" + "testing" + + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func TestSPMForIntersectionIndexMerge(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(a int, b int, c int, d int, e int, index ia(a), index ib(b), index ic(c), index id(d), index ie(e))") + require.False(t, tk.HasPlan("select * from t where a = 10 and b = 20 and c > 30 and d is null and e in (0, 100)", "IndexMerge")) + require.True(t, + tk.HasPlan("select /*+ use_index_merge(t, ia, ib, ic, id, ie) */ * from t where a = 10 and b = 20 and c > 30 and d is null and e in (0, 100)", + "IndexMerge", + ), + ) + tk.MustExec(` +create global binding for + select * from t where a = 10 and b = 20 and c > 30 and d is null and e in (0, 100) +using + select /*+ use_index_merge(t, ia, ib, ic, id, ie) */ * from t where a = 10 and b = 20 and c > 30 and d is null and e in (0, 100) +`) + require.True(t, tk.HasPlan("select * from t where a = 10 and b = 20 and c > 30 and d is null and e in (0, 100)", "IndexMerge")) +} + +func TestPlanCacheForIntersectionIndexMerge(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(a int, b int, c int, d int, e int, index ia(a), index ib(b), index ic(c), index id(d), index ie(e))") + tk.MustExec("prepare stmt from 'select /*+ use_index_merge(t, ia, ib, ic, id, ie) */ * from t where a = 10 and b = ? and c > ? and d is null and e in (0, 100)'") + tk.MustExec("set @a=1, @b=3") + tk.MustQuery("execute stmt using @a,@b").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt using @a,@b").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("set @a=100, @b=500") + tk.MustQuery("execute stmt using @a,@b").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @a,@b").Check(testkit.Rows()) + require.True(t, tk.HasPlanForLastExecution("IndexMerge")) +} + +func TestHintForIntersectionIndexMerge(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomain(t) + handle := domain.StatsHandle() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t1(a int, b int, c int, d int, e int, index ia(a), index ibc(b, c),index ic(c), index id(d), index ie(e))" + + "partition by range(c) (" + + "partition p0 values less than (10)," + + "partition p1 values less than (20)," + + "partition p2 values less than (30)," + + "partition p3 values less than (maxvalue))") + tk.MustExec("insert into t1 values (10, 20, 5, 5, 3), (20, 20, 50, 5, 200), (20, 20, 10, 5, 5), (10, 30, 5, 3, 1)") + tk.MustExec("create definer='root'@'localhost' view vh as " + + "select /*+ use_index_merge(t1, ia, ibc, id) */ * from t1 where a = 10 and b = 20 and c < 30 and d in (2,5)") + tk.MustExec("create definer='root'@'localhost' view v as " + + "select * from t1 where a = 10 and b = 20 and c < 30 and d in (2,5)") + tk.MustExec("create definer='root'@'localhost' view v1 as " + + "select * from t1 where a = 10 and b = 20") + tk.MustExec("create table t2(a int, b int, c int, d int, e int, index ia(a), index ibc(b, c), index id(d), index ie(e))" + + "partition by range columns (c, d) (" + + "partition p0 values less than (10, 20)," + + "partition p1 values less than (30, 40)," + + "partition p2 values less than (50, 60)," + + "partition p3 values less than (maxvalue, maxvalue))") + tk.MustExec("insert into t2 values (10, 20, 5, 5, 3), (20, 20, 20, 5, 100), (100, 30, 5, 3, 100)") + tk.MustExec("create table t3(a int, b int, c int, d int, e int, index ia(a), index ibc(b, c), index id(d), index ie(e))" + + "partition by hash (e) partitions 5") + tk.MustExec("insert into t3 values (10, 20, 5, 5, 3), (20, 20, 20, 5, 100), (10, 30, 5, 3, 100)") + tk.MustExec("create table t4(a int, b int, c int, d int, e int, index ia(a), index ibc(b, c), index id(d), index ie(e))" + + "partition by list (d) (" + + "partition p0 values in (1,2,3,4,5)," + + "partition p1 values in (6,7,8,9,10)," + + "partition p2 values in (11,12,13,14,15)," + + "partition p3 values in (16,17,18,19,20))") + tk.MustExec("insert into t4 values (30, 20, 5, 8, 100), (20, 20, 20, 3, 2), (10, 30, 5, 3, 100)") + tk.MustExec("create table t5(" + + "s1 varchar(20) collate utf8mb4_bin," + + "s2 varchar(30) collate ascii_bin," + + "s3 varchar(50) collate utf8_unicode_ci," + + "s4 varchar(20) collate gbk_chinese_ci," + + "index is1(s1), index is2(s2), index is3(s3), index is4(s4))") + tk.MustExec("insert into t5 values ('Abc', 'zzzz', 'aa', 'ccc'), ('abc', 'zzzz', 'CCC', 'ccc')") + tk.MustExec("create table t6(" + + "s1 varchar(20) collate utf8mb4_bin," + + "s2 varchar(30) collate ascii_bin," + + "s3 varchar(50) collate utf8_unicode_ci," + + "s4 varchar(20) collate gbk_chinese_ci," + + "primary key (s1, s2(10)) nonclustered," + + "index is1(s1), index is2(s2), index is3(s3), index is4(s4))") + tk.MustExec("insert into t6 values ('Abc', 'zzzz', 'A啊A', 'Cdaa'), ('Abc', 'zczz', 'A啊', 'Cda')") + tk.MustExec("create table t7(" + + "a tinyint unsigned," + + "b bit(3)," + + "c float," + + "d decimal(10,3)," + + "e datetime," + + "f timestamp(5)," + + "g year," + + "primary key (d) nonclustered," + + "index ia(a), unique index ib(b), index ic(c), index ie(e), index iff(f), index ig(g))") + tk.MustExec("insert into t7 values (100, 6, 12.2, 56, '2022-11-22 17:00', '2022-12-21 00:00', 2021)," + + "(20, 7, 12.4, 30, '2022-12-22 17:00', '2016-12-21 00:00', 2021)") + tk.MustExec("create table t8(" + + "s1 mediumtext collate utf8mb4_general_ci," + + "s2 varbinary(20)," + + "s3 tinyblob," + + "s4 enum('测试', 'aA', '??') collate gbk_chinese_ci," + + "s5 set('^^^', 'tEsT', '2') collate utf8_general_ci," + + "primary key (s1(10)) nonclustered," + + "unique index is2(s2(20)), index is3(s3(20)), index is4(s4), index is5(s5))") + tk.MustExec("insert into t8 values('啊aabbccdd', 'abcc', 'cccc', 'aa', '2,test')," + + "('啊aabb', 'abcdc', 'aaaa', '??', '2')") + + require.NoError(t, handle.HandleDDLEvent(<-handle.DDLEventCh())) + require.Nil(t, handle.Update(domain.InfoSchema())) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("analyze table t1,t2,t3,t4") + require.Nil(t, handle.Update(domain.InfoSchema())) + + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := core.GetIndexMergeSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + matchSetStmt, err := regexp.Compile("^set") + require.NoError(t, err) + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + }) + ok := matchSetStmt.MatchString(ts) + if ok { + tk.MustExec(ts) + continue + } + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + // Expect no warnings. + tk.MustQuery("show warnings").Check(testkit.Rows()) + } +} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index e03a2052d8ada..cdd8b07ca3c6a 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -2351,7 +2351,7 @@ func TestIssue16837(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b int,c int,d int,e int,unique key idx_ab(a,b),unique key(c),unique key(d))") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows( - "IndexMerge 0.01 root ", + "IndexMerge 0.01 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_ab(a, b) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:c(c) range:[1,1], keep order:false, stats:pseudo", "└─Selection(Probe) 0.01 cop[tikv] or(eq(test.t.a, 1), and(eq(test.t.e, 1), eq(test.t.c, 1)))", @@ -2592,7 +2592,7 @@ func TestIssue16407(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b char(100),key(a),key(b(10)))") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a=10 or b='x'").Check(testkit.Rows( - "IndexMerge 0.04 root ", + "IndexMerge 0.04 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[\"x\",\"x\"], keep order:false, stats:pseudo", "└─Selection(Probe) 0.04 cop[tikv] or(eq(test.t.a, 10), eq(test.t.b, \"x\"))", @@ -5040,7 +5040,7 @@ func TestIndexMergeTableFilter(t *testing.T) { tk.MustExec("insert into t values(10,1,1,10)") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a=10 or (b=10 and c=10)").Check(testkit.Rows( - "IndexMerge 0.02 root ", + "IndexMerge 0.02 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", "└─Selection(Probe) 0.02 cop[tikv] or(eq(test.t.a, 10), and(eq(test.t.b, 10), eq(test.t.c, 10)))", @@ -5050,7 +5050,7 @@ func TestIndexMergeTableFilter(t *testing.T) { "10 1 1 10", )) tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where (a=10 and d=10) or (b=10 and c=10)").Check(testkit.Rows( - "IndexMerge 0.00 root ", + "IndexMerge 0.00 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", "└─Selection(Probe) 0.00 cop[tikv] or(and(eq(test.t.a, 10), eq(test.t.d, 10)), and(eq(test.t.b, 10), eq(test.t.c, 10)))", @@ -5701,36 +5701,36 @@ func TestIssue29221(t *testing.T) { tk.MustExec("set @@session.sql_select_limit=3;") tk.MustQuery("explain format = 'brief' select * from t where a = 1 or b = 1;").Check(testkit.Rows( "Limit 3.00 root offset:0, count:3", - "└─IndexMerge 3.00 root ", + "└─IndexMerge 3.00 root type: union", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a = 1 or b = 1;").Check(testkit.Rows( "Limit 3.00 root offset:0, count:3", - "└─IndexMerge 3.00 root ", + "└─IndexMerge 3.00 root type: union", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustExec("set @@session.sql_select_limit=18446744073709551615;") tk.MustQuery("explain format = 'brief' select * from t where a = 1 or b = 1;").Check(testkit.Rows( - "IndexMerge 19.99 root ", + "IndexMerge 19.99 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 19.99 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("explain format = 'brief' select * from t where a = 1 or b = 1 limit 3;").Check(testkit.Rows( "Limit 3.00 root offset:0, count:3", - "└─IndexMerge 3.00 root ", + "└─IndexMerge 3.00 root type: union", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a = 1 or b = 1;").Check(testkit.Rows( - "IndexMerge 19.99 root ", + "IndexMerge 19.99 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 19.99 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a = 1 or b = 1 limit 3;").Check(testkit.Rows( "Limit 3.00 root offset:0, count:3", - "└─IndexMerge 3.00 root ", + "└─IndexMerge 3.00 root type: union", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo")) diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 8bd3264684fb7..26d0497cdeddd 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -130,3 +130,7 @@ func GetFlatPlanSuiteData() testdata.TestData { func GetBinaryPlanSuiteData() testdata.TestData { return testDataMap["binary_plan_suite"] } + +func GetIndexMergeSuiteData() testdata.TestData { + return testDataMap["index_merge_suite"] +} diff --git a/planner/core/stats.go b/planner/core/stats.go index 6fe03f87d6a80..f377feac91030 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -467,10 +467,19 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, func (ds *DataSource) generateAndPruneIndexMergePath(indexMergeConds []expression.Expression, needPrune bool) error { regularPathCount := len(ds.possibleAccessPaths) + // 1. Generate possible IndexMerge paths for `OR`. err := ds.generateIndexMergeOrPaths(indexMergeConds) if err != nil { return err } + // 2. Generate possible IndexMerge paths for `AND`. + indexMergeAndPath := ds.generateIndexMergeAndPaths(regularPathCount) + if indexMergeAndPath != nil { + ds.possibleAccessPaths = append(ds.possibleAccessPaths, indexMergeAndPath) + } + + // 3. If needed, append a warning if no IndexMerge is generated. + // If without hints, it means that `enableIndexMerge` is true if len(ds.indexMergeHints) == 0 { return nil @@ -481,6 +490,9 @@ func (ds *DataSource) generateAndPruneIndexMergePath(indexMergeConds []expressio ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable")) return nil } + + // 4. If needPrune is true, prune non-IndexMerge paths. + // Do not need to consider the regular paths in find_best_task(). // So we can use index merge's row count as DataSource's row count. if needPrune { @@ -615,7 +627,9 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) return nil } -// isInIndexMergeHints checks whether current index or primary key is in IndexMerge hints. +// isInIndexMergeHints returns true if the input index name is not excluded by the IndexMerge hints, which means either +// (1) there's no IndexMerge hint, (2) there's IndexMerge hint but no specified index names, or (3) the input index +// name is specified in the IndexMerge hints. func (ds *DataSource) isInIndexMergeHints(name string) bool { if len(ds.indexMergeHints) == 0 { return true @@ -633,6 +647,34 @@ func (ds *DataSource) isInIndexMergeHints(name string) bool { return false } +// indexMergeHintsHasSpecifiedIdx returns true if there's IndexMerge hint, and it has specified index names. +func (ds *DataSource) indexMergeHintsHasSpecifiedIdx() bool { + for _, hint := range ds.indexMergeHints { + if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { + continue + } + if len(hint.indexHint.IndexNames) > 0 { + return true + } + } + return false +} + +// indexMergeHintsHasSpecifiedIdx return true if the input index name is specified in the IndexMerge hint. +func (ds *DataSource) isSpecifiedInIndexMergeHints(name string) bool { + for _, hint := range ds.indexMergeHints { + if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { + continue + } + for _, hintName := range hint.indexHint.IndexNames { + if strings.EqualFold(strings.ToLower(name), strings.ToLower(hintName.String())) { + return true + } + } + } + return false +} + // accessPathsForConds generates all possible index paths for conditions. func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, usedIndexCount int) []*util.AccessPath { var results = make([]*util.AccessPath, 0, usedIndexCount) @@ -754,6 +796,103 @@ func (ds *DataSource) buildIndexMergeOrPath(filters []expression.Expression, par return indexMergePath } +// generateIndexMergeAndPaths generates IndexMerge paths for `AND` (a.k.a. intersection type IndexMerge) +func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.AccessPath { + // For now, we only consider intersection type IndexMerge when the index names are specified in the hints. + if !ds.indexMergeHintsHasSpecifiedIdx() { + return nil + } + + // 1. Collect partial paths from normal paths. + var partialPaths []*util.AccessPath + for i := 0; i < normalPathCnt; i++ { + originalPath := ds.possibleAccessPaths[i] + // No need to consider table path as a partial path. + if ds.possibleAccessPaths[i].IsTablePath() { + continue + } + if !ds.isSpecifiedInIndexMergeHints(originalPath.Index.Name.L) { + continue + } + // If the path contains a full range, ignore it. + if ranger.HasFullRange(originalPath.Ranges, false) { + continue + } + newPath := originalPath.Clone() + partialPaths = append(partialPaths, newPath) + } + if len(partialPaths) < 2 { + return nil + } + + // 2. Collect filters that can't be covered by the partial paths and deduplicate them. + finalFilters := make([]expression.Expression, 0) + partialFilters := make([]expression.Expression, 0, len(partialPaths)) + hashCodeSet := make(map[string]struct{}) + for _, path := range partialPaths { + // Classify filters into coveredConds and notCoveredConds. + coveredConds := make([]expression.Expression, 0, len(path.AccessConds)+len(path.IndexFilters)) + notCoveredConds := make([]expression.Expression, 0, len(path.IndexFilters)+len(path.TableFilters)) + // AccessConds can be covered by partial path. + coveredConds = append(coveredConds, path.AccessConds...) + for i, cond := range path.IndexFilters { + // IndexFilters can be covered by partial path if it can be pushed down to TiKV. + if !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, []expression.Expression{cond}, ds.ctx.GetClient(), kv.TiKV) { + path.IndexFilters = append(path.IndexFilters[:i], path.IndexFilters[i+1:]...) + notCoveredConds = append(notCoveredConds, cond) + } else { + coveredConds = append(coveredConds, cond) + } + } + // TableFilters can't be covered by partial path. + notCoveredConds = append(notCoveredConds, path.TableFilters...) + + // Record covered filters in hashCodeSet. + // Note that we only record filters that not appear in the notCoveredConds. It's possible that a filter appear + // in both coveredConds and notCoveredConds (e.g. because of prefix index). So we need this extra check to + // avoid wrong deduplication. + notCoveredHashCodeSet := make(map[string]struct{}) + for _, cond := range notCoveredConds { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + notCoveredHashCodeSet[hashCode] = struct{}{} + } + for _, cond := range coveredConds { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + if _, ok := notCoveredHashCodeSet[hashCode]; !ok { + hashCodeSet[hashCode] = struct{}{} + } + } + + finalFilters = append(finalFilters, notCoveredConds...) + partialFilters = append(partialFilters, coveredConds...) + } + + // Remove covered filters from finalFilters and deduplicate finalFilters. + dedupedFinalFilters := make([]expression.Expression, 0, len(finalFilters)) + for _, cond := range finalFilters { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + if _, ok := hashCodeSet[hashCode]; !ok { + dedupedFinalFilters = append(dedupedFinalFilters, cond) + hashCodeSet[hashCode] = struct{}{} + } + } + + // 3. Estimate the row count after partial paths. + sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, partialFilters, nil) + if err != nil { + logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) + sel = SelectionFactor + } + + indexMergePath := &util.AccessPath{ + PartialIndexPaths: partialPaths, + IndexMergeIsIntersection: true, + TableFilters: dedupedFinalFilters, + CountAfterAccess: sel * ds.tableStats.RowCount, + } + return indexMergePath +} + // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { if p.stats != nil { diff --git a/planner/core/testdata/index_merge_suite_in.json b/planner/core/testdata/index_merge_suite_in.json index 55c9c1cca1f7c..db7ebacdb29c7 100644 --- a/planner/core/testdata/index_merge_suite_in.json +++ b/planner/core/testdata/index_merge_suite_in.json @@ -9,5 +9,35 @@ "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (c < 1 or g > 2)", "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (e < 1 or f > 2)" ] + }, + { + "name": "TestHintForIntersectionIndexMerge", + "cases": [ + "set @@tidb_partition_prune_mode = 'dynamic'", + "select * from vh", + "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", + "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", + "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", + "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", + "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", + "set @@tidb_partition_prune_mode = 'static'", + "select * from vh", + "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100" + ] } -] \ No newline at end of file +] diff --git a/planner/core/testdata/index_merge_suite_out.json b/planner/core/testdata/index_merge_suite_out.json index 4fdd67789e205..3d67e5e372251 100644 --- a/planner/core/testdata/index_merge_suite_out.json +++ b/planner/core/testdata/index_merge_suite_out.json @@ -9,5 +9,427 @@ "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,g],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(gt(test.t.c, 5), lt(test.t.f, 7))]}]", "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.e, 1), gt(test.t.f, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.e, 1), gt(test.t.f, 2))]}]" ] + }, + { + "Name": "TestHintForIntersectionIndexMerge", + "Cases": [ + { + "SQL": "set @@tidb_partition_prune_mode = 'dynamic'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t2 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.44 root partition:p0 type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t3, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.44 cop[tikv] table:t3 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t4, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t4 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is1(s1) range:[\"Abc\",\"Abc\"], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:is2(s2) range:(\"zzz\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:is3(s3) range:[-inf,\"B啊a\"), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is4(s4) range:[\"CcC\",\"CcC\"], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz aa ccc" + ] + }, + { + "SQL": "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", + "Plan": [ + "IndexMerge 0.03 root type: intersection", + "├─IndexRangeScan(Build) 33.33 cop[tikv] table:t6, index:PRIMARY(s1, s2) range:(\"Abc\" \"zzz\",\"Abc\" +inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t6, index:is3(s3) range:[\"A啊a\",\"A啊a\"], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.03 cop[tikv] gt(test.t6.s2, \"zzz\"), not(like(test.t6.s4, \"Cd_\", 92))", + " └─TableRowIDScan 0.03 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz A啊A Cdaa" + ] + }, + { + "SQL": "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:PRIMARY(d) range:(54.321,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ia(a) range:[100,100], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:ib(b) range:(\"0x05\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ic(c) range:[-inf,12.3), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ie(e) range:[2022-11-22 17:00:00,2022-11-22 17:00:00], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:iff(f) range:(2020-06-23 10:00:00.00000,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ig(g) range:[-inf,2025), keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t7 keep order:false, stats:pseudo" + ], + "Result": [ + "100 \u0006 12.2 56.000 2022-11-22 17:00:00 2022-12-21 00:00:00.00000 2021" + ] + }, + { + "SQL": "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", + "Plan": [ + "Selection 1.42 root eq(test.t8.s5, \"test,2\")", + "└─IndexMerge 0.59 root type: intersection", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(0x616263,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(0x636261,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t8, index:is4(s4) range:[\"aA\",\"aA\"], [\"??\",\"??\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.59 cop[tikv] gt(test.t8.s3, \"cba\"), like(test.t8.s1, \"啊A%\", 92)", + " └─TableRowIDScan 2.22 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": [ + "啊aabbccdd abcc cccc aA tEsT,2" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 1.50 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t1, index:ibc(b, c) range: decided by [eq(test.t1.b, 20) eq(test.t1.c, test.t2.a)], keep order:false", + " └─TableRowIDScan(Probe) 1.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─Selection(Build) 7.20 cop[tikv] gt(test.t1.c, test.t2.a)", + " │ └─IndexRangeScan 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─TableRowIDScan(Probe) 3.60 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─Selection(Probe) 3.60 cop[tikv] gt(test.t1.e, test.t2.a)", + " └─TableRowIDScan 4.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "20" + ] + }, + { + "SQL": "set @@tidb_partition_prune_mode = 'static'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t2, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t2, partition:p2, index:ia(a) range:(10,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t2, partition:p2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t2, partition:p2, index:id(d) range:[-inf,45), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, partition:p2, index:ie(e) range:[100,100], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.50 root type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.50 cop[tikv] table:t3, partition:p0 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, partition:p0, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t4, partition:p0 keep order:false", + "└─IndexMerge 1.00 root type: intersection", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ie(e) range:[100,100], keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t4, partition:p1 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + } + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 607f6d2947787..5527622476829 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -339,7 +339,7 @@ { "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", "Plan": [ - "IndexMerge 2.00 root ", + "IndexMerge 2.00 root type: union", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -348,7 +348,7 @@ { "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", "Plan": [ - "IndexMerge 2.00 root ", + "IndexMerge 2.00 root type: union", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -386,7 +386,7 @@ { "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, c) */ * from t where b = 1 and (a = 1 or c = 1)", "Plan": [ - "IndexMerge 0.02 root ", + "IndexMerge 0.02 root type: union", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:c(c) range:[1,1], keep order:false, stats:pseudo", "└─Selection(Probe) 0.02 cop[tikv] eq(test.t.b, 1)", @@ -942,7 +942,7 @@ "SQL": "select /*+ use_index_merge(t partition(p0)) */ * from t where t.b = 1 or t.c = \"8\"", "Plan": [ "PartitionUnion 59.97 root ", - "├─IndexMerge 19.99 root ", + "├─IndexMerge 19.99 root type: union", "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, partition:p0, index:b(b) range:[1,1], keep order:false, stats:pseudo", "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, partition:p0, index:c(c) range:[\"8\",\"8\"], keep order:false, stats:pseudo", "│ └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", @@ -959,11 +959,11 @@ "SQL": "select /*+ use_index_merge(t partition(p0, p1) primary, b) */ * from t where t.a = 1 or t.b = 2", "Plan": [ "PartitionUnion 33.00 root ", - "├─IndexMerge 11.00 root ", + "├─IndexMerge 11.00 root type: union", "│ ├─TableRangeScan(Build) 1.00 cop[tikv] table:t, partition:p0 range:[1,1], keep order:false, stats:pseudo", "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, partition:p0, index:b(b) range:[2,2], keep order:false, stats:pseudo", "│ └─TableRowIDScan(Probe) 11.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", - "├─IndexMerge 11.00 root ", + "├─IndexMerge 11.00 root type: union", "│ ├─TableRangeScan(Build) 1.00 cop[tikv] table:t, partition:p1 range:[1,1], keep order:false, stats:pseudo", "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, partition:p1, index:b(b) range:[2,2], keep order:false, stats:pseudo", "│ └─TableRowIDScan(Probe) 11.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", @@ -1094,7 +1094,7 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", "Plan": [ - "IndexMerge 3.00 root ", + "IndexMerge 3.00 root type: union", "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" @@ -1108,7 +1108,7 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", "Plan": [ - "IndexMerge 1.67 root ", + "IndexMerge 1.67 root type: union", "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" @@ -1338,7 +1338,7 @@ { "SQL": "select * from pt where id = 4 or c < 7", "Plan": [ - "IndexMerge_11 3330.01 root partition:all ", + "IndexMerge_11 3330.01 root partition:all type: union", "├─IndexRangeScan_8(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo", "├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo", "└─TableRowIDScan_10(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo" @@ -1347,7 +1347,7 @@ { "SQL": "select * from pt where id > 4 or c = 7", "Plan": [ - "IndexMerge_11 3340.00 root partition:all ", + "IndexMerge_11 3340.00 root partition:all type: union", "├─IndexRangeScan_8(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo", "├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo", "└─TableRowIDScan_10(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo" @@ -1487,7 +1487,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(test.t1.key1)->Column#10", - "└─IndexMerge 0.02 root ", + "└─IndexMerge 0.02 root type: union", " ├─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", " └─Selection(Probe) 0.02 cop[tikv] or(and(eq(test.t1.key4, 42), not(isnull(test.t1.key6))), and(eq(test.t1.key1, 4), eq(test.t1.key3, 6)))", @@ -2211,7 +2211,7 @@ "Plan": [ "Projection 15.99 root 1->Column#5", "└─Selection 15.99 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", - " └─IndexMerge 19.99 root ", + " └─IndexMerge 19.99 root type: union", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" @@ -2225,7 +2225,7 @@ "Plan": [ "Projection 17.99 root 1->Column#5", "└─Selection 0.04 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.t1.c1, 10)), 10)))", - " └─IndexMerge 19.99 root ", + " └─IndexMerge 19.99 root type: union", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"ab\",\"ab\"], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" @@ -2239,7 +2239,7 @@ "Plan": [ "Projection 15.99 root 1->Column#6", "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.tt1.c3)), \"10\")))", - " └─IndexMerge 19.99 root ", + " └─IndexMerge 19.99 root type: union", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo" @@ -2253,7 +2253,7 @@ "Plan": [ "Projection 2.40 root 1->Column#3", "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))))", - " └─IndexMerge 3.00 root ", + " └─IndexMerge 3.00 root type: union", " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", " ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" @@ -2267,7 +2267,7 @@ "Plan": [ "Projection 5098.44 root 1->Column#5", "└─Selection 2825.66 root or(lt(test.tt3.c1, -10), and(lt(test.tt3.c2, 10), eq(reverse(cast(test.tt3.c3, var_string(20))), \"2\")))", - " └─IndexMerge 5542.21 root ", + " └─IndexMerge 5542.21 root type: union", " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo", " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", " └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:tt3 keep order:false, stats:pseudo" @@ -2302,7 +2302,7 @@ " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root ", + " └─IndexMerge 63.35 root type: union", " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", @@ -2324,7 +2324,7 @@ " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root ", + " └─IndexMerge 63.35 root type: union", " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", @@ -2345,7 +2345,7 @@ " ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 30263.46 root ", + " └─IndexMerge 30263.46 root type: union", " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", @@ -2367,7 +2367,7 @@ " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", " └─StreamAgg(Probe) 10000.00 root funcs:min(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", " └─Projection 11.05 root test.tt2.c_decimal, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 11.05 root ", + " └─IndexMerge 11.05 root type: union", " ├─Selection(Build) 10.00 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", " │ └─IndexRangeScan 10000.00 cop[tikv] table:tt2, index:c_decimal(c_decimal) range:[9.060000,9.060000], keep order:false, stats:pseudo", " ├─Selection(Build) 33233.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", @@ -2390,7 +2390,7 @@ " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", " └─Projection 17.91 root test.tt2.c_decimal, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 17.91 root ", + " └─IndexMerge 17.91 root type: union", " ├─Selection(Build) 10000.00 cop[tikv] lt(7, test.tt1.c_decimal)", " │ └─TableRangeScan 10000.00 cop[tikv] table:tt2 range:[7,7], keep order:false, stats:pseudo", " ├─Selection(Build) 33333.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", @@ -6066,7 +6066,7 @@ { "SQL": "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(t5@qb_v4, idx_a, idx_b) */ * from v4;", "Plan": [ - "IndexMerge 5548.89 root ", + "IndexMerge 5548.89 root type: union", "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 5548.89 cop[tikv] table:t5 keep order:false, stats:pseudo" @@ -6076,7 +6076,7 @@ { "SQL": "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(@qb_v4 t5, idx_b, idx_a) */ * from v4;", "Plan": [ - "IndexMerge 5548.89 root ", + "IndexMerge 5548.89 root type: union", "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", "└─TableRowIDScan(Probe) 5548.89 cop[tikv] table:t5 keep order:false, stats:pseudo" @@ -8160,7 +8160,7 @@ { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", "Plan": [ - "IndexMerge 8.00 root ", + "IndexMerge 8.00 root type: union", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", "├─Selection(Build) 1.00 cop[tikv] 1", "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", @@ -8171,7 +8171,7 @@ { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", "Plan": [ - "IndexMerge 8.00 root ", + "IndexMerge 8.00 root type: union", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", "├─Selection(Build) 1.00 cop[tikv] 1", "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", @@ -8182,7 +8182,7 @@ { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", "Plan": [ - "IndexMerge 8.00 root ", + "IndexMerge 8.00 root type: union", "├─Selection(Build) 1.00 cop[tikv] 1", "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", "├─Selection(Build) 1.00 cop[tikv] 1", @@ -8194,7 +8194,7 @@ { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", "Plan": [ - "IndexMerge 0.29 root ", + "IndexMerge 0.29 root type: union", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", diff --git a/testkit/testkit.go b/testkit/testkit.go index 25479bc862d96..50ba5c24aaec1 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -231,6 +231,18 @@ func (tk *TestKit) HasPlan(sql string, plan string, args ...interface{}) bool { return false } +// HasPlanForLastExecution checks if the execution plan of the last execution contains specific plan. +func (tk *TestKit) HasPlanForLastExecution(plan string) bool { + connID := tk.session.GetSessionVars().ConnectionID + rs := tk.MustQuery(fmt.Sprintf("explain for connection %d", connID)) + for i := range rs.rows { + if strings.Contains(rs.rows[i][0], plan) { + return true + } + } + return false +} + // HasKeywordInOperatorInfo checks if the result execution plan contains specific keyword in the operator info. func (tk *TestKit) HasKeywordInOperatorInfo(sql string, keyword string, args ...interface{}) bool { rs := tk.MustQuery("explain "+sql, args...) From 06072d64a8d70c9b51a50b36f8609fd8e3236772 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 30 Nov 2022 11:02:00 +0800 Subject: [PATCH 08/13] Explain: change the explain analyze output info (#39444) ref pingcap/tidb#39273 --- util/execdetails/execdetails.go | 2 +- util/execdetails/execdetails_test.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index e3652b2c49aec..aef6ecc88b18f 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -549,7 +549,7 @@ func (context *TiFlashScanContext) Clone() TiFlashScanContext { } } 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) + return fmt.Sprintf("tiflash_scan:{dtfile:{total_scanned_packs:%d, total_skipped_packs:%d, total_scanned_rows:%d, total_skipped_rows:%d, total_rs_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 diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index a7c17ca43db87..88e194637e94d 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -225,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}, 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()) + require.Equal(t, "tiflash_task:{proc max:2ns, min:1ns, avg: 1ns, p80:2ns, p95:2ns, iters:3, tasks:2, threads:2}, tiflash_scan:{dtfile:{total_scanned_packs:1, total_skipped_packs:0, total_scanned_rows:8192, total_skipped_rows:0, total_rs_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.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, "time:1s, loops:2, threads:1, tiflash_scan:{dtfile:{total_scanned_packs:1, total_skipped_packs:0, total_scanned_rows:8192, total_skipped_rows:0, total_rs_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:{dtfile:{total_scanned_packs:3, total_skipped_packs:11, total_scanned_rows:20192, total_skipped_rows:86000, total_rs_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) From e98e9594bce66288e7c0021aeefc3d4e8d44157b Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 30 Nov 2022 12:23:59 +0800 Subject: [PATCH 09/13] *: add telemetry support for IndexMerge (#39469) close pingcap/tidb#39475 --- executor/adapter.go | 1 + executor/builder.go | 3 +++ metrics/telemetry.go | 26 +++++++++++++++++++++++++ session/session.go | 6 ++++++ telemetry/data_feature_usage.go | 14 ++++++++++++++ telemetry/data_feature_usage_test.go | 29 ++++++++++++++++++++++++++++ 6 files changed, 79 insertions(+) diff --git a/executor/adapter.go b/executor/adapter.go index aaa7aa8c4b8c9..a7942f2638a02 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -198,6 +198,7 @@ type TelemetryInfo struct { UseFlashbackToCluster bool PartitionTelemetry *PartitionTelemetryInfo AccountLockTelemetry *AccountLockTelemetryInfo + UseIndexMerge bool } // PartitionTelemetryInfo records table partition telemetry information during execution. diff --git a/executor/builder.go b/executor/builder.go index 8391964095877..e3c4e5533b68f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3977,6 +3977,9 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd } func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMergeReader) Executor { + if b.Ti != nil { + b.Ti.UseIndexMerge = true + } ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) if err := b.validCanReadTemporaryOrCacheTable(ts.Table); err != nil { b.err = err diff --git a/metrics/telemetry.go b/metrics/telemetry.go index 7460ca5ceb04c..591823f9952d9 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", }) + TelemetryIndexMergeUsage = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "telemetry", + Name: "index_merge_usage", + Help: "Counter of usage of index merge", + }) TelemetryCompactPartitionCnt = prometheus.NewCounter( prometheus.CounterOpts{ Namespace: "tidb", @@ -396,3 +403,22 @@ func GetDDLUsageCounter() DDLUsageCounter { FlashbackClusterUsed: readCounter(TelemetryFlashbackClusterCnt), } } + +// IndexMergeUsageCounter records the usages of IndexMerge feature. +type IndexMergeUsageCounter struct { + IndexMergeUsed int64 `json:"index_merge_used"` +} + +// Sub returns the difference of two counters. +func (i IndexMergeUsageCounter) Sub(rhs IndexMergeUsageCounter) IndexMergeUsageCounter { + return IndexMergeUsageCounter{ + IndexMergeUsed: i.IndexMergeUsed - rhs.IndexMergeUsed, + } +} + +// GetIndexMergeCounter gets the IndexMerge usage counter. +func GetIndexMergeCounter() IndexMergeUsageCounter { + return IndexMergeUsageCounter{ + IndexMergeUsed: readCounter(TelemetryIndexMergeUsage), + } +} diff --git a/session/session.go b/session/session.go index 437c663999b83..9104c4186bf74 100644 --- a/session/session.go +++ b/session/session.go @@ -150,6 +150,8 @@ var ( telemetryLockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("lockUser") telemetryUnlockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("unlockUser") telemetryCreateOrAlterUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("createOrAlterUser") + + telemetryIndexMerge = metrics.TelemetryIndexMergeUsage ) // Session context, it is consistent with the lifecycle of a client connection. @@ -3588,6 +3590,10 @@ func (s *session) updateTelemetryMetric(es *executor.ExecStmt) { telemetryCTEUsageNotCTE.Inc() } + if ti.UseIndexMerge { + telemetryIndexMerge.Inc() + } + if ti.UseMultiSchemaChange { telemetryMultiSchemaChangeUsage.Inc() } diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 3766945536d9c..6dec8edcb023a 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -58,6 +58,7 @@ type featureUsage struct { DDLUsageCounter *m.DDLUsageCounter `json:"DDLUsageCounter"` EnableGlobalMemoryControl bool `json:"enableGlobalMemoryControl"` AutoIDNoCache bool `json:"autoIDNoCache"` + IndexMergeUsageCounter *m.IndexMergeUsageCounter `json:"indexMergeUsageCounter"` } type placementPolicyUsage struct { @@ -108,6 +109,8 @@ func getFeatureUsage(ctx context.Context, sctx sessionctx.Context) (*featureUsag usage.EnableGlobalMemoryControl = getGlobalMemoryControl() + usage.IndexMergeUsageCounter = getIndexMergeUsageInfo() + return &usage, nil } @@ -244,6 +247,7 @@ var initialTablePartitionCounter m.TablePartitionUsageCounter var initialSavepointStmtCounter int64 var initialLazyPessimisticUniqueCheckSetCount int64 var initialDDLUsageCounter m.DDLUsageCounter +var initialIndexMergeCounter m.IndexMergeUsageCounter // getTxnUsageInfo gets the usage info of transaction related features. It's exported for tests. func getTxnUsageInfo(ctx sessionctx.Context) *TxnUsage { @@ -402,3 +406,13 @@ func getDDLUsageInfo(ctx sessionctx.Context) *m.DDLUsageCounter { func getGlobalMemoryControl() bool { return memory.ServerMemoryLimit.Load() > 0 } + +func postReportIndexMergeUsage() { + initialIndexMergeCounter = m.GetIndexMergeCounter() +} + +func getIndexMergeUsageInfo() *m.IndexMergeUsageCounter { + curr := m.GetIndexMergeCounter() + diff := curr.Sub(initialIndexMergeCounter) + return &diff +} diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index ebe93d6dbaa52..369073009c0a4 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -554,3 +554,32 @@ func TestGlobalMemoryControl(t *testing.T) { require.NoError(t, err) require.False(t, usage.EnableGlobalMemoryControl) } + +func TestIndexMergeUsage(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t1(c1 int, c2 int, index idx1(c1), index idx2(c2))") + res := tk.MustQuery("explain select /*+ use_index_merge(t1, idx1, idx2) */ * from t1 where c1 = 1 and c2 = 1").Rows() + require.Contains(t, res[0][0], "IndexMerge") + + usage, err := telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.Equal(t, usage.IndexMergeUsageCounter.IndexMergeUsed, int64(0)) + + tk.MustExec("select /*+ use_index_merge(t1, idx1, idx2) */ * from t1 where c1 = 1 and c2 = 1") + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(1), usage.IndexMergeUsageCounter.IndexMergeUsed) + + tk.MustExec("select /*+ use_index_merge(t1, idx1, idx2) */ * from t1 where c1 = 1 or c2 = 1") + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(2), usage.IndexMergeUsageCounter.IndexMergeUsed) + + tk.MustExec("select /*+ no_index_merge() */ * from t1 where c1 = 1 or c2 = 1") + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(2), usage.IndexMergeUsageCounter.IndexMergeUsed) +} From 1de2bf6e546ea0ca7540ac3cb0da7f7f67da1a57 Mon Sep 17 00:00:00 2001 From: Jianjun Liao <36503113+Leavrth@users.noreply.github.com> Date: Wed, 30 Nov 2022 12:52:00 +0800 Subject: [PATCH 10/13] Backup: add checkpoint (#38648) close pingcap/tidb#38647 --- br/pkg/backup/BUILD.bazel | 1 + br/pkg/backup/client.go | 293 ++++++++++++-- br/pkg/backup/push.go | 34 +- br/pkg/backup/schema.go | 57 ++- br/pkg/backup/schema_test.go | 26 +- br/pkg/checkpoint/BUILD.bazel | 32 ++ br/pkg/checkpoint/checkpoint.go | 580 +++++++++++++++++++++++++++ br/pkg/checkpoint/checkpoint_test.go | 175 ++++++++ br/pkg/restore/client.go | 2 +- br/pkg/rtree/rtree.go | 7 + br/pkg/storage/local.go | 18 +- br/pkg/storage/local_test.go | 27 ++ br/pkg/summary/collector.go | 16 + br/pkg/summary/summary.go | 9 + br/pkg/task/backup.go | 100 ++++- br/pkg/task/backup_raw.go | 11 +- br/pkg/task/stream.go | 2 +- br/pkg/version/version.go | 15 +- br/pkg/version/version_test.go | 23 ++ 19 files changed, 1349 insertions(+), 79 deletions(-) create mode 100644 br/pkg/checkpoint/BUILD.bazel create mode 100644 br/pkg/checkpoint/checkpoint.go create mode 100644 br/pkg/checkpoint/checkpoint_test.go diff --git a/br/pkg/backup/BUILD.bazel b/br/pkg/backup/BUILD.bazel index c8cad292f4607..65ff4288987a1 100644 --- a/br/pkg/backup/BUILD.bazel +++ b/br/pkg/backup/BUILD.bazel @@ -12,6 +12,7 @@ go_library( importpath = "github.com/pingcap/tidb/br/pkg/backup", visibility = ["//visibility:public"], deps = [ + "//br/pkg/checkpoint", "//br/pkg/checksum", "//br/pkg/conn", "//br/pkg/conn/util", diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 865e7fa2f3078..0241789e65103 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -3,7 +3,9 @@ package backup import ( + "bytes" "context" + "encoding/base64" "encoding/hex" "encoding/json" "fmt" @@ -21,6 +23,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/checkpoint" "github.com/pingcap/tidb/br/pkg/conn" connutil "github.com/pingcap/tidb/br/pkg/conn/util" berrors "github.com/pingcap/tidb/br/pkg/errors" @@ -90,6 +93,10 @@ type Client struct { backend *backuppb.StorageBackend apiVersion kvrpcpb.APIVersion + cipher *backuppb.CipherInfo + checkpointMeta *checkpoint.CheckpointMetadata + checkpointRunner *checkpoint.CheckpointRunner + gcTTL int64 } @@ -101,9 +108,18 @@ func NewBackupClient(ctx context.Context, mgr ClientMgr) *Client { return &Client{ clusterID: clusterID, mgr: mgr, + + cipher: nil, + checkpointMeta: nil, + checkpointRunner: nil, } } +// SetCipher for checkpoint to encrypt sst file's metadata +func (bc *Client) SetCipher(cipher *backuppb.CipherInfo) { + bc.cipher = cipher +} + // GetTS gets a new timestamp from PD. func (bc *Client) GetCurerntTS(ctx context.Context) (uint64, error) { p, l, err := bc.mgr.GetPDClient().GetTS(ctx) @@ -120,6 +136,11 @@ func (bc *Client) GetTS(ctx context.Context, duration time.Duration, ts uint64) backupTS uint64 err error ) + + if bc.checkpointMeta != nil { + log.Info("reuse checkpoint BackupTS", zap.Uint64("backup-ts", bc.checkpointMeta.BackupTS)) + return bc.checkpointMeta.BackupTS, nil + } if ts > 0 { backupTS = ts } else { @@ -160,6 +181,15 @@ func (bc *Client) SetLockFile(ctx context.Context) error { "This file exists to remind other backup jobs won't use this path")) } +// GetSafePointID get the gc-safe-point's service-id from either checkpoint or immediate generation +func (bc *Client) GetSafePointID() string { + if bc.checkpointMeta != nil { + log.Info("reuse the checkpoint gc-safepoint service id", zap.String("service-id", bc.checkpointMeta.GCServiceId)) + return bc.checkpointMeta.GCServiceId + } + return utils.MakeSafePointID() +} + // SetGCTTL set gcTTL for client. func (bc *Client) SetGCTTL(ttl int64) { if ttl <= 0 { @@ -204,13 +234,145 @@ func (bc *Client) SetStorageAndCheckNotInUse( "there may be some backup files in the path already, "+ "please specify a correct backup directory!", bc.storage.URI()+"/"+metautil.MetaFile) } - err = CheckBackupStorageIsLocked(ctx, bc.storage) + // use checkpoint mode if checkpoint meta exists + exist, err = bc.storage.FileExists(ctx, checkpoint.CheckpointMetaPath) if err != nil { - return err + return errors.Annotatef(err, "error occurred when checking %s file", checkpoint.CheckpointMetaPath) + } + + // if there is no checkpoint meta, then checkpoint mode is not used + // or it is the first execution + if exist { + // load the config's hash to keep the config unchanged. + log.Info("load the checkpoint meta, so the existence of lockfile is allowed.") + bc.checkpointMeta, err = checkpoint.LoadCheckpointMetadata(ctx, bc.storage) + if err != nil { + return errors.Annotatef(err, "error occurred when loading %s file", checkpoint.CheckpointMetaPath) + } + } else { + err = CheckBackupStorageIsLocked(ctx, bc.storage) + if err != nil { + return err + } } + + return nil +} + +// CheckCheckpoint check whether the configs are the same +func (bc *Client) CheckCheckpoint(hash []byte) error { + if bc.checkpointMeta != nil && !bytes.Equal(bc.checkpointMeta.ConfigHash, hash) { + return errors.Annotatef(berrors.ErrInvalidArgument, "failed to backup to %v, "+ + "because the checkpoint mode is used, "+ + "but the hashs of the configs are not the same. Please check the config", + bc.storage.URI(), + ) + } + + // first execution or not using checkpoint mode yet + // or using the same config can pass the check return nil } +func (bc *Client) GetCheckpointRunner() *checkpoint.CheckpointRunner { + return bc.checkpointRunner +} + +// StartCheckpointMeta will +// 1. saves the initial status into the external storage; +// 2. load the checkpoint data from external storage +// 3. start checkpoint runner +func (bc *Client) StartCheckpointRunner( + ctx context.Context, + cfgHash []byte, + backupTS uint64, + ranges []rtree.Range, + safePointID string, + progressCallBack func(ProgressUnit), +) (err error) { + if bc.checkpointMeta == nil { + bc.checkpointMeta = &checkpoint.CheckpointMetadata{ + GCServiceId: safePointID, + ConfigHash: cfgHash, + BackupTS: backupTS, + Ranges: ranges, + } + + // sync the checkpoint meta to the external storage at first + if err := checkpoint.SaveCheckpointMetadata(ctx, bc.storage, bc.checkpointMeta); err != nil { + return errors.Trace(err) + } + } else { + // otherwise, the checkpoint meta is loaded from the external storage, + // no need to save it again + // besides, there are exist checkpoint data need to be loaded before start checkpoint runner + bc.checkpointMeta.CheckpointDataMap, err = bc.loadCheckpointRanges(ctx, progressCallBack) + if err != nil { + return errors.Trace(err) + } + } + + bc.checkpointRunner = checkpoint.StartCheckpointRunner(ctx, bc.storage, bc.cipher) + return nil +} + +func (bc *Client) WaitForFinishCheckpoint() { + if bc.checkpointRunner != nil { + bc.checkpointRunner.WaitForFinish() + } +} + +// GetProgressRange loads the checkpoint(finished) sub-ranges of the current range, and calculate its incompleted sub-ranges. +func (bc *Client) GetProgressRange(r rtree.Range) (*rtree.ProgressRange, error) { + // use groupKey to distinguish different ranges + groupKey := base64.URLEncoding.EncodeToString(r.StartKey) + if bc.checkpointMeta != nil && len(bc.checkpointMeta.CheckpointDataMap) > 0 { + rangeTree, exists := bc.checkpointMeta.CheckpointDataMap[groupKey] + if exists { + incomplete := rangeTree.GetIncompleteRange(r.StartKey, r.EndKey) + delete(bc.checkpointMeta.CheckpointDataMap, groupKey) + return &rtree.ProgressRange{ + Res: rangeTree, + Incomplete: incomplete, + Origin: r, + GroupKey: groupKey, + }, nil + } + } + + // the origin range are not recorded in checkpoint + // return the default progress range + return &rtree.ProgressRange{ + Res: rtree.NewRangeTree(), + Incomplete: []rtree.Range{ + r, + }, + Origin: r, + GroupKey: groupKey, + }, nil +} + +// LoadCheckpointRange loads the checkpoint(finished) sub-ranges of the current range, and calculate its incompleted sub-ranges. +func (bc *Client) loadCheckpointRanges(ctx context.Context, progressCallBack func(ProgressUnit)) (map[string]rtree.RangeTree, error) { + rangeDataMap := make(map[string]rtree.RangeTree) + + pastDureTime, err := checkpoint.WalkCheckpointFile(ctx, bc.storage, bc.cipher, func(groupKey string, rg *rtree.Range) { + rangeTree, exists := rangeDataMap[groupKey] + if !exists { + rangeTree = rtree.NewRangeTree() + rangeDataMap[groupKey] = rangeTree + } + rangeTree.Put(rg.StartKey, rg.EndKey, rg.Files) + progressCallBack(RegionUnit) + }) + + // we should adjust start-time of the summary to `pastDureTime` earlier + log.Info("past cost time", zap.Duration("cost", pastDureTime)) + summary.AdjustStartTimeToEarlierTime(pastDureTime) + + return rangeDataMap, errors.Trace(err) +} + // SetStorage sets ExternalStorage for client. func (bc *Client) SetStorage( ctx context.Context, @@ -239,6 +401,22 @@ func (bc *Client) SetApiVersion(v kvrpcpb.APIVersion) { bc.apiVersion = v } +// Client.BuildBackupRangeAndSchema calls BuildBackupRangeAndSchema, +// if the checkpoint mode is used, return the ranges from checkpoint meta +func (bc *Client) BuildBackupRangeAndSchema( + storage kv.Storage, + tableFilter filter.Filter, + backupTS uint64, + isFullBackup bool, +) ([]rtree.Range, *Schemas, []*backuppb.PlacementPolicy, error) { + if bc.checkpointMeta == nil { + return BuildBackupRangeAndSchema(storage, tableFilter, backupTS, isFullBackup, true) + } + _, schemas, policies, err := BuildBackupRangeAndSchema(storage, tableFilter, backupTS, isFullBackup, false) + schemas.SetCheckpointChecksum(bc.checkpointMeta.CheckpointChecksum) + return bc.checkpointMeta.Ranges, schemas, policies, errors.Trace(err) +} + // CheckBackupStorageIsLocked checks whether backups is locked. // which means we found other backup progress already write // some data files into the same backup directory or cloud prefix. @@ -252,7 +430,7 @@ func CheckBackupStorageIsLocked(ctx context.Context, s storage.ExternalStorage) // should return error to break the walkDir when found lock file and other .sst files. if strings.HasSuffix(path, ".sst") { return errors.Annotatef(berrors.ErrInvalidArgument, "backup lock file and sst file exist in %v, "+ - "there are some backup files in the path already, "+ + "there are some backup files in the path already, but hasn't checkpoint metadata, "+ "please specify a correct backup directory!", s.URI()+"/"+metautil.LockFile) } return nil @@ -319,6 +497,7 @@ func BuildBackupRangeAndSchema( tableFilter filter.Filter, backupTS uint64, isFullBackup bool, + buildRange bool, ) ([]rtree.Range, *Schemas, []*backuppb.PlacementPolicy, error) { snapshot := storage.GetSnapshot(kv.NewVersion(backupTS)) m := meta.NewSnapshotMeta(snapshot) @@ -435,15 +614,17 @@ func BuildBackupRangeAndSchema( backupSchemas.AddSchema(dbInfo, tableInfo) - tableRanges, err := BuildTableRanges(tableInfo) - if err != nil { - return nil, nil, nil, errors.Trace(err) - } - for _, r := range tableRanges { - ranges = append(ranges, rtree.Range{ - StartKey: r.StartKey, - EndKey: r.EndKey, - }) + if buildRange { + tableRanges, err := BuildTableRanges(tableInfo) + if err != nil { + return nil, nil, nil, errors.Trace(err) + } + for _, r := range tableRanges { + ranges = append(ranges, rtree.Range{ + StartKey: r.StartKey, + EndKey: r.EndKey, + }) + } } } } @@ -604,10 +785,13 @@ func (bc *Client) BackupRanges( id := id req := request req.StartKey, req.EndKey = r.StartKey, r.EndKey - + pr, err := bc.GetProgressRange(r) + if err != nil { + return errors.Trace(err) + } workerPool.ApplyOnErrorGroup(eg, func() error { elctx := logutil.ContextWithField(ectx, logutil.RedactAny("range-sn", id)) - err := bc.BackupRange(elctx, req, metaWriter, progressCallBack) + err := bc.BackupRange(elctx, req, pr, metaWriter, progressCallBack) if err != nil { // The error due to context cancel, stack trace is meaningless, the stack shall be suspended (also clear) if errors.Cause(err) == context.Canceled { @@ -618,6 +802,7 @@ func (bc *Client) BackupRanges( return nil }) } + return eg.Wait() } @@ -625,7 +810,8 @@ func (bc *Client) BackupRanges( // Returns an array of files backed up. func (bc *Client) BackupRange( ctx context.Context, - req backuppb.BackupRequest, + request backuppb.BackupRequest, + progressRange *rtree.ProgressRange, metaWriter *metautil.MetaWriter, progressCallBack func(ProgressUnit), ) (err error) { @@ -633,17 +819,17 @@ func (bc *Client) BackupRange( defer func() { elapsed := time.Since(start) logutil.CL(ctx).Info("backup range completed", - logutil.Key("startKey", req.StartKey), logutil.Key("endKey", req.EndKey), + logutil.Key("startKey", progressRange.Origin.StartKey), logutil.Key("endKey", progressRange.Origin.EndKey), zap.Duration("take", elapsed)) - key := "range start:" + hex.EncodeToString(req.StartKey) + " end:" + hex.EncodeToString(req.EndKey) + key := "range start:" + hex.EncodeToString(progressRange.Origin.StartKey) + " end:" + hex.EncodeToString(progressRange.Origin.EndKey) if err != nil { summary.CollectFailureUnit(key, err) } }() logutil.CL(ctx).Info("backup range started", - logutil.Key("startKey", req.StartKey), logutil.Key("endKey", req.EndKey), - zap.Uint64("rateLimit", req.RateLimit), - zap.Uint32("concurrency", req.Concurrency)) + logutil.Key("startKey", progressRange.Origin.StartKey), logutil.Key("endKey", progressRange.Origin.EndKey), + zap.Uint64("rateLimit", request.RateLimit), + zap.Uint32("concurrency", request.Concurrency)) var allStores []*metapb.Store allStores, err = conn.GetAllTiKVStoresWithRetry(ctx, bc.mgr.GetPDClient(), connutil.SkipTiFlash) @@ -652,35 +838,57 @@ func (bc *Client) BackupRange( } logutil.CL(ctx).Info("backup push down started") - push := newPushDown(bc.mgr, len(allStores)) - results, err := push.pushBackup(ctx, req, allStores, progressCallBack) - if err != nil { - return errors.Trace(err) + // either the `incomplete` is origin range itself, + // or the `incomplete` is sub-ranges split by checkpoint of origin range + if len(progressRange.Incomplete) > 0 { + // don't make the origin request dirty, + // since fineGrainedBackup need to use it. + req := request + if len(progressRange.Incomplete) > 1 { + subRanges := make([]*kvrpcpb.KeyRange, 0, len(progressRange.Incomplete)) + for _, r := range progressRange.Incomplete { + subRanges = append(subRanges, &kvrpcpb.KeyRange{ + StartKey: r.StartKey, + EndKey: r.EndKey, + }) + } + req.SubRanges = subRanges + } else { + // compatible with older version of TiKV + req.StartKey = progressRange.Incomplete[0].StartKey + req.EndKey = progressRange.Incomplete[0].EndKey + } + + push := newPushDown(bc.mgr, len(allStores)) + err = push.pushBackup(ctx, req, progressRange, allStores, bc.checkpointRunner, progressCallBack) + if err != nil { + return errors.Trace(err) + } } - logutil.CL(ctx).Info("backup push down completed", zap.Int("small-range-count", results.Len())) + logutil.CL(ctx).Info("backup push down completed", zap.Int("small-range-count", progressRange.Res.Len())) // Find and backup remaining ranges. // TODO: test fine grained backup. - if err := bc.fineGrainedBackup(ctx, req, results, progressCallBack); err != nil { + if err := bc.fineGrainedBackup(ctx, request, progressRange, progressCallBack); err != nil { return errors.Trace(err) } // update progress of range unit progressCallBack(RangeUnit) - if req.IsRawKv { + if request.IsRawKv { logutil.CL(ctx).Info("raw ranges backed up", - logutil.Key("startKey", req.StartKey), - logutil.Key("endKey", req.EndKey), - zap.String("cf", req.Cf)) + logutil.Key("startKey", progressRange.Origin.StartKey), + logutil.Key("endKey", progressRange.Origin.EndKey), + zap.String("cf", request.Cf)) } else { logutil.CL(ctx).Info("transactional range backup completed", - zap.Reflect("StartTS", req.StartVersion), - zap.Reflect("EndTS", req.EndVersion)) + zap.Reflect("StartTS", request.StartVersion), + zap.Reflect("EndTS", request.EndVersion)) } var ascendErr error - results.Ascend(func(i btree.Item) bool { + progressRange.Res.Ascend(func(i btree.Item) bool { r := i.(*rtree.Range) for _, f := range r.Files { summary.CollectSuccessUnit(summary.TotalKV, 1, f.TotalKvs) @@ -699,7 +907,7 @@ func (bc *Client) BackupRange( } // Check if there are duplicated files. - checkDupFiles(&results) + checkDupFiles(&progressRange.Res) return nil } @@ -732,7 +940,7 @@ func (bc *Client) findRegionLeader(ctx context.Context, key []byte, isRawKv bool func (bc *Client) fineGrainedBackup( ctx context.Context, req backuppb.BackupRequest, - rangeTree rtree.RangeTree, + pr *rtree.ProgressRange, progressCallBack func(ProgressUnit), ) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { @@ -759,7 +967,7 @@ func (bc *Client) fineGrainedBackup( bo := tikv.NewBackoffer(ctx, backupFineGrainedMaxBackoff) for { // Step1, check whether there is any incomplete range - incomplete := rangeTree.GetIncompleteRange(req.StartKey, req.EndKey) + incomplete := pr.Res.GetIncompleteRange(req.StartKey, req.EndKey) if len(incomplete) == 0 { return nil } @@ -827,7 +1035,18 @@ func (bc *Client) fineGrainedBackup( logutil.Key("fine-grained-range-start", resp.StartKey), logutil.Key("fine-grained-range-end", resp.EndKey), ) - rangeTree.Put(resp.StartKey, resp.EndKey, resp.Files) + if bc.checkpointRunner != nil { + if err := bc.checkpointRunner.Append( + ctx, + pr.GroupKey, + resp.StartKey, + resp.EndKey, + resp.Files, + ); err != nil { + return errors.Annotate(err, "failed to flush checkpoint when fineGrainedBackup") + } + } + pr.Res.Put(resp.StartKey, resp.EndKey, resp.Files) apiVersion := resp.ApiVersion bc.SetApiVersion(apiVersion) diff --git a/br/pkg/backup/push.go b/br/pkg/backup/push.go index 45c2b9acca01c..2ffffe690ffe5 100644 --- a/br/pkg/backup/push.go +++ b/br/pkg/backup/push.go @@ -13,6 +13,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/checkpoint" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/redact" @@ -54,9 +55,11 @@ func newPushDown(mgr ClientMgr, capacity int) *pushDown { func (push *pushDown) pushBackup( ctx context.Context, req backuppb.BackupRequest, + pr *rtree.ProgressRange, stores []*metapb.Store, + checkpointRunner *checkpoint.CheckpointRunner, progressCallBack func(ProgressUnit), -) (rtree.RangeTree, error) { +) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("pushDown.pushBackup", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -64,10 +67,9 @@ func (push *pushDown) pushBackup( } // Push down backup tasks to all tikv instances. - res := rtree.NewRangeTree() failpoint.Inject("noop-backup", func(_ failpoint.Value) { logutil.CL(ctx).Warn("skipping normal backup, jump to fine-grained backup, meow :3", logutil.Key("start-key", req.StartKey), logutil.Key("end-key", req.EndKey)) - failpoint.Return(res, nil) + failpoint.Return(nil) }) wg := new(sync.WaitGroup) @@ -84,7 +86,7 @@ func (push *pushDown) pushBackup( // BR should be able to backup even some of stores disconnected. // The regions managed by this store can be retried at fine-grained backup then. logutil.CL(lctx).Warn("fail to connect store, skipping", zap.Error(err)) - return res, nil + return nil } wg.Add(1) go func() { @@ -125,7 +127,7 @@ func (push *pushDown) pushBackup( store := respAndStore.GetStore() if !ok { // Finished. - return res, nil + return nil } failpoint.Inject("backup-timeout-error", func(val failpoint.Value) { msg := val.(string) @@ -165,7 +167,19 @@ func (push *pushDown) pushBackup( }) if resp.GetError() == nil { // None error means range has been backuped successfully. - res.Put( + if checkpointRunner != nil { + if err := checkpointRunner.Append( + ctx, + pr.GroupKey, + resp.StartKey, + resp.EndKey, + resp.Files, + ); err != nil { + // the error is only from flush operator + return errors.Annotate(err, "failed to flush checkpoint") + } + } + pr.Res.Put( resp.GetStartKey(), resp.GetEndKey(), resp.GetFiles()) // Update progress @@ -181,7 +195,7 @@ func (push *pushDown) pushBackup( case *backuppb.Error_ClusterIdError: logutil.CL(ctx).Error("backup occur cluster ID error", zap.Reflect("error", v)) - return res, errors.Annotatef(berrors.ErrKVClusterIDMismatch, "%v", errPb) + return errors.Annotatef(berrors.ErrKVClusterIDMismatch, "%v", errPb) default: if utils.MessageIsRetryableStorageError(errPb.GetMsg()) { logutil.CL(ctx).Warn("backup occur storage error", zap.String("error", errPb.GetMsg())) @@ -204,7 +218,7 @@ func (push *pushDown) pushBackup( if len(errMsg) <= 0 { errMsg = errPb.Msg } - return res, errors.Annotatef(berrors.ErrKVStorage, "error happen in store %v at %s: %s %s", + return errors.Annotatef(berrors.ErrKVStorage, "error happen in store %v at %s: %s %s", store.GetId(), redact.String(store.GetAddress()), req.StorageBackend.String(), @@ -214,10 +228,10 @@ func (push *pushDown) pushBackup( } case err := <-push.errCh: if !berrors.Is(err, berrors.ErrFailedToConnect) { - return res, errors.Annotatef(err, "failed to backup range [%s, %s)", redact.Key(req.StartKey), redact.Key(req.EndKey)) + return errors.Annotatef(err, "failed to backup range [%s, %s)", redact.Key(req.StartKey), redact.Key(req.EndKey)) } logutil.CL(ctx).Warn("skipping disconnected stores", logutil.ShortError(err)) - return res, nil + return nil } } } diff --git a/br/pkg/backup/schema.go b/br/pkg/backup/schema.go index 066043c224064..bb0cf7f884189 100644 --- a/br/pkg/backup/schema.go +++ b/br/pkg/backup/schema.go @@ -12,6 +12,7 @@ import ( "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/checkpoint" "github.com/pingcap/tidb/br/pkg/checksum" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/logutil" @@ -44,14 +45,22 @@ type schemaInfo struct { type Schemas struct { // name -> schema schemas map[string]*schemaInfo + + // checkpoint: table id -> checksum + checkpointChecksum map[int64]*checkpoint.ChecksumItem } func NewBackupSchemas() *Schemas { return &Schemas{ - schemas: make(map[string]*schemaInfo), + schemas: make(map[string]*schemaInfo), + checkpointChecksum: nil, } } +func (ss *Schemas) SetCheckpointChecksum(checkpointChecksum map[int64]*checkpoint.ChecksumItem) { + ss.checkpointChecksum = checkpointChecksum +} + func (ss *Schemas) AddSchema( dbInfo *model.DBInfo, tableInfo *model.TableInfo, ) { @@ -73,6 +82,7 @@ func (ss *Schemas) AddSchema( func (ss *Schemas) BackupSchemas( ctx context.Context, metaWriter *metautil.MetaWriter, + checkpointRunner *checkpoint.CheckpointRunner, store kv.Storage, statsHandle *handle.Handle, backupTS uint64, @@ -100,6 +110,11 @@ func (ss *Schemas) BackupSchemas( schema.dbInfo.Name = utils.TemporaryDBName(schema.dbInfo.Name.O) } + var checksum *checkpoint.ChecksumItem + var exists bool = false + if ss.checkpointChecksum != nil { + checksum, exists = ss.checkpointChecksum[schema.tableInfo.ID] + } workerPool.ApplyOnErrorGroup(errg, func() error { if schema.tableInfo != nil { logger := log.With( @@ -109,16 +124,38 @@ func (ss *Schemas) BackupSchemas( if !skipChecksum { logger.Info("Calculate table checksum start") - start := time.Now() - err := schema.calculateChecksum(ectx, store.GetClient(), backupTS, copConcurrency) - if err != nil { - return errors.Trace(err) + if exists && checksum != nil { + schema.crc64xor = checksum.Crc64xor + schema.totalKvs = checksum.TotalKvs + schema.totalBytes = checksum.TotalBytes + logger.Info("Calculate table checksum completed (from checkpoint)", + zap.Uint64("Crc64Xor", schema.crc64xor), + zap.Uint64("TotalKvs", schema.totalKvs), + zap.Uint64("TotalBytes", schema.totalBytes)) + } else { + start := time.Now() + err := schema.calculateChecksum(ectx, store.GetClient(), backupTS, copConcurrency) + if err != nil { + return errors.Trace(err) + } + calculateCost := time.Since(start) + var flushCost time.Duration + if checkpointRunner != nil { + // if checkpoint runner is running and the checksum is not from checkpoint + // then flush the checksum by the checkpoint runner + startFlush := time.Now() + if err = checkpointRunner.FlushChecksum(ctx, schema.tableInfo.ID, schema.crc64xor, schema.totalKvs, schema.totalBytes, calculateCost.Seconds()); err != nil { + return errors.Trace(err) + } + flushCost = time.Since(startFlush) + } + logger.Info("Calculate table checksum completed", + zap.Uint64("Crc64Xor", schema.crc64xor), + zap.Uint64("TotalKvs", schema.totalKvs), + zap.Uint64("TotalBytes", schema.totalBytes), + zap.Duration("calculate-take", calculateCost), + zap.Duration("flush-take", flushCost)) } - logger.Info("Calculate table checksum completed", - zap.Uint64("Crc64Xor", schema.crc64xor), - zap.Uint64("TotalKvs", schema.totalKvs), - zap.Uint64("TotalBytes", schema.totalBytes), - zap.Duration("take", time.Since(start))) } if statsHandle != nil { if err := schema.dumpStatsToJSON(statsHandle); err != nil { diff --git a/br/pkg/backup/schema_test.go b/br/pkg/backup/schema_test.go index bed9d834d2e10..08d560bf03c25 100644 --- a/br/pkg/backup/schema_test.go +++ b/br/pkg/backup/schema_test.go @@ -108,7 +108,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { testFilter, err := filter.Parse([]string{"test.t1"}) require.NoError(t, err) _, backupSchemas, _, err := backup.BuildBackupRangeAndSchema( - m.Storage, testFilter, math.MaxUint64, false) + m.Storage, testFilter, math.MaxUint64, false, true) require.NoError(t, err) require.NotNil(t, backupSchemas) @@ -116,7 +116,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { fooFilter, err := filter.Parse([]string{"foo.t1"}) require.NoError(t, err) _, backupSchemas, _, err = backup.BuildBackupRangeAndSchema( - m.Storage, fooFilter, math.MaxUint64, false) + m.Storage, fooFilter, math.MaxUint64, false, true) require.NoError(t, err) require.Nil(t, backupSchemas) @@ -125,7 +125,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { noFilter, err := filter.Parse([]string{"*.*", "!mysql.*"}) require.NoError(t, err) _, backupSchemas, _, err = backup.BuildBackupRangeAndSchema( - m.Storage, noFilter, math.MaxUint64, false) + m.Storage, noFilter, math.MaxUint64, false, true) require.NoError(t, err) require.NotNil(t, backupSchemas) @@ -137,7 +137,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { var policies []*backuppb.PlacementPolicy _, backupSchemas, policies, err = backup.BuildBackupRangeAndSchema( - m.Storage, testFilter, math.MaxUint64, false) + m.Storage, testFilter, math.MaxUint64, false, true) require.NoError(t, err) require.Equal(t, 1, backupSchemas.Len()) // we expect no policies collected, because it's not full backup. @@ -151,7 +151,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) require.Equal(t, int64(1), updateCh.get()) require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) @@ -170,7 +170,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { tk.MustExec("insert into t2 values (11);") _, backupSchemas, policies, err = backup.BuildBackupRangeAndSchema( - m.Storage, noFilter, math.MaxUint64, true) + m.Storage, noFilter, math.MaxUint64, true, true) require.NoError(t, err) require.Equal(t, 2, backupSchemas.Len()) // we expect the policy fivereplicas collected in full backup. @@ -180,7 +180,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, m.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) require.Equal(t, int64(2), updateCh.get()) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) @@ -219,7 +219,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { f, err := filter.Parse([]string{"test.t3"}) require.NoError(t, err) - _, backupSchemas, _, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false) + _, backupSchemas, _, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false, true) require.NoError(t, err) require.Equal(t, 1, backupSchemas.Len()) @@ -234,7 +234,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) require.NoError(t, err) @@ -253,7 +253,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { // recover the statistics. tk.MustExec("analyze table t3;") - _, backupSchemas, _, err = backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false) + _, backupSchemas, _, err = backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false, true) require.NoError(t, err) require.Equal(t, 1, backupSchemas.Len()) @@ -262,7 +262,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, m.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter2, nil, m.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) require.NoError(t, err) @@ -294,7 +294,7 @@ func TestBackupSchemasForSystemTable(t *testing.T) { f, err := filter.Parse([]string{"mysql.systable*"}) require.NoError(t, err) - _, backupSchemas, _, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false) + _, backupSchemas, _, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64, false, true) require.NoError(t, err) require.Equal(t, systemTablesCount, backupSchemas.Len()) @@ -305,7 +305,7 @@ func TestBackupSchemasForSystemTable(t *testing.T) { updateCh := new(simpleProgress) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) - err = backupSchemas.BackupSchemas(ctx, metaWriter2, m.Storage, nil, + err = backupSchemas.BackupSchemas(ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, true, updateCh) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) diff --git a/br/pkg/checkpoint/BUILD.bazel b/br/pkg/checkpoint/BUILD.bazel new file mode 100644 index 0000000000000..baae284d545fb --- /dev/null +++ b/br/pkg/checkpoint/BUILD.bazel @@ -0,0 +1,32 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "checkpoint", + srcs = ["checkpoint.go"], + importpath = "github.com/pingcap/tidb/br/pkg/checkpoint", + visibility = ["//visibility:public"], + deps = [ + "//br/pkg/metautil", + "//br/pkg/rtree", + "//br/pkg/storage", + "//br/pkg/summary", + "//br/pkg/utils", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/brpb", + "@com_github_pingcap_log//:log", + "@org_uber_go_zap//:zap", + ], +) + +go_test( + name = "checkpoint_test", + srcs = ["checkpoint_test.go"], + deps = [ + ":checkpoint", + "//br/pkg/rtree", + "//br/pkg/storage", + "@com_github_pingcap_kvproto//pkg/brpb", + "@com_github_pingcap_kvproto//pkg/encryptionpb", + "@com_github_stretchr_testify//require", + ], +) diff --git a/br/pkg/checkpoint/checkpoint.go b/br/pkg/checkpoint/checkpoint.go new file mode 100644 index 0000000000000..a1634779009b8 --- /dev/null +++ b/br/pkg/checkpoint/checkpoint.go @@ -0,0 +1,580 @@ +// 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 checkpoint + +import ( + "bytes" + "context" + "crypto/sha256" + "encoding/base64" + "encoding/json" + "fmt" + "math/rand" + "strings" + "sync" + "time" + + "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/metautil" + "github.com/pingcap/tidb/br/pkg/rtree" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/summary" + "github.com/pingcap/tidb/br/pkg/utils" + "go.uber.org/zap" +) + +const ( + CheckpointMetaPath = "checkpoint.meta" + CheckpointDir = "/checkpoints" + + CheckpointDataDir = CheckpointDir + "/data" + CheckpointChecksumDir = CheckpointDir + "/checksum" +) + +const MaxChecksumTotalCost float64 = 60.0 + +const tickDuration = 30 * time.Second + +type CheckpointMessage struct { + // start-key of the origin range + GroupKey string + + Group *rtree.Range +} + +// A Checkpoint Range File is like this: +// +// ChecksumData +// +----------------+ RangeGroupData RangeGroups +// | DureTime | +--------------------------+ encrypted +-------------+ +// | RangeGroupData-+---> | RangeGroupsEncriptedData-+----------> | GroupKey | +// | RangeGroupData | | Checksum | | Range | +// | ... | | CipherIv | | ... | +// | RangeGroupData | | Size | | Range | +// +----------------+ +--------------------------+ +-------------+ + +type RangeGroups struct { + GroupKey string `json:"group-key"` + Groups []*rtree.Range `json:"groups"` +} + +type RangeGroupData struct { + RangeGroupsEncriptedData []byte + Checksum []byte + CipherIv []byte + + Size int +} + +type CheckpointData struct { + DureTime time.Duration `json:"dure-time"` + RangeGroupMetas []*RangeGroupData `json:"range-group-metas"` +} + +// A Checkpoint Checksum File is like this: +// +// ChecksumInfo ChecksumItems ChecksumItem +// +-------------+ +--------------+ +--------------+ +// | Content---+-> | ChecksumItem-+---> | TableID | +// | Checksum | | ChecksumItem | | Crc64xor | +// +-------------+ | ... | | TotalKvs | +// | ChecksumItem | | TotalBytes | +// +--------------+ +--------------+ + +type ChecksumItem struct { + TableID int64 `json:"table-id"` + Crc64xor uint64 `json:"crc64-xor"` + TotalKvs uint64 `json:"total-kvs"` + TotalBytes uint64 `json:"total-bytes"` +} + +type ChecksumItems struct { + Items []*ChecksumItem `json:"checksum-items"` +} + +type ChecksumInfo struct { + Content []byte `json:"content"` + Checksum []byte `json:"checksum"` +} + +type ChecksumRunner struct { + sync.Mutex + + checksumItems ChecksumItems + + // when the total time cost is large than the threshold, + // begin to flush checksum + totalCost float64 + + err error + wg sync.WaitGroup + workerPool utils.WorkerPool +} + +func NewChecksumRunner() *ChecksumRunner { + return &ChecksumRunner{ + workerPool: *utils.NewWorkerPool(4, "checksum flush worker"), + } +} + +// FlushChecksum save the checksum in the memory temporarily +// and flush to the external storage if checksum take much time +func (cr *ChecksumRunner) FlushChecksum( + ctx context.Context, + s storage.ExternalStorage, + tableID int64, + crc64xor uint64, + totalKvs uint64, + totalBytes uint64, + timeCost float64, +) error { + checksumItem := &ChecksumItem{ + TableID: tableID, + Crc64xor: crc64xor, + TotalKvs: totalKvs, + TotalBytes: totalBytes, + } + var toBeFlushedChecksumItems *ChecksumItems = nil + cr.Lock() + if cr.err != nil { + err := cr.err + cr.Unlock() + return err + } + if cr.checksumItems.Items == nil { + // reset the checksumInfo + cr.totalCost = 0 + cr.checksumItems.Items = make([]*ChecksumItem, 0) + } + cr.totalCost += timeCost + cr.checksumItems.Items = append(cr.checksumItems.Items, checksumItem) + if cr.totalCost > MaxChecksumTotalCost { + toBeFlushedChecksumItems = &ChecksumItems{ + Items: cr.checksumItems.Items, + } + cr.checksumItems.Items = nil + } + cr.Unlock() + + // now lock is free + if toBeFlushedChecksumItems == nil { + return nil + } + + // create a goroutine to flush checksumInfo to external storage + cr.wg.Add(1) + cr.workerPool.Apply(func() { + defer cr.wg.Done() + recordErr := func(err error) { + cr.Lock() + cr.err = err + cr.Unlock() + } + + content, err := json.Marshal(toBeFlushedChecksumItems) + if err != nil { + recordErr(err) + return + } + + checksum := sha256.Sum256(content) + checksumInfo := &ChecksumInfo{ + Content: content, + Checksum: checksum[:], + } + + data, err := json.Marshal(checksumInfo) + if err != nil { + recordErr(err) + return + } + + fname := fmt.Sprintf("%s/t%d_and__", CheckpointChecksumDir, tableID) + err = s.WriteFile(ctx, fname, data) + if err != nil { + recordErr(err) + return + } + }) + return nil +} + +type CheckpointRunner struct { + meta map[string]*RangeGroups + + checksumRunner *ChecksumRunner + + storage storage.ExternalStorage + cipher *backuppb.CipherInfo + + appendCh chan *CheckpointMessage + metaCh chan map[string]*RangeGroups + errCh chan error + + wg sync.WaitGroup +} + +// only for test +func StartCheckpointRunnerForTest(ctx context.Context, storage storage.ExternalStorage, cipher *backuppb.CipherInfo, tick time.Duration) *CheckpointRunner { + runner := &CheckpointRunner{ + meta: make(map[string]*RangeGroups), + + checksumRunner: NewChecksumRunner(), + + storage: storage, + cipher: cipher, + + appendCh: make(chan *CheckpointMessage), + metaCh: make(chan map[string]*RangeGroups), + errCh: make(chan error), + } + + runner.startCheckpointLoop(ctx, tick) + return runner +} + +func StartCheckpointRunner(ctx context.Context, storage storage.ExternalStorage, cipher *backuppb.CipherInfo) *CheckpointRunner { + runner := &CheckpointRunner{ + meta: make(map[string]*RangeGroups), + + checksumRunner: NewChecksumRunner(), + + storage: storage, + cipher: cipher, + + appendCh: make(chan *CheckpointMessage), + metaCh: make(chan map[string]*RangeGroups), + errCh: make(chan error), + } + + runner.startCheckpointLoop(ctx, tickDuration) + return runner +} + +func (r *CheckpointRunner) FlushChecksum(ctx context.Context, tableID int64, crc64xor uint64, totalKvs uint64, totalBytes uint64, timeCost float64) error { + return r.checksumRunner.FlushChecksum(ctx, r.storage, tableID, crc64xor, totalKvs, totalBytes, timeCost) +} + +func (r *CheckpointRunner) Append( + ctx context.Context, + groupKey string, + startKey []byte, + endKey []byte, + files []*backuppb.File, +) error { + select { + case <-ctx.Done(): + return nil + case err := <-r.errCh: + return err + case r.appendCh <- &CheckpointMessage{ + GroupKey: groupKey, + Group: &rtree.Range{ + StartKey: startKey, + EndKey: endKey, + Files: files, + }, + }: + return nil + } +} + +// Note: Cannot be parallel with `Append` function +func (r *CheckpointRunner) WaitForFinish() { + // can not append anymore + close(r.appendCh) + // wait the range flusher exit + r.wg.Wait() + // wait the checksum flusher exit + r.checksumRunner.wg.Wait() +} + +// Send the meta to the flush goroutine, and reset the CheckpointRunner's meta +func (r *CheckpointRunner) flushMeta(ctx context.Context, errCh chan error) error { + meta := r.meta + r.meta = make(map[string]*RangeGroups) + // do flush + select { + case <-ctx.Done(): + case err := <-errCh: + return err + case r.metaCh <- meta: + } + return nil +} + +// start a goroutine to flush the meta, which is sent from `checkpoint looper`, to the external storage +func (r *CheckpointRunner) startCheckpointRunner(ctx context.Context, wg *sync.WaitGroup) chan error { + errCh := make(chan error, 1) + wg.Add(1) + flushWorker := func(ctx context.Context, errCh chan error) { + defer wg.Done() + for { + select { + case <-ctx.Done(): + return + case meta, ok := <-r.metaCh: + if !ok { + log.Info("stop checkpoint flush worker") + return + } + if err := r.doFlush(ctx, meta); err != nil { + errCh <- err + return + } + } + } + } + + go flushWorker(ctx, errCh) + return errCh +} + +func (r *CheckpointRunner) startCheckpointLoop(ctx context.Context, tickDuration time.Duration) { + r.wg.Add(1) + checkpointLoop := func(ctx context.Context) { + defer r.wg.Done() + cctx, cancel := context.WithCancel(ctx) + defer cancel() + var wg sync.WaitGroup + errCh := r.startCheckpointRunner(cctx, &wg) + ticker := time.NewTicker(tickDuration) + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + if err := r.flushMeta(ctx, errCh); err != nil { + r.errCh <- err + return + } + case msg, ok := <-r.appendCh: + if !ok { + log.Info("stop checkpoint runner") + if err := r.flushMeta(ctx, errCh); err != nil { + r.errCh <- err + } + // close the channel to flush worker + // and wait it to consumes all the metas + close(r.metaCh) + wg.Wait() + return + } + groups, exist := r.meta[msg.GroupKey] + if !exist { + groups = &RangeGroups{ + GroupKey: msg.GroupKey, + Groups: make([]*rtree.Range, 0), + } + r.meta[msg.GroupKey] = groups + } + groups.Groups = append(groups.Groups, msg.Group) + case err := <-errCh: + // pass flush worker's error back + r.errCh <- err + return + } + } + } + + go checkpointLoop(ctx) +} + +// flush the meta to the external storage +func (r *CheckpointRunner) doFlush(ctx context.Context, meta map[string]*RangeGroups) error { + if len(meta) == 0 { + return nil + } + + checkpointData := &CheckpointData{ + DureTime: summary.NowDureTime(), + RangeGroupMetas: make([]*RangeGroupData, 0, len(meta)), + } + + var fname []byte = nil + + for _, group := range meta { + if len(group.Groups) == 0 { + continue + } + + // use the first item's group-key and sub-range-key as the filename + if len(fname) == 0 { + fname = append(append([]byte(group.GroupKey), '.', '.'), group.Groups[0].StartKey...) + } + + // Flush the metaFile to storage + content, err := json.Marshal(group) + if err != nil { + return errors.Trace(err) + } + + encryptBuff, iv, err := metautil.Encrypt(content, r.cipher) + if err != nil { + return errors.Trace(err) + } + + checksum := sha256.Sum256(content) + + checkpointData.RangeGroupMetas = append(checkpointData.RangeGroupMetas, &RangeGroupData{ + RangeGroupsEncriptedData: encryptBuff, + Checksum: checksum[:], + Size: len(content), + CipherIv: iv, + }) + } + + if len(checkpointData.RangeGroupMetas) > 0 { + data, err := json.Marshal(checkpointData) + if err != nil { + return errors.Trace(err) + } + + checksum := sha256.Sum256(fname) + checksumEncoded := base64.URLEncoding.EncodeToString(checksum[:]) + path := fmt.Sprintf("%s/%s_%d.cpt", CheckpointDataDir, checksumEncoded, rand.Uint64()) + if err := r.storage.WriteFile(ctx, path, data); err != nil { + return errors.Trace(err) + } + } + return nil +} + +// walk the whole checkpoint range files and retrieve the metadatat of backed up ranges +// and return the total time cost in the past executions +func WalkCheckpointFile(ctx context.Context, s storage.ExternalStorage, cipher *backuppb.CipherInfo, fn func(groupKey string, rg *rtree.Range)) (time.Duration, error) { + // records the total time cost in the past executions + var pastDureTime time.Duration = 0 + err := s.WalkDir(ctx, &storage.WalkOption{SubDir: CheckpointDataDir}, func(path string, size int64) error { + if strings.HasSuffix(path, ".cpt") { + content, err := s.ReadFile(ctx, path) + if err != nil { + return errors.Trace(err) + } + + checkpointData := &CheckpointData{} + if err = json.Unmarshal(content, checkpointData); err != nil { + return errors.Trace(err) + } + + if checkpointData.DureTime > pastDureTime { + pastDureTime = checkpointData.DureTime + } + for _, meta := range checkpointData.RangeGroupMetas { + decryptContent, err := metautil.Decrypt(meta.RangeGroupsEncriptedData, cipher, meta.CipherIv) + if err != nil { + return errors.Trace(err) + } + + checksum := sha256.Sum256(decryptContent) + if !bytes.Equal(meta.Checksum, checksum[:]) { + log.Error("checkpoint checksum info's checksum mismatch, skip it", + zap.ByteString("expect", meta.Checksum), + zap.ByteString("got", checksum[:]), + ) + continue + } + + group := &RangeGroups{} + if err = json.Unmarshal(decryptContent, group); err != nil { + return errors.Trace(err) + } + + for _, g := range group.Groups { + fn(group.GroupKey, g) + } + } + } + return nil + }) + + return pastDureTime, errors.Trace(err) +} + +type CheckpointMetadata struct { + GCServiceId string `json:"gc-service-id"` + ConfigHash []byte `json:"config-hash"` + BackupTS uint64 `json:"backup-ts"` + Ranges []rtree.Range `json:"ranges"` + + CheckpointChecksum map[int64]*ChecksumItem `json:"-"` + CheckpointDataMap map[string]rtree.RangeTree `json:"-"` +} + +// load checkpoint metadata from the external storage +func LoadCheckpointMetadata(ctx context.Context, s storage.ExternalStorage) (*CheckpointMetadata, error) { + data, err := s.ReadFile(ctx, CheckpointMetaPath) + if err != nil { + return nil, errors.Trace(err) + } + m := &CheckpointMetadata{} + err = json.Unmarshal(data, m) + if err != nil { + return nil, errors.Trace(err) + } + m.CheckpointChecksum, err = loadCheckpointChecksum(ctx, s) + return m, errors.Trace(err) +} + +// walk the whole checkpoint checksum files and retrieve checksum information of tables calculated +func loadCheckpointChecksum(ctx context.Context, s storage.ExternalStorage) (map[int64]*ChecksumItem, error) { + checkpointChecksum := make(map[int64]*ChecksumItem) + + err := s.WalkDir(ctx, &storage.WalkOption{SubDir: CheckpointChecksumDir}, func(path string, size int64) error { + data, err := s.ReadFile(ctx, path) + if err != nil { + return errors.Trace(err) + } + info := &ChecksumInfo{} + err = json.Unmarshal(data, info) + if err != nil { + return errors.Trace(err) + } + + checksum := sha256.Sum256(info.Content) + if !bytes.Equal(info.Checksum, checksum[:]) { + log.Error("checkpoint checksum info's checksum mismatch, skip it", + zap.ByteString("expect", info.Checksum), + zap.ByteString("got", checksum[:]), + ) + return nil + } + + items := &ChecksumItems{} + err = json.Unmarshal(info.Content, items) + if err != nil { + return errors.Trace(err) + } + + for _, c := range items.Items { + checkpointChecksum[c.TableID] = c + } + return nil + }) + return checkpointChecksum, errors.Trace(err) +} + +// save the checkpoint metadata into the external storage +func SaveCheckpointMetadata(ctx context.Context, s storage.ExternalStorage, meta *CheckpointMetadata) error { + data, err := json.Marshal(meta) + if err != nil { + return errors.Trace(err) + } + + err = s.WriteFile(ctx, CheckpointMetaPath, data) + return errors.Trace(err) +} diff --git a/br/pkg/checkpoint/checkpoint_test.go b/br/pkg/checkpoint/checkpoint_test.go new file mode 100644 index 0000000000000..f52b1c6ad2b02 --- /dev/null +++ b/br/pkg/checkpoint/checkpoint_test.go @@ -0,0 +1,175 @@ +// 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 checkpoint_test + +import ( + "context" + "os" + "strings" + "testing" + "time" + + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/pingcap/tidb/br/pkg/checkpoint" + "github.com/pingcap/tidb/br/pkg/rtree" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" +) + +func TestCheckpointMeta(t *testing.T) { + ctx := context.Background() + base := t.TempDir() + s, err := storage.NewLocalStorage(base) + require.NoError(t, err) + + checkpointMeta := &checkpoint.CheckpointMetadata{ + ConfigHash: []byte("123456"), + BackupTS: 123456, + } + + err = checkpoint.SaveCheckpointMetadata(ctx, s, checkpointMeta) + require.NoError(t, err) + + checkpointMeta2, err := checkpoint.LoadCheckpointMetadata(ctx, s) + require.NoError(t, err) + require.Equal(t, checkpointMeta.ConfigHash, checkpointMeta2.ConfigHash) + require.Equal(t, checkpointMeta.BackupTS, checkpointMeta2.BackupTS) +} + +func TestCheckpointRunner(t *testing.T) { + ctx := context.Background() + base := t.TempDir() + s, err := storage.NewLocalStorage(base) + require.NoError(t, err) + os.MkdirAll(base+checkpoint.CheckpointDataDir, 0o755) + os.MkdirAll(base+checkpoint.CheckpointChecksumDir, 0o755) + + cipher := &backuppb.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: []byte("01234567890123456789012345678901"), + } + checkpointRunner := checkpoint.StartCheckpointRunnerForTest(ctx, s, cipher, 5*time.Second) + + data := map[string]struct { + StartKey string + EndKey string + Name string + Name2 string + }{ + "a": { + StartKey: "a", + EndKey: "b", + Name: "c", + Name2: "d", + }, + "A": { + StartKey: "A", + EndKey: "B", + Name: "C", + Name2: "D", + }, + "1": { + StartKey: "1", + EndKey: "2", + Name: "3", + Name2: "4", + }, + } + + data2 := map[string]struct { + StartKey string + EndKey string + Name string + Name2 string + }{ + "+": { + StartKey: "+", + EndKey: "-", + Name: "*", + Name2: "/", + }, + } + + for _, d := range data { + err = checkpointRunner.Append(ctx, "a", []byte(d.StartKey), []byte(d.EndKey), []*backuppb.File{ + {Name: d.Name}, + {Name: d.Name2}, + }) + require.NoError(t, err) + } + + checkpointRunner.FlushChecksum(ctx, 1, 1, 1, 1, checkpoint.MaxChecksumTotalCost-20.0) + checkpointRunner.FlushChecksum(ctx, 2, 2, 2, 2, 40.0) + // now the checksum is flushed, because the total time cost is larger than `MaxChecksumTotalCost` + checkpointRunner.FlushChecksum(ctx, 3, 3, 3, 3, checkpoint.MaxChecksumTotalCost-20.0) + time.Sleep(6 * time.Second) + // the checksum has not been flushed even though after 6 seconds, + // because the total time cost is less than `MaxChecksumTotalCost` + checkpointRunner.FlushChecksum(ctx, 4, 4, 4, 4, 40.0) + + for _, d := range data2 { + err = checkpointRunner.Append(ctx, "+", []byte(d.StartKey), []byte(d.EndKey), []*backuppb.File{ + {Name: d.Name}, + {Name: d.Name2}, + }) + require.NoError(t, err) + } + + checkpointRunner.WaitForFinish() + + checker := func(groupKey string, resp *rtree.Range) { + require.NotNil(t, resp) + d, ok := data[string(resp.StartKey)] + if !ok { + d, ok = data2[string(resp.StartKey)] + require.True(t, ok) + } + require.Equal(t, d.StartKey, string(resp.StartKey)) + require.Equal(t, d.EndKey, string(resp.EndKey)) + require.Equal(t, d.Name, resp.Files[0].Name) + require.Equal(t, d.Name2, resp.Files[1].Name) + } + + _, err = checkpoint.WalkCheckpointFile(ctx, s, cipher, checker) + require.NoError(t, err) + + checkpointMeta := &checkpoint.CheckpointMetadata{ + ConfigHash: []byte("123456"), + BackupTS: 123456, + } + + err = checkpoint.SaveCheckpointMetadata(ctx, s, checkpointMeta) + require.NoError(t, err) + meta, err := checkpoint.LoadCheckpointMetadata(ctx, s) + require.NoError(t, err) + + var i int64 + for i = 1; i <= 4; i++ { + require.Equal(t, meta.CheckpointChecksum[i].Crc64xor, uint64(i)) + } + + // only 2 checksum files exists, they are t2_and__ and t4_and__ + count := 0 + err = s.WalkDir(ctx, &storage.WalkOption{SubDir: checkpoint.CheckpointChecksumDir}, func(s string, i int64) error { + count += 1 + if !strings.Contains(s, "t2") { + require.True(t, strings.Contains(s, "t4")) + } + return nil + }) + require.NoError(t, err) + require.Equal(t, count, 2) +} diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index e5bd63815b60a..49b9b7bb7f58e 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -2583,7 +2583,7 @@ func (rc *Client) SaveSchemas( schemas := TidyOldSchemas(sr) schemasConcurrency := uint(mathutil.Min(64, schemas.Len())) - err := schemas.BackupSchemas(ctx, metaWriter, nil, nil, rc.restoreTS, schemasConcurrency, 0, true, nil) + err := schemas.BackupSchemas(ctx, metaWriter, nil, nil, nil, rc.restoreTS, schemasConcurrency, 0, true, nil) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/rtree/rtree.go b/br/pkg/rtree/rtree.go index 9f12b22daca75..f17ebf38df510 100644 --- a/br/pkg/rtree/rtree.go +++ b/br/pkg/rtree/rtree.go @@ -217,3 +217,10 @@ func (rangeTree *RangeTree) GetIncompleteRange( } return incomplete } + +type ProgressRange struct { + Res RangeTree + Incomplete []Range + Origin Range + GroupKey string +} diff --git a/br/pkg/storage/local.go b/br/pkg/storage/local.go index 68dc760cc1c9a..2d6e0571faa93 100644 --- a/br/pkg/storage/local.go +++ b/br/pkg/storage/local.go @@ -10,6 +10,8 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" ) const ( @@ -38,7 +40,21 @@ func (l *LocalStorage) WriteFile(_ context.Context, name string, data []byte) er // to an empty file if write is not finished. tmpPath := filepath.Join(l.base, name) + ".tmp" if err := os.WriteFile(tmpPath, data, localFilePerm); err != nil { - return errors.Trace(err) + path := filepath.Dir(tmpPath) + log.Info("failed to write file, try to mkdir the path", zap.String("path", path)) + exists, existErr := pathExists(path) + if existErr != nil { + return errors.Annotatef(err, "after failed to write file, failed to check path exists : %v", existErr) + } + if exists { + return errors.Trace(err) + } + if mkdirErr := mkdirAll(path); mkdirErr != nil { + return errors.Annotatef(err, "after failed to write file, failed to mkdir : %v", mkdirErr) + } + if err := os.WriteFile(tmpPath, data, localFilePerm); err != nil { + return errors.Trace(err) + } } if err := os.Rename(tmpPath, filepath.Join(l.base, name)); err != nil { return errors.Trace(err) diff --git a/br/pkg/storage/local_test.go b/br/pkg/storage/local_test.go index 82e7435ae29be..db1ba424b9d6b 100644 --- a/br/pkg/storage/local_test.go +++ b/br/pkg/storage/local_test.go @@ -9,6 +9,7 @@ import ( "runtime" "testing" + "github.com/pingcap/errors" "github.com/stretchr/testify/require" ) @@ -99,4 +100,30 @@ func TestWalkDirWithSoftLinkFile(t *testing.T) { }) require.NoError(t, err) require.Equal(t, 1, i) + + // test file not exists + exists, err := store.FileExists(context.TODO(), "/123/456") + require.NoError(t, err) + require.False(t, exists) + + // test walk nonexistent directory + err = store.WalkDir(context.TODO(), &WalkOption{SubDir: "123/456"}, func(path string, size int64) error { + return errors.New("find file") + }) + require.NoError(t, err) + // write file to a nonexistent directory + err = store.WriteFile(context.TODO(), "/123/456/789.txt", []byte(data)) + require.NoError(t, err) + exists, err = store.FileExists(context.TODO(), "/123/456") + require.NoError(t, err) + require.True(t, exists) + + // test walk existent directory + err = store.WalkDir(context.TODO(), &WalkOption{SubDir: "123/456"}, func(path string, size int64) error { + if path == "123/456/789.txt" { + return nil + } + return errors.Errorf("find other file: %s", path) + }) + require.NoError(t, err) } diff --git a/br/pkg/summary/collector.go b/br/pkg/summary/collector.go index 705c26df3e4ac..1a16fb6dc9cfc 100644 --- a/br/pkg/summary/collector.go +++ b/br/pkg/summary/collector.go @@ -46,6 +46,10 @@ type LogCollector interface { SetSuccessStatus(success bool) + NowDureTime() time.Duration + + AdjustStartTimeToEarlierTime(t time.Duration) + Summary(name string) Log(msg string, fields ...zap.Field) @@ -163,6 +167,18 @@ func logKeyFor(key string) string { return strings.ReplaceAll(key, " ", "-") } +func (tc *logCollector) NowDureTime() time.Duration { + tc.mu.Lock() + defer tc.mu.Unlock() + return time.Since(tc.startTime) +} + +func (tc *logCollector) AdjustStartTimeToEarlierTime(t time.Duration) { + tc.mu.Lock() + defer tc.mu.Unlock() + tc.startTime = tc.startTime.Add(-t) +} + func (tc *logCollector) Summary(name string) { tc.mu.Lock() defer func() { diff --git a/br/pkg/summary/summary.go b/br/pkg/summary/summary.go index 7ae488785760e..45c8fbbc55997 100644 --- a/br/pkg/summary/summary.go +++ b/br/pkg/summary/summary.go @@ -43,6 +43,15 @@ func SetSuccessStatus(success bool) { collector.SetSuccessStatus(success) } +// NowDureTime returns the duration between start time and current time +func NowDureTime() time.Duration { + return collector.NowDureTime() +} + +func AdjustStartTimeToEarlierTime(t time.Duration) { + collector.AdjustStartTimeToEarlierTime(t) +} + // Summary outputs summary log. func Summary(name string) { collector.Summary(name) diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index dba72170649de..3402640c703b2 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -4,6 +4,8 @@ package task import ( "context" + "crypto/sha256" + "encoding/json" "fmt" "os" "strconv" @@ -26,6 +28,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics/handle" @@ -45,11 +48,13 @@ const ( flagRemoveSchedulers = "remove-schedulers" flagIgnoreStats = "ignore-stats" flagUseBackupMetaV2 = "use-backupmeta-v2" + flagUseCheckpoint = "use-checkpoint" flagGCTTL = "gcttl" defaultBackupConcurrency = 4 maxBackupConcurrency = 256 + checkpointDefaultGCTTL = 72 * 60 // 72 minutes ) const ( @@ -77,6 +82,7 @@ type BackupConfig struct { RemoveSchedulers bool `json:"remove-schedulers" toml:"remove-schedulers"` IgnoreStats bool `json:"ignore-stats" toml:"ignore-stats"` UseBackupMetaV2 bool `json:"use-backupmeta-v2"` + UseCheckpoint bool `json:"use-checkpoint" toml:"use-checkpoint"` CompressionConfig // for ebs-based backup @@ -126,6 +132,9 @@ func DefineBackupFlags(flags *pflag.FlagSet) { // but will generate v1 meta due to this flag is false. the behaviour is as same as v4.0.15, v4.0.16. // finally v4.0.17 will set this flag to true, and generate v2 meta. _ = flags.MarkHidden(flagUseBackupMetaV2) + + flags.Bool(flagUseCheckpoint, true, "use checkpoint mode") + _ = flags.MarkHidden(flagUseCheckpoint) } // ParseFromFlags parses the backup-related flags from the flag set. @@ -150,10 +159,21 @@ func (cfg *BackupConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return errors.Trace(err) } + noCheckpoint, err := flags.GetBool(flagUseCheckpoint) + if err != nil { + return errors.Trace(err) + } + cfg.UseCheckpoint = !noCheckpoint gcTTL, err := flags.GetInt64(flagGCTTL) if err != nil { return errors.Trace(err) } + // if use checkpoint and gcTTL is the default value + // update gcttl to checkpoint's default gc ttl + if cfg.UseCheckpoint && gcTTL == utils.DefaultBRGCSafePointTTL { + gcTTL = checkpointDefaultGCTTL + log.Info("use checkpoint's default GC TTL", zap.Int64("GC TTL", gcTTL)) + } cfg.GCTTL = gcTTL compressionCfg, err := parseCompressionFlags(flags) @@ -269,6 +289,23 @@ func (cfg *BackupConfig) Adjust() { } } +// a rough hash for checkpoint checker +func (cfg *BackupConfig) Hash() ([]byte, error) { + config := &BackupConfig{ + LastBackupTS: cfg.LastBackupTS, + IgnoreStats: cfg.IgnoreStats, + UseCheckpoint: cfg.UseCheckpoint, + Config: cfg.Config, + } + data, err := json.Marshal(config) + if err != nil { + return nil, errors.Trace(err) + } + hash := sha256.Sum256(data) + + return hash[:], nil +} + func isFullBackup(cmdName string) bool { return cmdName == FullBackupCmd } @@ -301,6 +338,12 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig return errors.Trace(err) } defer mgr.Close() + // after version check, check the cluster whether support checkpoint mode + if cfg.UseCheckpoint { + err = version.CheckCheckpointSupport() + log.Warn("unable to use checkpoint mode, fall back to normal mode", zap.Error(err)) + cfg.UseCheckpoint = false + } var statsHandle *handle.Handle if !skipStats { statsHandle = mgr.GetDomain().StatsHandle() @@ -321,6 +364,10 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig } client := backup.NewBackupClient(ctx, mgr) + + // set cipher only for checkpoint + client.SetCipher(&cfg.CipherInfo) + opts := storage.ExternalStorageOptions{ NoCredentials: cfg.NoCreds, SendCredentials: cfg.SendCreds, @@ -329,6 +376,16 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig if err = client.SetStorageAndCheckNotInUse(ctx, u, &opts); err != nil { return errors.Trace(err) } + // if checkpoint mode is unused at this time but there is checkpoint meta, + // CheckCheckpoint will stop backing up + cfgHash, err := cfg.Hash() + if err != nil { + return errors.Trace(err) + } + err = client.CheckCheckpoint(cfgHash) + if err != nil { + return errors.Trace(err) + } err = client.SetLockFile(ctx) if err != nil { return errors.Trace(err) @@ -340,10 +397,11 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig return errors.Trace(err) } g.Record("BackupTS", backupTS) + safePointID := client.GetSafePointID() sp := utils.BRServiceSafePoint{ BackupTS: backupTS, TTL: client.GetGCTTL(), - ID: utils.MakeSafePointID(), + ID: safePointID, } // use lastBackupTS as safePoint if exists @@ -353,7 +411,26 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig } log.Info("current backup safePoint job", zap.Object("safePoint", sp)) - err = utils.StartServiceSafePointKeeper(ctx, mgr.GetPDClient(), sp) + cctx, gcSafePointKeeperCancel := context.WithCancel(ctx) + gcSafePointKeeperRemovable := false + defer func() { + // don't reset the gc-safe-point if checkpoint mode is used and backup is not finished + if cfg.UseCheckpoint && !gcSafePointKeeperRemovable { + return + } + log.Info("start to remove gc-safepoint keeper") + // close the gc safe point keeper at first + gcSafePointKeeperCancel() + // set the ttl to 0 to remove the gc-safe-point + sp.TTL = 0 + if err := utils.UpdateServiceSafePoint(ctx, mgr.GetPDClient(), sp); err != nil { + log.Warn("failed to update service safe point, backup may fail if gc triggered", + zap.Error(err), + ) + } + log.Info("finish removing gc-safepoint keeper") + }() + err = utils.StartServiceSafePointKeeper(cctx, mgr.GetPDClient(), sp) if err != nil { return errors.Trace(err) } @@ -392,7 +469,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig return errors.Trace(err) } - ranges, schemas, policies, err := backup.BuildBackupRangeAndSchema(mgr.GetStorage(), cfg.TableFilter, backupTS, isFullBackup(cmdName)) + ranges, schemas, policies, err := client.BuildBackupRangeAndSchema(mgr.GetStorage(), cfg.TableFilter, backupTS, isFullBackup(cmdName)) if err != nil { return errors.Trace(err) } @@ -500,6 +577,18 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig }) } } + + if cfg.UseCheckpoint { + if err = client.StartCheckpointRunner(ctx, cfgHash, backupTS, ranges, safePointID, progressCallBack); err != nil { + return errors.Trace(err) + } + defer func() { + if !gcSafePointKeeperRemovable { + log.Info("wait for flush checkpoint...") + client.WaitForFinishCheckpoint() + } + }() + } metawriter.StartWriteMetasAsync(ctx, metautil.AppendDataFile) err = client.BackupRanges(ctx, ranges, req, uint(cfg.Concurrency), metawriter, progressCallBack) if err != nil { @@ -529,7 +618,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig schemasConcurrency := uint(mathutil.Min(backup.DefaultSchemaConcurrency, schemas.Len())) err = schemas.BackupSchemas( - ctx, metawriter, mgr.GetStorage(), statsHandle, backupTS, schemasConcurrency, cfg.ChecksumConcurrency, skipChecksum, updateCh) + ctx, metawriter, client.GetCheckpointRunner(), mgr.GetStorage(), statsHandle, backupTS, schemasConcurrency, cfg.ChecksumConcurrency, skipChecksum, updateCh) if err != nil { return errors.Trace(err) } @@ -538,6 +627,9 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig if err != nil { return errors.Trace(err) } + // Since backupmeta is flushed on the external storage, + // we can remove the gc safepoint keeper + gcSafePointKeeperRemovable = true // Checksum has finished, close checksum progress. updateCh.Close() diff --git a/br/pkg/task/backup_raw.go b/br/pkg/task/backup_raw.go index 4d1d35d37f140..2b46347327501 100644 --- a/br/pkg/task/backup_raw.go +++ b/br/pkg/task/backup_raw.go @@ -213,9 +213,18 @@ func RunBackupRaw(c context.Context, g glue.Glue, cmdName string, cfg *RawKvConf CompressionLevel: cfg.CompressionLevel, CipherInfo: &cfg.CipherInfo, } + rg := rtree.Range{ + StartKey: backupRange.StartKey, + EndKey: backupRange.EndKey, + } + progressRange := &rtree.ProgressRange{ + Res: rtree.NewRangeTree(), + Incomplete: []rtree.Range{rg}, + Origin: rg, + } metaWriter := metautil.NewMetaWriter(client.GetStorage(), metautil.MetaFileSize, false, metautil.MetaFile, &cfg.CipherInfo) metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDataFile) - err = client.BackupRange(ctx, req, metaWriter, progressCallBack) + err = client.BackupRange(ctx, req, progressRange, metaWriter, progressCallBack) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 8006793ffdb89..fdcc728a9ce5f 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -424,7 +424,7 @@ func (s *streamMgr) backupFullSchemas(ctx context.Context, g glue.Glue) error { } schemasConcurrency := uint(mathutil.Min(backup.DefaultSchemaConcurrency, schemas.Len())) - err = schemas.BackupSchemas(ctx, metaWriter, s.mgr.GetStorage(), nil, + err = schemas.BackupSchemas(ctx, metaWriter, nil, s.mgr.GetStorage(), nil, s.cfg.StartTS, schemasConcurrency, 0, true, nil) if err != nil { return errors.Trace(err) diff --git a/br/pkg/version/version.go b/br/pkg/version/version.go index 2af5cc3491535..14a0201ad4325 100644 --- a/br/pkg/version/version.go +++ b/br/pkg/version/version.go @@ -33,7 +33,8 @@ var ( versionHash = regexp.MustCompile("-[0-9]+-g[0-9a-f]{7,}") - pitrSupportBatchKVFiles bool = true + checkpointSupportError error = nil + pitrSupportBatchKVFiles bool = true ) // NextMajorVersion returns the next major version. @@ -204,6 +205,14 @@ func CheckVersionForBR(s *metapb.Store, tikvVersion *semver.Version) error { } } + // reset the checkpoint support error + checkpointSupportError = nil + if tikvVersion.Major < 6 || (tikvVersion.Major == 6 && tikvVersion.Minor < 5) { + // checkpoint mode only support after v6.5.0 + checkpointSupportError = errors.Annotatef(berrors.ErrVersionMismatch, "TiKV node %s version %s is too low when use checkpoint, please update tikv's version to at least v6.5.0", + s.Address, tikvVersion) + } + // don't warn if we are the master build, which always have the version v4.0.0-beta.2-* if build.GitBranch != "master" && tikvVersion.Compare(*BRVersion) > 0 { log.Warn(fmt.Sprintf("BR version is outdated, please consider use version %s of BR", tikvVersion)) @@ -311,6 +320,10 @@ func FetchVersion(ctx context.Context, db utils.QueryExecutor) (string, error) { return versionInfo, nil } +func CheckCheckpointSupport() error { + return checkpointSupportError +} + func CheckPITRSupportBatchKVFiles() bool { return pitrSupportBatchKVFiles } diff --git a/br/pkg/version/version_test.go b/br/pkg/version/version_test.go index 8871e0ae9f04e..96c8d9a306ef7 100644 --- a/br/pkg/version/version_test.go +++ b/br/pkg/version/version_test.go @@ -228,6 +228,29 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) require.NoError(t, err) + require.Error(t, CheckCheckpointSupport()) + } + + { + build.ReleaseVersion = "v6.0.0-rc.2" + mock.getAllStores = func() []*metapb.Store { + // TiKV v6.0.0-rc.1 with BR v6.0.0-rc.2 is ok + return []*metapb.Store{{Version: "v6.0.0-rc.1"}} + } + err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) + require.NoError(t, err) + require.Error(t, CheckCheckpointSupport()) + } + + { + build.ReleaseVersion = "v6.5.0-rc.2" + mock.getAllStores = func() []*metapb.Store { + // TiKV v6.5.0-rc.1 with BR v6.5.0-rc.2 is ok + return []*metapb.Store{{Version: "v6.5.0-rc.1"}} + } + err := CheckClusterVersion(context.Background(), &mock, CheckVersionForBR) + require.NoError(t, err) + require.NoError(t, CheckCheckpointSupport()) } { From 328aef88b04cda6d3660a44604ca19b1ad57051f Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 30 Nov 2022 13:17:59 +0800 Subject: [PATCH 11/13] lightning: add precheck about downstream CDC/PiTR (#39338) close pingcap/tidb#39346 --- Makefile | 2 +- br/pkg/lightning/common/errors.go | 1 + br/pkg/lightning/restore/BUILD.bazel | 7 + br/pkg/lightning/restore/check_info.go | 7 + br/pkg/lightning/restore/precheck.go | 3 + br/pkg/lightning/restore/precheck_impl.go | 154 ++++++++++++++++++ .../lightning/restore/precheck_impl_test.go | 87 ++++++++++ br/pkg/lightning/restore/restore.go | 4 + br/pkg/utils/db.go | 5 +- errors.toml | 5 + 10 files changed, 272 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 2f2dba4b010f0..030944e263418 100644 --- a/Makefile +++ b/Makefile @@ -442,7 +442,7 @@ bazel_golangcilinter: -- run $$($(PACKAGE_DIRECTORIES)) --config ./.golangci.yaml bazel_brietest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/brietest/... bazel_pessimistictest: failpoint-enable bazel_ci_prepare diff --git a/br/pkg/lightning/common/errors.go b/br/pkg/lightning/common/errors.go index c2fc3dbaa901f..1b23ff99fc1b4 100644 --- a/br/pkg/lightning/common/errors.go +++ b/br/pkg/lightning/common/errors.go @@ -51,6 +51,7 @@ var ( ErrCheckTableEmpty = errors.Normalize("check table empty error", errors.RFCCodeText("Lightning:PreCheck:ErrCheckTableEmpty")) ErrCheckCSVHeader = errors.Normalize("check csv header error", errors.RFCCodeText("Lightning:PreCheck:ErrCheckCSVHeader")) ErrCheckDataSource = errors.Normalize("check data source error", errors.RFCCodeText("Lightning:PreCheck:ErrCheckDataSource")) + ErrCheckCDCPiTR = errors.Normalize("check TiCDC/PiTR task error", errors.RFCCodeText("Lightning:PreCheck:ErrCheckCDCPiTR")) ErrOpenCheckpoint = errors.Normalize("open checkpoint error", errors.RFCCodeText("Lightning:Checkpoint:ErrOpenCheckpoint")) ErrReadCheckpoint = errors.Normalize("read checkpoint error", errors.RFCCodeText("Lightning:Checkpoint:ErrReadCheckpoint")) diff --git a/br/pkg/lightning/restore/BUILD.bazel b/br/pkg/lightning/restore/BUILD.bazel index 175e69e22d02f..ef5aeb106585b 100644 --- a/br/pkg/lightning/restore/BUILD.bazel +++ b/br/pkg/lightning/restore/BUILD.bazel @@ -39,6 +39,7 @@ go_library( "//br/pkg/pdutil", "//br/pkg/redact", "//br/pkg/storage", + "//br/pkg/streamhelper", "//br/pkg/utils", "//br/pkg/version", "//br/pkg/version/build", @@ -77,6 +78,9 @@ go_library( "@com_github_pingcap_tipb//go-tipb", "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_pd_client//:client", + "@io_etcd_go_etcd_client_v3//:client", + "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//keepalive", "@org_golang_x_exp//maps", "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", @@ -124,6 +128,7 @@ go_test( "//br/pkg/lightning/worker", "//br/pkg/mock", "//br/pkg/storage", + "//br/pkg/streamhelper", "//br/pkg/version/build", "//ddl", "//errno", @@ -158,6 +163,8 @@ go_test( "@com_github_tikv_pd_client//:client", "@com_github_xitongsys_parquet_go//writer", "@com_github_xitongsys_parquet_go_source//buffer", + "@io_etcd_go_etcd_client_v3//:client", + "@io_etcd_go_etcd_tests_v3//integration", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index cc4b3b734ebaa..aab9e5ebacef5 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -155,3 +155,10 @@ func (rc *Controller) checkSourceSchema(ctx context.Context) error { } return rc.doPreCheckOnItem(ctx, CheckSourceSchemaValid) } + +func (rc *Controller) checkCDCPiTR(ctx context.Context) error { + if rc.cfg.TikvImporter.Backend == config.BackendTiDB { + return nil + } + return rc.doPreCheckOnItem(ctx, CheckTargetUsingCDCPITR) +} diff --git a/br/pkg/lightning/restore/precheck.go b/br/pkg/lightning/restore/precheck.go index 7dc578053492d..a76854556a165 100644 --- a/br/pkg/lightning/restore/precheck.go +++ b/br/pkg/lightning/restore/precheck.go @@ -25,6 +25,7 @@ const ( CheckTargetClusterVersion CheckItemID = "CHECK_TARGET_CLUSTER_VERSION" CheckLocalDiskPlacement CheckItemID = "CHECK_LOCAL_DISK_PLACEMENT" CheckLocalTempKVDir CheckItemID = "CHECK_LOCAL_TEMP_KV_DIR" + CheckTargetUsingCDCPITR CheckItemID = "CHECK_TARGET_USING_CDC_PITR" ) type CheckResult struct { @@ -139,6 +140,8 @@ func (b *PrecheckItemBuilder) BuildPrecheckItem(checkID CheckItemID) (PrecheckIt return NewLocalDiskPlacementCheckItem(b.cfg), nil case CheckLocalTempKVDir: return NewLocalTempKVDirCheckItem(b.cfg, b.preInfoGetter), nil + case CheckTargetUsingCDCPITR: + return NewCDCPITRCheckItem(b.cfg), nil default: return nil, errors.Errorf("unsupported check item: %v", checkID) } diff --git a/br/pkg/lightning/restore/precheck_impl.go b/br/pkg/lightning/restore/precheck_impl.go index 64e15d5aa828a..b3c3c2fc00f25 100644 --- a/br/pkg/lightning/restore/precheck_impl.go +++ b/br/pkg/lightning/restore/precheck_impl.go @@ -14,6 +14,7 @@ package restore import ( + "bytes" "context" "fmt" "path/filepath" @@ -21,6 +22,7 @@ import ( "strconv" "strings" "sync" + "time" "github.com/docker/go-units" "github.com/pingcap/errors" @@ -32,6 +34,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/store/pdtypes" @@ -40,9 +43,12 @@ import ( "github.com/pingcap/tidb/util/engine" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/set" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) type clusterResourceCheckItem struct { @@ -672,6 +678,154 @@ func (ci *checkpointCheckItem) checkpointIsValid(ctx context.Context, tableInfo return msgs, nil } +// CDCPITRCheckItem check downstream has enabled CDC or PiTR. It's exposed to let +// caller override the Instruction message. +type CDCPITRCheckItem struct { + cfg *config.Config + Instruction string + // used in test + etcdCli *clientv3.Client +} + +// NewCDCPITRCheckItem creates a checker to check downstream has enabled CDC or PiTR. +func NewCDCPITRCheckItem(cfg *config.Config) PrecheckItem { + return &CDCPITRCheckItem{ + cfg: cfg, + Instruction: "local backend is not compatible with them. Please switch to tidb backend then try again.", + } +} + +// GetCheckItemID implements PrecheckItem interface. +func (ci *CDCPITRCheckItem) GetCheckItemID() CheckItemID { + return CheckTargetUsingCDCPITR +} + +func dialEtcdWithCfg(ctx context.Context, cfg *config.Config) (*clientv3.Client, error) { + cfg2, err := cfg.ToTLS() + if err != nil { + return nil, err + } + tlsConfig := cfg2.TLSConfig() + + return clientv3.New(clientv3.Config{ + TLS: tlsConfig, + Endpoints: []string{cfg.TiDB.PdAddr}, + AutoSyncInterval: 30 * time.Second, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, + Timeout: 3 * time.Second, + PermitWithoutStream: false, + }), + grpc.WithBlock(), + grpc.WithReturnConnectionError(), + }, + Context: ctx, + }) +} + +// Check implements PrecheckItem interface. +func (ci *CDCPITRCheckItem) Check(ctx context.Context) (*CheckResult, error) { + theResult := &CheckResult{ + Item: ci.GetCheckItemID(), + Severity: Critical, + } + + if ci.cfg.TikvImporter.Backend != config.BackendLocal { + theResult.Passed = true + theResult.Message = "TiDB Lightning is not using local backend, skip this check" + return theResult, nil + } + + if ci.etcdCli == nil { + var err error + ci.etcdCli, err = dialEtcdWithCfg(ctx, ci.cfg) + if err != nil { + return nil, errors.Trace(err) + } + //nolint: errcheck + defer ci.etcdCli.Close() + } + + errorMsg := make([]string, 0, 2) + + pitrCli := streamhelper.NewMetaDataClient(ci.etcdCli) + tasks, err := pitrCli.GetAllTasks(ctx) + if err != nil { + return nil, errors.Trace(err) + } + if len(tasks) > 0 { + names := make([]string, 0, len(tasks)) + for _, task := range tasks { + names = append(names, task.Info.GetName()) + } + errorMsg = append(errorMsg, fmt.Sprintf("found PiTR log streaming task(s): %v,", names)) + } + + // check etcd KV of CDC >= v6.2 + cdcPrefix := "/tidb/cdc/" + capturePath := []byte("/__cdc_meta__/capture/") + nameSet := make(map[string][]string, 1) + resp, err := ci.etcdCli.Get(ctx, cdcPrefix, clientv3.WithPrefix(), clientv3.WithKeysOnly()) + if err != nil { + return nil, errors.Trace(err) + } + for _, kv := range resp.Kvs { + // example: /tidb/cdc//__cdc_meta__/capture/ + k := kv.Key[len(cdcPrefix):] + clusterID, captureID, found := bytes.Cut(k, capturePath) + if found { + nameSet[string(clusterID)] = append(nameSet[string(clusterID)], string(captureID)) + } + } + if len(nameSet) == 0 { + // check etcd KV of CDC <= v6.1 + cdcPrefixV61 := "/tidb/cdc/capture/" + resp, err = ci.etcdCli.Get(ctx, cdcPrefixV61, clientv3.WithPrefix(), clientv3.WithKeysOnly()) + if err != nil { + return nil, errors.Trace(err) + } + for _, kv := range resp.Kvs { + // example: /tidb/cdc/capture/ + k := kv.Key[len(cdcPrefixV61):] + if len(k) == 0 { + continue + } + nameSet[""] = append(nameSet[""], string(k)) + } + } + + if len(nameSet) > 0 { + var captureMsgBuf strings.Builder + captureMsgBuf.WriteString("found CDC capture(s): ") + isFirst := true + for clusterID, captureIDs := range nameSet { + if !isFirst { + captureMsgBuf.WriteString(", ") + } + isFirst = false + captureMsgBuf.WriteString("clusterID: ") + captureMsgBuf.WriteString(clusterID) + captureMsgBuf.WriteString(" captureID(s): ") + captureMsgBuf.WriteString(fmt.Sprintf("%v", captureIDs)) + } + captureMsgBuf.WriteString(",") + errorMsg = append(errorMsg, captureMsgBuf.String()) + } + + if len(errorMsg) > 0 { + errorMsg = append(errorMsg, ci.Instruction) + theResult.Passed = false + theResult.Message = strings.Join(errorMsg, "\n") + } else { + theResult.Passed = true + theResult.Message = "no CDC or PiTR task found" + } + + return theResult, nil +} + type schemaCheckItem struct { cfg *config.Config preInfoGetter PreRestoreInfoGetter diff --git a/br/pkg/lightning/restore/precheck_impl_test.go b/br/pkg/lightning/restore/precheck_impl_test.go index 88f3cf8f9a30b..2811937a71c82 100644 --- a/br/pkg/lightning/restore/precheck_impl_test.go +++ b/br/pkg/lightning/restore/precheck_impl_test.go @@ -24,7 +24,11 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/restore/mock" ropts "github.com/pingcap/tidb/br/pkg/lightning/restore/opts" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" ) type precheckImplSuite struct { @@ -581,3 +585,86 @@ func (s *precheckImplSuite) TestTableEmptyCheckBasic() { s.T().Logf("check result message: %s", result.Message) s.Require().False(result.Passed) } + +func (s *precheckImplSuite) TestCDCPITRCheckItem() { + integration.BeforeTestExternal(s.T()) + testEtcdCluster := integration.NewClusterV3(s.T(), &integration.ClusterConfig{Size: 1}) + defer testEtcdCluster.Terminate(s.T()) + + ctx := context.Background() + cfg := &config.Config{ + TikvImporter: config.TikvImporter{ + Backend: config.BackendLocal, + }, + } + ci := NewCDCPITRCheckItem(cfg) + checker := ci.(*CDCPITRCheckItem) + checker.etcdCli = testEtcdCluster.RandClient() + result, err := ci.Check(ctx) + s.Require().NoError(err) + s.Require().NotNil(result) + s.Require().Equal(ci.GetCheckItemID(), result.Item) + s.Require().Equal(Critical, result.Severity) + s.Require().True(result.Passed) + s.Require().Equal("no CDC or PiTR task found", result.Message) + + cli := testEtcdCluster.RandClient() + brCli := streamhelper.NewMetaDataClient(cli) + backend, _ := storage.ParseBackend("noop://", nil) + taskInfo, err := streamhelper.NewTaskInfo("br_name"). + FromTS(1). + UntilTS(1000). + WithTableFilter("*.*", "!mysql"). + ToStorage(backend). + Check() + s.Require().NoError(err) + err = brCli.PutTask(ctx, *taskInfo) + s.Require().NoError(err) + checkEtcdPut := func(key string) { + _, err := cli.Put(ctx, key, "") + s.Require().NoError(err) + } + // TiCDC >= v6.2 + checkEtcdPut("/tidb/cdc/default/__cdc_meta__/capture/3ecd5c98-0148-4086-adfd-17641995e71f") + checkEtcdPut("/tidb/cdc/default/__cdc_meta__/meta/meta-version") + checkEtcdPut("/tidb/cdc/default/__cdc_meta__/meta/ticdc-delete-etcd-key-count") + checkEtcdPut("/tidb/cdc/default/__cdc_meta__/owner/22318498f4dd6639") + checkEtcdPut("/tidb/cdc/default/default/changefeed/info/test") + checkEtcdPut("/tidb/cdc/default/default/changefeed/info/test-1") + checkEtcdPut("/tidb/cdc/default/default/changefeed/status/test") + checkEtcdPut("/tidb/cdc/default/default/changefeed/status/test-1") + checkEtcdPut("/tidb/cdc/default/default/task/position/3ecd5c98-0148-4086-adfd-17641995e71f/test-1") + checkEtcdPut("/tidb/cdc/default/default/upstream/7168358383033671922") + + result, err = ci.Check(ctx) + s.Require().NoError(err) + s.Require().False(result.Passed) + s.Require().Equal("found PiTR log streaming task(s): [br_name],\n"+ + "found CDC capture(s): clusterID: default captureID(s): [3ecd5c98-0148-4086-adfd-17641995e71f],\n"+ + "local backend is not compatible with them. Please switch to tidb backend then try again.", + result.Message) + + _, err = cli.Delete(ctx, "/tidb/cdc/", clientv3.WithPrefix()) + s.Require().NoError(err) + + // TiCDC <= v6.1 + checkEtcdPut("/tidb/cdc/capture/f14cb04d-5ba1-410e-a59b-ccd796920e9d") + checkEtcdPut("/tidb/cdc/changefeed/info/test") + checkEtcdPut("/tidb/cdc/job/test") + checkEtcdPut("/tidb/cdc/owner/223184ad80a88b0b") + checkEtcdPut("/tidb/cdc/task/position/f14cb04d-5ba1-410e-a59b-ccd796920e9d/test") + + result, err = ci.Check(ctx) + s.Require().NoError(err) + s.Require().False(result.Passed) + s.Require().Equal("found PiTR log streaming task(s): [br_name],\n"+ + "found CDC capture(s): clusterID: captureID(s): [f14cb04d-5ba1-410e-a59b-ccd796920e9d],\n"+ + "local backend is not compatible with them. Please switch to tidb backend then try again.", + result.Message) + + checker.cfg.TikvImporter.Backend = config.BackendTiDB + result, err = ci.Check(ctx) + s.Require().NoError(err) + s.Require().True(result.Passed) + s.Require().Equal("TiDB Lightning is not using local backend, skip this check", result.Message) +} diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 210435640473f..380642774fe55 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -2136,6 +2136,10 @@ func (rc *Controller) preCheckRequirements(ctx context.Context) error { return common.ErrCheckClusterRegion.Wrap(err).GenWithStackByArgs() } } + // even if checkpoint exists, we still need to make sure CDC/PiTR task is not running. + if err := rc.checkCDCPiTR(ctx); err != nil { + return common.ErrCheckCDCPiTR.Wrap(err).GenWithStackByArgs() + } } } diff --git a/br/pkg/utils/db.go b/br/pkg/utils/db.go index 57c0d5a1e280f..6cc9f82a1229a 100644 --- a/br/pkg/utils/db.go +++ b/br/pkg/utils/db.go @@ -98,14 +98,14 @@ func IsLogBackupEnabled(ctx sqlexec.RestrictedSQLExecutor) (bool, error) { return true, nil } -// CheckLogBackupTaskExist increases the count of log backup task. +// LogBackupTaskCountInc increases the count of log backup task. func LogBackupTaskCountInc() { LogBackupTaskMutex.Lock() logBackupTaskCount++ LogBackupTaskMutex.Unlock() } -// CheckLogBackupTaskExist decreases the count of log backup task. +// LogBackupTaskCountDec decreases the count of log backup task. func LogBackupTaskCountDec() { LogBackupTaskMutex.Lock() logBackupTaskCount-- @@ -122,6 +122,7 @@ func IsLogBackupInUse(ctx sessionctx.Context) bool { return CheckLogBackupEnabled(ctx) && CheckLogBackupTaskExist() } +// GetTidbNewCollationEnabled returns the variable name of NewCollationEnabled. func GetTidbNewCollationEnabled() string { return tidbNewCollationEnabled } diff --git a/errors.toml b/errors.toml index df952cc7af45a..191a7189f95ce 100644 --- a/errors.toml +++ b/errors.toml @@ -461,6 +461,11 @@ error = ''' update pd error ''' +["Lightning:PreCheck:ErrCheckCDCPiTR"] +error = ''' +check TiCDC/PiTR task error +''' + ["Lightning:PreCheck:ErrCheckCSVHeader"] error = ''' check csv header error From 40f0bd5255bac0c9d5cc8dc46a584a83ee38300a Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 30 Nov 2022 13:42:00 +0800 Subject: [PATCH 12/13] executor: add index_merge_intersection cases (#39323) --- executor/index_merge_reader_test.go | 212 ++++++++++++++++++++++++++++ testkit/testutil/require.go | 12 ++ 2 files changed, 224 insertions(+) diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 4d719adbdf781..79d2d8b895a81 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -23,7 +23,9 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testutil" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" ) @@ -242,32 +244,52 @@ func TestIndexMergeInTransaction(t *testing.T) { "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", "└─Selection_8(Probe) 1106.67 cop[tikv] lt(test.t1.c3, 10)", " └─TableRowIDScan_7 3330.01 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("explain select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where c1 < 10 and c2 < 10 and c3 < 10;").Check(testkit.Rows( + "IndexMerge_9 367.05 root type: intersection", + "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", + "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + "├─IndexRangeScan_7(Build) 3323.33 cop[tikv] table:t1, index:c3(c3) range:[-inf,10), keep order:false, stats:pseudo", + "└─TableRowIDScan_8(Probe) 367.05 cop[tikv] table:t1 keep order:false, stats:pseudo")) // Test with normal key. tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < 10 and c2 < -1) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < -1 and c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("insert into t1 values(1, 1, 1, 1);") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < 10 and c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < 10 and c2 < 10) and c3 > 10;").Check(testkit.Rows()) + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < 10 and c2 < 10) and c3 > 10;").Check(testkit.Rows("1 1 100 1")) + tk.MustExec("delete from t1;") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c1, c2, c3) */ * from t1 where (c1 < 10 and c2 < 10) and c3 > 10;").Check(testkit.Rows()) // Test with primary key, so the partialPlan is TableScan. tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c2, c3, primary) */ * from t1 where (pk < -1 and c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c2, c3, primary) */ * from t1 where (pk < 10 and c2 < -1) and c3 < 10;").Check(testkit.Rows()) tk.MustExec("insert into t1 values(1, 1, 1, 1);") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) + tk.MustQuery("select /*+ use_index_merge(t1, c2, c3, primary) */ * from t1 where (pk < 10 and c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) tk.MustExec("update t1 set c3 = 100 where c3 = 1;") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c2, c3, primary) */ * from t1 where (pk < 10 and c2 < 10) and c3 > 10;").Check(testkit.Rows("1 1 100 1")) tk.MustExec("delete from t1;") tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < -1 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < -1) and c3 < 10;").Check(testkit.Rows()) + tk.MustQuery("select /*+ use_index_merge(t1, c2, c3, primary) */ * from t1 where (pk < 10 and c2 < 10) and c3 > 10;").Check(testkit.Rows()) tk.MustExec("commit;") if i == 1 { @@ -566,3 +588,193 @@ func TestPessimisticLockOnPartitionForIndexMerge(t *testing.T) { // TODO: add support for index merge reader in dynamic tidb_partition_prune_mode } + +func TestIndexMergeIntersectionConcurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int, c2 bigint, c3 bigint, primary key(c1), key(c2), key(c3)) partition by hash(c1) partitions 10;") + tk.MustExec("insert into t1 values(1, 1, 3000), (2, 1, 1)") + tk.MustExec("analyze table t1;") + tk.MustExec("set tidb_partition_prune_mode = 'dynamic'") + res := tk.MustQuery("explain select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Rows() + require.Contains(t, res[1][0], "IndexMerge") + + // Default is tidb_executor_concurrency. + res = tk.MustQuery("select @@tidb_executor_concurrency;").Sort().Rows() + defExecCon := res[0][0].(string) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", fmt.Sprintf("return(%s)", defExecCon))) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency")) + }() + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + + tk.MustExec("set tidb_executor_concurrency = 10") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(10)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + // workerCnt = min(part_num, concurrency) + tk.MustExec("set tidb_executor_concurrency = 20") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(10)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + tk.MustExec("set tidb_executor_concurrency = 2") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(2)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + + tk.MustExec("set tidb_index_merge_intersection_concurrency = 9") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(9)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + tk.MustExec("set tidb_index_merge_intersection_concurrency = 21") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(10)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + tk.MustExec("set tidb_index_merge_intersection_concurrency = 3") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(3)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + + // Concurrency only works for dynamic pruning partition table, so real concurrency is 1. + tk.MustExec("set tidb_partition_prune_mode = 'static'") + tk.MustExec("set tidb_index_merge_intersection_concurrency = 9") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(1)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) + + // Concurrency only works for dynamic pruning partition table. so real concurrency is 1. + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int, c2 bigint, c3 bigint, primary key(c1), key(c2), key(c3));") + tk.MustExec("insert into t1 values(1, 1, 3000), (2, 1, 1)") + tk.MustExec("set tidb_index_merge_intersection_concurrency = 9") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionConcurrency", "return(1)")) + tk.MustQuery("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Check(testkit.Rows("1")) +} + +func TestIntersectionWithDifferentConcurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + var execCon []int + tblSchemas := []string{ + // partition table + "create table t1(c1 int, c2 bigint, c3 bigint, primary key(c1), key(c2), key(c3)) partition by hash(c1) partitions 10;", + // non-partition table + "create table t1(c1 int, c2 bigint, c3 bigint, primary key(c1), key(c2), key(c3));", + } + + for tblIdx, tblSchema := range tblSchemas { + if tblIdx == 0 { + // Test different intersectionProcessWorker with partition table(10 partitions). + execCon = []int{1, 3, 10, 11, 20} + } else { + // Default concurrency. + execCon = []int{5} + } + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec(tblSchema) + + const queryCnt int = 10 + const rowCnt int = 1000 + curRowCnt := 0 + insertStr := "insert into t1 values" + for i := 0; i < rowCnt; i++ { + if i != 0 { + insertStr += ", " + } + insertStr += fmt.Sprintf("(%d, %d, %d)", i, rand.Int(), rand.Int()) + curRowCnt++ + } + tk.MustExec(insertStr) + tk.MustExec("analyze table t1") + + for _, concurrency := range execCon { + tk.MustExec(fmt.Sprintf("set tidb_executor_concurrency = %d", concurrency)) + for i := 0; i < 2; i++ { + if i == 0 { + // Dynamic mode. + tk.MustExec("set tidb_partition_prune_mode = 'dynamic'") + res := tk.MustQuery("explain select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024") + require.Contains(t, res.Rows()[1][0], "IndexMerge") + } else { + tk.MustExec("set tidb_partition_prune_mode = 'static'") + res := tk.MustQuery("explain select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024") + if tblIdx == 0 { + // partition table + require.Contains(t, res.Rows()[1][0], "PartitionUnion") + require.Contains(t, res.Rows()[2][0], "IndexMerge") + } else { + require.Contains(t, res.Rows()[1][0], "IndexMerge") + } + } + for i := 0; i < queryCnt; i++ { + c3 := rand.Intn(1024) + res := tk.MustQuery(fmt.Sprintf("select /*+ no_index_merge() */ c1 from t1 where c2 < 1024 and c3 > %d", c3)).Sort().Rows() + tk.MustQuery(fmt.Sprintf("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > %d", c3)).Sort().Check(res) + } + + // In tranaction + for i := 0; i < queryCnt; i++ { + tk.MustExec("begin;") + r := rand.Intn(3) + if r == 0 { + tk.MustExec(fmt.Sprintf("update t1 set c3 = %d where c1 = %d", rand.Int(), rand.Intn(rowCnt))) + } else if r == 1 { + tk.MustExec(fmt.Sprintf("delete from t1 where c1 = %d", rand.Intn(rowCnt))) + } else if r == 2 { + tk.MustExec(fmt.Sprintf("insert into t1 values(%d, %d, %d)", curRowCnt, rand.Int(), rand.Int())) + curRowCnt++ + } + c3 := rand.Intn(1024) + res := tk.MustQuery(fmt.Sprintf("select /*+ no_index_merge() */ c1 from t1 where c2 < 1024 and c3 > %d", c3)).Sort().Rows() + tk.MustQuery(fmt.Sprintf("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > %d", c3)).Sort().Check(res) + tk.MustExec("commit;") + } + } + } + tk.MustExec("drop table t1") + } +} + +func TestIntersectionWorkerPanic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int, c2 bigint, c3 bigint, primary key(c1), key(c2), key(c3)) partition by hash(c1) partitions 10;") + tk.MustExec("insert into t1 values(1, 1, 3000), (2, 1, 1)") + tk.MustExec("analyze table t1;") + tk.MustExec("set tidb_partition_prune_mode = 'dynamic'") + res := tk.MustQuery("explain select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024").Rows() + require.Contains(t, res[1][0], "IndexMerge") + + // Test panic in intersection. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionWorkerPanic", "panic")) + err := tk.QueryToErr("select /*+ use_index_merge(t1, primary, c2, c3) */ c1 from t1 where c2 < 1024 and c3 > 1024") + require.Contains(t, err.Error(), "IndexMergeReaderExecutor") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIndexMergeIntersectionWorkerPanic")) +} + +func TestIntersectionMemQuota(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(pk varchar(100) primary key, c1 int, c2 int, index idx1(c1), index idx2(c2))") + + insertStr := "insert into t1 values" + for i := 0; i < 20; i++ { + if i != 0 { + insertStr += ", " + } + insertStr += fmt.Sprintf("('%s', %d, %d)", testutil.RandStringRunes(100), 1, 1) + } + tk.MustExec(insertStr) + res := tk.MustQuery("explain select /*+ use_index_merge(t1, primary, idx1, idx2) */ c1 from t1 where c1 < 1024 and c2 < 1024").Rows() + require.Contains(t, res[1][0], "IndexMerge") + + tk.MustExec("set global tidb_mem_oom_action='CANCEL'") + defer tk.MustExec("set global tidb_mem_oom_action = DEFAULT") + tk.MustExec("set @@tidb_mem_quota_query = 4000") + err := tk.QueryToErr("select /*+ use_index_merge(t1, primary, idx1, idx2) */ c1 from t1 where c1 < 1024 and c2 < 1024") + require.Contains(t, err.Error(), "Out Of Memory Quota!") +} diff --git a/testkit/testutil/require.go b/testkit/testutil/require.go index 90b157fcb7591..09e8e871312ae 100644 --- a/testkit/testutil/require.go +++ b/testkit/testutil/require.go @@ -17,6 +17,7 @@ package testutil import ( + "math/rand" "testing" "github.com/pingcap/tidb/kv" @@ -75,3 +76,14 @@ func CompareUnorderedStringSlice(a []string, b []string) bool { } return len(m) == 0 } + +var letterRunes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") + +// RandStringRunes generate random string of length n. +func RandStringRunes(n int) string { + b := make([]rune, n) + for i := range b { + b[i] = letterRunes[rand.Intn(len(letterRunes))] + } + return string(b) +} From a48fe7913dc9e3b2d66130ba8369abd7b6a84883 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 30 Nov 2022 13:57:59 +0800 Subject: [PATCH 13/13] executor: close resultset (#39474) --- Makefile | 12 ++--- executor/ddl_test.go | 109 ++++++++++++++----------------------------- 2 files changed, 42 insertions(+), 79 deletions(-) diff --git a/Makefile b/Makefile index 030944e263418..66b3ba0686917 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) --remote_download_minimal \ + bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \ //... --//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 @@ -446,23 +446,23 @@ bazel_brietest: failpoint-enable bazel_ci_prepare -- //tests/realtikvtest/brietest/... bazel_pessimistictest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/pessimistictest/... bazel_sessiontest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/sessiontest/... bazel_statisticstest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/statisticstest/... bazel_txntest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/txntest/... bazel_addindextest: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \ + bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \ -- //tests/realtikvtest/addindextest/... bazel_lint: bazel_prepare diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 3c97d87f1a280..d4e9ed4c1b425 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -79,10 +79,8 @@ func TestInTxnExecDDLFail(t *testing.T) { tk.MustExec("insert into t values (1);") tk.MustExec("begin;") tk.MustExec("insert into t values (1);") - _, err := tk.Exec("truncate table t;") - require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") - result := tk.MustQuery("select count(*) from t") - result.Check(testkit.Rows("1")) + tk.MustGetErrMsg("truncate table t;", "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tk.MustQuery("select count(*) from t").Check(testkit.Rows("1")) } func TestInTxnExecDDLInvalid(t *testing.T) { @@ -212,11 +210,9 @@ func TestCreateView(t *testing.T) { // test create a exist view tk.MustExec("CREATE VIEW view_t AS select id , name from source_table") defer tk.MustExec("DROP VIEW IF EXISTS view_t") - _, err := tk.Exec("CREATE VIEW view_t AS select id , name from source_table") - require.EqualError(t, err, "[schema:1050]Table 'test.view_t' already exists") + tk.MustGetErrMsg("CREATE VIEW view_t AS select id , name from source_table", "[schema:1050]Table 'test.view_t' already exists") // create view on nonexistent table - _, err = tk.Exec("create view v1 (c,d) as select a,b from t1") - require.EqualError(t, err, "[schema:1146]Table 'test.t1' doesn't exist") + tk.MustGetErrMsg("create view v1 (c,d) as select a,b from t1", "[schema:1146]Table 'test.t1' doesn't exist") // simple view tk.MustExec("create table t1 (a int ,b int)") tk.MustExec("insert into t1 values (1,2), (1,3), (2,4), (2,5), (3,10)") @@ -231,26 +227,22 @@ func TestCreateView(t *testing.T) { // view with select wild card tk.MustExec("create view v5 as select * from t1") tk.MustExec("create view v6 (c,d) as select * from t1") - _, err = tk.Exec("create view v7 (c,d,e) as select * from t1") - require.Equal(t, dbterror.ErrViewWrongList.Error(), err.Error()) + tk.MustGetErrCode("create view v7 (c,d,e) as select * from t1", errno.ErrViewWrongList) // drop multiple views in a statement tk.MustExec("drop view v1,v2,v3,v4,v5,v6") // view with variable tk.MustExec("create view v1 (c,d) as select a,b+@@global.max_user_connections from t1") - _, err = tk.Exec("create view v1 (c,d) as select a,b from t1 where a = @@global.max_user_connections") - require.EqualError(t, err, "[schema:1050]Table 'test.v1' already exists") + tk.MustGetErrMsg("create view v1 (c,d) as select a,b from t1 where a = @@global.max_user_connections", "[schema:1050]Table 'test.v1' already exists") tk.MustExec("drop view v1") // view with different col counts - _, err = tk.Exec("create view v1 (c,d,e) as select a,b from t1 ") - require.Equal(t, dbterror.ErrViewWrongList.Error(), err.Error()) - _, err = tk.Exec("create view v1 (c) as select a,b from t1 ") - require.Equal(t, dbterror.ErrViewWrongList.Error(), err.Error()) + tk.MustGetErrCode("create view v1 (c,d,e) as select a,b from t1 ", errno.ErrViewWrongList) + tk.MustGetErrCode("create view v1 (c) as select a,b from t1 ", errno.ErrViewWrongList) // view with or_replace flag tk.MustExec("drop view if exists v1") tk.MustExec("create view v1 (c,d) as select a,b from t1") tk.MustExec("create or replace view v1 (c,d) as select a,b from t1 ") tk.MustExec("create table if not exists t1 (a int ,b int)") - _, err = tk.Exec("create or replace view t1 as select * from t1") + err := tk.ExecToErr("create or replace view t1 as select * from t1") require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "t1", "VIEW").Error(), err.Error()) // create view using prepare tk.MustExec(`prepare stmt from "create view v10 (x) as select 1";`) @@ -259,8 +251,7 @@ func TestCreateView(t *testing.T) { // create view on union tk.MustExec("drop table if exists t1, t2") tk.MustExec("drop view if exists v") - _, err = tk.Exec("create view v as select * from t1 union select * from t2") - require.True(t, terror.ErrorEqual(err, infoschema.ErrTableNotExists)) + tk.MustGetDBError("create view v as select * from t1 union select * from t2", infoschema.ErrTableNotExists) tk.MustExec("create table t1(a int, b int)") tk.MustExec("create table t2(a int, b int)") tk.MustExec("insert into t1 values(1,2), (1,1), (1,2)") @@ -268,14 +259,12 @@ func TestCreateView(t *testing.T) { tk.MustExec("create definer='root'@'localhost' view v as select * from t1 union select * from t2") tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 2", "1 3")) tk.MustExec("alter table t1 drop column a") - _, err = tk.Exec("select * from v") - require.True(t, terror.ErrorEqual(err, plannercore.ErrViewInvalid)) + tk.MustGetDBError("select * from v", plannercore.ErrViewInvalid) tk.MustExec("alter table t1 add column a int") tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 3", " 1", " 2")) tk.MustExec("alter table t1 drop column a") tk.MustExec("alter table t2 drop column b") - _, err = tk.Exec("select * from v") - require.True(t, terror.ErrorEqual(err, plannercore.ErrViewInvalid)) + tk.MustGetDBError("select * from v", plannercore.ErrViewInvalid) tk.MustExec("drop view v") tk.MustExec("create view v as (select * from t1)") @@ -294,8 +283,7 @@ func TestCreateView(t *testing.T) { tk.MustExec("create table test_v_nested(a int)") tk.MustExec("create definer='root'@'localhost' view v_nested as select * from test_v_nested") tk.MustExec("create definer='root'@'localhost' view v_nested2 as select * from v_nested") - _, err = tk.Exec("create or replace definer='root'@'localhost' view v_nested as select * from v_nested2") - require.True(t, terror.ErrorEqual(err, plannercore.ErrNoSuchTable)) + tk.MustGetDBError("create or replace definer='root'@'localhost' view v_nested as select * from v_nested2", plannercore.ErrNoSuchTable) tk.MustExec("drop table test_v_nested") tk.MustExec("drop view v_nested, v_nested2") @@ -322,8 +310,7 @@ func TestViewRecursion(t *testing.T) { tk.MustExec("create definer='root'@'localhost' view recursive_view2 as select * from recursive_view1") tk.MustExec("drop table t") tk.MustExec("rename table recursive_view2 to t") - _, err := tk.Exec("select * from recursive_view1") - require.True(t, terror.ErrorEqual(err, plannercore.ErrViewRecursive)) + tk.MustGetDBError("select * from recursive_view1", plannercore.ErrViewRecursive) tk.MustExec("drop view recursive_view1, t") } @@ -333,8 +320,8 @@ func TestIssue16250(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table if not exists t(a int)") tk.MustExec("create view view_issue16250 as select * from t") - _, err := tk.Exec("truncate table view_issue16250") - require.EqualError(t, err, "[schema:1146]Table 'test.view_issue16250' doesn't exist") + tk.MustGetErrMsg("truncate table view_issue16250", + "[schema:1146]Table 'test.view_issue16250' doesn't exist") } func TestIssue24771(t *testing.T) { @@ -564,11 +551,11 @@ func TestAlterTableAddColumn(t *testing.T) { tk.MustExec("alter table alter_test add column c3 varchar(50) default 'CURRENT_TIMESTAMP'") tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") - _, err = tk.Exec("alter table alter_view add column c4 varchar(50)") + err = tk.ExecToErr("alter table alter_view add column c4 varchar(50)") require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop view alter_view") tk.MustExec("create sequence alter_seq") - _, err = tk.Exec("alter table alter_seq add column c int") + err = tk.ExecToErr("alter table alter_seq add column c int") require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop sequence alter_seq") } @@ -591,11 +578,11 @@ func TestAlterTableAddColumns(t *testing.T) { require.Nil(t, r.Close()) tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") - _, err = tk.Exec("alter table alter_view add column (c4 varchar(50), c5 varchar(50))") + err = tk.ExecToErr("alter table alter_view add column (c4 varchar(50), c5 varchar(50))") require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop view alter_view") tk.MustExec("create sequence alter_seq") - _, err = tk.Exec("alter table alter_seq add column (c1 int, c2 varchar(10))") + err = tk.ExecToErr("alter table alter_seq add column (c1 int, c2 varchar(10))") require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop sequence alter_seq") } @@ -662,8 +649,7 @@ func TestAlterTableModifyColumn(t *testing.T) { tk.MustExec("drop table if exists modify_column_multiple_collate;") tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - _, err = tk.Exec("alter table modify_column_multiple_collate modify column a char(1) charset utf8mb4 collate utf8mb4_bin;") - require.NoError(t, err) + tk.MustExec("alter table modify_column_multiple_collate modify column a char(1) charset utf8mb4 collate utf8mb4_bin;") tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) require.NoError(t, err) require.Equal(t, "utf8mb4", tt.Cols()[0].GetCharset()) @@ -919,10 +905,8 @@ func TestShardRowIDBits(t *testing.T) { tk.MustExec("insert into t1 values(1)") // continue inserting will fail. - _, err = tk.Exec("insert into t1 values(2)") - require.Truef(t, autoid.ErrAutoincReadFailed.Equal(err), "err:%v", err) - _, err = tk.Exec("insert into t1 values(3)") - require.Truef(t, autoid.ErrAutoincReadFailed.Equal(err), "err:%v", err) + tk.MustGetDBError("insert into t1 values(2)", autoid.ErrAutoincReadFailed) + tk.MustGetDBError("insert into t1 values(3)", autoid.ErrAutoincReadFailed) } func TestAutoRandomBitsData(t *testing.T) { @@ -1164,8 +1148,7 @@ func TestSetDDLReorgWorkerCnt(t *testing.T) { err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) - _, err = tk.Exec("set @@global.tidb_ddl_reorg_worker_cnt = invalid_val") - require.Truef(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) + tk.MustGetDBError("set @@global.tidb_ddl_reorg_worker_cnt = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) @@ -1207,8 +1190,7 @@ func TestSetDDLReorgBatchSize(t *testing.T) { err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) require.Equal(t, variable.MaxDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) - _, err = tk.Exec("set @@global.tidb_ddl_reorg_batch_size = invalid_val") - require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) + tk.MustGetDBError("set @@global.tidb_ddl_reorg_batch_size = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) @@ -1315,8 +1297,7 @@ func TestSetDDLErrorCountLimit(t *testing.T) { err = ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) require.Equal(t, int64(math.MaxInt64), variable.GetDDLErrorCountLimit()) - _, err = tk.Exec("set @@global.tidb_ddl_error_count_limit = invalid_val") - require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) + tk.MustGetDBError("set @@global.tidb_ddl_error_count_limit = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_error_count_limit = 100") err = ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) @@ -1373,39 +1354,21 @@ func TestCheckDefaultFsp(t *testing.T) { tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) - _, err := tk.Exec("create table t ( tt timestamp default now(1));") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") - - _, err = tk.Exec("create table t ( tt timestamp(1) default current_timestamp);") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") - - _, err = tk.Exec("create table t ( tt timestamp(1) default now(2));") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") + tk.MustGetErrMsg("create table t ( tt timestamp default now(1));", "[ddl:1067]Invalid default value for 'tt'") + tk.MustGetErrMsg("create table t ( tt timestamp(1) default current_timestamp);", "[ddl:1067]Invalid default value for 'tt'") + tk.MustGetErrMsg("create table t ( tt timestamp(1) default now(2));", "[ddl:1067]Invalid default value for 'tt'") tk.MustExec("create table t ( tt timestamp(1) default now(1));") tk.MustExec("create table t2 ( tt timestamp default current_timestamp());") tk.MustExec("create table t3 ( tt timestamp default current_timestamp(0));") - _, err = tk.Exec("alter table t add column ttt timestamp default now(2);") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") - - _, err = tk.Exec("alter table t add column ttt timestamp(5) default current_timestamp;") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") - - _, err = tk.Exec("alter table t add column ttt timestamp(5) default now(2);") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") - - _, err = tk.Exec("alter table t modify column tt timestamp(1) default now();") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") - - _, err = tk.Exec("alter table t modify column tt timestamp(4) default now(5);") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") - - _, err = tk.Exec("alter table t change column tt tttt timestamp(4) default now(5);") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tttt'") - - _, err = tk.Exec("alter table t change column tt tttt timestamp(1) default now();") - require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tttt'") + tk.MustGetErrMsg("alter table t add column ttt timestamp default now(2);", "[ddl:1067]Invalid default value for 'ttt'") + tk.MustGetErrMsg("alter table t add column ttt timestamp(5) default current_timestamp;", "[ddl:1067]Invalid default value for 'ttt'") + tk.MustGetErrMsg("alter table t add column ttt timestamp(5) default now(2);", "[ddl:1067]Invalid default value for 'ttt'") + tk.MustGetErrMsg("alter table t modify column tt timestamp(1) default now();", "[ddl:1067]Invalid default value for 'tt'") + tk.MustGetErrMsg("alter table t modify column tt timestamp(4) default now(5);", "[ddl:1067]Invalid default value for 'tt'") + tk.MustGetErrMsg("alter table t change column tt tttt timestamp(4) default now(5);", "[ddl:1067]Invalid default value for 'tttt'") + tk.MustGetErrMsg("alter table t change column tt tttt timestamp(1) default now();", "[ddl:1067]Invalid default value for 'tttt'") } func TestTimestampMinDefaultValue(t *testing.T) {