From 9d56715b6a3f9a322d0e01c44853e695e88506b9 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Fri, 31 May 2024 16:56:37 +0800 Subject: [PATCH 1/6] test --- pkg/executor/admin.go | 9 ++++----- pkg/tablecodec/tablecodec.go | 29 ++++------------------------- pkg/util/admin/admin.go | 7 +++++-- 3 files changed, 13 insertions(+), 32 deletions(-) diff --git a/pkg/executor/admin.go b/pkg/executor/admin.go index a9266bade1139..dc7756425c014 100644 --- a/pkg/executor/admin.go +++ b/pkg/executor/admin.go @@ -381,6 +381,9 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists if err != nil { return nil, err } + if e.index.Meta().Global { + handle = kv.NewPartitionHandle(e.physicalID, handle) + } idxVals, err := e.buildIndexedValues(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes, idxValLen) if err != nil { return nil, err @@ -614,11 +617,7 @@ func (e *CleanupIndexExec) getIdxColTypes() []*types.FieldType { func (e *CleanupIndexExec) batchGetRecord(txn kv.Transaction) (map[string][]byte, error) { e.idxValues.Range(func(h kv.Handle, _ any) bool { - if ph, ok := h.(kv.PartitionHandle); ok { - e.batchKeys = append(e.batchKeys, tablecodec.EncodeRecordKey(tablecodec.GenTableRecordPrefix(ph.PartitionID), ph.Handle)) - } else { - e.batchKeys = append(e.batchKeys, tablecodec.EncodeRecordKey(e.table.RecordPrefix(), h)) - } + e.batchKeys = append(e.batchKeys, tablecodec.EncodeRecordKey(e.table.RecordPrefix(), h)) return true }) values, err := txn.BatchGet(context.Background(), e.batchKeys) diff --git a/pkg/tablecodec/tablecodec.go b/pkg/tablecodec/tablecodec.go index 27ee0ea24e65b..713d3537e806b 100644 --- a/pkg/tablecodec/tablecodec.go +++ b/pkg/tablecodec/tablecodec.go @@ -109,6 +109,9 @@ func CutRowKeyPrefix(key kv.Key) []byte { // EncodeRecordKey encodes the recordPrefix, row handle into a kv.Key. func EncodeRecordKey(recordPrefix kv.Key, h kv.Handle) kv.Key { buf := make([]byte, 0, len(recordPrefix)+h.Len()) + if ph, ok := h.(kv.PartitionHandle); ok { + recordPrefix = GenTableRecordPrefix(ph.PartitionID) + } buf = append(buf, recordPrefix...) buf = append(buf, h.Encoded()...) return buf @@ -1678,31 +1681,7 @@ func encodeCommonHandle(idxVal []byte, h kv.Handle) []byte { // DecodeHandleInUniqueIndexValue decodes handle in data. func DecodeHandleInUniqueIndexValue(data []byte, isCommonHandle bool) (kv.Handle, error) { - if !isCommonHandle { - dLen := len(data) - if dLen <= MaxOldEncodeValueLen { - return kv.IntHandle(int64(binary.BigEndian.Uint64(data))), nil - } - 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]) - handleEndOff := 4 + handleLen - h, err := kv.NewCommonHandle(data[4:handleEndOff]) - if err != nil { - return nil, err - } - return h, nil + return decodeHandleInIndexValue(data) } func encodePartitionID(idxVal []byte, partitionID int64) []byte { diff --git a/pkg/util/admin/admin.go b/pkg/util/admin/admin.go index 8297bb866d555..cc1c326ea4462 100644 --- a/pkg/util/admin/admin.go +++ b/pkg/util/admin/admin.go @@ -184,7 +184,7 @@ func CheckRecordAndIndex(ctx context.Context, sessCtx sessionctx.Context, txn kv return true, nil } - err := iterRecords(sessCtx, txn, t, startKey, cols, filterFunc) + err := iterRecords(sessCtx, txn, t, startKey, cols, idx.Meta().Global, filterFunc) if err != nil { return errors.Trace(err) } @@ -204,7 +204,7 @@ func makeRowDecoder(t table.Table, sctx sessionctx.Context) (*decoder.RowDecoder return decoder.NewRowDecoder(t, t.Cols(), decodeColsMap), nil } -func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Table, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { +func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Table, startKey kv.Key, cols []*table.Column, isGlobalIndex bool, fn table.RecordIterFunc) error { prefix := t.RecordPrefix() keyUpperBound := prefix.PrefixNext() @@ -234,6 +234,9 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab if err != nil { return errors.Trace(err) } + if isGlobalIndex { + handle = kv.NewPartitionHandle(tablecodec.DecodeTableID(it.Key()), handle) + } rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), nil) if err != nil { From 2a6466eeb0043c9ffe8562250d47a28dc7eeba30 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Mon, 3 Jun 2024 11:55:59 +0800 Subject: [PATCH 2/6] remove DecodeHandleInUniqueIndexValue func --- pkg/ddl/index_merge_tmp.go | 2 +- pkg/executor/admin.go | 3 +- pkg/executor/batch_checker.go | 6 ++-- pkg/executor/batch_point_get.go | 2 +- pkg/executor/insert.go | 6 ++-- pkg/executor/insert_common.go | 2 +- pkg/executor/point_get.go | 2 +- pkg/executor/point_get_test.go | 2 +- pkg/executor/replace.go | 2 +- pkg/server/conn.go | 3 +- pkg/table/tables/index.go | 26 ++++++++-------- pkg/tablecodec/bench_test.go | 30 +++++++++++++++++++ pkg/tablecodec/tablecodec.go | 13 ++++---- pkg/tablecodec/tablecodec_test.go | 2 +- .../pessimistictest/pessimistic_test.go | 2 +- 15 files changed, 64 insertions(+), 39 deletions(-) diff --git a/pkg/ddl/index_merge_tmp.go b/pkg/ddl/index_merge_tmp.go index ba4619f3c8d33..f55658988e7ea 100644 --- a/pkg/ddl/index_merge_tmp.go +++ b/pkg/ddl/index_merge_tmp.go @@ -84,7 +84,7 @@ func checkTempIndexKey(txn kv.Transaction, tmpRec *temporaryIndexRecord, originI return nil } // For distinct index key values, prevent deleting an unexpected index KV in original index. - hdInVal, err := tablecodec.DecodeHandleInUniqueIndexValue(originIdxVal, tblInfo.Meta().IsCommonHandle) + hdInVal, err := tablecodec.DecodeHandleInIndexValue(originIdxVal) if err != nil { return errors.Trace(err) } diff --git a/pkg/executor/admin.go b/pkg/executor/admin.go index dc7756425c014..f38b8bb42eba7 100644 --- a/pkg/executor/admin.go +++ b/pkg/executor/admin.go @@ -475,12 +475,11 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) // 1. unique-key is duplicate and the handle is equal, skip it. // 2. unique-key is duplicate and the handle is not equal, data is not consistent, log it and skip it. // 3. non-unique-key is duplicate, skip it. - isCommonHandle := e.table.Meta().IsCommonHandle for i, key := range e.batchKeys { val, found := values[string(key)] if found { if distinctFlags[i] { - handle, err1 := tablecodec.DecodeHandleInUniqueIndexValue(val, isCommonHandle) + handle, err1 := tablecodec.DecodeHandleInIndexValue(val) if err1 != nil { return err1 } diff --git a/pkg/executor/batch_checker.go b/pkg/executor/batch_checker.go index e4a34f12bd4ab..798df3ffce561 100644 --- a/pkg/executor/batch_checker.go +++ b/pkg/executor/batch_checker.go @@ -39,7 +39,6 @@ import ( type keyValueWithDupInfo struct { newKey kv.Key dupErr error - commonHandle bool } type toBeCheckedRow struct { @@ -213,9 +212,8 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D return nil, err1 } uniqueKeys = append(uniqueKeys, &keyValueWithDupInfo{ - newKey: key, - dupErr: kv.ErrKeyExists.FastGenByArgs(colValStr, fmt.Sprintf("%s.%s", v.TableMeta().Name.String(), v.Meta().Name.String())), - commonHandle: t.Meta().IsCommonHandle, + newKey: key, + dupErr: kv.ErrKeyExists.FastGenByArgs(colValStr, fmt.Sprintf("%s.%s", v.TableMeta().Name.String(), v.Meta().Name.String())), }) } } diff --git a/pkg/executor/batch_point_get.go b/pkg/executor/batch_point_get.go index 96f469fcff71d..4b0262b44014d 100644 --- a/pkg/executor/batch_point_get.go +++ b/pkg/executor/batch_point_get.go @@ -290,7 +290,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if len(handleVal) == 0 { continue } - handle, err1 := tablecodec.DecodeHandleInUniqueIndexValue(handleVal, e.tblInfo.IsCommonHandle) + handle, err1 := tablecodec.DecodeHandleInIndexValue(handleVal) if err1 != nil { return err1 } diff --git a/pkg/executor/insert.go b/pkg/executor/insert.go index 24c7cf6066587..86895458e9ecd 100644 --- a/pkg/executor/insert.go +++ b/pkg/executor/insert.go @@ -161,12 +161,12 @@ func prefetchConflictedOldRows(ctx context.Context, txn kv.Transaction, rows []t for _, uk := range r.uniqueKeys { if val, found := values[string(uk.newKey)]; found { if tablecodec.IsTempIndexKey(uk.newKey) { - // If it is a temp index, the value cannot be decoded by DecodeHandleInUniqueIndexValue. + // If it is a temp index, the value cannot be decoded by DecodeHandleInIndexValue. // Since this function is an optimization, we can skip prefetching the rows referenced by // temp indexes. continue } - handle, err := tablecodec.DecodeHandleInUniqueIndexValue(val, uk.commonHandle) + handle, err := tablecodec.DecodeHandleInIndexValue(val) if err != nil { return err } @@ -253,7 +253,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D } for _, uk := range r.uniqueKeys { - _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID, uk.commonHandle) + _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID) if err != nil { return err } diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index 4915286e920fa..007e8fa099cab 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -1178,7 +1178,7 @@ func (e *InsertValues) handleDuplicateKey(ctx context.Context, txn kv.Transactio } return true, nil } - _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID, uk.commonHandle) + _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID) if err != nil { return false, err } diff --git a/pkg/executor/point_get.go b/pkg/executor/point_get.go index d7ff703552af1..5f2472744e899 100644 --- a/pkg/executor/point_get.go +++ b/pkg/executor/point_get.go @@ -329,7 +329,7 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { } var iv kv.Handle - iv, err = tablecodec.DecodeHandleInUniqueIndexValue(e.handleVal, e.tblInfo.IsCommonHandle) + iv, err = tablecodec.DecodeHandleInIndexValue(e.handleVal) if err != nil { return err } diff --git a/pkg/executor/point_get_test.go b/pkg/executor/point_get_test.go index 3d7a061466600..d4058f3e16f78 100644 --- a/pkg/executor/point_get_test.go +++ b/pkg/executor/point_get_test.go @@ -88,7 +88,7 @@ func TestReturnValues(t *testing.T) { txnCtx := tk.Session().GetSessionVars().TxnCtx val, ok := txnCtx.GetKeyInPessimisticLockCache(pk) require.True(t, ok) - handle, err := tablecodec.DecodeHandleInUniqueIndexValue(val, false) + handle, err := tablecodec.DecodeHandleInIndexValue(val) require.NoError(t, err) rowKey := tablecodec.EncodeRowKeyWithHandle(tid, handle) _, ok = txnCtx.GetKeyInPessimisticLockCache(rowKey) diff --git a/pkg/executor/replace.go b/pkg/executor/replace.go index 687ddaf00a333..bc9a5d7447f82 100644 --- a/pkg/executor/replace.go +++ b/pkg/executor/replace.go @@ -120,7 +120,7 @@ func (e *ReplaceExec) replaceRow(ctx context.Context, r toBeCheckedRow) error { // 3. error: the error. func (e *ReplaceExec) removeIndexRow(ctx context.Context, txn kv.Transaction, r toBeCheckedRow) (rowUnchanged, foundDupKey bool, err error) { for _, uk := range r.uniqueKeys { - _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID, uk.commonHandle) + _, handle, err := tables.FetchDuplicatedHandle(ctx, uk.newKey, true, txn, e.Table.Meta().ID) if err != nil { return false, false, err } diff --git a/pkg/server/conn.go b/pkg/server/conn.go index 7f3c3caea09ec..360c3b2fe71b1 100644 --- a/pkg/server/conn.go +++ b/pkg/server/conn.go @@ -1951,8 +1951,7 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm return nil, err1 } for idxKey, idxVal := range idxVals { - isCommonHd := isCommonHandle[idxKey] - h, err2 := tablecodec.DecodeHandleInUniqueIndexValue(idxVal, isCommonHd) + h, err2 := tablecodec.DecodeHandleInIndexValue(idxVal) if err2 != nil { return nil, err2 } diff --git a/pkg/table/tables/index.go b/pkg/table/tables/index.go index 71443ea8776fd..ccd351efd2258 100644 --- a/pkg/table/tables/index.go +++ b/pkg/table/tables/index.go @@ -308,7 +308,7 @@ func (c *index) Create(sctx table.MutateContext, txn kv.Transaction, indexedValu val = tempVal.Encode(value) } needPresumeNotExists, err := needPresumeKeyNotExistsFlag(ctx, txn, key, tempKey, h, - keyIsTempIdxKey, c.tblInfo.IsCommonHandle, c.tblInfo.ID) + keyIsTempIdxKey, c.tblInfo.ID) if err != nil { return nil, err } @@ -356,7 +356,7 @@ func (c *index) Create(sctx table.MutateContext, txn kv.Transaction, indexedValu if keyIsTempIdxKey && !tempIdxVal.IsEmpty() { value = tempIdxVal.Current().Value } - handle, err := tablecodec.DecodeHandleInUniqueIndexValue(value, c.tblInfo.IsCommonHandle) + handle, err := tablecodec.DecodeHandleInIndexValue(value) if err != nil { return nil, err } @@ -366,7 +366,7 @@ func (c *index) Create(sctx table.MutateContext, txn kv.Transaction, indexedValu } func needPresumeKeyNotExistsFlag(ctx context.Context, txn kv.Transaction, key, tempKey kv.Key, - h kv.Handle, keyIsTempIdxKey bool, isCommon bool, tblID int64) (needFlag bool, err error) { + h kv.Handle, keyIsTempIdxKey bool, tblID int64) (needFlag bool, err error) { var uniqueTempKey kv.Key if keyIsTempIdxKey { uniqueTempKey = key @@ -375,7 +375,7 @@ func needPresumeKeyNotExistsFlag(ctx context.Context, txn kv.Transaction, key, t } else { return true, nil } - foundKey, dupHandle, err := FetchDuplicatedHandle(ctx, uniqueTempKey, true, txn, tblID, isCommon) + foundKey, dupHandle, err := FetchDuplicatedHandle(ctx, uniqueTempKey, true, txn, tblID) if err != nil { return false, err } @@ -417,7 +417,7 @@ func (c *index) Delete(ctx table.MutateContext, txn kv.Transaction, indexedValue return err } if len(originVal) > 0 { - oh, err := tablecodec.DecodeHandleInUniqueIndexValue(originVal, c.tblInfo.IsCommonHandle) + oh, err := tablecodec.DecodeHandleInIndexValue(originVal) if err != nil { return err } @@ -526,7 +526,7 @@ func (c *index) Exist(ec errctx.Context, loc *time.Location, txn kv.Transaction, if len(tempKey) > 0 { key = tempKey } - foundKey, dupHandle, err := FetchDuplicatedHandle(context.TODO(), key, distinct, txn, c.tblInfo.ID, c.tblInfo.IsCommonHandle) + foundKey, dupHandle, err := FetchDuplicatedHandle(context.TODO(), key, distinct, txn, c.tblInfo.ID) if err != nil || !foundKey { return false, nil, err } @@ -540,9 +540,9 @@ func (c *index) Exist(ec errctx.Context, loc *time.Location, txn kv.Transaction, // FetchDuplicatedHandle is used to find the duplicated row's handle for a given unique index key. func FetchDuplicatedHandle(ctx context.Context, key kv.Key, distinct bool, - txn kv.Transaction, tableID int64, isCommon bool) (foundKey bool, dupHandle kv.Handle, err error) { + txn kv.Transaction, tableID int64) (foundKey bool, dupHandle kv.Handle, err error) { if tablecodec.IsTempIndexKey(key) { - return fetchDuplicatedHandleForTempIndexKey(ctx, key, distinct, txn, tableID, isCommon) + return fetchDuplicatedHandleForTempIndexKey(ctx, key, distinct, txn, tableID) } // The index key is not from temp index. val, err := getKeyInTxn(ctx, txn, key) @@ -550,14 +550,14 @@ func FetchDuplicatedHandle(ctx context.Context, key kv.Key, distinct bool, return false, nil, err } if distinct { - h, err := tablecodec.DecodeHandleInUniqueIndexValue(val, isCommon) + h, err := tablecodec.DecodeHandleInIndexValue(val) return true, h, err } return true, nil, nil } func fetchDuplicatedHandleForTempIndexKey(ctx context.Context, tempKey kv.Key, distinct bool, - txn kv.Transaction, tableID int64, isCommon bool) (foundKey bool, dupHandle kv.Handle, err error) { + txn kv.Transaction, tableID int64) (foundKey bool, dupHandle kv.Handle, err error) { tempRawVal, err := getKeyInTxn(ctx, txn, tempKey) if err != nil { return false, nil, err @@ -570,7 +570,7 @@ func fetchDuplicatedHandleForTempIndexKey(ctx context.Context, tempKey kv.Key, d return false, nil, err } if distinct { - originHandle, err := tablecodec.DecodeHandleInUniqueIndexValue(originVal, isCommon) + originHandle, err := tablecodec.DecodeHandleInIndexValue(originVal) if err != nil { return false, nil, err } @@ -591,7 +591,7 @@ func fetchDuplicatedHandleForTempIndexKey(ctx context.Context, tempKey kv.Key, d return false, nil, err } if distinct { - originHandle, err := tablecodec.DecodeHandleInUniqueIndexValue(originVal, isCommon) + originHandle, err := tablecodec.DecodeHandleInIndexValue(originVal) if err != nil { return false, nil, err } @@ -614,7 +614,7 @@ func fetchDuplicatedHandleForTempIndexKey(ctx context.Context, tempKey kv.Key, d } // The value in temp index is not the delete marker. if distinct { - h, err := tablecodec.DecodeHandleInUniqueIndexValue(curElem.Value, isCommon) + h, err := tablecodec.DecodeHandleInIndexValue(curElem.Value) return true, h, err } return true, nil, nil diff --git a/pkg/tablecodec/bench_test.go b/pkg/tablecodec/bench_test.go index e6118e25bd5c3..780428389801a 100644 --- a/pkg/tablecodec/bench_test.go +++ b/pkg/tablecodec/bench_test.go @@ -16,9 +16,12 @@ package tablecodec import ( "testing" + "time" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/benchdaily" + "github.com/pingcap/tidb/pkg/util/codec" ) func BenchmarkEncodeRowKeyWithHandle(b *testing.B) { @@ -55,6 +58,33 @@ func BenchmarkDecodeRowKey(b *testing.B) { } } +func BenchmarkDecodeIndexKeyIntHandle(b *testing.B) { + var idxVal []byte + // When handle values greater than 255, it will have a memory alloc. + idxVal = append(idxVal, EncodeHandleInUniqueIndexValue(kv.IntHandle(256), false)...) + + for i := 0; i < b.N; i++ { + DecodeHandleInIndexValue(idxVal) + } +} + +func BenchmarkDecodeIndexKeyCommonHandle(b *testing.B) { + var idxVal []byte + idxVal = append(idxVal, 0) + // index version + idxVal = append(idxVal, IndexVersionFlag) + idxVal = append(idxVal, byte(1)) + + // common handle + encoded, _ := codec.EncodeKey(time.UTC, nil, types.MakeDatums(1, 2)...) + h, _ := kv.NewCommonHandle(encoded) + idxVal = encodeCommonHandle(idxVal, h) + + for i := 0; i < b.N; i++ { + DecodeHandleInIndexValue(idxVal) + } +} + func TestBenchDaily(t *testing.T) { benchdaily.Run( BenchmarkEncodeRowKeyWithHandle, diff --git a/pkg/tablecodec/tablecodec.go b/pkg/tablecodec/tablecodec.go index 713d3537e806b..5d5a9e5843f6b 100644 --- a/pkg/tablecodec/tablecodec.go +++ b/pkg/tablecodec/tablecodec.go @@ -963,7 +963,7 @@ func DecodeIndexHandle(key, value []byte, colsLen int) (kv.Handle, error) { if len(b) > 0 { return decodeHandleInIndexKey(b) } else if len(value) >= 8 { - return decodeHandleInIndexValue(value) + return DecodeHandleInIndexValue(value) } // Should never execute to here. return nil, errors.Errorf("no handle in index key: %v, value: %v", key, value) @@ -980,7 +980,11 @@ func decodeHandleInIndexKey(keySuffix []byte) (kv.Handle, error) { return kv.NewCommonHandle(keySuffix) } -func decodeHandleInIndexValue(value []byte) (handle kv.Handle, err error) { +// DecodeHandleInIndexValue decodes handle in unqiue index value. +func DecodeHandleInIndexValue(value []byte) (handle kv.Handle, err error) { + if len(value) <= MaxOldEncodeValueLen { + return decodeIntHandleInIndexValue(value), nil + } seg := SplitIndexValue(value) if len(seg.IntHandle) != 0 { handle = decodeIntHandleInIndexValue(seg.IntHandle) @@ -1679,11 +1683,6 @@ func encodeCommonHandle(idxVal []byte, h kv.Handle) []byte { return idxVal } -// DecodeHandleInUniqueIndexValue decodes handle in data. -func DecodeHandleInUniqueIndexValue(data []byte, isCommonHandle bool) (kv.Handle, error) { - return decodeHandleInIndexValue(data) -} - func encodePartitionID(idxVal []byte, partitionID int64) []byte { idxVal = append(idxVal, PartitionIDFlag) idxVal = codec.EncodeInt(idxVal, partitionID) diff --git a/pkg/tablecodec/tablecodec_test.go b/pkg/tablecodec/tablecodec_test.go index 0a3b3bbbe56f2..fc7787fbb7650 100644 --- a/pkg/tablecodec/tablecodec_test.go +++ b/pkg/tablecodec/tablecodec_test.go @@ -656,7 +656,7 @@ func TestTempIndexValueCodec(t *testing.T) { remain, err = newTempIdxVal.DecodeOne(val) require.NoError(t, err) require.Equal(t, 0, len(remain)) - handle, err := DecodeHandleInUniqueIndexValue(newTempIdxVal.Value, false) + handle, err := DecodeHandleInIndexValue(newTempIdxVal.Value) require.NoError(t, err) require.Equal(t, handle.IntValue(), int64(100)) require.EqualValues(t, tempIdxVal, newTempIdxVal) diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index fe6e36137258d..e557496268858 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -2919,7 +2919,7 @@ func TestRCPointWriteLockIfExists(t *testing.T) { txnCtx = tk.Session().GetSessionVars().TxnCtx val, ok := txnCtx.GetKeyInPessimisticLockCache(secIdxKey1) require.Equal(t, true, ok) - handle, err := tablecodec.DecodeHandleInUniqueIndexValue(val, false) + handle, err := tablecodec.DecodeHandleInIndexValue(val) require.NoError(t, err) require.Equal(t, kv.IntHandle(1), handle) _, ok = txnCtx.GetKeyInPessimisticLockCache(key1) From 4691829f13bae915fd359ab36c060360a3cd8e46 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Mon, 3 Jun 2024 13:42:22 +0800 Subject: [PATCH 3/6] add tests --- tests/integrationtest/r/globalindex/insert.result | 8 ++++++++ tests/integrationtest/t/globalindex/insert.test | 8 ++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/integrationtest/r/globalindex/insert.result create mode 100644 tests/integrationtest/t/globalindex/insert.test diff --git a/tests/integrationtest/r/globalindex/insert.result b/tests/integrationtest/r/globalindex/insert.result new file mode 100644 index 0000000000000..0af2477c5799a --- /dev/null +++ b/tests/integrationtest/r/globalindex/insert.result @@ -0,0 +1,8 @@ +set tidb_enable_global_index = true; +drop table if exists t; +create table t(a int, b int, unique index idx(a)) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2) on duplicate key update a=1, b=3; +select * from t use index (idx); +a b +1 3 +set tidb_enable_global_index = false; diff --git a/tests/integrationtest/t/globalindex/insert.test b/tests/integrationtest/t/globalindex/insert.test new file mode 100644 index 0000000000000..e98da452ce65c --- /dev/null +++ b/tests/integrationtest/t/globalindex/insert.test @@ -0,0 +1,8 @@ +set tidb_enable_global_index = true; + +drop table if exists t; +create table t(a int, b int, unique index idx(a)) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2) on duplicate key update a=1, b=3; +select * from t use index (idx); + +set tidb_enable_global_index = false; From ac19463bd80f7c3500ac469ab1099258a9e0cf92 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Mon, 3 Jun 2024 13:48:32 +0800 Subject: [PATCH 4/6] make fmt --- pkg/executor/batch_checker.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/executor/batch_checker.go b/pkg/executor/batch_checker.go index 798df3ffce561..d7f83edeb459f 100644 --- a/pkg/executor/batch_checker.go +++ b/pkg/executor/batch_checker.go @@ -37,8 +37,8 @@ import ( ) type keyValueWithDupInfo struct { - newKey kv.Key - dupErr error + newKey kv.Key + dupErr error } type toBeCheckedRow struct { From 81afbd94ed5eae9ef84ee50e0d26b6075fb42029 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Mon, 3 Jun 2024 16:37:27 +0800 Subject: [PATCH 5/6] fix 53750 --- pkg/executor/insert_common.go | 6 +++++- tests/integrationtest/r/globalindex/update.result | 7 +++++++ tests/integrationtest/t/globalindex/update.test | 7 +++++++ 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index 007e8fa099cab..a4289e758e89f 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -1373,7 +1373,11 @@ func (e *InsertValues) removeRow( return true, nil } - err = r.t.RemoveRecord(e.Ctx().GetTableCtx(), handle, oldRow) + if ph, ok := handle.(kv.PartitionHandle); ok { + err = e.Table.(table.PartitionedTable).GetPartition(ph.PartitionID).RemoveRecord(e.Ctx().GetTableCtx(), handle, oldRow) + } else { + err = r.t.RemoveRecord(e.Ctx().GetTableCtx(), handle, oldRow) + } if err != nil { return false, err } diff --git a/tests/integrationtest/r/globalindex/update.result b/tests/integrationtest/r/globalindex/update.result index 7db38cbe8ba8f..f475c84aefd2e 100644 --- a/tests/integrationtest/r/globalindex/update.result +++ b/tests/integrationtest/r/globalindex/update.result @@ -40,4 +40,11 @@ a b c 2 11 11 8 8 8 13 12 12 +drop table t; +create table t(a varchar(70), b mediumint(9), unique index idx_a(a), unique index idx_b(b)) partition by key(b) partitions 5; +insert into t values ('',826534 ); +replace into t values ('',826536 ); +select * from t; +a b + 826536 set tidb_enable_global_index=default; diff --git a/tests/integrationtest/t/globalindex/update.test b/tests/integrationtest/t/globalindex/update.test index 90edbf9447dd3..ccc14ffdd9b59 100644 --- a/tests/integrationtest/t/globalindex/update.test +++ b/tests/integrationtest/t/globalindex/update.test @@ -34,4 +34,11 @@ update t set a = 13 where a = 12; explain select * from t use index(idx) order by a; select * from t use index(idx) order by a; +# https://github.com/pingcap/tidb/issues/53750 +drop table t; +create table t(a varchar(70), b mediumint(9), unique index idx_a(a), unique index idx_b(b)) partition by key(b) partitions 5; +insert into t values ('',826534 ); +replace into t values ('',826536 ); +select * from t; + set tidb_enable_global_index=default; From 17ac9fe4db0d3c01aca6415d7299b9bbc6bf7a30 Mon Sep 17 00:00:00 2001 From: Jason Mo Date: Mon, 3 Jun 2024 17:16:11 +0800 Subject: [PATCH 6/6] update --- pkg/executor/insert_common.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index a4289e758e89f..da1d9bd5f6703 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -1374,7 +1374,7 @@ func (e *InsertValues) removeRow( } if ph, ok := handle.(kv.PartitionHandle); ok { - err = e.Table.(table.PartitionedTable).GetPartition(ph.PartitionID).RemoveRecord(e.Ctx().GetTableCtx(), handle, oldRow) + err = e.Table.(table.PartitionedTable).GetPartition(ph.PartitionID).RemoveRecord(e.Ctx().GetTableCtx(), ph.Handle, oldRow) } else { err = r.t.RemoveRecord(e.Ctx().GetTableCtx(), handle, oldRow) }