331 lines
9.8 KiB
Go
331 lines
9.8 KiB
Go
// Copyright 2018 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package executor
|
|
|
|
import (
|
|
"context"
|
|
"strings"
|
|
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/tidb/pkg/errno"
|
|
"github.com/pingcap/tidb/pkg/expression"
|
|
"github.com/pingcap/tidb/pkg/kv"
|
|
"github.com/pingcap/tidb/pkg/meta/model"
|
|
"github.com/pingcap/tidb/pkg/parser/terror"
|
|
"github.com/pingcap/tidb/pkg/sessionctx"
|
|
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
|
|
"github.com/pingcap/tidb/pkg/table"
|
|
"github.com/pingcap/tidb/pkg/table/tables"
|
|
"github.com/pingcap/tidb/pkg/tablecodec"
|
|
"github.com/pingcap/tidb/pkg/types"
|
|
"github.com/pingcap/tidb/pkg/util"
|
|
"github.com/pingcap/tidb/pkg/util/chunk"
|
|
"github.com/pingcap/tidb/pkg/util/codec"
|
|
"github.com/pingcap/tidb/pkg/util/logutil"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
type keyValueWithDupInfo struct {
|
|
newKey kv.Key
|
|
dupErr error
|
|
}
|
|
|
|
type toBeCheckedRow struct {
|
|
row []types.Datum
|
|
handleKey *keyValueWithDupInfo
|
|
uniqueKeys []*keyValueWithDupInfo
|
|
// t is the table or partition this row belongs to.
|
|
t table.Table
|
|
ignored bool
|
|
}
|
|
|
|
// getKeysNeedCheck gets keys converted from to-be-insert rows to record keys and unique index keys,
|
|
// which need to be checked whether they are duplicate keys.
|
|
func getKeysNeedCheck(sctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([]toBeCheckedRow, error) {
|
|
nUnique := 0
|
|
for _, v := range t.Indices() {
|
|
if !tables.IsIndexWritable(v) {
|
|
continue
|
|
}
|
|
if v.Meta().Unique {
|
|
nUnique++
|
|
}
|
|
}
|
|
toBeCheckRows := make([]toBeCheckedRow, 0, len(rows))
|
|
|
|
var (
|
|
tblHandleCols []*table.Column
|
|
pkIdxInfo *model.IndexInfo
|
|
)
|
|
// Get handle column if PK is handle.
|
|
if t.Meta().PKIsHandle {
|
|
for _, col := range t.Cols() {
|
|
if col.IsPKHandleColumn(t.Meta()) {
|
|
tblHandleCols = append(tblHandleCols, col)
|
|
break
|
|
}
|
|
}
|
|
} else if t.Meta().IsCommonHandle {
|
|
pkIdxInfo = tables.FindPrimaryIndex(t.Meta())
|
|
for _, idxCol := range pkIdxInfo.Columns {
|
|
tblHandleCols = append(tblHandleCols, t.Cols()[idxCol.Offset])
|
|
}
|
|
}
|
|
|
|
var err error
|
|
for _, row := range rows {
|
|
toBeCheckRows, err = getKeysNeedCheckOneRow(sctx, t, row, nUnique, tblHandleCols, pkIdxInfo, toBeCheckRows)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
return toBeCheckRows, nil
|
|
}
|
|
|
|
func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.Datum, nUnique int, handleCols []*table.Column,
|
|
pkIdxInfo *model.IndexInfo, result []toBeCheckedRow) ([]toBeCheckedRow, error) {
|
|
var err error
|
|
if p, ok := t.(table.PartitionedTable); ok {
|
|
t, err = p.GetPartitionByRow(ctx.GetExprCtx().GetEvalCtx(), row)
|
|
if err != nil {
|
|
if terr, ok := errors.Cause(err).(*terror.Error); ok && (terr.Code() == errno.ErrNoPartitionForGivenValue || terr.Code() == errno.ErrRowDoesNotMatchGivenPartitionSet) {
|
|
ec := ctx.GetSessionVars().StmtCtx.ErrCtx()
|
|
if err = ec.HandleError(terr); err != nil {
|
|
return nil, err
|
|
}
|
|
result = append(result, toBeCheckedRow{ignored: true})
|
|
return result, nil
|
|
}
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
uniqueKeys := make([]*keyValueWithDupInfo, 0, nUnique)
|
|
// Append record keys and errors.
|
|
var handle kv.Handle
|
|
if t.Meta().IsCommonHandle {
|
|
var err error
|
|
handle, err = buildHandleFromDatumRow(ctx.GetSessionVars().StmtCtx, row, handleCols, pkIdxInfo)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else if len(handleCols) > 0 {
|
|
handle = kv.IntHandle(row[handleCols[0].Offset].GetInt64())
|
|
}
|
|
var handleKey *keyValueWithDupInfo
|
|
if handle != nil {
|
|
handleKey = &keyValueWithDupInfo{
|
|
newKey: tablecodec.EncodeRecordKey(t.RecordPrefix(), handle),
|
|
}
|
|
|
|
var keyCols []string
|
|
var err error
|
|
if t.Meta().IsCommonHandle {
|
|
data := make([]types.Datum, len(handleCols))
|
|
for i, col := range handleCols {
|
|
data[i] = row[col.Offset]
|
|
}
|
|
keyCols, err = dataToStrings(data)
|
|
} else {
|
|
var s string
|
|
s, err = row[handleCols[0].Offset].ToString()
|
|
keyCols = []string{s}
|
|
}
|
|
if err != nil {
|
|
var handleData []types.Datum
|
|
handleData, err = handle.Data()
|
|
if err == nil {
|
|
keyCols, err = dataToStrings(handleData)
|
|
}
|
|
}
|
|
|
|
if err == nil {
|
|
handleKey.dupErr = kv.GenKeyExistsErr(keyCols, t.Meta().Name.String()+".PRIMARY")
|
|
} else {
|
|
logutil.BgLogger().Warn("get key string failed",
|
|
zap.Error(err), zap.Stringer("handle", handle))
|
|
handleKey.dupErr = kv.ErrKeyExists
|
|
}
|
|
}
|
|
|
|
// extraColumns is used to fetch values while processing "add/drop/modify/change column" operation.
|
|
extraColumns := 0
|
|
for _, col := range t.WritableCols() {
|
|
// if there is a changing column, append the dependency column for index fetch values
|
|
if col.ChangeStateInfo != nil && col.State != model.StatePublic {
|
|
value, err := table.CastValue(ctx, row[col.DependencyColumnOffset], col.ColumnInfo, false, false)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
row = append(row, value)
|
|
extraColumns++
|
|
continue
|
|
}
|
|
|
|
if col.State != model.StatePublic {
|
|
// only append origin default value for index fetch values
|
|
if col.Offset >= len(row) {
|
|
value, err := table.GetColOriginDefaultValue(ctx.GetExprCtx(), col.ToInfo())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
row = append(row, value)
|
|
extraColumns++
|
|
}
|
|
}
|
|
}
|
|
// append unique keys and errors
|
|
for _, v := range t.Indices() {
|
|
if !tables.IsIndexWritable(v) {
|
|
continue
|
|
}
|
|
if !v.Meta().Unique {
|
|
continue
|
|
}
|
|
if t.Meta().IsCommonHandle && v.Meta().Primary {
|
|
continue
|
|
}
|
|
colVals, err1 := v.FetchValues(row, nil)
|
|
if err1 != nil {
|
|
return nil, err1
|
|
}
|
|
// Pass handle = 0 to GenIndexKey,
|
|
// due to we only care about distinct key.
|
|
sc := ctx.GetSessionVars().StmtCtx
|
|
iter := v.GenIndexKVIter(sc.ErrCtx(), sc.TimeZone(), colVals, kv.IntHandle(0), nil)
|
|
for iter.Valid() {
|
|
key, _, distinct, err1 := iter.Next(nil, nil)
|
|
if err1 != nil {
|
|
return nil, err1
|
|
}
|
|
// Skip the non-distinct keys.
|
|
if !distinct {
|
|
continue
|
|
}
|
|
// If index is used ingest ways, then we should check key from temp index.
|
|
if v.Meta().State != model.StatePublic && v.Meta().BackfillState != model.BackfillStateInapplicable {
|
|
_, key, _ = tables.GenTempIdxKeyByState(v.Meta(), key)
|
|
}
|
|
colStrVals, err1 := dataToStrings(colVals)
|
|
if err1 != nil {
|
|
return nil, err1
|
|
}
|
|
uniqueKeys = append(uniqueKeys, &keyValueWithDupInfo{
|
|
newKey: key,
|
|
dupErr: kv.GenKeyExistsErr(colStrVals, v.TableMeta().Name.String()+"."+v.Meta().Name.String()),
|
|
})
|
|
}
|
|
}
|
|
row = row[:len(row)-extraColumns]
|
|
result = append(result, toBeCheckedRow{
|
|
row: row,
|
|
handleKey: handleKey,
|
|
uniqueKeys: uniqueKeys,
|
|
t: t,
|
|
})
|
|
return result, nil
|
|
}
|
|
|
|
func buildHandleFromDatumRow(sctx *stmtctx.StatementContext, row []types.Datum, tblHandleCols []*table.Column, pkIdxInfo *model.IndexInfo) (kv.Handle, error) {
|
|
pkDts := make([]types.Datum, 0, len(tblHandleCols))
|
|
for i, col := range tblHandleCols {
|
|
d := row[col.Offset]
|
|
if pkIdxInfo != nil && len(pkIdxInfo.Columns) > 0 {
|
|
tablecodec.TruncateIndexValue(&d, pkIdxInfo.Columns[i], col.ColumnInfo)
|
|
}
|
|
pkDts = append(pkDts, d)
|
|
}
|
|
handleBytes, err := codec.EncodeKey(sctx.TimeZone(), nil, pkDts...)
|
|
err = sctx.HandleError(err)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
handle, err := kv.NewCommonHandle(handleBytes)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return handle, nil
|
|
}
|
|
|
|
func dataToStrings(data []types.Datum) ([]string, error) {
|
|
strs := make([]string, 0, len(data))
|
|
for _, datum := range data {
|
|
str, err := datum.ToString()
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
if datum.Kind() == types.KindBytes || datum.Kind() == types.KindMysqlBit || datum.Kind() == types.KindBinaryLiteral {
|
|
// Same as MySQL, remove all 0x00 on the tail,
|
|
// but keep one 0x00 at least.
|
|
if datum.Kind() == types.KindBytes {
|
|
str = strings.TrimRight(str, string(rune(0x00)))
|
|
if len(str) == 0 {
|
|
str = string(rune(0x00))
|
|
}
|
|
}
|
|
str = util.FmtNonASCIIPrintableCharToHex(str, len(str), true)
|
|
}
|
|
strs = append(strs, str)
|
|
}
|
|
return strs, nil
|
|
}
|
|
|
|
// getOldRow gets the table record row from storage for batch check.
|
|
// t could be a normal table or a partition, but it must not be a PartitionedTable.
|
|
func getOldRow(ctx context.Context, sctx sessionctx.Context, txn kv.Transaction, t table.Table, handle kv.Handle,
|
|
genExprs []expression.Expression) ([]types.Datum, error) {
|
|
oldValue, err := kv.GetValue(ctx, txn, tablecodec.EncodeRecordKey(t.RecordPrefix(), handle))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
cols := t.WritableCols()
|
|
oldRow, oldRowMap, err := tables.DecodeRawRowData(sctx.GetExprCtx(), t.Meta(), handle, cols, oldValue)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
// Fill write-only and write-reorg columns with originDefaultValue if not found in oldValue.
|
|
gIdx := 0
|
|
exprCtx := sctx.GetExprCtx()
|
|
for _, col := range cols {
|
|
if col.State != model.StatePublic && oldRow[col.Offset].IsNull() {
|
|
_, found := oldRowMap[col.ID]
|
|
if !found {
|
|
oldRow[col.Offset], err = table.GetColOriginDefaultValue(exprCtx, col.ToInfo())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
if col.IsGenerated() && col.State == model.StatePublic {
|
|
// only the virtual column needs fill back.
|
|
// Insert doesn't fill the generated columns at non-public state.
|
|
if !col.GeneratedStored {
|
|
val, err := genExprs[gIdx].Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.MutRowFromDatums(oldRow).ToRow())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
oldRow[col.Offset], err = table.CastValue(sctx, val, col.ToInfo(), false, false)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
gIdx++
|
|
}
|
|
}
|
|
return oldRow, nil
|
|
}
|