diff --git a/ddl/db_test.go b/ddl/db_test.go index 87da7667c9893..1405bd99bff56 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -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 { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c68516b33e4fc..7c708595d8b2a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -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)) @@ -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 { diff --git a/ddl/index.go b/ddl/index.go index 971e86ebbe043..2a521542f0909 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -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. } @@ -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 } @@ -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) } @@ -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. diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index cc3ce01690348..18dd9a975fceb 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -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) diff --git a/executor/admin.go b/executor/admin.go index 93f47168f91e1..4769fb760d744 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -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 } @@ -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 } @@ -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 } diff --git a/executor/admin_test.go b/executor/admin_test.go index 996dbc303c19d..427fa7ab54678 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -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) @@ -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) @@ -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) @@ -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()) @@ -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()) @@ -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) @@ -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) @@ -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) @@ -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) @@ -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) @@ -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) @@ -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) @@ -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) diff --git a/executor/distsql.go b/executor/distsql.go index fb40162326dca..f57968b572f6a 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -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 diff --git a/executor/distsql_test.go b/executor/distsql_test.go index c0874bc9a70cb..e5f079963b285 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -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) diff --git a/executor/executor_test.go b/executor/executor_test.go index 2e78603f125db..e17c287f8632d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -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) @@ -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))) @@ -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) diff --git a/executor/split_test.go b/executor/split_test.go index 5476e57f9c899..b457a08a560da 100644 --- a/executor/split_test.go +++ b/executor/split_test.go @@ -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, diff --git a/executor/write_test.go b/executor/write_test.go index b1be828284415..435385e64d3a2 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -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) diff --git a/expression/integration_test.go b/expression/integration_test.go index 19af7711f452c..2139611018d22 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -7885,6 +7885,76 @@ func (s *testIntegrationSuite) TestIssue17476(c *C) { tk.MustQuery(`SELECT * FROM (table_int_float_varchar AS tmp3) WHERE (col_varchar_6 AND NULL) IS NULL AND col_int_6=0;`).Check(testkit.Rows("13 0 -0.1 ")) } +func (s *testIntegrationSerialSuite) TestClusteredIndexAndNewCollationIndexEncodeDecodeV5(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_enable_clustered_index=1;") + tk.MustExec("create table t(a int, b char(10) collate utf8mb4_bin, c char(10) collate utf8mb4_general_ci," + + "d varchar(10) collate utf8mb4_bin, e varchar(10) collate utf8mb4_general_ci, f char(10) collate utf8mb4_unicode_ci, g varchar(10) collate utf8mb4_unicode_ci, " + + "primary key(a, b, c, d, e, f, g), key a(a), unique key ua(a), key b(b), unique key ub(b), key c(c), unique key uc(c)," + + "key d(d), unique key ud(d),key e(e), unique key ue(e), key f(f), key g(g), unique key uf(f), unique key ug(g))") + + tk.MustExec("insert into t values (1, '啊 ', '啊 ', '啊 ', '啊 ', '啊 ', '啊 ')") + // Single Read. + tk.MustQuery("select * from t ").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + + tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(f)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(uf)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(g)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ug)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + + tk.MustExec("alter table t add column h varchar(10) collate utf8mb4_general_ci default '🐸'") + tk.MustExec("alter table t add column i varchar(10) collate utf8mb4_general_ci default '🐸'") + tk.MustExec("alter table t add index h(h)") + tk.MustExec("alter table t add unique index uh(h)") + + tk.MustQuery("select * from t use index(h)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(uh)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + + // Double read. + tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustExec("admin check table t") + tk.MustExec("admin recover index t a") + tk.MustExec("alter table t add column n char(10) COLLATE utf8mb4_unicode_ci") + tk.MustExec("alter table t add index n(n)") + tk.MustExec("update t set n = '吧';") + tk.MustQuery("select * from t").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) + tk.MustQuery("select * from t use index(n)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) + tk.MustExec("admin check table t") + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a varchar(255) COLLATE utf8_general_ci primary key clustered, b int) partition by range columns(a) " + + "(partition p0 values less than ('0'), partition p1 values less than MAXVALUE);") + tk.MustExec("alter table t add index b(b);") + tk.MustExec("insert into t values ('0', 1);") + tk.MustQuery("select * from t use index(b);").Check(testkit.Rows("0 1")) + tk.MustQuery("select * from t use index();").Check(testkit.Rows("0 1")) + tk.MustExec("admin check table t") +} + func (s *testIntegrationSuite) TestIssue11645(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) diff --git a/go.sum b/go.sum index 9719772821148..ede41b0600c60 100644 --- a/go.sum +++ b/go.sum @@ -39,8 +39,10 @@ github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmx github.com/VividCortex/mysqlerr v0.0.0-20200629151747-c28746d985dd/go.mod h1:f3HiCrHjHBdcm6E83vGaXh1KomZMA2P6aeo3hKx/wg0= github.com/Xeoncross/go-aesctr-with-hmac v0.0.0-20200623134604-12b17a7ff502/go.mod h1:pmnBM9bxWSiHvC/gSWunUIyDvGn33EkP2CUjxFKtTTM= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4 h1:Hs82Z41s6SdL1CELW+XaDYmOH4hkBN4/N9og/AsOv7E= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= github.com/appleboy/gin-jwt/v2 v2.6.3/go.mod h1:MfPYA4ogzvOcVkRwAxT7quHOtQmVKDpTwxyUrC2DNw0= @@ -807,6 +809,7 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.1 h1:zvIju4sqAGvwKspUQOhwnpcqSbzi7/H6QomNNjTL4sk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/infoschema/tables.go b/infoschema/tables.go index 57444d9f2f73e..3aecf2f7d2454 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -277,6 +277,7 @@ func buildTableMeta(tableName string, cs []columnInfo) *model.TableInfo { tblInfo.PKIsHandle = true default: tblInfo.IsCommonHandle = true + tblInfo.CommonHandleVersion = 1 index := &model.IndexInfo{ Name: model.NewCIStr("primary"), State: model.StatePublic, diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 2e0455b014afc..5fae1d70ec686 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -962,11 +962,10 @@ func (ds *DataSource) isCoveringIndex(columns, indexColumns []*expression.Column if !coveredByPlainIndex && !coveredByClusteredIndex { return false } - isClusteredNewCollationIdx := collate.NewCollationEnabled() && col.GetType().EvalType() == types.ETString && !mysql.HasBinaryFlag(col.GetType().Flag) - if !coveredByPlainIndex && coveredByClusteredIndex && isClusteredNewCollationIdx { + if !coveredByPlainIndex && coveredByClusteredIndex && isClusteredNewCollationIdx && ds.table.Meta().CommonHandleVersion == 0 { return false } } diff --git a/planner/core/handle_cols.go b/planner/core/handle_cols.go index b66aa964d5e51..57ce33a49b4d5 100644 --- a/planner/core/handle_cols.go +++ b/planner/core/handle_cols.go @@ -256,3 +256,18 @@ func (ib *IntHandleCols) GetFieldsTypes() []*types.FieldType { func NewIntHandleCols(col *expression.Column) HandleCols { return &IntHandleCols{col: col} } + +// GetCommonHandleDatum gets the original data for the common handle. +func GetCommonHandleDatum(cols HandleCols, row chunk.Row) []types.Datum { + if cols.IsInt() { + return nil + } + cb := cols.(*CommonHandleCols) + + datumBuf := make([]types.Datum, 0, 4) + for _, col := range cb.columns { + datumBuf = append(datumBuf, row.GetDatum(col.Index, col.RetType)) + } + + return datumBuf +} diff --git a/session/schema_amender.go b/session/schema_amender.go index 64b883dc15967..5088bfb81e6e5 100644 --- a/session/schema_amender.go +++ b/session/schema_amender.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/expression" @@ -406,6 +407,19 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc return nil } +func getCommonHandleDatum(tbl table.Table, row chunk.Row) []types.Datum { + if !tbl.Meta().IsCommonHandle { + return nil + } + datumBuf := make([]types.Datum, 0, 4) + for _, col := range tbl.Cols() { + if mysql.HasPriKeyFlag(col.Flag) { + datumBuf = append(datumBuf, row.GetDatum(col.Offset, &col.FieldType)) + } + } + return datumBuf +} + func (a *amendOperationAddIndexInfo) genIndexKeyValue(ctx context.Context, sctx sessionctx.Context, kvMap map[string][]byte, key []byte, kvHandle kv.Handle, keyOnly bool) ([]byte, []byte, error) { chk := a.chk @@ -428,6 +442,8 @@ func (a *amendOperationAddIndexInfo) genIndexKeyValue(ctx context.Context, sctx idxVals = append(idxVals, chk.GetRow(0).GetDatum(oldCol.Offset, &oldCol.FieldType)) } + rsData := tables.TryGetHandleRestoredDataWrapper(a.tblInfoAtCommit, getCommonHandleDatum(a.tblInfoAtCommit, chk.GetRow(0)), nil) + // Generate index key buf. newIdxKey, distinct, err := tablecodec.GenIndexKey(sctx.GetSessionVars().StmtCtx, a.tblInfoAtCommit.Meta(), a.indexInfoAtCommit.Meta(), a.tblInfoAtCommit.Meta().ID, idxVals, kvHandle, nil) @@ -440,9 +456,8 @@ func (a *amendOperationAddIndexInfo) genIndexKeyValue(ctx context.Context, sctx } // Generate index value buf. - containsNonBinaryString := tables.ContainsNonBinaryString(a.indexInfoAtCommit.Meta().Columns, a.tblInfoAtCommit.Meta().Columns) - newIdxVal, err := tablecodec.GenIndexValue(sctx.GetSessionVars().StmtCtx, a.tblInfoAtCommit.Meta(), - a.indexInfoAtCommit.Meta(), containsNonBinaryString, distinct, false, idxVals, kvHandle) + needRsData := tables.NeedRestoredData(a.indexInfoAtCommit.Meta().Columns, a.tblInfoAtCommit.Meta().Columns) + newIdxVal, err := tablecodec.GenIndexValuePortal(sctx.GetSessionVars().StmtCtx, a.tblInfoAtCommit.Meta(), a.indexInfoAtCommit.Meta(), needRsData, distinct, false, idxVals, kvHandle, 0, rsData) if err != nil { logutil.Logger(ctx).Warn("amend generate index values failed", zap.Error(err)) return nil, nil, errors.Trace(err) diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index 23a53292b4619..91ff6a3e29cc4 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -196,8 +196,7 @@ func prepareTestData(se *session, mutations *tikv.PlainMutations, oldTblInfo tab idxKey, _, err := tablecodec.GenIndexKey(se.sessionVars.StmtCtx, newTblInfo.Meta(), info.indexInfoAtCommit.Meta(), newTblInfo.Meta().ID, indexDatums, kvHandle, nil) c.Assert(err, IsNil) - idxVal, err = tablecodec.GenIndexValue(se.sessionVars.StmtCtx, newTblInfo.Meta(), info.indexInfoAtCommit.Meta(), - false, info.indexInfoAtCommit.Meta().Unique, false, indexDatums, kvHandle) + idxVal, err = tablecodec.GenIndexValuePortal(se.sessionVars.StmtCtx, newTblInfo.Meta(), info.indexInfoAtCommit.Meta(), false, info.indexInfoAtCommit.Meta().Unique, false, indexDatums, kvHandle, 0, nil) c.Assert(err, IsNil) return idxKey, idxVal } diff --git a/table/index.go b/table/index.go index e0554f16ff8d3..af823e80b998b 100644 --- a/table/index.go +++ b/table/index.go @@ -64,7 +64,7 @@ type Index interface { // Meta returns IndexInfo. Meta() *model.IndexInfo // Create supports insert into statement. - Create(ctx sessionctx.Context, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle, opts ...CreateIdxOptFunc) (kv.Handle, error) + Create(ctx sessionctx.Context, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error // Drop supports drop table, drop index statements. @@ -73,8 +73,6 @@ type Index interface { Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) - // GenIndexValue generates an index value. - GenIndexValue(sc *stmtctx.StatementContext, indexedValues []types.Datum, distinct bool, untouched bool, h kv.Handle) (val []byte, err error) // Seek supports where clause. Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter IndexIterator, hit bool, err error) // SeekFirst supports aggregate min and ascend order by. diff --git a/table/tables/index.go b/table/tables/index.go index df4267d0ade50..95169fdad1bab 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -20,7 +20,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -80,27 +79,26 @@ func (c *indexIter) Next() (val []types.Datum, h kv.Handle, err error) { // index is the data structure for index data in the KV store. type index struct { - idxInfo *model.IndexInfo - tblInfo *model.TableInfo - prefix kv.Key - containNonBinaryString bool - phyTblID int64 + idxInfo *model.IndexInfo + tblInfo *model.TableInfo + prefix kv.Key + needRestoredData bool + phyTblID int64 } -// ContainsNonBinaryString checks whether the index columns contains non binary string column, the input -// colInfos should be column info correspond to the table contains the index. -func ContainsNonBinaryString(idxCols []*model.IndexColumn, colInfos []*model.ColumnInfo) bool { +// NeedRestoredData checks whether the index columns needs restored data. +func NeedRestoredData(idxCols []*model.IndexColumn, colInfos []*model.ColumnInfo) bool { for _, idxCol := range idxCols { col := colInfos[idxCol.Offset] - if col.EvalType() == types.ETString && !mysql.HasBinaryFlag(col.Flag) { + if types.NeedRestoredData(&col.FieldType) { return true } } return false } -func (c *index) checkContainNonBinaryString() bool { - return ContainsNonBinaryString(c.idxInfo.Columns, c.tblInfo.Columns) +func (c *index) checkNeedRestoredData() bool { + return NeedRestoredData(c.idxInfo.Columns, c.tblInfo.Columns) } // NewIndex builds a new Index object. @@ -120,7 +118,7 @@ func NewIndex(physicalID int64, tblInfo *model.TableInfo, indexInfo *model.Index prefix: prefix, phyTblID: physicalID, } - index.containNonBinaryString = index.checkContainNonBinaryString() + index.needRestoredData = index.checkNeedRestoredData() return index } @@ -139,48 +137,10 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types. return tablecodec.GenIndexKey(sc, c.tblInfo, c.idxInfo, idxTblID, indexedValues, h, buf) } -func (c *index) GenIndexValue(sc *stmtctx.StatementContext, indexedValues []types.Datum, distinct bool, untouched bool, h kv.Handle) (val []byte, err error) { - return tablecodec.GenIndexValueNew(sc, c.tblInfo, c.idxInfo, c.containNonBinaryString, distinct, untouched, indexedValues, h, c.phyTblID) -} - // Create creates a new entry in the kvIndex data. // If the index is unique and there is an existing entry with the same key, // Create will return the existing entry's handle as the first return value, ErrKeyExists as the second return value. -// Value layout: -// +--New Encoding (with restore data, or common handle, or index is global) -// | -// | Layout: TailLen | Options | Padding | [IntHandle] | [UntouchedFlag] -// | Length: 1 | len(options) | len(padding) | 8 | 1 -// | -// | TailLen: len(padding) + len(IntHandle) + len(UntouchedFlag) -// | Options: Encode some value for new features, such as common handle, new collations or global index. -// | See below for more information. -// | Padding: Ensure length of value always >= 10. (or >= 11 if UntouchedFlag exists.) -// | IntHandle: Only exists when table use int handles and index is unique. -// | UntouchedFlag: Only exists when index is untouched. -// | -// | Layout of Options: -// | -// | Segment: Common Handle | Global Index | New Collation -// | Layout: CHandle Flag | CHandle Len | CHandle | PidFlag | PartitionID | restoreData -// | Length: 1 | 2 | len(CHandle) | 1 | 8 | len(restoreData) -// | -// | Common Handle Segment: Exists when unique index used common handles. -// | Global Index Segment: Exists when index is global. -// | New Collation Segment: Exists when new collation is used and index contains non-binary string. -// | -// +--Old Encoding (without restore data, integer handle, local) -// -// Layout: [Handle] | [UntouchedFlag] -// Length: 8 | 1 -// -// Handle: Only exists in unique index. -// UntouchedFlag: Only exists when index is untouched. -// -// If neither Handle nor UntouchedFlag exists, value will be one single byte '0' (i.e. []byte{'0'}). -// Length of value <= 9, use to distinguish from the new encoding. -// -func (c *index) Create(sctx sessionctx.Context, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle, opts ...table.CreateIdxOptFunc) (kv.Handle, error) { +func (c *index) Create(sctx sessionctx.Context, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...table.CreateIdxOptFunc) (kv.Handle, error) { if c.Meta().Unique { us.CacheTableInfo(c.phyTblID, c.tblInfo) } @@ -213,8 +173,7 @@ func (c *index) Create(sctx sessionctx.Context, us kv.UnionStore, indexedValues // save the key buffer to reuse. writeBufs.IndexKeyBuf = key - idxVal, err := tablecodec.GenIndexValueNew(sctx.GetSessionVars().StmtCtx, c.tblInfo, c.idxInfo, - c.containNonBinaryString, distinct, opt.Untouched, indexedValues, h, c.phyTblID) + idxVal, err := tablecodec.GenIndexValuePortal(sctx.GetSessionVars().StmtCtx, c.tblInfo, c.idxInfo, c.needRestoredData, distinct, opt.Untouched, indexedValues, h, c.phyTblID, handleRestoreData) if err != nil { return nil, err } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 2a0264fa91475..6004533c8a8ba 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -90,7 +90,7 @@ func (s *testIndexSuite) TestIndex(c *C) { values := types.MakeDatums(1, 2) mockCtx := mock.NewContext() - _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1), nil) c.Assert(err, IsNil) it, err := index.SeekFirst(txn) @@ -122,7 +122,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue, Commentf("err %v", err)) it.Close() - _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(0)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(0), nil) c.Assert(err, IsNil) _, err = index.SeekFirst(txn) @@ -177,10 +177,10 @@ func (s *testIndexSuite) TestIndex(c *C) { txn, err = s.s.Begin() c.Assert(err, IsNil) - _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1), nil) c.Assert(err, IsNil) - _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(2)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(2), nil) c.Assert(err, NotNil) it, err = index.SeekFirst(txn) @@ -215,7 +215,7 @@ func (s *testIndexSuite) TestIndex(c *C) { // Test the function of Next when the value of unique key is nil. values2 := types.MakeDatums(nil, nil) - _, err = index.Create(mockCtx, txn.GetUnionStore(), values2, kv.IntHandle(2)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values2, kv.IntHandle(2), nil) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) c.Assert(err, IsNil) @@ -257,7 +257,7 @@ func (s *testIndexSuite) TestCombineIndexSeek(c *C) { mockCtx := mock.NewContext() values := types.MakeDatums("abc", "def") - _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1)) + _, err = index.Create(mockCtx, txn.GetUnionStore(), values, kv.IntHandle(1), nil) c.Assert(err, IsNil) index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) @@ -298,12 +298,11 @@ func (s *testIndexSuite) TestSingleColumnCommonHandle(c *C) { for _, idx := range []table.Index{idxUnique, idxNonUnique} { key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) c.Assert(err, IsNil) - _, err = idx.Create(mockCtx, txn.GetUnionStore(), idxColVals, commonHandle) + _, err = idx.Create(mockCtx, txn.GetUnionStore(), idxColVals, commonHandle, nil) c.Assert(err, IsNil) val, err := txn.Get(context.Background(), key) c.Assert(err, IsNil) - colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, - createRowcodecColInfo(tblInfo, idx.Meta())) + colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, createRowcodecColInfo(tblInfo, idx.Meta())) c.Assert(err, IsNil) c.Assert(colVals, HasLen, 2) _, d, err := codec.DecodeOne(colVals[0]) @@ -319,8 +318,7 @@ func (s *testIndexSuite) TestSingleColumnCommonHandle(c *C) { unTouchedVal := append([]byte{1}, val[1:]...) unTouchedVal = append(unTouchedVal, kv.UnCommitIndexKVFlag) - _, err = tablecodec.DecodeIndexKV(key, unTouchedVal, 1, tablecodec.HandleDefault, - createRowcodecColInfo(tblInfo, idx.Meta())) + _, err = tablecodec.DecodeIndexKV(key, unTouchedVal, 1, tablecodec.HandleDefault, createRowcodecColInfo(tblInfo, idx.Meta())) c.Assert(err, IsNil) } } @@ -338,6 +336,16 @@ func (s *testIndexSuite) TestMultiColumnCommonHandle(c *C) { idxNonUnique = idx } } + var a, b *model.ColumnInfo + for _, col := range tblInfo.Columns { + if col.Name.String() == "a" { + a = col + } else if col.Name.String() == "b" { + b = col + } + } + c.Assert(a, NotNil) + c.Assert(b, NotNil) txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -354,12 +362,22 @@ func (s *testIndexSuite) TestMultiColumnCommonHandle(c *C) { for _, idx := range []table.Index{idxUnique, idxNonUnique} { key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) c.Assert(err, IsNil) - _, err = idx.Create(mockCtx, txn.GetUnionStore(), idxColVals, commonHandle) + _, err = idx.Create(mockCtx, txn.GetUnionStore(), idxColVals, commonHandle, nil) c.Assert(err, IsNil) val, err := txn.Get(context.Background(), key) c.Assert(err, IsNil) - colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, - createRowcodecColInfo(tblInfo, idx.Meta())) + colInfo := createRowcodecColInfo(tblInfo, idx.Meta()) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: a.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(a), + }) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: b.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(b), + }) + colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, colInfo) c.Assert(err, IsNil) c.Assert(colVals, HasLen, 3) _, d, err := codec.DecodeOne(colVals[0]) diff --git a/table/tables/tables.go b/table/tables/tables.go index 85339cdbd2197..fc924560a4390 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/generatedexpr" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stringutil" @@ -473,7 +474,7 @@ func (t *TableCommon) rebuildIndices(ctx sessionctx.Context, txn kv.Transaction, if err != nil { return err } - if err := t.buildIndexForRow(ctx, h, newVs, idx, txn, untouched, opts...); err != nil { + if err := t.buildIndexForRow(ctx, h, newVs, newData, idx, txn, untouched, opts...); err != nil { return err } } @@ -846,7 +847,8 @@ func (t *TableCommon) addIndices(sctx sessionctx.Context, recordID kv.Handle, r idxMeta := v.Meta() dupErr = kv.ErrKeyExists.FastGenByArgs(entryKey, idxMeta.Name.String()) } - if dupHandle, err := v.Create(sctx, txn.GetUnionStore(), indexVals, recordID, opts...); err != nil { + rsData := TryGetHandleRestoredDataWrapper(t, r, nil) + if dupHandle, err := v.Create(sctx, txn.GetUnionStore(), indexVals, recordID, rsData, opts...); err != nil { if kv.ErrKeyExists.Equal(err) { return dupHandle, dupErr } @@ -1138,13 +1140,14 @@ func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, h kv.Handle, } // buildIndexForRow implements table.Table BuildIndexForRow interface. -func (t *TableCommon) buildIndexForRow(ctx sessionctx.Context, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction, untouched bool, popts ...table.CreateIdxOptFunc) error { +func (t *TableCommon) buildIndexForRow(ctx sessionctx.Context, h kv.Handle, vals []types.Datum, newData []types.Datum, idx table.Index, txn kv.Transaction, untouched bool, popts ...table.CreateIdxOptFunc) error { var opts []table.CreateIdxOptFunc opts = append(opts, popts...) if untouched { opts = append(opts, table.IndexIsUntouched) } - if _, err := idx.Create(ctx, txn.GetUnionStore(), vals, h, opts...); err != nil { + rsData := TryGetHandleRestoredDataWrapper(t, newData, nil) + if _, err := idx.Create(ctx, txn.GetUnionStore(), vals, h, rsData, opts...); err != nil { if kv.ErrKeyExists.Equal(err) { // Make error message consistent with MySQL. entryKey, err1 := t.genIndexKeyStr(vals) @@ -1682,6 +1685,50 @@ func (t *TableCommon) GetSequenceCommon() *sequenceCommon { return t.sequence } +// TryGetHandleRestoredDataWrapper tries to get the restored data for handle if needed. The argument can be a slice or a map. +func TryGetHandleRestoredDataWrapper(t table.Table, row []types.Datum, rowMap map[int64]types.Datum) []types.Datum { + if !collate.NewCollationEnabled() || !t.Meta().IsCommonHandle || t.Meta().CommonHandleVersion == 0 { + return nil + } + + useIDMap := false + if len(rowMap) > 0 { + useIDMap = true + } + + var datum types.Datum + rsData := make([]types.Datum, 0, 4) + pkCols := TryGetCommonPkColumns(t) + for _, col := range pkCols { + if !types.NeedRestoredData(&col.FieldType) { + continue + } + if collate.IsBinCollation(col.Collate) { + if useIDMap { + datum = rowMap[col.ID] + } else { + datum = row[col.Offset] + } + rsData = append(rsData, types.NewIntDatum(stringutil.GetTailSpaceCount(datum.GetString()))) + } else { + if useIDMap { + rsData = append(rsData, rowMap[col.ID]) + } else { + rsData = append(rsData, row[col.Offset]) + } + } + } + + for _, idx := range t.Meta().Indices { + if idx.Primary { + tablecodec.TruncateIndexValues(t.Meta(), idx, rsData) + break + } + } + + return rsData +} + func getSequenceAllocator(allocs autoid.Allocators) (autoid.Allocator, error) { for _, alloc := range allocs { if alloc.GetType() == autoid.SequenceType { diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 957243cda5f42..e9b6f291ff2cd 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -17,6 +17,7 @@ import ( "bytes" "encoding/binary" "math" + "strings" "time" "unicode/utf8" @@ -34,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tidb/util/stringutil" ) var ( @@ -64,6 +66,8 @@ const ( CommonHandleFlag byte = 127 // PartitionIDFlag is the flag used to decode the partition ID in global index value. PartitionIDFlag byte = 126 + // IndexVersionFlag is the flag used to decode the index's version info. + IndexVersionFlag byte = 125 // RestoreDataFlag is the flag that RestoreData begin with. // See rowcodec.Encoder.Encode and rowcodec.row.toBytes RestoreDataFlag byte = rowcodec.CodecVer @@ -719,6 +723,19 @@ func reEncodeHandle(handle kv.Handle, unsigned bool) ([][]byte, error) { return [][]byte{intHandleBytes}, err } +// reEncodeHandleConsiderNewCollation encodes the handle as a Datum so it can be properly decoded later. +func reEncodeHandleConsiderNewCollation(handle kv.Handle, columns []rowcodec.ColInfo, restoreData []byte, unsigned bool) ([][]byte, error) { + handleColLen := handle.NumCols() + cHandleBytes := make([][]byte, 0, handleColLen) + for i := 0; i < handleColLen; i++ { + cHandleBytes = append(cHandleBytes, handle.EncodedCol(i)) + } + if len(restoreData) == 0 { + return cHandleBytes, nil + } + return decodeRestoredValuesV5(columns, cHandleBytes, restoreData) +} + func decodeRestoredValues(columns []rowcodec.ColInfo, restoredVal []byte) ([][]byte, error) { colIDs := make(map[int64]int, len(columns)) for i, col := range columns { @@ -733,6 +750,78 @@ func decodeRestoredValues(columns []rowcodec.ColInfo, restoredVal []byte) ([][]b return resultValues, nil } +// decodeRestoredValuesV5 decodes index values whose format is introduced in TiDB 5.0. +// Unlike the format in TiDB 4.0, the new format is optimized for storage space: +// 1. If the index is a composed index, only the non-binary string column's value need to write to value, not all. +// 2. If a string column's collation is _bin, then we only write the number of the truncated spaces to value. +// 3. If a string column is char, not varchar, then we use the sortKey directly. +func decodeRestoredValuesV5(columns []rowcodec.ColInfo, keyVal [][]byte, restoredVal []byte) ([][]byte, error) { + colIDs := make(map[int64]int, len(columns)) + result := make([][]byte, len(columns)) + // restoredData2All is the slice from the offset in restoredColumns to the offset in columns. + restoredData2All := make([]int, len(columns)) + restoredColumns := make([]rowcodec.ColInfo, 0, len(columns)) + j := 0 + + // Collect some information, restoredColumns means the columns whose value need to restore from the index value. + for i, col := range columns { + if types.NeedRestoredData(col.Ft) { + colIDs[col.ID] = j + restoredData2All[j] = i + j++ + copyColInfo := rowcodec.ColInfo{ + ID: col.ID, + Ft: columns[i].Ft, + } + if collate.IsBinCollation(col.Ft.Collate) { + // Change the fieldType from string to uint since we store the number of the truncated spaces. + copyColInfo.Ft = types.NewFieldType(mysql.TypeLonglong) + } + restoredColumns = append(restoredColumns, copyColInfo) + } else { + // Use the value in index key directly. + result[i] = keyVal[i] + } + } + + // We don't need to decode handle here, and colIDs >= 0 always. + rd := rowcodec.NewByteDecoder(restoredColumns, []int64{-1}, nil, nil) + restoredValues, err := rd.DecodeToBytesNoHandle(colIDs, restoredVal) + if err != nil { + return nil, errors.Trace(err) + } + + // Restore value. If it is the _bin collation, we use the sortKey and restore value together to get the original value. + // Otherwise, use the restore value directly. + for _, offset := range colIDs { + rv := restoredValues[offset] + allOffset := restoredData2All[offset] + if collate.IsBinCollation(columns[allOffset].Ft.Collate) { + noPaddingStr, err := DecodeColumnValue(keyVal[allOffset], columns[allOffset].Ft, nil) + if err != nil { + return nil, errors.Trace(err) + } + paddingCount, err := DecodeColumnValue(restoredValues[offset], types.NewFieldType(mysql.TypeLonglong), nil) + if err != nil { + return nil, errors.Trace(err) + } + // Skip if padding count is 0. + if paddingCount.GetInt64() == 0 { + result[allOffset] = keyVal[allOffset] + continue + } + noPaddingStr.SetString(noPaddingStr.GetString()+strings.Repeat(" ", int(paddingCount.GetInt64())), noPaddingStr.Collation()) + result[allOffset] = result[allOffset][:0] + result[allOffset] = append(result[allOffset], rowcodec.BytesFlag) + result[allOffset] = codec.EncodeBytes(result[allOffset], noPaddingStr.GetBytes()) + } else { + result[allOffset] = rv + } + } + + return result, nil +} + func decodeIndexKvOldCollation(key, value []byte, colsLen int, hdStatus HandleStatus) ([][]byte, error) { resultValues, b, err := CutIndexKeyNew(key, colsLen) if err != nil { @@ -765,11 +854,25 @@ func decodeIndexKvOldCollation(key, value []byte, colsLen int, hdStatus HandleSt return resultValues, nil } +func getIndexVersion(value []byte) int { + if len(value) <= MaxOldEncodeValueLen { + return 0 + } + tailLen := int(value[0]) + if (tailLen == 0 || tailLen == 1) && value[1] == IndexVersionFlag { + return int(value[2]) + } + return 0 +} + // DecodeIndexKV uses to decode index key values. func DecodeIndexKV(key, value []byte, colsLen int, hdStatus HandleStatus, columns []rowcodec.ColInfo) ([][]byte, error) { if len(value) <= MaxOldEncodeValueLen { return decodeIndexKvOldCollation(key, value, colsLen, hdStatus) } + if getIndexVersion(value) == 1 { + return decodeIndexKvForClusteredIndexVersion1(key, value, colsLen, hdStatus, columns) + } return decodeIndexKvGeneral(key, value, colsLen, hdStatus, columns) } @@ -800,6 +903,10 @@ func decodeHandleInIndexKey(keySuffix []byte) (kv.Handle, error) { } func decodeHandleInIndexValue(value []byte) (kv.Handle, error) { + if getIndexVersion(value) == 1 { + seg := SplitIndexValueForClusteredIndexVersion1(value) + return kv.NewCommonHandle(seg.CommonHandle) + } if len(value) > MaxOldEncodeValueLen { tailLen := value[0] if tailLen >= 8 { @@ -964,15 +1071,143 @@ func GenIndexKey(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo return } -// GenIndexValue creates encoded index value and returns the result, only support local index -func GenIndexValue(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, containNonBinaryString bool, - distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle) ([]byte, error) { - return GenIndexValueNew(sc, tblInfo, idxInfo, containNonBinaryString, distinct, untouched, indexedValues, h, 0) +// GenIndexValuePortal is the portal for generating index value. +// Value layout: +// +-- IndexValueVersion0 (with restore data, or common handle, or index is global) +// | +// | Layout: TailLen | Options | Padding | [IntHandle] | [UntouchedFlag] +// | Length: 1 | len(options) | len(padding) | 8 | 1 +// | +// | TailLen: len(padding) + len(IntHandle) + len(UntouchedFlag) +// | Options: Encode some value for new features, such as common handle, new collations or global index. +// | See below for more information. +// | Padding: Ensure length of value always >= 10. (or >= 11 if UntouchedFlag exists.) +// | IntHandle: Only exists when table use int handles and index is unique. +// | UntouchedFlag: Only exists when index is untouched. +// | +// +-- Old Encoding (without restore data, integer handle, local) +// | +// | Layout: [Handle] | [UntouchedFlag] +// | Length: 8 | 1 +// | +// | Handle: Only exists in unique index. +// | UntouchedFlag: Only exists when index is untouched. +// | +// | If neither Handle nor UntouchedFlag exists, value will be one single byte '0' (i.e. []byte{'0'}). +// | Length of value <= 9, use to distinguish from the new encoding. +// | +// +-- IndexValueForClusteredIndexVersion1 +// | +// | Layout: TailLen | VersionFlag | Version | Options | [UntouchedFlag] +// | Length: 1 | 1 | 1 | len(options) | 1 +// | +// | TailLen: len(UntouchedFlag) +// | Options: Encode some value for new features, such as common handle, new collations or global index. +// | See below for more information. +// | UntouchedFlag: Only exists when index is untouched. +// | +// | Layout of Options: +// | +// | Segment: Common Handle | Global Index | New Collation +// | Layout: CHandle Flag | CHandle Len | CHandle | PidFlag | PartitionID | restoreData +// | Length: 1 | 2 | len(CHandle) | 1 | 8 | len(restoreData) +// | +// | Common Handle Segment: Exists when unique index used common handles. +// | Global Index Segment: Exists when index is global. +// | New Collation Segment: Exists when new collation is used and index or handle contains non-binary string. +// | In v4.0, restored data contains all the index values. For example, (a int, b char(10)) and index (a, b). +// | The restored data contains both the values of a and b. +// | In v5.0, restored data contains only non-binary data(except for char and _bin). In the above example, the restored data contains only the value of b. +// | Besides, if the collation of b is _bin, then restored data is an integer indicate the spaces are truncated. Then we use sortKey +// | and the restored data together to restore original data. +func GenIndexValuePortal(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, needRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, restoredData []types.Datum) ([]byte, error) { + if tblInfo.IsCommonHandle && tblInfo.CommonHandleVersion == 1 { + return GenIndexValueForClusteredIndexVersion1(sc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, restoredData) + } + return genIndexValueVersion0(sc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID) +} + +// TryGetCommonPkColumnRestoredIds get the IDs of primary key columns which need restored data if the table has common handle. +// Caller need to make sure the table has common handle. +func TryGetCommonPkColumnRestoredIds(tbl *model.TableInfo) []int64 { + var pkColIds []int64 + var pkIdx *model.IndexInfo + for _, idx := range tbl.Indices { + if idx.Primary { + pkIdx = idx + break + } + } + if pkIdx == nil { + return pkColIds + } + for _, idxCol := range pkIdx.Columns { + if types.NeedRestoredData(&tbl.Columns[idxCol.Offset].FieldType) { + pkColIds = append(pkColIds, tbl.Columns[idxCol.Offset].ID) + } + } + return pkColIds } -// GenIndexValueNew create index value for both local and global index. -func GenIndexValueNew(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, containNonBinaryString bool, - distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64) ([]byte, error) { +// GenIndexValueForClusteredIndexVersion1 generates the index value for the clustered index with version 1(New in v5.0.0). +func GenIndexValueForClusteredIndexVersion1(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, IdxValNeedRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, handleRestoredData []types.Datum) ([]byte, error) { + idxVal := make([]byte, 1) + tailLen := 0 + // Version info. + idxVal = append(idxVal, IndexVersionFlag) + idxVal = append(idxVal, byte(1)) + + if distinct { + idxVal = encodeCommonHandle(idxVal, h) + } + if idxInfo.Global { + idxVal = encodePartitionID(idxVal, partitionID) + } + if collate.NewCollationEnabled() && (IdxValNeedRestoredData || len(handleRestoredData) > 0) { + colIds := make([]int64, 0, len(idxInfo.Columns)) + allRestoredData := make([]types.Datum, 0, len(handleRestoredData)+len(idxInfo.Columns)) + for i, idxCol := range idxInfo.Columns { + col := tblInfo.Columns[idxCol.Offset] + // If the column is the primary key's column, + // the restored data will be written later. Skip writing it here to avoid redundancy. + if mysql.HasPriKeyFlag(col.Flag) { + continue + } + if types.NeedRestoredData(&col.FieldType) { + colIds = append(colIds, col.ID) + if collate.IsBinCollation(col.Collate) { + allRestoredData = append(allRestoredData, types.NewUintDatum(uint64(stringutil.GetTailSpaceCount(indexedValues[i].GetString())))) + } else { + allRestoredData = append(allRestoredData, indexedValues[i]) + } + } + } + + if len(handleRestoredData) > 0 { + pkColIds := TryGetCommonPkColumnRestoredIds(tblInfo) + colIds = append(colIds, pkColIds...) + allRestoredData = append(allRestoredData, handleRestoredData...) + } + + rd := rowcodec.Encoder{Enable: true} + rowRestoredValue, err := rd.Encode(sc, colIds, allRestoredData, nil) + if err != nil { + return nil, err + } + idxVal = append(idxVal, rowRestoredValue...) + } + + if untouched { + tailLen = 1 + idxVal = append(idxVal, kv.UnCommitIndexKVFlag) + } + idxVal[0] = byte(tailLen) + + return idxVal, nil +} + +// genIndexValueVersion0 create index value for both local and global index. +func genIndexValueVersion0(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, IdxValNeedRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64) ([]byte, error) { idxVal := make([]byte, 1) newEncode := false tailLen := 0 @@ -984,7 +1219,7 @@ func GenIndexValueNew(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, id idxVal = encodePartitionID(idxVal, partitionID) newEncode = true } - if collate.NewCollationEnabled() && containNonBinaryString { + if collate.NewCollationEnabled() && IdxValNeedRestoredData { colIds := make([]int64, len(idxInfo.Columns)) for i, col := range idxInfo.Columns { colIds[i] = tblInfo.Columns[col.Offset].ID @@ -1100,6 +1335,15 @@ func DecodeHandleInUniqueIndexValue(data []byte, isCommonHandle bool) (kv.Handle } return kv.IntHandle(int64(binary.BigEndian.Uint64(data[dLen-int(data[0]):]))), nil } + if getIndexVersion(data) == 1 { + seg := SplitIndexValueForClusteredIndexVersion1(data) + h, err := kv.NewCommonHandle(seg.CommonHandle) + if err != nil { + return nil, err + } + return h, nil + } + tailLen := int(data[0]) data = data[:len(data)-tailLen] handleLen := uint16(data[2])<<8 + uint16(data[3]) @@ -1149,6 +1393,76 @@ func SplitIndexValue(value []byte) (segs IndexValueSegments) { return } +// SplitIndexValueForClusteredIndexVersion1 splits index value into segments. +func SplitIndexValueForClusteredIndexVersion1(value []byte) (segs IndexValueSegments) { + tailLen := int(value[0]) + // Skip the tailLen and version info. + value = value[3 : len(value)-tailLen] + if len(value) > 0 && value[0] == CommonHandleFlag { + handleLen := uint16(value[1])<<8 + uint16(value[2]) + handleEndOff := 3 + handleLen + segs.CommonHandle = value[3:handleEndOff] + value = value[handleEndOff:] + } + if len(value) > 0 && value[0] == PartitionIDFlag { + segs.PartitionID = value[1:9] + value = value[9:] + } + if len(value) > 0 && value[0] == RestoreDataFlag { + segs.RestoredValues = value + } + return +} + +func decodeIndexKvForClusteredIndexVersion1(key, value []byte, colsLen int, hdStatus HandleStatus, columns []rowcodec.ColInfo) ([][]byte, error) { + var resultValues [][]byte + var keySuffix []byte + var handle kv.Handle + var err error + segs := SplitIndexValueForClusteredIndexVersion1(value) + resultValues, keySuffix, err = CutIndexKeyNew(key, colsLen) + if err != nil { + return nil, err + } + if segs.RestoredValues != nil { + resultValues, err = decodeRestoredValuesV5(columns[:colsLen], resultValues, segs.RestoredValues) + if err != nil { + return nil, err + } + } + if hdStatus == HandleNotNeeded { + return resultValues, nil + } + if segs.CommonHandle != nil { + // In unique common handle index. + handle, err = kv.NewCommonHandle(segs.CommonHandle) + } else { + // In non-unique index, decode handle in keySuffix. + handle, err = kv.NewCommonHandle(keySuffix) + } + if err != nil { + return nil, err + } + handleBytes, err := reEncodeHandleConsiderNewCollation(handle, columns[colsLen:], segs.RestoredValues, hdStatus == HandleIsUnsigned) + if err != nil { + return nil, err + } + resultValues = append(resultValues, handleBytes...) + if segs.PartitionID != nil { + _, pid, err := codec.DecodeInt(segs.PartitionID) + if err != nil { + return nil, err + } + datum := types.NewIntDatum(pid) + pidBytes, err := codec.EncodeValue(nil, nil, datum) + if err != nil { + return nil, err + } + resultValues = append(resultValues, pidBytes) + } + return resultValues, nil +} + // decodeIndexKvGeneral decodes index key value pair of new layout in an extensible way. func decodeIndexKvGeneral(key, value []byte, colsLen int, hdStatus HandleStatus, columns []rowcodec.ColInfo) ([][]byte, error) { var resultValues [][]byte diff --git a/types/etc.go b/types/etc.go index 80a63d57ffd02..90d7da96287d3 100644 --- a/types/etc.go +++ b/types/etc.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/opcode" "github.com/pingcap/parser/terror" ast "github.com/pingcap/parser/types" + "github.com/pingcap/tidb/util/collate" ) // IsTypeBlob returns a boolean indicating whether the tp is a blob type. @@ -104,6 +105,15 @@ func IsNonBinaryStr(ft *FieldType) bool { return false } +// NeedRestoredData returns if a type needs restored data. +// If the type is char and the collation is _bin, NeedRestoredData() returns false. +func NeedRestoredData(ft *FieldType) bool { + if IsNonBinaryStr(ft) && !(collate.IsBinCollation(ft.Collate) && !IsTypeVarchar(ft.Tp)) { + return true + } + return false +} + // IsString returns a boolean indicating // whether the field type is a string type. func IsString(tp byte) bool { diff --git a/util/collate/collate.go b/util/collate/collate.go index e4d3160890a73..86a599318b587 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -278,6 +278,12 @@ func IsCICollation(collate string) bool { collate == "utf8_unicode_ci" || collate == "utf8mb4_unicode_ci" } +// IsBinCollation returns if the collation is 'xx_bin' +func IsBinCollation(collate string) bool { + return collate == "ascii_bin" || collate == "latin1_bin" || + collate == "utf8_bin" || collate == "utf8mb4_bin" +} + func init() { newCollatorMap = make(map[string]Collator) newCollatorIDMap = make(map[int]Collator) diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index 4b453df81b394..04d71dda0d19a 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -171,7 +171,7 @@ func (s *testDecoderSuite) TestClusterIndexRowDecoder(c *C) { cols := []*model.ColumnInfo{c1, c2, c3} - tblInfo := &model.TableInfo{ID: 1, Columns: cols, Indices: []*model.IndexInfo{pk}, IsCommonHandle: true} + tblInfo := &model.TableInfo{ID: 1, Columns: cols, Indices: []*model.IndexInfo{pk}, IsCommonHandle: true, CommonHandleVersion: 1} tbl := tables.MockTableFromMeta(tblInfo) ctx := mock.NewContext() diff --git a/util/stringutil/string_util.go b/util/stringutil/string_util.go index d4ef3166e0e3c..ae0722d8b0069 100644 --- a/util/stringutil/string_util.go +++ b/util/stringutil/string_util.go @@ -366,3 +366,12 @@ func BuildStringFromLabels(labels map[string]string) string { returned := r.String() return returned[:len(returned)-1] } + +// GetTailSpaceCount returns the number of tailed spaces. +func GetTailSpaceCount(str string) int64 { + length := len(str) + for length > 0 && str[length-1] == ' ' { + length-- + } + return int64(len(str) - length) +}