Skip to content

Commit

Permalink
*: new secondary index value format (#20220)
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored Mar 3, 2021
1 parent a4b4d7a commit b5a9076
Show file tree
Hide file tree
Showing 27 changed files with 609 additions and 135 deletions.
3 changes: 1 addition & 2 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1928,8 +1928,7 @@ func checkGlobalIndexRow(c *C, ctx sessionctx.Context, tblInfo *model.TableInfo,
c.Assert(err, IsNil)
value, err := txn.Get(context.Background(), key)
c.Assert(err, IsNil)
colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns),
tablecodec.HandleDefault, idxColInfos)
colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns), tablecodec.HandleDefault, idxColInfos)
c.Assert(err, IsNil)
c.Assert(colVals, HasLen, len(idxVals)+2)
for i, val := range idxVals {
Expand Down
6 changes: 6 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1438,6 +1438,9 @@ func buildTableInfo(
tbInfo.PKIsHandle = true
} else {
tbInfo.IsCommonHandle = noBinlog
if tbInfo.IsCommonHandle {
tbInfo.CommonHandleVersion = 1
}
if !noBinlog {
errMsg := "cannot build clustered index table because the binlog is ON"
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf(errMsg))
Expand All @@ -1449,6 +1452,9 @@ func buildTableInfo(
tbInfo.PKIsHandle = !alterPKConf
} else {
tbInfo.IsCommonHandle = !alterPKConf && ctx.GetSessionVars().EnableClusteredIndex && noBinlog
if tbInfo.IsCommonHandle {
tbInfo.CommonHandleVersion = 1
}
}
}
if tbInfo.PKIsHandle || tbInfo.IsCommonHandle {
Expand Down
10 changes: 7 additions & 3 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -821,6 +821,7 @@ type indexRecord struct {
handle kv.Handle
key []byte // It's used to lock a record. Record it to reduce the encoding time.
vals []types.Datum // It's the index values.
rsData []types.Datum // It's the restored data for handle.
skip bool // skip indicates that the index key is already exists, we should not add it.
}

Expand Down Expand Up @@ -922,7 +923,9 @@ func (w *baseIndexWorker) getIndexRecord(idxInfo *model.IndexInfo, handle kv.Han
}
idxVal[j] = idxColumnVal
}
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal}

rsData := tables.TryGetHandleRestoredDataWrapper(w.table, nil, w.rowMap)
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal, rsData: rsData}
return idxRecord, nil
}

Expand Down Expand Up @@ -1128,7 +1131,8 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i
} else if w.distinctCheckFlags[i] {
// The keys in w.batchCheckKeys also maybe duplicate,
// so we need to backfill the not found key into `batchVals` map.
val, err := w.index.GenIndexValue(stmtCtx, idxRecords[i].vals, w.distinctCheckFlags[i], false, idxRecords[i].handle)
needRsData := tables.NeedRestoredData(w.index.Meta().Columns, w.table.Meta().Columns)
val, err := tablecodec.GenIndexValuePortal(stmtCtx, w.table.Meta(), w.index.Meta(), needRsData, w.distinctCheckFlags[i], false, idxRecords[i].vals, idxRecords[i].handle, 0, idxRecords[i].rsData)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -1185,7 +1189,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC
}

// Create the index.
handle, err := w.index.Create(w.sessCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle)
handle, err := w.index.Create(w.sessCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle, idxRecord.rsData)
if err != nil {
if kv.ErrKeyExists.Equal(err) && idxRecord.handle.Equal(handle) {
// Index already exists, skip it.
Expand Down
2 changes: 1 addition & 1 deletion ddl/reorg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func (s *testDDLSuite) TestReorg(c *C) {
Job: job,
currElement: e,
}
mockTbl := tables.MockTableFromMeta(&model.TableInfo{IsCommonHandle: s.IsCommonHandle})
mockTbl := tables.MockTableFromMeta(&model.TableInfo{IsCommonHandle: s.IsCommonHandle, CommonHandleVersion: 1})
err = d.generalWorker().runReorgJob(m, rInfo, mockTbl.Meta(), d.lease, f)
c.Assert(err, NotNil)

Expand Down
6 changes: 4 additions & 2 deletions executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,7 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err
type recoverRows struct {
handle kv.Handle
idxVals []types.Datum
rsData []types.Datum
skip bool
}

Expand Down Expand Up @@ -377,7 +378,8 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists
}
idxVals := extractIdxVals(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes, idxValLen)
e.idxValsBufs[result.scanRowCount] = idxVals
e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, skip: false})
rsData := tables.TryGetHandleRestoredDataWrapper(e.table, plannercore.GetCommonHandleDatum(e.handleCols, row), nil)
e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, rsData: rsData, skip: false})
result.scanRowCount++
result.currentHandle = handle
}
Expand Down Expand Up @@ -463,7 +465,7 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa
return result, err
}

