From 52d16c960411219b63701db8064157450a9ca09e Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 09:47:29 +0800 Subject: [PATCH 1/7] enhance task type in explain Signed-off-by: xufei --- executor/builder.go | 6 +- planner/core/common_plans.go | 9 +- planner/core/initialize.go | 45 +- planner/core/physical_plans.go | 20 +- .../core/testdata/enforce_mpp_suite_out.json | 118 +- .../core/testdata/integration_suite_out.json | 3444 ++++++++--------- planner/core/testdata/plan_suite_out.json | 522 +-- 7 files changed, 2092 insertions(+), 2072 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index f5a4a629c3660..9d662ab19d964 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3143,7 +3143,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea plans: v.TablePlans, tablePlan: v.GetTablePlan(), storeType: v.StoreType, - batchCop: v.BatchCop, + batchCop: v.ReadType == plannercore.BatchCop, } e.buildVirtualColumnInfo() if containsLimit(dagReq.Executors) { @@ -3208,7 +3208,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E } }) if useMPPExecution(b.ctx, v) { - plannercore.SetMppOrBatchCopForTableScan(v.GetTablePlan()) return b.buildMPPGather(v) } ts, err := v.GetTableScan() @@ -3216,9 +3215,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E b.err = err return nil } - if v.BatchCop { - ts.IsMPPOrBatchCop = true - } ret, err := buildNoRangeTableReader(b, v) if err != nil { b.err = err diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 1f5b10c3932f6..af5af6f60de4a 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1395,9 +1395,14 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st return errors.Errorf("the store type %v is unknown", x.StoreType) } storeType = x.StoreType.Name() - taskName := "cop" - if x.BatchCop { + taskName := "" + switch x.ReadType { + case Cop: + taskName = "cop" + case BatchCop: taskName = "batchCop" + case MPP: + taskName = "mpp" } err = e.explainPlanInRowFormat(x.tablePlan, taskName+"["+storeType+"]", "", childIndent, true) case *PhysicalIndexReader: diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 3070b9859a561..91c3de7ecf6db 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -417,35 +417,44 @@ func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *Phys // Init initializes PhysicalTableReader. func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalTableReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableReader, &p, offset) + p.ReadType = Cop if p.tablePlan != nil { p.TablePlans = flattenPushDownPlan(p.tablePlan) p.schema = p.tablePlan.Schema() if p.StoreType == kv.TiFlash { - tableScans := p.GetTableScans() - // When PhysicalTableReader's store type is tiflash, has table scan - // and all table scans contained are not keepOrder, try to use batch cop. - if len(tableScans) > 0 { - for _, tableScan := range tableScans { - if tableScan.KeepOrder { - return &p + _, ok := p.tablePlan.(*PhysicalExchangeSender) + if ok { + p.ReadType = MPP + } else { + tableScans := p.GetTableScans() + // When PhysicalTableReader's store type is tiflash, has table scan + // and all table scans contained are not keepOrder, try to use batch cop. + if len(tableScans) > 0 { + for _, tableScan := range tableScans { + if tableScan.KeepOrder { + return &p + } } - } - // When allow batch cop is 1, only agg / topN uses batch cop. - // When allow batch cop is 2, every query uses batch cop. - switch ctx.GetSessionVars().AllowBatchCop { - case 1: - for _, plan := range p.TablePlans { - switch plan.(type) { - case *PhysicalHashAgg, *PhysicalStreamAgg, *PhysicalTopN: - p.BatchCop = true + // When allow batch cop is 1, only agg / topN uses batch cop. + // When allow batch cop is 2, every query uses batch cop. + switch ctx.GetSessionVars().AllowBatchCop { + case 1: + for _, plan := range p.TablePlans { + switch plan.(type) { + case *PhysicalHashAgg, *PhysicalStreamAgg, *PhysicalTopN: + p.ReadType = BatchCop + } } + case 2: + p.ReadType = BatchCop } - case 2: - p.BatchCop = true } } } + if p.ReadType == BatchCop || p.ReadType == MPP { + setMppOrBatchCopForTableScan(p.tablePlan) + } } return &p } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 984ce5a419cae..e076ab171c538 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -73,6 +73,17 @@ type tableScanAndPartitionInfo struct { partitionInfo PartitionInfo } +type ReadReqType uint8 + +const ( + // Cop means read from storage by cop request. + Cop ReadReqType = iota + // BatchCop means read from storage by BatchCop request, only used for TiFlash + BatchCop + // MPP means read from storage by MPP request, only used for TiFlash + MPP +) + // PhysicalTableReader is the table reader in tidb. type PhysicalTableReader struct { physicalSchemaProducer @@ -85,7 +96,7 @@ type PhysicalTableReader struct { StoreType kv.StoreType // BatchCop = true means the cop task in the physical table reader will be executed in batch mode(use in TiFlash only) - BatchCop bool + ReadType ReadReqType IsCommonHandle bool @@ -129,14 +140,13 @@ func (p *PhysicalTableReader) GetTableScan() (*PhysicalTableScan, error) { return tableScans[0], nil } -// SetMppOrBatchCopForTableScan set IsMPPOrBatchCop for all TableScan. -func SetMppOrBatchCopForTableScan(curPlan PhysicalPlan) { +func setMppOrBatchCopForTableScan(curPlan PhysicalPlan) { if ts, ok := curPlan.(*PhysicalTableScan); ok { ts.IsMPPOrBatchCop = true } children := curPlan.Children() for _, child := range children { - SetMppOrBatchCopForTableScan(child) + setMppOrBatchCopForTableScan(child) } } @@ -173,7 +183,7 @@ func (p *PhysicalTableReader) Clone() (PhysicalPlan, error) { } cloned.physicalSchemaProducer = *base cloned.StoreType = p.StoreType - cloned.BatchCop = p.BatchCop + cloned.ReadType = p.ReadType cloned.IsCommonHandle = p.IsCommonHandle if cloned.tablePlan, err = p.tablePlan.Clone(); err != nil { return nil, err diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index 3315ea985f791..32e92bf19d031 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -94,10 +94,10 @@ "Plan": [ "HashAgg_21 1.00 11910.73 root funcs:count(Column#6)->Column#4", "└─TableReader_23 1.00 11877.13 root data:ExchangeSender_22", - " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_22 1.00 285050.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_20 10.00 285020.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -111,10 +111,10 @@ "Plan": [ "HashAgg_24 1.00 33.89 root funcs:count(Column#6)->Column#4", "└─TableReader_26 1.00 0.29 root data:ExchangeSender_25", - " └─ExchangeSender_25 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─Selection_23 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_22 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_25 1.00 285050.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_23 10.00 285020.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 255020.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -133,10 +133,10 @@ "Plan": [ "HashAgg_21 1.00 33.89 root funcs:count(Column#6)->Column#4", "└─TableReader_23 1.00 0.29 root data:ExchangeSender_22", - " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_22 1.00 285050.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_20 10.00 285020.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -150,10 +150,10 @@ "Plan": [ "HashAgg_25 1.00 33.60 root funcs:count(Column#6)->Column#4", "└─TableReader_27 1.00 0.29 root data:ExchangeSender_26", - " └─ExchangeSender_26 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─Selection_24 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_23 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_26 1.00 285050.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_10 1.00 285050.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_24 10.00 285020.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_23 10000.00 255020.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -174,10 +174,10 @@ "Plan": [ "HashAgg_22 1.00 33.60 root funcs:count(Column#6)->Column#4", "└─TableReader_24 1.00 0.29 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_23 1.00 285050.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_10 1.00 285050.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_21 10.00 285020.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_20 10000.00 255020.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null } @@ -418,9 +418,9 @@ "Plan": [ "HashAgg_24 8000.00 root group by:test.t.b, funcs:count(Column#7)->Column#4", "└─TableReader_26 8000.00 root data:ExchangeSender_25", - " └─ExchangeSender_25 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg_11 8000.00 batchCop[tiflash] group by:test.t.b, funcs:count(1)->Column#7", - " └─TableFullScan_21 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_25 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_11 8000.00 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#7", + " └─TableFullScan_21 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because when `new_collation_enabled` is true, HashJoin or HashAgg with string key is not supported now.", @@ -432,14 +432,14 @@ "SQL": "EXPLAIN SELECT * from t t1 join t t2 on t1.b=t2.b; -- 2. new collation FIXME", "Plan": [ "TableReader_34 12487.50 root data:ExchangeSender_33", - "└─ExchangeSender_33 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_32 12487.50 cop[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", - " ├─ExchangeReceiver_15(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender_14 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection_13 9990.00 cop[tiflash] not(isnull(test.t.b))", - " │ └─TableFullScan_12 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─Selection_17(Probe) 9990.00 cop[tiflash] not(isnull(test.t.b))", - " └─TableFullScan_16 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + "└─ExchangeSender_33 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_32 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver_15(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_13 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection_17(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan_16 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" ], "Warn": null } @@ -556,12 +556,12 @@ "SQL": "EXPLAIN SELECT * from t join s; -- can use mpp", "Plan": [ "TableReader_27 100000000.00 root data:ExchangeSender_26", - "└─ExchangeSender_26 100000000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_25 100000000.00 cop[tiflash] CARTESIAN inner join", - " ├─ExchangeReceiver_13(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender_12 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan_11 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan_14(Probe) 10000.00 cop[tiflash] table:s keep order:false, stats:pseudo" + "└─ExchangeSender_26 100000000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_25 100000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver_13(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_11 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_14(Probe) 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo" ], "Warn": null }, @@ -574,12 +574,12 @@ "SQL": "EXPLAIN SELECT * from t join s; -- can use mpp", "Plan": [ "TableReader_27 100000000.00 root data:ExchangeSender_26", - "└─ExchangeSender_26 100000000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_25 100000000.00 cop[tiflash] CARTESIAN inner join", - " ├─ExchangeReceiver_13(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender_12 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan_11 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan_14(Probe) 10000.00 cop[tiflash] table:s keep order:false, stats:pseudo" + "└─ExchangeSender_26 100000000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_25 100000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver_13(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_11 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_14(Probe) 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo" ], "Warn": null }, @@ -592,14 +592,14 @@ "SQL": "explain select a from t where t.a>1 or t.a in (select a from t); -- 7. left outer semi join", "Plan": [ "TableReader_48 8000.00 root data:ExchangeSender_47", - "└─ExchangeSender_47 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection_8 8000.00 cop[tiflash] test.t.a", - " └─Selection_45 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#3)", - " └─HashJoin_46 10000.00 cop[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", - " ├─ExchangeReceiver_26(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender_25 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan_24 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan_23(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender_47 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.a", + " └─Selection_45 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_46 10000.00 mpp[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_26(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_25 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_23(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -607,14 +607,14 @@ "SQL": "explain select a from t where t.a>1 or t.a not in (select a from t); -- now it's supported -- 8. anti left outer semi join", "Plan": [ "TableReader_48 8000.00 root data:ExchangeSender_47", - "└─ExchangeSender_47 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection_8 8000.00 cop[tiflash] test.t.a", - " └─Selection_45 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#3)", - " └─HashJoin_46 10000.00 cop[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", - " ├─ExchangeReceiver_26(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender_25 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan_24 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan_23(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender_47 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.a", + " └─Selection_45 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_46 10000.00 mpp[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_26(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_25 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_23(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 0024560973972..d9fe99f235939 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1332,7 +1332,7 @@ "└─IndexRangeScan 20.00 cop[tikv] table:tt, index:a(a) range:[10,10], [20,20], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable." + "Warning 1105 IndexMerge is inapplicable" ] }, { @@ -1342,7 +1342,7 @@ "└─IndexRangeScan 6666.67 cop[tikv] table:tt, index:a(a) range:[-inf,10), [15,15], (20,+inf], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable." + "Warning 1105 IndexMerge is inapplicable" ] } ] @@ -2393,12 +2393,12 @@ "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ "TableReader_24 3.00 3.33 root data:ExchangeSender_23", - "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", - " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_21 3.00 68.00 batchCop[tiflash] ", - " └─ExchangeSender_20 3.00 68.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", - " └─TableFullScan_19 3.00 65.00 batchCop[tiflash] table:t2 keep order:false" + "└─ExchangeSender_23 3.00 77.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_22 3.00 0.00 mpp[tiflash] Column#4", + " └─HashAgg_8 3.00 77.00 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_21 3.00 68.00 mpp[tiflash] ", + " └─ExchangeSender_20 3.00 68.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─TableFullScan_19 3.00 65.00 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2444,14 +2444,14 @@ "Plan": [ "StreamAgg_14 1.00 18.93 root funcs:count(1)->Column#7", "└─TableReader_46 3.00 9.93 root data:ExchangeSender_45", - " └─ExchangeSender_45 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_42 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_21(Build) 3.00 77.00 cop[tiflash] ", - " │ └─ExchangeSender_20 3.00 77.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection_19 3.00 74.00 cop[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_18 3.00 65.00 cop[tiflash] table:t1 keep order:false", - " └─Selection_23(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_22 3.00 65.00 cop[tiflash] table:t2 keep order:false" + " └─ExchangeSender_45 3.00 235.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_42 3.00 235.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_21(Build) 3.00 77.00 mpp[tiflash] ", + " │ └─ExchangeSender_20 3.00 77.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_19 3.00 74.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_18 3.00 65.00 mpp[tiflash] table:t1 keep order:false", + " └─Selection_23(Probe) 3.00 74.00 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_22 3.00 65.00 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2462,14 +2462,14 @@ " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", - " │ └─ExchangeSender_34 3.00 106.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection_33 3.00 103.00 cop[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan_32 3.00 94.00 cop[tiflash] table:t1 keep order:false", - " └─Selection_37(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_36 3.00 65.00 cop[tiflash] table:t2 keep order:false" + " └─ExchangeSender_38 3.00 264.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 264.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 106.00 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 106.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_33 3.00 103.00 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 94.00 mpp[tiflash] table:t1 keep order:false", + " └─Selection_37(Probe) 3.00 74.00 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 65.00 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2488,9 +2488,9 @@ " └─Projection_27(Probe) 1.00 21.95 root 1->Column#26", " └─Limit_30 1.00 3.35 root offset:0, count:1", " └─TableReader_38 1.00 3.35 root data:ExchangeSender_37", - " └─ExchangeSender_37 1.00 79.50 cop[tiflash] ExchangeType: PassThrough", - " └─Limit_36 1.00 79.50 cop[tiflash] offset:0, count:1", - " └─TableFullScan_35 1.00 79.50 cop[tiflash] table:t2 keep order:false" + " └─ExchangeSender_37 1.00 79.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_36 1.00 79.50 mpp[tiflash] offset:0, count:1", + " └─TableFullScan_35 1.00 79.50 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2784,14 +2784,14 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2799,24 +2799,24 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#17", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[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 cop[tiflash] table:fact_t keep order:false" + " └─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.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" ] }, { @@ -2824,14 +2824,14 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2839,13 +2839,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2853,13 +2853,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + " └─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" ] }, { @@ -2867,14 +2867,14 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2882,13 +2882,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2896,13 +2896,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2910,13 +2910,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + " └─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" ] }, { @@ -2924,13 +2924,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + " └─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" ] }, { @@ -2938,14 +2938,14 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#12", "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2953,14 +2953,14 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#12", "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2968,12 +2968,12 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#12", "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2981,12 +2981,12 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#12", "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[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 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -2994,15 +2994,15 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -3010,14 +3010,14 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -3025,14 +3025,14 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 8.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" + " └─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" ] }, { @@ -3040,12 +3040,12 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#11", "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + " └─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" ] } ] @@ -3057,14 +3057,14 @@ "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t); -- left semi", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", - " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", - " └─HashJoin 10000.00 cop[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 mpp[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, @@ -3072,14 +3072,14 @@ "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t where b1 or t1.a not in (select a from test.t); -- left anti", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", - " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", - " └─HashJoin 10000.00 cop[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 mpp[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, @@ -3102,14 +3102,14 @@ "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a not in (select a from test.t where b1 or t1.b in (select a from test.t); -- cartesian left semi", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", - " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", - " └─HashJoin 10000.00 cop[tiflash] CARTESIAN left outer semi join, other cond:eq(test.t.b, test.t.a)", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 mpp[tiflash] CARTESIAN left outer semi join, other cond:eq(test.t.b, test.t.a)", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, @@ -3132,14 +3132,14 @@ "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select b from test.t where b1 or t1.b not in (select a from test.t); -- cartesian left anti", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", - " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", - " └─HashJoin 10000.00 cop[tiflash] CARTESIAN anti left outer semi join, other cond:eq(test.t.b, test.t.a)", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 mpp[tiflash] CARTESIAN anti left outer semi join, other cond:eq(test.t.b, test.t.a)", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, @@ -3162,14 +3162,14 @@ "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.b not in (select a from test.t where bColumn#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + " └─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" ] }, { @@ -3310,13 +3310,13 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + " └─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" ] } ] @@ -3329,15 +3329,15 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + " └─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" ] }, { @@ -3345,15 +3345,15 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + " └─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" ] } ] @@ -3366,15 +3366,15 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", - " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + " └─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" ] }, { @@ -3382,15 +3382,15 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#7", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", - " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", - " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + " └─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" ] } ] @@ -3403,17 +3403,17 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_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" ] }, { @@ -3421,31 +3421,31 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#18)->Column#17", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─ExchangeReceiver(Probe) 64.00 batchCop[tiflash] ", - " └─ExchangeSender 64.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: binary]", - " └─HashJoin 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─ExchangeReceiver(Probe) 32.00 batchCop[tiflash] ", - " └─ExchangeSender 32.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: binary]", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#18", + " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", + " └─ExchangeReceiver(Probe) 64.00 mpp[tiflash] ", + " └─ExchangeSender 64.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: binary]", + " └─HashJoin 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", + " └─ExchangeReceiver(Probe) 32.00 mpp[tiflash] ", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: binary]", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_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)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3453,17 +3453,17 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_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" ] }, { @@ -3471,27 +3471,27 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#18)->Column#17", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#18", + " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_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" ] }, { @@ -3499,16 +3499,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 mpp[tiflash] left outer 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]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3516,16 +3516,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 mpp[tiflash] right outer 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]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3533,17 +3533,17 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[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) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.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) 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" ] }, { @@ -3551,16 +3551,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[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) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.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) 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]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3568,24 +3568,24 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#22)->Column#19", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#22", - " └─HashJoin 204.80 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Projection(Probe) 102.40 batchCop[tiflash] test.fact_t.d1_k", - " └─Selection 102.40 batchCop[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", - " └─HashJoin 128.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:t2 keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#22", + " └─HashJoin 204.80 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_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", + " └─Projection(Probe) 102.40 mpp[tiflash] test.fact_t.d1_k", + " └─Selection 102.40 mpp[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", + " └─HashJoin 128.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 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:t1 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:t2 keep order:false" ] }, { @@ -3593,16 +3593,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[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) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.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) 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]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3610,16 +3610,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[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) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.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) 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]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { @@ -3627,16 +3627,16 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[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) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.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) 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", + " └─ExchangeReceiver(Probe) 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" ] }, { @@ -3644,17 +3644,17 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#13)->Column#12", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -3662,17 +3662,17 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#13)->Column#12", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[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 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -3680,15 +3680,15 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#13)->Column#12", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] }, { @@ -3696,15 +3696,15 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#13)->Column#12", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[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 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─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" ] } ] @@ -3716,65 +3716,65 @@ "SQL": "explain format = 'brief' select v from t3 as a left join (select t1.v1, t1.v2, t1.v1 + t1.v2 as v from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2) b on a.v1 = b.v1 and a.v2 = b.v2", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 cop[tiflash] Column#13", - " └─HashJoin 1.00 cop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t1.v1) eq(test.t3.v2, test.t1.v2)]", - " ├─ExchangeReceiver(Build) 1.00 cop[tiflash] ", - " │ └─ExchangeSender 1.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#23, collate: binary], [name: Column#24, collate: binary]", - " │ └─Projection 1.00 cop[tiflash] test.t3.v1, test.t3.v2, cast(test.t3.v1, decimal(20,2))->Column#23, cast(test.t3.v2, decimal(20,2))->Column#24", - " │ └─TableFullScan 1.00 cop[tiflash] table:a keep order:false", - " └─Projection(Probe) 2.00 cop[tiflash] test.t1.v1, test.t1.v2, plus(test.t1.v1, test.t1.v2)->Column#13", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.t1.v1)), not(isnull(test.t1.v2))", - " │ └─TableFullScan 2.00 cop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 cop[tiflash] ", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: binary], [name: Column#16, collate: binary]", - " └─Projection 8.00 cop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#15, cast(test.t2.v2, decimal(20,2))->Column#16", - " └─Selection 8.00 cop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 cop[tiflash] table:t2 keep order:false" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#13", + " └─HashJoin 1.00 mpp[tiflash] left outer join, equal:[eq(test.t3.v1, test.t1.v1) eq(test.t3.v2, test.t1.v2)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#23, collate: binary], [name: Column#24, collate: binary]", + " │ └─Projection 1.00 mpp[tiflash] test.t3.v1, test.t3.v2, cast(test.t3.v1, decimal(20,2))->Column#23, cast(test.t3.v2, decimal(20,2))->Column#24", + " │ └─TableFullScan 1.00 mpp[tiflash] table:a keep order:false", + " └─Projection(Probe) 2.00 mpp[tiflash] test.t1.v1, test.t1.v2, plus(test.t1.v1, test.t1.v2)->Column#13", + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.t1.v1)), not(isnull(test.t1.v2))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: binary], [name: Column#16, collate: binary]", + " └─Projection 8.00 mpp[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#15, cast(test.t2.v2, decimal(20,2))->Column#16", + " └─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" ] }, { "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2 group by t2.v1, t2.v2", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#22)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#22", - " └─HashJoin 2.00 batchCop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", - " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#15, collate: binary]", - " └─Projection 8.00 batchCop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#14, cast(test.t2.v2, decimal(20,2))->Column#15", - " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 mpp[tiflash] Column#9, test.t2.v1, test.t2.v2", + " └─HashAgg 2.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#22)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─ExchangeReceiver 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─HashAgg 2.00 mpp[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#22", + " └─HashJoin 2.00 mpp[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", + " │ └─TableFullScan 2.00 mpp[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#15, collate: binary]", + " └─Projection 8.00 mpp[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#14, cast(test.t2.v2, decimal(20,2))->Column#15", + " └─Selection 8.00 mpp[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t3 left join t2 on t3.v1 = t2.v1 and t3.v2 = t2.v2 group by t2.v1, t2.v2", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 1.00 batchCop[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", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#16", - " └─HashJoin 1.00 batchCop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", - " │ └─TableFullScan 1.00 batchCop[tiflash] table:t3 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", - " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", - " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" + "└─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", + " └─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" ] } ] @@ -3831,102 +3831,102 @@ "SQL": "explain format = 'brief' select * from table_1 a, table_1 b where a.value = b.value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 mpp[tiflash] table:b keep order:false" ] }, { "SQL": "explain format = 'brief' select * from table_1 a, table_2 b where a.value = b.value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " └─Selection 2.00 mpp[tiflash] not(isnull(test.table_2.value))", + " └─TableFullScan 2.00 mpp[tiflash] table:b keep order:false" ] }, { "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and b.value = c.value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_2.value, test.table_1.value)]", - " ├─HashJoin(Build) 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_2.value, test.table_1.value)]", + " ├─HashJoin(Build) 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " │ ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " │ │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " │ └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_2.value))", + " │ └─TableFullScan 2.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.table_1.value, collate: utf8mb4_bin]", + " └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 mpp[tiflash] table:c keep order:false" ] }, { "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and a.value = c.value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " │ └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " │ └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " │ ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " │ │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " │ └─ExchangeReceiver(Probe) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_2.value))", + " │ └─TableFullScan 2.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.table_1.value, collate: utf8mb4_general_ci]", + " └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 mpp[tiflash] table:c keep order:false" ] }, { "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_1 group by value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_1.value", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_1.value)->test.table_1.value", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:count(1)->Column#7", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 mpp[tiflash] Column#4, test.table_1.value", + " └─HashAgg 2.00 mpp[tiflash] group by:test.table_1.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_1.value)->test.table_1.value", + " └─ExchangeReceiver 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " └─HashAgg 2.00 mpp[tiflash] group by:test.table_1.value, funcs:count(1)->Column#7", + " └─TableFullScan 2.00 mpp[tiflash] table:table_1 keep order:false" ] }, { "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_2 group by value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_2.value", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_2.value)->test.table_2.value", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:count(1)->Column#7", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_2 keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 mpp[tiflash] Column#4, test.table_2.value", + " └─HashAgg 2.00 mpp[tiflash] group by:test.table_2.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_2.value)->test.table_2.value", + " └─ExchangeReceiver 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " └─HashAgg 2.00 mpp[tiflash] group by:test.table_2.value, funcs:count(1)->Column#7", + " └─TableFullScan 2.00 mpp[tiflash] table:table_2 keep order:false" ] } ] @@ -3938,26 +3938,26 @@ "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.id = b.id", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.id, test.table_1.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:b keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.id, test.table_1.id)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:b keep order:false" ] }, { "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.value = b.value", "Plan": [ "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─Selection(Probe) 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 mpp[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:a keep order:false", + " └─Selection(Probe) 2.00 mpp[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 mpp[tiflash] table:b keep order:false" ] } ] @@ -3970,14 +3970,14 @@ "Plan": [ "Projection 2.00 root test.table_1.id, Column#4, Column#5", "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#4, div(Column#5, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#5, test.table_1.id", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.id, funcs:sum(Column#15)->Column#13, funcs:sum(Column#16)->Column#4, funcs:sum(Column#17)->Column#14, funcs:sum(Column#18)->Column#5, funcs:firstrow(test.table_1.id)->test.table_1.id", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.id, collate: binary]", - " └─HashAgg 2.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#16, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#18", - " └─Projection 2.00 batchCop[tiflash] plus(test.table_1.value, 1)->Column#25, plus(test.table_1.value, 1)->Column#26, test.table_1.value, test.table_1.value, test.table_1.id", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#4, div(Column#5, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#5, test.table_1.id", + " └─HashAgg 2.00 mpp[tiflash] group by:test.table_1.id, funcs:sum(Column#15)->Column#13, funcs:sum(Column#16)->Column#4, funcs:sum(Column#17)->Column#14, funcs:sum(Column#18)->Column#5, funcs:firstrow(test.table_1.id)->test.table_1.id", + " └─ExchangeReceiver 2.00 mpp[tiflash] ", + " └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.id, collate: binary]", + " └─HashAgg 2.00 mpp[tiflash] group by:Column#29, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#16, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#18", + " └─Projection 2.00 mpp[tiflash] plus(test.table_1.value, 1)->Column#25, plus(test.table_1.value, 1)->Column#26, test.table_1.value, test.table_1.value, test.table_1.id", + " └─TableFullScan 2.00 mpp[tiflash] table:table_1 keep order:false" ] } ] @@ -4504,10 +4504,10 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#9)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(Column#11)->Column#9", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4515,9 +4515,9 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#8)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4525,10 +4525,10 @@ "Plan": [ "HashAgg 1.00 root funcs:sum(Column#9)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#9", - " └─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" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#11)->Column#9", + " └─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" ] }, { @@ -4536,10 +4536,10 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#10)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#10", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─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" ] }, { @@ -4547,9 +4547,9 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#9)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4557,113 +4557,113 @@ "Plan": [ "HashAgg 1.00 root funcs:sum(Column#10)->Column#6", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#10", - " └─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" + " └─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" ] }, { "SQL": "desc format = 'brief' select B.b+A.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] plus(Column#5, Column#10)->Column#11", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] plus(Column#5, Column#10)->Column#11", + " └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 10000.00 cop[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 mpp[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", "Plan": [ "TableReader 12487.50 root data:ExchangeSender", - "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 10000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 10000.00 mpp[tiflash] minus(test.t.id, 2)->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] Column#10, Column#5", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#10, Column#5", + " └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 8000.00 mpp[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 mpp[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select id from t as A where exists (select 1 from t where t.id=A.id)", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 cop[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:A keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8000.00 cop[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 mpp[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:A keep order:false, stats:pseudo" ] }, { @@ -4671,24 +4671,24 @@ "Plan": [ "Sort 8000.00 root test.t.id", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] mul(Column#5, 2)->Column#6, test.t.id", - " └─Projection 8000.00 batchCop[tiflash] div(Column#5, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#5, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#21)->Column#20, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#26, funcs:count(Column#24)->Column#21, funcs:sum(Column#25)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.value, 2)->Column#24, plus(test.t.value, 2)->Column#25, test.t.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] mul(Column#5, 2)->Column#6, test.t.id", + " └─Projection 8000.00 mpp[tiflash] div(Column#5, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#5, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#21)->Column#20, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#26, funcs:count(Column#24)->Column#21, funcs:sum(Column#25)->Column#22", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.value, 2)->Column#24, plus(test.t.value, 2)->Column#25, test.t.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] } ] @@ -4701,13 +4701,13 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#12)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─Union 20000.00 batchCop[tiflash] ", - " ├─Projection 10000.00 batchCop[tiflash] cast(test.t.a, int(11) BINARY)->Column#9, test.t.b", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 batchCop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", + " └─Union 20000.00 mpp[tiflash] ", + " ├─Projection 10000.00 mpp[tiflash] cast(test.t.a, int(11) BINARY)->Column#9, test.t.b", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 mpp[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -4715,13 +4715,13 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#16)->Column#15", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#16", - " └─Union 20000.00 batchCop[tiflash] ", - " ├─Projection 10000.00 batchCop[tiflash] cast(test.t.a, int(11) BINARY)->Column#13, test.t.b", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 batchCop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#14", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#16", + " └─Union 20000.00 mpp[tiflash] ", + " ├─Projection 10000.00 mpp[tiflash] cast(test.t.a, int(11) BINARY)->Column#13, test.t.b", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 mpp[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -4729,14 +4729,14 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#14", - " └─Union 20000.00 batchCop[tiflash] ", - " ├─Projection 10000.00 batchCop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", - " │ └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 batchCop[tiflash] test.t1.a, cast(test.t1.c, double BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#14", + " └─Union 20000.00 mpp[tiflash] ", + " ├─Projection 10000.00 mpp[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 mpp[tiflash] test.t1.a, cast(test.t1.c, double BINARY)->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -4744,12 +4744,12 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#14", - " └─Union 10000.00 batchCop[tiflash] ", - " └─Projection 10000.00 batchCop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#14", + " └─Union 10000.00 mpp[tiflash] ", + " └─Projection 10000.00 mpp[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " └─Projection 10000.00 mpp[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4773,17 +4773,17 @@ "Plan": [ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c1, test.t.c2, test.t.c3", "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#21, collate: binary], [name: Column#22, collate: binary]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(13,8) BINARY)->Column#21, cast(Column#15, decimal(10,5) BINARY)->Column#22", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, mul(test.t.c1, 3)->Column#13, plus(test.t.c1, 1)->Column#15", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#16, collate: binary]", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, div(test.t.c3, 2)->Column#14, minus(test.t.c2, 10)->Column#16", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + " └─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#21, collate: binary], [name: Column#22, collate: binary]", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(13,8) BINARY)->Column#21, cast(Column#15, decimal(10,5) BINARY)->Column#22", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, mul(test.t.c1, 3)->Column#13, plus(test.t.c1, 1)->Column#15", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#16, collate: binary]", + " └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, div(test.t.c3, 2)->Column#14, minus(test.t.c2, 10)->Column#16", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" ] }, { @@ -4791,44 +4791,44 @@ "Plan": [ "Projection 7976.02 root test.t.c1, test.t.c2, test.t.c5, Column#7, test.t.c1, test.t.c2, test.t.c3, Column#14", "└─TableReader 7976.02 root data:ExchangeSender", - " └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", - " ├─ExchangeReceiver(Build) 7976.02 batchCop[tiflash] ", - " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#31, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", - " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5", - " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:sum(Column#15)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", - " │ └─ExchangeReceiver 7976.02 batchCop[tiflash] ", - " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#15", - " │ └─Selection 9970.03 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 7984.01 batchCop[tiflash] ", - " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#32, collate: binary], [name: Column#33, collate: binary]", - " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3, cast(test.t.c3, decimal(10,5))->Column#32, cast(test.t.c1, decimal(40,20))->Column#33", - " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3", - " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:sum(Column#23)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", - " └─ExchangeReceiver 7984.01 batchCop[tiflash] ", - " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c1, collate: binary]", - " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#23", - " └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 7976.02 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7976.02 mpp[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", + " ├─ExchangeReceiver(Build) 7976.02 mpp[tiflash] ", + " │ └─ExchangeSender 7976.02 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#31, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 7976.02 mpp[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", + " │ └─Projection 7976.02 mpp[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5", + " │ └─HashAgg 7976.02 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:sum(Column#15)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", + " │ └─ExchangeReceiver 7976.02 mpp[tiflash] ", + " │ └─ExchangeSender 7976.02 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─HashAgg 7976.02 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#15", + " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 7984.01 mpp[tiflash] ", + " └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#32, collate: binary], [name: Column#33, collate: binary]", + " └─Projection 7984.01 mpp[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3, cast(test.t.c3, decimal(10,5))->Column#32, cast(test.t.c1, decimal(40,20))->Column#33", + " └─Projection 7984.01 mpp[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3", + " └─HashAgg 7984.01 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:sum(Column#23)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", + " └─ExchangeReceiver 7984.01 mpp[tiflash] ", + " └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c1, collate: binary]", + " └─HashAgg 7984.01 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#23", + " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c2 and t1.c3 = t2.c3 and t1.c4 = t2.c4 and t1.c5 = t2.c5", "Plan": [ "TableReader 12462.54 root data:ExchangeSender", - "└─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c2) eq(test.t.c3, test.t.c3) eq(test.t.c4, test.t.c4) eq(test.t.c5, test.t.c5)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + "└─ExchangeSender 12462.54 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12462.54 mpp[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c2) eq(test.t.c3, test.t.c3) eq(test.t.c4, test.t.c4) eq(test.t.c5, test.t.c5)]", + " ├─ExchangeReceiver(Build) 9970.03 mpp[tiflash] ", + " │ └─ExchangeSender 9970.03 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 mpp[tiflash] ", + " └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", + " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" ] }, { @@ -4836,18 +4836,18 @@ "Plan": [ "Projection 12462.54 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12462.54 root data:ExchangeSender", - " └─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", - " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#14, cast(test.t.c3, decimal(10,5))->Column#16, cast(test.t.c1, decimal(40,20))->Column#17", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 12462.54 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12462.54 mpp[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9970.03 mpp[tiflash] ", + " │ └─ExchangeSender 9970.03 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 9970.03 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", + " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 mpp[tiflash] ", + " └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", + " └─Projection 9980.01 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#14, cast(test.t.c3, decimal(10,5))->Column#16, cast(test.t.c1, decimal(40,20))->Column#17", + " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -4855,17 +4855,17 @@ "Plan": [ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, plus(test.t.c1, test.t.c2)->Column#13", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary]", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, div(test.t.c2, test.t.c3)->Column#14", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + " └─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(Column#13, Column#14)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, plus(test.t.c1, test.t.c2)->Column#13", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary]", + " └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, div(test.t.c2, test.t.c3)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" ] }, { @@ -4873,18 +4873,18 @@ "Plan": [ "Projection 7984.01 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 7984.01 root data:ExchangeSender", - " └─ExchangeSender 7984.01 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", - " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#13, cast(test.t.c3, decimal(10,5))->Column#15, cast(test.t.c1, decimal(40,20))->Column#17", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7984.01 mpp[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", + " ├─ExchangeReceiver(Build) 9970.03 mpp[tiflash] ", + " │ └─ExchangeSender 9970.03 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 9970.03 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", + " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 mpp[tiflash] ", + " └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", + " └─Projection 9980.01 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#13, cast(test.t.c3, decimal(10,5))->Column#15, cast(test.t.c1, decimal(40,20))->Column#17", + " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -4892,55 +4892,55 @@ "Plan": [ "Projection 19492.21 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 19492.21 root data:ExchangeSender", - " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#27, collate: binary]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo", - " └─Projection(Probe) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#25, collate: binary]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c5, collate: binary]", - " └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary]", - " │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary]", - " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 19492.21 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 mpp[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#27, collate: binary]", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 keep order:false, stats:pseudo", + " └─Projection(Probe) 15593.77 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#25, collate: binary]", + " │ └─Projection 10000.00 mpp[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 mpp[tiflash] ", + " └─ExchangeSender 12475.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c5, collate: binary]", + " └─HashJoin 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary]", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.c1))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin], [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", - " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", - " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: utf8mb4_bin]", - " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin], [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", + " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: utf8mb4_bin]", + " │ └─Projection 9990.00 mpp[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] @@ -4953,10 +4953,10 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#8)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(Column#9)->Column#8", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4964,9 +4964,9 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#7)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4974,10 +4974,10 @@ "Plan": [ "HashAgg 1.00 root funcs:sum(Column#8)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#8", + " └─Projection 10000.00 mpp[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -4985,22 +4985,22 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#6)->Column#4", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#6", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select count(*), id from t group by id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5008,33 +5008,33 @@ "Plan": [ "Projection 8000.00 root Column#4, plus(test.t.id, 1)->Column#5", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:sum(Column#11)->Column#4, funcs:firstrow(Column#12)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:count(1)->Column#11, funcs:firstrow(Column#16)->Column#12", - " └─Projection 10000.00 batchCop[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:sum(Column#11)->Column#4, funcs:firstrow(Column#12)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:count(1)->Column#11, funcs:firstrow(Column#16)->Column#12", + " └─Projection 10000.00 mpp[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", "Plan": [ "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5043,9 +5043,9 @@ "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", "├─HashAgg(Build) 1.00 root funcs:count(Column#11)->Column#7", "│ └─TableReader 1.00 root data:ExchangeSender", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#11", - "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#11", + "│ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" @@ -5055,13 +5055,13 @@ "SQL": "desc format = 'brief' select avg(value) as b,id from t group by id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#9, 0), 1, Column#9), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#10)->Column#9, funcs:sum(Column#11)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#10, funcs:sum(test.t.value)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#9, 0), 1, Column#9), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#10)->Column#9, funcs:sum(Column#11)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#10, funcs:sum(test.t.value)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5069,14 +5069,14 @@ "Plan": [ "HashAgg 1.00 root funcs:sum(Column#18)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#4)->Column#18", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#4", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#16)->Column#15, funcs:sum(Column#17)->Column#4", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#16, funcs:sum(test.t.value)->Column#17", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#4)->Column#18", + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#4", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#16)->Column#15, funcs:sum(Column#17)->Column#4", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#16, funcs:sum(test.t.value)->Column#17", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5085,13 +5085,13 @@ "Projection 6400.00 root test.t.id", "└─Selection 6400.00 root gt(Column#4, 0)", " └─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#10, funcs:sum(Column#12)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#11, funcs:sum(test.t.value)->Column#12", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#10, funcs:sum(Column#12)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#11, funcs:sum(test.t.value)->Column#12", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5099,13 +5099,13 @@ "Plan": [ "Selection 6400.00 root gt(Column#4, 0)", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5113,75 +5113,75 @@ "Plan": [ "Projection 8000.00 root plus(Column#4, 1)->Column#5, test.t.id", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 7992.00 batchCop[tiflash] Column#7", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 7992.00 mpp[tiflash] Column#7", + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", + " └─Projection 12487.50 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#17", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", + " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.id, Column#13", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#17", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select count(distinct value),id from t group by id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.value, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5196,118 +5196,118 @@ "SQL": "desc format = 'brief' select * from t join ( select count(distinct value), id from t group by id) as A on A.id = t.id", "Plan": [ "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t join ( select count(1/value), id from t group by id) as A on A.id = t.id", "Plan": [ "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", - " │ └─Projection 9990.00 batchCop[tiflash] div(1, test.t.value)->Column#18, test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", + " │ └─Projection 9990.00 mpp[tiflash] div(1, test.t.value)->Column#18, test.t.id", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", "Plan": [ "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 6400.00 batchCop[tiflash] Column#4", - " └─HashAgg 6400.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#19)->Column#4", - " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#19, test.t.value", - " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", - " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#4", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#20, funcs:sum(Column#19)->Column#4", + " └─Projection 6400.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#19, test.t.value", + " └─Projection 6400.00 mpp[tiflash] test.t.id, test.t.value", + " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " └─ExchangeReceiver 6400.00 mpp[tiflash] ", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", + " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, test.t.value, ", + " └─Selection 8000.00 mpp[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/", "Plan": [ "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 6400.00 batchCop[tiflash] Column#8", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#8", - " └─Selection 8000.00 batchCop[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))", - " └─HashJoin 10000.00 batchCop[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", - " ├─Projection(Build) 7984.01 batchCop[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 7984.01 batchCop[tiflash] ", - " │ └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24", - " │ └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:B keep order:false, stats:pseudo" + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#8", + " └─HashAgg 6400.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#8", + " └─Selection 8000.00 mpp[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))", + " └─HashJoin 10000.00 mpp[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", + " ├─Projection(Build) 7984.01 mpp[tiflash] Column#7, test.t.id, test.t.value", + " │ └─HashAgg 7984.01 mpp[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7984.01 mpp[tiflash] ", + " │ └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7984.01 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:B keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select count(distinct value) from t", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#4", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#4", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.t.value)->Column#4", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#4)->Column#5", - " └─Projection 1.00 batchCop[tiflash] Column#4", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct Column#4)->Column#5", + " └─Projection 1.00 mpp[tiflash] Column#4", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.t.value)->Column#4", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select count(distinct value), count(value), avg(value) from t", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#4, Column#5, div(Column#6, cast(case(eq(Column#7, 0), 1, Column#7), decimal(20,0) BINARY))->Column#6", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#7, funcs:sum(Column#10)->Column#6", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, funcs:count(test.t.value)->Column#8, funcs:count(test.t.value)->Column#9, funcs:sum(test.t.value)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#4, Column#5, div(Column#6, cast(case(eq(Column#7, 0), 1, Column#7), decimal(20,0) BINARY))->Column#6", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.t.value)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#7, funcs:sum(Column#10)->Column#6", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t.value, funcs:count(test.t.value)->Column#8, funcs:count(test.t.value)->Column#9, funcs:sum(test.t.value)->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] } ] @@ -5319,39 +5319,39 @@ "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", "Plan": [ "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, Column#8)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 6400.00 batchCop[tiflash] plus(Column#7, test.t.id)->Column#8", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(Column#7, test.t.id)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#11", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.id, Column#8)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 mpp[tiflash] plus(Column#7, test.t.id)->Column#8", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(Column#7, test.t.id)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#7, test.t.id", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#11", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5359,20 +5359,20 @@ "Plan": [ "Projection 7992.00 root test.t.id, test.t.value, Column#7, test.t.id", "└─TableReader 7992.00 root data:ExchangeSender", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 6393.60 batchCop[tiflash] ", - " │ └─ExchangeSender 6393.60 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 6393.60 batchCop[tiflash] lt(plus(test.t.value, cast(Column#7, decimal(20,0) BINARY)), 10)", - " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#10)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary], [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#10", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 6393.60 mpp[tiflash] ", + " │ └─ExchangeSender 6393.60 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 6393.60 mpp[tiflash] lt(plus(test.t.value, cast(Column#7, decimal(20,0) BINARY)), 10)", + " │ └─Projection 7992.00 mpp[tiflash] Column#7, test.t.id, test.t.value", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#10)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary], [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#10", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5381,9 +5381,9 @@ "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", "├─HashAgg(Build) 1.00 root funcs:count(Column#10)->Column#7", "│ └─TableReader 1.00 root data:ExchangeSender", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#10", - "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#10", + "│ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" @@ -5393,79 +5393,79 @@ "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 7992.00 batchCop[tiflash] Column#7", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#8", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 7992.00 mpp[tiflash] Column#7", + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7", + " └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#8", + " └─Projection 12487.50 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 mpp[tiflash] Column#7, test.t.id", + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", "Plan": [ "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#17)->Column#11, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#16)->Column#13", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#16", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#17)->Column#11, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 mpp[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", + " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.id, Column#13", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#16)->Column#13", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#16", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5473,15 +5473,15 @@ "Plan": [ "TopN 1.00 root test.t.value, offset:0, count:1", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TopN 1.00 batchCop[tiflash] test.t.value, offset:0, count:1", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 1.00 mpp[tiflash] test.t.value, offset:0, count:1", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5491,17 +5491,17 @@ "└─TopN 1.00 root Column#8, offset:0, count:1", " └─Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#8", " └─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value", - " └─TopN 1.00 batchCop[tiflash] Column#7, offset:0, count:1", - " └─Projection 12487.50 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t.id, test.t.value, test.t.id, test.t.value", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 12487.50 mpp[tiflash] test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5510,15 +5510,15 @@ "HashAgg 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 batchCop[tiflash] ExchangeType: PassThrough", - " └─TopN 20.00 batchCop[tiflash] test.t.value, offset:0, count:20", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 20.00 mpp[tiflash] test.t.value, offset:0, count:20", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5526,15 +5526,15 @@ "Plan": [ "Limit 1.00 root offset:0, count:1", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 1.00 cop[tiflash] offset:0, count:1", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 mpp[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5542,15 +5542,15 @@ "Plan": [ "Limit 1.00 root offset:0, count:1", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 1.00 cop[tiflash] offset:0, count:1", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 mpp[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -5559,15 +5559,15 @@ "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", "└─Limit 20.00 root offset:0, count:20", " └─TableReader 20.00 root data:ExchangeSender", - " └─ExchangeSender 20.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 20.00 cop[tiflash] offset:0, count:20", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 16.02 cop[tiflash] table:t1 keep order:false, stats:pseudo" + " └─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 20.00 mpp[tiflash] offset:0, count:20", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 16.02 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] } ] @@ -5686,19 +5686,19 @@ " ├─Selection(Build) 0.80 root ne(Column#27, 0)", " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", " │ └─TableReader 1.00 root data:ExchangeSender", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", - " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 mpp[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 12487.50 root data:ExchangeSender", - " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))", - " └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:table2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" ] } ] @@ -5711,10 +5711,10 @@ "Plan": [ "HashAgg 1.00 root funcs:group_concat(Column#7 separator \",\")->Column#5", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#7", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#7", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5724,14 +5724,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5741,13 +5741,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5757,14 +5757,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5774,13 +5774,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.col_1", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.col_1", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5790,14 +5790,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13, Column#14 order by Column#15 separator \",\")->Column#5, funcs:sum(Column#16)->Column#6, funcs:max(Column#17)->Column#7", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#14, test.ts.col_0, Column#10, Column#11", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#10, funcs:max(test.ts.col_0)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#12, Column#13, Column#14 order by Column#15 separator \",\")->Column#5, funcs:sum(Column#16)->Column#6, funcs:max(Column#17)->Column#7", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#14, test.ts.col_0, Column#10, Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#10, funcs:max(test.ts.col_0)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5807,14 +5807,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5824,14 +5824,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5841,13 +5841,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5857,14 +5857,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5874,13 +5874,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5890,14 +5890,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5907,13 +5907,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#19, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#19, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5923,15 +5923,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(distinct Column#32, Column#33, Column#34 order by Column#35 separator \",\")->Column#5, funcs:sum(Column#36)->Column#6, funcs:max(Column#37)->Column#7, funcs:sum(Column#38)->Column#20, funcs:sum(Column#39)->Column#8", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#34, test.ts.col_0, Column#21, Column#22, Column#23, Column#24, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#28, Column#29, Column#30, Column#31, funcs:count(1)->Column#21, funcs:max(Column#25)->Column#22, funcs:count(Column#26)->Column#23, funcs:sum(Column#27)->Column#24", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#40, funcs:group_concat(distinct Column#32, Column#33, Column#34 order by Column#35 separator \",\")->Column#5, funcs:sum(Column#36)->Column#6, funcs:max(Column#37)->Column#7, funcs:sum(Column#38)->Column#20, funcs:sum(Column#39)->Column#8", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#34, test.ts.col_0, Column#21, Column#22, Column#23, Column#24, test.ts.col_2", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#28, Column#29, Column#30, Column#31, funcs:count(1)->Column#21, funcs:max(Column#25)->Column#22, funcs:count(Column#26)->Column#23, funcs:sum(Column#27)->Column#24", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5941,13 +5941,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#18", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#18", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5957,15 +5957,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 order by Column#29 separator \",\")->Column#5, funcs:sum(Column#30)->Column#6, funcs:max(Column#31)->Column#7, funcs:sum(Column#32)->Column#14, funcs:sum(Column#33)->Column#8", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_0, Column#15, Column#16, Column#17, Column#18", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#22, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 order by Column#29 separator \",\")->Column#5, funcs:sum(Column#30)->Column#6, funcs:max(Column#31)->Column#7, funcs:sum(Column#32)->Column#14, funcs:sum(Column#33)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_0, Column#15, Column#16, Column#17, Column#18", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#22, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5975,14 +5975,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#27, 0), 1, Column#27), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:min(Column#30)->Column#7, funcs:sum(Column#31)->Column#27, funcs:sum(Column#32)->Column#8", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(Column#33, Column#34, Column#35 separator \",\")->Column#28, funcs:count(Column#36)->Column#29, funcs:min(Column#37)->Column#30, funcs:count(Column#38)->Column#31, funcs:sum(Column#39)->Column#32", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#39, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#27, 0), 1, Column#27), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:min(Column#30)->Column#7, funcs:sum(Column#31)->Column#27, funcs:sum(Column#32)->Column#8", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#40, funcs:group_concat(Column#33, Column#34, Column#35 separator \",\")->Column#28, funcs:count(Column#36)->Column#29, funcs:min(Column#37)->Column#30, funcs:count(Column#38)->Column#31, funcs:sum(Column#39)->Column#32", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#39, test.ts.col_2", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -5992,15 +5992,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(distinct Column#33, Column#34, Column#35 separator \",\")->Column#5, funcs:sum(Column#36)->Column#6, funcs:max(Column#37)->Column#7, funcs:sum(Column#38)->Column#20, funcs:sum(Column#39)->Column#8", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, Column#21, Column#22, Column#23, Column#24, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, Column#30, Column#31, Column#32, funcs:count(Column#25)->Column#21, funcs:max(Column#26)->Column#22, funcs:count(Column#27)->Column#23, funcs:sum(Column#28)->Column#24", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#40, funcs:group_concat(distinct Column#33, Column#34, Column#35 separator \",\")->Column#5, funcs:sum(Column#36)->Column#6, funcs:max(Column#37)->Column#7, funcs:sum(Column#38)->Column#20, funcs:sum(Column#39)->Column#8", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, Column#21, Column#22, Column#23, Column#24, test.ts.col_2", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#29, Column#30, Column#31, Column#32, funcs:count(Column#25)->Column#21, funcs:max(Column#26)->Column#22, funcs:count(Column#27)->Column#23, funcs:sum(Column#28)->Column#24", + " └─Projection 10000.00 mpp[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6011,10 +6011,10 @@ "Plan": [ "HashAgg 1.00 root funcs:group_concat(Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:avg(Column#17, Column#18)->Column#8", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#24, Column#25, Column#26 separator \",\")->Column#14, funcs:count(Column#27)->Column#15, funcs:min(Column#28)->Column#16, funcs:count(Column#29)->Column#17, funcs:sum(Column#30)->Column#18", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#24, Column#25, Column#26 separator \",\")->Column#14, funcs:count(Column#27)->Column#15, funcs:min(Column#28)->Column#16, funcs:count(Column#29)->Column#17, funcs:sum(Column#30)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6024,15 +6024,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:max(Column#30)->Column#7, funcs:sum(Column#31)->Column#14, funcs:sum(Column#32)->Column#8", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#28, Column#15, Column#16, Column#17, Column#18", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#22, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:max(Column#30)->Column#7, funcs:sum(Column#31)->Column#14, funcs:sum(Column#32)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#28, Column#15, Column#16, Column#17, Column#18", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#22, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6042,13 +6042,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:group_concat(Column#21, Column#22, Column#23 separator \",\")->Column#5, funcs:count(Column#24)->Column#6, funcs:group_concat(Column#25 order by Column#26 separator \",\")->Column#7, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#29, funcs:group_concat(Column#21, Column#22, Column#23 separator \",\")->Column#5, funcs:count(Column#24)->Column#6, funcs:group_concat(Column#25 order by Column#26 separator \",\")->Column#7, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6058,13 +6058,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#24, funcs:group_concat(distinct Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:count(Column#22)->Column#13, funcs:sum(Column#23)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#23, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#24, funcs:group_concat(distinct Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:count(Column#22)->Column#13, funcs:sum(Column#23)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#23, test.ts.col_2", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6074,13 +6074,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#18, Column#19 separator \",\")->Column#5, funcs:count(Column#20)->Column#6, funcs:group_concat(Column#21, Column#22 order by Column#23, Column#24 separator \",\")->Column#7, funcs:min(Column#25)->Column#8, funcs:count(Column#26)->Column#15, funcs:sum(Column#27)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#18, Column#19 separator \",\")->Column#5, funcs:count(Column#20)->Column#6, funcs:group_concat(Column#21, Column#22 order by Column#23, Column#24 separator \",\")->Column#7, funcs:min(Column#25)->Column#8, funcs:count(Column#26)->Column#15, funcs:sum(Column#27)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6090,13 +6090,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#12, 0), 1, Column#12), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:max(Column#22)->Column#8, funcs:count(Column#23)->Column#12, funcs:sum(Column#24)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#16, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#12, 0), 1, Column#12), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:max(Column#22)->Column#8, funcs:count(Column#23)->Column#12, funcs:sum(Column#24)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#16, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6106,15 +6106,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#29, Column#30, Column#31 separator \",\")->Column#5, funcs:count(distinct Column#32)->Column#6, funcs:group_concat(Column#33 separator \",\")->Column#7, funcs:max(Column#34)->Column#8, funcs:sum(Column#35)->Column#15, funcs:sum(Column#36)->Column#9", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_2, Column#16, Column#17, Column#18, Column#19", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#25, Column#26, Column#27, Column#28, funcs:group_concat(Column#20, Column#21 separator \",\")->Column#16, funcs:max(Column#22)->Column#17, funcs:count(Column#23)->Column#18, funcs:sum(Column#24)->Column#19", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#21, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#29, Column#30, Column#31 separator \",\")->Column#5, funcs:count(distinct Column#32)->Column#6, funcs:group_concat(Column#33 separator \",\")->Column#7, funcs:max(Column#34)->Column#8, funcs:sum(Column#35)->Column#15, funcs:sum(Column#36)->Column#9", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_2, Column#16, Column#17, Column#18, Column#19", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#25, Column#26, Column#27, Column#28, funcs:group_concat(Column#20, Column#21 separator \",\")->Column#16, funcs:max(Column#22)->Column#17, funcs:count(Column#23)->Column#18, funcs:sum(Column#24)->Column#19", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#21, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6124,15 +6124,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#21, 0), 1, Column#21), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:group_concat(distinct Column#35, Column#36, Column#37 separator \",\")->Column#5, funcs:count(distinct Column#38)->Column#6, funcs:group_concat(Column#39 separator \",\")->Column#7, funcs:max(Column#40)->Column#8, funcs:sum(Column#41)->Column#21, funcs:sum(Column#42)->Column#9", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#37, test.ts.col_2, Column#22, Column#23, Column#24, Column#25, test.ts.col_0", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_0, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#31, Column#32, Column#33, Column#34, funcs:group_concat(Column#26, Column#27 separator \",\")->Column#22, funcs:max(Column#28)->Column#23, funcs:count(Column#29)->Column#24, funcs:sum(Column#30)->Column#25", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#27, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#21, 0), 1, Column#21), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#43, funcs:group_concat(distinct Column#35, Column#36, Column#37 separator \",\")->Column#5, funcs:count(distinct Column#38)->Column#6, funcs:group_concat(Column#39 separator \",\")->Column#7, funcs:max(Column#40)->Column#8, funcs:sum(Column#41)->Column#21, funcs:sum(Column#42)->Column#9", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#37, test.ts.col_2, Column#22, Column#23, Column#24, Column#25, test.ts.col_0", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_0, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#31, Column#32, Column#33, Column#34, funcs:group_concat(Column#26, Column#27 separator \",\")->Column#22, funcs:max(Column#28)->Column#23, funcs:count(Column#29)->Column#24, funcs:sum(Column#30)->Column#25", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#27, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6142,14 +6142,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", @@ -6161,14 +6161,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"01\", 0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"01\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6178,14 +6178,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#8, var_string(20))->Column#10, cast(Column#9, var_string(20))->Column#11", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6195,14 +6195,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#7, var_string(20))->Column#8, cast(Column#7, var_string(20))->Column#9", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#7, var_string(20))->Column#8, cast(Column#7, var_string(20))->Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6212,14 +6212,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#13, var_string(20))->Column#15, cast(Column#14, var_string(20))->Column#16, Column#12", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, 10, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, cast(Column#14, var_string(20))->Column#16, Column#12", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, 10, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6229,14 +6229,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#14, funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#11, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13, Column#10", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#14, funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#11, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13, Column#10", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6246,14 +6246,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#13, var_string(20))->Column#15, Column#14, Column#12", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, Column#14, Column#12", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", @@ -6269,13 +6269,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#7, Column#7 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#7, Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6285,13 +6285,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"Gg\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"Gg\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6301,13 +6301,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"GG-10\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"GG-10\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6317,14 +6317,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] Column#8, cast(Column#9, var_string(20))->Column#11", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] Column#8, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", @@ -6336,13 +6336,13 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6352,14 +6352,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6369,14 +6369,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#7 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#9, funcs:firstrow(Column#8)->Column#7", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#8, test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#9, funcs:firstrow(Column#8)->Column#7", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#8, test.ts.col_0", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6386,14 +6386,14 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, Column#11, funcs:firstrow(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_1, test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:firstrow(Column#9)->Column#8", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_1, test.ts.col_0", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6403,15 +6403,15 @@ "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15 order by Column#16 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#15, Column#11, test.ts.col_1", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, Column#14, funcs:firstrow(Column#12)->Column#11", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#12, test.ts.col_1, gt(cast(test.ts.col_0, double BINARY), 10)->Column#14", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15 order by Column#16 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] cast(Column#10, var_string(20))->Column#15, Column#11, test.ts.col_1", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:firstrow(Column#12)->Column#11", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#12, test.ts.col_1, gt(cast(test.ts.col_0, double BINARY), 10)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" @@ -6444,13 +6444,13 @@ "Projection 8000.00 root Column#5", "└─Sort 8000.00 root test.t.id", " └─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6458,13 +6458,13 @@ "Plan": [ "Sort 8000.00 root Column#5", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#8", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6472,15 +6472,15 @@ "Plan": [ "Sort 8000.00 root Column#5", "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.t.id, test.t.name", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.name)->test.t.name", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.t.id, test.t.name", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.name)->test.t.name", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6488,20 +6488,20 @@ "Plan": [ "Sort 9990.00 root test.t.id", "└─TableReader 9990.00 root data:ExchangeSender", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 9990.00 batchCop[tiflash] test.t.id, test.t.id, test.t.value, test.t.name", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.id, test.t.id, test.t.value, test.t.name", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.id", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6509,46 +6509,46 @@ "Plan": [ "Sort 12487.50 root test.t.id", "└─TableReader 12487.50 root data:ExchangeSender", - " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 12487.50 cop[tiflash] test.t.id, test.t.value, test.t.name, test.t.id, test.t.value, test.t.name", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.t.id, test.t.value, test.t.name, test.t.id, test.t.value, test.t.name", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from ((select id from t order by 1) union all (select id+1 from t order by 1))c", "Plan": [ "TableReader 20000.00 root data:ExchangeSender", - "└─ExchangeSender 20000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Union 20000.00 cop[tiflash] ", - " ├─Projection 10000.00 cop[tiflash] cast(test.t.id, bigint(20) BINARY)->Column#10", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] plus(test.t.id, 1)->Column#10", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 20000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Union 20000.00 mpp[tiflash] ", + " ├─Projection 10000.00 mpp[tiflash] cast(test.t.id, bigint(20) BINARY)->Column#10", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { "SQL": "desc format = 'brief' select * from ((select count(*) from (select id,name from t order by id)a group by name,id order by id) union all (select id+1 from t order by 1))c", "Plan": [ "TableReader 18000.00 root data:ExchangeSender", - "└─ExchangeSender 18000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Union 18000.00 batchCop[tiflash] ", - " ├─Projection 8000.00 batchCop[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.t.id", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 batchCop[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "└─ExchangeSender 18000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Union 18000.00 mpp[tiflash] ", + " ├─Projection 8000.00 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", + " │ └─Projection 8000.00 mpp[tiflash] Column#5", + " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 mpp[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6556,8 +6556,8 @@ "Plan": [ "Sort 10000.00 root test.t.name", "└─TableReader 10000.00 root data:ExchangeSender", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] } ] @@ -6570,14 +6570,14 @@ "Plan": [ "HashAgg 1.00 root funcs:sum(Column#15)->Column#14", "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(test.partsupp.ps_supplycost)->Column#15", - " └─Projection 12500.00 batchCop[tiflash] test.partsupp.ps_supplycost", - " └─HashJoin 12500.00 batchCop[tiflash] inner join, equal:[eq(test.supplier.s_suppkey, test.partsupp.ps_suppkey)]", - " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:supplier keep order:false", - " └─TableFullScan(Probe) 800000.00 batchCop[tiflash] table:partsupp keep order:false" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(test.partsupp.ps_supplycost)->Column#15", + " └─Projection 12500.00 mpp[tiflash] test.partsupp.ps_supplycost", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.supplier.s_suppkey, test.partsupp.ps_suppkey)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:supplier keep order:false", + " └─TableFullScan(Probe) 800000.00 mpp[tiflash] table:partsupp keep order:false" ] } ] @@ -6589,18 +6589,18 @@ "SQL": "explain format = 'brief' select * from rp_t where a = 1 or a = 20", "Plan": [ "TableReader 20.00 root partition:p0,p3 data:ExchangeSender", - "└─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Selection 20.00 batchCop[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:rp_t keep order:false, stats:pseudo" + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t keep order:false, stats:pseudo, PartitionTableScan:true" ] }, { "SQL": "explain format = 'brief' select * from hp_t where a = 1 or a = 20", "Plan": [ "TableReader 20.00 root partition:p0,p1 data:ExchangeSender", - "└─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Selection 20.00 batchCop[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:hp_t keep order:false, stats:pseudo" + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t keep order:false, stats:pseudo, PartitionTableScan:true" ] }, { @@ -6608,10 +6608,10 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#5)->Column#3", "└─TableReader 1.00 root partition:p0,p3 data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#5", - " └─Selection 20.00 batchCop[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:rp_t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#5", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t keep order:false, stats:pseudo, PartitionTableScan:true" ] }, { @@ -6619,10 +6619,10 @@ "Plan": [ "HashAgg 1.00 root funcs:count(Column#5)->Column#3", "└─TableReader 1.00 root partition:p0,p1 data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#5", - " └─Selection 20.00 batchCop[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:hp_t keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#5", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t keep order:false, stats:pseudo, PartitionTableScan:true" ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index df15027594cd4..12998ed47236e 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2685,252 +2685,252 @@ "SQL": "select count(*) from employee group by deptid+1", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#14, funcs:count(1)->Column#13", - " └─Projection 10000.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#14", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#14, funcs:count(1)->Column#13", + " └─Projection 10000.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select count(distinct deptid) a from employee", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee join employee e1 using(deptid)", "Plan": [ "TableReader 12487.50 root data:ExchangeSender", - "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 12487.50 cop[tiflash] test.employee.deptid, test.employee.empid, test.employee.salary, test.employee.empid, test.employee.salary", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo" + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.employee.deptid, test.employee.empid, test.employee.salary, test.employee.empid, test.employee.salary", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo" ] }, { "SQL": "select count(distinct a) from (select count(distinct deptid) a from employee) x", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#6", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#5)->Column#6", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#6", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct Column#5)->Column#6", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b+1", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#7", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#7", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#15, funcs:count(Column#14)->Column#13", - " └─Projection 1.00 batchCop[tiflash] Column#5, plus(Column#6, 1)->Column#15", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#7", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#7", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#15, funcs:count(Column#14)->Column#13", + " └─Projection 1.00 mpp[tiflash] Column#5, plus(Column#6, 1)->Column#15", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#7", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#6, funcs:count(Column#5)->Column#7", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#7", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#6, funcs:count(Column#5)->Column#7", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee join (select count(distinct deptid) a, count(distinct empid) b from employee) e1", "Plan": [ "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 10000.00 batchCop[tiflash] CARTESIAN inner join", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 1.00 batchCop[tiflash] Column#9, Column#10", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#9, funcs:count(distinct test.employee.empid)->Column#10", - " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 mpp[tiflash] Column#9, Column#10", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#9, funcs:count(distinct test.employee.empid)->Column#10", + " │ └─ExchangeReceiver 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee e1 join (select count(distinct deptid) a from employee) e2 on e1.deptid = e2.a", "Plan": [ "TableReader 1.25 root data:ExchangeSender", - "└─ExchangeSender 1.25 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 1.25 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#9)]", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 1.00 batchCop[tiflash] Column#9", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#9", - " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + "└─ExchangeSender 1.25 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 1.25 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#9)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 mpp[tiflash] Column#9", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#9", + " │ └─ExchangeReceiver 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select count(distinct deptid) a from employee) e1 join employee e2 on e1.a = e2.deptid", "Plan": [ "TableReader 1.25 root data:ExchangeSender", - "└─ExchangeSender 1.25 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.25 batchCop[tiflash] Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", - " └─HashJoin 1.25 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#5)]", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 1.00 batchCop[tiflash] Column#5", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + "└─ExchangeSender 1.25 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.25 mpp[tiflash] Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 1.25 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#5)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 mpp[tiflash] Column#5", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e2 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select count(distinct deptid) a from employee) e1 join (select count(distinct deptid) b from employee) e2 on e1.a=e2.b", "Plan": [ "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 1.00 batchCop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 1.00 batchCop[tiflash] Column#5", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Projection(Probe) 1.00 batchCop[tiflash] Column#10", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 1.00 mpp[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 mpp[tiflash] Column#5", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 mpp[tiflash] Column#10", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", "Plan": [ "TableReader 12487.50 root data:ExchangeSender", - "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e2 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e2 keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#9, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d", "Plan": [ "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6400.00 batchCop[tiflash] inner join, equal:[eq(Column#6, Column#12)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─Projection(Probe) 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", - " └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6400.00 mpp[tiflash] inner join, equal:[eq(Column#6, Column#12)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─Projection(Probe) 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", + " └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " └─Projection 8000.00 mpp[tiflash] Column#11, test.employee.deptid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", + " └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { @@ -2947,13 +2947,13 @@ "Projection 1.25 root Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", "└─HashJoin 1.25 root inner join, equal:[eq(test.employee.deptid, Column#5)]", " ├─TableReader(Build) 1.00 root data:ExchangeSender", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─Projection 1.00 batchCop[tiflash] Column#5", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─Projection 1.00 mpp[tiflash] Column#5", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 mpp[tiflash] ", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" @@ -2964,117 +2964,117 @@ "Plan": [ "HashJoin 1.00 root inner join, equal:[eq(Column#5, Column#10)]", "├─TableReader(Build) 1.00 root data:ExchangeSender", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─Projection 1.00 batchCop[tiflash] Column#10", - "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", - "│ └─ExchangeReceiver 1.00 batchCop[tiflash] ", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - "│ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Projection 1.00 mpp[tiflash] Column#10", + "│ └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", + "│ └─ExchangeReceiver 1.00 mpp[tiflash] ", + "│ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + "│ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", "└─TableReader(Probe) 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", "Plan": [ "TableReader 12487.50 root data:ExchangeSender", - "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e2 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", - " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", + " └─Projection 9990.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e2 keep order:false, stats:pseudo" ] }, { "SQL": "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", "Plan": [ "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#10, Column#9", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", - " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#10, Column#9", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#9, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", + " └─Projection 9990.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 mpp[tiflash] table:e1 keep order:false, stats:pseudo" ] }, { "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d", "Plan": [ "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6400.00 batchCop[tiflash] inner join, equal:[eq(Column#6, Column#12)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", - " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", - " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 6400.00 batchCop[tiflash] ", - " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", - " └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", - " └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", - " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", - " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6400.00 mpp[tiflash] inner join, equal:[eq(Column#6, Column#12)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", + " │ └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", + " │ └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 6400.00 mpp[tiflash] ", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─Projection 6400.00 mpp[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", + " └─Selection 6400.00 mpp[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " └─Projection 8000.00 mpp[tiflash] Column#11, test.employee.deptid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", + " └─Projection 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", + " └─TableFullScan 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ] } ] From 798f33a1a6cf9aadaa85068ad9ed0c3a61997be3 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 10:08:03 +0800 Subject: [PATCH 2/7] refine code Signed-off-by: xufei --- executor/builder.go | 2 +- planner/core/common_plans.go | 2 +- planner/core/initialize.go | 77 ++++++++++++++++++---------------- planner/core/physical_plans.go | 7 ++-- 4 files changed, 47 insertions(+), 41 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 9d662ab19d964..d5881f6fd4a49 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3143,7 +3143,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea plans: v.TablePlans, tablePlan: v.GetTablePlan(), storeType: v.StoreType, - batchCop: v.ReadType == plannercore.BatchCop, + batchCop: v.ReadReqType == plannercore.BatchCop, } e.buildVirtualColumnInfo() if containsLimit(dagReq.Executors) { diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index af5af6f60de4a..590f186429bcb 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1396,7 +1396,7 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st } storeType = x.StoreType.Name() taskName := "" - switch x.ReadType { + switch x.ReadReqType { case Cop: taskName = "cop" case BatchCop: diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 91c3de7ecf6db..bdfb1c3b4da78 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -414,47 +414,52 @@ func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *Phys return &p } -// Init initializes PhysicalTableReader. -func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalTableReader { - p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableReader, &p, offset) - p.ReadType = Cop - if p.tablePlan != nil { - p.TablePlans = flattenPushDownPlan(p.tablePlan) - p.schema = p.tablePlan.Schema() - if p.StoreType == kv.TiFlash { - _, ok := p.tablePlan.(*PhysicalExchangeSender) - if ok { - p.ReadType = MPP - } else { - tableScans := p.GetTableScans() - // When PhysicalTableReader's store type is tiflash, has table scan - // and all table scans contained are not keepOrder, try to use batch cop. - if len(tableScans) > 0 { - for _, tableScan := range tableScans { - if tableScan.KeepOrder { - return &p - } - } +func (p PhysicalTableReader) AdjustReadReqType(ctx sessionctx.Context) { + if p.StoreType == kv.TiFlash { + _, ok := p.tablePlan.(*PhysicalExchangeSender) + if ok { + p.ReadReqType = MPP + return + } + tableScans := p.GetTableScans() + // When PhysicalTableReader's store type is tiflash, has table scan + // and all table scans contained are not keepOrder, try to use batch cop. + if len(tableScans) > 0 { + for _, tableScan := range tableScans { + if tableScan.KeepOrder { + return + } + } - // When allow batch cop is 1, only agg / topN uses batch cop. - // When allow batch cop is 2, every query uses batch cop. - switch ctx.GetSessionVars().AllowBatchCop { - case 1: - for _, plan := range p.TablePlans { - switch plan.(type) { - case *PhysicalHashAgg, *PhysicalStreamAgg, *PhysicalTopN: - p.ReadType = BatchCop - } - } - case 2: - p.ReadType = BatchCop + // When allow batch cop is 1, only agg / topN uses batch cop. + // When allow batch cop is 2, every query uses batch cop. + switch ctx.GetSessionVars().AllowBatchCop { + case 1: + for _, plan := range p.TablePlans { + switch plan.(type) { + case *PhysicalHashAgg, *PhysicalStreamAgg, *PhysicalTopN: + p.ReadReqType = BatchCop } } + case 2: + p.ReadReqType = BatchCop } } - if p.ReadType == BatchCop || p.ReadType == MPP { - setMppOrBatchCopForTableScan(p.tablePlan) - } + } +} + +// Init initializes PhysicalTableReader. +func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalTableReader { + p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableReader, &p, offset) + p.ReadReqType = Cop + if p.tablePlan == nil { + return &p + } + p.TablePlans = flattenPushDownPlan(p.tablePlan) + p.schema = p.tablePlan.Schema() + p.AdjustReadReqType(ctx) + if p.ReadReqType == BatchCop || p.ReadReqType == MPP { + setMppOrBatchCopForTableScan(p.tablePlan) } return &p } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index e076ab171c538..9419d3aecdba6 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -95,8 +95,9 @@ type PhysicalTableReader struct { // StoreType indicates table read from which type of store. StoreType kv.StoreType - // BatchCop = true means the cop task in the physical table reader will be executed in batch mode(use in TiFlash only) - ReadType ReadReqType + // ReadReqType is the read request type for current physical table reader, there are 3 kinds of read request: Cop, + // BatchCop and MPP, currently, the latter two are only used in TiFlash + ReadReqType ReadReqType IsCommonHandle bool @@ -183,7 +184,7 @@ func (p *PhysicalTableReader) Clone() (PhysicalPlan, error) { } cloned.physicalSchemaProducer = *base cloned.StoreType = p.StoreType - cloned.ReadType = p.ReadType + cloned.ReadReqType = p.ReadReqType cloned.IsCommonHandle = p.IsCommonHandle if cloned.tablePlan, err = p.tablePlan.Clone(); err != nil { return nil, err From 2944bc01e1d706f1ccc3e2bd53b0b11c12fcc113 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 10:23:30 +0800 Subject: [PATCH 3/7] refine Signed-off-by: xufei --- planner/core/common_plans.go | 14 ++------------ planner/core/initialize.go | 1 + planner/core/physical_plans.go | 13 +++++++++++++ 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 590f186429bcb..487cab724d2b8 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1387,24 +1387,14 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st switch x := p.(type) { case *PhysicalTableReader: - var storeType string switch x.StoreType { case kv.TiKV, kv.TiFlash, kv.TiDB: // expected do nothing default: return errors.Errorf("the store type %v is unknown", x.StoreType) } - storeType = x.StoreType.Name() - taskName := "" - switch x.ReadReqType { - case Cop: - taskName = "cop" - case BatchCop: - taskName = "batchCop" - case MPP: - taskName = "mpp" - } - err = e.explainPlanInRowFormat(x.tablePlan, taskName+"["+storeType+"]", "", childIndent, true) + taskName := x.ReadReqType.Name() + "[" + x.StoreType.Name() + "]" + err = e.explainPlanInRowFormat(x.tablePlan, taskName, "", childIndent, true) case *PhysicalIndexReader: err = e.explainPlanInRowFormat(x.indexPlan, "cop[tikv]", "", childIndent, true) case *PhysicalIndexLookUpReader: diff --git a/planner/core/initialize.go b/planner/core/initialize.go index bdfb1c3b4da78..f71346b94dacf 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -439,6 +439,7 @@ func (p PhysicalTableReader) AdjustReadReqType(ctx sessionctx.Context) { switch plan.(type) { case *PhysicalHashAgg, *PhysicalStreamAgg, *PhysicalTopN: p.ReadReqType = BatchCop + return } } case 2: diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 9419d3aecdba6..1f87084d18dab 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -84,6 +84,19 @@ const ( MPP ) +// Name returns the name of read request type. +func (r ReadReqType) Name() string { + switch r { + case BatchCop: + return "batchCop" + case MPP: + return "mpp" + default: + // return cop by default + return "cop" + } +} + // PhysicalTableReader is the table reader in tidb. type PhysicalTableReader struct { physicalSchemaProducer From e9b74e80a4d573b8ef6c3cab4d883cbff128f133 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 10:51:27 +0800 Subject: [PATCH 4/7] refine Signed-off-by: xufei --- planner/core/physical_plans.go | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 1f87084d18dab..866c765260aa3 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -154,6 +154,7 @@ func (p *PhysicalTableReader) GetTableScan() (*PhysicalTableScan, error) { return tableScans[0], nil } +// setMppOrBatchCopForTableScan set IsMPPOrBatchCop for all TableScan. func setMppOrBatchCopForTableScan(curPlan PhysicalPlan) { if ts, ok := curPlan.(*PhysicalTableScan); ok { ts.IsMPPOrBatchCop = true From e12fa86d6d0ca45ae58042a107a58320163d5948 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 11:56:30 +0800 Subject: [PATCH 5/7] fix ci Signed-off-by: xufei --- planner/core/initialize.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/initialize.go b/planner/core/initialize.go index f71346b94dacf..2d7e23fe46d76 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -414,7 +414,7 @@ func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *Phys return &p } -func (p PhysicalTableReader) AdjustReadReqType(ctx sessionctx.Context) { +func (p *PhysicalTableReader) AdjustReadReqType(ctx sessionctx.Context) { if p.StoreType == kv.TiFlash { _, ok := p.tablePlan.(*PhysicalExchangeSender) if ok { From 7b2a65a976a324f1e96cb2ab3fabd2fe8a73998f Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 12:07:43 +0800 Subject: [PATCH 6/7] fix ci Signed-off-by: xufei --- planner/core/physical_plans.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 866c765260aa3..1af92ba75b59f 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -73,11 +73,11 @@ type tableScanAndPartitionInfo struct { partitionInfo PartitionInfo } -type ReadReqType uint8 +type readReqType uint8 const ( // Cop means read from storage by cop request. - Cop ReadReqType = iota + Cop readReqType = iota // BatchCop means read from storage by BatchCop request, only used for TiFlash BatchCop // MPP means read from storage by MPP request, only used for TiFlash @@ -85,7 +85,7 @@ const ( ) // Name returns the name of read request type. -func (r ReadReqType) Name() string { +func (r readReqType) Name() string { switch r { case BatchCop: return "batchCop" @@ -110,7 +110,7 @@ type PhysicalTableReader struct { // ReadReqType is the read request type for current physical table reader, there are 3 kinds of read request: Cop, // BatchCop and MPP, currently, the latter two are only used in TiFlash - ReadReqType ReadReqType + ReadReqType readReqType IsCommonHandle bool From 8e09000a962bbbc0308674cba6d74139ada75a9f Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 23 Mar 2022 12:08:23 +0800 Subject: [PATCH 7/7] fix ci Signed-off-by: xufei --- planner/core/initialize.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 2d7e23fe46d76..9b1a4bf858c6c 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -414,7 +414,7 @@ func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *Phys return &p } -func (p *PhysicalTableReader) AdjustReadReqType(ctx sessionctx.Context) { +func (p *PhysicalTableReader) adjustReadReqType(ctx sessionctx.Context) { if p.StoreType == kv.TiFlash { _, ok := p.tablePlan.(*PhysicalExchangeSender) if ok { @@ -458,7 +458,7 @@ func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalT } p.TablePlans = flattenPushDownPlan(p.tablePlan) p.schema = p.tablePlan.Schema() - p.AdjustReadReqType(ctx) + p.adjustReadReqType(ctx) if p.ReadReqType == BatchCop || p.ReadReqType == MPP { setMppOrBatchCopForTableScan(p.tablePlan) }