Skip to content

Commit

Permalink
executor: cleanup useless code in batch checker (#19511)
Browse files Browse the repository at this point in the history
Signed-off-by: Shuaipeng Yu <jackysp@gmail.com>
  • Loading branch information
jackysp authored Sep 1, 2020
1 parent 64a5e4f commit 968d0d7
Show file tree
Hide file tree
Showing 4 changed files with 14 additions and 28 deletions.
20 changes: 3 additions & 17 deletions executor/batch_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,20 +28,14 @@ import (
"github.com/pingcap/tidb/util/stringutil"
)

type keyValue struct {
key kv.Key
value []byte
}

type keyValueWithDupInfo struct {
newKV keyValue
newKey kv.Key
dupErr error
commonHandle bool
}

type toBeCheckedRow struct {
row []types.Datum
rowValue []byte
handleKey *keyValueWithDupInfo
uniqueKeys []*keyValueWithDupInfo
// t is the table or partition this row belongs to.
Expand Down Expand Up @@ -110,10 +104,6 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D
}

uniqueKeys := make([]*keyValueWithDupInfo, 0, nUnique)
newRowValue, err := encodeNewRow(ctx, t, row)
if err != nil {
return nil, err
}
// Append record keys and errors.
var handle kv.Handle
if t.Meta().IsCommonHandle {
Expand All @@ -135,10 +125,7 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D
return kv.GetDuplicateErrorHandleString(handle)
}
handleKey = &keyValueWithDupInfo{
newKV: keyValue{
key: t.RecordKey(handle),
value: newRowValue,
},
newKey: t.RecordKey(handle),
dupErr: kv.ErrKeyExists.FastGenByArgs(stringutil.MemoizeStr(fn), "PRIMARY"),
}
}
Expand Down Expand Up @@ -179,7 +166,7 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D
return nil, err1
}
uniqueKeys = append(uniqueKeys, &keyValueWithDupInfo{
newKV: keyValue{key: key},
newKey: key,
dupErr: kv.ErrKeyExists.FastGenByArgs(colValStr, v.Meta().Name),
commonHandle: t.Meta().IsCommonHandle,
})
Expand All @@ -189,7 +176,6 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D
}
result = append(result, toBeCheckedRow{
row: row,
rowValue: newRowValue,
handleKey: handleKey,
uniqueKeys: uniqueKeys,
t: t,
Expand Down
12 changes: 6 additions & 6 deletions executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,10 +121,10 @@ func prefetchUniqueIndices(ctx context.Context, txn kv.Transaction, rows []toBeC
batchKeys := make([]kv.Key, 0, nKeys)
for _, r := range rows {
if r.handleKey != nil {
batchKeys = append(batchKeys, r.handleKey.newKV.key)
batchKeys = append(batchKeys, r.handleKey.newKey)
}
for _, k := range r.uniqueKeys {
batchKeys = append(batchKeys, k.newKV.key)
batchKeys = append(batchKeys, k.newKey)
}
}
return txn.BatchGet(ctx, batchKeys)
Expand All @@ -140,7 +140,7 @@ func prefetchConflictedOldRows(ctx context.Context, txn kv.Transaction, rows []t
batchKeys := make([]kv.Key, 0, len(rows))
for _, r := range rows {
for _, uk := range r.uniqueKeys {
if val, found := values[string(uk.newKV.key)]; found {
if val, found := values[string(uk.newKey)]; found {
handle, err := tablecodec.DecodeHandleInUniqueIndexValue(val, uk.commonHandle)
if err != nil {
return err
Expand Down Expand Up @@ -209,7 +209,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D

for i, r := range toBeCheckedRows {
if r.handleKey != nil {
handle, err := tablecodec.DecodeRowKey(r.handleKey.newKV.key)
handle, err := tablecodec.DecodeRowKey(r.handleKey.newKey)
if err != nil {
return err
}
Expand All @@ -224,7 +224,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D
}

for _, uk := range r.uniqueKeys {
val, err := txn.Get(ctx, uk.newKV.key)
val, err := txn.Get(ctx, uk.newKey)
if err != nil {
if kv.IsErrNotFound(err) {
continue
Expand All @@ -242,7 +242,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D
// Data index inconsistent? A unique key provide the handle information, but the
// handle points to nothing.
logutil.BgLogger().Error("get old row failed when insert on dup",
zap.String("uniqueKey", hex.EncodeToString(uk.newKV.key)),
zap.String("uniqueKey", hex.EncodeToString(uk.newKey)),
zap.Stringer("handle", handle),
zap.String("toBeInsertedRow", types.DatumsToStrNoErr(r.row)))
}
Expand Down
4 changes: 2 additions & 2 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -975,7 +975,7 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D
for i, r := range toBeCheckedRows {
skip := false
if r.handleKey != nil {
_, err := txn.Get(ctx, r.handleKey.newKV.key)
_, err := txn.Get(ctx, r.handleKey.newKey)
if err == nil {
e.ctx.GetSessionVars().StmtCtx.AppendWarning(r.handleKey.dupErr)
continue
Expand All @@ -985,7 +985,7 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D
}
}
for _, uk := range r.uniqueKeys {
_, err := txn.Get(ctx, uk.newKV.key)
_, err := txn.Get(ctx, uk.newKey)
if err == nil {
// If duplicate keys were found in BatchGet, mark row = nil.
e.ctx.GetSessionVars().StmtCtx.AppendWarning(uk.dupErr)
Expand Down
6 changes: 3 additions & 3 deletions executor/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,12 +96,12 @@ func (e *ReplaceExec) replaceRow(ctx context.Context, r toBeCheckedRow) error {
}

if r.handleKey != nil {
handle, err := tablecodec.DecodeRowKey(r.handleKey.newKV.key)
handle, err := tablecodec.DecodeRowKey(r.handleKey.newKey)
if err != nil {
return err
}

if _, err := txn.Get(ctx, r.handleKey.newKV.key); err == nil {
if _, err := txn.Get(ctx, r.handleKey.newKey); err == nil {
rowUnchanged, err := e.removeRow(ctx, txn, handle, r)
if err != nil {
return err
Expand Down Expand Up @@ -147,7 +147,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) (bool, bool, error) {
for _, uk := range r.uniqueKeys {
val, err := txn.Get(ctx, uk.newKV.key)
val, err := txn.Get(ctx, uk.newKey)
if err != nil {
if kv.IsErrNotFound(err) {
continue
Expand Down

0 comments on commit 968d0d7

Please sign in to comment.