Skip to content

Commit

Permalink
table/tables: fix buildListColumnsPruner issue in the list partition (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala authored and wshwsh12 committed Mar 7, 2022
1 parent 2506c86 commit a8cc9ba
Showing 1 changed file with 37 additions and 11 deletions.
48 changes: 37 additions & 11 deletions table/tables/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,6 +253,14 @@ type ForListColumnPruning struct {
valueTp *types.FieldType
valueMap map[string]ListPartitionLocation
sorted *btree.BTree

// To deal with the location partition failure caused by inconsistent NewCollationEnabled values(see issue #32416).
// The following fields are used to delay building valueMap.
ctx sessionctx.Context
tblInfo *model.TableInfo
schema *expression.Schema
names types.NameSlice
colIdx int
}

// ListPartitionGroup indicate the group index of the column value in a partition.
Expand Down Expand Up @@ -649,7 +657,6 @@ func (lp *ForListPruning) buildListColumnsPruner(ctx sessionctx.Context, tblInfo
columns []*expression.Column, names types.NameSlice) error {
pi := tblInfo.GetPartitionInfo()
schema := expression.NewSchema(columns...)
p := parser.New()
colPrunes := make([]*ForListColumnPruning, 0, len(pi.Columns))
for colIdx := range pi.Columns {
colInfo := model.FindColumnInfo(tblInfo.Columns, pi.Columns[colIdx].L)
Expand All @@ -661,15 +668,17 @@ func (lp *ForListPruning) buildListColumnsPruner(ctx sessionctx.Context, tblInfo
return table.ErrUnknownColumn.GenWithStackByArgs(pi.Columns[colIdx].L)
}
colPrune := &ForListColumnPruning{
ctx: ctx,
tblInfo: tblInfo,
schema: schema,
names: names,
colIdx: colIdx,
ExprCol: columns[idx],
valueTp: &colInfo.FieldType,
valueMap: make(map[string]ListPartitionLocation),
sorted: btree.New(btreeDegree),
}
err := colPrune.buildPartitionValueMapAndSorted(ctx, tblInfo, colIdx, schema, names, p)
if err != nil {
return err
}

colPrunes = append(colPrunes, colPrune)
}
lp.ColPrunes = colPrunes
Expand Down Expand Up @@ -753,13 +762,13 @@ func (lp *ForListPruning) locateListColumnsPartitionByRow(ctx sessionctx.Context
// buildListPartitionValueMapAndSorted builds list columns partition value map for the specified column.
// it also builds list columns partition value btree for the specified column.
// colIdx is the specified column index in the list columns.
func (lp *ForListColumnPruning) buildPartitionValueMapAndSorted(ctx sessionctx.Context, tblInfo *model.TableInfo, colIdx int,
schema *expression.Schema, names types.NameSlice, p *parser.Parser) error {
pi := tblInfo.GetPartitionInfo()
sc := ctx.GetSessionVars().StmtCtx
func (lp *ForListColumnPruning) buildPartitionValueMapAndSorted() error {
p := parser.New()
pi := lp.tblInfo.GetPartitionInfo()
sc := lp.ctx.GetSessionVars().StmtCtx
for partitionIdx, def := range pi.Definitions {
for groupIdx, vs := range def.InValues {
keyBytes, err := lp.genConstExprKey(ctx, sc, vs[colIdx], schema, names, p)
keyBytes, err := lp.genConstExprKey(lp.ctx, sc, vs[lp.colIdx], lp.schema, lp.names, p)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -805,11 +814,20 @@ func (lp *ForListColumnPruning) genKey(sc *stmtctx.StatementContext, v types.Dat
if err != nil {
return nil, errors.Trace(err)
}
return codec.EncodeKey(sc, nil, v)
valByte, err := codec.EncodeKey(sc, nil, v)
return valByte, err
}

// LocatePartition locates partition by the column value
func (lp *ForListColumnPruning) LocatePartition(sc *stmtctx.StatementContext, v types.Datum) (ListPartitionLocation, error) {
// To deal with the location partition failure caused by inconsistent NewCollationEnabled values(see issue #32416).
if len(lp.valueMap) == 0 {
err := lp.buildPartitionValueMapAndSorted()
if err != nil {
return nil, err
}
}

key, err := lp.genKey(sc, v)
if err != nil {
return nil, errors.Trace(err)
Expand All @@ -823,6 +841,14 @@ func (lp *ForListColumnPruning) LocatePartition(sc *stmtctx.StatementContext, v

// LocateRanges locates partition ranges by the column range
func (lp *ForListColumnPruning) LocateRanges(sc *stmtctx.StatementContext, r *ranger.Range) ([]ListPartitionLocation, error) {
// To deal with the location partition failure caused by inconsistent NewCollationEnabled values(see issue #32416).
if len(lp.valueMap) == 0 {
err := lp.buildPartitionValueMapAndSorted()
if err != nil {
return nil, err
}
}

var err error
var lowKey, highKey []byte
lowVal := r.LowVal[0]
Expand Down

0 comments on commit a8cc9ba

Please sign in to comment.