Skip to content

Commit 204d780

Browse files
authored
planner, util: move FastIntSet to util (#47723)
1 parent aebf22d commit 204d780

13 files changed

+195
-168
lines changed

pkg/expression/BUILD.bazel

+1-1
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,6 @@ go_library(
8282
"//pkg/parser/opcode",
8383
"//pkg/parser/terror",
8484
"//pkg/parser/types",
85-
"//pkg/planner/funcdep",
8685
"//pkg/privilege",
8786
"//pkg/sessionctx",
8887
"//pkg/sessionctx/stmtctx",
@@ -98,6 +97,7 @@ go_library(
9897
"//pkg/util/encrypt",
9998
"//pkg/util/generatedexpr",
10099
"//pkg/util/hack",
100+
"//pkg/util/intset",
101101
"//pkg/util/logutil",
102102
"//pkg/util/mathutil",
103103
"//pkg/util/mock",

pkg/expression/grouping_sets.go

+13-13
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@ import (
1919

2020
"github.com/pingcap/tidb/pkg/kv"
2121
"github.com/pingcap/tidb/pkg/parser/mysql"
22-
fd "github.com/pingcap/tidb/pkg/planner/funcdep"
2322
"github.com/pingcap/tidb/pkg/sessionctx"
2423
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
24+
"github.com/pingcap/tidb/pkg/util/intset"
2525
"github.com/pingcap/tidb/pkg/util/size"
2626
"github.com/pingcap/tipb/go-tipb"
2727
)
@@ -178,7 +178,7 @@ func (gss GroupingSets) TargetOne(normalAggArgs []Expression) int {
178178
return 0
179179
}
180180
// for other normal agg args like: count(a), count(a+b), count(not(a is null)) and so on.
181-
normalAggArgsIDSet := fd.NewFastIntSet()
181+
normalAggArgsIDSet := intset.NewFastIntSet()
182182
for _, one := range columnInNormalAggArgs {
183183
normalAggArgsIDSet.Insert(int(one.UniqueID))
184184
}
@@ -202,7 +202,7 @@ func (gss GroupingSets) TargetOne(normalAggArgs []Expression) int {
202202
func (gss GroupingSets) NeedCloneColumn() bool {
203203
// for grouping sets like: {<a,c>},{<c>} / {<a,c>},{<b,c>}
204204
// the column c should be copied one more time here, otherwise it will be filled with null values and not visible for the other grouping set again.
205-
setIDs := make([]*fd.FastIntSet, 0, len(gss))
205+
setIDs := make([]*intset.FastIntSet, 0, len(gss))
206206
for _, groupingSet := range gss {
207207
setIDs = append(setIDs, groupingSet.AllColIDs())
208208
}
@@ -231,8 +231,8 @@ func (gs GroupingSet) IsEmpty() bool {
231231
}
232232

233233
// AllColIDs collect all the grouping col's uniqueID. (here assuming that all the grouping expressions are single col)
234-
func (gs GroupingSet) AllColIDs() *fd.FastIntSet {
235-
res := fd.NewFastIntSet()
234+
func (gs GroupingSet) AllColIDs() *intset.FastIntSet {
235+
res := intset.NewFastIntSet()
236236
for _, groupingExprs := range gs {
237237
// on the condition that every grouping expression is single column.
238238
// eg: group by a, b, c
@@ -313,8 +313,8 @@ func (gss GroupingSets) IsEmpty() bool {
313313
}
314314

315315
// AllSetsColIDs is used to collect all the column id inside into a fast int set.
316-
func (gss GroupingSets) AllSetsColIDs() *fd.FastIntSet {
317-
res := fd.NewFastIntSet()
316+
func (gss GroupingSets) AllSetsColIDs() *intset.FastIntSet {
317+
res := intset.NewFastIntSet()
318318
for _, groupingSet := range gss {
319319
res.UnionWith(*groupingSet.AllColIDs())
320320
}
@@ -361,8 +361,8 @@ func (g GroupingExprs) IsEmpty() bool {
361361

362362
// SubSetOf is used to do the logical computation of subset between two grouping expressions.
363363
func (g GroupingExprs) SubSetOf(other GroupingExprs) bool {
364-
oldOne := fd.NewFastIntSet()
365-
newOne := fd.NewFastIntSet()
364+
oldOne := intset.NewFastIntSet()
365+
newOne := intset.NewFastIntSet()
366366
for _, one := range g {
367367
oldOne.Insert(int(one.(*Column).UniqueID))
368368
}
@@ -373,8 +373,8 @@ func (g GroupingExprs) SubSetOf(other GroupingExprs) bool {
373373
}
374374

375375
// IDSet is used to collect column ids inside grouping expressions into a fast int set.
376-
func (g GroupingExprs) IDSet() *fd.FastIntSet {
377-
res := fd.NewFastIntSet()
376+
func (g GroupingExprs) IDSet() *intset.FastIntSet {
377+
res := intset.NewFastIntSet()
378378
for _, one := range g {
379379
res.Insert(int(one.(*Column).UniqueID))
380380
}
@@ -493,7 +493,7 @@ func AdjustNullabilityFromGroupingSets(gss GroupingSets, schema *Schema) {
493493
// set, so it won't be filled with null value at any time, the nullable change is unnecessary.
494494
groupingIDs := gss.AllSetsColIDs()
495495
// cache the grouping ids set to avoid fetch them multi times below.
496-
groupingIDsSlice := make([]*fd.FastIntSet, 0, len(gss))
496+
groupingIDsSlice := make([]*intset.FastIntSet, 0, len(gss))
497497
for _, oneGroupingSet := range gss {
498498
groupingIDsSlice = append(groupingIDsSlice, oneGroupingSet.AllColIDs())
499499
}
@@ -570,7 +570,7 @@ func (gss GroupingSets) DistinctSize() (int, []uint64, map[int]map[uint64]struct
570570
func (gss GroupingSets) DistinctSizeWithThreshold(N int) (int, []uint64, map[int]map[uint64]struct{}) {
571571
// all the group by item are col, deduplicate from id-set.
572572
distinctGroupingIDsPos := make([]int, 0, len(gss))
573-
originGroupingIDsSlice := make([]*fd.FastIntSet, 0, len(gss))
573+
originGroupingIDsSlice := make([]*intset.FastIntSet, 0, len(gss))
574574

575575
for _, oneGroupingSet := range gss {
576576
curIDs := oneGroupingSet.AllColIDs()

pkg/planner/core/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -143,6 +143,7 @@ go_library(
143143
"//pkg/util/hack",
144144
"//pkg/util/hint",
145145
"//pkg/util/intest",
146+
"//pkg/util/intset",
146147
"//pkg/util/kvcache",
147148
"//pkg/util/logutil",
148149
"//pkg/util/mathutil",

pkg/planner/core/logical_plan_builder.go

+10-9
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,7 @@ import (
6161
"github.com/pingcap/tidb/pkg/util/dbterror"
6262
"github.com/pingcap/tidb/pkg/util/hack"
6363
"github.com/pingcap/tidb/pkg/util/hint"
64+
"github.com/pingcap/tidb/pkg/util/intset"
6465
"github.com/pingcap/tidb/pkg/util/logutil"
6566
"github.com/pingcap/tidb/pkg/util/plancodec"
6667
"github.com/pingcap/tidb/pkg/util/set"
@@ -1809,7 +1810,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields
18091810
if fields[offset].AuxiliaryColInAgg {
18101811
continue
18111812
}
1812-
item := fd.NewFastIntSet()
1813+
item := intset.NewFastIntSet()
18131814
switch x := expr.(type) {
18141815
case *expression.Column:
18151816
item.Insert(int(x.UniqueID))
@@ -1851,7 +1852,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields
18511852
baseCols := expression.ExtractColumns(expr)
18521853
errShowCol := baseCols[0]
18531854
for _, col := range baseCols {
1854-
colSet := fd.NewFastIntSet(int(col.UniqueID))
1855+
colSet := intset.NewFastIntSet(int(col.UniqueID))
18551856
if !colSet.SubsetOf(strictClosure) {
18561857
errShowCol = col
18571858
break
@@ -1876,7 +1877,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields
18761877
}
18771878
if fds.GroupByCols.Only1Zero() {
18781879
// maxOneRow is delayed from agg's ExtractFD logic since some details listed in it.
1879-
projectionUniqueIDs := fd.NewFastIntSet()
1880+
projectionUniqueIDs := intset.NewFastIntSet()
18801881
for _, expr := range proj.Exprs {
18811882
switch x := expr.(type) {
18821883
case *expression.Column:
@@ -5317,15 +5318,15 @@ func (ds *DataSource) ExtractFD() *fd.FDSet {
53175318
// Once the all conditions are not equal to nil, built it again.
53185319
if ds.fdSet == nil || ds.allConds != nil {
53195320
fds := &fd.FDSet{HashCodeToUniqueID: make(map[string]int)}
5320-
allCols := fd.NewFastIntSet()
5321+
allCols := intset.NewFastIntSet()
53215322
// should use the column's unique ID avoiding fdSet conflict.
53225323
for _, col := range ds.TblCols {
53235324
// todo: change it to int64
53245325
allCols.Insert(int(col.UniqueID))
53255326
}
53265327
// int pk doesn't store its index column in indexInfo.
53275328
if ds.tableInfo.PKIsHandle {
5328-
keyCols := fd.NewFastIntSet()
5329+
keyCols := intset.NewFastIntSet()
53295330
for _, col := range ds.TblCols {
53305331
if mysql.HasPriKeyFlag(col.RetType.GetFlag()) {
53315332
keyCols.Insert(int(col.UniqueID))
@@ -5351,7 +5352,7 @@ func (ds *DataSource) ExtractFD() *fd.FDSet {
53515352
}
53525353
// other indices including common handle.
53535354
for _, idx := range ds.tableInfo.Indices {
5354-
keyCols := fd.NewFastIntSet()
5355+
keyCols := intset.NewFastIntSet()
53555356
allColIsNotNull := true
53565357
if ds.isForUpdateRead && changed {
53575358
latestIndex, ok := latestIndexes[idx.ID]
@@ -5410,14 +5411,14 @@ func (ds *DataSource) ExtractFD() *fd.FDSet {
54105411
// the generated column is sequentially dependent on the forward column.
54115412
// a int, b int as (a+1), c int as (b+1), here we can build the strict FD down:
54125413
// {a} -> {b}, {b} -> {c}, put the maintenance of the dependencies between generated columns to the FD graph.
5413-
notNullCols := fd.NewFastIntSet()
5414+
notNullCols := intset.NewFastIntSet()
54145415
for _, col := range ds.TblCols {
54155416
if col.VirtualExpr != nil {
5416-
dependencies := fd.NewFastIntSet()
5417+
dependencies := intset.NewFastIntSet()
54175418
dependencies.Insert(int(col.UniqueID))
54185419
// dig out just for 1 level.
54195420
directBaseCol := expression.ExtractColumns(col.VirtualExpr)
5420-
determinant := fd.NewFastIntSet()
5421+
determinant := intset.NewFastIntSet()
54215422
for _, col := range directBaseCol {
54225423
determinant.Insert(int(col.UniqueID))
54235424
}

pkg/planner/core/logical_plans.go

+28-27
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@ import (
3636
"github.com/pingcap/tidb/pkg/statistics"
3737
"github.com/pingcap/tidb/pkg/table"
3838
"github.com/pingcap/tidb/pkg/types"
39+
"github.com/pingcap/tidb/pkg/util/intset"
3940
"github.com/pingcap/tidb/pkg/util/logutil"
4041
"github.com/pingcap/tidb/pkg/util/ranger"
4142
"github.com/pingcap/tidb/pkg/util/size"
@@ -298,7 +299,7 @@ func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expres
298299
outerFD, innerFD := p.children[0].ExtractFD(), p.children[1].ExtractFD()
299300
innerCondition := p.RightConditions
300301
outerCondition := p.LeftConditions
301-
outerCols, innerCols := fd.NewFastIntSet(), fd.NewFastIntSet()
302+
outerCols, innerCols := intset.NewFastIntSet(), intset.NewFastIntSet()
302303
for _, col := range p.children[0].Schema().Columns {
303304
outerCols.Insert(int(col.UniqueID))
304305
}
@@ -326,7 +327,7 @@ func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expres
326327
equivUniqueIDs := extractEquivalenceCols(allConds, p.SCtx(), filterFD)
327328

328329
filterFD.AddConstants(constUniqueIDs)
329-
equivOuterUniqueIDs := fd.NewFastIntSet()
330+
equivOuterUniqueIDs := intset.NewFastIntSet()
330331
equivAcrossNum := 0
331332
for _, equiv := range equivUniqueIDs {
332333
filterFD.AddEquivalence(equiv[0], equiv[1])
@@ -354,7 +355,7 @@ func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expres
354355
// other condition may contain right side cols, it doesn't affect the judgement of intersection of non-left-equiv cols.
355356
outConditionCols = append(outConditionCols, expression.ExtractColumnsFromExpressions(nil, p.OtherConditions, nil)...)
356357
}
357-
outerConditionUniqueIDs := fd.NewFastIntSet()
358+
outerConditionUniqueIDs := intset.NewFastIntSet()
358359
for _, col := range outConditionCols {
359360
outerConditionUniqueIDs.Insert(int(col.UniqueID))
360361
}
@@ -857,8 +858,8 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet {
857858
// basically extract the children's fdSet.
858859
fds := p.logicalSchemaProducer.ExtractFD()
859860
// collect the output columns' unique ID.
860-
outputColsUniqueIDs := fd.NewFastIntSet()
861-
notnullColsUniqueIDs := fd.NewFastIntSet()
861+
outputColsUniqueIDs := intset.NewFastIntSet()
862+
notnullColsUniqueIDs := intset.NewFastIntSet()
862863
outputColsUniqueIDsArray := make([]int, 0, len(p.Schema().Columns))
863864
// here schema extended columns may contain expr, const and column allocated with uniqueID.
864865
for _, one := range p.Schema().Columns {
@@ -885,7 +886,7 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet {
885886
constantUniqueID = outputColsUniqueIDsArray[idx]
886887
fds.RegisterUniqueID(string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)), constantUniqueID)
887888
}
888-
fds.AddConstants(fd.NewFastIntSet(constantUniqueID))
889+
fds.AddConstants(intset.NewFastIntSet(constantUniqueID))
889890
case *expression.ScalarFunction:
890891
// t1(a,b,c), t2(m,n)
891892
// select a, (select c+n from t2 where m=b) from t1;
@@ -908,9 +909,9 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet {
908909
} else {
909910
// since the scalar's hash code has been registered before, the equivalence exists between the unique ID
910911
// allocated by phase of building-projection-for-scalar and that of previous registered unique ID.
911-
fds.AddEquivalence(fd.NewFastIntSet(scalarUniqueID), fd.NewFastIntSet(outputColsUniqueIDsArray[idx]))
912+
fds.AddEquivalence(intset.NewFastIntSet(scalarUniqueID), intset.NewFastIntSet(outputColsUniqueIDsArray[idx]))
912913
}
913-
determinants := fd.NewFastIntSet()
914+
determinants := intset.NewFastIntSet()
914915
extractedColumns := expression.ExtractColumns(x)
915916
extractedCorColumns := expression.ExtractCorColumns(x)
916917
for _, one := range extractedColumns {
@@ -927,7 +928,7 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet {
927928
if notnull || determinants.SubsetOf(fds.NotNullCols) {
928929
notnullColsUniqueIDs.Insert(scalarUniqueID)
929930
}
930-
fds.AddStrictFunctionalDependency(determinants, fd.NewFastIntSet(scalarUniqueID))
931+
fds.AddStrictFunctionalDependency(determinants, intset.NewFastIntSet(scalarUniqueID))
931932
}
932933
}
933934

@@ -1013,10 +1014,10 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
10131014
// basically extract the children's fdSet.
10141015
fds := la.logicalSchemaProducer.ExtractFD()
10151016
// collect the output columns' unique ID.
1016-
outputColsUniqueIDs := fd.NewFastIntSet()
1017-
notnullColsUniqueIDs := fd.NewFastIntSet()
1018-
groupByColsUniqueIDs := fd.NewFastIntSet()
1019-
groupByColsOutputCols := fd.NewFastIntSet()
1017+
outputColsUniqueIDs := intset.NewFastIntSet()
1018+
notnullColsUniqueIDs := intset.NewFastIntSet()
1019+
groupByColsUniqueIDs := intset.NewFastIntSet()
1020+
groupByColsOutputCols := intset.NewFastIntSet()
10201021
// Since the aggregation is build ahead of projection, the latter one will reuse the column with UniqueID allocated in aggregation
10211022
// via aggMapper, so we don't need unnecessarily maintain the <aggDes, UniqueID> mapping in the FDSet like expr did, just treating
10221023
// it as normal column.
@@ -1051,7 +1052,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
10511052
fds.RegisterUniqueID(hashCode, scalarUniqueID)
10521053
groupByColsUniqueIDs.Insert(scalarUniqueID)
10531054
}
1054-
determinants := fd.NewFastIntSet()
1055+
determinants := intset.NewFastIntSet()
10551056
extractedColumns := expression.ExtractColumns(x)
10561057
extractedCorColumns := expression.ExtractCorColumns(x)
10571058
for _, one := range extractedColumns {
@@ -1066,7 +1067,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
10661067
if notnull || determinants.SubsetOf(fds.NotNullCols) {
10671068
notnullColsUniqueIDs.Insert(scalarUniqueID)
10681069
}
1069-
fds.AddStrictFunctionalDependency(determinants, fd.NewFastIntSet(scalarUniqueID))
1070+
fds.AddStrictFunctionalDependency(determinants, intset.NewFastIntSet(scalarUniqueID))
10701071
}
10711072
}
10721073

@@ -1078,7 +1079,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
10781079
//
10791080
// and since any_value will NOT be pushed down to agg schema, which means every firstRow aggDes in the agg logical operator
10801081
// is meaningless to build the FD with. Let's only store the non-firstRow FD down: {group by items} ~~> {real aggDes}
1081-
realAggFuncUniqueID := fd.NewFastIntSet()
1082+
realAggFuncUniqueID := intset.NewFastIntSet()
10821083
for i, aggDes := range la.AggFuncs {
10831084
if aggDes.Name != "firstrow" {
10841085
realAggFuncUniqueID.Insert(int(la.schema.Columns[i].UniqueID))
@@ -1095,7 +1096,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
10951096
// 0 unique id is only used for here.
10961097
groupByColsUniqueIDs.Insert(0)
10971098
for i, ok := realAggFuncUniqueID.Next(0); ok; i, ok = realAggFuncUniqueID.Next(i + 1) {
1098-
fds.AddStrictFunctionalDependency(groupByColsUniqueIDs, fd.NewFastIntSet(i))
1099+
fds.AddStrictFunctionalDependency(groupByColsUniqueIDs, intset.NewFastIntSet(i))
10991100
}
11001101
} else {
11011102
// eliminating input columns that are un-projected.
@@ -1107,7 +1108,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet {
11071108
// 1: it can always distinguish and group the all-null/part-null group column rows.
11081109
// 2: the rows with all/part null group column are unique row after group operation.
11091110
// 3: there won't be two same group key with different agg values, so strict FD secured.
1110-
fds.AddStrictFunctionalDependency(groupByColsUniqueIDs, fd.NewFastIntSet(i))
1111+
fds.AddStrictFunctionalDependency(groupByColsUniqueIDs, intset.NewFastIntSet(i))
11111112
}
11121113

11131114
// agg funcDes has been tag not null flag when building aggregation.
@@ -1211,7 +1212,7 @@ type LogicalSelection struct {
12111212
Conditions []expression.Expression
12121213
}
12131214

1214-
func extractNotNullFromConds(conditions []expression.Expression, p LogicalPlan) fd.FastIntSet {
1215+
func extractNotNullFromConds(conditions []expression.Expression, p LogicalPlan) intset.FastIntSet {
12151216
// extract the column NOT NULL rejection characteristic from selection condition.
12161217
// CNF considered only, DNF doesn't have its meanings (cause that condition's eval may don't take effect)
12171218
//
@@ -1224,7 +1225,7 @@ func extractNotNullFromConds(conditions []expression.Expression, p LogicalPlan)
12241225
// 2: `b` must be null since only `NULL is NULL` is evaluated as true.
12251226
//
12261227
// As a result, `a` will be extracted as not-null column to abound the FDSet.
1227-
notnullColsUniqueIDs := fd.NewFastIntSet()
1228+
notnullColsUniqueIDs := intset.NewFastIntSet()
12281229
for _, condition := range conditions {
12291230
var cols []*expression.Column
12301231
cols = expression.ExtractColumnsFromExpressions(cols, []expression.Expression{condition}, nil)
@@ -1237,13 +1238,13 @@ func extractNotNullFromConds(conditions []expression.Expression, p LogicalPlan)
12371238
return notnullColsUniqueIDs
12381239
}
12391240

1240-
func extractConstantCols(conditions []expression.Expression, sctx sessionctx.Context, fds *fd.FDSet) fd.FastIntSet {
1241+
func extractConstantCols(conditions []expression.Expression, sctx sessionctx.Context, fds *fd.FDSet) intset.FastIntSet {
12411242
// extract constant cols
12421243
// eg: where a=1 and b is null and (1+c)=5.
12431244
// TODO: Some columns can only be determined to be constant from multiple constraints (e.g. x <= 1 AND x >= 1)
12441245
var (
12451246
constObjs []expression.Expression
1246-
constUniqueIDs = fd.NewFastIntSet()
1247+
constUniqueIDs = intset.NewFastIntSet()
12471248
)
12481249
constObjs = expression.ExtractConstantEqColumnsOrScalar(sctx, constObjs, conditions)
12491250
for _, constObj := range constObjs {
@@ -1264,10 +1265,10 @@ func extractConstantCols(conditions []expression.Expression, sctx sessionctx.Con
12641265
return constUniqueIDs
12651266
}
12661267

1267-
func extractEquivalenceCols(conditions []expression.Expression, sctx sessionctx.Context, fds *fd.FDSet) [][]fd.FastIntSet {
1268+
func extractEquivalenceCols(conditions []expression.Expression, sctx sessionctx.Context, fds *fd.FDSet) [][]intset.FastIntSet {
12681269
var equivObjsPair [][]expression.Expression
12691270
equivObjsPair = expression.ExtractEquivalenceColumns(equivObjsPair, conditions)
1270-
equivUniqueIDs := make([][]fd.FastIntSet, 0, len(equivObjsPair))
1271+
equivUniqueIDs := make([][]intset.FastIntSet, 0, len(equivObjsPair))
12711272
for _, equivObjPair := range equivObjsPair {
12721273
// lhs of equivalence.
12731274
var (
@@ -1301,7 +1302,7 @@ func extractEquivalenceCols(conditions []expression.Expression, sctx sessionctx.
13011302
rhsUniqueID = scalarUniqueID
13021303
}
13031304
}
1304-
equivUniqueIDs = append(equivUniqueIDs, []fd.FastIntSet{fd.NewFastIntSet(lhsUniqueID), fd.NewFastIntSet(rhsUniqueID)})
1305+
equivUniqueIDs = append(equivUniqueIDs, []intset.FastIntSet{intset.NewFastIntSet(lhsUniqueID), intset.NewFastIntSet(rhsUniqueID)})
13051306
}
13061307
return equivUniqueIDs
13071308
}
@@ -1311,8 +1312,8 @@ func (p *LogicalSelection) ExtractFD() *fd.FDSet {
13111312
// basically extract the children's fdSet.
13121313
fds := p.baseLogicalPlan.ExtractFD()
13131314
// collect the output columns' unique ID.
1314-
outputColsUniqueIDs := fd.NewFastIntSet()
1315-
notnullColsUniqueIDs := fd.NewFastIntSet()
1315+
outputColsUniqueIDs := intset.NewFastIntSet()
1316+
notnullColsUniqueIDs := intset.NewFastIntSet()
13161317
// eg: select t2.a, count(t2.b) from t1 join t2 using (a) where t1.a = 1
13171318
// join's schema will miss t2.a while join.full schema has. since selection
13181319
// itself doesn't contain schema, extracting schema should tell them apart.

0 commit comments

Comments
 (0)