Skip to content

Commit

Permalink
address comments, fix the Exist check in index
Browse files Browse the repository at this point in the history
Signed-off-by: xiongjiwei <[email protected]>
  • Loading branch information
xiongjiwei committed Jan 3, 2023
1 parent 16304c1 commit 4e40892
Show file tree
Hide file tree
Showing 5 changed files with 34 additions and 41 deletions.
1 change: 1 addition & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,6 +433,7 @@ func buildIndexLookUpChecker(b *executorBuilder, p *plannercore.PhysicalIndexLoo

tps := make([]*types.FieldType, 0, fullColLen)
for _, col := range is.Columns {
// tps is used to decode the index, we should use the element type of the array if any.
tps = append(tps, col.FieldType.ArrayType())
}

Expand Down
20 changes: 2 additions & 18 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -1256,24 +1257,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta
col := w.idxTblCols[i]
idxVal := idxRow.GetDatum(i, w.idxColTps[i])
tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo)
var cmpRes int
if col.FieldType.IsArray() {
// If it is multi-valued index, we should check the JSON contains the indexed value.
bj := vals[i].GetMysqlJSON()
count := bj.GetElemCount()
for elemIdx := 0; elemIdx < count; elemIdx++ {
jsonDatum := types.NewJSONDatum(bj.ArrayGetElem(elemIdx))
cmpRes, err = jsonDatum.Compare(sctx, &idxVal, collate.GetBinaryCollator())
if err != nil {
return errors.Trace(err)
}
if cmpRes == 0 {
break
}
}
} else {
cmpRes, err = idxVal.Compare(sctx, &vals[i], collators[i])
}
cmpRes, err := tables.CompareIndexAndVal(sctx, vals[i], idxVal, collators[i], col.FieldType.IsArray())
if err != nil {
return ir().ReportAdminCheckInconsistentWithColInfo(ctx,
handle,
Expand Down
3 changes: 3 additions & 0 deletions parser/types/field_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,9 @@ func (ft *FieldType) IsArray() bool {

// ArrayType return the type of the array.
func (ft *FieldType) ArrayType() *FieldType {
if !ft.array {
return ft
}
clone := ft.Clone()
clone.SetArray(false)
return clone
Expand Down
3 changes: 1 addition & 2 deletions table/tables/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -464,7 +464,7 @@ func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedV
case KeyInTempIndexConflict:
return true, h1, kv.ErrKeyExists
case KeyInTempIndexIsItself:
return true, h, nil
continue
}
}

Expand All @@ -486,7 +486,6 @@ func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedV
if !handle.Equal(h) {
return true, handle, kv.ErrKeyExists
}
return true, handle, nil
}
}
return true, h, nil
Expand Down
48 changes: 27 additions & 21 deletions table/tables/mutation_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -347,27 +347,9 @@ func compareIndexData(
cols[indexInfo.Columns[i].Offset].ColumnInfo,
)

var comparison int
var err error
// If it is multi-valued index, we should check the JSON contains the indexed value.
if cols[indexInfo.Columns[i].Offset].ColumnInfo.FieldType.IsArray() && expectedDatum.Kind() == types.KindMysqlJSON {
bj := expectedDatum.GetMysqlJSON()
count := bj.GetElemCount()
for elemIdx := 0; elemIdx < count; elemIdx++ {
jsonDatum := types.NewJSONDatum(bj.ArrayGetElem(elemIdx))
comparison, err = jsonDatum.Compare(sc, &decodedMutationDatum, collate.GetBinaryCollator())
if err != nil {
return errors.Trace(err)
}
if comparison == 0 {
break
}
}
} else {
comparison, err = decodedMutationDatum.Compare(sc, &expectedDatum, collate.GetCollator(decodedMutationDatum.Collation()))
if err != nil {
return errors.Trace(err)
}
comparison, err := CompareIndexAndVal(sc, expectedDatum, decodedMutationDatum, collate.GetCollator(decodedMutationDatum.Collation()), cols[indexInfo.Columns[i].Offset].ColumnInfo.FieldType.IsArray())
if err != nil {
return errors.Trace(err)
}

if comparison != 0 {
Expand All @@ -382,6 +364,30 @@ func compareIndexData(
return nil
}

// CompareIndexAndVal compare index valued and row value.
func CompareIndexAndVal(sctx *stmtctx.StatementContext, rowVal types.Datum, idxVal types.Datum, collator collate.Collator, cmpMVIndex bool) (int, error) {
var cmpRes int
var err error
if cmpMVIndex {
// If it is multi-valued index, we should check the JSON contains the indexed value.
bj := rowVal.GetMysqlJSON()
count := bj.GetElemCount()
for elemIdx := 0; elemIdx < count; elemIdx++ {
jsonDatum := types.NewJSONDatum(bj.ArrayGetElem(elemIdx))
cmpRes, err = jsonDatum.Compare(sctx, &idxVal, collate.GetBinaryCollator())
if err != nil {
return 0, errors.Trace(err)
}
if cmpRes == 0 {
break
}
}
} else {
cmpRes, err = idxVal.Compare(sctx, &rowVal, collator)
}
return cmpRes, err
}

// getColumnMaps tries to get the columnMaps from transaction options. If there isn't one, it builds one and stores it.
// It saves redundant computations of the map.
func getColumnMaps(txn kv.Transaction, t *TableCommon) columnMaps {
Expand Down

0 comments on commit 4e40892

Please sign in to comment.