Skip to content

Commit

Permalink
session, table: fix listColumnPartition data race (pingcap#33199)
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala authored Apr 27, 2022
1 parent 591c205 commit 1a19f95
Show file tree
Hide file tree
Showing 3 changed files with 53 additions and 24 deletions.
31 changes: 31 additions & 0 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,12 @@ import (
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/dbterror"
Expand Down Expand Up @@ -2007,3 +2009,32 @@ func oldPasswordUpgrade(pass string) (string, error) {
newpass := fmt.Sprintf("*%X", hash2)
return newpass, nil
}

// rebuildAllPartitionValueMapAndSorted rebuilds all value map and sorted info for list column partitions with InfoSchema.
func rebuildAllPartitionValueMapAndSorted(s *session) {
type partitionExpr interface {
PartitionExpr() (*tables.PartitionExpr, error)
}

p := parser.New()
is := s.GetInfoSchema().(infoschema.InfoSchema)
for _, dbInfo := range is.AllSchemas() {
for _, t := range is.SchemaTables(dbInfo.Name) {
pi := t.Meta().GetPartitionInfo()
if pi == nil || pi.Type != model.PartitionTypeList {
continue
}

pe, err := t.(partitionExpr).PartitionExpr()
if err != nil {
panic("partition table gets partition expression failed")
}
for _, cp := range pe.ColPrunes {
if err = cp.RebuildPartitionValueMapAndSorted(p); err != nil {
logutil.BgLogger().Warn("build list column partition value map and sorted failed")
break
}
}
}
}
}
2 changes: 2 additions & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2795,6 +2795,8 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) {
return nil, err
}
collate.SetNewCollationEnabledForTest(newCollationEnabled)
// To deal with the location partition failure caused by inconsistent NewCollationEnabled values(see issue #32416).
rebuildAllPartitionValueMapAndSorted(ses[0])

err = updateMemoryConfigAndSysVar(ses[0])
if err != nil {
Expand Down
44 changes: 20 additions & 24 deletions table/tables/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,6 @@ type ForListColumnPruning struct {
ExprCol *expression.Column
valueTp *types.FieldType
valueMap map[string]ListPartitionLocation
mu sync.RWMutex
sorted *btree.BTree

// To deal with the location partition failure caused by inconsistent NewCollationEnabled values(see issue #32416).
Expand Down Expand Up @@ -658,6 +657,7 @@ 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 @@ -679,7 +679,10 @@ func (lp *ForListPruning) buildListColumnsPruner(ctx sessionctx.Context, tblInfo
valueMap: make(map[string]ListPartitionLocation),
sorted: btree.New(btreeDegree),
}

err := colPrune.buildPartitionValueMapAndSorted(p)
if err != nil {
return err
}
colPrunes = append(colPrunes, colPrune)
}
lp.ColPrunes = colPrunes
Expand Down Expand Up @@ -760,22 +763,28 @@ func (lp *ForListPruning) locateListColumnsPartitionByRow(ctx sessionctx.Context
return location[0].PartIdx, nil
}

// buildListPartitionValueMapAndSorted builds list columns partition value map for the specified column.
// it also builds list columns partition value btree for the specified column.
// buildPartitionValueMapAndSorted 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() error {
lp.mu.RLock()
func (lp *ForListColumnPruning) buildPartitionValueMapAndSorted(p *parser.Parser) error {
l := len(lp.valueMap)
lp.mu.RUnlock()
if l != 0 {
return nil
}

p := parser.New()
return lp.buildListPartitionValueMapAndSorted(p)
}

// RebuildPartitionValueMapAndSorted rebuilds list columns partition value map for the specified column.
func (lp *ForListColumnPruning) RebuildPartitionValueMapAndSorted(p *parser.Parser) error {
lp.valueMap = make(map[string]ListPartitionLocation, len(lp.valueMap))
lp.sorted.Clear(false)
return lp.buildListPartitionValueMapAndSorted(p)
}

func (lp *ForListColumnPruning) buildListPartitionValueMapAndSorted(p *parser.Parser) error {
pi := lp.tblInfo.GetPartitionInfo()
sc := lp.ctx.GetSessionVars().StmtCtx
lp.mu.Lock()
defer lp.mu.Unlock()
for partitionIdx, def := range pi.Definitions {
for groupIdx, vs := range def.InValues {
keyBytes, err := lp.genConstExprKey(lp.ctx, sc, vs[lp.colIdx], lp.schema, lp.names, p)
Expand Down Expand Up @@ -830,19 +839,11 @@ func (lp *ForListColumnPruning) genKey(sc *stmtctx.StatementContext, v types.Dat

// 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).
err := lp.buildPartitionValueMapAndSorted()
if err != nil {
return nil, err
}

key, err := lp.genKey(sc, v)
if err != nil {
return nil, errors.Trace(err)
}
lp.mu.RLock()
location, ok := lp.valueMap[string(key)]
lp.mu.RUnlock()
if !ok {
return nil, nil
}
Expand All @@ -851,13 +852,8 @@ 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).
err := lp.buildPartitionValueMapAndSorted()
if err != nil {
return nil, err
}

var lowKey, highKey []byte
var err error
lowVal := r.LowVal[0]
if r.LowVal[0].Kind() == types.KindMinNotNull {
lowVal = types.GetMinValue(lp.ExprCol.GetType())
Expand Down

0 comments on commit 1a19f95

Please sign in to comment.