Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
  • Loading branch information
Howie59 committed Apr 27, 2021
2 parents b74d2d5 + d67beef commit 9f13862
Show file tree
Hide file tree
Showing 79 changed files with 2,400 additions and 1,004 deletions.
18 changes: 9 additions & 9 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -194,29 +194,29 @@ test t4 1 expr_idx 1 NULL NULL (`a` + `b` + 1) 2 YES NO
explain format = 'brief' select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#5
└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#11
└─StreamAgg 1.00 root funcs:firstrow(Column#9)->Column#7
└─TableReader 1.00 root data:StreamAgg
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#9
└─Selection 10.00 cop[tikv] eq(test.t1.c3, 100)
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain format = 'brief' select 1 from (select count(c2), count(c3) from t1) k;
id estRows task access object operator info
Projection 1.00 root 1->Column#6
└─StreamAgg 1.00 root funcs:firstrow(Column#16)->Column#11
└─StreamAgg 1.00 root funcs:firstrow(Column#14)->Column#9
└─TableReader 1.00 root data:StreamAgg
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#16
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#14
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain format = 'brief' select count(1) from (select max(c2), count(c3) as m from t1) k;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#6
└─StreamAgg 1.00 root funcs:firstrow(Column#15)->Column#10
└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#8
└─TableReader 1.00 root data:StreamAgg
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#15
└─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain format = 'brief' select count(1) from (select count(c2) from t1 group by c3) k;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#5
└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#9
└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#7
└─TableReader 10000.00 root data:TableFullScan
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
set @@session.tidb_opt_insubq_to_join_and_agg=0;
Expand Down Expand Up @@ -498,8 +498,8 @@ PRIMARY KEY (`id`)
explain format = 'brief' SELECT COUNT(1) FROM (SELECT COALESCE(b.region_name, '不详') region_name, SUM(a.registration_num) registration_num FROM (SELECT stat_date, show_date, region_id, 0 registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202 UNION ALL SELECT stat_date, show_date, region_id, registration_num registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202) a LEFT JOIN test02 b ON a.region_id = b.id WHERE registration_num > 0 AND a.stat_date >= '20191202' AND a.stat_date <= '20191202' GROUP BY a.stat_date , a.show_date , COALESCE(b.region_name, '不详') ) JLS;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#22
└─HashAgg 8000.00 root group by:Column#34, Column#35, Column#36, funcs:firstrow(1)->Column#33
└─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#36
└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:firstrow(1)->Column#31
└─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#34
└─HashJoin 10000.01 root left outer join, equal:[eq(Column#16, test.test02.id)]
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:b keep order:false, stats:pseudo
Expand Down
15 changes: 7 additions & 8 deletions cmd/explaintest/r/explain_join_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -7,14 +7,13 @@ load stats 's/explain_join_stats_lo.json';
explain format = 'brief' select count(*) from e, lo where lo.a=e.a and e.b=22336;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#5
└─Projection 19977.00 root 1->Column#6
└─HashJoin 19977.00 root inner join, equal:[eq(test.lo.a, test.e.a)]
├─TableReader(Build) 250.00 root data:TableFullScan
│ └─TableFullScan 250.00 cop[tikv] table:lo keep order:false
└─IndexLookUp(Probe) 19977.00 root
├─IndexRangeScan(Build) 19977.00 cop[tikv] table:e, index:idx_b(b) range:[22336,22336], keep order:false
└─Selection(Probe) 19977.00 cop[tikv] not(isnull(test.e.a))
└─TableRowIDScan 19977.00 cop[tikv] table:e keep order:false
└─HashJoin 19977.00 root inner join, equal:[eq(test.lo.a, test.e.a)]
├─TableReader(Build) 250.00 root data:TableFullScan
│ └─TableFullScan 250.00 cop[tikv] table:lo keep order:false
└─IndexLookUp(Probe) 19977.00 root
├─IndexRangeScan(Build) 19977.00 cop[tikv] table:e, index:idx_b(b) range:[22336,22336], keep order:false
└─Selection(Probe) 19977.00 cop[tikv] not(isnull(test.e.a))
└─TableRowIDScan 19977.00 cop[tikv] table:e keep order:false
explain format = 'brief' select /*+ TIDB_INLJ(e) */ count(*) from e, lo where lo.a=e.a and e.b=22336;
id estRows task access object operator info
StreamAgg 1.00 root funcs:count(1)->Column#5
Expand Down
1 change: 1 addition & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -592,6 +592,7 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error {
}

if historyJob.Error != nil {
logutil.BgLogger().Info("[ddl] DDL job is failed", zap.Int64("jobID", jobID))
return errors.Trace(historyJob.Error)
}
// Only for JobStateCancelled job which is adding columns or drop columns.
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -563,7 +563,7 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error {
d.mu.hook.OnJobUpdated(job)
d.mu.RUnlock()

if job.IsSynced() || job.IsCancelled() {
if job.IsSynced() || job.IsCancelled() || job.IsRollbackDone() {
asyncNotify(d.ddlJobDoneCh)
}
}
Expand Down
4 changes: 2 additions & 2 deletions executor/aggfuncs/func_group_concat_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,12 +44,12 @@ func (s *testSuite) TestGroupConcat(c *C) {
s.testMultiArgsAggFunc(c, test2)

defer func() {
err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, types.NewStringDatum("1024"))
err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, "1024")
c.Assert(err, IsNil)
}()
// minimum GroupConcatMaxLen is 4
for i := 4; i <= 7; i++ {
err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, types.NewStringDatum(fmt.Sprint(i)))
err := variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, fmt.Sprint(i))
c.Assert(err, IsNil)
test2 = buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00"[:i])
test2.orderBy = true
Expand Down
40 changes: 16 additions & 24 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1204,6 +1204,12 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo
} else {
e.buildTypes, e.probeTypes = rightTypes, leftTypes
}
for _, key := range e.buildKeys {
e.buildTypes[key.Index].Flag = key.RetType.Flag
}
for _, key := range e.probeKeys {
e.probeTypes[key.Index].Flag = key.RetType.Flag
}
return e
}

Expand Down Expand Up @@ -2719,24 +2725,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E
return ret
}

func buildPartitionTable(b *executorBuilder, tblInfo *model.TableInfo, partitionInfo *plannercore.PartitionInfo, e Executor, n nextPartition) (Executor, error) {
tmp, _ := b.is.TableByID(tblInfo.ID)
tbl := tmp.(table.PartitionedTable)
partitions, err := partitionPruning(b.ctx, tbl, partitionInfo.PruningConds, partitionInfo.PartitionNames, partitionInfo.Columns, partitionInfo.ColumnNames)
if err != nil {
return nil, err
}

if len(partitions) == 0 {
return &TableDualExec{baseExecutor: *e.base()}, nil
}
return &PartitionTableExecutor{
baseExecutor: *e.base(),
partitions: partitions,
nextPartition: n,
}, nil
}

func buildIndexRangeForEachPartition(ctx sessionctx.Context, usedPartitions []table.PhysicalTable, contentPos []int64,
lookUpContent []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (map[int64][]*ranger.Range, error) {
contentBucket := make(map[int64][]*indexJoinLookUpContent)
Expand Down Expand Up @@ -3064,13 +3052,16 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo
return ret
}

nextPartition := nextPartitionForIndexLookUp{exec: ret}
exec, err := buildPartitionTable(b, ts.Table, &v.PartitionInfo, ret, nextPartition)
tmp, _ := b.is.TableByID(is.Table.ID)
tbl := tmp.(table.PartitionedTable)
partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames)
if err != nil {
b.err = err
return nil
}
return exec
ret.partitionTableMode = true
ret.prunedPartitions = partitions
return ret
}

func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalIndexMergeReader) (*IndexMergeReaderExecutor, error) {
Expand Down Expand Up @@ -3170,13 +3161,14 @@ func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMerg
return ret
}

