Skip to content

Commit 5922b74

Browse files
winoroscoocood
authored andcommittedNov 7, 2017
ranger: rerange file and rename some method. (pingcap#5028)
1 parent 54a1e0e commit 5922b74

File tree

6 files changed

+564
-542
lines changed

6 files changed

+564
-542
lines changed
 

‎plan/physical_plan_builder.go

+44-1
Original file line numberDiff line numberDiff line change
@@ -878,7 +878,7 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou
878878
if len(is.filterCondition) > 0 {
879879
var indexConds, tableConds []expression.Expression
880880
if copTask.tablePlan != nil {
881-
indexConds, tableConds = ranger.DetachIndexFilterConditions(is.filterCondition, is.Index.Columns, is.Table)
881+
indexConds, tableConds = splitIndexFilterConditions(is.filterCondition, is.Index.Columns, is.Table)
882882
} else {
883883
indexConds = is.filterCondition
884884
}
@@ -921,6 +921,49 @@ func matchIndicesProp(idxCols []*model.IndexColumn, propCols []*expression.Colum
921921
return true
922922
}
923923

924+
func splitIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn,
925+
table *model.TableInfo) (indexConds, tableConds []expression.Expression) {
926+
var pkName model.CIStr
927+
if table.PKIsHandle {
928+
for _, colInfo := range table.Columns {
929+
if mysql.HasPriKeyFlag(colInfo.Flag) {
930+
pkName = colInfo.Name
931+
break
932+
}
933+
}
934+
}
935+
var indexConditions, tableConditions []expression.Expression
936+
for _, cond := range conditions {
937+
if checkIndexCondition(cond, indexColumns, pkName) {
938+
indexConditions = append(indexConditions, cond)
939+
} else {
940+
tableConditions = append(tableConditions, cond)
941+
}
942+
}
943+
return indexConditions, tableConditions
944+
}
945+
946+
// checkIndexCondition will check whether all columns of condition is index columns or primary key column.
947+
func checkIndexCondition(condition expression.Expression, indexColumns []*model.IndexColumn, pkName model.CIStr) bool {
948+
cols := expression.ExtractColumns(condition)
949+
for _, col := range cols {
950+
if pkName.L == col.ColName.L {
951+
continue
952+
}
953+
isIndexColumn := false
954+
for _, indCol := range indexColumns {
955+
if col.ColName.L == indCol.Name.L && indCol.Length == types.UnspecifiedLength {
956+
isIndexColumn = true
957+
break
958+
}
959+
}
960+
if !isIndexColumn {
961+
return false
962+
}
963+
}
964+
return true
965+
}
966+
924967
func (p *DataSource) forceToTableScan() PhysicalPlan {
925968
ts := PhysicalTableScan{
926969
Table: p.tableInfo,

‎util/ranger/refiner.go ‎util/ranger/checker.go

+1-125
Original file line numberDiff line numberDiff line change
@@ -14,33 +14,12 @@
1414
package ranger
1515

1616
import (
17-
"math"
18-
19-
"github.com/juju/errors"
2017
"github.com/pingcap/tidb/ast"
2118
"github.com/pingcap/tidb/expression"
2219
"github.com/pingcap/tidb/model"
23-
"github.com/pingcap/tidb/mysql"
24-
"github.com/pingcap/tidb/sessionctx/variable"
2520
"github.com/pingcap/tidb/types"
2621
)
2722

28-
// fullRange is (-∞, +∞).
29-
var fullRange = []point{
30-
{start: true},
31-
{value: types.MaxValueDatum()},
32-
}
33-
34-
// FullIntRange is (-∞, +∞) for IntColumnRange.
35-
func FullIntRange() []types.IntColumnRange {
36-
return []types.IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}}
37-
}
38-
39-
// FullIndexRange is (-∞, +∞) for IndexRange.
40-
func FullIndexRange() []*types.IndexRange {
41-
return []*types.IndexRange{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}
42-
}
43-
4423
// getEQFunctionOffset judge if the expression is a eq function like A = 1 where a is an index.
4524
// If so, it will return the offset of A in index columns. e.g. for index(C,B,A), A's offset is 2.
4625
func getEQFunctionOffset(expr expression.Expression, cols []*model.IndexColumn) int {
@@ -68,110 +47,6 @@ func getEQFunctionOffset(expr expression.Expression, cols []*model.IndexColumn)
6847
return -1
6948
}
7049

71-
func removeAccessConditions(conditions, accessConds []expression.Expression) []expression.Expression {
72-
for i := len(conditions) - 1; i >= 0; i-- {
73-
for _, cond := range accessConds {
74-
if cond == conditions[i] {
75-
conditions = append(conditions[:i], conditions[i+1:]...)
76-
break
77-
}
78-
}
79-
}
80-
return conditions
81-
}
82-
83-
// checkIndexCondition will check whether all columns of condition is index columns or primary key column.
84-
func checkIndexCondition(condition expression.Expression, indexColumns []*model.IndexColumn, pKName model.CIStr) bool {
85-
cols := expression.ExtractColumns(condition)
86-
for _, col := range cols {
87-
if pKName.L == col.ColName.L {
88-
continue
89-
}
90-
isIndexColumn := false
91-
for _, indCol := range indexColumns {
92-
if col.ColName.L == indCol.Name.L && indCol.Length == types.UnspecifiedLength {
93-
isIndexColumn = true
94-
break
95-
}
96-
}
97-
if !isIndexColumn {
98-
return false
99-
}
100-
}
101-
return true
102-
}
103-
104-
// DetachIndexFilterConditions will detach the access conditions from other conditions.
105-
func DetachIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, table *model.TableInfo) ([]expression.Expression, []expression.Expression) {
106-
var pKName model.CIStr
107-
if table.PKIsHandle {
108-
for _, colInfo := range table.Columns {
109-
if mysql.HasPriKeyFlag(colInfo.Flag) {
110-
pKName = colInfo.Name
111-
break
112-
}
113-
}
114-
}
115-
var indexConditions, tableConditions []expression.Expression
116-
for _, cond := range conditions {
117-
if checkIndexCondition(cond, indexColumns, pKName) {
118-
indexConditions = append(indexConditions, cond)
119-
} else {
120-
tableConditions = append(tableConditions, cond)
121-
}
122-
}
123-
return indexConditions, tableConditions
124-
}
125-
126-
// DetachColumnConditions distinguishes between access conditions and filter conditions from conditions.
127-
func DetachColumnConditions(conditions []expression.Expression, colName model.CIStr) ([]expression.Expression, []expression.Expression) {
128-
if colName.L == "" {
129-
return nil, conditions
130-
}
131-
132-
var accessConditions, filterConditions []expression.Expression
133-
checker := conditionChecker{
134-
colName: colName,
135-
length: types.UnspecifiedLength,
136-
}
137-
for _, cond := range conditions {
138-
cond = expression.PushDownNot(cond, false, nil)
139-
if !checker.check(cond) {
140-
filterConditions = append(filterConditions, cond)
141-
continue
142-
}
143-
accessConditions = append(accessConditions, cond)
144-
// TODO: it will lead to repeated computation cost.
145-
if checker.shouldReserve {
146-
filterConditions = append(filterConditions, cond)
147-
checker.shouldReserve = false
148-
}
149-
}
150-
151-
return accessConditions, filterConditions
152-
}
153-
154-
// BuildTableRange will build range of pk for PhysicalTableScan
155-
func BuildTableRange(accessConditions []expression.Expression, sc *variable.StatementContext) ([]types.IntColumnRange, error) {
156-
if len(accessConditions) == 0 {
157-
return FullIntRange(), nil
158-
}
159-
160-
rb := builder{sc: sc}
161-
rangePoints := fullRange
162-
for _, cond := range accessConditions {
163-
rangePoints = rb.intersection(rangePoints, rb.build(cond))
164-
if rb.err != nil {
165-
return nil, errors.Trace(rb.err)
166-
}
167-
}
168-
ranges, err := points2TableRanges(sc, rangePoints)
169-
if err != nil {
170-
return nil, errors.Trace(rb.err)
171-
}
172-
return ranges, nil
173-
}
174-
17550
// conditionChecker checks if this condition can be pushed to index plan.
17651
type conditionChecker struct {
17752
idx *model.IndexInfo
@@ -284,6 +159,7 @@ func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool
284159
pattern, ok := scalar.GetArgs()[1].(*expression.Constant)
285160
if !ok {
286161
return false
162+
287163
}
288164
if pattern.Value.IsNull() {
289165
return false

‎util/ranger/new_refiner.go ‎util/ranger/detacher.go

+38-202
Original file line numberDiff line numberDiff line change
@@ -14,135 +14,13 @@
1414
package ranger
1515

1616
import (
17-
"github.com/juju/errors"
1817
"github.com/pingcap/tidb/ast"
1918
"github.com/pingcap/tidb/context"
2019
"github.com/pingcap/tidb/expression"
21-
"github.com/pingcap/tidb/sessionctx/variable"
20+
"github.com/pingcap/tidb/model"
2221
"github.com/pingcap/tidb/types"
2322
)
2423

25-
func buildIndexRange(sc *variable.StatementContext, cols []*expression.Column, lengths []int,
26-
accessCondition []expression.Expression) ([]*types.IndexRange, error) {
27-
rb := builder{sc: sc}
28-
var (
29-
ranges []*types.IndexRange
30-
eqAndInCount int
31-
err error
32-
)
33-
for eqAndInCount = 0; eqAndInCount < len(accessCondition) && eqAndInCount < len(cols); eqAndInCount++ {
34-
if sf, ok := accessCondition[eqAndInCount].(*expression.ScalarFunction); !ok || (sf.FuncName.L != ast.EQ && sf.FuncName.L != ast.In) {
35-
break
36-
}
37-
// Build ranges for equal or in access conditions.
38-
point := rb.build(accessCondition[eqAndInCount])
39-
if rb.err != nil {
40-
return nil, errors.Trace(rb.err)
41-
}
42-
if eqAndInCount == 0 {
43-
ranges, err = points2IndexRanges(sc, point, cols[eqAndInCount].RetType)
44-
} else {
45-
ranges, err = appendPoints2IndexRanges(sc, ranges, point, cols[eqAndInCount].RetType)
46-
}
47-
if err != nil {
48-
return nil, errors.Trace(err)
49-
}
50-
}
51-
rangePoints := fullRange
52-
// Build rangePoints for non-equal access conditions.
53-
for i := eqAndInCount; i < len(accessCondition); i++ {
54-
rangePoints = rb.intersection(rangePoints, rb.build(accessCondition[i]))
55-
if rb.err != nil {
56-
return nil, errors.Trace(err)
57-
}
58-
}
59-
if eqAndInCount == 0 {
60-
ranges, err = points2IndexRanges(sc, rangePoints, cols[0].RetType)
61-
} else if eqAndInCount < len(accessCondition) {
62-
ranges, err = appendPoints2IndexRanges(sc, ranges, rangePoints, cols[eqAndInCount].RetType)
63-
}
64-
65-
// Take prefix index into consideration.
66-
if hasPrefix(lengths) {
67-
fixPrefixColRange(ranges, lengths)
68-
}
69-
70-
if len(ranges) > 0 && len(ranges[0].LowVal) < len(cols) {
71-
for _, ran := range ranges {
72-
if ran.HighExclude || ran.LowExclude {
73-
if ran.HighExclude {
74-
ran.HighVal = append(ran.HighVal, types.NewDatum(nil))
75-
} else {
76-
ran.HighVal = append(ran.HighVal, types.MaxValueDatum())
77-
}
78-
if ran.LowExclude {
79-
ran.LowVal = append(ran.LowVal, types.MaxValueDatum())
80-
} else {
81-
ran.LowVal = append(ran.LowVal, types.NewDatum(nil))
82-
}
83-
}
84-
}
85-
}
86-
return ranges, nil
87-
}
88-
89-
func hasPrefix(lengths []int) bool {
90-
for _, l := range lengths {
91-
if l != types.UnspecifiedLength {
92-
return true
93-
}
94-
}
95-
return false
96-
}
97-
98-
func fixPrefixColRange(ranges []*types.IndexRange, lengths []int) {
99-
for _, ran := range ranges {
100-
for i := 0; i < len(ran.LowVal); i++ {
101-
fixRangeDatum(&ran.LowVal[i], lengths[i])
102-
}
103-
ran.LowExclude = false
104-
for i := 0; i < len(ran.HighVal); i++ {
105-
fixRangeDatum(&ran.HighVal[i], lengths[i])
106-
}
107-
ran.HighExclude = false
108-
}
109-
}
110-
111-
func fixRangeDatum(v *types.Datum, length int) {
112-
// If this column is prefix and the prefix length is smaller than the range, cut it.
113-
if length != types.UnspecifiedLength && length < len(v.GetBytes()) {
114-
v.SetBytes(v.GetBytes()[:length])
115-
}
116-
}
117-
118-
// getEQColOffset judge if the expression is a eq function that one side is constant and another is column.
119-
// If so, it will return the offset of this column in the slice.
120-
func getEQColOffset(expr expression.Expression, cols []*expression.Column) int {
121-
f, ok := expr.(*expression.ScalarFunction)
122-
if !ok || f.FuncName.L != ast.EQ {
123-
return -1
124-
}
125-
if c, ok := f.GetArgs()[0].(*expression.Column); ok {
126-
if _, ok := f.GetArgs()[1].(*expression.Constant); ok {
127-
for i, col := range cols {
128-
if col.Equal(c, nil) {
129-
return i
130-
}
131-
}
132-
}
133-
}
134-
if c, ok := f.GetArgs()[1].(*expression.Column); ok {
135-
if _, ok := f.GetArgs()[0].(*expression.Constant); ok {
136-
for i, col := range cols {
137-
if col.Equal(c, nil) {
138-
return i
139-
}
140-
}
141-
}
142-
}
143-
return -1
144-
}
145-
14624
// detachColumnCNFConditions detaches the condition for calculating range from the other conditions.
14725
// Please make sure that the top level is CNF form.
14826
func detachColumnCNFConditions(conditions []expression.Expression, checker *conditionChecker) ([]expression.Expression, []expression.Expression) {
@@ -266,38 +144,57 @@ func DetachIndexConditions(conditions []expression.Expression, cols []*expressio
266144
return accessConds, filterConds
267145
}
268146

269-
// buildColumnRange builds the range for sampling histogram to calculate the row count.
270-
func buildColumnRange(conds []expression.Expression, sc *variable.StatementContext, tp *types.FieldType) ([]*types.ColumnRange, error) {
271-
if len(conds) == 0 {
272-
return []*types.ColumnRange{{Low: types.Datum{}, High: types.MaxValueDatum()}}, nil
273-
}
274-
275-
rb := builder{sc: sc}
276-
rangePoints := fullRange
277-
for _, cond := range conds {
278-
rangePoints = rb.intersection(rangePoints, rb.build(cond))
279-
if rb.err != nil {
280-
return nil, errors.Trace(rb.err)
147+
func removeAccessConditions(conditions, accessConds []expression.Expression) []expression.Expression {
148+
for i := len(conditions) - 1; i >= 0; i-- {
149+
for _, cond := range accessConds {
150+
if cond == conditions[i] {
151+
conditions = append(conditions[:i], conditions[i+1:]...)
152+
break
153+
}
281154
}
282155
}
283-
ranges, err := points2ColumnRanges(sc, rangePoints, tp)
284-
if err != nil {
285-
return nil, errors.Trace(rb.err)
286-
}
287-
return ranges, nil
156+
return conditions
288157
}
289158

290159
// DetachCondsForSelectivity detaches the conditions used for range calculation from other useless conditions.
291160
func DetachCondsForSelectivity(conds []expression.Expression, rangeType int, cols []*expression.Column,
292161
lengths []int) (accessConditions, otherConditions []expression.Expression) {
293162
if rangeType == IntRangeType || rangeType == ColumnRangeType {
294-
return DetachColumnConditions(conds, cols[0].ColName)
163+
return detachColumnConditions(conds, cols[0].ColName)
295164
} else if rangeType == IndexRangeType {
296165
return DetachIndexConditions(conds, cols, lengths)
297166
}
298167
return nil, conds
299168
}
300169

170+
// detachColumnConditions distinguishes between access conditions and filter conditions from conditions.
171+
func detachColumnConditions(conditions []expression.Expression, colName model.CIStr) ([]expression.Expression, []expression.Expression) {
172+
if colName.L == "" {
173+
return nil, conditions
174+
}
175+
176+
var accessConditions, filterConditions []expression.Expression
177+
checker := conditionChecker{
178+
colName: colName,
179+
length: types.UnspecifiedLength,
180+
}
181+
for _, cond := range conditions {
182+
cond = expression.PushDownNot(cond, false, nil)
183+
if !checker.check(cond) {
184+
filterConditions = append(filterConditions, cond)
185+
continue
186+
}
187+
accessConditions = append(accessConditions, cond)
188+
// TODO: it will lead to repeated computation cost.
189+
if checker.shouldReserve {
190+
filterConditions = append(filterConditions, cond)
191+
checker.shouldReserve = false
192+
}
193+
}
194+
195+
return accessConditions, filterConditions
196+
}
197+
301198
// DetachCondsForTableRange detaches the conditions used for range calculation form other useless conditions for
302199
// calculating the table range.
303200
func DetachCondsForTableRange(ctx context.Context, conds []expression.Expression, col *expression.Column) (accessContditions, otherConditions []expression.Expression) {
@@ -310,64 +207,3 @@ func DetachCondsForTableRange(ctx context.Context, conds []expression.Expression
310207
}
311208
return detachColumnCNFConditions(conds, checker)
312209
}
313-
314-
// BuildRange is a method which can calculate IntColumnRange, ColumnRange, IndexRange.
315-
func BuildRange(sc *variable.StatementContext, conds []expression.Expression, rangeType int, cols []*expression.Column,
316-
lengths []int) (retRanges []types.Range, _ error) {
317-
if rangeType == IntRangeType {
318-
ranges, err := BuildTableRange(conds, sc)
319-
if err != nil {
320-
return nil, errors.Trace(err)
321-
}
322-
retRanges = make([]types.Range, 0, len(ranges))
323-
for _, ran := range ranges {
324-
retRanges = append(retRanges, ran)
325-
}
326-
} else if rangeType == ColumnRangeType {
327-
ranges, err := buildColumnRange(conds, sc, cols[0].RetType)
328-
if err != nil {
329-
return nil, errors.Trace(err)
330-
}
331-
retRanges = make([]types.Range, 0, len(ranges))
332-
for _, ran := range ranges {
333-
retRanges = append(retRanges, ran)
334-
}
335-
} else if rangeType == IndexRangeType {
336-
ranges, err := buildIndexRange(sc, cols, lengths, conds)
337-
if err != nil {
338-
return nil, errors.Trace(err)
339-
}
340-
retRanges = make([]types.Range, 0, len(ranges))
341-
for _, ran := range ranges {
342-
retRanges = append(retRanges, ran)
343-
}
344-
}
345-
return
346-
}
347-
348-
// Ranges2IntRanges changes []types.Range to []types.IntColumnRange
349-
func Ranges2IntRanges(ranges []types.Range) []types.IntColumnRange {
350-
retRanges := make([]types.IntColumnRange, 0, len(ranges))
351-
for _, ran := range ranges {
352-
retRanges = append(retRanges, ran.Convert2IntRange())
353-
}
354-
return retRanges
355-
}
356-
357-
// Ranges2ColumnRanges changes []types.Range to []*types.ColumnRange
358-
func Ranges2ColumnRanges(ranges []types.Range) []*types.ColumnRange {
359-
retRanges := make([]*types.ColumnRange, 0, len(ranges))
360-
for _, ran := range ranges {
361-
retRanges = append(retRanges, ran.Convert2ColumnRange())
362-
}
363-
return retRanges
364-
}
365-
366-
// Ranges2IndexRanges changes []types.Range to []*types.IndexRange
367-
func Ranges2IndexRanges(ranges []types.Range) []*types.IndexRange {
368-
retRanges := make([]*types.IndexRange, 0, len(ranges))
369-
for _, ran := range ranges {
370-
retRanges = append(retRanges, ran.Convert2IndexRange())
371-
}
372-
return retRanges
373-
}

‎util/ranger/range.go ‎util/ranger/points.go

+16-213
Original file line numberDiff line numberDiff line change
@@ -114,6 +114,22 @@ func (r *pointSorter) Swap(i, j int) {
114114
r.points[i], r.points[j] = r.points[j], r.points[i]
115115
}
116116

117+
// fullRange is (-∞, +∞).
118+
var fullRange = []point{
119+
{start: true},
120+
{value: types.MaxValueDatum()},
121+
}
122+
123+
// FullIntRange is (-∞, +∞) for IntColumnRange.
124+
func FullIntRange() []types.IntColumnRange {
125+
return []types.IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}}
126+
}
127+
128+
// FullIndexRange is (-∞, +∞) for IndexRange.
129+
func FullIndexRange() []*types.IndexRange {
130+
return []*types.IndexRange{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}
131+
}
132+
117133
// builder is the range builder struct.
118134
type builder struct {
119135
err error
@@ -487,216 +503,3 @@ func (r *builder) merge(a, b []point, union bool) []point {
487503
}
488504
return merged
489505
}
490-
491-
// points2IndexRanges build index ranges from range points.
492-
// Only the first column in the index is built, extra column ranges will be appended by
493-
// appendPoints2IndexRanges.
494-
func points2IndexRanges(sc *variable.StatementContext, rangePoints []point, tp *types.FieldType) ([]*types.IndexRange, error) {
495-
indexRanges := make([]*types.IndexRange, 0, len(rangePoints)/2)
496-
for i := 0; i < len(rangePoints); i += 2 {
497-
startPoint, err := convertPoint(sc, rangePoints[i], tp)
498-
if err != nil {
499-
return nil, errors.Trace(err)
500-
}
501-
endPoint, err := convertPoint(sc, rangePoints[i+1], tp)
502-
if err != nil {
503-
return nil, errors.Trace(err)
504-
}
505-
less, err := rangePointLess(sc, startPoint, endPoint)
506-
if err != nil {
507-
return nil, errors.Trace(err)
508-
}
509-
if !less {
510-
continue
511-
}
512-
ir := &types.IndexRange{
513-
LowVal: []types.Datum{startPoint.value},
514-
LowExclude: startPoint.excl,
515-
HighVal: []types.Datum{endPoint.value},
516-
HighExclude: endPoint.excl,
517-
}
518-
indexRanges = append(indexRanges, ir)
519-
}
520-
return indexRanges, nil
521-
}
522-
523-
func convertPoint(sc *variable.StatementContext, point point, tp *types.FieldType) (point, error) {
524-
switch point.value.Kind() {
525-
case types.KindMaxValue, types.KindMinNotNull:
526-
return point, nil
527-
}
528-
casted, err := point.value.ConvertTo(sc, tp)
529-
if err != nil {
530-
return point, errors.Trace(err)
531-
}
532-
valCmpCasted, err := point.value.CompareDatum(sc, &casted)
533-
if err != nil {
534-
return point, errors.Trace(err)
535-
}
536-
point.value = casted
537-
if valCmpCasted == 0 {
538-
return point, nil
539-
}
540-
if point.start {
541-
if point.excl {
542-
if valCmpCasted < 0 {
543-
// e.g. "a > 1.9" convert to "a >= 2".
544-
point.excl = false
545-
}
546-
} else {
547-
if valCmpCasted > 0 {
548-
// e.g. "a >= 1.1 convert to "a > 1"
549-
point.excl = true
550-
}
551-
}
552-
} else {
553-
if point.excl {
554-
if valCmpCasted > 0 {
555-
// e.g. "a < 1.1" convert to "a <= 1"
556-
point.excl = false
557-
}
558-
} else {
559-
if valCmpCasted < 0 {
560-
// e.g. "a <= 1.9" convert to "a < 2"
561-
point.excl = true
562-
}
563-
}
564-
}
565-
return point, nil
566-
}
567-
568-
// appendPoints2IndexRanges appends additional column ranges for multi-column index.
569-
// The additional column ranges can only be appended to point ranges.
570-
// for example we have an index (a, b), if the condition is (a > 1 and b = 2)
571-
// then we can not build a conjunctive ranges for this index.
572-
func appendPoints2IndexRanges(sc *variable.StatementContext, origin []*types.IndexRange, rangePoints []point,
573-
ft *types.FieldType) ([]*types.IndexRange, error) {
574-
var newIndexRanges []*types.IndexRange
575-
for i := 0; i < len(origin); i++ {
576-
oRange := origin[i]
577-
if !oRange.IsPoint(sc) {
578-
newIndexRanges = append(newIndexRanges, oRange)
579-
} else {
580-
newRanges, err := appendPoints2IndexRange(sc, oRange, rangePoints, ft)
581-
if err != nil {
582-
return nil, errors.Trace(err)
583-
}
584-
newIndexRanges = append(newIndexRanges, newRanges...)
585-
}
586-
}
587-
return newIndexRanges, nil
588-
}
589-
590-
func appendPoints2IndexRange(sc *variable.StatementContext, origin *types.IndexRange, rangePoints []point,
591-
ft *types.FieldType) ([]*types.IndexRange, error) {
592-
newRanges := make([]*types.IndexRange, 0, len(rangePoints)/2)
593-
for i := 0; i < len(rangePoints); i += 2 {
594-
startPoint, err := convertPoint(sc, rangePoints[i], ft)
595-
if err != nil {
596-
return nil, errors.Trace(err)
597-
}
598-
endPoint, err := convertPoint(sc, rangePoints[i+1], ft)
599-
if err != nil {
600-
return nil, errors.Trace(err)
601-
}
602-
less, err := rangePointLess(sc, startPoint, endPoint)
603-
if err != nil {
604-
return nil, errors.Trace(err)
605-
}
606-
if !less {
607-
continue
608-
}
609-
610-
lowVal := make([]types.Datum, len(origin.LowVal)+1)
611-
copy(lowVal, origin.LowVal)
612-
lowVal[len(origin.LowVal)] = startPoint.value
613-
614-
highVal := make([]types.Datum, len(origin.HighVal)+1)
615-
copy(highVal, origin.HighVal)
616-
highVal[len(origin.HighVal)] = endPoint.value
617-
618-
ir := &types.IndexRange{
619-
LowVal: lowVal,
620-
LowExclude: startPoint.excl,
621-
HighVal: highVal,
622-
HighExclude: endPoint.excl,
623-
}
624-
newRanges = append(newRanges, ir)
625-
}
626-
return newRanges, nil
627-
}
628-
629-
// points2TableRanges will construct the range slice with the given range points
630-
func points2TableRanges(sc *variable.StatementContext, rangePoints []point) ([]types.IntColumnRange, error) {
631-
tableRanges := make([]types.IntColumnRange, 0, len(rangePoints)/2)
632-
for i := 0; i < len(rangePoints); i += 2 {
633-
startPoint := rangePoints[i]
634-
if startPoint.value.IsNull() || startPoint.value.Kind() == types.KindMinNotNull {
635-
startPoint.value.SetInt64(math.MinInt64)
636-
}
637-
startInt, err := startPoint.value.ToInt64(sc)
638-
if err != nil {
639-
return nil, errors.Trace(err)
640-
}
641-
startDatum := types.NewDatum(startInt)
642-
cmp, err := startDatum.CompareDatum(sc, &startPoint.value)
643-
if err != nil {
644-
return nil, errors.Trace(err)
645-
}
646-
if cmp < 0 || (cmp == 0 && startPoint.excl) {
647-
startInt++
648-
}
649-
endPoint := rangePoints[i+1]
650-
if endPoint.value.IsNull() {
651-
endPoint.value.SetInt64(math.MinInt64)
652-
} else if endPoint.value.Kind() == types.KindMaxValue {
653-
endPoint.value.SetInt64(math.MaxInt64)
654-
}
655-
endInt, err := endPoint.value.ToInt64(sc)
656-
if err != nil {
657-
return nil, errors.Trace(err)
658-
}
659-
endDatum := types.NewDatum(endInt)
660-
cmp, err = endDatum.CompareDatum(sc, &endPoint.value)
661-
if err != nil {
662-
return nil, errors.Trace(err)
663-
}
664-
if cmp > 0 || (cmp == 0 && endPoint.excl) {
665-
endInt--
666-
}
667-
if startInt > endInt {
668-
continue
669-
}
670-
tableRanges = append(tableRanges, types.IntColumnRange{LowVal: startInt, HighVal: endInt})
671-
}
672-
return tableRanges, nil
673-
}
674-
675-
func points2ColumnRanges(sc *variable.StatementContext, points []point, tp *types.FieldType) ([]*types.ColumnRange, error) {
676-
columnRanges := make([]*types.ColumnRange, 0, len(points)/2)
677-
for i := 0; i < len(points); i += 2 {
678-
startPoint, err := convertPoint(sc, points[i], tp)
679-
if err != nil {
680-
return nil, errors.Trace(err)
681-
}
682-
endPoint, err := convertPoint(sc, points[i+1], tp)
683-
if err != nil {
684-
return nil, errors.Trace(err)
685-
}
686-
less, err := rangePointLess(sc, startPoint, endPoint)
687-
if err != nil {
688-
return nil, errors.Trace(err)
689-
}
690-
if !less {
691-
continue
692-
}
693-
cr := &types.ColumnRange{
694-
Low: startPoint.value,
695-
LowExcl: startPoint.excl,
696-
High: endPoint.value,
697-
HighExcl: endPoint.excl,
698-
}
699-
columnRanges = append(columnRanges, cr)
700-
}
701-
return columnRanges, nil
702-
}

‎util/ranger/ranger.go

+464
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,464 @@
1+
// Copyright 2017 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// See the License for the specific language governing permissions and
12+
// limitations under the License.
13+
14+
package ranger
15+
16+
import (
17+
"math"
18+
19+
"github.com/juju/errors"
20+
"github.com/pingcap/tidb/ast"
21+
"github.com/pingcap/tidb/expression"
22+
"github.com/pingcap/tidb/sessionctx/variable"
23+
"github.com/pingcap/tidb/types"
24+
)
25+
26+
// points2IndexRanges build index ranges from range points.
27+
// Only the first column in the index is built, extra column ranges will be appended by
28+
// appendPoints2IndexRanges.
29+
func points2IndexRanges(sc *variable.StatementContext, rangePoints []point, tp *types.FieldType) ([]*types.IndexRange, error) {
30+
indexRanges := make([]*types.IndexRange, 0, len(rangePoints)/2)
31+
for i := 0; i < len(rangePoints); i += 2 {
32+
startPoint, err := convertPoint(sc, rangePoints[i], tp)
33+
if err != nil {
34+
return nil, errors.Trace(err)
35+
}
36+
endPoint, err := convertPoint(sc, rangePoints[i+1], tp)
37+
if err != nil {
38+
return nil, errors.Trace(err)
39+
}
40+
less, err := rangePointLess(sc, startPoint, endPoint)
41+
if err != nil {
42+
return nil, errors.Trace(err)
43+
}
44+
if !less {
45+
continue
46+
}
47+
ir := &types.IndexRange{
48+
LowVal: []types.Datum{startPoint.value},
49+
LowExclude: startPoint.excl,
50+
HighVal: []types.Datum{endPoint.value},
51+
HighExclude: endPoint.excl,
52+
}
53+
indexRanges = append(indexRanges, ir)
54+
}
55+
return indexRanges, nil
56+
}
57+
58+
func convertPoint(sc *variable.StatementContext, point point, tp *types.FieldType) (point, error) {
59+
switch point.value.Kind() {
60+
case types.KindMaxValue, types.KindMinNotNull:
61+
return point, nil
62+
}
63+
casted, err := point.value.ConvertTo(sc, tp)
64+
if err != nil {
65+
return point, errors.Trace(err)
66+
}
67+
valCmpCasted, err := point.value.CompareDatum(sc, &casted)
68+
if err != nil {
69+
return point, errors.Trace(err)
70+
}
71+
point.value = casted
72+
if valCmpCasted == 0 {
73+
return point, nil
74+
}
75+
if point.start {
76+
if point.excl {
77+
if valCmpCasted < 0 {
78+
// e.g. "a > 1.9" convert to "a >= 2".
79+
point.excl = false
80+
}
81+
} else {
82+
if valCmpCasted > 0 {
83+
// e.g. "a >= 1.1 convert to "a > 1"
84+
point.excl = true
85+
}
86+
}
87+
} else {
88+
if point.excl {
89+
if valCmpCasted > 0 {
90+
// e.g. "a < 1.1" convert to "a <= 1"
91+
point.excl = false
92+
}
93+
} else {
94+
if valCmpCasted < 0 {
95+
// e.g. "a <= 1.9" convert to "a < 2"
96+
point.excl = true
97+
}
98+
}
99+
}
100+
return point, nil
101+
}
102+
103+
// appendPoints2IndexRanges appends additional column ranges for multi-column index.
104+
// The additional column ranges can only be appended to point ranges.
105+
// for example we have an index (a, b), if the condition is (a > 1 and b = 2)
106+
// then we can not build a conjunctive ranges for this index.
107+
func appendPoints2IndexRanges(sc *variable.StatementContext, origin []*types.IndexRange, rangePoints []point,
108+
ft *types.FieldType) ([]*types.IndexRange, error) {
109+
var newIndexRanges []*types.IndexRange
110+
for i := 0; i < len(origin); i++ {
111+
oRange := origin[i]
112+
if !oRange.IsPoint(sc) {
113+
newIndexRanges = append(newIndexRanges, oRange)
114+
} else {
115+
newRanges, err := appendPoints2IndexRange(sc, oRange, rangePoints, ft)
116+
if err != nil {
117+
return nil, errors.Trace(err)
118+
}
119+
newIndexRanges = append(newIndexRanges, newRanges...)
120+
}
121+
}
122+
return newIndexRanges, nil
123+
}
124+
125+
func appendPoints2IndexRange(sc *variable.StatementContext, origin *types.IndexRange, rangePoints []point,
126+
ft *types.FieldType) ([]*types.IndexRange, error) {
127+
newRanges := make([]*types.IndexRange, 0, len(rangePoints)/2)
128+
for i := 0; i < len(rangePoints); i += 2 {
129+
startPoint, err := convertPoint(sc, rangePoints[i], ft)
130+
if err != nil {
131+
return nil, errors.Trace(err)
132+
}
133+
endPoint, err := convertPoint(sc, rangePoints[i+1], ft)
134+
if err != nil {
135+
return nil, errors.Trace(err)
136+
}
137+
less, err := rangePointLess(sc, startPoint, endPoint)
138+
if err != nil {
139+
return nil, errors.Trace(err)
140+
}
141+
if !less {
142+
continue
143+
}
144+
145+
lowVal := make([]types.Datum, len(origin.LowVal)+1)
146+
copy(lowVal, origin.LowVal)
147+
lowVal[len(origin.LowVal)] = startPoint.value
148+
149+
highVal := make([]types.Datum, len(origin.HighVal)+1)
150+
copy(highVal, origin.HighVal)
151+
highVal[len(origin.HighVal)] = endPoint.value
152+
153+
ir := &types.IndexRange{
154+
LowVal: lowVal,
155+
LowExclude: startPoint.excl,
156+
HighVal: highVal,
157+
HighExclude: endPoint.excl,
158+
}
159+
newRanges = append(newRanges, ir)
160+
}
161+
return newRanges, nil
162+
}
163+
164+
// points2TableRanges will construct the range slice with the given range points
165+
func points2TableRanges(sc *variable.StatementContext, rangePoints []point) ([]types.IntColumnRange, error) {
166+
tableRanges := make([]types.IntColumnRange, 0, len(rangePoints)/2)
167+
for i := 0; i < len(rangePoints); i += 2 {
168+
startPoint := rangePoints[i]
169+
if startPoint.value.IsNull() || startPoint.value.Kind() == types.KindMinNotNull {
170+
startPoint.value.SetInt64(math.MinInt64)
171+
}
172+
startInt, err := startPoint.value.ToInt64(sc)
173+
if err != nil {
174+
return nil, errors.Trace(err)
175+
}
176+
startDatum := types.NewDatum(startInt)
177+
cmp, err := startDatum.CompareDatum(sc, &startPoint.value)
178+
if err != nil {
179+
return nil, errors.Trace(err)
180+
}
181+
if cmp < 0 || (cmp == 0 && startPoint.excl) {
182+
startInt++
183+
}
184+
endPoint := rangePoints[i+1]
185+
if endPoint.value.IsNull() {
186+
endPoint.value.SetInt64(math.MinInt64)
187+
} else if endPoint.value.Kind() == types.KindMaxValue {
188+
endPoint.value.SetInt64(math.MaxInt64)
189+
}
190+
endInt, err := endPoint.value.ToInt64(sc)
191+
if err != nil {
192+
return nil, errors.Trace(err)
193+
}
194+
endDatum := types.NewDatum(endInt)
195+
cmp, err = endDatum.CompareDatum(sc, &endPoint.value)
196+
if err != nil {
197+
return nil, errors.Trace(err)
198+
}
199+
if cmp > 0 || (cmp == 0 && endPoint.excl) {
200+
endInt--
201+
}
202+
if startInt > endInt {
203+
continue
204+
}
205+
tableRanges = append(tableRanges, types.IntColumnRange{LowVal: startInt, HighVal: endInt})
206+
}
207+
return tableRanges, nil
208+
}
209+
210+
func points2ColumnRanges(sc *variable.StatementContext, points []point, tp *types.FieldType) ([]*types.ColumnRange, error) {
211+
columnRanges := make([]*types.ColumnRange, 0, len(points)/2)
212+
for i := 0; i < len(points); i += 2 {
213+
startPoint, err := convertPoint(sc, points[i], tp)
214+
if err != nil {
215+
return nil, errors.Trace(err)
216+
}
217+
endPoint, err := convertPoint(sc, points[i+1], tp)
218+
if err != nil {
219+
return nil, errors.Trace(err)
220+
}
221+
less, err := rangePointLess(sc, startPoint, endPoint)
222+
if err != nil {
223+
return nil, errors.Trace(err)
224+
}
225+
if !less {
226+
continue
227+
}
228+
cr := &types.ColumnRange{
229+
Low: startPoint.value,
230+
LowExcl: startPoint.excl,
231+
High: endPoint.value,
232+
HighExcl: endPoint.excl,
233+
}
234+
columnRanges = append(columnRanges, cr)
235+
}
236+
return columnRanges, nil
237+
}
238+
239+
// buildTableRange will build range of pk for PhysicalTableScan
240+
func buildTableRange(accessConditions []expression.Expression, sc *variable.StatementContext) ([]types.IntColumnRange, error) {
241+
if len(accessConditions) == 0 {
242+
return FullIntRange(), nil
243+
}
244+
245+
rb := builder{sc: sc}
246+
rangePoints := fullRange
247+
for _, cond := range accessConditions {
248+
rangePoints = rb.intersection(rangePoints, rb.build(cond))
249+
if rb.err != nil {
250+
return nil, errors.Trace(rb.err)
251+
}
252+
}
253+
ranges, err := points2TableRanges(sc, rangePoints)
254+
if err != nil {
255+
return nil, errors.Trace(err)
256+
}
257+
return ranges, nil
258+
}
259+
260+
// buildColumnRange builds the range for sampling histogram to calculate the row count.
261+
func buildColumnRange(conds []expression.Expression, sc *variable.StatementContext, tp *types.FieldType) ([]*types.ColumnRange, error) {
262+
if len(conds) == 0 {
263+
return []*types.ColumnRange{{Low: types.Datum{}, High: types.MaxValueDatum()}}, nil
264+
}
265+
266+
rb := builder{sc: sc}
267+
rangePoints := fullRange
268+
for _, cond := range conds {
269+
rangePoints = rb.intersection(rangePoints, rb.build(cond))
270+
if rb.err != nil {
271+
return nil, errors.Trace(rb.err)
272+
}
273+
}
274+
ranges, err := points2ColumnRanges(sc, rangePoints, tp)
275+
if err != nil {
276+
return nil, errors.Trace(err)
277+
}
278+
return ranges, nil
279+
}
280+
281+
func buildIndexRange(sc *variable.StatementContext, cols []*expression.Column, lengths []int,
282+
accessCondition []expression.Expression) ([]*types.IndexRange, error) {
283+
rb := builder{sc: sc}
284+
var (
285+
ranges []*types.IndexRange
286+
eqAndInCount int
287+
err error
288+
)
289+
for eqAndInCount = 0; eqAndInCount < len(accessCondition) && eqAndInCount < len(cols); eqAndInCount++ {
290+
if sf, ok := accessCondition[eqAndInCount].(*expression.ScalarFunction); !ok || (sf.FuncName.L != ast.EQ && sf.FuncName.L != ast.In) {
291+
break
292+
}
293+
// Build ranges for equal or in access conditions.
294+
point := rb.build(accessCondition[eqAndInCount])
295+
if rb.err != nil {
296+
return nil, errors.Trace(rb.err)
297+
}
298+
if eqAndInCount == 0 {
299+
ranges, err = points2IndexRanges(sc, point, cols[eqAndInCount].RetType)
300+
} else {
301+
ranges, err = appendPoints2IndexRanges(sc, ranges, point, cols[eqAndInCount].RetType)
302+
}
303+
if err != nil {
304+
return nil, errors.Trace(err)
305+
}
306+
}
307+
rangePoints := fullRange
308+
// Build rangePoints for non-equal access conditions.
309+
for i := eqAndInCount; i < len(accessCondition); i++ {
310+
rangePoints = rb.intersection(rangePoints, rb.build(accessCondition[i]))
311+
if rb.err != nil {
312+
return nil, errors.Trace(rb.err)
313+
}
314+
}
315+
if eqAndInCount == 0 {
316+
ranges, err = points2IndexRanges(sc, rangePoints, cols[0].RetType)
317+
} else if eqAndInCount < len(accessCondition) {
318+
ranges, err = appendPoints2IndexRanges(sc, ranges, rangePoints, cols[eqAndInCount].RetType)
319+
}
320+
if err != nil {
321+
return nil, errors.Trace(err)
322+
}
323+
324+
// Take prefix index into consideration.
325+
if hasPrefix(lengths) {
326+
fixPrefixColRange(ranges, lengths)
327+
}
328+
329+
if len(ranges) > 0 && len(ranges[0].LowVal) < len(cols) {
330+
for _, ran := range ranges {
331+
if ran.HighExclude || ran.LowExclude {
332+
if ran.HighExclude {
333+
ran.HighVal = append(ran.HighVal, types.NewDatum(nil))
334+
} else {
335+
ran.HighVal = append(ran.HighVal, types.MaxValueDatum())
336+
}
337+
if ran.LowExclude {
338+
ran.LowVal = append(ran.LowVal, types.MaxValueDatum())
339+
} else {
340+
ran.LowVal = append(ran.LowVal, types.NewDatum(nil))
341+
}
342+
}
343+
}
344+
}
345+
return ranges, nil
346+
}
347+
348+
func hasPrefix(lengths []int) bool {
349+
for _, l := range lengths {
350+
if l != types.UnspecifiedLength {
351+
return true
352+
}
353+
}
354+
return false
355+
}
356+
357+
func fixPrefixColRange(ranges []*types.IndexRange, lengths []int) {
358+
for _, ran := range ranges {
359+
for i := 0; i < len(ran.LowVal); i++ {
360+
fixRangeDatum(&ran.LowVal[i], lengths[i])
361+
}
362+
ran.LowExclude = false
363+
for i := 0; i < len(ran.HighVal); i++ {
364+
fixRangeDatum(&ran.HighVal[i], lengths[i])
365+
}
366+
ran.HighExclude = false
367+
}
368+
}
369+
370+
func fixRangeDatum(v *types.Datum, length int) {
371+
// If this column is prefix and the prefix length is smaller than the range, cut it.
372+
if length != types.UnspecifiedLength && length < len(v.GetBytes()) {
373+
v.SetBytes(v.GetBytes()[:length])
374+
}
375+
}
376+
377+
// getEQColOffset judge if the expression is a eq function that one side is constant and another is column.
378+
// If so, it will return the offset of this column in the slice.
379+
func getEQColOffset(expr expression.Expression, cols []*expression.Column) int {
380+
f, ok := expr.(*expression.ScalarFunction)
381+
if !ok || f.FuncName.L != ast.EQ {
382+
return -1
383+
}
384+
if c, ok := f.GetArgs()[0].(*expression.Column); ok {
385+
if _, ok := f.GetArgs()[1].(*expression.Constant); ok {
386+
for i, col := range cols {
387+
if col.Equal(c, nil) {
388+
return i
389+
}
390+
}
391+
}
392+
}
393+
if c, ok := f.GetArgs()[1].(*expression.Column); ok {
394+
if _, ok := f.GetArgs()[0].(*expression.Constant); ok {
395+
for i, col := range cols {
396+
if col.Equal(c, nil) {
397+
return i
398+
}
399+
}
400+
}
401+
}
402+
return -1
403+
}
404+
405+
// BuildRange is a method which can calculate IntColumnRange, ColumnRange, IndexRange.
406+
func BuildRange(sc *variable.StatementContext, conds []expression.Expression, rangeType int, cols []*expression.Column,
407+
lengths []int) (retRanges []types.Range, _ error) {
408+
if rangeType == IntRangeType {
409+
ranges, err := buildTableRange(conds, sc)
410+
if err != nil {
411+
return nil, errors.Trace(err)
412+
}
413+
retRanges = make([]types.Range, 0, len(ranges))
414+
for _, ran := range ranges {
415+
retRanges = append(retRanges, ran)
416+
}
417+
} else if rangeType == ColumnRangeType {
418+
ranges, err := buildColumnRange(conds, sc, cols[0].RetType)
419+
if err != nil {
420+
return nil, errors.Trace(err)
421+
}
422+
retRanges = make([]types.Range, 0, len(ranges))
423+
for _, ran := range ranges {
424+
retRanges = append(retRanges, ran)
425+
}
426+
} else if rangeType == IndexRangeType {
427+
ranges, err := buildIndexRange(sc, cols, lengths, conds)
428+
if err != nil {
429+
return nil, errors.Trace(err)
430+
}
431+
retRanges = make([]types.Range, 0, len(ranges))
432+
for _, ran := range ranges {
433+
retRanges = append(retRanges, ran)
434+
}
435+
}
436+
return
437+
}
438+
439+
// Ranges2IntRanges changes []types.Range to []types.IntColumnRange
440+
func Ranges2IntRanges(ranges []types.Range) []types.IntColumnRange {
441+
retRanges := make([]types.IntColumnRange, 0, len(ranges))
442+
for _, ran := range ranges {
443+
retRanges = append(retRanges, ran.Convert2IntRange())
444+
}
445+
return retRanges
446+
}
447+
448+
// Ranges2ColumnRanges changes []types.Range to []*types.ColumnRange
449+
func Ranges2ColumnRanges(ranges []types.Range) []*types.ColumnRange {
450+
retRanges := make([]*types.ColumnRange, 0, len(ranges))
451+
for _, ran := range ranges {
452+
retRanges = append(retRanges, ran.Convert2ColumnRange())
453+
}
454+
return retRanges
455+
}
456+
457+
// Ranges2IndexRanges changes []types.Range to []*types.IndexRange
458+
func Ranges2IndexRanges(ranges []types.Range) []*types.IndexRange {
459+
retRanges := make([]*types.IndexRange, 0, len(ranges))
460+
for _, ran := range ranges {
461+
retRanges = append(retRanges, ran.Convert2IndexRange())
462+
}
463+
return retRanges
464+
}

‎util/ranger/range_test.go ‎util/ranger/ranger_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -682,7 +682,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) {
682682
col := expression.ColInfo2Col(sel.Schema().Columns, ds.TableInfo().Columns[tt.colPos])
683683
c.Assert(col, NotNil)
684684
var filter []expression.Expression
685-
conds, filter = ranger.DetachColumnConditions(conds, col.ColName)
685+
conds, filter = ranger.DetachCondsForSelectivity(conds, ranger.ColumnRangeType, []*expression.Column{col}, nil)
686686
c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr))
687687
c.Assert(fmt.Sprintf("%s", filter), Equals, tt.filterConds, Commentf("wrong filter conditions for expr: %s", tt.exprStr))
688688
result, err := ranger.BuildRange(new(variable.StatementContext), conds, ranger.ColumnRangeType, []*expression.Column{col}, nil)

0 commit comments

Comments
 (0)
Please sign in to comment.