_, err = e.index.Create(e.ctx, txn.GetUnionStore(), row.idxVals, row.handle)
_, err = e.index.Create(e.ctx, txn.GetUnionStore(), row.idxVals, row.handle, row.rsData)
if err != nil {
return result, err
}
Expand Down
52 changes: 26 additions & 26 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -386,21 +386,21 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) {

txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(-100))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(-100), nil)
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(100))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(100), nil)
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(8), kv.IntHandle(100))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(8), kv.IntHandle(100), nil)
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(nil), kv.IntHandle(101))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(nil), kv.IntHandle(101), nil)
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(nil), kv.IntHandle(102))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(nil), kv.IntHandle(102), nil)
c.Assert(err, IsNil)
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(200))
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(200), nil)
c.Assert(err, IsNil)
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(-200))
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(6), kv.IntHandle(-200), nil)
c.Assert(err, IsNil)
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(8), kv.IntHandle(-200))
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(8), kv.IntHandle(-200), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -455,9 +455,9 @@ func (s *testSuite5) TestAdminCleanupIndexForPartitionTable(c *C) {

txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(handle))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(handle), nil)
c.Assert(err, IsNil)
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(handle))
_, err = indexOpr3.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(handle), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -537,11 +537,11 @@ func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) {

txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(7, 10), kv.IntHandle(-100))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(7, 10), kv.IntHandle(-100), nil)
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(4, 6), kv.IntHandle(100))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(4, 6), kv.IntHandle(100), nil)
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(-7, 4), kv.IntHandle(101))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(-7, 4), kv.IntHandle(101), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -590,9 +590,9 @@ func (s *testSuite5) TestAdminCleanupIndexMore(c *C) {
for i := 0; i < 2000; i++ {
c1 := int64(2*i + 7)
c2 := int64(2*i + 8)
_, err = indexOpr1.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(c1, c2), kv.IntHandle(c1))
_, err = indexOpr1.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(c1, c2), kv.IntHandle(c1), nil)
c.Assert(err, IsNil, Commentf(errors.ErrorStack(err)))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(c2), kv.IntHandle(c1))
_, err = indexOpr2.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(c2), kv.IntHandle(c1), nil)
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
Expand Down Expand Up @@ -669,11 +669,11 @@ func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
for _, di := range c2DanglingIdx {
_, err := indexOpr2.Create(s.ctx, txn.GetUnionStore(), di.idxVal, di.handle)
_, err := indexOpr2.Create(s.ctx, txn.GetUnionStore(), di.idxVal, di.handle, nil)
c.Assert(err, IsNil)
}
for _, di := range c3DanglingIdx {
_, err := indexOpr3.Create(s.ctx, txn.GetUnionStore(), di.idxVal, di.handle)
_, err := indexOpr3.Create(s.ctx, txn.GetUnionStore(), di.idxVal, di.handle, nil)
c.Assert(err, IsNil)
}
err = txn.Commit(context.Background())
Expand Down Expand Up @@ -742,7 +742,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) {
// Manual recover index.
txn, err = s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i), kv.IntHandle(i))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i), kv.IntHandle(i), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand All @@ -756,7 +756,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) {
indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i+8))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i+8), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand All @@ -779,7 +779,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) {
indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -842,7 +842,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) {
// Index c2 has one more values than table data: 0, and the handle 0 hasn't correlative record.
txn, err = s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand All @@ -858,9 +858,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) {
err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0))
c.Assert(err, IsNil)
// Make sure the index value "19" is smaller "21". Then we scan to "19" before "21".
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10), nil)
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(13), kv.IntHandle(2))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(13), kv.IntHandle(2), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand All @@ -886,7 +886,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) {
// Index c2 has one line of data is 19, the corresponding table data is 20.
txn, err = s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(12), kv.IntHandle(2))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(12), kv.IntHandle(2), nil)
c.Assert(err, IsNil)
err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10))
c.Assert(err, IsNil)
Expand All @@ -901,7 +901,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) {
c.Assert(err, IsNil)
err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10))
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -1058,7 +1058,7 @@ func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) {
idxOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo)
txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = idxOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(2), kv.IntHandle(100))
_, err = idxOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(2), kv.IntHandle(100), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down
3 changes: 1 addition & 2 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -915,9 +915,8 @@ func (e *IndexLookUpExecutor) getHandle(row chunk.Row, handleIdx []int,
// original value(the primary key) here.
// We use a trick to avoid encoding the "sortKey" again by changing the charset
// collation to `binary`.
// TODO: Add the restore value to the secondary index to remove this trick.
rtp := e.handleCols[i].RetType
if collate.NewCollationEnabled() && rtp.EvalType() == types.ETString &&
if collate.NewCollationEnabled() && e.table.Meta().CommonHandleVersion == 0 && rtp.EvalType() == types.ETString &&
!mysql.HasBinaryFlag(rtp.Flag) && tp == getHandleFromIndex {
rtp = rtp.Clone()
rtp.Collate = charset.CollationBin
Expand Down
2 changes: 1 addition & 1 deletion executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) {
for i := 0; i < 10; i++ {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = idxOp.Create(ctx, txn.GetUnionStore(), types.MakeDatums(i+10), kv.IntHandle(100+i))
_, err = idxOp.Create(ctx, txn.GetUnionStore(), types.MakeDatums(i+10), kv.IntHandle(100+i), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down
6 changes: 3 additions & 3 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -456,7 +456,7 @@ func (s *testSuite3) TestAdmin(c *C) {
tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test"))
c.Assert(err, IsNil)
c.Assert(tb.Indices(), HasLen, 1)
_, err = tb.Indices()[0].Create(mock.NewContext(), txn.GetUnionStore(), types.MakeDatums(int64(10)), kv.IntHandle(1))
_, err = tb.Indices()[0].Create(mock.NewContext(), txn.GetUnionStore(), types.MakeDatums(int64(10)), kv.IntHandle(1), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -3690,7 +3690,7 @@ func (s *testSuite) TestCheckIndex(c *C) {
// table data (handle, data): (1, 10), (2, 20), (4, 40)
txn, err = s.store.Begin()
c.Assert(err, IsNil)
_, err = idx.Create(mockCtx, txn.GetUnionStore(), types.MakeDatums(int64(30)), kv.IntHandle(3))
_, err = idx.Create(mockCtx, txn.GetUnionStore(), types.MakeDatums(int64(30)), kv.IntHandle(3), nil)
c.Assert(err, IsNil)
key := tablecodec.EncodeRowKey(tb.Meta().ID, kv.IntHandle(4).Encoded())
setColValue(c, txn, key, types.NewDatum(int64(40)))
Expand All @@ -3705,7 +3705,7 @@ func (s *testSuite) TestCheckIndex(c *C) {
// table data (handle, data): (1, 10), (2, 20), (4, 40)
txn, err = s.store.Begin()
c.Assert(err, IsNil)
_, err = idx.Create(mockCtx, txn.GetUnionStore(), types.MakeDatums(int64(40)), kv.IntHandle(4))
_, err = idx.Create(mockCtx, txn.GetUnionStore(), types.MakeDatums(int64(40)), kv.IntHandle(4), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down
7 changes: 4 additions & 3 deletions executor/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -368,9 +368,10 @@ func (s *testSplitIndex) TestSplitTable(c *C) {

func (s *testSplitIndex) TestClusterIndexSplitTable(c *C) {
tbInfo := &model.TableInfo{
Name: model.NewCIStr("t"),
ID: 1,
IsCommonHandle: true,
Name: model.NewCIStr("t"),
ID: 1,
IsCommonHandle: true,
CommonHandleVersion: 1,
Indices: []*model.IndexInfo{
{
ID: 1,
Expand Down
2 changes: 1 addition & 1 deletion executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2689,7 +2689,7 @@ func (s *testSuite7) TestReplaceLog(c *C) {

txn, err := s.store.Begin()
c.Assert(err, IsNil)
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1))
_, err = indexOpr.Create(s.ctx, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1), nil)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
Expand Down
Loading

0 comments on commit b5a9076

Please sign in to comment.