nextPartition := nextPartitionForIndexMerge{ret}
exec, err := buildPartitionTable(b, ts.Table, &v.PartitionInfo, ret, nextPartition)
tmp, _ := b.is.TableByID(ts.Table.ID)
partitions, err := partitionPruning(b.ctx, tmp.(table.PartitionedTable), v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames)
if err != nil {
b.err = err
return nil
}
return exec
ret.partitionTableMode, ret.prunedPartitions = true, partitions
return ret
}

// dataReaderBuilder build an executor.
Expand Down
5 changes: 2 additions & 3 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/gcutil"
Expand Down Expand Up @@ -238,11 +237,11 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error {
sessionVars := e.ctx.GetSessionVars()
if err == nil && strings.ToLower(sessionVars.CurrentDB) == dbName.L {
sessionVars.CurrentDB = ""
err = variable.SetSessionSystemVar(sessionVars, variable.CharsetDatabase, types.NewStringDatum(mysql.DefaultCharset))
err = variable.SetSessionSystemVar(sessionVars, variable.CharsetDatabase, mysql.DefaultCharset)
if err != nil {
return err
}
err = variable.SetSessionSystemVar(sessionVars, variable.CollationDatabase, types.NewStringDatum(mysql.DefaultCollationName))
err = variable.SetSessionSystemVar(sessionVars, variable.CollationDatabase, mysql.DefaultCollationName)
if err != nil {
return err
}
Expand Down
Loading

0 comments on commit 9f13862

Please sign in to comment.